diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockCache.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockCache.java index 2990696ee1b08..7e26e916b3368 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockCache.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockCache.java @@ -55,12 +55,15 @@ public interface BlockCache extends Closeable { /** * Gets the block having the given {@code blockNumber}. - * + * If the block is not present then the method returns + * false and {@code buffer} is unchanged. * @param blockNumber the id of the desired block. * @param buffer contents of the desired block are copied to this buffer. + * @return true if the block was found. * @throws IOException if there is an error reading the given block. + * @throws IllegalArgumentException if buffer is null. */ - void get(int blockNumber, ByteBuffer buffer) throws IOException; + boolean get(int blockNumber, ByteBuffer buffer) throws IOException; /** * Puts the given block in this cache. diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockData.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockData.java index ecb8bc7243be0..323a8ec77aff6 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockData.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockData.java @@ -89,6 +89,13 @@ public BlockData(long fileSize, int blockSize) { ? 1 : 0); this.state = new State[this.numBlocks]; + markBlocksAsNotReady(); + } + + /** + * Mark all the blocks as not ready. + */ + public void markBlocksAsNotReady() { for (int b = 0; b < this.numBlocks; b++) { setState(b, State.NOT_READY); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockManager.java index 45f0aabe7dcd9..078bbe05c9c6a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockManager.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockManager.java @@ -125,8 +125,9 @@ public void requestPrefetch(int blockNumber) { /** * Requests cancellation of any previously issued prefetch requests. + * @param reason why? */ - public void cancelPrefetches() { + public void cancelPrefetches(final CancelReason reason) { // Do nothing because we do not support prefetches. } @@ -142,4 +143,16 @@ public void requestCaching(BufferData data) { @Override public void close() { } + + /** + * Reason for cancelling prefetches. + */ + public enum CancelReason { + /** Stream has switched to random IO. */ + RandomIO, + /** Stream closed completely. */ + Close, + /** Stream unbuffered. */ + Unbuffer + } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockManagerParameters.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockManagerParameters.java index 581d53016df74..0ffc631959f3e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockManagerParameters.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockManagerParameters.java @@ -22,6 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.LocalDirAllocator; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.statistics.DurationTrackerFactory; /** @@ -30,6 +31,11 @@ @InterfaceAudience.Private public final class BlockManagerParameters { + /** + * The path of the underlying file; for logging. + */ + private Path path; + /** * Asynchronous tasks are performed in this pool. */ @@ -224,4 +230,20 @@ public BlockManagerParameters withTrackerFactory( return this; } + /** + * @return The path of the underlying file. + */ + public Path getPath() { + return path; + } + + /** + * Set the path. + * @param value new value + * @return the builder + */ + public BlockManagerParameters withPath(final Path value) { + path = value; + return this; + } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BufferData.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BufferData.java index de68269ab700c..d8a0c216106f0 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BufferData.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BufferData.java @@ -51,7 +51,7 @@ public enum State { /** * Buffer has been acquired but has no data. */ - BLANK, + EMPTY, /** * This block is being prefetched. @@ -114,7 +114,7 @@ public BufferData(int blockNumber, ByteBuffer buffer) { this.blockNumber = blockNumber; this.buffer = buffer; - this.state = State.BLANK; + this.state = State.EMPTY; } /** @@ -181,7 +181,7 @@ public synchronized Future getActionFuture() { public synchronized void setPrefetch(Future actionFuture) { Validate.checkNotNull(actionFuture, "actionFuture"); - this.updateState(State.PREFETCHING, State.BLANK); + this.updateState(State.PREFETCHING, State.EMPTY); this.action = actionFuture; } @@ -289,7 +289,7 @@ public boolean stateEqualsOneOf(State... states) { public String toString() { return String.format( - "[%03d] id: %03d, %s: buf: %s, checksum: %d, future: %s", + "[%03d] id: %03d, State: %s: buffer: %s, checksum: %d, future: %s", this.blockNumber, System.identityHashCode(this), this.state, @@ -300,7 +300,7 @@ public String toString() { private String getFutureStr(Future f) { if (f == null) { - return "--"; + return "(none)"; } else { return this.action.isDone() ? "done" : "not done"; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/CachingBlockManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/CachingBlockManager.java index 86612a6d2c8b7..3623c161fdec2 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/CachingBlockManager.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/CachingBlockManager.java @@ -26,6 +26,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; @@ -37,20 +38,24 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.LocalDirAllocator; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.statistics.DurationTracker; import org.apache.hadoop.fs.statistics.DurationTrackerFactory; +import org.apache.hadoop.fs.store.LogExactlyOnce; import static java.util.Objects.requireNonNull; - import static org.apache.hadoop.fs.impl.prefetch.Validate.checkNotNegative; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; +import static org.apache.hadoop.util.functional.FutureIO.awaitFuture; /** * Provides read access to the underlying file one block at a time. - * Improve read performance by prefetching and locall caching blocks. + * Improve read performance by prefetching and locally caching blocks. */ public abstract class CachingBlockManager extends BlockManager { private static final Logger LOG = LoggerFactory.getLogger(CachingBlockManager.class); + + private static final LogExactlyOnce LOG_CACHING_DISABLED = new LogExactlyOnce(LOG); private static final int TIMEOUT_MINUTES = 60; /** @@ -85,7 +90,10 @@ public abstract class CachingBlockManager extends BlockManager { */ private final BlockOperations ops; - private boolean closed; + /** + * True if the manager has been closed. + */ + private final AtomicBoolean closed = new AtomicBoolean(false); /** * If a single caching operation takes more than this time (in seconds), @@ -104,6 +112,11 @@ public abstract class CachingBlockManager extends BlockManager { private final LocalDirAllocator localDirAllocator; + /** + * The path of the underlying file; for logging. + */ + private Path path; + /** * Constructs an instance of a {@code CachingBlockManager}. * @@ -115,25 +128,27 @@ public CachingBlockManager(@Nonnull final BlockManagerParameters blockManagerPar Validate.checkPositiveInteger(blockManagerParameters.getBufferPoolSize(), "bufferPoolSize"); - this.futurePool = requireNonNull(blockManagerParameters.getFuturePool()); + this.path = requireNonNull(blockManagerParameters.getPath(), "block manager path"); + this.futurePool = requireNonNull(blockManagerParameters.getFuturePool(), "future pool"); this.bufferPoolSize = blockManagerParameters.getBufferPoolSize(); this.numCachingErrors = new AtomicInteger(); this.numReadErrors = new AtomicInteger(); this.cachingDisabled = new AtomicBoolean(); this.prefetchingStatistics = requireNonNull( - blockManagerParameters.getPrefetchingStatistics()); - this.conf = requireNonNull(blockManagerParameters.getConf()); + blockManagerParameters.getPrefetchingStatistics(), "prefetching statistics"); + this.conf = requireNonNull(blockManagerParameters.getConf(), "configuratin"); - if (this.getBlockData().getFileSize() > 0) { - this.bufferPool = new BufferPool(bufferPoolSize, this.getBlockData().getBlockSize(), + if (getBlockData().getFileSize() > 0) { + this.bufferPool = new BufferPool(bufferPoolSize, getBlockData().getBlockSize(), this.prefetchingStatistics); - this.cache = this.createCache(blockManagerParameters.getMaxBlocksCount(), + this.cache = createCache(blockManagerParameters.getMaxBlocksCount(), blockManagerParameters.getTrackerFactory()); } this.ops = new BlockOperations(); - this.ops.setDebug(false); + this.ops.setDebug(LOG.isDebugEnabled()); this.localDirAllocator = blockManagerParameters.getLocalDirAllocator(); + prefetchingStatistics.setPrefetchDiskCachingState(true); } /** @@ -152,7 +167,7 @@ public BufferData get(int blockNumber) throws IOException { boolean done; do { - if (closed) { + if (isClosed()) { throw new IOException("this stream is already closed"); } @@ -201,7 +216,7 @@ private boolean getInternal(BufferData data) throws IOException { return true; } - data.throwIfStateIncorrect(BufferData.State.BLANK); + data.throwIfStateIncorrect(BufferData.State.EMPTY); read(data); return true; } @@ -214,7 +229,7 @@ private boolean getInternal(BufferData data) throws IOException { */ @Override public void release(BufferData data) { - if (closed) { + if (isClosed()) { return; } @@ -227,16 +242,14 @@ public void release(BufferData data) { @Override public synchronized void close() { - if (closed) { + if (closed.getAndSet(true)) { return; } - closed = true; - final BlockOperations.Operation op = ops.close(); // Cancel any prefetches in progress. - cancelPrefetches(); + cancelPrefetches(CancelReason.Close); cleanupWithLogger(LOG, cache); @@ -257,25 +270,28 @@ public synchronized void close() { public void requestPrefetch(int blockNumber) { checkNotNegative(blockNumber, "blockNumber"); - if (closed) { + if (isClosed()) { return; } // We initiate a prefetch only if we can acquire a buffer from the shared pool. + LOG.debug("{}: Requesting prefetch for block {}", path, blockNumber); BufferData data = bufferPool.tryAcquire(blockNumber); if (data == null) { + LOG.debug("{}: no buffer acquired for block {}", path, blockNumber); return; } + LOG.debug("{}: acquired {}", path, data); // Opportunistic check without locking. - if (!data.stateEqualsOneOf(BufferData.State.BLANK)) { + if (!data.stateEqualsOneOf(BufferData.State.EMPTY)) { // The block is ready or being prefetched/cached. return; } synchronized (data) { // Reconfirm state after locking. - if (!data.stateEqualsOneOf(BufferData.State.BLANK)) { + if (!data.stateEqualsOneOf(BufferData.State.EMPTY)) { // The block is ready or being prefetched/cached. return; } @@ -290,16 +306,26 @@ public void requestPrefetch(int blockNumber) { /** * Requests cancellation of any previously issued prefetch requests. + * If the reason was switching to random IO, any active prefetched blocks + * are still cached. + * @param reason why were prefetches cancelled? */ @Override - public void cancelPrefetches() { + public void cancelPrefetches(final CancelReason reason) { + LOG.debug("{}: Cancelling prefetches: {}", path, reason); BlockOperations.Operation op = ops.cancelPrefetches(); - for (BufferData data : bufferPool.getAll()) { - // We add blocks being prefetched to the local cache so that the prefetch is not wasted. - if (data.stateEqualsOneOf(BufferData.State.PREFETCHING, BufferData.State.READY)) { - requestCaching(data); + if (reason == CancelReason.RandomIO) { + for (BufferData data : bufferPool.getAll()) { + // We add blocks being prefetched to the local cache so that the prefetch is not wasted. + // this only done if the reason is random IO-related, not due to close/unbuffer + if (data.stateEqualsOneOf(BufferData.State.PREFETCHING, BufferData.State.READY)) { + requestCaching(data); + } } + } else { + // free the buffers + bufferPool.getAll().forEach(BufferData::setDone); } ops.end(op); @@ -308,9 +334,9 @@ public void cancelPrefetches() { private void read(BufferData data) throws IOException { synchronized (data) { try { - readBlock(data, false, BufferData.State.BLANK); + readBlock(data, false, BufferData.State.EMPTY); } catch (IOException e) { - LOG.error("error reading block {}", data.getBlockNumber(), e); + LOG.debug("{}: error reading block {}", path, data.getBlockNumber(), e); throw e; } } @@ -331,13 +357,17 @@ private void prefetch(BufferData data, Instant taskQueuedStartTime) throws IOExc private void readBlock(BufferData data, boolean isPrefetch, BufferData.State... expectedState) throws IOException { - if (closed) { + if (isClosed()) { return; } BlockOperations.Operation op = null; DurationTracker tracker = null; + int bytesFetched = 0; + // to be filled in later. + long offset = 0; + int size = 0; synchronized (data) { try { if (data.stateEqualsOneOf(BufferData.State.DONE, BufferData.State.READY)) { @@ -348,6 +378,10 @@ private void readBlock(BufferData data, boolean isPrefetch, BufferData.State... data.throwIfStateIncorrect(expectedState); int blockNumber = data.getBlockNumber(); + final BlockData blockData = getBlockData(); + + offset = blockData.getStartOffset(data.getBlockNumber()); + size = blockData.getSize(data.getBlockNumber()); // Prefer reading from cache over reading from network. if (cache.containsBlock(blockNumber)) { @@ -361,17 +395,27 @@ private void readBlock(BufferData data, boolean isPrefetch, BufferData.State... tracker = prefetchingStatistics.prefetchOperationStarted(); op = ops.prefetch(data.getBlockNumber()); } else { + tracker = prefetchingStatistics.blockFetchOperationStarted(); op = ops.getRead(data.getBlockNumber()); } - long offset = getBlockData().getStartOffset(data.getBlockNumber()); - int size = getBlockData().getSize(data.getBlockNumber()); + ByteBuffer buffer = data.getBuffer(); buffer.clear(); read(buffer, offset, size); buffer.flip(); data.setReady(expectedState); + bytesFetched = size; + LOG.debug("Completed {} of block {} [{}-{}]", + isPrefetch ? "prefetch" : "read", + data.getBlockNumber(), + offset, offset + size); } catch (Exception e) { + LOG.debug("Failure in {} of block {} [{}-{}]", + isPrefetch ? "prefetch" : "read", + data.getBlockNumber(), + offset, offset + size, + e); if (isPrefetch && tracker != null) { tracker.failed(); } @@ -384,20 +428,46 @@ private void readBlock(BufferData data, boolean isPrefetch, BufferData.State... ops.end(op); } - if (isPrefetch) { - prefetchingStatistics.prefetchOperationCompleted(); - if (tracker != null) { - tracker.close(); - } + // update the statistics + prefetchingStatistics.fetchOperationCompleted(isPrefetch, bytesFetched); + if (tracker != null) { + tracker.close(); + LOG.debug("fetch completed: {}", tracker); } } } } + /** + * True if the manager has been closed. + */ + private boolean isClosed() { + return closed.get(); + } + + /** + * Disable caching; updates stream statistics and logs exactly once + * at info. + * @param endOp operation which measured the duration of the write. + */ + private void disableCaching(final BlockOperations.End endOp) { + if (!cachingDisabled.getAndSet(true)) { + String message = String.format( + "%s: Caching disabled because of slow operation (%.1f sec)", + path, endOp.duration()); + LOG_CACHING_DISABLED.info(message); + prefetchingStatistics.setPrefetchDiskCachingState(false); + } + } + + private boolean isCachingDisabled() { + return cachingDisabled.get(); + } + /** * Read task that is submitted to the future pool. */ - private static class PrefetchTask implements Supplier { + private class PrefetchTask implements Supplier { private final BufferData data; private final CachingBlockManager blockManager; private final Instant taskQueuedStartTime; @@ -413,50 +483,61 @@ public Void get() { try { blockManager.prefetch(data, taskQueuedStartTime); } catch (Exception e) { - LOG.info("error prefetching block {}. {}", data.getBlockNumber(), e.getMessage()); - LOG.debug("error prefetching block {}", data.getBlockNumber(), e); + LOG.info("{}: error prefetching block {}. {}", path, data.getBlockNumber(), e.getMessage()); + LOG.debug("{}: error prefetching block {}", path, data.getBlockNumber(), e); } return null; } } - private static final BufferData.State[] EXPECTED_STATE_AT_CACHING = - new BufferData.State[] { - BufferData.State.PREFETCHING, BufferData.State.READY - }; + /** + * Required state of a block for it to be cacheable. + */ + private static final BufferData.State[] EXPECTED_STATE_AT_CACHING = { + BufferData.State.PREFETCHING, BufferData.State.READY + }; /** * Requests that the given block should be copied to the local cache. + * If the stream is closed or caching disabled, the request is denied. + *

+ * If the block is in being prefetched, the + *

* The block must not be accessed by the caller after calling this method - * because it will released asynchronously relative to the caller. - * + * because it may be released asynchronously relative to the caller. + *

+ * @param data the block to cache.. * @throws IllegalArgumentException if data is null. */ @Override public void requestCaching(BufferData data) { - if (closed) { - return; - } + Validate.checkNotNull(data, "data"); - if (cachingDisabled.get()) { + final int blockNumber = data.getBlockNumber(); + LOG.debug("{}: Block {}: request caching of {}", path, blockNumber, data); + + if (isClosed() || isCachingDisabled()) { data.setDone(); return; } - Validate.checkNotNull(data, "data"); - // Opportunistic check without locking. if (!data.stateEqualsOneOf(EXPECTED_STATE_AT_CACHING)) { + LOG.debug("{}: Block {}: Block in wrong state to cache: {}", + path, blockNumber, data.getState()); return; } synchronized (data) { // Reconfirm state after locking. if (!data.stateEqualsOneOf(EXPECTED_STATE_AT_CACHING)) { + LOG.debug("{}: Block {}: Block in wrong state to cache: {}", + path, blockNumber, data.getState()); return; } if (cache.containsBlock(data.getBlockNumber())) { + LOG.debug("{}: Block {}: Block is already in cache", path, blockNumber); data.setDone(); return; } @@ -486,29 +567,38 @@ private void addToCacheAndRelease(BufferData data, Future blockFuture, prefetchingStatistics.executorAcquired( Duration.between(taskQueuedStartTime, Instant.now())); - if (closed) { + if (isClosed()) { return; } - if (cachingDisabled.get()) { + final int blockNumber = data.getBlockNumber(); + LOG.debug("{}: Block {}: Preparing to cache block", path, blockNumber); + + if (isCachingDisabled()) { + LOG.debug("{}: Block {}: Preparing caching disabled, not prefetching", path, blockNumber); data.setDone(); return; } + LOG.debug("{}: Block {}: awaiting any read to complete", path, blockNumber); try { - blockFuture.get(TIMEOUT_MINUTES, TimeUnit.MINUTES); + // wait for data; state of caching may change during this period. + awaitFuture(blockFuture, TIMEOUT_MINUTES, TimeUnit.MINUTES); if (data.stateEqualsOneOf(BufferData.State.DONE)) { // There was an error during prefetch. + LOG.debug("{}: Block {}: prefetch failure", path, blockNumber); return; } - } catch (Exception e) { - LOG.info("error waiting on blockFuture: {}. {}", data, e.getMessage()); - LOG.debug("error waiting on blockFuture: {}", data, e); + } catch (IOException | TimeoutException e) { + LOG.info("{}: Error fetching block: {}. {}", path, data, e.toString()); + LOG.debug("{}: Error fetching block: {}", path, data, e); data.setDone(); return; } - if (cachingDisabled.get()) { + if (isCachingDisabled()) { + // caching was disabled while waiting fro the read to complete. + LOG.debug("{}: Block {}: caching disabled while reading data", path, blockNumber); data.setDone(); return; } @@ -518,34 +608,32 @@ private void addToCacheAndRelease(BufferData data, Future blockFuture, synchronized (data) { try { if (data.stateEqualsOneOf(BufferData.State.DONE)) { + LOG.debug("{}: Block {}: block no longer in use; not adding", path, blockNumber); return; } - if (cache.containsBlock(data.getBlockNumber())) { + if (cache.containsBlock(blockNumber)) { + LOG.debug("{}: Block {}: already in cache; not adding", path, blockNumber); data.setDone(); return; } - op = ops.addToCache(data.getBlockNumber()); + op = ops.addToCache(blockNumber); ByteBuffer buffer = data.getBuffer().duplicate(); buffer.rewind(); - cachePut(data.getBlockNumber(), buffer); + cachePut(blockNumber, buffer); data.setDone(); } catch (Exception e) { numCachingErrors.incrementAndGet(); - LOG.info("error adding block to cache after wait: {}. {}", data, e.getMessage()); - LOG.debug("error adding block to cache after wait: {}", data, e); + LOG.info("{}: error adding block to cache: {}. {}", path, data, e.getMessage()); + LOG.debug("{}: error adding block to cache: {}", path, data, e); data.setDone(); } if (op != null) { BlockOperations.End endOp = (BlockOperations.End) ops.end(op); if (endOp.duration() > SLOW_CACHING_THRESHOLD) { - if (!cachingDisabled.getAndSet(true)) { - String message = String.format( - "Caching disabled because of slow operation (%.1f sec)", endOp.duration()); - LOG.warn(message); - } + disableCaching(endOp); } } } @@ -556,9 +644,10 @@ protected BlockCache createCache(int maxBlocksCount, DurationTrackerFactory trac } protected void cachePut(int blockNumber, ByteBuffer buffer) throws IOException { - if (closed) { + if (isClosed()) { return; } + LOG.debug("{}: Block {}: Caching", path, buffer); cache.put(blockNumber, buffer, conf, localDirAllocator); } @@ -634,15 +723,13 @@ BufferData getData(int blockNumber) { @Override public String toString() { - StringBuilder sb = new StringBuilder(); - - sb.append("cache("); - sb.append(cache.toString()); - sb.append("); "); - sb.append("pool: "); - sb.append(bufferPool.toString()); + String sb = "path: " + path + "; " + + "; cache(" + cache + "); " + + "pool: " + bufferPool + + "; numReadErrors: " + numReadErrors.get() + + "; numCachingErrors: " + numCachingErrors.get(); - return sb.toString(); + return sb; } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/EmptyPrefetchingStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/EmptyPrefetchingStatistics.java index d20a32728681f..ded78002a8e47 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/EmptyPrefetchingStatistics.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/EmptyPrefetchingStatistics.java @@ -67,6 +67,11 @@ public void prefetchOperationCompleted() { } + @Override + public void fetchOperationCompleted(final boolean prefetch, final long bytesFetched) { + + } + @Override public void executorAcquired(Duration timeInQueue) { @@ -81,5 +86,7 @@ public void memoryAllocated(int size) { public void memoryFreed(int size) { } + + } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/FilePosition.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/FilePosition.java index 286bdd7ae8996..84275d9c09faa 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/FilePosition.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/FilePosition.java @@ -64,12 +64,14 @@ public final class FilePosition { */ private long readStartOffset; + private long totalBytesRead; + // Read stats after a seek (mostly for debugging use). - private int numSingleByteReads; + private long numSingleByteReads; - private int numBytesRead; + private long numBytesRead; - private int numBufferReads; + private long numBufferReads; /** * Constructs an instance of {@link FilePosition}. @@ -245,8 +247,13 @@ public boolean bufferFullyRead() { && (numBytesRead == buffer.limit()); } + /** + * Increment the read statistic counters. + * @param n number of bytes read + */ public void incrementBytesRead(int n) { numBytesRead += n; + totalBytesRead += n; if (n == 1) { numSingleByteReads++; } else { @@ -254,18 +261,29 @@ public void incrementBytesRead(int n) { } } - public int numBytesRead() { + public long numBytesRead() { return numBytesRead; } - public int numSingleByteReads() { + public long numSingleByteReads() { return numSingleByteReads; } - public int numBufferReads() { + public long numBufferReads() { return numBufferReads; } + /** + * Total bytes read: is never reset. + * @return count of bytes read. + */ + public long getTotalBytesRead() { + return totalBytesRead; + } + + /** + * Reset the read statistics. + */ private void resetReadStats() { numBytesRead = 0; numSingleByteReads = 0; @@ -292,6 +310,9 @@ public String toString() { currentBufferState)); sb.append("\n"); } + sb.append(String.format( + "totalBytesRead=%d; numBytesRead=%d; numBufferReads=%d; numSingleByteReads=%d;", + totalBytesRead, numBytesRead, numBufferReads, numSingleByteReads)); return sb.toString(); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/PrefetchingStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/PrefetchingStatistics.java index d1ea600be720f..a266b0ac4f978 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/PrefetchingStatistics.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/PrefetchingStatistics.java @@ -17,56 +17,99 @@ * under the License. */ -package org.apache.hadoop.fs.impl.prefetch; - -import java.time.Duration; - -import org.apache.hadoop.fs.statistics.DurationTracker; -import org.apache.hadoop.fs.statistics.IOStatisticsSource; - -public interface PrefetchingStatistics extends IOStatisticsSource { - - /** - * A prefetch operation has started. - * @return duration tracker - */ - DurationTracker prefetchOperationStarted(); - - /** - * A block has been saved to the file cache. - */ - void blockAddedToFileCache(); - - /** - * A block has been removed from the file cache. - */ - void blockRemovedFromFileCache(); - - /** - * A block has been evicted from the file cache. - */ - void blockEvictedFromFileCache(); - - /** - * A prefetch operation has completed. - */ - void prefetchOperationCompleted(); - - /** - * An executor has been acquired, either for prefetching or caching. - * @param timeInQueue time taken to acquire an executor. - */ - void executorAcquired(Duration timeInQueue); - - /** - * A new buffer has been added to the buffer pool. - * @param size size of the new buffer - */ - void memoryAllocated(int size); - - /** - * Previously allocated memory has been freed. - * @param size size of memory freed. - */ - void memoryFreed(int size); -} + package org.apache.hadoop.fs.impl.prefetch; + + import java.time.Duration; + + import org.apache.hadoop.fs.statistics.DurationTracker; + import org.apache.hadoop.fs.statistics.IOStatisticsSource; + + import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.stubDurationTracker; + + public interface PrefetchingStatistics extends IOStatisticsSource { + + /** + * A prefetch operation has started. + * @return duration tracker + */ + DurationTracker prefetchOperationStarted(); + + /** + * A block fetch operation has started. + * @return duration tracker + */ + default DurationTracker blockFetchOperationStarted() { + return stubDurationTracker(); + } + + /** + * A block has been saved to the file cache. + */ + void blockAddedToFileCache(); + + /** + * A block has been removed from the file cache. + */ + void blockRemovedFromFileCache(); + + /** + * A block has been evicted from the file cache. + */ + void blockEvictedFromFileCache(); + + /** + * A prefetch operation has completed. + */ + void prefetchOperationCompleted(); + + /** + * A fetch/prefetch operation has completed. + * @param prefetch true if this was a prefetch + * @param bytesFetched number of bytes fetch + */ + void fetchOperationCompleted(boolean prefetch, long bytesFetched); + + /** + * An executor has been acquired, either for prefetching or caching. + * @param timeInQueue time taken to acquire an executor. + */ + void executorAcquired(Duration timeInQueue); + + /** + * A new buffer has been added to the buffer pool. + * @param size size of the new buffer + */ + void memoryAllocated(int size); + + /** + * Previously allocated memory has been freed. + * @param size size of memory freed. + */ + void memoryFreed(int size); + + /** + * Publish the prefetching state through a gauge. + * @param prefetchEnabled is prefetching enabled? + * @param blocks number of blocks to fetch + * @param blocksize block size. + */ + default void setPrefetchState(boolean prefetchEnabled, int blocks, int blocksize) { + + } + + /** + * Publish the disk caching state through a gauge. + * @param cacheEnabled will blocks be cached? + */ + default void setPrefetchDiskCachingState(boolean cacheEnabled) { + + } + + /** + * Bytes read from buffer (rather than via any direct http request). + * @param bytes number of bytes read. + */ + default void bytesReadFromBuffer(long bytes) { + + } + } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/SingleFilePerBlockCache.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/SingleFilePerBlockCache.java index ecf7e38f4f7cc..56f76405da87f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/SingleFilePerBlockCache.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/SingleFilePerBlockCache.java @@ -39,8 +39,12 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.ReentrantReadWriteLock; +import javax.annotation.Nullable; + +import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,12 +53,14 @@ import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.statistics.DurationTracker; import org.apache.hadoop.fs.statistics.DurationTrackerFactory; -import org.apache.hadoop.util.Preconditions; +import org.apache.hadoop.util.DurationInfo; import static java.util.Objects.requireNonNull; import static org.apache.hadoop.fs.impl.prefetch.Validate.checkNotNull; import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.stubDurationTrackerFactory; import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_FILE_CACHE_EVICTION; +import static org.apache.hadoop.util.Preconditions.checkArgument; +import static org.apache.hadoop.util.Preconditions.checkState; /** * Provides functionality necessary for caching blocks of data read from FileSystem. @@ -65,8 +71,10 @@ public class SingleFilePerBlockCache implements BlockCache { /** * Blocks stored in this cache. + * A concurrent hash map is used here, but it is still important for cache operations to + * be thread safe. */ - private final Map blocks; + private final Map blocks = new ConcurrentHashMap<>(); /** * Total max blocks count, to be considered as baseline for LRU cache eviction. @@ -76,7 +84,7 @@ public class SingleFilePerBlockCache implements BlockCache { /** * The lock to be shared by LRU based linked list updates. */ - private final ReentrantReadWriteLock blocksLock; + private final ReentrantReadWriteLock blocksLock = new ReentrantReadWriteLock(); /** * Head of the linked list. @@ -97,9 +105,9 @@ public class SingleFilePerBlockCache implements BlockCache { * Number of times a block was read from this cache. * Used for determining cache utilization factor. */ - private int numGets = 0; + private final AtomicInteger numGets = new AtomicInteger(); - private final AtomicBoolean closed; + private final AtomicBoolean closed = new AtomicBoolean(false); private final PrefetchingStatistics prefetchingStatistics; @@ -144,7 +152,7 @@ private enum LockType { @Override public String toString() { return String.format( - "([%03d] %s: size = %d, checksum = %d)", + "([%03d] %s: size = %,d, checksum = %d)", blockNumber, path, size, checksum); } @@ -222,13 +230,10 @@ private void setNext(Entry next) { */ public SingleFilePerBlockCache(PrefetchingStatistics prefetchingStatistics, int maxBlocksCount, - DurationTrackerFactory trackerFactory) { + @Nullable DurationTrackerFactory trackerFactory) { this.prefetchingStatistics = requireNonNull(prefetchingStatistics); - this.closed = new AtomicBoolean(false); + checkArgument(maxBlocksCount > 0, "maxBlocksCount should be more than 0"); this.maxBlocksCount = maxBlocksCount; - Preconditions.checkArgument(maxBlocksCount > 0, "maxBlocksCount should be more than 0"); - blocks = new ConcurrentHashMap<>(); - blocksLock = new ReentrantReadWriteLock(); this.trackerFactory = trackerFactory != null ? trackerFactory : stubDurationTrackerFactory(); } @@ -245,7 +250,7 @@ public boolean containsBlock(int blockNumber) { * Gets the blocks in this cache. */ @Override - public Iterable blocks() { + public synchronized Iterable blocks() { return Collections.unmodifiableList(new ArrayList<>(blocks.keySet())); } @@ -257,19 +262,20 @@ public int size() { return blocks.size(); } - /** - * Gets the block having the given {@code blockNumber}. - * - * @throws IllegalArgumentException if buffer is null. - */ @Override - public void get(int blockNumber, ByteBuffer buffer) throws IOException { + public synchronized boolean get(int blockNumber, ByteBuffer buffer) throws IOException { if (closed.get()) { - return; + return false; } checkNotNull(buffer, "buffer"); + if (!blocks.containsKey(blockNumber)) { + // no block found + return false; + } + + // block found. read it. Entry entry = getEntry(blockNumber); entry.takeLock(Entry.LockType.READ); try { @@ -280,8 +286,16 @@ public void get(int blockNumber, ByteBuffer buffer) throws IOException { } finally { entry.releaseLock(Entry.LockType.READ); } + return true; } + /** + * Read the contents of a file into a bytebuffer. + * @param path local path + * @param buffer destination. + * @return bytes read. + * @throws IOException read failure. + */ protected int readFile(Path path, ByteBuffer buffer) throws IOException { int numBytesRead = 0; int numBytes; @@ -294,57 +308,64 @@ protected int readFile(Path path, ByteBuffer buffer) throws IOException { return numBytesRead; } - private Entry getEntry(int blockNumber) { + /** + * Get an entry in the cache. + * Increases the value of {@link #numGets} + * @param blockNumber block number + * @return the entry. + */ + private synchronized Entry getEntry(int blockNumber) { Validate.checkNotNegative(blockNumber, "blockNumber"); Entry entry = blocks.get(blockNumber); - if (entry == null) { - throw new IllegalStateException(String.format("block %d not found in cache", blockNumber)); - } - numGets++; + checkState(entry != null, "block %d not found in cache", blockNumber); + numGets.getAndIncrement(); addToLinkedListHead(entry); return entry; } /** - * Helper method to add the given entry to the head of the linked list. - * + * Add the given entry to the head of the linked list if + * is not already there. + * Locks {@link #blocksLock} first. * @param entry Block entry to add. */ private void addToLinkedListHead(Entry entry) { blocksLock.writeLock().lock(); try { - addToHeadOfLinkedList(entry); + maybePushToHeadOfBlockList(entry); } finally { blocksLock.writeLock().unlock(); } } /** - * Add the given entry to the head of the linked list. - * + * Maybe Add the given entry to the head of the block list. + * No-op if the block is already in the list. * @param entry Block entry to add. + * @return true if the block was added. */ - private void addToHeadOfLinkedList(Entry entry) { + private boolean maybePushToHeadOfBlockList(Entry entry) { if (head == null) { head = entry; tail = entry; } LOG.debug( - "Block num {} to be added to the head. Current head block num: {} and tail block num: {}", - entry.blockNumber, head.blockNumber, tail.blockNumber); + "Block {} to be added to the head. Current head block {} and tail block {}; {}", + entry.blockNumber, head.blockNumber, tail.blockNumber, entry); if (entry != head) { Entry prev = entry.getPrevious(); - Entry nxt = entry.getNext(); - // no-op if the block is already evicted + Entry next = entry.getNext(); + // no-op if the block is already block list if (!blocks.containsKey(entry.blockNumber)) { - return; + LOG.debug("Block {} is already in block list", entry.blockNumber); + return false; } if (prev != null) { - prev.setNext(nxt); + prev.setNext(next); } - if (nxt != null) { - nxt.setPrevious(prev); + if (next != null) { + next.setPrevious(prev); } entry.setPrevious(null); entry.setNext(head); @@ -354,6 +375,7 @@ private void addToHeadOfLinkedList(Entry entry) { tail = prev; } } + return true; } /** @@ -366,9 +388,10 @@ private void addToHeadOfLinkedList(Entry entry) { * @throws IOException if either local dir allocator fails to allocate file or if IO error * occurs while writing the buffer content to the file. * @throws IllegalArgumentException if buffer is null, or if buffer.limit() is zero or negative. + * @throws IllegalStateException if the cache file exists and is not empty */ @Override - public void put(int blockNumber, ByteBuffer buffer, Configuration conf, + public synchronized void put(int blockNumber, ByteBuffer buffer, Configuration conf, LocalDirAllocator localDirAllocator) throws IOException { if (closed.get()) { return; @@ -377,6 +400,8 @@ public void put(int blockNumber, ByteBuffer buffer, Configuration conf, checkNotNull(buffer, "buffer"); if (blocks.containsKey(blockNumber)) { + // this block already exists. + // verify the checksum matches Entry entry = blocks.get(blockNumber); entry.takeLock(Entry.LockType.READ); try { @@ -390,14 +415,11 @@ public void put(int blockNumber, ByteBuffer buffer, Configuration conf, Validate.checkPositiveInteger(buffer.limit(), "buffer.limit()"); - Path blockFilePath = getCacheFilePath(conf, localDirAllocator); + String blockInfo = String.format("-block-%04d", blockNumber); + Path blockFilePath = getCacheFilePath(conf, localDirAllocator, blockInfo, buffer.limit()); long size = Files.size(blockFilePath); - if (size != 0) { - String message = - String.format("[%d] temp file already has data. %s (%d)", + checkState(size == 0, "[%d] temp file already has data. %s (%d)", blockNumber, blockFilePath, size); - throw new IllegalStateException(message); - } writeFile(blockFilePath, buffer); long checksum = BufferData.getChecksum(buffer); @@ -421,8 +443,9 @@ public void put(int blockNumber, ByteBuffer buffer, Configuration conf, private void addToLinkedListAndEvictIfRequired(Entry entry) { blocksLock.writeLock().lock(); try { - addToHeadOfLinkedList(entry); - entryListSize++; + if (maybePushToHeadOfBlockList(entry)) { + entryListSize++; + } if (entryListSize > maxBlocksCount && !closed.get()) { Entry elementToPurge = tail; tail = tail.getPrevious(); @@ -444,12 +467,13 @@ private void addToLinkedListAndEvictIfRequired(Entry entry) { * @param elementToPurge Block entry to evict. */ private void deleteBlockFileAndEvictCache(Entry elementToPurge) { + LOG.debug("Evicting block {} from cache: {}", elementToPurge.blockNumber, elementToPurge); try (DurationTracker ignored = trackerFactory.trackDuration(STREAM_FILE_CACHE_EVICTION)) { boolean lockAcquired = elementToPurge.takeLock(Entry.LockType.WRITE, PrefetchConstants.PREFETCH_WRITE_LOCK_TIMEOUT, PrefetchConstants.PREFETCH_WRITE_LOCK_TIMEOUT_UNIT); if (!lockAcquired) { - LOG.error("Cache file {} deletion would not be attempted as write lock could not" + LOG.warn("Cache file {} deletion would not be attempted as write lock could not" + " be acquired within {} {}", elementToPurge.path, PrefetchConstants.PREFETCH_WRITE_LOCK_TIMEOUT, PrefetchConstants.PREFETCH_WRITE_LOCK_TIMEOUT_UNIT); @@ -460,9 +484,11 @@ private void deleteBlockFileAndEvictCache(Entry elementToPurge) { prefetchingStatistics.blockRemovedFromFileCache(); blocks.remove(elementToPurge.blockNumber); prefetchingStatistics.blockEvictedFromFileCache(); + } else { + LOG.debug("Cache file {} not found for deletion: {}", elementToPurge.path, elementToPurge); } } catch (IOException e) { - LOG.warn("Failed to delete cache file {}", elementToPurge.path, e); + LOG.warn("Failed to delete cache file {} for {}", elementToPurge.path, elementToPurge, e); } finally { elementToPurge.releaseLock(Entry.LockType.WRITE); } @@ -475,9 +501,17 @@ private void deleteBlockFileAndEvictCache(Entry elementToPurge) { StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING); + /** + * Write the contents of the buffer to the path. + * @param path file to create. + * @param buffer source buffer. + * @throws IOException + */ protected void writeFile(Path path, ByteBuffer buffer) throws IOException { buffer.rewind(); - try (WritableByteChannel writeChannel = Files.newByteChannel(path, CREATE_OPTIONS)) { + try (WritableByteChannel writeChannel = Files.newByteChannel(path, CREATE_OPTIONS); + DurationInfo d = new DurationInfo(LOG, "save %d bytes to %s", + buffer.remaining(), path)) { while (buffer.hasRemaining()) { writeChannel.write(buffer); } @@ -486,17 +520,20 @@ protected void writeFile(Path path, ByteBuffer buffer) throws IOException { /** * Return temporary file created based on the file path retrieved from local dir allocator. - * * @param conf The configuration object. * @param localDirAllocator Local dir allocator instance. + * @param blockInfo info about block to use in filename + * @param fileSize size or -1 if unknown * @return Path of the temporary file created. * @throws IOException if IO error occurs while local dir allocator tries to retrieve path * from local FS or file creation fails or permission set fails. */ protected Path getCacheFilePath(final Configuration conf, - final LocalDirAllocator localDirAllocator) + final LocalDirAllocator localDirAllocator, + final String blockInfo, + final long fileSize) throws IOException { - return getTempFilePath(conf, localDirAllocator); + return getTempFilePath(conf, localDirAllocator, blockInfo, fileSize); } @Override @@ -512,6 +549,7 @@ public void close() throws IOException { */ private void deleteCacheFiles() { int numFilesDeleted = 0; + LOG.debug("Prefetch cache close: Deleting {} cache files", blocks.size()); for (Entry entry : blocks.values()) { boolean lockAcquired = entry.takeLock(Entry.LockType.WRITE, PrefetchConstants.PREFETCH_WRITE_LOCK_TIMEOUT, @@ -548,21 +586,21 @@ public String toString() { return sb.toString(); } + /** + * Validate a block entry against a buffer, including checksum comparison. + * @param entry block entry + * @param buffer buffer + * @throws IllegalStateException if invalid. + */ private void validateEntry(Entry entry, ByteBuffer buffer) { - if (entry.size != buffer.limit()) { - String message = String.format( - "[%d] entry.size(%d) != buffer.limit(%d)", - entry.blockNumber, entry.size, buffer.limit()); - throw new IllegalStateException(message); - } + checkState(entry.size == buffer.limit(), + "[%d] entry.size(%d) != buffer.limit(%d)", + entry.blockNumber, entry.size, buffer.limit()); long checksum = BufferData.getChecksum(buffer); - if (entry.checksum != checksum) { - String message = String.format( - "[%d] entry.checksum(%d) != buffer checksum(%d)", - entry.blockNumber, entry.checksum, checksum); - throw new IllegalStateException(message); - } + checkState(entry.checksum == checksum, + "[%d] entry.checksum(%d) != buffer checksum(%d)", + entry.blockNumber, entry.checksum, checksum); } /** @@ -607,30 +645,11 @@ private String getStats() { return sb.toString(); } - private static final String CACHE_FILE_PREFIX = "fs-cache-"; - /** - * Determine if the cache space is available on the local FS. - * - * @param fileSize The size of the file. - * @param conf The configuration. - * @param localDirAllocator Local dir allocator instance. - * @return True if the given file size is less than the available free space on local FS, - * False otherwise. + * Prefix for cache files: {@value}. */ - public static boolean isCacheSpaceAvailable(long fileSize, Configuration conf, - LocalDirAllocator localDirAllocator) { - try { - Path cacheFilePath = getTempFilePath(conf, localDirAllocator); - long freeSpace = new File(cacheFilePath.toString()).getUsableSpace(); - LOG.info("fileSize = {}, freeSpace = {}", fileSize, freeSpace); - Files.deleteIfExists(cacheFilePath); - return fileSize < freeSpace; - } catch (IOException e) { - LOG.error("isCacheSpaceAvailable", e); - return false; - } - } + @VisibleForTesting + public static final String CACHE_FILE_PREFIX = "fs-cache-"; // The suffix (file extension) of each serialized index file. private static final String BINARY_FILE_SUFFIX = ".bin"; @@ -639,20 +658,23 @@ public static boolean isCacheSpaceAvailable(long fileSize, Configuration conf, * Create temporary file based on the file path retrieved from local dir allocator * instance. The file is created with .bin suffix. The created file has been granted * posix file permissions available in TEMP_FILE_ATTRS. - * * @param conf the configuration. * @param localDirAllocator the local dir allocator instance. + * @param blockInfo info about block to use in filename + * @param fileSize size or -1 if unknown * @return path of the file created. * @throws IOException if IO error occurs while local dir allocator tries to retrieve path * from local FS or file creation fails or permission set fails. */ private static Path getTempFilePath(final Configuration conf, - final LocalDirAllocator localDirAllocator) throws IOException { + final LocalDirAllocator localDirAllocator, + final String blockInfo, + final long fileSize) throws IOException { org.apache.hadoop.fs.Path path = - localDirAllocator.getLocalPathForWrite(CACHE_FILE_PREFIX, conf); + localDirAllocator.getLocalPathForWrite(CACHE_FILE_PREFIX, fileSize, conf); File dir = new File(path.getParent().toUri().getPath()); String prefix = path.getName(); - File tmpFile = File.createTempFile(prefix, BINARY_FILE_SUFFIX, dir); + File tmpFile = File.createTempFile(prefix, blockInfo + BINARY_FILE_SUFFIX, dir); Path tmpFilePath = Paths.get(tmpFile.toURI()); return Files.setPosixFilePermissions(tmpFilePath, TEMP_FILE_ATTRS); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java index 57c775721384e..0e0fe3a92096c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StreamStatisticNames.java @@ -429,6 +429,44 @@ public final class StreamStatisticNames { public static final String STREAM_READ_PREFETCH_OPERATIONS = "stream_read_prefetch_operations"; + /** + * Is this stream currently saving blocks to a cache? + * + * Value: {@value}. + */ + public static final String STREAM_READ_BLOCK_CACHE_ENABLED = + "stream_read_block_cache_enabled"; + + /** + * Is this stream currently prefetching data blocks. + * + * Value: {@value}. + */ + public static final String STREAM_READ_BLOCK_PREFETCH_ENABLED = + "stream_read_block_prefetch_enabled"; + + /** + * Maximum number of blocks to prefetch. + * + * Value: {@value}. + */ + public static final String STREAM_READ_BLOCK_PREFETCH_LIMIT = + "stream_read_block_prefetch_limit"; + + /** + * Block size for fetch/prefetch. + * + * Value: {@value}. + */ + public static final String STREAM_READ_BLOCK_SIZE = + "stream_read_block_size"; + + /** + * Total number of block fetch operations executed. + */ + public static final String STREAM_READ_BLOCK_FETCH_OPERATIONS + = "stream_read_block_fetch_operations"; + /** * Total number of block in disk cache. */ diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java index 6a5d01fb3b074..ad3fb1a0a88c5 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java @@ -31,6 +31,9 @@ import java.util.function.Function; import java.util.function.Supplier; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.fs.StorageStatistics; @@ -51,6 +54,12 @@ * Support for implementing IOStatistics interfaces. */ public final class IOStatisticsBinding { + /** + * Log changes at debug. + * Noisy, but occasionally useful. + */ + private static final Logger LOG = + LoggerFactory.getLogger(IOStatisticsBinding.class); /** Pattern used for each entry. */ public static final String ENTRY_PATTERN = "(%s=%s)"; @@ -548,6 +557,7 @@ public static B invokeTrackingDuration( } catch (IOException | RuntimeException e) { // input function failed: note it tracker.failed(); + LOG.debug("Operation failure with duration {}", tracker); // and rethrow throw e; } finally { @@ -555,6 +565,7 @@ public static B invokeTrackingDuration( // this is called after the catch() call would have // set the failed flag. tracker.close(); + LOG.debug("Operation success with duration {}", tracker); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StatisticDurationTracker.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StatisticDurationTracker.java index 04d30135f6bd3..3d2d26ca320ac 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StatisticDurationTracker.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StatisticDurationTracker.java @@ -31,6 +31,10 @@ * In the constructor, the counter with name of 'key' is * incremented -default is by 1, but can be set to other * values, including 0. + *

+ * If there is also a gauge with with the same key, + * then it is incremented in the constructor and decremented + * afterwards. */ public class StatisticDurationTracker extends OperationDuration implements DurationTracker { @@ -78,6 +82,9 @@ public StatisticDurationTracker( if (count > 0) { iostats.incrementCounter(key, count); } + if (iostats.gauges().containsKey(key)) { + iostats.incrementGauge(key, 1); + } } @Override @@ -102,6 +109,9 @@ public void close() { iostats.incrementCounter(name); } iostats.addTimedOperation(name, asDuration()); + if (iostats.gauges().containsKey(key)) { + iostats.incrementGauge(key, -1); + } } @Override diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstreambuilder.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstreambuilder.md index c318a6a479b73..4ced03d300c8f 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstreambuilder.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstreambuilder.md @@ -600,6 +600,12 @@ The S3A Connector supports custom options for readahead and seek policy. | `fs.s3a.readahead.range` | `long` | readahead range in bytes | | `fs.s3a.experimental.input.fadvise` | `String` | seek policy. Superceded by `fs.option.openfile.read.policy` | | `fs.s3a.input.async.drain.threshold` | `long` | threshold to switch to asynchronous draining of the stream. (Since 3.3.5) | +| `fs.s3a.prefetch.block.size` | `int` | Block size in bytes for prefetching. | +| `fs.s3a.prefetch.block.count` | `int` | Number of blocks for prefetching. | + +Irrespective of the value of `fs.s3a.prefetch.block.count`, the maximum number of active +prefetches in a single filesystem instance is limited to the value of the same option +in the configuration used to create the filesystem instance; callers can only choose smaller values. If the option set contains a SQL statement in the `fs.s3a.select.sql` statement, then the file is opened as an S3 Select query. diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java index 53594f4db8858..7f7f0a016eed9 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java @@ -20,6 +20,7 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FSDataOutputStreamBuilder; import org.apache.hadoop.fs.FileContext; import org.apache.hadoop.fs.FileRange; import org.apache.hadoop.fs.FileStatus; @@ -158,7 +159,9 @@ public static void writeAndRead(FileSystem fs, * whether file overwrite operations should be enabled * @param fs filesystem * @param path path to write to + * @param src source buffer * @param len length of data + * @param buffersize buffer size. * @param overwrite should the create option allow overwrites? * @throws IOException IO problems */ @@ -179,7 +182,9 @@ public static void writeDataset(FileSystem fs, * * @param fs filesystem * @param path path to write to + * @param src source buffer * @param len length of data + * @param buffersize buffer size. * @param overwrite should the create option allow overwrites? * @param useBuilder should use builder API to create file? * @throws IOException IO problems diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBufferData.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBufferData.java index ee5f95ca6bbb6..1102aa17730fb 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBufferData.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBufferData.java @@ -75,7 +75,7 @@ public void testValidStateUpdates() { ByteBuffer buffer = ByteBuffer.allocate(1); BufferData data = new BufferData(1, buffer); - assertEquals(BufferData.State.BLANK, data.getState()); + assertEquals(BufferData.State.EMPTY, data.getState()); CompletableFuture actionFuture = new CompletableFuture<>(); actionFuture.complete(null); @@ -92,7 +92,7 @@ public void testValidStateUpdates() { assertNotSame(actionFuture, actionFuture2); List states = Arrays.asList( - BufferData.State.BLANK, + BufferData.State.EMPTY, BufferData.State.PREFETCHING, BufferData.State.CACHING, BufferData.State.READY @@ -116,7 +116,7 @@ public void testInvalidStateUpdates() throws Exception { actionFuture.complete(null); testInvalidStateUpdatesHelper( (d) -> d.setPrefetch(actionFuture), - BufferData.State.BLANK, + BufferData.State.EMPTY, BufferData.State.READY); testInvalidStateUpdatesHelper( @@ -137,7 +137,7 @@ public void testSetReady() throws Exception { assertNotEquals(BufferData.State.READY, data.getState()); assertEquals(0, data.getChecksum()); - data.setReady(BufferData.State.BLANK); + data.setReady(BufferData.State.EMPTY); assertEquals(BufferData.State.READY, data.getState()); assertNotEquals(0, data.getChecksum()); @@ -151,7 +151,7 @@ public void testSetReady() throws Exception { ExceptionAsserts.assertThrows( IllegalStateException.class, "Checksum cannot be changed once set", - () -> data.setReady(BufferData.State.BLANK)); + () -> data.setReady(BufferData.State.EMPTY)); // Verify that we detect post READY buffer modification. buffer.array()[2] = (byte) 42; @@ -197,7 +197,7 @@ private void testInvalidStateUpdatesHelper( ByteBuffer buffer = ByteBuffer.allocate(1); BufferData data = new BufferData(1, buffer); - data.updateState(validFromState[0], BufferData.State.BLANK); + data.updateState(validFromState[0], BufferData.State.EMPTY); List states = this.getStatesExcept(validFromState); BufferData.State prevState = validFromState[0]; String expectedMessage = @@ -217,7 +217,7 @@ private void testInvalidStateUpdatesHelper( static final List ALL_STATES = Arrays.asList( BufferData.State.UNKNOWN, - BufferData.State.BLANK, + BufferData.State.EMPTY, BufferData.State.PREFETCHING, BufferData.State.CACHING, BufferData.State.READY diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBufferPool.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBufferPool.java index b8375fe66dcb1..76da234805a1b 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBufferPool.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBufferPool.java @@ -102,13 +102,13 @@ public void testGetAndRelease() { assertEquals(2, pool.numCreated()); assertEquals(0, pool.numAvailable()); - data1.updateState(BufferData.State.READY, BufferData.State.BLANK); + data1.updateState(BufferData.State.READY, BufferData.State.EMPTY); pool.release(data1); assertEquals(2, pool.numCreated()); assertEquals(1, pool.numAvailable()); - data2.updateState(BufferData.State.READY, BufferData.State.BLANK); + data2.updateState(BufferData.State.READY, BufferData.State.EMPTY); pool.release(data2); assertEquals(2, pool.numCreated()); @@ -117,7 +117,7 @@ public void testGetAndRelease() { @Test public void testRelease() throws Exception { - testReleaseHelper(BufferData.State.BLANK, true); + testReleaseHelper(BufferData.State.EMPTY, true); testReleaseHelper(BufferData.State.PREFETCHING, true); testReleaseHelper(BufferData.State.CACHING, true); testReleaseHelper(BufferData.State.READY, false); @@ -131,7 +131,7 @@ private void testReleaseHelper(BufferData.State stateBeforeRelease, assertInitialState(pool, POOL_SIZE); BufferData data = this.acquire(pool, 1); - data.updateState(stateBeforeRelease, BufferData.State.BLANK); + data.updateState(stateBeforeRelease, BufferData.State.EMPTY); if (expectThrow) { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/Sizes.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/Sizes.java new file mode 100644 index 0000000000000..faaaba9ae8c19 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/Sizes.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.test; + +/** + * Sizes of data in KiB/MiB. + */ +public final class Sizes { + + public static final int S_256 = 256; + public static final int S_512 = 512; + public static final int S_1K = 1024; + public static final int S_2K = 2 * S_1K; + public static final int S_4K = 4 * S_1K; + public static final int S_8K = 8 * S_1K; + public static final int S_10K = 10 * S_1K; + public static final int S_16K = 16 * S_1K; + public static final int S_32K = 32 * S_1K; + public static final int S_64K = 64 * S_1K; + public static final int S_128K = 128 * S_1K; + public static final int S_256K = 256 * S_1K; + public static final int S_500K = 500 * S_1K; + public static final int S_512K = 512 * S_1K; + public static final int S_1M = S_1K * S_1K; + public static final int S_2M = 2 * S_1M; + public static final int S_4M = 4 * S_1M; + public static final int S_5M = 5 * S_1M; + public static final int S_8M = 8 * S_1M; + public static final int S_16M = 16 * S_1M; + public static final int S_10M = 10 * S_1M; + public static final int S_32M = 32 * S_1M; + public static final int S_64M = 64 * S_1M; + public static final double NANOSEC = 1.0e9; + + private Sizes() { + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index 4d52ee0d2f9c5..deb091b1a9fb2 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java @@ -1560,13 +1560,13 @@ private Constants() { // 64 MB of heap space (8 blocks x 8 MB each). /** - * The size of a single prefetched block in number of bytes. + * The size of a single prefetched block in number of bytes: {@value}. */ public static final String PREFETCH_BLOCK_SIZE_KEY = "fs.s3a.prefetch.block.size"; public static final int PREFETCH_BLOCK_DEFAULT_SIZE = 8 * 1024 * 1024; /** - * Maximum number of blocks prefetched at any given time. + * Maximum number of blocks prefetched at any given time: {@value}. */ public static final String PREFETCH_BLOCK_COUNT_KEY = "fs.s3a.prefetch.block.count"; public static final int PREFETCH_BLOCK_DEFAULT_COUNT = 8; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index c0e530cb5ce40..430723f06b14d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -357,6 +357,11 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, /** Log to warn of storage class configuration problems. */ private static final LogExactlyOnce STORAGE_CLASS_WARNING = new LogExactlyOnce(LOG); + /** + * Directory allocator, bonded to directory list set in + * {@link Constants#BUFFER_DIR} and used for storing temporary files + * including: upload file blocks, stage committer files and cached read blocks. + */ private LocalDirAllocator directoryAllocator; private String cannedACL; @@ -2063,7 +2068,7 @@ protected S3AReadOpContext createReadContext( prefetchBlockCount) .withAuditSpan(auditSpan); openFileHelper.applyDefaultOptions(roc); - return roc.build(); + return roc; } /** @@ -2624,8 +2629,8 @@ public S3ObjectAttributes createObjectAttributes( @Override public S3AReadOpContext createReadContext(final FileStatus fileStatus) { - return S3AFileSystem.this.createReadContext(fileStatus, - auditSpan); + return S3AFileSystem.this.createReadContext(fileStatus, auditSpan) + .build(); } @Override @@ -5549,7 +5554,11 @@ public boolean hasPathCapability(final Path path, final String capability) case CommonPathCapabilities.VIRTUAL_BLOCK_LOCATIONS: return true; - /* + // Is prefetching enabled? + case PREFETCH_ENABLED_KEY: + return prefetchEnabled; + + /* * Marker policy capabilities are handed off. */ case STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_KEEP: diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java index b84f19fcd8773..8652babf13331 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java @@ -220,6 +220,10 @@ public S3AInstrumentation(URI name) { storeBuilder.withDurationTracking(stat.getSymbol()); }); + // plus any gauges which are also counters; the auto-registration + // doesn't handle these. + storeBuilder.withGauges(StreamStatisticNames.STREAM_READ_BLOCK_FETCH_OPERATIONS); + //todo need a config for the quantiles interval? int interval = 1; throttleRateQuantile = quantiles(STORE_IO_THROTTLE_RATE, @@ -889,8 +893,14 @@ private InputStreamStatistics( StreamStatisticNames.STREAM_READ_VECTORED_READ_BYTES_DISCARDED, StreamStatisticNames.STREAM_READ_VERSION_MISMATCHES, StreamStatisticNames.STREAM_EVICT_BLOCKS_FROM_FILE_CACHE) - .withGauges(STREAM_READ_GAUGE_INPUT_POLICY, + .withGauges( + STREAM_READ_GAUGE_INPUT_POLICY, + STREAM_READ_BLOCK_CACHE_ENABLED.getSymbol(), + STREAM_READ_BLOCK_FETCH_OPERATIONS.getSymbol(), STREAM_READ_BLOCKS_IN_FILE_CACHE.getSymbol(), + STREAM_READ_BLOCK_PREFETCH_ENABLED.getSymbol(), + STREAM_READ_BLOCK_PREFETCH_LIMIT.getSymbol(), + STREAM_READ_BLOCK_SIZE.getSymbol(), STREAM_READ_ACTIVE_PREFETCH_OPERATIONS.getSymbol(), STREAM_READ_ACTIVE_MEMORY_IN_USE.getSymbol() ) @@ -902,7 +912,8 @@ private InputStreamStatistics( StreamStatisticNames.STREAM_READ_PREFETCH_OPERATIONS, StreamStatisticNames.STREAM_READ_REMOTE_BLOCK_READ, StreamStatisticNames.STREAM_READ_BLOCK_ACQUIRE_AND_READ, - StreamStatisticNames.STREAM_FILE_CACHE_EVICTION) + StreamStatisticNames.STREAM_FILE_CACHE_EVICTION, + StreamStatisticNames.STREAM_READ_BLOCK_FETCH_OPERATIONS) .build(); setIOStatistics(st); aborted = st.getCounterReference( @@ -1397,6 +1408,11 @@ public DurationTracker prefetchOperationStarted() { return trackDuration(StreamStatisticNames.STREAM_READ_PREFETCH_OPERATIONS); } + @Override + public DurationTracker blockFetchOperationStarted() { + return trackDuration(StreamStatisticNames.STREAM_READ_BLOCK_FETCH_OPERATIONS); + } + @Override public void blockAddedToFileCache() { incAllGauges(STREAM_READ_BLOCKS_IN_FILE_CACHE, 1); @@ -1417,6 +1433,26 @@ public void prefetchOperationCompleted() { incAllGauges(STREAM_READ_ACTIVE_PREFETCH_OPERATIONS, -1); } + @Override + public void fetchOperationCompleted(final boolean prefetch, final long bytesFetched) { + if (prefetch) { + incAllGauges(STREAM_READ_ACTIVE_PREFETCH_OPERATIONS, -1); + } + if (bytesFetched > 0) { + totalBytesRead.addAndGet(bytesFetched); + } + } + + /** + * {@inheritDoc}. + * If the byte counter is positive, increment bytesRead. + */ + @Override + public void bytesReadFromBuffer(long bytes) { + if (bytes > 0) { + bytesRead.addAndGet(bytes); + } + } @Override public void memoryAllocated(int size) { incAllGauges(STREAM_READ_ACTIVE_MEMORY_IN_USE, size); @@ -1426,6 +1462,39 @@ public void memoryAllocated(int size) { public void memoryFreed(int size) { incAllGauges(STREAM_READ_ACTIVE_MEMORY_IN_USE, -size); } + + @Override + public void setPrefetchState(final boolean prefetchEnabled, + final int blocks, + final int blocksize) { + setGaugeToBoolean(STREAM_READ_BLOCK_PREFETCH_ENABLED, prefetchEnabled); + setGauge(STREAM_READ_BLOCK_PREFETCH_LIMIT, blocks); + setGauge(STREAM_READ_BLOCK_SIZE, blocksize); + } + + @Override + public void setPrefetchDiskCachingState(final boolean cacheEnabled) { + setGaugeToBoolean(STREAM_READ_BLOCK_CACHE_ENABLED, cacheEnabled); + } + + /** + * Set a gauge. + * @param st statistic + * @param value value + */ + private void setGauge(final Statistic st, final long value) { + localIOStatistics().setGauge(st.getSymbol(), value); + } + + /** + * Set a gauge to a boolean value, mapping true to 1, false to 0, + * in the classic C Language style. + * @param st statistic + * @param flag flag to use. + */ + private void setGaugeToBoolean(final Statistic st, final boolean flag) { + setGauge(st, flag ? 1 : 0); + } } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java index 55351f0c81396..686b0f3142577 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java @@ -18,6 +18,8 @@ package org.apache.hadoop.fs.s3a; +import java.util.Optional; + import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -32,6 +34,7 @@ import org.apache.hadoop.util.Preconditions; import static java.util.Objects.requireNonNull; +import static java.util.Optional.empty; /** * Read-specific operation context struct. @@ -78,11 +81,25 @@ public class S3AReadOpContext extends S3AOpContext { // S3 reads are prefetched asynchronously using this future pool. private ExecutorServiceFuturePool futurePool; - // Size in bytes of a single prefetch block. - private final int prefetchBlockSize; + /** + * Size in bytes of a single prefetch block. + */ + private int prefetchBlockSize; + + /** + * Size of prefetch queue (in number of blocks). + */ + private int prefetchBlockCount; + + /** + * Where does the read start from, if known. + */ + private Optional splitStart = empty(); - // Size of prefetch queue (in number of blocks). - private final int prefetchBlockCount; + /** + * What is the split end, if known? + */ + private Optional splitEnd = empty(); /** * Instantiate. @@ -283,6 +300,62 @@ public int getPrefetchBlockCount() { return this.prefetchBlockCount; } + /** + * Where does the read start from, if known. + * @return split start. + */ + public Optional getSplitStart() { + return splitStart; + } + + /** + * Set split start. + * @param value new value -must not be null + * @return the builder + */ + public S3AReadOpContext withSplitStart(final Optional value) { + splitStart = requireNonNull(value); + return this; + } + + /** + * Set split end. + * @param value new value -must not be null + * @return the builder + */ + public S3AReadOpContext withSplitEnd(final Optional value) { + splitEnd = requireNonNull(value); + return this; + } + + /** + * Set builder value. + * @param value new value + * @return the builder + */ + public S3AReadOpContext withPrefetchBlockSize(final int value) { + prefetchBlockSize = value; + return this; + } + + /** + * Set builder value. + * @param value new value + * @return the builder + */ + public S3AReadOpContext withPrefetchBlockCount(final int value) { + prefetchBlockCount = value; + return this; + } + + /** + * What is the split end, if known? + * @return split end. + */ + public Optional getSplitEnd() { + return splitEnd; + } + @Override public String toString() { final StringBuilder sb = new StringBuilder( diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java index 07133a48e8e35..e221817f78878 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java @@ -444,6 +444,33 @@ public enum Statistic { StreamStatisticNames.STREAM_READ_ACTIVE_MEMORY_IN_USE, "Gauge of active memory in use", TYPE_GAUGE), + STREAM_READ_BLOCK_FETCH_OPERATIONS( + StreamStatisticNames.STREAM_READ_BLOCK_FETCH_OPERATIONS, + "Tracker/Gauge of active block fetches", + TYPE_DURATION), + STREAM_READ_BLOCK_ACQUIRE_AND_READ( + StreamStatisticNames.STREAM_READ_BLOCK_ACQUIRE_AND_READ, + "Time spent blocked waiting for blocks prefetch/read to complete", + TYPE_DURATION), + + /* Stream read block prefetch/cache gauges */ + /* As with all gauges; aggregation is of limited value */ + STREAM_READ_BLOCK_CACHE_ENABLED( + StreamStatisticNames.STREAM_READ_BLOCK_CACHE_ENABLED, + "gauge to signal the stream caches blocks", + TYPE_GAUGE), + STREAM_READ_BLOCK_PREFETCH_ENABLED( + StreamStatisticNames.STREAM_READ_BLOCK_PREFETCH_ENABLED, + "gauge to signal the stream prefetches blocks", + TYPE_GAUGE), + STREAM_READ_BLOCK_PREFETCH_LIMIT( + StreamStatisticNames.STREAM_READ_BLOCK_PREFETCH_LIMIT, + "gauge to log prefetch limit", + TYPE_GAUGE), + STREAM_READ_BLOCK_SIZE( + StreamStatisticNames.STREAM_READ_BLOCK_SIZE, + "gauge to log stream block read size", + TYPE_GAUGE), /* Stream Write statistics */ diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java index 16bae4b816457..44c001305a45f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java @@ -279,8 +279,9 @@ private class LoggingAuditSpan extends AbstractAuditSpanImpl { * Attach Range of data for GetObject Request. * @param request the sdk request to be modified * @param executionAttributes execution attributes for this request + * @return the extracted range (bytes) or null if not a GetObject request/or no range */ - private void attachRangeFromRequest(SdkHttpRequest request, + private String attachRangeFromRequest(SdkHttpRequest request, ExecutionAttributes executionAttributes) { String operationName = executionAttributes.getAttribute(AwsExecutionAttribute.OPERATION_NAME); @@ -290,10 +291,13 @@ private void attachRangeFromRequest(SdkHttpRequest request, String[] rangeHeader = request.headers().get("Range").get(0).split("="); // only set header if range unit is bytes if (rangeHeader[0].equals("bytes")) { - referrer.set(AuditConstants.PARAM_RANGE, rangeHeader[1]); + final String range = rangeHeader[1]; + referrer.set(AuditConstants.PARAM_RANGE, range); + return range; } } } + return null; } private final String description; @@ -391,7 +395,7 @@ public SdkHttpRequest modifyHttpRequest(Context.ModifyHttpRequest context, SdkRequest sdkRequest = context.request(); // attach range for GetObject requests - attachRangeFromRequest(httpRequest, executionAttributes); + final String range = attachRangeFromRequest(httpRequest, executionAttributes); // for delete op, attach the number of files to delete attachDeleteKeySizeAttribute(sdkRequest); @@ -407,11 +411,13 @@ public SdkHttpRequest modifyHttpRequest(Context.ModifyHttpRequest context, .build(); } if (LOG.isDebugEnabled()) { - LOG.debug("[{}] {} Executing {} with {}; {}", + String extra = range == null ? "" : (" range=" + range + ";"); + LOG.debug("[{}] {} Executing {} with {};{} {}", currentThreadID(), getSpanId(), getOperationName(), analyzer.analyze(context.request()), + extra, header); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java index 5bb64ddc28920..0b1bca23f2c5d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java @@ -42,6 +42,7 @@ import static org.apache.hadoop.fs.s3a.Constants.FIPS_ENDPOINT; import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_CREATE_PERFORMANCE; import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_CREATE_PERFORMANCE_ENABLED; +import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY; import static org.apache.hadoop.fs.s3a.Constants.STORE_CAPABILITY_AWS_V2; import static org.apache.hadoop.fs.s3a.impl.S3ExpressStorage.STORE_CAPABILITY_S3_EXPRESS_STORAGE; import static org.apache.hadoop.fs.s3a.Constants.STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_DELETE; @@ -122,6 +123,8 @@ private InternalConstants() { Set keys = Stream.of( Constants.ASYNC_DRAIN_THRESHOLD, Constants.INPUT_FADVISE, + Constants.PREFETCH_BLOCK_COUNT_KEY, + Constants.PREFETCH_BLOCK_SIZE_KEY, Constants.READAHEAD_RANGE) .collect(Collectors.toSet()); keys.addAll(FS_OPTION_OPENFILE_STANDARD_OPTIONS); @@ -287,7 +290,9 @@ private InternalConstants() { STORE_CAPABILITY_S3_EXPRESS_STORAGE, FS_S3A_CREATE_PERFORMANCE_ENABLED, DIRECTORY_OPERATIONS_PURGE_UPLOADS, - ENABLE_MULTI_DELETE)); + ENABLE_MULTI_DELETE, + PREFETCH_ENABLED_KEY + )); /** * AWS V4 Auth Scheme to use when creating signers: {@value}. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OpenFileSupport.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OpenFileSupport.java index b841e8f786dc4..911f0da0b2f4a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OpenFileSupport.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OpenFileSupport.java @@ -21,6 +21,7 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.util.Collection; +import java.util.Optional; import java.util.Set; import org.slf4j.Logger; @@ -38,6 +39,8 @@ import org.apache.hadoop.fs.s3a.select.SelectConstants; import org.apache.hadoop.fs.store.LogExactlyOnce; +import static java.util.Objects.requireNonNull; +import static java.util.Optional.empty; import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_BUFFER_SIZE; import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY; @@ -46,6 +49,8 @@ import static org.apache.hadoop.fs.impl.AbstractFSBuilderImpl.rejectUnknownMandatoryKeys; import static org.apache.hadoop.fs.s3a.Constants.ASYNC_DRAIN_THRESHOLD; import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADVISE; +import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_COUNT_KEY; +import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_SIZE_KEY; import static org.apache.hadoop.fs.s3a.Constants.READAHEAD_RANGE; import static org.apache.hadoop.util.Preconditions.checkArgument; @@ -252,17 +257,20 @@ public OpenFileInformation prepareToOpenFile( } FSBuilderSupport builderSupport = new FSBuilderSupport(options); // determine start and end of file. - long splitStart = builderSupport.getPositiveLong(FS_OPTION_OPENFILE_SPLIT_START, 0); + Optional splitStart = getOptionalLong(options, FS_OPTION_OPENFILE_SPLIT_START); // split end - long splitEnd = builderSupport.getLong( - FS_OPTION_OPENFILE_SPLIT_END, LENGTH_UNKNOWN); - - if (splitStart > 0 && splitStart > splitEnd) { - LOG.warn("Split start {} is greater than split end {}, resetting", + Optional splitEnd = getOptionalLong(options, FS_OPTION_OPENFILE_SPLIT_END); + // if there's a mismatch between start and end, set both to empty + if (splitEnd.isPresent() && splitEnd.get() < splitStart.orElse(0L)) { + LOG.debug("Split start {} is greater than split end {}, resetting", splitStart, splitEnd); - splitStart = 0; + splitStart = empty(); + splitEnd = empty(); } + Optional prefetchBlockSize = getOptionalInteger(options, PREFETCH_BLOCK_SIZE_KEY); + Optional prefetchBlockCount = getOptionalInteger(options, PREFETCH_BLOCK_COUNT_KEY); + // read end is the open file value fileLength = builderSupport.getPositiveLong(FS_OPTION_OPENFILE_LENGTH, fileLength); @@ -296,6 +304,8 @@ public OpenFileInformation prepareToOpenFile( S3AInputPolicy.getFirstSupportedPolicy(policies, defaultInputPolicy)) .withReadAheadRange( builderSupport.getPositiveLong(READAHEAD_RANGE, defaultReadAhead)) + .withPrefetchBlockCount(prefetchBlockCount) + .withPrefetchBlockSize(prefetchBlockSize) .withSplitStart(splitStart) .withSplitEnd(splitEnd) .withStatus(fileStatus) @@ -336,8 +346,8 @@ public OpenFileInformation openSimpleFile(final int bufferSize) { .withFileLength(LENGTH_UNKNOWN) .withInputPolicy(defaultInputPolicy) .withReadAheadRange(defaultReadAhead) - .withSplitStart(0) - .withSplitEnd(LENGTH_UNKNOWN) + .withSplitStart(empty()) + .withSplitEnd(empty()) .build(); } @@ -352,6 +362,36 @@ public String toString() { '}'; } + + /** + * Get a long value with resilience to unparseable values. + * @param options configuration to parse + * @param key key to log + * @return long value or empty() + */ + public Optional getOptionalLong(final Configuration options, String key) { + final String v = options.getTrimmed(key, ""); + if (v.isEmpty()) { + return empty(); + } + try { + return Optional.of(Long.parseLong(v)); + } catch (NumberFormatException e) { + return empty(); + } + } + + /** + * Get an int value with resilience to unparseable values. + * @param options configuration to parse + * @param key key to log + * @return long value or empty() + */ + public Optional getOptionalInteger(final Configuration options, String key) { + return getOptionalLong(options, key) + .map(l -> l.intValue()); + } + /** * The information on a file needed to open it. */ @@ -373,15 +413,24 @@ public static final class OpenFileInformation { private int bufferSize; /** - * Where does the read start from. 0 unless known. + * Where does the read start from, if known. */ - private long splitStart; + private Optional splitStart = empty(); /** - * What is the split end? - * Negative if not known. + * What is the split end, if known? + */ + private Optional splitEnd = empty(); + + /** + * Prefetch block size. */ - private long splitEnd = -1; + private Optional prefetchBlockSize = empty(); + + /** + * Prefetch block count. + */ + private Optional prefetchBlockCount = empty(); /** * What is the file length? @@ -429,11 +478,27 @@ public int getBufferSize() { return bufferSize; } - public long getSplitStart() { + public Optional getPrefetchBlockSize() { + return prefetchBlockSize; + } + + public Optional getPrefetchBlockCount() { + return prefetchBlockCount; + } + + /** + * Where does the read start from, if known. + * @return split start. + */ + public Optional getSplitStart() { return splitStart; } - public long getSplitEnd() { + /** + * What is the split end, if known? + * @return split end. + */ + public Optional getSplitEnd() { return splitEnd; } @@ -444,6 +509,8 @@ public String toString() { ", inputPolicy=" + inputPolicy + ", changePolicy=" + changePolicy + ", readAheadRange=" + readAheadRange + + ", prefetchBlockSize=" + prefetchBlockSize + + ", prefetchBlockCount=" + prefetchBlockCount + ", splitStart=" + splitStart + ", splitEnd=" + splitEnd + ", bufferSize=" + bufferSize + @@ -514,8 +581,8 @@ public OpenFileInformation withBufferSize(final int value) { * @param value new value * @return the builder */ - public OpenFileInformation withSplitStart(final long value) { - splitStart = value; + public OpenFileInformation withPrefetchBlockSize(final Optional value) { + prefetchBlockSize = value; return this; } @@ -524,11 +591,32 @@ public OpenFileInformation withSplitStart(final long value) { * @param value new value * @return the builder */ - public OpenFileInformation withSplitEnd(final long value) { - splitEnd = value; + public OpenFileInformation withPrefetchBlockCount(final Optional value) { + prefetchBlockCount = value; return this; } + /** + * Set split start. + * @param value new value -must not be null + * @return the builder + */ + public OpenFileInformation withSplitStart(final Optional value) { + splitStart = requireNonNull(value); + return this; + } + + /** + * Set split end. + * @param value new value -must not be null + * @return the builder + */ + public OpenFileInformation withSplitEnd(final Optional value) { + splitEnd = requireNonNull(value); + return this; + } + + /** * Set builder value. * @param value new value @@ -556,11 +644,16 @@ public OpenFileInformation withAsyncDrainThreshold(final long value) { * @return the context */ public S3AReadOpContext applyOptions(S3AReadOpContext roc) { - return roc - .withInputPolicy(inputPolicy) + roc.withInputPolicy(inputPolicy) .withChangeDetectionPolicy(changePolicy) .withAsyncDrainThreshold(asyncDrainThreshold) - .withReadahead(readAheadRange); + .withReadahead(readAheadRange) + .withSplitStart(splitStart) + .withSplitEnd(splitEnd); + prefetchBlockCount.map(roc::withPrefetchBlockCount); + prefetchBlockSize.map(roc::withPrefetchBlockSize); + return roc; + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ACachingInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ACachingInputStream.java index e05ad7e38b5b8..58048b84d762a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ACachingInputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ACachingInputStream.java @@ -52,12 +52,18 @@ public class S3ACachingInputStream extends S3ARemoteInputStream { private static final Logger LOG = LoggerFactory.getLogger( S3ACachingInputStream.class); + public static final int BLOCKS_TO_PREFETCH_AFTER_SEEK = 1; + /** - * Number of blocks queued for prefching. + * Number of blocks queued for prefetching. */ private final int numBlocksToPrefetch; - private final BlockManager blockManager; + private final Configuration conf; + + private final LocalDirAllocator localDirAllocator; + + private BlockManager blockManager; /** * Initializes a new instance of the {@code S3ACachingInputStream} class. @@ -79,35 +85,77 @@ public S3ACachingInputStream( S3AInputStreamStatistics streamStatistics, Configuration conf, LocalDirAllocator localDirAllocator) { + super(context, s3Attributes, client, streamStatistics); + this.conf = conf; + this.localDirAllocator = localDirAllocator; + this.numBlocksToPrefetch = getContext().getPrefetchBlockCount(); + demandCreateBlockManager(); - this.numBlocksToPrefetch = this.getContext().getPrefetchBlockCount(); - int bufferPoolSize = this.numBlocksToPrefetch + 1; - BlockManagerParameters blockManagerParamsBuilder = - new BlockManagerParameters() - .withFuturePool(this.getContext().getFuturePool()) - .withBlockData(this.getBlockData()) - .withBufferPoolSize(bufferPoolSize) - .withConf(conf) - .withLocalDirAllocator(localDirAllocator) - .withMaxBlocksCount( - conf.getInt(PREFETCH_MAX_BLOCKS_COUNT, DEFAULT_PREFETCH_MAX_BLOCKS_COUNT)) - .withPrefetchingStatistics(getS3AStreamStatistics()) - .withTrackerFactory(getS3AStreamStatistics()); - this.blockManager = this.createBlockManager(blockManagerParamsBuilder, - this.getReader()); int fileSize = (int) s3Attributes.getLen(); LOG.debug("Created caching input stream for {} (size = {})", this.getName(), fileSize); + streamStatistics.setPrefetchState( + true, + numBlocksToPrefetch, + context.getPrefetchBlockSize()); + } + + /** + * Demand create the block manager. + */ + private synchronized void demandCreateBlockManager() { + if (blockManager == null) { + LOG.debug("{}: creating block manager", getName()); + int bufferPoolSize = this.numBlocksToPrefetch + BLOCKS_TO_PREFETCH_AFTER_SEEK; + final S3AReadOpContext readOpContext = this.getContext(); + BlockManagerParameters blockManagerParamsBuilder = + new BlockManagerParameters() + .withPath(readOpContext.getPath()) + .withFuturePool(readOpContext.getFuturePool()) + .withBlockData(getBlockData()) + .withBufferPoolSize(bufferPoolSize) + .withConf(conf) + .withLocalDirAllocator(localDirAllocator) + .withMaxBlocksCount( + conf.getInt(PREFETCH_MAX_BLOCKS_COUNT, DEFAULT_PREFETCH_MAX_BLOCKS_COUNT)) + .withPrefetchingStatistics(getS3AStreamStatistics()) + .withTrackerFactory(getS3AStreamStatistics()); + blockManager = createBlockManager(blockManagerParamsBuilder, getReader()); + } } @Override public void close() throws IOException { // Close the BlockManager first, cancelling active prefetches, // deleting cached files and freeing memory used by buffer pool. - blockManager.close(); - super.close(); - LOG.info("closed: {}", getName()); + if (!isClosed()) { + closeBlockManager(); + super.close(); + LOG.info("closed: {}", getName()); + } + } + + /** + * Close the stream and the block manager. + * @param unbuffer is this an unbuffer operation? + * @return true if the stream was closed. + */ + @Override + protected boolean closeStream(final boolean unbuffer) { + final boolean b = super.closeStream(unbuffer); + closeBlockManager(); + return b; + } + + /** + * Close the block manager and set to null, if + * it is not already in this state. + */ + private synchronized void closeBlockManager() { + if (blockManager != null) { + blockManager.close(); + } } @Override @@ -115,9 +163,11 @@ protected boolean ensureCurrentBuffer() throws IOException { if (isClosed()) { return false; } + demandCreateBlockManager(); long readPos = getNextReadPos(); if (!getBlockData().isValidOffset(readPos)) { + // the block exists return false; } @@ -130,30 +180,46 @@ protected boolean ensureCurrentBuffer() throws IOException { return true; } + boolean resetPrefetching; + + // We are jumping out of the current buffer. + // if the buffer data is valid, decide whether to cache it or not. if (filePosition.isValid()) { - // We are jumping out of the current buffer. There are two cases to consider: + + // There are two cases to consider: if (filePosition.bufferFullyRead()) { // This buffer was fully read: // it is very unlikely that this buffer will be needed again; // therefore we release the buffer without caching. blockManager.release(filePosition.data()); } else { + // there's been a partial read. // We will likely need this buffer again (as observed empirically for Parquet) // therefore we issue an async request to cache this buffer. blockManager.requestCaching(filePosition.data()); } filePosition.invalidate(); + // prefetch policy is based on read order + resetPrefetching = outOfOrderRead; + } else { + // the data wasn't valid. + // do not treat this as an OOO read: leave all + // TODO: get the policy right. we want to + // leave prefetches alone if they are equal to or later than the current read pos + // but do abort any which have been skipped. + resetPrefetching = false; } int prefetchCount; - if (outOfOrderRead) { + if (resetPrefetching) { LOG.debug("lazy-seek({})", getOffsetStr(readPos)); - blockManager.cancelPrefetches(); + blockManager.cancelPrefetches(BlockManager.CancelReason.RandomIO); // We prefetch only 1 block immediately after a seek operation. - prefetchCount = 1; + prefetchCount = BLOCKS_TO_PREFETCH_AFTER_SEEK; } else { // A sequential read results in a prefetch. + // but prefetchCount = numBlocksToPrefetch; } @@ -168,8 +234,7 @@ protected boolean ensureCurrentBuffer() throws IOException { } BufferData data = invokeTrackingDuration( - getS3AStreamStatistics() - .trackDuration(STREAM_READ_BLOCK_ACQUIRE_AND_READ), + getS3AStreamStatistics().trackDuration(STREAM_READ_BLOCK_ACQUIRE_AND_READ), () -> blockManager.get(toBlockNumber)); filePosition.setData(data, startOffset, readPos); @@ -178,16 +243,26 @@ protected boolean ensureCurrentBuffer() throws IOException { @Override public String toString() { - if (isClosed()) { - return "closed"; - } - StringBuilder sb = new StringBuilder(); sb.append(String.format("%s%n", super.toString())); - sb.append(blockManager.toString()); + if (isClosed()) { + sb.append("closed"); + } else { + sb.append("file position: ").append(getFilePosition()); + // block manager may be null. + sb.append("; block manager: ").append(blockManager); + } return sb.toString(); } + /** + * Construct an instance of a {@code S3ACachingBlockManager}. + * + * @param blockManagerParameters block manager parameters + * @param reader block reader + * @return the block manager + * @throws IllegalArgumentException if reader is null. + */ protected BlockManager createBlockManager( @Nonnull final BlockManagerParameters blockManagerParameters, final S3ARemoteObjectReader reader) { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3AInMemoryInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3AInMemoryInputStream.java index e8bfe946f4abf..ccdf624c0311d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3AInMemoryInputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3AInMemoryInputStream.java @@ -32,18 +32,29 @@ import org.apache.hadoop.fs.s3a.S3ObjectAttributes; import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; +import static org.apache.hadoop.util.Preconditions.checkArgument; +import static org.apache.hadoop.util.Preconditions.checkState; + /** * Provides an {@code InputStream} that allows reading from an S3 file. * The entire file is read into memory before reads can begin. * - * Use of this class is recommended only for small files that can fit - * entirely in memory. + * Use of this class is recommended only for small files. + * When {@link #unbuffer()} is called, the memory is released. */ public class S3AInMemoryInputStream extends S3ARemoteInputStream { private static final Logger LOG = LoggerFactory.getLogger( S3AInMemoryInputStream.class); + /** + * The file size; cut down from long. + */ + private final int fileSize; + + /** + * Buffer containing the read data. + */ private ByteBuffer buffer; /** @@ -64,12 +75,38 @@ public S3AInMemoryInputStream( S3AInputStream.InputStreamCallbacks client, S3AInputStreamStatistics streamStatistics) { super(context, s3Attributes, client, streamStatistics); - int fileSize = (int) s3Attributes.getLen(); - this.buffer = ByteBuffer.allocate(fileSize); + final long len = s3Attributes.getLen(); + checkArgument(len < Integer.MAX_VALUE && len >= 0, + "Unsupported file size: %s", len); + fileSize = (int) len; + LOG.debug("Created in memory input stream for {} (size = {})", this.getName(), + fileSize); + } + + /** + * Allocates a buffer for the input stream. + * + * @throws IllegalStateException if the buffer is already allocated. + */ + private void allocateBuffer() { + checkState(buffer == null, "buffer for {} already allocated", getName()); + buffer = ByteBuffer.allocate(fileSize); + getS3AStreamStatistics().memoryAllocated(fileSize); LOG.debug("Created in-memory input stream for {} (size = {})", getName(), fileSize); } + /** + * Set the buffer to null so that GC will clean it up. + * Harmless to call on a released buffer. + */ + private void releaseBuffer() { + if (buffer != null) { + getS3AStreamStatistics().memoryFreed(fileSize); + buffer = null; + } + } + /** * Ensures that a non-empty valid buffer is available for immediate reading. * It returns true when at least one such buffer is available for reading. @@ -86,6 +123,9 @@ protected boolean ensureCurrentBuffer() throws IOException { if (getBlockData().getFileSize() == 0) { return false; } + if (buffer == null) { + allocateBuffer(); + } FilePosition filePosition = getFilePosition(); if (filePosition.isValid()) { @@ -105,4 +145,12 @@ protected boolean ensureCurrentBuffer() throws IOException { return filePosition.buffer().hasRemaining(); } + + @Override + protected boolean closeStream(final boolean unbuffer) { + final boolean b = super.closeStream(unbuffer); + releaseBuffer(); + return b; + } + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchingInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchingInputStream.java index 9b9ee12ad7502..a1326e810d562 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchingInputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchingInputStream.java @@ -29,6 +29,7 @@ import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CanSetReadahead; +import org.apache.hadoop.fs.CanUnbuffer; import org.apache.hadoop.fs.FSExceptionMessages; import org.apache.hadoop.fs.FSInputStream; import org.apache.hadoop.fs.LocalDirAllocator; @@ -49,7 +50,7 @@ */ public class S3APrefetchingInputStream extends FSInputStream - implements CanSetReadahead, StreamCapabilities, IOStatisticsSource { + implements CanSetReadahead, StreamCapabilities, IOStatisticsSource, CanUnbuffer { private static final Logger LOG = LoggerFactory.getLogger( S3APrefetchingInputStream.class); @@ -114,7 +115,7 @@ public S3APrefetchingInputStream( client, streamStatistics); } else { - LOG.debug("Creating in caching input stream for {}", context.getPath()); + LOG.debug("Creating caching input stream for {}", context.getPath()); this.inputStream = new S3ACachingInputStream( context, s3Attributes, @@ -154,6 +155,19 @@ public synchronized long getPos() throws IOException { return lastReadCurrentPos; } + /** + * Ensure the stream is active by initializing the inner input stream if + * neeeded. + * @throws IOException failure + */ + protected synchronized void ensureStreamActive() throws IOException { + throwIfClosed(); + + if (inputStream.underlyingResourcesClosed()) { + inputStream.initializeUnderlyingResources(); + } + } + /** * Reads and returns one byte from this stream. * @@ -162,7 +176,7 @@ public synchronized long getPos() throws IOException { */ @Override public synchronized int read() throws IOException { - throwIfClosed(); + ensureStreamActive(); return inputStream.read(); } @@ -180,10 +194,17 @@ public synchronized int read() throws IOException { @Override public synchronized int read(byte[] buffer, int offset, int len) throws IOException { - throwIfClosed(); + ensureStreamActive(); return inputStream.read(buffer, offset, len); } + @Override + public synchronized void unbuffer() { + if (inputStream != null) { + inputStream.unbuffer(); + } + } + /** * Closes this stream and releases all acquired resources. * @@ -286,4 +307,13 @@ public boolean seekToNewSource(long targetPos) throws IOException { public boolean markSupported() { return false; } + + @Override + public void readFully(final long position, + final byte[] buffer, + final int offset, + final int length) throws IOException { + ensureStreamActive(); + inputStream.readFully(position, buffer, offset, length); + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteInputStream.java index 38d740bd74f94..7cb22ee0ce179 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteInputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteInputStream.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.io.InputStream; import java.nio.ByteBuffer; +import java.util.concurrent.atomic.AtomicBoolean; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -30,7 +31,9 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.CanSetReadahead; +import org.apache.hadoop.fs.CanUnbuffer; import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.fs.FSInputStream; import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.fs.impl.prefetch.BlockData; import org.apache.hadoop.fs.impl.prefetch.FilePosition; @@ -42,6 +45,7 @@ import org.apache.hadoop.fs.s3a.impl.ChangeTracker; import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsAggregator; import org.apache.hadoop.fs.statistics.IOStatisticsSource; import static java.util.Objects.requireNonNull; @@ -50,8 +54,8 @@ * Provides an {@link InputStream} that allows reading from an S3 file. */ public abstract class S3ARemoteInputStream - extends InputStream - implements CanSetReadahead, StreamCapabilities, IOStatisticsSource { + extends FSInputStream + implements CanSetReadahead, StreamCapabilities, IOStatisticsSource, CanUnbuffer { private static final Logger LOG = LoggerFactory.getLogger( S3ARemoteInputStream.class); @@ -74,7 +78,13 @@ public abstract class S3ARemoteInputStream /** * Indicates whether the stream has been closed. */ - private volatile boolean closed; + private final AtomicBoolean closed = new AtomicBoolean(false); + + /** + * Indicates whether resources have been freed, used when the stream is closed + * via unbuffer. + */ + private final AtomicBoolean underlyingResourcesClosed = new AtomicBoolean(true); /** * Internal position within the file. Updated lazily @@ -107,8 +117,14 @@ public abstract class S3ARemoteInputStream private final ChangeTracker changeTracker; + /** + * IOStatistics. + */ private final IOStatistics ioStatistics; + /** Aggregator used to aggregate per thread IOStatistics. */ + private final IOStatisticsAggregator threadIOStatistics; + /** * Initializes a new instance of the {@code S3ARemoteInputStream} class. * @@ -132,25 +148,31 @@ public S3ARemoteInputStream( this.client = requireNonNull(client); this.streamStatistics = requireNonNull(streamStatistics); this.ioStatistics = streamStatistics.getIOStatistics(); - this.name = S3ARemoteObject.getPath(s3Attributes); + this.name = context.getPath().toUri().toString(); this.changeTracker = new ChangeTracker( this.name, context.getChangeDetectionPolicy(), this.streamStatistics.getChangeTrackerStatistics(), s3Attributes); + this.threadIOStatistics = requireNonNull(context.getIOStatisticsAggregator()); + this.nextReadPos = 0; setInputPolicy(context.getInputPolicy()); setReadahead(context.getReadahead()); + initializeUnderlyingResources(); + } + /** + * Initializes those resources that the stream uses but are released during unbuffer. + */ + protected final void initializeUnderlyingResources() { + underlyingResourcesClosed.set(false); long fileSize = s3Attributes.getLen(); int bufferSize = context.getPrefetchBlockSize(); - - this.blockData = new BlockData(fileSize, bufferSize); - this.fpos = new FilePosition(fileSize, bufferSize); - this.remoteObject = getS3File(); - this.reader = new S3ARemoteObjectReader(remoteObject); - - this.nextReadPos = 0; + remoteObject = getS3File(); + reader = new S3ARemoteObjectReader(remoteObject); + blockData = new BlockData(fileSize, bufferSize); + fpos = new FilePosition(fileSize, bufferSize); } /** @@ -180,7 +202,7 @@ public S3AInputStreamStatistics getS3AStreamStatistics() { * @param readahead the number of bytes to read ahead each time.. */ @Override - public synchronized void setReadahead(Long readahead) { + public final void setReadahead(Long readahead) { // We support read head by prefetching therefore we ignore the supplied value. if (readahead != null) { Validate.checkNotNegative(readahead, "readahead"); @@ -196,7 +218,8 @@ public synchronized void setReadahead(Long readahead) { @Override public boolean hasCapability(String capability) { return capability.equalsIgnoreCase(StreamCapabilities.IOSTATISTICS) - || capability.equalsIgnoreCase(StreamCapabilities.READAHEAD); + || capability.equalsIgnoreCase(StreamCapabilities.READAHEAD) + || capability.equalsIgnoreCase(StreamCapabilities.UNBUFFER); } /** @@ -216,6 +239,11 @@ private void setInputPolicy(S3AInputPolicy inputPolicy) { public int available() throws IOException { throwIfClosed(); + // buffer is closed. + if (underlyingResourcesClosed.get()) { + return 0; + } + // Update the current position in the current buffer, if possible. if (!fpos.setAbsolute(nextReadPos)) { return 0; @@ -265,12 +293,18 @@ public void seek(long pos) throws IOException { public int read() throws IOException { throwIfClosed(); - if (remoteObject.size() == 0 - || nextReadPos >= remoteObject.size()) { + if (remoteObject.size() == 0) { + LOG.debug("Rejecting read on empty file"); + return -1; + } + + if (nextReadPos >= remoteObject.size()) { + LOG.debug("Rejecting read past EOF"); return -1; } if (!ensureCurrentBuffer()) { + LOG.debug("Empty buffer in cache"); return -1; } @@ -308,17 +342,23 @@ public int read(byte[] buffer) throws IOException { @Override public int read(byte[] buffer, int offset, int len) throws IOException { throwIfClosed(); - + validatePositionedReadArgs(nextReadPos, buffer, offset, len); if (len == 0) { return 0; } - if (remoteObject.size() == 0 - || nextReadPos >= remoteObject.size()) { + if (remoteObject.size() == 0) { + LOG.debug("Rejecting read on empty file"); + return -1; + } + + if (nextReadPos >= remoteObject.size()) { + LOG.debug("Rejecting read past EOF"); return -1; } if (!ensureCurrentBuffer()) { + LOG.debug("Empty buffer in cache"); return -1; } @@ -339,49 +379,67 @@ public int read(byte[] buffer, int offset, int len) throws IOException { numBytesRemaining -= bytesToRead; numBytesRead += bytesToRead; } - return numBytesRead; } - protected S3ARemoteObject getFile() { + /** + * Forward to superclass after updating the {@code readFully()} IOStatistics. + * {@inheritDoc} + */ + @Override + public void readFully(final long position, + final byte[] buffer, + final int offset, + final int length) throws IOException { + throwIfClosed(); + validatePositionedReadArgs(position, buffer, offset, length); + streamStatistics.readFullyOperationStarted(position, length); + super.readFully(position, buffer, offset, length); + } + + protected final S3ARemoteObject getRemoteObject() { return remoteObject; } - protected S3ARemoteObjectReader getReader() { + protected final S3ARemoteObjectReader getReader() { return reader; } - protected S3ObjectAttributes getS3ObjectAttributes() { + protected final S3ObjectAttributes getS3ObjectAttributes() { return s3Attributes; } - protected FilePosition getFilePosition() { + protected final FilePosition getFilePosition() { return fpos; } - protected String getName() { + protected final String getName() { return name; } - protected boolean isClosed() { - return closed; + protected final boolean isClosed() { + return closed.get(); + } + + protected final boolean underlyingResourcesClosed() { + return underlyingResourcesClosed.get(); } - protected long getNextReadPos() { + protected final long getNextReadPos() { return nextReadPos; } - protected BlockData getBlockData() { + protected final BlockData getBlockData() { return blockData; } - protected S3AReadOpContext getContext() { + protected final S3AReadOpContext getContext() { return context; } private void incrementBytesRead(int bytesRead) { if (bytesRead > 0) { - streamStatistics.bytesRead(bytesRead); + streamStatistics.bytesReadFromBuffer(bytesRead); if (getContext().getStats() != null) { getContext().getStats().incrementBytesRead(bytesRead); } @@ -399,6 +457,12 @@ protected S3ARemoteObject getS3File() { ); } + /** + * Get string info on offset, mapping to block number:offset. + * @param offset absolute position + * @return a string of block number and offset, with a block number of "-1:" + * if the offset is invalid. + */ protected String getOffsetStr(long offset) { int blockNumber = -1; @@ -409,6 +473,36 @@ protected String getOffsetStr(long offset) { return String.format("%d:%d", blockNumber, offset); } + @Override + public synchronized void unbuffer() { + LOG.debug("{}: unbuffered", getName()); + if (closeStream(true)) { + getS3AStreamStatistics().unbuffered(); + } + } + + /** + * Close the stream in close() or unbuffer(). + * @param unbuffer is this an unbuffer operation? + * @return true if the stream was closed; false means it was already closed. + */ + protected boolean closeStream(final boolean unbuffer) { + + if (underlyingResourcesClosed.getAndSet(true)) { + return false; + } + + // release all the blocks + blockData = null; + + reader.close(); + reader = null; + // trigger GC. + remoteObject = null; + fpos.invalidate(); + return true; + } + /** * Closes this stream and releases all acquired resources. * @@ -416,20 +510,17 @@ protected String getOffsetStr(long offset) { */ @Override public void close() throws IOException { - if (closed) { + if (closed.getAndSet(true)) { return; } - closed = true; + closeStream(false); - blockData = null; - reader.close(); - reader = null; - remoteObject = null; - fpos.invalidate(); try { client.close(); } finally { streamStatistics.close(); + // Collect ThreadLevel IOStats + threadIOStatistics.aggregate(streamStatistics.getIOStatistics()); } client = null; } @@ -452,7 +543,7 @@ public String toString() { } protected void throwIfClosed() throws IOException { - if (closed) { + if (closed.get()) { throw new IOException( name + ": " + FSExceptionMessages.STREAM_IS_CLOSED); } @@ -479,7 +570,7 @@ public void reset() { } @Override - public long skip(long n) { - throw new UnsupportedOperationException("skip not supported"); + public boolean seekToNewSource(long targetPos) throws IOException { + return false; } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObject.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObject.java index ec6e3700226e0..34d08d88079b1 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObject.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObject.java @@ -29,6 +29,7 @@ import org.apache.hadoop.fs.impl.prefetch.Validate; import org.apache.hadoop.fs.s3a.Invoker; +import org.apache.hadoop.fs.s3a.Retries; import org.apache.hadoop.fs.s3a.S3AInputStream; import org.apache.hadoop.fs.s3a.S3AReadOpContext; import org.apache.hadoop.fs.s3a.S3AUtils; @@ -36,7 +37,8 @@ import org.apache.hadoop.fs.s3a.impl.ChangeTracker; import org.apache.hadoop.fs.s3a.impl.SDKStreamDrainer; import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; -import org.apache.hadoop.fs.statistics.DurationTracker; + +import static org.apache.hadoop.fs.s3a.Invoker.onceTrackingDuration; /** @@ -82,6 +84,8 @@ public class S3ARemoteObject { */ private static final int DRAIN_BUFFER_SIZE = 16384; + public static final String OPERATION_GET = "GET"; + /** * Initializes a new instance of the {@code S3ARemoteObject} class. * @@ -115,7 +119,7 @@ public S3ARemoteObject( this.client = client; this.streamStatistics = streamStatistics; this.changeTracker = changeTracker; - this.uri = this.getPath(); + this.uri = context.getPath().toUri().toString(); } /** @@ -142,18 +146,7 @@ public S3AInputStreamStatistics getStatistics() { * @return the path of this file. */ public String getPath() { - return getPath(s3Attributes); - } - - /** - * Gets the path corresponding to the given s3Attributes. - * - * @param s3Attributes attributes of an S3 object. - * @return the path corresponding to the given s3Attributes. - */ - public static String getPath(S3ObjectAttributes s3Attributes) { - return String.format("s3a://%s/%s", s3Attributes.getBucket(), - s3Attributes.getKey()); + return uri; } /** @@ -178,6 +171,7 @@ public long size() { * @throws IllegalArgumentException if offset is greater than or equal to file size. * @throws IllegalArgumentException if size is greater than the remaining bytes. */ + @Retries.OnceTranslated public ResponseInputStream openForRead(long offset, int size) throws IOException { Validate.checkNotNegative(offset, "offset"); @@ -192,23 +186,26 @@ public ResponseInputStream openForRead(long offset, int size) .build(); String operation = String.format( - "%s %s at %d", S3AInputStream.OPERATION_OPEN, uri, offset); - DurationTracker tracker = streamStatistics.initiateGetRequest(); - ResponseInputStream object = null; - - try { - object = Invoker.once(operation, uri, () -> client.getObject(request)); - } catch (IOException e) { - tracker.failed(); - throw e; - } finally { - tracker.close(); - } + "%s %s at %d size %d", S3AInputStream.OPERATION_OPEN, uri, offset, size); + ResponseInputStream object; + + // initiate the GET. This completes once the request returns the response headers; + // the data is read later. + object = onceTrackingDuration(operation, uri, streamStatistics.initiateGetRequest(), + () -> client.getObject(request)); changeTracker.processResponse(object.response(), operation, offset); return object; } + /** + * Close the input stream, draining it first. + * If the number of bytes is above a configured threshold, + * the stream is drained asynchronously + * @param inputStream stream to close + * @param numRemainingBytes number of bytes left in the stream. + * @throws IllegalArgumentException unknown stream. + */ void close(ResponseInputStream inputStream, int numRemainingBytes) { SDKStreamDrainer drainer = new SDKStreamDrainer( uri, diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObjectReader.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObjectReader.java index b49b2699f916b..bdcc02bc5120e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObjectReader.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObjectReader.java @@ -22,14 +22,16 @@ import java.io.Closeable; import java.io.EOFException; import java.io.IOException; -import java.net.SocketTimeoutException; import java.nio.ByteBuffer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.impl.prefetch.Validate; +import org.apache.hadoop.fs.s3a.HttpChannelEOFException; import org.apache.hadoop.fs.s3a.Invoker; +import org.apache.hadoop.fs.s3a.RangeNotSatisfiableEOFException; +import org.apache.hadoop.fs.s3a.Retries; import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; import software.amazon.awssdk.core.ResponseInputStream; @@ -78,13 +80,14 @@ public S3ARemoteObjectReader(S3ARemoteObject remoteObject) { * @param offset the absolute offset into the underlying file where reading starts. * @param size the number of bytes to be read. * - * @return number of bytes actually read. + * @return number of bytes actually read. -1 if the file is closed. * @throws IOException if there is an error reading from the file. * * @throws IllegalArgumentException if buffer is null. * @throws IllegalArgumentException if offset is outside of the range [0, file size]. * @throws IllegalArgumentException if size is zero or negative. */ + @Retries.RetryTranslated public int read(ByteBuffer buffer, long offset, int size) throws IOException { Validate.checkNotNull(buffer, "buffer"); Validate.checkWithinRange(offset, "offset", 0, this.remoteObject.size()); @@ -103,42 +106,80 @@ public void close() { this.closed = true; } + /** + * Reads one block from S3. + *

+ * There are no retries on base EOFExceptions. + * {@link HttpChannelEOFException} will be retried. + * {@link RangeNotSatisfiableEOFException} will be downgraded to + * partial read, so data may be returned. + * @param buffer destination. + * @param offset object offset + * @param size size to retrieve. + * @return bytes read. + * @throws EOFException if this was raised. + * @throws IOException IO failure. + */ + @Retries.RetryTranslated private int readOneBlockWithRetries(ByteBuffer buffer, long offset, int size) throws IOException { this.streamStatistics.readOperationStarted(offset, size); Invoker invoker = this.remoteObject.getReadInvoker(); - int invokerResponse = - invoker.retry("read", this.remoteObject.getPath(), true, + final String path = this.remoteObject.getPath(); + EOFException invokerResponse = + invoker.retry(String.format("read %s [%d-%d]", path, offset, offset + size), + path, true, trackDurationOfOperation(streamStatistics, STREAM_READ_REMOTE_BLOCK_READ, () -> { try { this.readOneBlock(buffer, offset, size); + } catch (HttpChannelEOFException e) { + // EOF subclasses with are rethrown as errors. + this.remoteObject.getStatistics().readException(); + throw e; } catch (EOFException e) { // the base implementation swallows EOFs. - return -1; - } catch (SocketTimeoutException e) { - throw e; + return e; } catch (IOException e) { this.remoteObject.getStatistics().readException(); throw e; } - return 0; + return null; })); int numBytesRead = buffer.position(); buffer.limit(numBytesRead); + if (numBytesRead != size) { + // fewer bytes came back; log at debug + LOG.debug("Expected to read {} bytes but got {}", size, numBytesRead); + } this.remoteObject.getStatistics() .readOperationCompleted(size, numBytesRead); - if (invokerResponse < 0) { - return invokerResponse; - } else { - return numBytesRead; + if (invokerResponse != null) { + if (invokerResponse instanceof RangeNotSatisfiableEOFException) { + // the range wasn't satisfiable, but some may have been read. + return numBytesRead; + } else { + throw invokerResponse; + } } + + // how much was read? + return numBytesRead; } + /** + * GET one block from S3. + * @param buffer buffer to fill up. + * @param offset offset within the object. + * @param size size to retrieve. + * @throws IOException IO failure. + * @throws HttpChannelEOFException if the channel is closed during the read. + */ + @Retries.OnceTranslated private void readOneBlock(ByteBuffer buffer, long offset, int size) throws IOException { int readSize = Math.min(size, buffer.remaining()); @@ -162,7 +203,7 @@ private void readOneBlock(ByteBuffer buffer, long offset, int size) String message = String.format( "Unexpected end of stream: buffer[%d], readSize = %d, numRemainingBytes = %d", buffer.capacity(), readSize, numRemainingBytes); - throw new EOFException(message); + throw new HttpChannelEOFException(remoteObject.getPath(), message, null); } if (numBytes > 0) { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/EmptyS3AStatisticsContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/EmptyS3AStatisticsContext.java index e47656efd1800..650ad1466d94a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/EmptyS3AStatisticsContext.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/EmptyS3AStatisticsContext.java @@ -231,6 +231,11 @@ public void prefetchOperationCompleted() { } + @Override + public void fetchOperationCompleted(final boolean prefetch, final long bytesFetched) { + + } + @Override public void blockAddedToFileCache() { diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/prefetching.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/prefetching.md index 8bb85008e3624..dd50cdc879850 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/prefetching.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/prefetching.md @@ -37,11 +37,11 @@ Multiple blocks may be read in parallel. ### Configuring the stream -|Property |Meaning |Default | -|---|---|---| -|`fs.s3a.prefetch.enabled` |Enable the prefetch input stream |`false` | -|`fs.s3a.prefetch.block.size` |Size of a block |`8M` | -|`fs.s3a.prefetch.block.count` |Number of blocks to prefetch |`8` | +| Property | Meaning | Default | +|-------------------------------|----------------------------------|---------| +| `fs.s3a.prefetch.enabled` | Enable the prefetch input stream | `false` | +| `fs.s3a.prefetch.block.size` | Size of a block | `8M` | +| `fs.s3a.prefetch.block.count` | Number of blocks to prefetch | `8` | The default size of a block is 8MB, and the minimum allowed block size is 1 byte. Decreasing block size will increase the number of blocks to be read for a file. @@ -100,7 +100,7 @@ the `S3InMemoryInputStream` will be used. If the caller makes the following read calls: -``` +```java in.read(buffer, 0, 3MB); in.read(buffer, 0, 2MB); ``` diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractOpen.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractOpen.java index 82a7a3c63b37f..4dc4cc3a5d0e6 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractOpen.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractOpen.java @@ -19,8 +19,11 @@ package org.apache.hadoop.fs.contract.s3a; import java.io.FileNotFoundException; +import java.util.Collection; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -32,18 +35,51 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.PREFETCH_OPTIONS; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.setPrefetchOption; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.prepareTestConfiguration; import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * S3A contract tests opening files. */ +@RunWith(Parameterized.class) public class ITestS3AContractOpen extends AbstractContractOpenTest { + public ITestS3AContractOpen(final boolean prefetch) { + this.prefetch = prefetch; + } + + /** + * Parameterization. + */ + @Parameterized.Parameters(name = "prefetch={0}") + public static Collection params() { + return PREFETCH_OPTIONS; + } + + /** + * Prefetch flag. + */ + private final boolean prefetch; + @Override protected AbstractFSContract createContract(Configuration conf) { return new S3AContract(conf); } + /** + * Create a configuration. + * @return a configuration + */ + @Override + protected Configuration createConfiguration() { + final Configuration conf = prepareTestConfiguration(super.createConfiguration()); + disableFilesystemCaching(conf); + return setPrefetchOption(conf, prefetch); + } + /** * S3A always declares zero byte files as encrypted. * @return true, always. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java index dd41583de3fe4..00209e503357b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractSeek.java @@ -47,11 +47,12 @@ import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_DEFAULT; import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_RANDOM; import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.setPrefetchOption; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.prepareTestConfiguration; import static org.apache.hadoop.util.Preconditions.checkNotNull; import static org.apache.hadoop.fs.s3a.Constants.INPUT_FADVISE; import static org.apache.hadoop.fs.s3a.Constants.READAHEAD_RANGE; import static org.apache.hadoop.fs.s3a.Constants.SSL_CHANNEL_MODE; -import static org.apache.hadoop.fs.s3a.S3ATestConstants.FS_S3A_IMPL_DISABLE_CACHE; import static org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory. SSLChannelMode.Default_JSSE; import static org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory. @@ -75,32 +76,42 @@ public class ITestS3AContractSeek extends AbstractContractSeekTest { private final String seekPolicy; private final DelegatingSSLSocketFactory.SSLChannelMode sslChannelMode; + /** + * Prefetch flag. + */ + private final boolean prefetch; + public static final int DATASET_LEN = READAHEAD * 2; public static final byte[] DATASET = ContractTestUtils.dataset(DATASET_LEN, 'a', 32); /** * This test suite is parameterized for the different seek policies - * which S3A Supports. + * which S3A Supports, openssl policy and whether prefetch is enabled or + * not. + * The openssl option is only executed if the required openssl library + * is found on the path; skipped otherwise. * @return a list of seek policies to test. */ - @Parameterized.Parameters + @Parameterized.Parameters(name = "policy={0}-ssl={1}-prefetch={2}") public static Collection params() { return Arrays.asList(new Object[][]{ - {FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL, Default_JSSE}, - {FS_OPTION_OPENFILE_READ_POLICY_RANDOM, OpenSSL}, - {FS_OPTION_OPENFILE_READ_POLICY_DEFAULT, Default_JSSE_with_GCM}, + {FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL, Default_JSSE, false}, + {FS_OPTION_OPENFILE_READ_POLICY_RANDOM, OpenSSL, true}, // requires openssl + {FS_OPTION_OPENFILE_READ_POLICY_DEFAULT, Default_JSSE_with_GCM, true}, }); } /** * Run the test with a chosen seek policy. * @param seekPolicy fadvise policy to use. + * @param prefetch */ public ITestS3AContractSeek(final String seekPolicy, - final DelegatingSSLSocketFactory.SSLChannelMode sslChannelMode) { + final DelegatingSSLSocketFactory.SSLChannelMode sslChannelMode, final boolean prefetch) { this.seekPolicy = seekPolicy; this.sslChannelMode = sslChannelMode; + this.prefetch = prefetch; } /** @@ -112,7 +123,8 @@ public ITestS3AContractSeek(final String seekPolicy, */ @Override protected Configuration createConfiguration() { - Configuration conf = super.createConfiguration(); + Configuration conf = setPrefetchOption( + prepareTestConfiguration(super.createConfiguration()), prefetch); // purge any per-bucket overrides. try { URI bucketURI = new URI(checkNotNull(conf.get("fs.contract.test.fs.s3a"))); @@ -136,16 +148,6 @@ protected AbstractFSContract createContract(Configuration conf) { return new S3AContract(conf); } - @Override - public void teardown() throws Exception { - super.teardown(); - S3AFileSystem fs = getFileSystem(); - if (fs != null && fs.getConf().getBoolean(FS_S3A_IMPL_DISABLE_CACHE, - false)) { - fs.close(); - } - } - /** * This subclass of the {@code path(path)} operation adds the seek policy * to the end to guarantee uniqueness across different calls of the same diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractUnbuffer.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractUnbuffer.java index 2c7149ff5cb15..89d96f5c7f88c 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractUnbuffer.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractUnbuffer.java @@ -18,12 +18,51 @@ package org.apache.hadoop.fs.contract.s3a; +import java.util.Collection; + +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.contract.AbstractContractUnbufferTest; import org.apache.hadoop.fs.contract.AbstractFSContract; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.PREFETCH_OPTIONS; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.setPrefetchOption; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.prepareTestConfiguration; + +@RunWith(Parameterized.class) public class ITestS3AContractUnbuffer extends AbstractContractUnbufferTest { + /** + * Parameterization. + */ + @Parameterized.Parameters(name = "prefetch={0}") + public static Collection params() { + return PREFETCH_OPTIONS; + } + + /** + * Prefetch flag. + */ + private final boolean prefetch; + + public ITestS3AContractUnbuffer(final boolean prefetch) { + this.prefetch = prefetch; + } + + /** + * Create a configuration. + * @return a configuration + */ + @Override + protected Configuration createConfiguration() { + final Configuration conf = prepareTestConfiguration(super.createConfiguration()); + disableFilesystemCaching(conf); + return setPrefetchOption(conf, prefetch); + } + @Override protected AbstractFSContract createContract(Configuration conf) { return new S3AContract(conf); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/S3AContract.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/S3AContract.java index 0d3dd4c2f6606..d585bd525fe8b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/S3AContract.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/S3AContract.java @@ -18,14 +18,20 @@ package org.apache.hadoop.fs.contract.s3a; +import java.io.IOException; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.AbstractBondedFSContract; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ATestUtils; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.closeIfUncached; + /** * The contract of S3A: only enabled if the test bucket is provided. + * If the configuration supplied disabled caching of the test filesystem, + * then it will be closed in {@link #teardown()}. */ public class S3AContract extends AbstractBondedFSContract { @@ -72,4 +78,15 @@ public String getScheme() { public Path getTestPath() { return S3ATestUtils.createTestPath(super.getTestPath()); } + + /** + * teardown: If the fs was created without caching, + * close it. + * @throws IOException if the superclass teardown raises it. + */ + @Override + public void teardown() throws IOException { + closeIfUncached(getTestFileSystem()); + super.teardown(); + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingCacheFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingCacheFiles.java index 5e6731ed520ad..22ec3d4285937 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingCacheFiles.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingCacheFiles.java @@ -38,11 +38,10 @@ import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest; import static org.apache.hadoop.fs.s3a.Constants.BUFFER_DIR; -import static org.apache.hadoop.fs.s3a.Constants.ENDPOINT; +import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_COUNT_KEY; import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_SIZE_KEY; import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY; import static org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils.getExternalData; -import static org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils.isUsingDefaultExternalDataFile; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; /** @@ -58,6 +57,11 @@ public class ITestS3APrefetchingCacheFiles extends AbstractS3ACostTest { public static final int PREFETCH_OFFSET = 10240; + /** + * Block prefetch count: {@value}. + */ + public static final int BLOCK_COUNT = 8; + private Path testFile; /** The FS with the external file. */ @@ -86,16 +90,18 @@ public void setUp() throws Exception { @Override public Configuration createConfiguration() { Configuration configuration = super.createConfiguration(); - if (isUsingDefaultExternalDataFile(configuration)) { - S3ATestUtils.removeBaseAndBucketOverrides(configuration, - PREFETCH_ENABLED_KEY, - ENDPOINT); - } + S3ATestUtils.removeBaseAndBucketOverrides(configuration, + PREFETCH_ENABLED_KEY, + PREFETCH_BLOCK_COUNT_KEY, + PREFETCH_BLOCK_SIZE_KEY); configuration.setBoolean(PREFETCH_ENABLED_KEY, true); // use a small block size unless explicitly set in the test config. configuration.setInt(PREFETCH_BLOCK_SIZE_KEY, BLOCK_SIZE); + configuration.setInt(PREFETCH_BLOCK_COUNT_KEY, BLOCK_COUNT); + // patch buffer dir with a unique path for test isolation. - final String bufferDirBase = configuration.get(BUFFER_DIR); + + final String bufferDirBase = "target/prefetch"; bufferDir = bufferDirBase + "/" + UUID.randomUUID(); configuration.set(BUFFER_DIR, bufferDir); return configuration; @@ -131,7 +137,7 @@ public void testCacheFileExistence() throws Throwable { in.read(prefetchBlockSize * 2, buffer, 0, prefetchBlockSize); - File tmpFileDir = new File(conf.get(BUFFER_DIR)); + File tmpFileDir = new File(bufferDir); final LocalFileSystem localFs = FileSystem.getLocal(conf); Path bufferDirPath = new Path(tmpFileDir.toURI()); ContractTestUtils.assertIsDirectory(localFs, bufferDirPath); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingInputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingInputStream.java deleted file mode 100644 index 4998cbc946e12..0000000000000 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingInputStream.java +++ /dev/null @@ -1,294 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.s3a; - -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.contract.ContractTestUtils; -import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest; -import org.apache.hadoop.fs.s3a.prefetch.S3APrefetchingInputStream; -import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; -import org.apache.hadoop.fs.statistics.IOStatistics; -import org.apache.hadoop.test.LambdaTestUtils; - -import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_SIZE_KEY; -import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY; -import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticMaximum; -import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; -import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticGaugeValue; -import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_EXECUTOR_ACQUIRED; -import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_HTTP_GET_REQUEST; -import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MAX; -import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_ACTIVE_MEMORY_IN_USE; -import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BLOCKS_IN_FILE_CACHE; -import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_OPENED; -import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_PREFETCH_OPERATIONS; - -/** - * Test the prefetching input stream, validates that the underlying S3ACachingInputStream and - * S3AInMemoryInputStream are working as expected. - */ -public class ITestS3APrefetchingInputStream extends AbstractS3ACostTest { - - public ITestS3APrefetchingInputStream() { - super(true); - } - - private static final Logger LOG = - LoggerFactory.getLogger(ITestS3APrefetchingInputStream.class); - - private static final int S_500 = 512; - private static final int S_1K = S_500 * 2; - private static final int S_1M = S_1K * S_1K; - private int numBlocks; - - // Size should be > block size so S3ACachingInputStream is used - private long largeFileSize; - - // Size should be < block size so S3AInMemoryInputStream is used - private static final int SMALL_FILE_SIZE = S_1K * 9; - - private static final int TIMEOUT_MILLIS = 5000; - private static final int INTERVAL_MILLIS = 500; - private static final int BLOCK_SIZE = S_1K * 10; - - @Override - public Configuration createConfiguration() { - Configuration conf = super.createConfiguration(); - S3ATestUtils.removeBaseAndBucketOverrides(conf, PREFETCH_ENABLED_KEY); - S3ATestUtils.removeBaseAndBucketOverrides(conf, PREFETCH_BLOCK_SIZE_KEY); - conf.setBoolean(PREFETCH_ENABLED_KEY, true); - conf.setInt(PREFETCH_BLOCK_SIZE_KEY, BLOCK_SIZE); - return conf; - } - - private void createLargeFile() throws Exception { - byte[] data = ContractTestUtils.dataset(S_1K * 72, 'x', 26); - Path largeFile = methodPath(); - FileSystem largeFileFS = getFileSystem(); - ContractTestUtils.writeDataset(getFileSystem(), largeFile, data, data.length, 16, true); - FileStatus fileStatus = largeFileFS.getFileStatus(largeFile); - largeFileSize = fileStatus.getLen(); - numBlocks = calculateNumBlocks(largeFileSize, BLOCK_SIZE); - } - - private static int calculateNumBlocks(long largeFileSize, int blockSize) { - if (largeFileSize == 0) { - return 0; - } else { - return ((int) (largeFileSize / blockSize)) + (largeFileSize % blockSize > 0 ? 1 : 0); - } - } - - @Test - public void testReadLargeFileFully() throws Throwable { - describe("read a large file fully, uses S3ACachingInputStream"); - skipIfClientSideEncryption(); - IOStatistics ioStats; - createLargeFile(); - - try (FSDataInputStream in = getFileSystem().open(methodPath())) { - ioStats = in.getIOStatistics(); - - byte[] buffer = new byte[S_1M * 10]; - long bytesRead = 0; - - while (bytesRead < largeFileSize) { - in.readFully(buffer, 0, (int) Math.min(buffer.length, largeFileSize - bytesRead)); - bytesRead += buffer.length; - // Blocks are fully read, no blocks should be cached - verifyStatisticGaugeValue(ioStats, STREAM_READ_BLOCKS_IN_FILE_CACHE, - 0); - } - - // Assert that first block is read synchronously, following blocks are prefetched - verifyStatisticCounterValue(ioStats, STREAM_READ_PREFETCH_OPERATIONS, - numBlocks - 1); - verifyStatisticCounterValue(ioStats, ACTION_HTTP_GET_REQUEST, numBlocks); - verifyStatisticCounterValue(ioStats, STREAM_READ_OPENED, numBlocks); - } - // Verify that once stream is closed, all memory is freed - verifyStatisticGaugeValue(ioStats, STREAM_READ_ACTIVE_MEMORY_IN_USE, 0); - assertThatStatisticMaximum(ioStats, - ACTION_EXECUTOR_ACQUIRED + SUFFIX_MAX).isGreaterThan(0); - } - - @Test - public void testReadLargeFileFullyLazySeek() throws Throwable { - describe("read a large file using readFully(position,buffer,offset,length)," - + " uses S3ACachingInputStream"); - skipIfClientSideEncryption(); - IOStatistics ioStats; - createLargeFile(); - - try (FSDataInputStream in = getFileSystem().open(methodPath())) { - ioStats = in.getIOStatistics(); - - byte[] buffer = new byte[S_1M * 10]; - long bytesRead = 0; - - while (bytesRead < largeFileSize) { - in.readFully(bytesRead, buffer, 0, (int) Math.min(buffer.length, - largeFileSize - bytesRead)); - bytesRead += buffer.length; - // Blocks are fully read, no blocks should be cached - verifyStatisticGaugeValue(ioStats, STREAM_READ_BLOCKS_IN_FILE_CACHE, - 0); - } - - // Assert that first block is read synchronously, following blocks are prefetched - verifyStatisticCounterValue(ioStats, STREAM_READ_PREFETCH_OPERATIONS, - numBlocks - 1); - verifyStatisticCounterValue(ioStats, ACTION_HTTP_GET_REQUEST, numBlocks); - verifyStatisticCounterValue(ioStats, STREAM_READ_OPENED, numBlocks); - } - // Verify that once stream is closed, all memory is freed - verifyStatisticGaugeValue(ioStats, STREAM_READ_ACTIVE_MEMORY_IN_USE, 0); - } - - @Test - public void testRandomReadLargeFile() throws Throwable { - describe("random read on a large file, uses S3ACachingInputStream"); - skipIfClientSideEncryption(); - IOStatistics ioStats; - createLargeFile(); - - try (FSDataInputStream in = getFileSystem().open(methodPath())) { - ioStats = in.getIOStatistics(); - - byte[] buffer = new byte[BLOCK_SIZE]; - - // Don't read block 0 completely so it gets cached on read after seek - in.read(buffer, 0, BLOCK_SIZE - S_500 * 10); - - // Seek to block 2 and read all of it - in.seek(BLOCK_SIZE * 2); - in.read(buffer, 0, BLOCK_SIZE); - - // Seek to block 4 but don't read: noop. - in.seek(BLOCK_SIZE * 4); - - // Backwards seek, will use cached block 0 - in.seek(S_500 * 5); - in.read(); - - // Expected to get block 0 (partially read), 1 (prefetch), 2 (fully read), 3 (prefetch) - // Blocks 0, 1, 3 were not fully read, so remain in the file cache - LambdaTestUtils.eventually(TIMEOUT_MILLIS, INTERVAL_MILLIS, () -> { - LOG.info("IO stats: {}", ioStats); - verifyStatisticCounterValue(ioStats, ACTION_HTTP_GET_REQUEST, 4); - verifyStatisticCounterValue(ioStats, STREAM_READ_OPENED, 4); - verifyStatisticCounterValue(ioStats, STREAM_READ_PREFETCH_OPERATIONS, 2); - verifyStatisticGaugeValue(ioStats, STREAM_READ_BLOCKS_IN_FILE_CACHE, 3); - }); - } - LambdaTestUtils.eventually(TIMEOUT_MILLIS, INTERVAL_MILLIS, () -> { - LOG.info("IO stats: {}", ioStats); - verifyStatisticGaugeValue(ioStats, STREAM_READ_BLOCKS_IN_FILE_CACHE, 0); - verifyStatisticGaugeValue(ioStats, STREAM_READ_ACTIVE_MEMORY_IN_USE, 0); - }); - } - - @Test - public void testRandomReadSmallFile() throws Throwable { - describe("random read on a small file, uses S3AInMemoryInputStream"); - - byte[] data = ContractTestUtils.dataset(SMALL_FILE_SIZE, 'a', 26); - Path smallFile = path("randomReadSmallFile"); - ContractTestUtils.writeDataset(getFileSystem(), smallFile, data, data.length, 16, true); - - try (FSDataInputStream in = getFileSystem().open(smallFile)) { - IOStatistics ioStats = in.getIOStatistics(); - - byte[] buffer = new byte[SMALL_FILE_SIZE]; - - in.read(buffer, 0, S_1K * 2); - in.seek(S_1K * 7); - in.read(buffer, 0, S_1K * 2); - - verifyStatisticCounterValue(ioStats, ACTION_HTTP_GET_REQUEST, 1); - verifyStatisticCounterValue(ioStats, STREAM_READ_OPENED, 1); - verifyStatisticCounterValue(ioStats, STREAM_READ_PREFETCH_OPERATIONS, 0); - // The buffer pool is not used - verifyStatisticGaugeValue(ioStats, STREAM_READ_ACTIVE_MEMORY_IN_USE, 0); - // no prefetch ops, so no action_executor_acquired - assertThatStatisticMaximum(ioStats, - ACTION_EXECUTOR_ACQUIRED + SUFFIX_MAX).isEqualTo(-1); - } - } - - @Test - public void testStatusProbesAfterClosingStream() throws Throwable { - describe("When the underlying input stream is closed, the prefetch input stream" - + " should still support some status probes"); - - byte[] data = ContractTestUtils.dataset(SMALL_FILE_SIZE, 'a', 26); - Path smallFile = methodPath(); - ContractTestUtils.writeDataset(getFileSystem(), smallFile, data, data.length, 16, true); - - FSDataInputStream in = getFileSystem().open(smallFile); - - byte[] buffer = new byte[SMALL_FILE_SIZE]; - in.read(buffer, 0, S_1K * 2); - in.seek(S_1K * 7); - in.read(buffer, 0, S_1K * 2); - - long pos = in.getPos(); - IOStatistics ioStats = in.getIOStatistics(); - S3AInputStreamStatistics inputStreamStatistics = - ((S3APrefetchingInputStream) (in.getWrappedStream())).getS3AStreamStatistics(); - - assertNotNull("Prefetching input IO stats should not be null", ioStats); - assertNotNull("Prefetching input stream stats should not be null", inputStreamStatistics); - assertNotEquals("Position retrieved from prefetching input stream should be greater than 0", 0, - pos); - - in.close(); - - // status probes after closing the input stream - long newPos = in.getPos(); - IOStatistics newIoStats = in.getIOStatistics(); - S3AInputStreamStatistics newInputStreamStatistics = - ((S3APrefetchingInputStream) (in.getWrappedStream())).getS3AStreamStatistics(); - - assertNotNull("Prefetching input IO stats should not be null", newIoStats); - assertNotNull("Prefetching input stream stats should not be null", newInputStreamStatistics); - assertNotEquals("Position retrieved from prefetching input stream should be greater than 0", 0, - newPos); - - // compare status probes after closing of the stream with status probes done before - // closing the stream - assertEquals("Position retrieved through stream before and after closing should match", pos, - newPos); - assertEquals("IO stats retrieved through stream before and after closing should match", ioStats, - newIoStats); - assertEquals("Stream stats retrieved through stream before and after closing should match", - inputStreamStatistics, newInputStreamStatistics); - - assertFalse("seekToNewSource() not supported with prefetch", in.seekToNewSource(10)); - } - -} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AUnbuffer.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AUnbuffer.java index 3a2d1b1b09a49..92859a4f3e859 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AUnbuffer.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AUnbuffer.java @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.s3a; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.StreamCapabilities; @@ -35,6 +36,8 @@ import java.io.IOException; import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.setPrefetchOption; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.getInputStreamStatistics; import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_BYTES; import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_BYTES_READ_CLOSE; import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_TOTAL_BYTES; @@ -49,6 +52,8 @@ * {@link org.apache.hadoop.fs.contract.s3a.ITestS3AContractUnbuffer} tests, * these tests leverage the fact that isObjectStreamOpen exposes if the * underlying stream has been closed or not. + * Disables prefetching as the behaviors are so different that the stats are + * not assertable. */ public class ITestS3AUnbuffer extends AbstractS3ATestBase { @@ -56,6 +61,11 @@ public class ITestS3AUnbuffer extends AbstractS3ATestBase { private Path dest; + @Override + protected Configuration createConfiguration() { + return setPrefetchOption(super.createConfiguration(), false); + } + @Override public void setup() throws Exception { super.setup(); @@ -75,7 +85,6 @@ public void testUnbuffer() throws IOException { // Open file, read half the data, and then call unbuffer try (FSDataInputStream inputStream = getFileSystem().open(dest)) { skipIfCannotUnbuffer(inputStream); - assertTrue(inputStream.getWrappedStream() instanceof S3AInputStream); int bytesToRead = 8; readAndAssertBytesRead(inputStream, bytesToRead); assertTrue(isObjectStreamOpen(inputStream)); @@ -178,9 +187,7 @@ public void testUnbufferStreamStatistics() throws IOException { totalBytesRead.assertDiffEquals(expectedTotalBytesRead); // Validate that the input stream stats are correct when the file is closed - S3AInputStreamStatistics streamStatistics = ((S3AInputStream) inputStream - .getWrappedStream()) - .getS3AStreamStatistics(); + S3AInputStreamStatistics streamStatistics = getInputStreamStatistics(inputStream); Assertions.assertThat(streamStatistics) .describedAs("Stream statistics %s", streamStatistics) .hasFieldOrPropertyWithValue("bytesRead", diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java index 66b25a054741e..7767c4399ca39 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java @@ -94,6 +94,8 @@ import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Set; import java.util.TreeSet; @@ -107,6 +109,7 @@ import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit; import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletion; import static org.apache.hadoop.fs.s3a.impl.S3ExpressStorage.STORE_CAPABILITY_S3_EXPRESS_STORAGE; +import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; import static org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils.getExternalData; import static org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils.requireDefaultExternalDataFile; import static org.apache.hadoop.test.GenericTestUtils.buildPaths; @@ -452,6 +455,24 @@ public static Path getLandsatCSVPath(Configuration conf) { return getExternalData(conf); } + /** + * Assume scale tests are enabled. + * Uses {@link #getTestPropertyBool(Configuration, String, boolean)} + * to resolve the option. + * + * @param conf configuration examine. + * @throws AssumptionViolatedException if disabled. + */ + public static void assumeScaleTestsEnabled(Configuration conf) { + boolean enabled = getTestPropertyBool( + conf, + KEY_SCALE_TESTS_ENABLED, + DEFAULT_SCALE_TESTS_ENABLED); + assume("Scale test disabled: to enable set property " + + KEY_SCALE_TESTS_ENABLED, + enabled); + } + /** * Verify the class of an exception. If it is not as expected, rethrow it. * Comparison is on the exact class, not subclass-of inference as @@ -486,6 +507,20 @@ public static void disableFilesystemCaching(Configuration conf) { conf.setBoolean(FS_S3A_IMPL_DISABLE_CACHE, true); } + /** + * Close any filesystem which is uncached (that is, + * if it's configuration has disabled caching and then + * instantiated via {@code Filesystem.get()} + * It does not work for any FS created by a {@code new} + + * {@code initialize()} sequence in test code. + * @param fs filesystem to close; nullable. + */ + public static void closeIfUncached(FileSystem fs) { + if (fs != null && fs.getConf().getBoolean(FS_S3A_IMPL_DISABLE_CACHE, false)) { + cleanupWithLogger(LOG, fs); + } + } + /** * Disable S3Express createSession calls. * @param conf configuration to patch @@ -731,11 +766,33 @@ public static Configuration prepareTestConfiguration(final Configuration conf) { boolean prefetchEnabled = getTestPropertyBool(conf, PREFETCH_ENABLED_KEY, PREFETCH_ENABLED_DEFAULT); - conf.setBoolean(PREFETCH_ENABLED_KEY, prefetchEnabled); + setPrefetchOption(conf, prefetchEnabled); return conf; } + /** + * Unset base/bucket prefetch options and set to the supplied option instead. + * @param conf configuration + * @param prefetch prefetch option + * @return the modified configuration. + */ + public static Configuration setPrefetchOption(final Configuration conf, + final boolean prefetch) { + removeBaseAndBucketOverrides(conf, PREFETCH_ENABLED_KEY); + conf.setBoolean(PREFETCH_ENABLED_KEY, prefetch); + return conf; + } + + /** + * The prefetch parameters to expose in a parameterized + * test to turn prefetching on/off. + */ + public static final Collection PREFETCH_OPTIONS = + Collections.unmodifiableList(Arrays.asList(new Object[][] { + {true}, {false} + })); + /** * build dir. * @return the directory for the project's build, as set by maven, @@ -915,6 +972,7 @@ public static void removeBaseAndBucketOverrides(final String bucket, /** * Remove any values from the test bucket and the base values too. + * If there is no configured test bucket, only unset the base values. * @param conf config * @param options list of fs.s3a options to remove */ @@ -924,7 +982,10 @@ public static void removeBaseAndBucketOverrides( for (String option : options) { conf.unset(option); } - removeBaseAndBucketOverrides(getTestBucketName(conf), conf, options); + if (conf.get(TEST_FS_S3A_NAME) != null) { + removeBaseAndBucketOverrides(getTestBucketName(conf), conf, options); + } + } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java index 6eccdc23dd5d5..5cd7b7b1e9e5d 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java @@ -185,7 +185,8 @@ private S3AInputStream getMockedS3AInputStream( S3AReadOpContext s3AReadOpContext = fs.createReadContext( s3AFileStatus, - NoopSpan.INSTANCE); + NoopSpan.INSTANCE) + .build(); return new S3AInputStream( s3AReadOpContext, diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileSupport.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileSupport.java index cf427c10e826a..7de992a86bb48 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileSupport.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileSupport.java @@ -379,7 +379,7 @@ public void testFileLength() throws Throwable { public void testSplitEndSetsLength() throws Throwable { long bigFile = 2L ^ 34; assertOpenFile(FS_OPTION_OPENFILE_SPLIT_END, Long.toString(bigFile)) - .matches(p -> p.getSplitEnd() == bigFile, "split end") + .matches(p -> p.getSplitEnd().get() == bigFile, "split end") .matches(p -> p.getFileLength() == -1, "file length") .matches(p -> p.getStatus() == null, "status"); } @@ -406,8 +406,8 @@ public void testSplitEndAndLength() throws Throwable { new OpenFileParameters() .withMandatoryKeys(s) .withOptions(conf))) - .matches(p -> p.getSplitStart() == 0, "split start") - .matches(p -> p.getSplitEnd() == splitEnd, "split end") + .matches(p -> !p.getSplitStart().isPresent(), "split start") + .matches(p -> !p.getSplitEnd().isPresent(), "split end") .matches(p -> p.getStatus().getLen() == len, "file length"); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java index 8e61225e17ef5..0400e83fe375d 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java @@ -41,6 +41,7 @@ import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.Statistic; import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.io.IOUtils; import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_FOOTER_CACHE; import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY; @@ -152,20 +153,26 @@ public void testOpenFileWithStatusOfOtherFS() throws Throwable { new Path("gopher:///localhost/" + testFile.getName())); // no IO in open - FSDataInputStream in = verifyMetrics(() -> - fs.openFile(testFile) - .withFileStatus(st2) - .build() - .get(), - always(NO_HEAD_OR_LIST), - with(STREAM_READ_OPENED, 0)); - - // the stream gets opened during read - long readLen = verifyMetrics(() -> - readStream(in), - always(NO_HEAD_OR_LIST), - with(STREAM_READ_OPENED, 1)); - assertEquals("bytes read from file", fileLength, readLen); + FSDataInputStream in = null; + try { + in = verifyMetrics(() -> + fs.openFile(testFile) + .withFileStatus(st2) + .build() + .get(), + always(NO_HEAD_OR_LIST), + with(STREAM_READ_OPENED, 0)); + + // the stream gets opened during read + final FSDataInputStream s = in; + long readLen = verifyMetrics(() -> + readStream(s), + always(NO_HEAD_OR_LIST), + with(STREAM_READ_OPENED, 1)); + assertEquals("bytes read from file", fileLength, readLen); + } finally { + IOUtils.closeStream(in); + } } @Test diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/ITestInMemoryInputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/ITestInMemoryInputStream.java new file mode 100644 index 0000000000000..13035c8357e6d --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/ITestInMemoryInputStream.java @@ -0,0 +1,226 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.prefetch; + +import org.assertj.core.api.Assertions; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.s3a.S3ATestUtils; +import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest; +import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsContext; + +import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.setPrefetchOption; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticMaximum; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticGaugeValue; +import static org.apache.hadoop.fs.statistics.IOStatisticsContext.getCurrentIOStatisticsContext; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_EXECUTOR_ACQUIRED; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_HTTP_GET_REQUEST; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MAX; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_ACTIVE_MEMORY_IN_USE; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BLOCK_CACHE_ENABLED; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BLOCK_PREFETCH_ENABLED; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BYTES; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_FULLY_OPERATIONS; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_OPENED; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_PREFETCH_OPERATIONS; +import static org.apache.hadoop.test.Sizes.S_16K; +import static org.apache.hadoop.test.Sizes.S_1K; +import static org.apache.hadoop.test.Sizes.S_4K; + +/** + * Test the prefetching input stream, validates that the + * {@link S3AInMemoryInputStream} is working as expected. + */ +public class ITestInMemoryInputStream extends AbstractS3ACostTest { + + private static final Logger LOG = + LoggerFactory.getLogger(ITestInMemoryInputStream.class); + + // Size should be < block size so S3AInMemoryInputStream is used + private static final int SMALL_FILE_SIZE = S_16K; + + /** + * Thread level IOStatistics; reset in setup(). + */ + private IOStatisticsContext ioStatisticsContext; + + @Override + public Configuration createConfiguration() { + Configuration conf = super.createConfiguration(); + S3ATestUtils.removeBaseAndBucketOverrides(conf, PREFETCH_ENABLED_KEY); + setPrefetchOption(conf, true); + return conf; + } + + @Override + public void setup() throws Exception { + super.setup(); + ioStatisticsContext = getCurrentIOStatisticsContext(); + ioStatisticsContext.reset(); + } + + private void printStreamStatistics(final FSDataInputStream in) { + LOG.info("Stream statistics\n{}", + ioStatisticsToPrettyString(in.getIOStatistics())); + } + + @Test + public void testRandomReadSmallFile() throws Throwable { + describe("random read on a small file, uses S3AInMemoryInputStream"); + + byte[] data = ContractTestUtils.dataset(SMALL_FILE_SIZE, 'a', 26); + Path smallFile = path("randomReadSmallFile"); + ContractTestUtils.writeDataset(getFileSystem(), smallFile, data, data.length, 16, true); + + int expectedReadBytes = 0; + try (FSDataInputStream in = getFileSystem().open(smallFile)) { + IOStatistics ioStats = in.getIOStatistics(); + + byte[] buffer = new byte[SMALL_FILE_SIZE]; + + in.read(buffer, 0, S_4K); + expectedReadBytes += S_4K; + verifyStatisticCounterValue(ioStats, STREAM_READ_BYTES, + expectedReadBytes); + + in.seek(S_1K * 12); + in.read(buffer, 0, S_4K); + expectedReadBytes += S_4K; + + verifyStatisticCounterValue(ioStats, STREAM_READ_BYTES, + expectedReadBytes); + printStreamStatistics(in); + + verifyStatisticCounterValue(ioStats, ACTION_HTTP_GET_REQUEST, 1); + verifyStatisticCounterValue(ioStats, STREAM_READ_OPENED, 1); + verifyStatisticCounterValue(ioStats, STREAM_READ_PREFETCH_OPERATIONS, 0); + // the whole file is loaded + verifyStatisticGaugeValue(ioStats, STREAM_READ_ACTIVE_MEMORY_IN_USE, SMALL_FILE_SIZE); + // there is no prefetching + verifyStatisticGaugeValue(ioStats, STREAM_READ_BLOCK_PREFETCH_ENABLED, 0); + // there is no caching + verifyStatisticGaugeValue(ioStats, STREAM_READ_BLOCK_CACHE_ENABLED, 0); + // no prefetch ops, so no action_executor_acquired + assertThatStatisticMaximum(ioStats, + ACTION_EXECUTOR_ACQUIRED + SUFFIX_MAX).isEqualTo(-1); + + // now read offset 0 again and again, expect no new costs + in.readFully(0, buffer); + verifyStatisticCounterValue(ioStats, ACTION_HTTP_GET_REQUEST, 1); + expectedReadBytes += buffer.length; + + verifyStatisticCounterValue(ioStats, STREAM_READ_BYTES, + expectedReadBytes); + // unbuffer + in.unbuffer(); + LOG.info("unbuffered {}", in); + verifyStatisticGaugeValue(ioStats, STREAM_READ_ACTIVE_MEMORY_IN_USE, 0); + + printStreamStatistics(in); + + in.readFully(0, buffer); + verifyStatisticCounterValue(ioStats, STREAM_READ_FULLY_OPERATIONS, 2); + verifyStatisticCounterValue(ioStats, ACTION_HTTP_GET_REQUEST, 2); + expectedReadBytes += buffer.length; + + verifyStatisticCounterValue(ioStats, STREAM_READ_BYTES, + expectedReadBytes); + verifyStatisticGaugeValue(ioStats, STREAM_READ_ACTIVE_MEMORY_IN_USE, SMALL_FILE_SIZE); + + } + + // thread IOStats are updated in close + final IOStatistics threadIOStats = ioStatisticsContext.getIOStatistics(); + verifyStatisticCounterValue(threadIOStats, + ACTION_HTTP_GET_REQUEST, 2); + verifyStatisticGaugeValue(threadIOStats, STREAM_READ_ACTIVE_MEMORY_IN_USE, 0); + verifyStatisticCounterValue(threadIOStats, STREAM_READ_BYTES, expectedReadBytes); + } + + @Test + public void testStatusProbesAfterClosingStream() throws Throwable { + describe("When the underlying input stream is closed, the prefetch input stream" + + " should still support some status probes"); + + byte[] data = ContractTestUtils.dataset(SMALL_FILE_SIZE, 'a', 26); + Path smallFile = methodPath(); + ContractTestUtils.writeDataset(getFileSystem(), smallFile, data, data.length, 16, true); + + + try (FSDataInputStream in = getFileSystem().open(smallFile)) { + byte[] buffer = new byte[SMALL_FILE_SIZE]; + in.read(buffer, 0, S_4K); + in.seek(S_1K * 12); + in.read(buffer, 0, S_4K); + + long pos = in.getPos(); + IOStatistics ioStats = in.getIOStatistics(); + S3AInputStreamStatistics inputStreamStatistics = + ((S3APrefetchingInputStream) (in.getWrappedStream())).getS3AStreamStatistics(); + + assertNotNull("Prefetching input IO stats should not be null", ioStats); + assertNotNull("Prefetching input stream stats should not be null", inputStreamStatistics); + assertNotEquals("Position retrieved from prefetching input stream should be greater than 0", + 0, pos); + + verifyStatisticCounterValue(ioStats, ACTION_HTTP_GET_REQUEST, 1); + + + // close the stream and still use it. + in.close(); + + // status probes after closing the input stream + long newPos = in.getPos(); + IOStatistics newIoStats = in.getIOStatistics(); + S3AInputStreamStatistics newInputStreamStatistics = + ((S3APrefetchingInputStream) (in.getWrappedStream())).getS3AStreamStatistics(); + + assertNotNull("Prefetching input IO stats should not be null", newIoStats); + assertNotNull("Prefetching input stream stats should not be null", newInputStreamStatistics); + assertNotEquals("Position retrieved from prefetching input stream should be greater than 0", + 0, newPos); + + // compare status probes after closing of the stream with status probes done before + // closing the stream + assertEquals("Position retrieved through stream before and after closing should match", pos, + newPos); + assertEquals("IO stats retrieved through stream before and after closing should match", + ioStats, newIoStats); + assertEquals("Stream stats retrieved through stream before and after closing should match", + inputStreamStatistics, newInputStreamStatistics); + + Assertions.assertThat(in.seekToNewSource(10)) + .describedAs("seekToNewSource() not supported with prefetch: %s", in) + .isFalse(); + } + + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/ITestS3APrefetchingLargeFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/ITestS3APrefetchingLargeFiles.java new file mode 100644 index 0000000000000..2e9f2ad979e8b --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/ITestS3APrefetchingLargeFiles.java @@ -0,0 +1,408 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.prefetch; + +import java.io.File; +import java.io.IOException; +import java.util.concurrent.atomic.AtomicLong; + +import org.assertj.core.api.Assertions; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.permission.FsAction; +import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsContext; +import org.apache.hadoop.test.LambdaTestUtils; + +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY; +import static org.apache.hadoop.fs.s3a.Constants.BUFFER_DIR; +import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_COUNT_KEY; +import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_SIZE_KEY; +import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticGauge; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticGaugeValue; +import static org.apache.hadoop.fs.statistics.IOStatisticsContext.getCurrentIOStatisticsContext; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_HTTP_GET_REQUEST; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_ACTIVE_MEMORY_IN_USE; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BLOCKS_IN_FILE_CACHE; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BLOCK_CACHE_ENABLED; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BLOCK_PREFETCH_ENABLED; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_OPENED; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_PREFETCH_OPERATIONS; +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_REMOTE_BLOCK_READ; +import static org.apache.hadoop.test.Sizes.S_1K; + +/** + * Test the prefetching input stream, validates that the underlying S3ACachingInputStream and + * S3AInMemoryInputStream are working as expected. + */ +public class ITestS3APrefetchingLargeFiles extends AbstractS3ACostTest { + + private static final Logger LOG = + LoggerFactory.getLogger(ITestS3APrefetchingLargeFiles.class); + + private static final int S_500 = 512; + + private int numBlocks; + + // Size should be > block size so S3ACachingInputStream is used + private long largeFileSize; + + private static final int TIMEOUT_MILLIS = 5000; + private static final int INTERVAL_MILLIS = 500; + + /** + * Size of blocks for this test {@value}. + */ + private static final int BLOCK_SIZE = S_1K * 10; + + /** + * Size of the large file {@value}. + */ + public static final int LARGE_FILE_SIZE = S_1K * 72; + + /** + * large file dataset. + */ + private static final byte[] DATASET = ContractTestUtils.dataset(LARGE_FILE_SIZE, 'a', 26); + + + @Rule + public TemporaryFolder tempFolder = new TemporaryFolder(); + private File tmpFileDir; + + /** + * Thread level IOStatistics; reset in setup(). + */ + private IOStatisticsContext ioStatisticsContext; + + @Override + public Configuration createConfiguration() { + Configuration conf = super.createConfiguration(); + removeBaseAndBucketOverrides(conf, + BUFFER_DIR, + PREFETCH_ENABLED_KEY, + PREFETCH_BLOCK_COUNT_KEY, + PREFETCH_BLOCK_SIZE_KEY); + conf.setBoolean(PREFETCH_ENABLED_KEY, true); + conf.setInt(PREFETCH_BLOCK_SIZE_KEY, BLOCK_SIZE); + conf.setInt(PREFETCH_BLOCK_COUNT_KEY, 2); + conf.set(BUFFER_DIR, tmpFileDir.getAbsolutePath()); + return conf; + } + + @Override + public void setup() throws Exception { + tmpFileDir = tempFolder.newFolder("cache"); + super.setup(); + + ioStatisticsContext = getCurrentIOStatisticsContext(); + ioStatisticsContext.reset(); + nameThread(); + skipIfClientSideEncryption(); + } + + /** + * Create a large file from the method path. + * + * @return the path. + */ + private Path createLargeFile() throws Exception { + Path largeFile = methodPath(); + FileSystem largeFileFS = getFileSystem(); + ContractTestUtils.writeDataset(getFileSystem(), largeFile, DATASET, DATASET.length, 16, true); + FileStatus fileStatus = largeFileFS.getFileStatus(largeFile); + largeFileSize = fileStatus.getLen(); + numBlocks = calculateNumBlocks(largeFileSize, BLOCK_SIZE); + return largeFile; + } + + private static int calculateNumBlocks(long largeFileSize, int blockSize) { + if (largeFileSize == 0) { + return 0; + } else { + return ((int) (largeFileSize / blockSize)) + (largeFileSize % blockSize > 0 ? 1 : 0); + } + } + + @Test + public void testReadLargeFileFully() throws Throwable { + describe("read a large file fully, uses S3ACachingInputStream"); + IOStatistics ioStats; + final Path path = createLargeFile(); + + try (FSDataInputStream in = getFileSystem().open(path)) { + ioStats = in.getIOStatistics(); + assertPrefetchingAndCachingEnabled(ioStats); + + byte[] buffer = new byte[S_1K]; + long bytesRead = 0; + + while (bytesRead < largeFileSize) { + in.readFully(buffer, 0, (int) Math.min(buffer.length, largeFileSize - bytesRead)); + bytesRead += buffer.length; + // Blocks are fully read, no blocks should be cached + verifyStatisticGaugeValue(ioStats, STREAM_READ_BLOCKS_IN_FILE_CACHE, + 0); + } + printStreamStatistics(in); + + // Assert that first block is read synchronously, following blocks are prefetched + verifyStatisticCounterValue(ioStats, STREAM_READ_PREFETCH_OPERATIONS, + numBlocks - 1); + verifyStatisticCounterValue(ioStats, ACTION_HTTP_GET_REQUEST, numBlocks); + verifyStatisticCounterValue(ioStats, STREAM_READ_OPENED, numBlocks); + + in.unbuffer(); + // Verify that once stream is closed, all memory is freed + verifyStatisticGaugeValue(ioStats, STREAM_READ_ACTIVE_MEMORY_IN_USE, 0); + assertPrefetchingAndCachingEnabled(ioStats); + } + // Verify that once stream is closed, all memory is freed + verifyStatisticGaugeValue(ioStats, STREAM_READ_ACTIVE_MEMORY_IN_USE, 0); + } + + /** + * Assert that prefetching and caching enabled -determined by examining gauges. + * @param ioStats iostatistics + */ + private static void assertPrefetchingAndCachingEnabled(final IOStatistics ioStats) { + // prefetching is on + verifyStatisticGaugeValue(ioStats, STREAM_READ_BLOCK_PREFETCH_ENABLED, 1); + // there is caching + verifyStatisticGaugeValue(ioStats, STREAM_READ_BLOCK_CACHE_ENABLED, 1); + } + + private void printStreamStatistics(final FSDataInputStream in) { + LOG.info("Stream statistics\n{}", + ioStatisticsToPrettyString(in.getIOStatistics())); + } + + @Test + public void testReadLargeFileFullyLazySeek() throws Throwable { + describe("read a large file using readFully(position,buffer,offset,length)," + + " uses S3ACachingInputStream"); + IOStatistics ioStats; + final Path path = createLargeFile(); + + try (FSDataInputStream in = getFileSystem().open(path)) { + ioStats = in.getIOStatistics(); + + byte[] buffer = new byte[S_1K]; + long bytesRead = 0; + + while (bytesRead < largeFileSize) { + in.readFully(bytesRead, buffer, 0, (int) Math.min(buffer.length, + largeFileSize - bytesRead)); + bytesRead += buffer.length; + // Blocks are fully read, no blocks should be cached + verifyStatisticGaugeValue(ioStats, STREAM_READ_BLOCKS_IN_FILE_CACHE, + 0); + } + printStreamStatistics(in); + + // Assert that first block is read synchronously, following blocks are prefetched + verifyStatisticCounterValue(ioStats, STREAM_READ_PREFETCH_OPERATIONS, + numBlocks - 1); + verifyStatisticCounterValue(ioStats, ACTION_HTTP_GET_REQUEST, numBlocks); + verifyStatisticCounterValue(ioStats, STREAM_READ_OPENED, numBlocks); + } + // Verify that once stream is closed, all memory is freed + verifyStatisticGaugeValue(ioStats, STREAM_READ_ACTIVE_MEMORY_IN_USE, 0); + + // thread IOStats are updated in close + final IOStatistics threadIOStats = ioStatisticsContext.getIOStatistics(); + verifyStatisticCounterValue(threadIOStats, + STREAM_READ_REMOTE_BLOCK_READ, numBlocks); + verifyStatisticGaugeValue(threadIOStats, STREAM_READ_ACTIVE_MEMORY_IN_USE, 0); + } + + /** + * Random read of a large file, with a small block size + * so as to reduce wait time for read completion. + * @throws Throwable + */ + @Test + public void testRandomReadLargeFile() throws Throwable { + describe("random read on a large file, uses S3ACachingInputStream"); + IOStatistics ioStats; + final Path path = createLargeFile(); + + describe("Commencing Read Sequence"); + try (FSDataInputStream in = getFileSystem() + .openFile(path) + .opt(FS_OPTION_OPENFILE_READ_POLICY, "random") + .build().get()) { + ioStats = in.getIOStatistics(); + + byte[] buffer = new byte[BLOCK_SIZE]; + + // there is no prefetch of block 0 + awaitCachedBlocks(ioStats, 0); + + // Don't read block 0 completely so it gets cached on read after seek + in.read(buffer, 0, BLOCK_SIZE - S_1K); + awaitCachedBlocks(ioStats, 1); + + // Seek to block 2 and read all of it + in.seek(BLOCK_SIZE * 2); + in.read(buffer, 0, BLOCK_SIZE); + + // so no increment in cache block count + awaitCachedBlocks(ioStats, 1); + + // Seek to block 4 but don't read: noop. + in.seek(BLOCK_SIZE * 4); + awaitCachedBlocks(ioStats, 1); + + // Backwards seek, will use cached block 0 + in.seek(S_500 * 5); + in.read(); + + AtomicLong iterations = new AtomicLong(); + + /* + these tests are really brittle to prefetching, timeouts and threading. + */ + // Expected to get block 0 (partially read), 1 (prefetch), 2 (fully read), 3 (prefetch) + // Blocks 0, 1, 3 were not fully read, so remain in the file cache + LambdaTestUtils.eventually(TIMEOUT_MILLIS, INTERVAL_MILLIS, () -> { + LOG.info("Attempt {}: {}", iterations.incrementAndGet(), + ioStatisticsToPrettyString(ioStats)); + verifyStatisticCounterValue(ioStats, ACTION_HTTP_GET_REQUEST, 4); + verifyStatisticCounterValue(ioStats, STREAM_READ_OPENED, 4); + }); + printStreamStatistics(in); + } + awaitCachedBlocks(ioStats, 0); + LambdaTestUtils.eventually(TIMEOUT_MILLIS, INTERVAL_MILLIS, () -> { + verifyStatisticGaugeValue(ioStats, STREAM_READ_ACTIVE_MEMORY_IN_USE, 0); + }); + + } + + /** + * Await the cached block count to match the expected value. + * @param ioStats live statistics + * @param count count of blocks + * @throws Exception failure + */ + public void awaitCachedBlocks(final IOStatistics ioStats, final long count) throws Exception { + LambdaTestUtils.eventually(TIMEOUT_MILLIS, INTERVAL_MILLIS, () -> { + verifyStatisticGaugeValue(ioStats, STREAM_READ_BLOCKS_IN_FILE_CACHE, count); + verifyStatisticGaugeValue(ioStats, STREAM_READ_ACTIVE_MEMORY_IN_USE, 0); + }); + } + /** + * Test to verify the existence of the cache file. + * Tries to perform inputStream read and seek ops to make the prefetching take place and + * asserts whether file with .bin suffix is present. It also verifies certain file stats. + */ + @Test + public void testCacheFileExistence() throws Throwable { + describe("Verify that FS cache files exist on local FS"); + + int prefetchBlockSize = BLOCK_SIZE; + final Path path = createLargeFile(); + LOG.info("Opening file {}", path); + + try (FSDataInputStream in = getFileSystem().open(path)) { + byte[] buffer = new byte[prefetchBlockSize]; + // caching is enabled + final IOStatistics ioStats = in.getIOStatistics(); + assertThatStatisticGauge(ioStats, STREAM_READ_BLOCK_CACHE_ENABLED) + .isEqualTo(1); + + // read less than the block size + LOG.info("reading first data block"); + in.readFully(buffer, 0, prefetchBlockSize - S_1K); + LOG.info("stream stats after a read {}", ioStatisticsToPrettyString(ioStats)); + + // there's been at least one fetch so far; others may be in progress. + assertThatStatisticCounter(ioStats, ACTION_HTTP_GET_REQUEST) + .isGreaterThanOrEqualTo(1); + + + //assertCacheFileExists(); + + LOG.info("reading rest of file"); + + in.readFully(prefetchBlockSize * 2, buffer, 0, prefetchBlockSize); + + LOG.info("stream stats after reading whole file {}", ioStatisticsToPrettyString(ioStats)); + + // and at least one block is in cache + assertThatStatisticGauge(ioStats, STREAM_READ_BLOCKS_IN_FILE_CACHE) + .isGreaterThanOrEqualTo(1); + + // a file which must be under the tempt dir + assertCacheFileExists(); + } + } + + /** + * Scan the temp dir and asserdt that there are cache files. + */ + private void assertCacheFileExists() throws IOException { + final Configuration conf = getFileSystem().getConf(); + Assertions.assertThat(tmpFileDir.isDirectory()) + .describedAs("The dir to keep cache files must exist %s", tmpFileDir); + File[] tmpFiles = tmpFileDir.listFiles(); + Assertions.assertThat(tmpFiles) + .describedAs("No cache files found under " + tmpFileDir) + .isNotNull() + .hasSizeGreaterThanOrEqualTo(1); + + for (File tmpFile : tmpFiles) { + Path path = new Path(tmpFile.getAbsolutePath()); + try (FileSystem localFs = FileSystem.getLocal(conf)) { + FileStatus stat = localFs.getFileStatus(path); + ContractTestUtils.assertIsFile(path, stat); + Assertions.assertThat(stat.getLen()) + .describedAs("%s: File length not matching with prefetchBlockSize", path) + .isEqualTo(BLOCK_SIZE); + + assertEquals("User permissions should be RW", FsAction.READ_WRITE, + stat.getPermission().getUserAction()); + assertEquals("Group permissions should be NONE", FsAction.NONE, + stat.getPermission().getGroupAction()); + assertEquals("Other permissions should be NONE", FsAction.NONE, + stat.getPermission().getOtherAction()); + } + } + } + + + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingLruEviction.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/ITestS3APrefetchingLruEviction.java similarity index 98% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingLruEviction.java rename to hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/ITestS3APrefetchingLruEviction.java index 0fa08f37cf909..63ab43380f834 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingLruEviction.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/ITestS3APrefetchingLruEviction.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.fs.s3a; +package org.apache.hadoop.fs.s3a.prefetch; import java.io.IOException; import java.io.UncheckedIOException; @@ -38,6 +38,7 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest; import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.test.LambdaTestUtils; @@ -51,6 +52,7 @@ import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_EVICT_BLOCKS_FROM_FILE_CACHE; import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_FILE_CACHE_EVICTION; import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BLOCKS_IN_FILE_CACHE; +import static org.apache.hadoop.test.Sizes.S_1K; /** * Test the prefetching input stream with LRU cache eviction on S3ACachingInputStream. @@ -76,7 +78,6 @@ public ITestS3APrefetchingLruEviction(final String maxBlocks) { private static final Logger LOG = LoggerFactory.getLogger(ITestS3APrefetchingLruEviction.class); - private static final int S_1K = 1024; private static final int S_500 = 512; private static final int SMALL_FILE_SIZE = S_1K * 56; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/MockS3ARemoteObject.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/MockS3ARemoteObject.java index 5fbbc3a127997..a4aa23561c67f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/MockS3ARemoteObject.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/MockS3ARemoteObject.java @@ -27,6 +27,8 @@ import software.amazon.awssdk.http.AbortableInputStream; import software.amazon.awssdk.services.s3.model.GetObjectRequest; import software.amazon.awssdk.services.s3.model.GetObjectResponse; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.impl.prefetch.Validate; import org.apache.hadoop.fs.s3a.S3AInputStream; @@ -36,7 +38,9 @@ /** * A mock s3 file with some fault injection. */ -class MockS3ARemoteObject extends S3ARemoteObject { +public class MockS3ARemoteObject extends S3ARemoteObject { + + private static final Logger LOG = LoggerFactory.getLogger(MockS3ARemoteObject.class); private byte[] contents; @@ -54,7 +58,7 @@ class MockS3ARemoteObject extends S3ARemoteObject { MockS3ARemoteObject(int size, boolean throwExceptionOnOpen) { super( - S3APrefetchFakes.createReadContext(null, KEY, size, 1, 1), + S3APrefetchFakes.createReadContext(null, "s3a://" + BUCKET + "/" + KEY, size, 1, 1), S3APrefetchFakes.createObjectAttributes(BUCKET, KEY, size), S3APrefetchFakes.createInputStreamCallbacks(BUCKET), EmptyS3AStatisticsContext.EMPTY_INPUT_STREAM_STATISTICS, @@ -75,6 +79,7 @@ public ResponseInputStream openForRead(long offset, int size) Validate.checkLessOrEqual(size, "size", size() - offset, "size() - offset"); if (throwExceptionOnOpen) { + LOG.debug("Raising IOException in open({}, {})", offset, size); throwExceptionOnOpen = false; throw new IOException("Throwing because throwExceptionOnOpen is true "); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchFakes.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchFakes.java index 1c509702188f3..b4eed922b0545 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchFakes.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchFakes.java @@ -340,7 +340,9 @@ protected void writeFile(Path path, ByteBuffer buffer) throws IOException { @Override protected Path getCacheFilePath(final Configuration conf, - final LocalDirAllocator localDirAllocator) + final LocalDirAllocator localDirAllocator, + final String blockInfo, + final long fileSize) throws IOException { fileCount++; return Paths.get(Long.toString(fileCount)); @@ -350,6 +352,13 @@ protected Path getCacheFilePath(final Configuration conf, public void close() throws IOException { this.files.clear(); } + + @Override + public String toString() { + return "FakeS3FilePerBlockCache{" + + "fileCount=" + fileCount + + "} " + super.toString(); + } } private static final Random RANDOM = new Random(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ABlockManager.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ABlockManager.java index c1b59d6f2e130..b17416690a28a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ABlockManager.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ABlockManager.java @@ -80,7 +80,7 @@ public void testGet() throws IOException { ByteBuffer buffer = data.getBuffer(); long startOffset = blockData.getStartOffset(b); for (int i = 0; i < BLOCK_SIZE; i++) { - assertEquals(startOffset + i, buffer.get()); + assertEquals("buffer value at offset " + i, startOffset + i, buffer.get()); } } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ACachingBlockManager.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ACachingBlockManager.java index 87e2b68f1e41b..ba8b780f290ec 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ACachingBlockManager.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ACachingBlockManager.java @@ -24,12 +24,19 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import org.assertj.core.api.Assertions; +import org.junit.After; +import org.junit.Before; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.LocalDirAllocator; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.impl.prefetch.BlockData; import org.apache.hadoop.fs.impl.prefetch.BlockManagerParameters; +import org.apache.hadoop.fs.impl.prefetch.BlockManager; import org.apache.hadoop.fs.impl.prefetch.BufferData; import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool; import org.apache.hadoop.fs.s3a.S3ATestUtils; @@ -49,22 +56,42 @@ */ public class TestS3ACachingBlockManager extends AbstractHadoopTestBase { + private static final Logger LOG = LoggerFactory.getLogger(TestS3ACachingBlockManager.class); + static final int FILE_SIZE = 15; static final int BLOCK_SIZE = 2; static final int POOL_SIZE = 3; - private final ExecutorService threadPool = Executors.newFixedThreadPool(4); + private ExecutorService threadPool; - private final ExecutorServiceFuturePool futurePool = - new ExecutorServiceFuturePool(threadPool); + private ExecutorServiceFuturePool futurePool; + + /** + * only used for logging. + */ + private Path testPath; private final S3AInputStreamStatistics streamStatistics = new EmptyS3AStatisticsContext().newInputStreamStatistics(); private final BlockData blockData = new BlockData(FILE_SIZE, BLOCK_SIZE); + @Before + public void setup() { + threadPool = Executors.newFixedThreadPool(4); + futurePool = new ExecutorServiceFuturePool(threadPool); + testPath = new Path("/"); + } + + @After + public void teardown() { + if (threadPool != null) { + threadPool.shutdown(); + } + } + private static final Configuration CONF = S3ATestUtils.prepareTestConfiguration(new Configuration()); @@ -72,13 +99,14 @@ public class TestS3ACachingBlockManager extends AbstractHadoopTestBase { public void testFuturePoolNull() throws Exception { MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, false); Configuration conf = new Configuration(); + BlockManagerParameters blockManagerParams = + new BlockManagerParameters() + .withBlockData(blockData) + .withBufferPoolSize(POOL_SIZE) + .withConf(conf) + .withPath(testPath) + .withPrefetchingStatistics(streamStatistics); try (S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File)) { - BlockManagerParameters blockManagerParams = - new BlockManagerParameters() - .withBlockData(blockData) - .withBufferPoolSize(POOL_SIZE) - .withPrefetchingStatistics(streamStatistics) - .withConf(conf); intercept(NullPointerException.class, () -> new S3ACachingBlockManager(blockManagerParams, reader)); @@ -90,13 +118,15 @@ public void testNullReader() throws Exception { Configuration conf = new Configuration(); BlockManagerParameters blockManagerParams = new BlockManagerParameters() - .withFuturePool(futurePool) .withBlockData(blockData) .withBufferPoolSize(POOL_SIZE) - .withPrefetchingStatistics(streamStatistics) .withConf(conf) + .withFuturePool(futurePool) .withMaxBlocksCount( - conf.getInt(PREFETCH_MAX_BLOCKS_COUNT, DEFAULT_PREFETCH_MAX_BLOCKS_COUNT)); + conf.getInt(PREFETCH_MAX_BLOCKS_COUNT, DEFAULT_PREFETCH_MAX_BLOCKS_COUNT)) + .withPath(testPath) + .withPrefetchingStatistics(streamStatistics); + intercept(IllegalArgumentException.class, "'reader' must not be null", () -> new S3ACachingBlockManager(blockManagerParams, null)); @@ -174,13 +204,15 @@ public void testArgChecks() throws Exception { Configuration conf = new Configuration(); BlockManagerParameters blockManagerParams = new BlockManagerParameters() - .withFuturePool(futurePool) .withBlockData(blockData) .withBufferPoolSize(POOL_SIZE) - .withPrefetchingStatistics(streamStatistics) .withConf(conf) + .withFuturePool(futurePool) .withMaxBlocksCount( - conf.getInt(PREFETCH_MAX_BLOCKS_COUNT, DEFAULT_PREFETCH_MAX_BLOCKS_COUNT)); + conf.getInt(PREFETCH_MAX_BLOCKS_COUNT, DEFAULT_PREFETCH_MAX_BLOCKS_COUNT)) + .withPath(testPath) + .withPrefetchingStatistics(streamStatistics); + // Should not throw. S3ACachingBlockManager blockManager = @@ -241,6 +273,7 @@ public int read(ByteBuffer buffer, long offset, int size) protected void cachePut(int blockNumber, ByteBuffer buffer) throws IOException { if (forceNextCachePutToFail) { + LOG.debug("Forcing put {} to fail", blockNumber); forceNextCachePutToFail = false; throw new RuntimeException("bar"); } else { @@ -336,24 +369,28 @@ private void testPrefetchHelper(boolean forcePrefetchFailure) blockManager.requestPrefetch(b); } - assertEquals(0, blockManager.numCached()); + assertNumCached(blockManager, 0); - blockManager.cancelPrefetches(); + // random IO prefetches will still complete; close/unbuffer will not + // add the result to the cache. + blockManager.cancelPrefetches(BlockManager.CancelReason.RandomIO); waitForCaching(blockManager, expectedNumSuccesses); - assertEquals(expectedNumErrors, this.totalErrors(blockManager)); - assertEquals(expectedNumSuccesses, blockManager.numCached()); + assertErrorCount(blockManager, expectedNumErrors); + assertNumCached(blockManager, expectedNumSuccesses); + } private BlockManagerParameters getBlockManagerParameters() { return new BlockManagerParameters() - .withFuturePool(futurePool) .withBlockData(blockData) .withBufferPoolSize(POOL_SIZE) - .withPrefetchingStatistics(streamStatistics) - .withLocalDirAllocator(new LocalDirAllocator(HADOOP_TMP_DIR)) .withConf(CONF) + .withFuturePool(futurePool) + .withLocalDirAllocator(new LocalDirAllocator(HADOOP_TMP_DIR)) .withMaxBlocksCount( - CONF.getInt(PREFETCH_MAX_BLOCKS_COUNT, DEFAULT_PREFETCH_MAX_BLOCKS_COUNT)); + CONF.getInt(PREFETCH_MAX_BLOCKS_COUNT, DEFAULT_PREFETCH_MAX_BLOCKS_COUNT)) + .withPath(testPath) + .withPrefetchingStatistics(streamStatistics); } // @Ignore @@ -365,15 +402,17 @@ public void testCachingOfPrefetched() Configuration conf = new Configuration(); BlockManagerParameters blockManagerParamsBuilder = new BlockManagerParameters() - .withFuturePool(futurePool) .withBlockData(blockData) .withBufferPoolSize(POOL_SIZE) - .withPrefetchingStatistics(streamStatistics) + .withConf(conf) + .withFuturePool(futurePool) .withLocalDirAllocator( new LocalDirAllocator(conf.get(BUFFER_DIR) != null ? BUFFER_DIR : HADOOP_TMP_DIR)) - .withConf(conf) .withMaxBlocksCount( - conf.getInt(PREFETCH_MAX_BLOCKS_COUNT, DEFAULT_PREFETCH_MAX_BLOCKS_COUNT)); + conf.getInt(PREFETCH_MAX_BLOCKS_COUNT, DEFAULT_PREFETCH_MAX_BLOCKS_COUNT)) + .withPath(testPath) + .withPrefetchingStatistics(streamStatistics); + S3ACachingBlockManager blockManager = new S3ACachingBlockManager(blockManagerParamsBuilder, reader); assertInitialState(blockManager); @@ -384,12 +423,18 @@ public void testCachingOfPrefetched() blockManager.requestCaching(data); } - waitForCaching(blockManager, Math.min(blockData.getNumBlocks(), - conf.getInt(PREFETCH_MAX_BLOCKS_COUNT, DEFAULT_PREFETCH_MAX_BLOCKS_COUNT))); - assertEquals(Math.min(blockData.getNumBlocks(), - conf.getInt(PREFETCH_MAX_BLOCKS_COUNT, DEFAULT_PREFETCH_MAX_BLOCKS_COUNT)), - blockManager.numCached()); - assertEquals(0, this.totalErrors(blockManager)); + final int expected = Math.min(blockData.getNumBlocks(), + conf.getInt(PREFETCH_MAX_BLOCKS_COUNT, DEFAULT_PREFETCH_MAX_BLOCKS_COUNT)); + waitForCaching(blockManager, expected); + assertNumCached(blockManager, expected); + assertErrorCount(blockManager, 0); + } + + private static void assertNumCached(final S3ACachingBlockManager blockManager, + final int expected) { + Assertions.assertThat(blockManager.numCached()) + .describedAs("blockManager.numCached() in %s", blockManager) + .isEqualTo(expected); } // @Ignore @@ -439,13 +484,19 @@ public void testCachingOfGetHelper(boolean forceCachingFailure) blockManager.numCached()); if (forceCachingFailure) { - assertEquals(expectedNumErrors, this.totalErrors(blockManager)); + assertErrorCount(blockManager, expectedNumErrors); } else { - assertEquals(0, this.totalErrors(blockManager)); + assertErrorCount(blockManager, 0); } } } + private void assertErrorCount(final S3ACachingBlockManager blockManager, final int expected) { + Assertions.assertThat(totalErrors(blockManager)) + .describedAs("Total errors of %s", blockManager) + .isEqualTo(expected); + } + private void waitForCaching( S3ACachingBlockManager blockManager, int expectedCount) @@ -459,9 +510,10 @@ private void waitForCaching( numTrys++; if (numTrys > 600) { String message = String.format( - "waitForCaching: expected: %d, actual: %d, read errors: %d, caching errors: %d", + "waitForCaching: expected: %d, actual: %d, read errors: %d, caching errors: %d in %s", expectedCount, count, blockManager.numReadErrors(), - blockManager.numCachingErrors()); + blockManager.numCachingErrors(), + blockManager); throw new IllegalStateException(message); } } @@ -473,6 +525,6 @@ private int totalErrors(S3ACachingBlockManager blockManager) { } private void assertInitialState(S3ACachingBlockManager blockManager) { - assertEquals(0, blockManager.numCached()); + assertNumCached(blockManager, 0); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteInputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteInputStream.java index 8ce26033c1182..24d1ffc0fc845 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteInputStream.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteInputStream.java @@ -33,11 +33,11 @@ import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool; import org.apache.hadoop.fs.s3a.S3AInputStream; import org.apache.hadoop.fs.s3a.S3AReadOpContext; -import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.S3ObjectAttributes; import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; import org.apache.hadoop.test.AbstractHadoopTestBase; +import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; @@ -65,7 +65,8 @@ public void testArgChecks() throws Exception { S3AInputStreamStatistics stats = readContext.getS3AStatisticsContext().newInputStreamStatistics(); - Configuration conf = S3ATestUtils.prepareTestConfiguration(new Configuration()); + Configuration conf = new Configuration(); + conf.setBoolean(PREFETCH_ENABLED_KEY, true); // Should not throw. new S3ACachingInputStream(readContext, attrs, client, stats, conf, null); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObject.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObject.java index b3788aac80834..a85a038d81627 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObject.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObject.java @@ -22,6 +22,8 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import org.apache.hadoop.fs.impl.prefetch.ExceptionAsserts; @@ -35,14 +37,26 @@ public class TestS3ARemoteObject extends AbstractHadoopTestBase { - private final ExecutorService threadPool = Executors.newFixedThreadPool(1); + private ExecutorService threadPool = Executors.newFixedThreadPool(1); - private final ExecutorServiceFuturePool futurePool = + private ExecutorServiceFuturePool futurePool = new ExecutorServiceFuturePool(threadPool); private final S3AInputStream.InputStreamCallbacks client = MockS3ARemoteObject.createClient("bucket"); + @Before + public void setup() { + threadPool = Executors.newFixedThreadPool(4); + futurePool = new ExecutorServiceFuturePool(threadPool); + } + + @After + public void teardown() { + if (threadPool != null) { + threadPool.shutdown(); + } + } @Test public void testArgChecks() throws Exception { S3AReadOpContext readContext = diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObjectReader.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObjectReader.java index db70c4f22bce9..79b6d5dc7ee1a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObjectReader.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObjectReader.java @@ -100,11 +100,11 @@ private void testGetHelper(boolean testWithRetry, long startOffset) int numBytesRead = reader.read(buffer, startOffset, readSize); int expectedNumBytesRead = Math.min(readSize, remainingSize); expectedNumBytesRead = Math.min(bufferSize, expectedNumBytesRead); - assertEquals(expectedNumBytesRead, numBytesRead); + assertEquals("bytes read", expectedNumBytesRead, numBytesRead); byte[] bytes = buffer.array(); for (int i = 0; i < expectedNumBytesRead; i++) { - assertEquals(startOffset + i, bytes[i]); + assertEquals("byte value", startOffset + i, bytes[i]); } } } diff --git a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties index 7442a357f9777..f0bbaab689fae 100644 --- a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties +++ b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties @@ -98,3 +98,6 @@ log4j.logger.org.apache.hadoop.fs.s3a.S3AStorageStatistics=INFO # uncomment this to get S3 Delete requests to return the list of deleted objects # log4.logger.org.apache.hadoop.fs.s3a.impl.RequestFactoryImpl=TRACE + +# prefetch hadoop common logs +log4j.logger.org.apache.hadoop.fs.impl.prefetch=DEBUG