From b542d91d783da759df7c99064638dbae1ce21e58 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 28 Mar 2022 11:05:34 +0100 Subject: [PATCH] HADOOP-18028. High performance S3A input stream. This is the the a rollup patch of the HADOOP-18028 S3A performance input stream feature branch. Contains HADOOP-18028. High performance S3A input stream (#4109) This is the the merge of the HADOOP-18028 S3A performance input stream. This patch on its own is incomplete and must be accompanied by all other commits with HADOOP-18028 in their git commit message. Consult the JIRA for that list Contributed by Bhalchandra Pandit. HADOOP-18180. Replace use of twitter util-core with java futures in S3A prefetching stream (#4115) Contributed by PJ Fanning. HADOOP-18177. Document prefetching architecture. (#4205) Contributed by Ahmar Suhail HADOOP-18175. fix test failures with prefetching s3a input stream (#4212) Contributed by Monthon Klongklaew HADOOP-18231. S3A prefetching: fix failing tests & drain stream async. (#4386) * adds in new test for prefetching input stream * creates streamStats before opening stream * updates numBlocks calculation method * fixes ITestS3AOpenCost.testOpenFileLongerLength * drains stream async * fixes failing unit test Contributed by Ahmar Suhail HADOOP-18254. Disable S3A prefetching by default. (#4469) Contributed by Ahmar Suhail HADOOP-18190. Collect IOStatistics during S3A prefetching (#4458) This adds iOStatisticsConnection to the S3PrefetchingInputStream class, with new statistic names in StreamStatistics. This stream is not (yet) IOStatisticsContext aware. Contributed by Ahmar Suhail. Change-Id: I48f217086531c12d6e2f0f91e39f17054a74d20f --- .../fs/statistics/StreamStatisticNames.java | 40 ++ .../apache/hadoop/fs/common/BlockCache.java | 70 ++ .../apache/hadoop/fs/common/BlockData.java | 248 +++++++ .../apache/hadoop/fs/common/BlockManager.java | 140 ++++ .../hadoop/fs/common/BlockOperations.java | 420 ++++++++++++ .../hadoop/fs/common/BoundedResourcePool.java | 182 +++++ .../apache/hadoop/fs/common/BufferData.java | 295 +++++++++ .../apache/hadoop/fs/common/BufferPool.java | 303 +++++++++ .../hadoop/fs/common/CachingBlockManager.java | 621 ++++++++++++++++++ .../fs/common/ExecutorServiceFuturePool.java | 70 ++ .../apache/hadoop/fs/common/FilePosition.java | 282 ++++++++ .../fs/common/PrefetchingStatistics.java | 67 ++ .../apache/hadoop/fs/common/ResourcePool.java | 71 ++ .../org/apache/hadoop/fs/common/Retryer.java | 87 +++ .../fs/common/SingleFilePerBlockCache.java | 347 ++++++++++ .../org/apache/hadoop/fs/common/Validate.java | 405 ++++++++++++ .../apache/hadoop/fs/common/package-info.java | 27 + .../org/apache/hadoop/fs/s3a/Constants.java | 25 + .../apache/hadoop/fs/s3a/S3AFileSystem.java | 64 +- .../hadoop/fs/s3a/S3AInstrumentation.java | 61 +- .../hadoop/fs/s3a/S3AReadOpContext.java | 53 +- .../org/apache/hadoop/fs/s3a/Statistic.java | 12 + .../hadoop/fs/s3a/read/S3BlockManager.java | 77 +++ .../fs/s3a/read/S3CachingBlockManager.java | 93 +++ .../fs/s3a/read/S3CachingInputStream.java | 207 ++++++ .../org/apache/hadoop/fs/s3a/read/S3File.java | 318 +++++++++ .../fs/s3a/read/S3InMemoryInputStream.java | 98 +++ .../hadoop/fs/s3a/read/S3InputStream.java | 459 +++++++++++++ .../fs/s3a/read/S3PrefetchingInputStream.java | 243 +++++++ .../apache/hadoop/fs/s3a/read/S3Reader.java | 169 +++++ .../hadoop/fs/s3a/read/package-info.java | 28 + .../statistics/S3AInputStreamStatistics.java | 3 +- .../impl/EmptyS3AStatisticsContext.java | 36 + .../site/markdown/tools/hadoop-aws/index.md | 24 + .../markdown/tools/hadoop-aws/prefetching.md | 192 ++++++ .../fs/common/EmptyPrefetchingStatistics.java | 76 +++ .../hadoop/fs/common/ExceptionAsserts.java | 60 ++ .../hadoop/fs/common/SampleDataForTests.java | 57 ++ .../hadoop/fs/common/TestBlockCache.java | 99 +++ .../hadoop/fs/common/TestBlockData.java | 158 +++++ .../hadoop/fs/common/TestBlockOperations.java | 105 +++ .../fs/common/TestBoundedResourcePool.java | 148 +++++ .../hadoop/fs/common/TestBufferData.java | 249 +++++++ .../hadoop/fs/common/TestBufferPool.java | 163 +++++ .../common/TestExecutorServiceFuturePool.java | 92 +++ .../hadoop/fs/common/TestFilePosition.java | 216 ++++++ .../apache/hadoop/fs/common/TestRetryer.java | 81 +++ .../apache/hadoop/fs/common/TestValidate.java | 322 +++++++++ .../hadoop/fs/s3a/ITestS3ARequesterPays.java | 16 +- .../hadoop/fs/s3a/ITestS3AUnbuffer.java | 10 + .../fs/s3a/ITestS3PrefetchingInputStream.java | 189 ++++++ .../apache/hadoop/fs/s3a/S3ATestUtils.java | 13 +- .../apache/hadoop/fs/s3a/TestS3AUnbuffer.java | 4 +- .../org/apache/hadoop/fs/s3a/read/Fakes.java | 382 +++++++++++ .../apache/hadoop/fs/s3a/read/MockS3File.java | 112 ++++ .../fs/s3a/read/TestS3BlockManager.java | 86 +++ .../s3a/read/TestS3CachingBlockManager.java | 337 ++++++++++ .../apache/hadoop/fs/s3a/read/TestS3File.java | 77 +++ .../hadoop/fs/s3a/read/TestS3InputStream.java | 251 +++++++ .../hadoop/fs/s3a/read/TestS3Reader.java | 106 +++ .../scale/ITestS3AInputStreamPerformance.java | 9 + 61 files changed, 9233 insertions(+), 22 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockCache.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockData.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockManager.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockOperations.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BoundedResourcePool.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BufferData.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BufferPool.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/CachingBlockManager.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/ExecutorServiceFuturePool.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/FilePosition.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/PrefetchingStatistics.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/ResourcePool.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/Retryer.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/SingleFilePerBlockCache.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/Validate.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/package-info.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3BlockManager.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3CachingBlockManager.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3CachingInputStream.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3File.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3InMemoryInputStream.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3InputStream.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3PrefetchingInputStream.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3Reader.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/package-info.java create mode 100644 hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/prefetching.md create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/EmptyPrefetchingStatistics.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/ExceptionAsserts.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/SampleDataForTests.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBlockCache.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBlockData.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBlockOperations.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBoundedResourcePool.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBufferData.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBufferPool.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestExecutorServiceFuturePool.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestFilePosition.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestRetryer.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestValidate.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3PrefetchingInputStream.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/Fakes.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/MockS3File.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3BlockManager.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3CachingBlockManager.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3File.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3InputStream.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3Reader.java 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 bb697ad8ccf0d..50bbf45505cec 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 @@ -415,6 +415,46 @@ public final class StreamStatisticNames { public static final String BLOCKS_RELEASED = "blocks_released"; + /** + * Total number of prefetching operations executed. + */ + public static final String STREAM_READ_PREFETCH_OPERATIONS + = "stream_read_prefetch_operations"; + + /** + * Total number of block in disk cache. + */ + public static final String STREAM_READ_BLOCKS_IN_FILE_CACHE + = "stream_read_blocks_in_cache"; + + /** + * Total number of active prefetch operations. + */ + public static final String STREAM_READ_ACTIVE_PREFETCH_OPERATIONS + = "stream_read_active_prefetch_operations"; + + /** + * Total bytes of memory in use by this input stream. + */ + public static final String STREAM_READ_ACTIVE_MEMORY_IN_USE + = "stream_read_active_memory_in_use"; + + /** + * count/duration of reading a remote block. + * + * Value: {@value}. + */ + public static final String STREAM_READ_REMOTE_BLOCK_READ + = "stream_read_block_read"; + + /** + * count/duration of acquiring a buffer and reading to it. + * + * Value: {@value}. + */ + public static final String STREAM_READ_BLOCK_ACQUIRE_AND_READ + = "stream_read_block_acquire_read"; + private StreamStatisticNames() { } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockCache.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockCache.java new file mode 100644 index 0000000000000..16354c7be5f66 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockCache.java @@ -0,0 +1,70 @@ +/* + * 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.common; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * Provides functionality necessary for caching blocks of data read from FileSystem. + */ +public interface BlockCache extends Closeable { + + /** + * Indicates whether the given block is in this cache. + * + * @param blockNumber the id of the given block. + * @return true if the given block is in this cache, false otherwise. + */ + boolean containsBlock(int blockNumber); + + /** + * Gets the blocks in this cache. + * + * @return the blocks in this cache. + */ + Iterable blocks(); + + /** + * Gets the number of blocks in this cache. + * + * @return the number of blocks in this cache. + */ + int size(); + + /** + * Gets the block having the given {@code blockNumber}. + * + * @param blockNumber the id of the desired block. + * @param buffer contents of the desired block are copied to this buffer. + * @throws IOException if there is an error reading the given block. + */ + void get(int blockNumber, ByteBuffer buffer) throws IOException; + + /** + * Puts the given block in this cache. + * + * @param blockNumber the id of the given block. + * @param buffer contents of the given block to be added to this cache. + * @throws IOException if there is an error writing the given block. + */ + void put(int blockNumber, ByteBuffer buffer) throws IOException; +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockData.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockData.java new file mode 100644 index 0000000000000..df226d0993881 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockData.java @@ -0,0 +1,248 @@ +/* + * 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.common; + +/** + * Holds information about blocks of data in a file. + */ +public class BlockData { + // State of each block of data. + enum State { + // Data is not yet ready to be read from this block (still being prefetched). + NOT_READY, + + // A read of this block has been enqueued in the prefetch queue. + QUEUED, + + // This block is ready to be read. That is, it has been fully read. + READY, + + // This block has been cached in the local disk cache. + CACHED + } + + // State of all blocks in a file. + private State[] state; + + // The size of a file. + private final long fileSize; + + // The file is divided into blocks of this size. + private final int blockSize; + + // The file has these many blocks. + private final int numBlocks; + + /** + * Constructs an instance of {@link BlockData}. + * + * @param fileSize the size of a file. + * @param blockSize the file is divided into blocks of this size. + * + * @throws IllegalArgumentException if fileSize is negative. + * @throws IllegalArgumentException if blockSize is negative. + * @throws IllegalArgumentException if blockSize is zero or negative. + */ + public BlockData(long fileSize, int blockSize) { + Validate.checkNotNegative(fileSize, "fileSize"); + if (fileSize == 0) { + Validate.checkNotNegative(blockSize, "blockSize"); + } else { + Validate.checkPositiveInteger(blockSize, "blockSize"); + } + + this.fileSize = fileSize; + this.blockSize = blockSize; + this.numBlocks = + (fileSize == 0) ? 0 : ((int) (fileSize / blockSize)) + (fileSize % blockSize > 0 ? 1 : 0); + this.state = new State[this.numBlocks]; + for (int b = 0; b < this.numBlocks; b++) { + this.setState(b, State.NOT_READY); + } + } + + /** + * Gets the size of each block. + * + * @return the size of each block. + */ + public int getBlockSize() { + return this.blockSize; + } + + /** + * Gets the size of the associated file. + * + * @return the size of the associated file. + */ + public long getFileSize() { + return this.fileSize; + } + + /** + * Gets the number of blocks in the associated file. + * + * @return the number of blocks in the associated file. + */ + public int getNumBlocks() { + return this.numBlocks; + } + + /** + * Indicates whether the given block is the last block in the associated file. + * + * @param blockNumber the id of the desired block. + * @return true if the given block is the last block in the associated file, false otherwise. + * + * @throws IllegalArgumentException if blockNumber is invalid. + */ + public boolean isLastBlock(int blockNumber) { + if (this.fileSize == 0) { + return false; + } + + throwIfInvalidBlockNumber(blockNumber); + + return blockNumber == (this.numBlocks - 1); + } + + /** + * Gets the id of the block that contains the given absolute offset. + * + * @param offset the absolute offset to check. + * @return the id of the block that contains the given absolute offset. + * + * @throws IllegalArgumentException if offset is invalid. + */ + public int getBlockNumber(long offset) { + throwIfInvalidOffset(offset); + + return (int) (offset / this.blockSize); + } + + /** + * Gets the size of the given block. + * + * @param blockNumber the id of the desired block. + * @return the size of the given block. + */ + public int getSize(int blockNumber) { + if (this.fileSize == 0) { + return 0; + } + + if (this.isLastBlock(blockNumber)) { + return (int) (this.fileSize - (((long) this.blockSize) * (this.numBlocks - 1))); + } else { + return this.blockSize; + } + } + + /** + * Indicates whether the given absolute offset is valid. + * + * @param offset absolute offset in the file.. + * @return true if the given absolute offset is valid, false otherwise. + */ + public boolean isValidOffset(long offset) { + return (offset >= 0) && (offset < this.fileSize); + } + + /** + * Gets the start offset of the given block. + + * @param blockNumber the id of the given block. + * @return the start offset of the given block. + * + * @throws IllegalArgumentException if blockNumber is invalid. + */ + public long getStartOffset(int blockNumber) { + throwIfInvalidBlockNumber(blockNumber); + + return blockNumber * (long) this.blockSize; + } + + /** + * Gets the relative offset corresponding to the given block and the absolute offset. + * + * @param blockNumber the id of the given block. + * @param offset absolute offset in the file. + * @return the relative offset corresponding to the given block and the absolute offset. + * + * @throws IllegalArgumentException if either blockNumber or offset is invalid. + */ + public int getRelativeOffset(int blockNumber, long offset) { + throwIfInvalidOffset(offset); + + return (int) (offset - this.getStartOffset(blockNumber)); + } + + /** + * Gets the state of the given block. + * + * @param blockNumber the id of the given block. + * @return the state of the given block. + * + * @throws IllegalArgumentException if blockNumber is invalid. + */ + public State getState(int blockNumber) { + throwIfInvalidBlockNumber(blockNumber); + + return this.state[blockNumber]; + } + + /** + * Sets the state of the given block to the given value. + * + * @param blockNumber the id of the given block. + * @param blockState the target state. + * + * @throws IllegalArgumentException if blockNumber is invalid. + */ + public void setState(int blockNumber, State blockState) { + throwIfInvalidBlockNumber(blockNumber); + + this.state[blockNumber] = blockState; + } + + // Debug helper. + public String getStateString() { + StringBuilder sb = new StringBuilder(); + int blockNumber = 0; + while (blockNumber < this.numBlocks) { + State tstate = this.getState(blockNumber); + int endBlockNumber = blockNumber; + while ((endBlockNumber < this.numBlocks) && (this.getState(endBlockNumber) == tstate)) { + endBlockNumber++; + } + sb.append(String.format("[%03d ~ %03d] %s%n", blockNumber, endBlockNumber - 1, tstate)); + blockNumber = endBlockNumber; + } + return sb.toString(); + } + + private void throwIfInvalidBlockNumber(int blockNumber) { + Validate.checkWithinRange(blockNumber, "blockNumber", 0, this.numBlocks - 1); + } + + private void throwIfInvalidOffset(long offset) { + Validate.checkWithinRange(offset, "offset", 0, this.fileSize - 1); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockManager.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockManager.java new file mode 100644 index 0000000000000..f5672aa6920a4 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockManager.java @@ -0,0 +1,140 @@ +/* + * 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.common; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * Provides read access to the underlying file one block at a time. + * + * This class is the simplest form of a {@code BlockManager} that does + * perform prefetching or caching. + */ +public abstract class BlockManager implements Closeable { + + // Information about each block of the underlying file. + private BlockData blockData; + + /** + * Constructs an instance of {@code BlockManager}. + * + * @param blockData information about each block of the underlying file. + * + * @throws IllegalArgumentException if blockData is null. + */ + public BlockManager(BlockData blockData) { + Validate.checkNotNull(blockData, "blockData"); + + this.blockData = blockData; + } + + /** + * Gets block data information. + * + * @return instance of {@code BlockData}. + */ + public BlockData getBlockData() { + return this.blockData; + } + + /** + * Gets the block having the given {@code blockNumber}. + * + * The entire block is read into memory and returned as a {@code BufferData}. + * The blocks are treated as a limited resource and must be released when + * one is done reading them. + * + * @param blockNumber the number of the block to be read and returned. + * @return {@code BufferData} having data from the given block. + * + * @throws IOException if there an error reading the given block. + * @throws IllegalArgumentException if blockNumber is negative. + */ + public BufferData get(int blockNumber) throws IOException { + Validate.checkNotNegative(blockNumber, "blockNumber"); + + int size = this.blockData.getSize(blockNumber); + ByteBuffer buffer = ByteBuffer.allocate(size); + long startOffset = this.blockData.getStartOffset(blockNumber); + this.read(buffer, startOffset, size); + buffer.flip(); + return new BufferData(blockNumber, buffer); + } + + /** + * Reads into the given {@code buffer} {@code size} bytes from the underlying file + * starting at {@code startOffset}. + * + * @param buffer the buffer to read data in to. + * @param startOffset the offset at which reading starts. + * @param size the number bytes to read. + * @return number of bytes read. + * @throws IOException if there an error reading the given block. + */ + public abstract int read(ByteBuffer buffer, long startOffset, int size) throws IOException; + + /** + * Releases resources allocated to the given block. + * + * @param data the {@code BufferData} to release. + * + * @throws IllegalArgumentException if data is null. + */ + public void release(BufferData data) { + Validate.checkNotNull(data, "data"); + + // Do nothing because we allocate a new buffer each time. + } + + /** + * Requests optional prefetching of the given block. + * + * @param blockNumber the id of the block to prefetch. + * + * @throws IllegalArgumentException if blockNumber is negative. + */ + public void requestPrefetch(int blockNumber) { + Validate.checkNotNegative(blockNumber, "blockNumber"); + + // Do nothing because we do not support prefetches. + } + + /** + * Requests cancellation of any previously issued prefetch requests. + */ + public void cancelPrefetches() { + // Do nothing because we do not support prefetches. + } + + /** + * Requests that the given block should be copied to the cache. Optional operation. + * + * @param data the {@code BufferData} instance to optionally cache. + */ + public void requestCaching(BufferData data) { + // Do nothing because we do not support caching. + } + + @Override + public void close() { + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockOperations.java new file mode 100644 index 0000000000000..2b322856293af --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockOperations.java @@ -0,0 +1,420 @@ +/* + * 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.common; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.DoubleSummaryStatistics; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Block level operations performed on a file. + * This class is meant to be used by {@code BlockManager}. + * It is separated out in its own file due to its size. + * + * This class is used for debugging/logging. Calls to this class + * can be safely removed without affecting the overall operation. + */ +public class BlockOperations { + private static final Logger LOG = LoggerFactory.getLogger(BlockOperations.class); + + public enum Kind { + UNKNOWN("??", "unknown", false), + CANCEL_PREFETCHES("CP", "cancelPrefetches", false), + CLOSE("CX", "close", false), + CACHE_PUT("C+", "putC", true), + GET_CACHED("GC", "getCached", true), + GET_PREFETCHED("GP", "getPrefetched", true), + GET_READ("GR", "getRead", true), + PREFETCH("PF", "prefetch", true), + RELEASE("RL", "release", true), + REQUEST_CACHING("RC", "requestCaching", true), + REQUEST_PREFETCH("RP", "requestPrefetch", true); + + private String shortName; + private String name; + private boolean hasBlock; + + Kind(String shortName, String name, boolean hasBlock) { + this.shortName = shortName; + this.name = name; + this.hasBlock = hasBlock; + } + + private static Map shortNameToKind = new HashMap<>(); + + public static Kind fromShortName(String shortName) { + if (shortNameToKind.size() == 0) { + for (Kind kind : Kind.values()) { + shortNameToKind.put(kind.shortName, kind); + } + } + return shortNameToKind.get(shortName); + } + } + + public static class Operation { + private final Kind kind; + private final int blockNumber; + private final long timestamp; + + public Operation(Kind kind, int blockNumber) { + this.kind = kind; + this.blockNumber = blockNumber; + this.timestamp = System.nanoTime(); + } + + public Kind getKind() { + return this.kind; + } + + public int getBlockNumber() { + return this.blockNumber; + } + + public long getTimestamp() { + return this.timestamp; + } + + public void getSummary(StringBuilder sb) { + if (this.kind.hasBlock) { + sb.append(String.format("%s(%d)", this.kind.shortName, this.blockNumber)); + } else { + sb.append(String.format("%s", this.kind.shortName)); + } + } + + public String getDebugInfo() { + if (this.kind.hasBlock) { + return String.format("--- %s(%d)", this.kind.name, this.blockNumber); + } else { + return String.format("... %s()", this.kind.name); + } + } + } + + public static class End extends Operation { + private Operation op; + + public End(Operation op) { + super(op.kind, op.blockNumber); + this.op = op; + } + + @Override + public void getSummary(StringBuilder sb) { + sb.append("E"); + super.getSummary(sb); + } + + @Override + public String getDebugInfo() { + return "***" + super.getDebugInfo().substring(3); + } + + public double duration() { + return (this.getTimestamp() - this.op.getTimestamp()) / 1e9; + } + } + + private ArrayList ops; + private boolean debugMode; + + public BlockOperations() { + this.ops = new ArrayList<>(); + } + + public synchronized void setDebug(boolean state) { + this.debugMode = state; + } + + private synchronized Operation add(Operation op) { + if (this.debugMode) { + LOG.info(op.getDebugInfo()); + } + ops.add(op); + return op; + } + + public Operation getPrefetched(int blockNumber) { + Validate.checkNotNegative(blockNumber, "blockNumber"); + + return this.add(new Operation(Kind.GET_PREFETCHED, blockNumber)); + } + + public Operation getCached(int blockNumber) { + Validate.checkNotNegative(blockNumber, "blockNumber"); + + return this.add(new Operation(Kind.GET_CACHED, blockNumber)); + } + + public Operation getRead(int blockNumber) { + Validate.checkNotNegative(blockNumber, "blockNumber"); + + return this.add(new Operation(Kind.GET_READ, blockNumber)); + } + + public Operation release(int blockNumber) { + Validate.checkNotNegative(blockNumber, "blockNumber"); + + return this.add(new Operation(Kind.RELEASE, blockNumber)); + } + + public Operation requestPrefetch(int blockNumber) { + Validate.checkNotNegative(blockNumber, "blockNumber"); + + return this.add(new Operation(Kind.REQUEST_PREFETCH, blockNumber)); + } + + public Operation prefetch(int blockNumber) { + Validate.checkNotNegative(blockNumber, "blockNumber"); + + return this.add(new Operation(Kind.PREFETCH, blockNumber)); + } + + public Operation cancelPrefetches() { + return this.add(new Operation(Kind.CANCEL_PREFETCHES, -1)); + } + + public Operation close() { + return this.add(new Operation(Kind.CLOSE, -1)); + } + + public Operation requestCaching(int blockNumber) { + Validate.checkNotNegative(blockNumber, "blockNumber"); + + return this.add(new Operation(Kind.REQUEST_CACHING, blockNumber)); + } + + public Operation addToCache(int blockNumber) { + Validate.checkNotNegative(blockNumber, "blockNumber"); + + return this.add(new Operation(Kind.CACHE_PUT, blockNumber)); + } + + public Operation end(Operation op) { + return this.add(new End(op)); + } + + private static void append(StringBuilder sb, String format, Object... args) { + sb.append(String.format(format, args)); + } + + public synchronized String getSummary(boolean showDebugInfo) { + StringBuilder sb = new StringBuilder(); + for (Operation op : this.ops) { + if (op != null) { + if (showDebugInfo) { + sb.append(op.getDebugInfo()); + sb.append("\n"); + } else { + op.getSummary(sb); + sb.append(";"); + } + } + } + + sb.append("\n"); + this.getDurationInfo(sb); + + return sb.toString(); + } + + public synchronized void getDurationInfo(StringBuilder sb) { + Map durations = new HashMap<>(); + for (Operation op : this.ops) { + if (op instanceof End) { + End endOp = (End) op; + DoubleSummaryStatistics stats = durations.get(endOp.getKind()); + if (stats == null) { + stats = new DoubleSummaryStatistics(); + durations.put(endOp.getKind(), stats); + } + stats.accept(endOp.duration()); + } + } + + List kinds = Arrays.asList( + Kind.GET_CACHED, + Kind.GET_PREFETCHED, + Kind.GET_READ, + Kind.CACHE_PUT, + Kind.PREFETCH, + Kind.REQUEST_CACHING, + Kind.REQUEST_PREFETCH, + Kind.CANCEL_PREFETCHES, + Kind.RELEASE, + Kind.CLOSE + ); + + for (Kind kind : kinds) { + append(sb, "%-18s : ", kind); + DoubleSummaryStatistics stats = durations.get(kind); + if (stats == null) { + append(sb, "--\n"); + } else { + append( + sb, + "#ops = %3d, total = %5.1f, min: %3.1f, avg: %3.1f, max: %3.1f\n", + stats.getCount(), + stats.getSum(), + stats.getMin(), + stats.getAverage(), + stats.getMax()); + } + } + } + + public synchronized void analyze(StringBuilder sb) { + Map> blockOps = new HashMap<>(); + + // Group-by block number. + for (Operation op : this.ops) { + if (op.blockNumber < 0) { + continue; + } + + List perBlockOps; + if (!blockOps.containsKey(op.blockNumber)) { + perBlockOps = new ArrayList<>(); + blockOps.put(op.blockNumber, perBlockOps); + } + + perBlockOps = blockOps.get(op.blockNumber); + perBlockOps.add(op); + } + + List prefetchedNotUsed = new ArrayList<>(); + List cachedNotUsed = new ArrayList<>(); + + for (Map.Entry> entry : blockOps.entrySet()) { + Integer blockNumber = entry.getKey(); + List perBlockOps = entry.getValue(); + Map kindCounts = new HashMap<>(); + Map endKindCounts = new HashMap<>(); + + for (Operation op : perBlockOps) { + if (op instanceof End) { + int endCount = endKindCounts.getOrDefault(op.kind, 0) + 1; + endKindCounts.put(op.kind, endCount); + } else { + int count = kindCounts.getOrDefault(op.kind, 0) + 1; + kindCounts.put(op.kind, count); + } + } + + for (Kind kind : kindCounts.keySet()) { + int count = kindCounts.getOrDefault(kind, 0); + int endCount = endKindCounts.getOrDefault(kind, 0); + if (count != endCount) { + append(sb, "[%d] %s : #ops(%d) != #end-ops(%d)\n", blockNumber, kind, count, endCount); + } + + if (count > 1) { + append(sb, "[%d] %s = %d\n", blockNumber, kind, count); + } + } + + int prefetchCount = kindCounts.getOrDefault(Kind.PREFETCH, 0); + int getPrefetchedCount = kindCounts.getOrDefault(Kind.GET_PREFETCHED, 0); + if ((prefetchCount > 0) && (getPrefetchedCount < prefetchCount)) { + prefetchedNotUsed.add(blockNumber); + } + + int cacheCount = kindCounts.getOrDefault(Kind.CACHE_PUT, 0); + int getCachedCount = kindCounts.getOrDefault(Kind.GET_CACHED, 0); + if ((cacheCount > 0) && (getCachedCount < cacheCount)) { + cachedNotUsed.add(blockNumber); + } + } + + if (prefetchedNotUsed.size() > 0) { + append(sb, "Prefetched but not used: %s\n", getIntList(prefetchedNotUsed)); + } + + if (cachedNotUsed.size() > 0) { + append(sb, "Cached but not used: %s\n", getIntList(cachedNotUsed)); + } + } + + private static String getIntList(Iterable nums) { + List numList = new ArrayList<>(); + for (Integer n : nums) { + numList.add(n.toString()); + } + return String.join(", ", numList); + } + + public static BlockOperations fromSummary(String summary) { + BlockOperations ops = new BlockOperations(); + ops.setDebug(true); + Pattern blockOpPattern = Pattern.compile("([A-Z+]+)(\\(([0-9]+)?\\))?"); + String[] tokens = summary.split(";"); + for (String token : tokens) { + Matcher matcher = blockOpPattern.matcher(token); + if (!matcher.matches()) { + String message = String.format("Unknown summary format: %s", token); + throw new IllegalArgumentException(message); + } + + String shortName = matcher.group(1); + String blockNumberStr = matcher.group(3); + int blockNumber = (blockNumberStr == null) ? -1 : Integer.parseInt(blockNumberStr); + Kind kind = Kind.fromShortName(shortName); + Kind endKind = null; + if (kind == null) { + if (shortName.charAt(0) == 'E') { + endKind = Kind.fromShortName(shortName.substring(1)); + } + } + + if (kind == null && endKind == null) { + String message = String.format("Unknown short name: %s (token = %s)", shortName, token); + throw new IllegalArgumentException(message); + } + + if (kind != null) { + ops.add(new Operation(kind, blockNumber)); + } else { + Operation op = null; + for (int i = ops.ops.size() - 1; i >= 0; i--) { + op = ops.ops.get(i); + if ((op.blockNumber == blockNumber) && (op.kind == endKind) && !(op instanceof End)) { + ops.add(new End(op)); + break; + } + } + + if (op == null) { + LOG.warn("Start op not found: %s(%d)", endKind, blockNumber); + } + } + } + + return ops; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BoundedResourcePool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BoundedResourcePool.java new file mode 100644 index 0000000000000..40f0a1cee5717 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BoundedResourcePool.java @@ -0,0 +1,182 @@ +/* + * 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.common; + +import java.util.Collections; +import java.util.IdentityHashMap; +import java.util.Set; +import java.util.concurrent.ArrayBlockingQueue; + +/** + * Manages a fixed pool of resources. + * + * Avoids creating a new resource if a previously created instance is already available. + */ +public abstract class BoundedResourcePool extends ResourcePool { + // The size of this pool. Fixed at creation time. + private final int size; + + // Items currently available in the pool. + private ArrayBlockingQueue items; + + // Items that have been created so far (regardless of whether they are currently available). + private Set createdItems; + + /** + * Constructs a resource pool of the given size. + * + * @param size the size of this pool. Cannot be changed post creation. + * + * @throws IllegalArgumentException if size is zero or negative. + */ + public BoundedResourcePool(int size) { + Validate.checkPositiveInteger(size, "size"); + + this.size = size; + this.items = new ArrayBlockingQueue(size); + + // The created items are identified based on their object reference. + this.createdItems = Collections.newSetFromMap(new IdentityHashMap()); + } + + /** + * Acquires a resource blocking if necessary until one becomes available. + */ + @Override + public T acquire() { + return this.acquireHelper(true); + } + + /** + * Acquires a resource blocking if one is immediately available. Otherwise returns null. + */ + @Override + public T tryAcquire() { + return this.acquireHelper(false); + } + + /** + * Releases a previously acquired resource. + * + * @throws IllegalArgumentException if item is null. + */ + @Override + public void release(T item) { + Validate.checkNotNull(item, "item"); + + synchronized (this.createdItems) { + if (!this.createdItems.contains(item)) { + throw new IllegalArgumentException("This item is not a part of this pool"); + } + } + + // Return if this item was released earlier. + // We cannot use this.items.contains() because that check is not based on reference equality. + for (T entry : this.items) { + if (entry == item) { + return; + } + } + + try { + this.items.put(item); + return; + } catch (InterruptedException e) { + throw new IllegalStateException("release() should never block"); + } + } + + @Override + public synchronized void close() { + for (T item : this.createdItems) { + this.close(item); + } + + this.items.clear(); + this.items = null; + + this.createdItems.clear(); + this.createdItems = null; + } + + /** + * Derived classes may implement a way to cleanup each item. + */ + @Override + protected synchronized void close(T item) { + // Do nothing in this class. Allow overriding classes to take any cleanup action. + } + + // Number of items created so far. Mostly for testing purposes. + public int numCreated() { + synchronized (this.createdItems) { + return this.createdItems.size(); + } + } + + // Number of items available to be acquired. Mostly for testing purposes. + public synchronized int numAvailable() { + return (this.size - this.numCreated()) + this.items.size(); + } + + // For debugging purposes. + @Override + public synchronized String toString() { + return String.format( + "size = %d, #created = %d, #in-queue = %d, #available = %d", + this.size, this.numCreated(), this.items.size(), this.numAvailable()); + } + + /** + * Derived classes must implement a way to create an instance of a resource. + */ + protected abstract T createNew(); + + private T acquireHelper(boolean canBlock) { + + // Prefer reusing an item if one is available. + // That avoids unnecessarily creating new instances. + T result = this.items.poll(); + if (result != null) { + return result; + } + + synchronized (this.createdItems) { + // Create a new instance if allowed by the capacity of this pool. + if (this.createdItems.size() < this.size) { + T item = this.createNew(); + this.createdItems.add(item); + return item; + } + } + + if (canBlock) { + try { + // Block for an instance to be available. + return this.items.take(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + return null; + } + } else { + return null; + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BufferData.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BufferData.java new file mode 100644 index 0000000000000..a855a1c2c390c --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BufferData.java @@ -0,0 +1,295 @@ +/* + * 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.common; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Future; +import java.util.zip.CRC32; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Holds the state of a ByteBuffer that is in use by {@code CachingBlockManager}. + * + * This class is not meant to be of general use. It exists into its own file due to its size. + * We use the term block and buffer interchangeably in this file because one buffer + * holds exactly one block of data. + * + * Holding all of the state associated with a block allows us to validate and control + * state transitions in a synchronized fashion. + */ +public class BufferData { + private static final Logger LOG = LoggerFactory.getLogger(BufferData.class); + + public enum State { + // Unknown / invalid state. + UNKNOWN, + + // Buffer has been acquired but has no data. + BLANK, + + // This block is being prefetched. + PREFETCHING, + + // This block is being added to the local cache. + CACHING, + + // This block has data and is ready to be read. + READY, + + // This block is no longer in-use and should not be used once in this state. + DONE + } + + // Number of the block associated with this buffer. + private final int blockNumber; + + // The buffer associated with this block. + private ByteBuffer buffer; + + // Current state of this block. + private volatile State state; + + // Future of the action being performed on this block (eg, prefetching or caching). + private Future action; + + // Checksum of the buffer contents once in READY state. + private long checksum = 0; + + /** + * Constructs an instances of this class. + * + * @param blockNumber Number of the block associated with this buffer. + * @param buffer The buffer associated with this block. + * + * @throws IllegalArgumentException if blockNumber is negative. + * @throws IllegalArgumentException if buffer is null. + */ + public BufferData(int blockNumber, ByteBuffer buffer) { + Validate.checkNotNegative(blockNumber, "blockNumber"); + Validate.checkNotNull(buffer, "buffer"); + + this.blockNumber = blockNumber; + this.buffer = buffer; + this.state = State.BLANK; + } + + /** + * Gets the id of this block. + * + * @return the id of this block. + */ + public int getBlockNumber() { + return this.blockNumber; + } + + /** + * Gets the buffer associated with this block. + * + * @return the buffer associated with this block. + */ + public ByteBuffer getBuffer() { + return this.buffer; + } + + /** + * Gets the state of this block. + * + * @return the state of this block. + */ + public State getState() { + return this.state; + } + + /** + * Gets the checksum of data in this block. + * + * @return the checksum of data in this block. + */ + public long getChecksum() { + return this.checksum; + } + + /** + * Computes CRC32 checksum of the given buffer's contents. + * + * @param buffer the buffer whose content's checksum is to be computed. + * @return the computed checksum. + */ + public static long getChecksum(ByteBuffer buffer) { + ByteBuffer tempBuffer = buffer.duplicate(); + tempBuffer.rewind(); + CRC32 crc32 = new CRC32(); + crc32.update(tempBuffer); + return crc32.getValue(); + } + + public synchronized Future getActionFuture() { + return this.action; + } + + /** + * Indicates that a prefetch operation is in progress. + * + * @param actionFuture the {@code Future} of a prefetch action. + * + * @throws IllegalArgumentException if actionFuture is null. + */ + public synchronized void setPrefetch(Future actionFuture) { + Validate.checkNotNull(actionFuture, "actionFuture"); + + this.updateState(State.PREFETCHING, State.BLANK); + this.action = actionFuture; + } + + /** + * Indicates that a caching operation is in progress. + * + * @param actionFuture the {@code Future} of a caching action. + * + * @throws IllegalArgumentException if actionFuture is null. + */ + public synchronized void setCaching(Future actionFuture) { + Validate.checkNotNull(actionFuture, "actionFuture"); + + this.throwIfStateIncorrect(State.PREFETCHING, State.READY); + this.state = State.CACHING; + this.action = actionFuture; + } + + /** + * Marks the completion of reading data into the buffer. + * The buffer cannot be modified once in this state. + * + * @param expectedCurrentState the collection of states from which transition to READY is allowed. + */ + public synchronized void setReady(State... expectedCurrentState) { + if (this.checksum != 0) { + throw new IllegalStateException("Checksum cannot be changed once set"); + } + + this.buffer = this.buffer.asReadOnlyBuffer(); + this.checksum = getChecksum(this.buffer); + this.buffer.rewind(); + this.updateState(State.READY, expectedCurrentState); + } + + /** + * Indicates that this block is no longer of use and can be reclaimed. + */ + public synchronized void setDone() { + if (this.checksum != 0) { + if (getChecksum(this.buffer) != this.checksum) { + throw new IllegalStateException("checksum changed after setReady()"); + } + } + this.state = State.DONE; + this.action = null; + } + + /** + * Updates the current state to the specified value. + * Asserts that the current state is as expected. + * + * @param newState the state to transition to. + * @param expectedCurrentState the collection of states from which + * transition to {@code newState} is allowed. + * + * @throws IllegalArgumentException if newState is null. + * @throws IllegalArgumentException if expectedCurrentState is null. + */ + public synchronized void updateState(State newState, State... expectedCurrentState) { + Validate.checkNotNull(newState, "newState"); + Validate.checkNotNull(expectedCurrentState, "expectedCurrentState"); + + this.throwIfStateIncorrect(expectedCurrentState); + this.state = newState; + } + + /** + * Helper that asserts the current state is one of the expected values. + * + * @param states the collection of allowed states. + * + * @throws IllegalArgumentException if states is null. + */ + public void throwIfStateIncorrect(State... states) { + Validate.checkNotNull(states, "states"); + + if (this.stateEqualsOneOf(states)) { + return; + } + + List statesStr = new ArrayList(); + for (State s : states) { + statesStr.add(s.toString()); + } + + String message = String.format( + "Expected buffer state to be '%s' but found: %s", String.join(" or ", statesStr), this); + throw new IllegalStateException(message); + } + + public boolean stateEqualsOneOf(State... states) { + State currentState = this.state; + + for (State s : states) { + if (currentState == s) { + return true; + } + } + + return false; + } + + public String toString() { + + return String.format( + "[%03d] id: %03d, %s: buf: %s, checksum: %d, future: %s", + this.blockNumber, + System.identityHashCode(this), + this.state, + this.getBufferStr(this.buffer), + this.checksum, + this.getFutureStr(this.action)); + } + + private String getFutureStr(Future f) { + if (f == null) { + return "--"; + } else { + return this.action.isDone() ? "done" : "not done"; + } + } + + private String getBufferStr(ByteBuffer buf) { + if (buf == null) { + return "--"; + } else { + return String.format( + "(id = %d, pos = %d, lim = %d)", + System.identityHashCode(buf), + buf.position(), buf.limit()); + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BufferPool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BufferPool.java new file mode 100644 index 0000000000000..bd7da11ddd8b7 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BufferPool.java @@ -0,0 +1,303 @@ +/* + * 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.common; + +import java.io.Closeable; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.IdentityHashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Future; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static java.util.Objects.requireNonNull; + +/** + * Manages a fixed pool of {@code ByteBuffer} instances. + * + * Avoids creating a new buffer if a previously created buffer is already available. + */ +public class BufferPool implements Closeable { + private static final Logger LOG = LoggerFactory.getLogger(BufferPool.class); + + // Max number of buffers in this pool. + private final int size; + + // Size in bytes of each buffer. + private final int bufferSize; + + // Invariants for internal state. + // -- a buffer is either in this.pool or in this.allocated + // -- transition between this.pool <==> this.allocated must be atomic + // -- only one buffer allocated for a given blockNumber + + // Underlying bounded resource pool. + private BoundedResourcePool pool; + + // Allows associating metadata to each buffer in the pool. + private Map allocated; + + private PrefetchingStatistics prefetchingStatistics; + + /** + * Initializes a new instance of the {@code BufferPool} class. + * + * @param size number of buffer in this pool. + * @param bufferSize size in bytes of each buffer. + * @param prefetchingStatistics statistics for this stream. + * + * @throws IllegalArgumentException if size is zero or negative. + * @throws IllegalArgumentException if bufferSize is zero or negative. + */ + public BufferPool(int size, int bufferSize, PrefetchingStatistics prefetchingStatistics) { + Validate.checkPositiveInteger(size, "size"); + Validate.checkPositiveInteger(bufferSize, "bufferSize"); + + this.size = size; + this.bufferSize = bufferSize; + this.allocated = new IdentityHashMap(); + this.prefetchingStatistics = requireNonNull(prefetchingStatistics); + this.pool = new BoundedResourcePool(size) { + @Override + public ByteBuffer createNew() { + ByteBuffer buffer = ByteBuffer.allocate(bufferSize); + prefetchingStatistics.memoryAllocated(bufferSize); + return buffer; + } + }; + } + + /** + * Gets a list of all blocks in this pool. + * + * @return a list of all blocks in this pool. + */ + public List getAll() { + synchronized (this.allocated) { + return Collections.unmodifiableList(new ArrayList(this.allocated.keySet())); + } + } + + /** + * Acquires a {@code ByteBuffer}; blocking if necessary until one becomes available. + * + * @param blockNumber the id of the block to acquire. + * @return the acquired block's {@code BufferData}. + */ + public synchronized BufferData acquire(int blockNumber) { + BufferData data; + final int maxRetryDelayMs = 600 * 1000; + final int statusUpdateDelayMs = 120 * 1000; + Retryer retryer = new Retryer(10, maxRetryDelayMs, statusUpdateDelayMs); + + do { + if (retryer.updateStatus()) { + LOG.warn("waiting to acquire block: {}", blockNumber); + LOG.info("state = {}", this.toString()); + this.releaseReadyBlock(blockNumber); + } + data = this.tryAcquire(blockNumber); + } + while ((data == null) && retryer.continueRetry()); + + if (data != null) { + return data; + } else { + String message = String.format("Wait failed for acquire(%d)", blockNumber); + throw new IllegalStateException(message); + } + } + + /** + * Acquires a buffer if one is immediately available. Otherwise returns null. + * + * @param blockNumber the id of the block to try acquire. + * @return the acquired block's {@code BufferData} or null. + */ + public synchronized BufferData tryAcquire(int blockNumber) { + return this.acquireHelper(blockNumber, false); + } + + private synchronized BufferData acquireHelper(int blockNumber, boolean canBlock) { + Validate.checkNotNegative(blockNumber, "blockNumber"); + + this.releaseDoneBlocks(); + + BufferData data = this.find(blockNumber); + if (data != null) { + return data; + } + + ByteBuffer buffer = canBlock ? this.pool.acquire() : this.pool.tryAcquire(); + if (buffer == null) { + return null; + } + + buffer.clear(); + data = new BufferData(blockNumber, buffer.duplicate()); + + synchronized (this.allocated) { + Validate.checkState(this.find(blockNumber) == null, "buffer data already exists"); + + this.allocated.put(data, buffer); + } + + return data; + } + + /** + * Releases resources for any blocks marked as 'done'. + */ + private synchronized void releaseDoneBlocks() { + for (BufferData data : this.getAll()) { + if (data.stateEqualsOneOf(BufferData.State.DONE)) { + this.release(data); + } + } + } + + /** + * If no blocks were released after calling releaseDoneBlocks() a few times, + * we may end up waiting forever. To avoid that situation, we try releasing + * a 'ready' block farthest away from the given block. + */ + private synchronized void releaseReadyBlock(int blockNumber) { + BufferData releaseTarget = null; + for (BufferData data : this.getAll()) { + if (data.stateEqualsOneOf(BufferData.State.READY)) { + if (releaseTarget == null) { + releaseTarget = data; + } else { + if (distance(data, blockNumber) > distance(releaseTarget, blockNumber)) { + releaseTarget = data; + } + } + } + } + + if (releaseTarget != null) { + LOG.warn("releasing 'ready' block: {}", releaseTarget); + releaseTarget.setDone(); + } + } + + private int distance(BufferData data, int blockNumber) { + return Math.abs(data.getBlockNumber() - blockNumber); + } + + /** + * Releases a previously acquired resource. + * + * @param data the {@code BufferData} instance to release. + * + * @throws IllegalArgumentException if data is null. + * @throws IllegalArgumentException if data cannot be released due to its state. + */ + public synchronized void release(BufferData data) { + Validate.checkNotNull(data, "data"); + + synchronized (data) { + Validate.checkArgument( + this.canRelease(data), + String.format("Unable to release buffer: %s", data)); + + ByteBuffer buffer = this.allocated.get(data); + if (buffer == null) { + // Likely released earlier. + return; + } + buffer.clear(); + this.pool.release(buffer); + this.allocated.remove(data); + } + + this.releaseDoneBlocks(); + } + + @Override + public synchronized void close() { + for (BufferData data : this.getAll()) { + Future actionFuture = data.getActionFuture(); + if (actionFuture != null) { + actionFuture.cancel(true); + } + } + + int currentPoolSize = pool.numCreated(); + + pool.close(); + pool = null; + + allocated.clear(); + allocated = null; + + prefetchingStatistics.memoryFreed(currentPoolSize * bufferSize); + } + + // For debugging purposes. + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(this.pool.toString()); + sb.append("\n"); + List allData = new ArrayList<>(this.getAll()); + Collections.sort(allData, (d1, d2) -> d1.getBlockNumber() - d2.getBlockNumber()); + for (BufferData data : allData) { + sb.append(data.toString()); + sb.append("\n"); + } + + return sb.toString(); + } + + // Number of ByteBuffers created so far. + public synchronized int numCreated() { + return this.pool.numCreated(); + } + + // Number of ByteBuffers available to be acquired. + public synchronized int numAvailable() { + this.releaseDoneBlocks(); + return this.pool.numAvailable(); + } + + private BufferData find(int blockNumber) { + synchronized (this.allocated) { + for (BufferData data : this.allocated.keySet()) { + if ((data.getBlockNumber() == blockNumber) + && !data.stateEqualsOneOf(BufferData.State.DONE)) { + return data; + } + } + } + + return null; + } + + private boolean canRelease(BufferData data) { + return data.stateEqualsOneOf( + BufferData.State.DONE, + BufferData.State.READY); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/CachingBlockManager.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/CachingBlockManager.java new file mode 100644 index 0000000000000..1207d3d031844 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/CachingBlockManager.java @@ -0,0 +1,621 @@ +/* + * 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.common; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.time.Duration; +import java.time.Instant; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.statistics.DurationTracker; + +import static java.util.Objects.requireNonNull; + +import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; + +/** + * Provides read access to the underlying file one block at a time. + * Improve read performance by prefetching and locall caching blocks. + */ +public abstract class CachingBlockManager extends BlockManager { + private static final Logger LOG = LoggerFactory.getLogger(CachingBlockManager.class); + private static final int TIMEOUT_MINUTES = 60; + + // Asynchronous tasks are performed in this pool. + private final ExecutorServiceFuturePool futurePool; + + // Pool of shared ByteBuffer instances. + private BufferPool bufferPool; + + // Size of the in-memory cache in terms of number of blocks. + // Total memory consumption is up to bufferPoolSize * blockSize. + private final int bufferPoolSize; + + // Local block cache. + private BlockCache cache; + + // Error counts. For testing purposes. + private final AtomicInteger numCachingErrors; + private final AtomicInteger numReadErrors; + + // Operations performed by this block manager. + private final BlockOperations ops; + + private boolean closed; + + // If a single caching operation takes more than this time (in seconds), + // we disable caching to prevent further perf degradation due to caching. + private static final int SLOW_CACHING_THRESHOLD = 5; + + // Once set to true, any further caching requests will be ignored. + private final AtomicBoolean cachingDisabled; + + private final PrefetchingStatistics prefetchingStatistics; + + /** + * Constructs an instance of a {@code CachingBlockManager}. + * + * @param futurePool asynchronous tasks are performed in this pool. + * @param blockData information about each block of the underlying file. + * @param bufferPoolSize size of the in-memory cache in terms of number of blocks. + * @param prefetchingStatistics statistics for this stream. + * + * @throws IllegalArgumentException if bufferPoolSize is zero or negative. + */ + public CachingBlockManager( + ExecutorServiceFuturePool futurePool, + BlockData blockData, + int bufferPoolSize, + PrefetchingStatistics prefetchingStatistics) { + super(blockData); + + Validate.checkPositiveInteger(bufferPoolSize, "bufferPoolSize"); + + this.futurePool = requireNonNull(futurePool); + this.bufferPoolSize = bufferPoolSize; + this.numCachingErrors = new AtomicInteger(); + this.numReadErrors = new AtomicInteger(); + this.cachingDisabled = new AtomicBoolean(); + this.prefetchingStatistics = requireNonNull(prefetchingStatistics); + + if (this.getBlockData().getFileSize() > 0) { + this.bufferPool = new BufferPool(bufferPoolSize, this.getBlockData().getBlockSize(), + this.prefetchingStatistics); + this.cache = this.createCache(); + } + + this.ops = new BlockOperations(); + this.ops.setDebug(false); + } + + /** + * Gets the block having the given {@code blockNumber}. + * + * @throws IllegalArgumentException if blockNumber is negative. + */ + @Override + public BufferData get(int blockNumber) throws IOException { + Validate.checkNotNegative(blockNumber, "blockNumber"); + + BufferData data = null; + final int maxRetryDelayMs = this.bufferPoolSize * 120 * 1000; + final int statusUpdateDelayMs = 120 * 1000; + Retryer retryer = new Retryer(10, maxRetryDelayMs, statusUpdateDelayMs); + boolean done; + + do { + if (this.closed) { + throw new IOException("this stream is already closed"); + } + + data = this.bufferPool.acquire(blockNumber); + done = this.getInternal(data); + + if (retryer.updateStatus()) { + LOG.warn("waiting to get block: {}", blockNumber); + LOG.info("state = {}", this.toString()); + } + } + while (!done && retryer.continueRetry()); + + if (done) { + return data; + } else { + String message = String.format("Wait failed for get(%d)", blockNumber); + throw new IllegalStateException(message); + } + } + + private boolean getInternal(BufferData data) throws IOException { + Validate.checkNotNull(data, "data"); + + // Opportunistic check without locking. + if (data.stateEqualsOneOf( + BufferData.State.PREFETCHING, + BufferData.State.CACHING, + BufferData.State.DONE)) { + return false; + } + + synchronized (data) { + // Reconfirm state after locking. + if (data.stateEqualsOneOf( + BufferData.State.PREFETCHING, + BufferData.State.CACHING, + BufferData.State.DONE)) { + return false; + } + + int blockNumber = data.getBlockNumber(); + if (data.getState() == BufferData.State.READY) { + BlockOperations.Operation op = this.ops.getPrefetched(blockNumber); + this.ops.end(op); + return true; + } + + data.throwIfStateIncorrect(BufferData.State.BLANK); + this.read(data); + return true; + } + } + + /** + * Releases resources allocated to the given block. + * + * @throws IllegalArgumentException if data is null. + */ + @Override + public void release(BufferData data) { + if (this.closed) { + return; + } + + Validate.checkNotNull(data, "data"); + + BlockOperations.Operation op = this.ops.release(data.getBlockNumber()); + this.bufferPool.release(data); + this.ops.end(op); + } + + @Override + public synchronized void close() { + if (this.closed) { + return; + } + + this.closed = true; + + final BlockOperations.Operation op = this.ops.close(); + + // Cancel any prefetches in progress. + this.cancelPrefetches(); + + cleanupWithLogger(LOG, this.cache); + + this.ops.end(op); + LOG.info(this.ops.getSummary(false)); + + this.bufferPool.close(); + this.bufferPool = null; + } + + /** + * Requests optional prefetching of the given block. + * The block is prefetched only if we can acquire a free buffer. + * + * @throws IllegalArgumentException if blockNumber is negative. + */ + @Override + public void requestPrefetch(int blockNumber) { + Validate.checkNotNegative(blockNumber, "blockNumber"); + + if (this.closed) { + return; + } + + // We initiate a prefetch only if we can acquire a buffer from the shared pool. + BufferData data = this.bufferPool.tryAcquire(blockNumber); + if (data == null) { + return; + } + + // Opportunistic check without locking. + if (!data.stateEqualsOneOf(BufferData.State.BLANK)) { + // The block is ready or being prefetched/cached. + return; + } + + synchronized (data) { + // Reconfirm state after locking. + if (!data.stateEqualsOneOf(BufferData.State.BLANK)) { + // The block is ready or being prefetched/cached. + return; + } + + BlockOperations.Operation op = this.ops.requestPrefetch(blockNumber); + PrefetchTask prefetchTask = new PrefetchTask(data, this, Instant.now()); + Future prefetchFuture = this.futurePool.executeFunction(prefetchTask); + data.setPrefetch(prefetchFuture); + this.ops.end(op); + } + } + + /** + * Requests cancellation of any previously issued prefetch requests. + */ + @Override + public void cancelPrefetches() { + BlockOperations.Operation op = this.ops.cancelPrefetches(); + + for (BufferData data : this.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)) { + this.requestCaching(data); + } + } + + this.ops.end(op); + } + + private void read(BufferData data) throws IOException { + synchronized (data) { + this.readBlock(data, false, BufferData.State.BLANK); + } + } + + private void prefetch(BufferData data, Instant taskQueuedStartTime) throws IOException { + synchronized (data) { + prefetchingStatistics.executorAcquired( + Duration.between(taskQueuedStartTime, Instant.now())); + this.readBlock( + data, + true, + BufferData.State.PREFETCHING, + BufferData.State.CACHING); + } + } + + private void readBlock(BufferData data, boolean isPrefetch, BufferData.State... expectedState) + throws IOException { + + if (this.closed) { + return; + } + + BlockOperations.Operation op = null; + DurationTracker tracker = null; + + synchronized (data) { + try { + if (data.stateEqualsOneOf(BufferData.State.DONE, BufferData.State.READY)) { + // DONE : Block was released, likely due to caching being disabled on slow perf. + // READY : Block was already fetched by another thread. No need to re-read. + return; + } + + data.throwIfStateIncorrect(expectedState); + int blockNumber = data.getBlockNumber(); + + // Prefer reading from cache over reading from network. + if (this.cache.containsBlock(blockNumber)) { + op = this.ops.getCached(blockNumber); + this.cache.get(blockNumber, data.getBuffer()); + data.setReady(expectedState); + return; + } + + if (isPrefetch) { + tracker = prefetchingStatistics.prefetchOperationStarted(); + op = this.ops.prefetch(data.getBlockNumber()); + } else { + op = this.ops.getRead(data.getBlockNumber()); + } + + long offset = this.getBlockData().getStartOffset(data.getBlockNumber()); + int size = this.getBlockData().getSize(data.getBlockNumber()); + ByteBuffer buffer = data.getBuffer(); + buffer.clear(); + this.read(buffer, offset, size); + buffer.flip(); + data.setReady(expectedState); + } catch (Exception e) { + String message = String.format("error during readBlock(%s)", data.getBlockNumber()); + LOG.error(message, e); + + if (isPrefetch && tracker != null) { + tracker.failed(); + } + + this.numReadErrors.incrementAndGet(); + data.setDone(); + throw e; + } finally { + if (op != null) { + this.ops.end(op); + } + + if (isPrefetch) { + prefetchingStatistics.prefetchOperationCompleted(); + if (tracker != null) { + tracker.close(); + } + } + } + } + } + + /** + * Read task that is submitted to the future pool. + */ + private static class PrefetchTask implements Supplier { + private final BufferData data; + private final CachingBlockManager blockManager; + private final Instant taskQueuedStartTime; + + PrefetchTask(BufferData data, CachingBlockManager blockManager, Instant taskQueuedStartTime) { + this.data = data; + this.blockManager = blockManager; + this.taskQueuedStartTime = taskQueuedStartTime; + } + + @Override + public Void get() { + try { + this.blockManager.prefetch(data, taskQueuedStartTime); + } catch (Exception e) { + LOG.error("error during prefetch", e); + } + return null; + } + } + + private static final BufferData.State[] EXPECTED_STATE_AT_CACHING = + new BufferData.State[] { + BufferData.State.PREFETCHING, BufferData.State.READY + }; + + /** + * Requests that the given block should be copied to the local cache. + * The block must not be accessed by the caller after calling this method + * because it will released asynchronously relative to the caller. + * + * @throws IllegalArgumentException if data is null. + */ + @Override + public void requestCaching(BufferData data) { + if (this.closed) { + return; + } + + if (this.cachingDisabled.get()) { + data.setDone(); + return; + } + + Validate.checkNotNull(data, "data"); + + // Opportunistic check without locking. + if (!data.stateEqualsOneOf(EXPECTED_STATE_AT_CACHING)) { + return; + } + + synchronized (data) { + // Reconfirm state after locking. + if (!data.stateEqualsOneOf(EXPECTED_STATE_AT_CACHING)) { + return; + } + + if (this.cache.containsBlock(data.getBlockNumber())) { + data.setDone(); + return; + } + + BufferData.State state = data.getState(); + + BlockOperations.Operation op = this.ops.requestCaching(data.getBlockNumber()); + Future blockFuture; + if (state == BufferData.State.PREFETCHING) { + blockFuture = data.getActionFuture(); + } else { + CompletableFuture cf = new CompletableFuture<>(); + cf.complete(null); + blockFuture = cf; + } + + CachePutTask task = new CachePutTask(data, blockFuture, this, Instant.now()); + Future actionFuture = this.futurePool.executeFunction(task); + data.setCaching(actionFuture); + this.ops.end(op); + } + } + + private void addToCacheAndRelease(BufferData data, Future blockFuture, + Instant taskQueuedStartTime) { + prefetchingStatistics.executorAcquired( + Duration.between(taskQueuedStartTime, Instant.now())); + + if (this.closed) { + return; + } + + if (this.cachingDisabled.get()) { + data.setDone(); + return; + } + + try { + blockFuture.get(TIMEOUT_MINUTES, TimeUnit.MINUTES); + if (data.stateEqualsOneOf(BufferData.State.DONE)) { + // There was an error during prefetch. + return; + } + } catch (Exception e) { + LOG.error("error waiting on blockFuture: {}", data, e); + data.setDone(); + return; + } + + if (this.cachingDisabled.get()) { + data.setDone(); + return; + } + + BlockOperations.Operation op = null; + + synchronized (data) { + try { + if (data.stateEqualsOneOf(BufferData.State.DONE)) { + return; + } + + if (this.cache.containsBlock(data.getBlockNumber())) { + data.setDone(); + return; + } + + op = this.ops.addToCache(data.getBlockNumber()); + ByteBuffer buffer = data.getBuffer().duplicate(); + buffer.rewind(); + this.cachePut(data.getBlockNumber(), buffer); + data.setDone(); + } catch (Exception e) { + this.numCachingErrors.incrementAndGet(); + String message = String.format("error adding block to cache after wait: %s", data); + LOG.error(message, e); + data.setDone(); + } + + if (op != null) { + BlockOperations.End endOp = (BlockOperations.End) this.ops.end(op); + if (endOp.duration() > SLOW_CACHING_THRESHOLD) { + if (!this.cachingDisabled.getAndSet(true)) { + String message = String.format( + "Caching disabled because of slow operation (%.1f sec)", endOp.duration()); + LOG.warn(message); + } + } + } + } + } + + protected BlockCache createCache() { + return new SingleFilePerBlockCache(prefetchingStatistics); + } + + protected void cachePut(int blockNumber, ByteBuffer buffer) throws IOException { + if (this.closed) { + return; + } + + this.cache.put(blockNumber, buffer); + } + + private static class CachePutTask implements Supplier { + private final BufferData data; + + // Block being asynchronously fetched. + private final Future blockFuture; + + // Block manager that manages this block. + private final CachingBlockManager blockManager; + + private final Instant taskQueuedStartTime; + + CachePutTask( + BufferData data, + Future blockFuture, + CachingBlockManager blockManager, + Instant taskQueuedStartTime) { + this.data = data; + this.blockFuture = blockFuture; + this.blockManager = blockManager; + this.taskQueuedStartTime = taskQueuedStartTime; + } + + @Override + public Void get() { + this.blockManager.addToCacheAndRelease(this.data, this.blockFuture, taskQueuedStartTime); + return null; + } + } + + /** + * Number of ByteBuffers available to be acquired. + * + * @return the number of available buffers. + */ + public int numAvailable() { + return this.bufferPool.numAvailable(); + } + + /** + * Number of caching operations completed. + * + * @return the number of cached buffers. + */ + public int numCached() { + return this.cache.size(); + } + + /** + * Number of errors encountered when caching. + * + * @return the number of errors encountered when caching. + */ + public int numCachingErrors() { + return this.numCachingErrors.get(); + } + + /** + * Number of errors encountered when reading. + * + * @return the number of errors encountered when reading. + */ + public int numReadErrors() { + return this.numReadErrors.get(); + } + + BufferData getData(int blockNumber) { + return this.bufferPool.tryAcquire(blockNumber); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + + sb.append("cache("); + sb.append(this.cache.toString()); + sb.append("); "); + + sb.append("pool: "); + sb.append(this.bufferPool.toString()); + + return sb.toString(); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/ExecutorServiceFuturePool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/ExecutorServiceFuturePool.java new file mode 100644 index 0000000000000..932a047c94008 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/ExecutorServiceFuturePool.java @@ -0,0 +1,70 @@ +/* + * 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.common; + +import java.util.Locale; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.function.Supplier; + +/** + * A FuturePool implementation backed by a java.util.concurrent.ExecutorService. + * + * If a piece of work has started, it cannot (currently) be cancelled. + * + * This class is a simplified version of com.twitter:util-core_2.11 + * ExecutorServiceFuturePool designed to avoid depending on that Scala library. + * One problem with using a Scala library is that many downstream projects + * (eg Apache Spark) use Scala, and they might want to use a different version of Scala + * from the version that Hadoop chooses to use. + * + */ +public class ExecutorServiceFuturePool { + private ExecutorService executor; + + public ExecutorServiceFuturePool(ExecutorService executor) { + this.executor = executor; + } + + /** + * @param f function to run in future on executor pool + * @return future + * @throws java.util.concurrent.RejectedExecutionException can be thrown + * @throws NullPointerException if f param is null + */ + public Future executeFunction(final Supplier f) { + return executor.submit(f::get); + } + + /** + * @param r runnable to run in future on executor pool + * @return future + * @throws java.util.concurrent.RejectedExecutionException can be thrown + * @throws NullPointerException if r param is null + */ + @SuppressWarnings("unchecked") + public Future executeRunnable(final Runnable r) { + return (Future) executor.submit(r::run); + } + + public String toString() { + return String.format(Locale.ROOT, "ExecutorServiceFuturePool(executor=%s)", executor); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/FilePosition.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/FilePosition.java new file mode 100644 index 0000000000000..24c46bc4d1f52 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/FilePosition.java @@ -0,0 +1,282 @@ +/* + * 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.common; + +import java.nio.ByteBuffer; + +/** + * Provides functionality related to tracking the position within a file. + * + * The file is accessed through an in memory buffer. The absolute position within + * the file is the sum of start offset of the buffer within the file and the relative + * offset of the current access location within the buffer. + * + * A file is made up of equal sized blocks. The last block may be of a smaller size. + * The size of a buffer associated with this file is typically the same as block size. + */ +public class FilePosition { + // Holds block based information about a file. + private BlockData blockData; + + // Information about the buffer in use. + private BufferData data; + + // Provides access to the underlying file. + private ByteBuffer buffer; + + // Start offset of the buffer relative to the start of a file. + private long bufferStartOffset; + + // Offset where reading starts relative to the start of a file. + private long readStartOffset; + + // Read stats after a seek (mostly for debugging use). + private int numSingleByteReads; + private int numBytesRead; + private int numBufferReads; + + /** + * Constructs an instance of {@link FilePosition}. + * + * @param fileSize size of the associated file. + * @param blockSize size of each block within the file. + * + * @throws IllegalArgumentException if fileSize is negative. + * @throws IllegalArgumentException if blockSize is zero or negative. + */ + public FilePosition(long fileSize, int blockSize) { + Validate.checkNotNegative(fileSize, "fileSize"); + if (fileSize == 0) { + Validate.checkNotNegative(blockSize, "blockSize"); + } else { + Validate.checkPositiveInteger(blockSize, "blockSize"); + } + + this.blockData = new BlockData(fileSize, blockSize); + + // The position is valid only when a valid buffer is associated with this file. + this.invalidate(); + } + + /** + * Associates a buffer with this file. + * + * @param bufferData the buffer associated with this file. + * @param startOffset Start offset of the buffer relative to the start of a file. + * @param readOffset Offset where reading starts relative to the start of a file. + * + * @throws IllegalArgumentException if bufferData is null. + * @throws IllegalArgumentException if startOffset is negative. + * @throws IllegalArgumentException if readOffset is negative. + * @throws IllegalArgumentException if readOffset is outside the range [startOffset, buffer end]. + */ + public void setData(BufferData bufferData, long startOffset, long readOffset) { + Validate.checkNotNull(bufferData, "bufferData"); + Validate.checkNotNegative(startOffset, "startOffset"); + Validate.checkNotNegative(readOffset, "readOffset"); + Validate.checkWithinRange( + readOffset, + "readOffset", + startOffset, + startOffset + bufferData.getBuffer().limit() - 1); + + this.data = bufferData; + this.buffer = bufferData.getBuffer().duplicate(); + this.bufferStartOffset = startOffset; + this.readStartOffset = readOffset; + this.setAbsolute(readOffset); + + this.resetReadStats(); + } + + public ByteBuffer buffer() { + throwIfInvalidBuffer(); + return this.buffer; + } + + public BufferData data() { + throwIfInvalidBuffer(); + return this.data; + } + + /** + * Gets the current absolute position within this file. + * + * @return the current absolute position within this file. + */ + public long absolute() { + throwIfInvalidBuffer(); + return this.bufferStartOffset + this.relative(); + } + + /** + * If the given {@code pos} lies within the current buffer, updates the current position to + * the specified value and returns true; otherwise returns false without changing the position. + * + * @param pos the absolute position to change the current position to if possible. + * @return true if the given current position was updated, false otherwise. + */ + public boolean setAbsolute(long pos) { + if (this.isValid() && this.isWithinCurrentBuffer(pos)) { + int relativePos = (int) (pos - this.bufferStartOffset); + this.buffer.position(relativePos); + return true; + } else { + return false; + } + } + + /** + * Gets the current position within this file relative to the start of the associated buffer. + * + * @return the current position within this file relative to the start of the associated buffer. + */ + public int relative() { + throwIfInvalidBuffer(); + return this.buffer.position(); + } + + /** + * Determines whether the given absolute position lies within the current buffer. + * + * @param pos the position to check. + * @return true if the given absolute position lies within the current buffer, false otherwise. + */ + public boolean isWithinCurrentBuffer(long pos) { + throwIfInvalidBuffer(); + long bufferEndOffset = this.bufferStartOffset + this.buffer.limit() - 1; + return (pos >= this.bufferStartOffset) && (pos <= bufferEndOffset); + } + + /** + * Gets the id of the current block. + * + * @return the id of the current block. + */ + public int blockNumber() { + throwIfInvalidBuffer(); + return this.blockData.getBlockNumber(this.bufferStartOffset); + } + + /** + * Determines whether the current block is the last block in this file. + * + * @return true if the current block is the last block in this file, false otherwise. + */ + public boolean isLastBlock() { + return this.blockData.isLastBlock(this.blockNumber()); + } + + /** + * Determines if the current position is valid. + * + * @return true if the current position is valid, false otherwise. + */ + public boolean isValid() { + return this.buffer != null; + } + + /** + * Marks the current position as invalid. + */ + public void invalidate() { + this.buffer = null; + this.bufferStartOffset = -1; + this.data = null; + } + + /** + * Gets the start of the current block's absolute offset. + * + * @return the start of the current block's absolute offset. + */ + public long bufferStartOffset() { + throwIfInvalidBuffer(); + return this.bufferStartOffset; + } + + /** + * Determines whether the current buffer has been fully read. + * + * @return true if the current buffer has been fully read, false otherwise. + */ + public boolean bufferFullyRead() { + throwIfInvalidBuffer(); + return (this.bufferStartOffset == this.readStartOffset) + && (this.relative() == this.buffer.limit()) + && (this.numBytesRead == this.buffer.limit()); + } + + public void incrementBytesRead(int n) { + this.numBytesRead += n; + if (n == 1) { + this.numSingleByteReads++; + } else { + this.numBufferReads++; + } + } + + public int numBytesRead() { + return this.numBytesRead; + } + + public int numSingleByteReads() { + return this.numSingleByteReads; + } + + public int numBufferReads() { + return this.numBufferReads; + } + + private void resetReadStats() { + numBytesRead = 0; + numSingleByteReads = 0; + numBufferReads = 0; + } + + public String toString() { + StringBuilder sb = new StringBuilder(); + if (this.buffer == null) { + sb.append("currentBuffer = null"); + } else { + int pos = this.buffer.position(); + int val; + if (pos >= this.buffer.limit()) { + val = -1; + } else { + val = this.buffer.get(pos); + } + String currentBufferState = + String.format("%d at pos: %d, lim: %d", val, pos, this.buffer.limit()); + sb.append(String.format( + "block: %d, pos: %d (CBuf: %s)%n", + this.blockNumber(), this.absolute(), + currentBufferState)); + sb.append("\n"); + } + return sb.toString(); + } + + private void throwIfInvalidBuffer() { + if (!this.isValid()) { + Validate.checkState(buffer != null, "'buffer' must not be null"); + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/PrefetchingStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/PrefetchingStatistics.java new file mode 100644 index 0000000000000..b1894f9769685 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/PrefetchingStatistics.java @@ -0,0 +1,67 @@ + /* + * 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.common; + +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 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); +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/ResourcePool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/ResourcePool.java new file mode 100644 index 0000000000000..11affb590f42c --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/ResourcePool.java @@ -0,0 +1,71 @@ +/* + * 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.common; + +import java.io.Closeable; + +/** + * Manages a fixed pool of resources. + * + * Avoids creating a new resource if a previously created instance is already available. + */ +public abstract class ResourcePool implements Closeable { + + /** + * Acquires a resource blocking if necessary until one becomes available. + * + * @return the acquired resource instance. + */ + public abstract T acquire(); + + /** + * Acquires a resource blocking if one is immediately available. Otherwise returns null. + + * @return the acquired resource instance (if immediately available) or null. + */ + public abstract T tryAcquire(); + + /** + * Releases a previously acquired resource. + * + * @param item the resource to release. + */ + public abstract void release(T item); + + @Override + public void close() { + } + + /** + * Derived classes may implement a way to cleanup each item. + * + * @param item the resource to close. + */ + protected void close(T item) { + // Do nothing in this class. Allow overriding classes to take any cleanup action. + } + + /** + * Derived classes must implement a way to create an instance of a resource. + * + * @return the created instance. + */ + protected abstract T createNew(); +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/Retryer.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/Retryer.java new file mode 100644 index 0000000000000..5aea7897008aa --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/Retryer.java @@ -0,0 +1,87 @@ +/* + * 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.common; + +/** + * Provides retry related functionality. + */ +public class Retryer { + // Maximum amount of delay (in ms) before retry fails. + private int maxDelay; + + // Per retry delay (in ms). + private int perRetryDelay; + + // The time interval (in ms) at which status update would be made. + private int statusUpdateInterval; + + // Current delay. + private int delay; + + /** + * Initializes a new instance of the {@code Retryer} class. + * + * @param perRetryDelay per retry delay (in ms). + * @param maxDelay maximum amount of delay (in ms) before retry fails. + * @param statusUpdateInterval time interval (in ms) at which status update would be made. + * + * @throws IllegalArgumentException if perRetryDelay is zero or negative. + * @throws IllegalArgumentException if maxDelay is less than or equal to perRetryDelay. + * @throws IllegalArgumentException if statusUpdateInterval is zero or negative. + */ + public Retryer(int perRetryDelay, int maxDelay, int statusUpdateInterval) { + Validate.checkPositiveInteger(perRetryDelay, "perRetryDelay"); + Validate.checkGreater(maxDelay, "maxDelay", perRetryDelay, "perRetryDelay"); + Validate.checkPositiveInteger(statusUpdateInterval, "statusUpdateInterval"); + + this.perRetryDelay = perRetryDelay; + this.maxDelay = maxDelay; + this.statusUpdateInterval = statusUpdateInterval; + } + + /** + * Returns true if retrying should continue, false otherwise. + * + * @return true if the caller should retry, false otherwise. + */ + public boolean continueRetry() { + if (this.delay >= this.maxDelay) { + return false; + } + + try { + Thread.sleep(this.perRetryDelay); + } catch (InterruptedException e) { + // Ignore the exception as required by the semantic of this class; + } + + this.delay += this.perRetryDelay; + return true; + } + + /** + * Returns true if status update interval has been reached. + * + * @return true if status update interval has been reached. + */ + public boolean updateStatus() { + return (this.delay > 0) && this.delay % this.statusUpdateInterval == 0; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/SingleFilePerBlockCache.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/SingleFilePerBlockCache.java new file mode 100644 index 0000000000000..7252c294bee1a --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/SingleFilePerBlockCache.java @@ -0,0 +1,347 @@ +/* + * 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.common; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.channels.WritableByteChannel; +import java.nio.file.Files; +import java.nio.file.OpenOption; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; +import java.nio.file.attribute.FileAttribute; +import java.nio.file.attribute.PosixFilePermission; +import java.nio.file.attribute.PosixFilePermissions; +import java.util.ArrayList; +import java.util.Collections; +import java.util.EnumSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static java.util.Objects.requireNonNull; + +/** + * Provides functionality necessary for caching blocks of data read from FileSystem. + * Each cache block is stored on the local disk as a separate file. + */ +public class SingleFilePerBlockCache implements BlockCache { + private static final Logger LOG = LoggerFactory.getLogger(SingleFilePerBlockCache.class); + + // Blocks stored in this cache. + private Map blocks = new ConcurrentHashMap<>(); + + // Number of times a block was read from this cache. + // Used for determining cache utilization factor. + private int numGets = 0; + + private boolean closed; + + private final PrefetchingStatistics prefetchingStatistics; + + // Cache entry. + // Each block is stored as a separate file. + private static class Entry { + private final int blockNumber; + private final Path path; + private final int size; + private final long checksum; + + Entry(int blockNumber, Path path, int size, long checksum) { + this.blockNumber = blockNumber; + this.path = path; + this.size = size; + this.checksum = checksum; + } + + @Override + public String toString() { + return String.format( + "([%03d] %s: size = %d, checksum = %d)", + this.blockNumber, this.path, this.size, this.checksum); + } + } + + /** + * Constructs an instance of a {@code SingleFilePerBlockCache}. + * + * @param prefetchingStatistics statistics for this stream. + */ + public SingleFilePerBlockCache(PrefetchingStatistics prefetchingStatistics) { + this.prefetchingStatistics = requireNonNull(prefetchingStatistics); + } + + /** + * Indicates whether the given block is in this cache. + */ + @Override + public boolean containsBlock(int blockNumber) { + return this.blocks.containsKey(blockNumber); + } + + /** + * Gets the blocks in this cache. + */ + @Override + public Iterable blocks() { + return Collections.unmodifiableList(new ArrayList(this.blocks.keySet())); + } + + /** + * Gets the number of blocks in this cache. + */ + @Override + public int size() { + return this.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 { + if (this.closed) { + return; + } + + Validate.checkNotNull(buffer, "buffer"); + + Entry entry = this.getEntry(blockNumber); + buffer.clear(); + this.readFile(entry.path, buffer); + buffer.rewind(); + + validateEntry(entry, buffer); + } + + protected int readFile(Path path, ByteBuffer buffer) throws IOException { + int numBytesRead = 0; + int numBytes; + FileChannel channel = FileChannel.open(path, StandardOpenOption.READ); + while ((numBytes = channel.read(buffer)) > 0) { + numBytesRead += numBytes; + } + buffer.limit(buffer.position()); + channel.close(); + return numBytesRead; + } + + private Entry getEntry(int blockNumber) { + Validate.checkNotNegative(blockNumber, "blockNumber"); + + Entry entry = this.blocks.get(blockNumber); + if (entry == null) { + throw new IllegalStateException(String.format("block %d not found in cache", blockNumber)); + } + this.numGets++; + return entry; + } + + /** + * Puts the given block in this cache. + * + * @throws IllegalArgumentException if buffer is null. + * @throws IllegalArgumentException if buffer.limit() is zero or negative. + */ + @Override + public void put(int blockNumber, ByteBuffer buffer) throws IOException { + if (this.closed) { + return; + } + + Validate.checkNotNull(buffer, "buffer"); + + if (this.blocks.containsKey(blockNumber)) { + Entry entry = this.blocks.get(blockNumber); + validateEntry(entry, buffer); + return; + } + + Validate.checkPositiveInteger(buffer.limit(), "buffer.limit()"); + + Path blockFilePath = getCacheFilePath(); + long size = Files.size(blockFilePath); + if (size != 0) { + String message = + String.format("[%d] temp file already has data. %s (%d)", + blockNumber, blockFilePath, size); + throw new IllegalStateException(message); + } + + this.writeFile(blockFilePath, buffer); + this.prefetchingStatistics.blockAddedToFileCache(); + long checksum = BufferData.getChecksum(buffer); + Entry entry = new Entry(blockNumber, blockFilePath, buffer.limit(), checksum); + this.blocks.put(blockNumber, entry); + } + + private static final Set CREATE_OPTIONS = + EnumSet.of(StandardOpenOption.WRITE, + StandardOpenOption.CREATE, + StandardOpenOption.TRUNCATE_EXISTING); + + protected void writeFile(Path path, ByteBuffer buffer) throws IOException { + buffer.rewind(); + WritableByteChannel writeChannel = Files.newByteChannel(path, CREATE_OPTIONS); + while (buffer.hasRemaining()) { + writeChannel.write(buffer); + } + writeChannel.close(); + } + + protected Path getCacheFilePath() throws IOException { + return getTempFilePath(); + } + + @Override + public void close() throws IOException { + if (this.closed) { + return; + } + + this.closed = true; + + LOG.info(this.getStats()); + int numFilesDeleted = 0; + + for (Entry entry : this.blocks.values()) { + try { + Files.deleteIfExists(entry.path); + this.prefetchingStatistics.blockRemovedFromFileCache(); + numFilesDeleted++; + } catch (IOException e) { + // Ignore while closing so that we can delete as many cache files as possible. + } + } + + if (numFilesDeleted > 0) { + LOG.info("Deleted {} cache files", numFilesDeleted); + } + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("stats: "); + sb.append(getStats()); + sb.append(", blocks:["); + sb.append(this.getIntList(this.blocks())); + sb.append("]"); + return sb.toString(); + } + + 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); + } + + 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); + } + } + + /** + * Produces a human readable list of blocks for the purpose of logging. + * This method minimizes the length of returned list by converting + * a contiguous list of blocks into a range. + * for example, + * 1, 3, 4, 5, 6, 8 becomes 1, 3~6, 8 + */ + private String getIntList(Iterable nums) { + List numList = new ArrayList<>(); + List numbers = new ArrayList(); + for (Integer n : nums) { + numbers.add(n); + } + Collections.sort(numbers); + + int index = 0; + while (index < numbers.size()) { + int start = numbers.get(index); + int prev = start; + int end = start; + while ((++index < numbers.size()) && ((end = numbers.get(index)) == prev + 1)) { + prev = end; + } + + if (start == prev) { + numList.add(Integer.toString(start)); + } else { + numList.add(String.format("%d~%d", start, prev)); + } + } + + return String.join(", ", numList); + } + + private String getStats() { + StringBuilder sb = new StringBuilder(); + sb.append(String.format( + "#entries = %d, #gets = %d", + this.blocks.size(), this.numGets)); + return sb.toString(); + } + + private static final String CACHE_FILE_PREFIX = "fs-cache-"; + + public static boolean isCacheSpaceAvailable(long fileSize) { + try { + Path cacheFilePath = getTempFilePath(); + 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; + } + } + + // The suffix (file extension) of each serialized index file. + private static final String BINARY_FILE_SUFFIX = ".bin"; + + // File attributes attached to any intermediate temporary file created during index creation. + private static final FileAttribute> TEMP_FILE_ATTRS = + PosixFilePermissions.asFileAttribute(EnumSet.of(PosixFilePermission.OWNER_READ, + PosixFilePermission.OWNER_WRITE)); + + private static Path getTempFilePath() throws IOException { + return Files.createTempFile( + CACHE_FILE_PREFIX, + BINARY_FILE_SUFFIX, + TEMP_FILE_ATTRS + ); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/Validate.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/Validate.java new file mode 100644 index 0000000000000..18e77fd1d0edd --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/Validate.java @@ -0,0 +1,405 @@ +/* + * 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.common; + +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Collection; + +/** + * A superset of Validate class in Apache commons lang3. + * + * It provides consistent message strings for frequently encountered checks. + * That simplifies callers because they have to supply only the name of the argument + * that failed a check instead of having to supply the entire message. + */ +public final class Validate { + private Validate() {} + + /** + * Validates that the given reference argument is not null. + * + * @param obj the argument reference to validate. + * @param argName the name of the argument being validated. + */ + public static void checkNotNull(Object obj, String argName) { + checkArgument(obj != null, "'%s' must not be null.", argName); + } + + /** + * Validates that the given integer argument is not zero or negative. + * + * @param value the argument value to validate + * @param argName the name of the argument being validated. + */ + public static void checkPositiveInteger(long value, String argName) { + checkArgument(value > 0, "'%s' must be a positive integer.", argName); + } + + /** + * Validates that the given integer argument is not negative. + * + * @param value the argument value to validate + * @param argName the name of the argument being validated. + */ + public static void checkNotNegative(long value, String argName) { + checkArgument(value >= 0, "'%s' must not be negative.", argName); + } + + /* + * Validates that the expression (that checks a required field is present) is true. + * + * @param isPresent indicates whether the given argument is present. + * @param argName the name of the argument being validated. + */ + public static void checkRequired(boolean isPresent, String argName) { + checkArgument(isPresent, "'%s' is required.", argName); + } + + /** + * Validates that the expression (that checks a field is valid) is true. + * + * @param isValid indicates whether the given argument is valid. + * @param argName the name of the argument being validated. + */ + public static void checkValid(boolean isValid, String argName) { + checkArgument(isValid, "'%s' is invalid.", argName); + } + + /** + * Validates that the expression (that checks a field is valid) is true. + * + * @param isValid indicates whether the given argument is valid. + * @param argName the name of the argument being validated. + * @param validValues the list of values that are allowed. + */ + public static void checkValid(boolean isValid, String argName, String validValues) { + checkArgument(isValid, "'%s' is invalid. Valid values are: %s.", argName, validValues); + } + + /** + * Validates that the given string is not null and has non-zero length. + * + * @param arg the argument reference to validate. + * @param argName the name of the argument being validated. + */ + public static void checkNotNullAndNotEmpty(String arg, String argName) { + Validate.checkNotNull(arg, argName); + Validate.checkArgument( + arg.length() > 0, + "'%s' must not be empty.", + argName); + } + + /** + * Validates that the given array is not null and has at least one element. + * + * @param the type of array's elements. + * @param array the argument reference to validate. + * @param argName the name of the argument being validated. + */ + public static void checkNotNullAndNotEmpty(T[] array, String argName) { + Validate.checkNotNull(array, argName); + checkNotEmpty(array.length, argName); + } + + /** + * Validates that the given array is not null and has at least one element. + * + * @param array the argument reference to validate. + * @param argName the name of the argument being validated. + */ + public static void checkNotNullAndNotEmpty(byte[] array, String argName) { + Validate.checkNotNull(array, argName); + checkNotEmpty(array.length, argName); + } + + /** + * Validates that the given array is not null and has at least one element. + * + * @param array the argument reference to validate. + * @param argName the name of the argument being validated. + */ + public static void checkNotNullAndNotEmpty(short[] array, String argName) { + Validate.checkNotNull(array, argName); + checkNotEmpty(array.length, argName); + } + + /** + * Validates that the given array is not null and has at least one element. + * + * @param array the argument reference to validate. + * @param argName the name of the argument being validated. + */ + public static void checkNotNullAndNotEmpty(int[] array, String argName) { + Validate.checkNotNull(array, argName); + checkNotEmpty(array.length, argName); + } + + /** + * Validates that the given array is not null and has at least one element. + * + * @param array the argument reference to validate. + * @param argName the name of the argument being validated. + */ + public static void checkNotNullAndNotEmpty(long[] array, String argName) { + Validate.checkNotNull(array, argName); + checkNotEmpty(array.length, argName); + } + + /** + * Validates that the given buffer is not null and has non-zero capacity. + * + * @param the type of iterable's elements. + * @param iter the argument reference to validate. + * @param argName the name of the argument being validated. + */ + public static void checkNotNullAndNotEmpty(Iterable iter, String argName) { + Validate.checkNotNull(iter, argName); + int minNumElements = iter.iterator().hasNext() ? 1 : 0; + checkNotEmpty(minNumElements, argName); + } + + /** + * Validates that the given set is not null and has an exact number of items. + * + * @param the type of collection's elements. + * @param collection the argument reference to validate. + * @param numElements the expected number of elements in the collection. + * @param argName the name of the argument being validated. + */ + public static void checkNotNullAndNumberOfElements( + Collection collection, int numElements, String argName) { + Validate.checkNotNull(collection, argName); + checkArgument( + collection.size() == numElements, + "Number of elements in '%s' must be exactly %s, %s given.", + argName, + numElements, + collection.size() + ); + } + + /** + * Validates that the given two values are equal. + * + * @param value1 the first value to check. + * @param value1Name the name of the first argument. + * @param value2 the second value to check. + * @param value2Name the name of the second argument. + */ + public static void checkValuesEqual( + long value1, + String value1Name, + long value2, + String value2Name) { + checkArgument( + value1 == value2, + "'%s' (%s) must equal '%s' (%s).", + value1Name, + value1, + value2Name, + value2); + } + + /** + * Validates that the first value is an integer multiple of the second value. + * + * @param value1 the first value to check. + * @param value1Name the name of the first argument. + * @param value2 the second value to check. + * @param value2Name the name of the second argument. + */ + public static void checkIntegerMultiple( + long value1, + String value1Name, + long value2, + String value2Name) { + checkArgument( + (value1 % value2) == 0, + "'%s' (%s) must be an integer multiple of '%s' (%s).", + value1Name, + value1, + value2Name, + value2); + } + + /** + * Validates that the first value is greater than the second value. + * + * @param value1 the first value to check. + * @param value1Name the name of the first argument. + * @param value2 the second value to check. + * @param value2Name the name of the second argument. + */ + public static void checkGreater( + long value1, + String value1Name, + long value2, + String value2Name) { + checkArgument( + value1 > value2, + "'%s' (%s) must be greater than '%s' (%s).", + value1Name, + value1, + value2Name, + value2); + } + + /** + * Validates that the first value is greater than or equal to the second value. + * + * @param value1 the first value to check. + * @param value1Name the name of the first argument. + * @param value2 the second value to check. + * @param value2Name the name of the second argument. + */ + public static void checkGreaterOrEqual( + long value1, + String value1Name, + long value2, + String value2Name) { + checkArgument( + value1 >= value2, + "'%s' (%s) must be greater than or equal to '%s' (%s).", + value1Name, + value1, + value2Name, + value2); + } + + /** + * Validates that the first value is less than or equal to the second value. + * + * @param value1 the first value to check. + * @param value1Name the name of the first argument. + * @param value2 the second value to check. + * @param value2Name the name of the second argument. + */ + public static void checkLessOrEqual( + long value1, + String value1Name, + long value2, + String value2Name) { + checkArgument( + value1 <= value2, + "'%s' (%s) must be less than or equal to '%s' (%s).", + value1Name, + value1, + value2Name, + value2); + } + + /** + * Validates that the given value is within the given range of values. + * + * @param value the value to check. + * @param valueName the name of the argument. + * @param minValueInclusive inclusive lower limit for the value. + * @param maxValueInclusive inclusive upper limit for the value. + */ + public static void checkWithinRange( + long value, + String valueName, + long minValueInclusive, + long maxValueInclusive) { + checkArgument( + (value >= minValueInclusive) && (value <= maxValueInclusive), + "'%s' (%s) must be within the range [%s, %s].", + valueName, + value, + minValueInclusive, + maxValueInclusive); + } + + /** + * Validates that the given value is within the given range of values. + * + * @param value the value to check. + * @param valueName the name of the argument. + * @param minValueInclusive inclusive lower limit for the value. + * @param maxValueInclusive inclusive upper limit for the value. + */ + public static void checkWithinRange( + double value, + String valueName, + double minValueInclusive, + double maxValueInclusive) { + checkArgument( + (value >= minValueInclusive) && (value <= maxValueInclusive), + "'%s' (%s) must be within the range [%s, %s].", + valueName, + value, + minValueInclusive, + maxValueInclusive); + } + + /** + * Validates that the given path exists. + * + * @param path the path to check. + * @param argName the name of the argument being validated. + */ + public static void checkPathExists(Path path, String argName) { + checkNotNull(path, argName); + checkArgument(Files.exists(path), "Path %s (%s) does not exist.", argName, path); + } + + /** + * Validates that the given path exists and is a directory. + * + * @param path the path to check. + * @param argName the name of the argument being validated. + */ + public static void checkPathExistsAsDir(Path path, String argName) { + checkPathExists(path, argName); + checkArgument( + Files.isDirectory(path), + "Path %s (%s) must point to a directory.", + argName, + path); + } + + /** + * Validates that the given path exists and is a file. + * + * @param path the path to check. + * @param argName the name of the argument being validated. + */ + public static void checkPathExistsAsFile(Path path, String argName) { + checkPathExists(path, argName); + checkArgument(Files.isRegularFile(path), "Path %s (%s) must point to a file.", argName, path); + } + + public static void checkArgument(boolean expression, String format, Object... args) { + org.apache.commons.lang3.Validate.isTrue(expression, format, args); + } + + public static void checkState(boolean expression, String format, Object... args) { + org.apache.commons.lang3.Validate.validState(expression, format, args); + } + + private static void checkNotEmpty(int arraySize, String argName) { + Validate.checkArgument( + arraySize > 0, + "'%s' must have at least one element.", + argName); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/package-info.java new file mode 100644 index 0000000000000..f6f7b6f9e01bc --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/package-info.java @@ -0,0 +1,27 @@ +/* + * 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. + */ + +/** + * block caching for use in object store clients. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.fs.common; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file 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 764a6adaca27d..b0f510055c91b 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 @@ -1203,4 +1203,29 @@ private Constants() { * Default maximum read size in bytes during vectored reads : {@value}. */ public static final int DEFAULT_AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE = 1253376; //1M + + /** + * Controls whether the prefetching input stream is enabled. + */ + public static final String PREFETCH_ENABLED_KEY = "fs.s3a.prefetch.enabled"; + + /** + * Default option as to whether the prefetching input stream is enabled. + */ + public static final boolean PREFETCH_ENABLED_DEFAULT = false; + + // If the default values are used, each file opened for reading will consume + // 64 MB of heap space (8 blocks x 8 MB each). + + /** + * The size of a single prefetched block in number of bytes. + */ + 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. + */ + 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 c49c368bbbe68..1299f03381ed0 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 @@ -79,6 +79,7 @@ import com.amazonaws.services.s3.transfer.model.UploadResult; import com.amazonaws.event.ProgressListener; +import org.apache.hadoop.fs.common.ExecutorServiceFuturePool; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -172,6 +173,7 @@ import org.apache.hadoop.fs.s3a.commit.PutTracker; import org.apache.hadoop.fs.s3a.commit.MagicCommitIntegration; import org.apache.hadoop.fs.s3a.impl.ChangeTracker; +import org.apache.hadoop.fs.s3a.read.S3PrefetchingInputStream; import org.apache.hadoop.fs.s3a.select.SelectBinding; import org.apache.hadoop.fs.s3a.select.SelectConstants; import org.apache.hadoop.fs.s3a.s3guard.S3Guard; @@ -289,6 +291,19 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, private TransferManager transfers; private ExecutorService boundedThreadPool; private ThreadPoolExecutor unboundedThreadPool; + + // S3 reads are prefetched asynchronously using this future pool. + private ExecutorServiceFuturePool futurePool; + + // If true, the prefetching input stream is used for reads. + private boolean prefetchEnabled; + + // Size in bytes of a single prefetch block. + private int prefetchBlockSize; + + // Size of prefetch queue (in number of blocks). + private int prefetchBlockCount; + private int executorCapacity; private long multiPartThreshold; public static final Logger LOG = LoggerFactory.getLogger(S3AFileSystem.class); @@ -489,6 +504,12 @@ public void initialize(URI name, Configuration originalConf) longBytesOption(conf, FS_S3A_BLOCK_SIZE, DEFAULT_BLOCKSIZE, 1); enableMultiObjectsDelete = conf.getBoolean(ENABLE_MULTI_DELETE, true); + this.prefetchEnabled = conf.getBoolean(PREFETCH_ENABLED_KEY, false); + this.prefetchBlockSize = intOption( + conf, PREFETCH_BLOCK_SIZE_KEY, PREFETCH_BLOCK_DEFAULT_SIZE, PREFETCH_BLOCK_DEFAULT_SIZE); + this.prefetchBlockCount = + intOption(conf, PREFETCH_BLOCK_COUNT_KEY, PREFETCH_BLOCK_DEFAULT_COUNT, 1); + initThreadPools(conf); int listVersion = conf.getInt(LIST_VERSION, DEFAULT_LIST_VERSION); @@ -594,11 +615,17 @@ public void initialize(URI name, Configuration originalConf) // amazon client exception: stop all services then throw the translation cleanupWithLogger(LOG, span); stopAllServices(); + if (this.futurePool != null) { + this.futurePool = null; + } throw translateException("initializing ", new Path(name), e); } catch (IOException | RuntimeException e) { // other exceptions: stop the services. cleanupWithLogger(LOG, span); stopAllServices(); + if (this.futurePool != null) { + this.futurePool = null; + } throw e; } } @@ -719,9 +746,11 @@ private void initThreadPools(Configuration conf) { MAX_TOTAL_TASKS, DEFAULT_MAX_TOTAL_TASKS, 1); long keepAliveTime = longOption(conf, KEEPALIVE_TIME, DEFAULT_KEEPALIVE_TIME, 0); + int numPrefetchThreads = this.prefetchEnabled ? this.prefetchBlockCount : 0; + boundedThreadPool = BlockingThreadPoolExecutorService.newInstance( maxThreads, - maxThreads + totalTasks, + maxThreads + totalTasks + numPrefetchThreads, keepAliveTime, TimeUnit.SECONDS, name + "-bounded"); unboundedThreadPool = new ThreadPoolExecutor( @@ -733,6 +762,9 @@ private void initThreadPools(Configuration conf) { unboundedThreadPool.allowCoreThreadTimeOut(true); executorCapacity = intOption(conf, EXECUTOR_CAPACITY, DEFAULT_EXECUTOR_CAPACITY, 1); + if (this.prefetchEnabled) { + this.futurePool = new ExecutorServiceFuturePool(boundedThreadPool); + } } /** @@ -1486,13 +1518,23 @@ private FSDataInputStream executeOpen( auditSpan); fileInformation.applyOptions(readContext); LOG.debug("Opening '{}'", readContext); - return new FSDataInputStream( - new S3AInputStream( - readContext.build(), - createObjectAttributes(path, fileStatus), - createInputStreamCallbacks(auditSpan), + + if (this.prefetchEnabled) { + return new FSDataInputStream( + new S3PrefetchingInputStream( + readContext.build(), + createObjectAttributes(path, fileStatus), + createInputStreamCallbacks(auditSpan), + inputStreamStats)); + } else { + return new FSDataInputStream( + new S3AInputStream( + readContext.build(), + createObjectAttributes(path, fileStatus), + createInputStreamCallbacks(auditSpan), inputStreamStats, unboundedThreadPool)); + } } /** @@ -1578,7 +1620,11 @@ protected S3AReadOpContext createReadContext( statisticsContext, fileStatus, vectoredIOContext, - IOStatisticsContext.getCurrentIOStatisticsContext().getAggregator()) + IOStatisticsContext.getCurrentIOStatisticsContext().getAggregator(), + fileStatus, + futurePool, + prefetchBlockSize, + prefetchBlockCount) .withAuditSpan(auditSpan); openFileHelper.applyDefaultOptions(roc); return roc.build(); @@ -3676,12 +3722,12 @@ public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path src, } protected CopyFromLocalOperation.CopyFromLocalOperationCallbacks - createCopyFromLocalCallbacks() throws IOException { + createCopyFromLocalCallbacks() throws IOException { LocalFileSystem local = getLocal(getConf()); return new CopyFromLocalCallbacksImpl(local); } - protected class CopyFromLocalCallbacksImpl implements + protected final class CopyFromLocalCallbacksImpl implements CopyFromLocalOperation.CopyFromLocalOperationCallbacks { private final LocalFileSystem local; 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 0a49f564616c9..46568ec2a8d3d 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 @@ -845,11 +845,19 @@ private InputStreamStatistics( StreamStatisticNames.STREAM_READ_VECTORED_OPERATIONS, StreamStatisticNames.STREAM_READ_VECTORED_READ_BYTES_DISCARDED, StreamStatisticNames.STREAM_READ_VERSION_MISMATCHES) - .withGauges(STREAM_READ_GAUGE_INPUT_POLICY) + .withGauges(STREAM_READ_GAUGE_INPUT_POLICY, + STREAM_READ_BLOCKS_IN_FILE_CACHE.getSymbol(), + STREAM_READ_ACTIVE_PREFETCH_OPERATIONS.getSymbol(), + STREAM_READ_ACTIVE_MEMORY_IN_USE.getSymbol() + ) .withDurationTracking(ACTION_HTTP_GET_REQUEST, + ACTION_EXECUTOR_ACQUIRED, StoreStatisticNames.ACTION_FILE_OPENED, StreamStatisticNames.STREAM_READ_REMOTE_STREAM_ABORTED, - StreamStatisticNames.STREAM_READ_REMOTE_STREAM_DRAINED) + StreamStatisticNames.STREAM_READ_REMOTE_STREAM_DRAINED, + StreamStatisticNames.STREAM_READ_PREFETCH_OPERATIONS, + StreamStatisticNames.STREAM_READ_REMOTE_BLOCK_READ, + StreamStatisticNames.STREAM_READ_BLOCK_ACQUIRE_AND_READ) .build(); setIOStatistics(st); aborted = st.getCounterReference( @@ -918,6 +926,18 @@ private long increment(String name, long value) { return incCounter(name, value); } + /** + * Increment the Statistic gauge and the local IOStatistics + * equivalent. + * @param statistic statistic + * @param v value. + * @return local IOStatistic value + */ + private long incAllGauges(Statistic statistic, long v) { + incrementGauge(statistic, v); + return incGauge(statistic.getSymbol(), v); + } + /** * {@inheritDoc}. * Increments the number of seek operations, @@ -1046,6 +1066,12 @@ public void readVectoredBytesDiscarded(int discarded) { bytesDiscardedInVectoredIO.addAndGet(discarded); } + @Override + public void executorAcquired(Duration timeInQueue) { + // update the duration fields in the IOStatistics. + localIOStatistics().addTimedOperation(ACTION_EXECUTOR_ACQUIRED, timeInQueue); + } + /** * {@code close()} merges the stream statistics into the filesystem's * instrumentation instance. @@ -1310,6 +1336,37 @@ public DurationTracker initiateInnerStreamClose(final boolean abort) { ? StreamStatisticNames.STREAM_READ_REMOTE_STREAM_ABORTED : StreamStatisticNames.STREAM_READ_REMOTE_STREAM_DRAINED); } + + @Override + public DurationTracker prefetchOperationStarted() { + incAllGauges(STREAM_READ_ACTIVE_PREFETCH_OPERATIONS, 1); + return trackDuration(StreamStatisticNames.STREAM_READ_PREFETCH_OPERATIONS); + } + + @Override + public void blockAddedToFileCache() { + incAllGauges(STREAM_READ_BLOCKS_IN_FILE_CACHE, 1); + } + + @Override + public void blockRemovedFromFileCache() { + incAllGauges(STREAM_READ_BLOCKS_IN_FILE_CACHE, -1); + } + + @Override + public void prefetchOperationCompleted() { + incAllGauges(STREAM_READ_ACTIVE_PREFETCH_OPERATIONS, -1); + } + + @Override + public void memoryAllocated(int size) { + incAllGauges(STREAM_READ_ACTIVE_MEMORY_IN_USE, size); + } + + @Override + public void memoryFreed(int size) { + incAllGauges(STREAM_READ_ACTIVE_MEMORY_IN_USE, -size); + } } /** 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 bbd86ef5ac222..afbd91a3694b8 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 @@ -21,6 +21,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.common.ExecutorServiceFuturePool; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; import org.apache.hadoop.fs.statistics.IOStatisticsAggregator; @@ -74,6 +75,15 @@ public class S3AReadOpContext extends S3AOpContext { /** Thread-level IOStatistics aggregator. **/ private final IOStatisticsAggregator ioStatisticsAggregator; + // 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 of prefetch queue (in number of blocks). + private final int prefetchBlockCount; + /** * Instantiate. * @param path path of read @@ -83,6 +93,9 @@ public class S3AReadOpContext extends S3AOpContext { * @param dstFileStatus target file status * @param vectoredIOContext context for vectored read operation. * @param ioStatisticsAggregator IOStatistics aggregator for each thread. + * @param futurePool the ExecutorServiceFuturePool instance used by async prefetches. + * @param prefetchBlockSize the size (in number of bytes) of each prefetched block. + * @param prefetchBlockCount maximum number of prefetched blocks. */ public S3AReadOpContext( final Path path, @@ -91,12 +104,23 @@ public S3AReadOpContext( S3AStatisticsContext instrumentation, FileStatus dstFileStatus, VectoredIOContext vectoredIOContext, - IOStatisticsAggregator ioStatisticsAggregator) { + IOStatisticsAggregator ioStatisticsAggregator, + ExecutorServiceFuturePool futurePool, + int prefetchBlockSize, + int prefetchBlockCount) { + super(invoker, stats, instrumentation, dstFileStatus); this.path = requireNonNull(path); this.vectoredIOContext = requireNonNull(vectoredIOContext, "vectoredIOContext"); this.ioStatisticsAggregator = ioStatisticsAggregator; + this.futurePool = futurePool; + Preconditions.checkArgument( + prefetchBlockSize > 0, "invalid prefetchBlockSize %d", prefetchBlockSize); + this.prefetchBlockSize = prefetchBlockSize; + Preconditions.checkArgument( + prefetchBlockCount > 0, "invalid prefetchBlockCount %d", prefetchBlockCount); + this.prefetchBlockCount = prefetchBlockCount; } /** @@ -232,6 +256,33 @@ public IOStatisticsAggregator getIOStatisticsAggregator() { return ioStatisticsAggregator; } + /** + * Gets the {@code ExecutorServiceFuturePool} used for asynchronous prefetches. + * + * @return the {@code ExecutorServiceFuturePool} used for asynchronous prefetches. + */ + public ExecutorServiceFuturePool getFuturePool() { + return this.futurePool; + } + + /** + * Gets the size in bytes of a single prefetch block. + * + * @return the size in bytes of a single prefetch block. + */ + public int getPrefetchBlockSize() { + return this.prefetchBlockSize; + } + + /** + * Gets the size of prefetch queue (in number of blocks). + * + * @return the size of prefetch queue (in number of blocks). + */ + public int getPrefetchBlockCount() { + return this.prefetchBlockCount; + } + @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 dfe9fdf2d8d37..66ff4f9720799 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 @@ -361,6 +361,18 @@ public enum Statistic { StreamStatisticNames.STREAM_READ_TOTAL_BYTES, "Total count of bytes read from an input stream", TYPE_COUNTER), + STREAM_READ_BLOCKS_IN_FILE_CACHE( + StreamStatisticNames.STREAM_READ_BLOCKS_IN_FILE_CACHE, + "Gauge of blocks in disk cache", + TYPE_GAUGE), + STREAM_READ_ACTIVE_PREFETCH_OPERATIONS( + StreamStatisticNames.STREAM_READ_ACTIVE_PREFETCH_OPERATIONS, + "Gauge of active prefetches", + TYPE_GAUGE), + STREAM_READ_ACTIVE_MEMORY_IN_USE( + StreamStatisticNames.STREAM_READ_ACTIVE_MEMORY_IN_USE, + "Gauge of active memory in use", + TYPE_GAUGE), /* Stream Write statistics */ diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3BlockManager.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3BlockManager.java new file mode 100644 index 0000000000000..f7c47ef7925e9 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3BlockManager.java @@ -0,0 +1,77 @@ +/* + * 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.read; + +import java.io.IOException; +import java.nio.ByteBuffer; + +import org.apache.hadoop.fs.common.BlockData; +import org.apache.hadoop.fs.common.BlockManager; +import org.apache.hadoop.fs.common.Validate; + +/** + * Provides read access to S3 file one block at a time. + * + * A naive implementation of a {@code BlockManager} that provides no prefetching or caching. + * Useful baseline for comparing performance difference against {@code S3CachingBlockManager}. + */ +public class S3BlockManager extends BlockManager { + + /** + * Reader that reads from S3 file. + */ + private final S3Reader reader; + + /** + * Constructs an instance of {@code S3BlockManager}. + * + * @param reader a reader that reads from S3 file. + * @param blockData information about each block of the S3 file. + * + * @throws IllegalArgumentException if reader is null. + * @throws IllegalArgumentException if blockData is null. + */ + public S3BlockManager(S3Reader reader, BlockData blockData) { + super(blockData); + + Validate.checkNotNull(reader, "reader"); + + this.reader = reader; + } + + /** + * Reads into the given {@code buffer} {@code size} bytes from the underlying file + * starting at {@code startOffset}. + * + * @param buffer the buffer to read data in to. + * @param startOffset the offset at which reading starts. + * @param size the number bytes to read. + * @return number of bytes read. + */ + @Override + public int read(ByteBuffer buffer, long startOffset, int size) throws IOException { + return this.reader.read(buffer, startOffset, size); + } + + @Override + public void close() { + this.reader.close(); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3CachingBlockManager.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3CachingBlockManager.java new file mode 100644 index 0000000000000..1c058087f3130 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3CachingBlockManager.java @@ -0,0 +1,93 @@ +/* + * 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.read; + +import java.io.IOException; +import java.nio.ByteBuffer; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.common.BlockData; +import org.apache.hadoop.fs.common.CachingBlockManager; +import org.apache.hadoop.fs.common.ExecutorServiceFuturePool; +import org.apache.hadoop.fs.common.Validate; +import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; + +/** + * Provides access to S3 file one block at a time. + */ +public class S3CachingBlockManager extends CachingBlockManager { + private static final Logger LOG = LoggerFactory.getLogger(S3CachingBlockManager.class); + + /** + * Reader that reads from S3 file. + */ + private final S3Reader reader; + + /** + * Constructs an instance of a {@code S3CachingBlockManager}. + * + * @param futurePool asynchronous tasks are performed in this pool. + * @param reader reader that reads from S3 file. + * @param blockData information about each block of the S3 file. + * @param bufferPoolSize size of the in-memory cache in terms of number of blocks. + * @param streamStatistics statistics for this stream. + * + * @throws IllegalArgumentException if reader is null. + */ + public S3CachingBlockManager( + ExecutorServiceFuturePool futurePool, + S3Reader reader, + BlockData blockData, + int bufferPoolSize, + S3AInputStreamStatistics streamStatistics) { + super(futurePool, blockData, bufferPoolSize, streamStatistics); + + Validate.checkNotNull(reader, "reader"); + + this.reader = reader; + } + + protected S3Reader getReader() { + return this.reader; + } + + /** + * Reads into the given {@code buffer} {@code size} bytes from the underlying file + * starting at {@code startOffset}. + * + * @param buffer the buffer to read data in to. + * @param startOffset the offset at which reading starts. + * @param size the number bytes to read. + * @return number of bytes read. + */ + @Override + public int read(ByteBuffer buffer, long startOffset, int size) throws IOException { + return this.reader.read(buffer, startOffset, size); + } + + @Override + public synchronized void close() { + this.reader.close(); + + super.close(); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3CachingInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3CachingInputStream.java new file mode 100644 index 0000000000000..b00119ac4e1ad --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3CachingInputStream.java @@ -0,0 +1,207 @@ +/* + * 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.read; + +import java.io.IOException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.common.BlockData; +import org.apache.hadoop.fs.common.BlockManager; +import org.apache.hadoop.fs.common.BufferData; +import org.apache.hadoop.fs.common.ExecutorServiceFuturePool; +import org.apache.hadoop.fs.s3a.S3AInputStream; +import org.apache.hadoop.fs.s3a.S3AReadOpContext; +import org.apache.hadoop.fs.s3a.S3ObjectAttributes; +import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; + +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BLOCK_ACQUIRE_AND_READ; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration; + +/** + * Provides an {@code InputStream} that allows reading from an S3 file. + * Prefetched blocks are cached to local disk if a seek away from the + * current block is issued. + */ +public class S3CachingInputStream extends S3InputStream { + private static final Logger LOG = LoggerFactory.getLogger(S3CachingInputStream.class); + + /** + * Number of blocks queued for prefching. + */ + private final int numBlocksToPrefetch; + + private final BlockManager blockManager; + + /** + * Initializes a new instance of the {@code S3CachingInputStream} class. + * + * @param context read-specific operation context. + * @param s3Attributes attributes of the S3 object being read. + * @param client callbacks used for interacting with the underlying S3 client. + * @param streamStatistics statistics for this stream. + * + * @throws IllegalArgumentException if context is null. + * @throws IllegalArgumentException if s3Attributes is null. + * @throws IllegalArgumentException if client is null. + */ + public S3CachingInputStream( + S3AReadOpContext context, + S3ObjectAttributes s3Attributes, + S3AInputStream.InputStreamCallbacks client, + S3AInputStreamStatistics streamStatistics) { + super(context, s3Attributes, client, streamStatistics); + + this.numBlocksToPrefetch = this.getContext().getPrefetchBlockCount(); + int bufferPoolSize = this.numBlocksToPrefetch + 1; + this.blockManager = this.createBlockManager( + this.getContext().getFuturePool(), + this.getReader(), + this.getBlockData(), + bufferPoolSize); + int fileSize = (int) s3Attributes.getLen(); + LOG.debug("Created caching input stream for {} (size = {})", this.getName(), fileSize); + } + + /** + * Moves the current read position so that the next read will occur at {@code pos}. + * + * @param pos the next read will take place at this position. + * + * @throws IllegalArgumentException if pos is outside of the range [0, file size]. + */ + @Override + public void seek(long pos) throws IOException { + this.throwIfClosed(); + this.throwIfInvalidSeek(pos); + + // The call to setAbsolute() returns true if the target position is valid and + // within the current block. Therefore, no additional work is needed when we get back true. + if (!this.getFilePosition().setAbsolute(pos)) { + LOG.info("seek({})", getOffsetStr(pos)); + // We could be here in two cases: + // -- the target position is invalid: + // We ignore this case here as the next read will return an error. + // -- it is valid but outside of the current block. + if (this.getFilePosition().isValid()) { + // There are two cases to consider: + // -- the seek was issued after this buffer was fully read. + // In this case, it is very unlikely that this buffer will be needed again; + // therefore we release the buffer without caching. + // -- if we are jumping out of the buffer before reading it completely then + // we will likely need this buffer again (as observed empirically for Parquet) + // therefore we issue an async request to cache this buffer. + if (!this.getFilePosition().bufferFullyRead()) { + this.blockManager.requestCaching(this.getFilePosition().data()); + } else { + this.blockManager.release(this.getFilePosition().data()); + } + this.getFilePosition().invalidate(); + this.blockManager.cancelPrefetches(); + } + this.setSeekTargetPos(pos); + } + } + + @Override + public void close() throws IOException { + // Close the BlockManager first, cancelling active prefetches, + // deleting cached files and freeing memory used by buffer pool. + this.blockManager.close(); + super.close(); + LOG.info("closed: {}", this.getName()); + } + + @Override + protected boolean ensureCurrentBuffer() throws IOException { + if (this.isClosed()) { + return false; + } + + if (this.getFilePosition().isValid() && this.getFilePosition().buffer().hasRemaining()) { + return true; + } + + long readPos; + int prefetchCount; + + if (this.getFilePosition().isValid()) { + // A sequential read results in a prefetch. + readPos = this.getFilePosition().absolute(); + prefetchCount = this.numBlocksToPrefetch; + } else { + // A seek invalidates the current position. + // We prefetch only 1 block immediately after a seek operation. + readPos = this.getSeekTargetPos(); + prefetchCount = 1; + } + + if (!this.getBlockData().isValidOffset(readPos)) { + return false; + } + + if (this.getFilePosition().isValid()) { + if (this.getFilePosition().bufferFullyRead()) { + this.blockManager.release(this.getFilePosition().data()); + } else { + this.blockManager.requestCaching(this.getFilePosition().data()); + } + } + + int toBlockNumber = this.getBlockData().getBlockNumber(readPos); + long startOffset = this.getBlockData().getStartOffset(toBlockNumber); + + for (int i = 1; i <= prefetchCount; i++) { + int b = toBlockNumber + i; + if (b < this.getBlockData().getNumBlocks()) { + this.blockManager.requestPrefetch(b); + } + } + + BufferData data = invokeTrackingDuration( + this.getS3AStreamStatistics().trackDuration(STREAM_READ_BLOCK_ACQUIRE_AND_READ), + () -> this.blockManager.get(toBlockNumber)); + + this.getFilePosition().setData(data, startOffset, readPos); + return true; + } + + @Override + public String toString() { + if (this.isClosed()) { + return "closed"; + } + + StringBuilder sb = new StringBuilder(); + sb.append(String.format("fpos = (%s)%n", this.getFilePosition())); + sb.append(this.blockManager.toString()); + return sb.toString(); + } + + protected BlockManager createBlockManager( + ExecutorServiceFuturePool futurePool, + S3Reader reader, + BlockData blockData, + int bufferPoolSize) { + return new S3CachingBlockManager(futurePool, reader, blockData, bufferPoolSize, + this.getS3AStreamStatistics()); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3File.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3File.java new file mode 100644 index 0000000000000..ac22976e5bc52 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3File.java @@ -0,0 +1,318 @@ +/* + * 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.read; + + +import java.io.IOException; +import java.io.InputStream; +import java.util.IdentityHashMap; +import java.util.Map; + +import com.amazonaws.services.s3.model.GetObjectRequest; +import com.amazonaws.services.s3.model.S3Object; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.common.Validate; +import org.apache.hadoop.fs.s3a.Invoker; +import org.apache.hadoop.fs.s3a.S3AInputStream; +import org.apache.hadoop.fs.s3a.S3AReadOpContext; +import org.apache.hadoop.fs.s3a.S3ObjectAttributes; +import org.apache.hadoop.fs.s3a.impl.ChangeTracker; +import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; +import org.apache.hadoop.fs.statistics.DurationTracker; + +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration; +import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; + +/** + * Encapsulates low level interactions with S3 object on AWS. + */ +public class S3File { + private static final Logger LOG = LoggerFactory.getLogger(S3File.class); + + /** + * Read-specific operation context. + */ + private final S3AReadOpContext context; + + /** + * S3 object attributes. + */ + private final S3ObjectAttributes s3Attributes; + + /** + * Callbacks used for interacting with the underlying S3 client. + */ + private final S3AInputStream.InputStreamCallbacks client; + + /** + * Used for reporting input stream access statistics. + */ + private final S3AInputStreamStatistics streamStatistics; + + /** + * Enforces change tracking related policies. + */ + private final ChangeTracker changeTracker; + + /** + * Maps a stream returned by openForRead() to the associated S3 object. + * That allows us to close the object when closing the stream. + */ + private Map s3Objects; + + /** + * uri of the object being read. + */ + private final String uri; + + /** + * size of a buffer to create when draining the stream. + */ + private static final int DRAIN_BUFFER_SIZE = 16384; + + /** + * Initializes a new instance of the {@code S3File} class. + * + * @param context read-specific operation context. + * @param s3Attributes attributes of the S3 object being read. + * @param client callbacks used for interacting with the underlying S3 client. + * @param streamStatistics statistics about stream access. + * @param changeTracker helps enforce change tracking policy. + * + * @throws IllegalArgumentException if context is null. + * @throws IllegalArgumentException if s3Attributes is null. + * @throws IllegalArgumentException if client is null. + * @throws IllegalArgumentException if streamStatistics is null. + * @throws IllegalArgumentException if changeTracker is null. + */ + public S3File( + S3AReadOpContext context, + S3ObjectAttributes s3Attributes, + S3AInputStream.InputStreamCallbacks client, + S3AInputStreamStatistics streamStatistics, + ChangeTracker changeTracker) { + + Validate.checkNotNull(context, "context"); + Validate.checkNotNull(s3Attributes, "s3Attributes"); + Validate.checkNotNull(client, "client"); + Validate.checkNotNull(streamStatistics, "streamStatistics"); + Validate.checkNotNull(changeTracker, "changeTracker"); + + this.context = context; + this.s3Attributes = s3Attributes; + this.client = client; + this.streamStatistics = streamStatistics; + this.changeTracker = changeTracker; + this.s3Objects = new IdentityHashMap<>(); + this.uri = this.getPath(); + } + + /** + * Gets an instance of {@code Invoker} for interacting with S3 API. + * + * @return an instance of {@code Invoker} for interacting with S3 API. + */ + public Invoker getReadInvoker() { + return this.context.getReadInvoker(); + } + + /** + * Gets an instance of {@code S3AInputStreamStatistics} used for reporting access metrics. + * + * @return an instance of {@code S3AInputStreamStatistics} used for reporting access metrics. + */ + public S3AInputStreamStatistics getStatistics() { + return this.streamStatistics; + } + + /** + * Gets the path of this file. + * + * @return the path of this file. + */ + public String getPath() { + return getPath(this.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()); + } + + /** + * Gets the size of this file. + * Its value is cached once obtained from AWS. + * + * @return the size of this file. + */ + public long size() { + return this.s3Attributes.getLen(); + } + + /** + * Opens a section of the file for reading. + * + * @param offset Start offset (0 based) of the section to read. + * @param size Size of the section to read. + * @return an {@code InputStream} corresponding to the given section of this file. + * + * @throws IOException if there is an error opening this file section for reading. + * @throws IllegalArgumentException if offset is negative. + * @throws IllegalArgumentException if offset is greater than or equal to file size. + * @throws IllegalArgumentException if size is greater than the remaining bytes. + */ + public InputStream openForRead(long offset, int size) throws IOException { + Validate.checkNotNegative(offset, "offset"); + Validate.checkLessOrEqual(offset, "offset", size(), "size()"); + Validate.checkLessOrEqual(size, "size", size() - offset, "size() - offset"); + + streamStatistics.streamOpened(); + final GetObjectRequest request = client.newGetRequest(this.s3Attributes.getKey()) + .withRange(offset, offset + size - 1); + this.changeTracker.maybeApplyConstraint(request); + + String operation = String.format( + "%s %s at %d", S3AInputStream.OPERATION_OPEN, uri, offset); + DurationTracker tracker = streamStatistics.initiateGetRequest(); + S3Object object = null; + + try { + object = Invoker.once(operation, uri, () -> client.getObject(request)); + } catch(IOException e) { + tracker.failed(); + throw e; + } finally { + tracker.close(); + } + + changeTracker.processResponse(object, operation, offset); + InputStream stream = object.getObjectContent(); + synchronized (this.s3Objects) { + this.s3Objects.put(stream, object); + } + + return stream; + } + + void close(InputStream inputStream, int numRemainingBytes) { + S3Object obj; + synchronized (this.s3Objects) { + obj = this.s3Objects.get(inputStream); + if (obj == null) { + throw new IllegalArgumentException("inputStream not found"); + } + this.s3Objects.remove(inputStream); + } + + if (numRemainingBytes <= this.context.getAsyncDrainThreshold()) { + // don't bother with async io. + drain(false, "close() operation", numRemainingBytes, obj, inputStream); + } else { + LOG.debug("initiating asynchronous drain of {} bytes", numRemainingBytes); + // schedule an async drain/abort with references to the fields so they + // can be reused + client.submit(() -> drain(false, "close() operation", numRemainingBytes, obj, inputStream)); + } + } + + /** + * drain the stream. This method is intended to be + * used directly or asynchronously, and measures the + * duration of the operation in the stream statistics. + * + * @param shouldAbort force an abort; used if explicitly requested. + * @param reason reason for stream being closed; used in messages + * @param remaining remaining bytes + * @param requestObject http request object; + * @param inputStream stream to close. + * @return was the stream aborted? + */ + private boolean drain( + final boolean shouldAbort, + final String reason, + final long remaining, + final S3Object requestObject, + final InputStream inputStream) { + + try { + return invokeTrackingDuration(streamStatistics.initiateInnerStreamClose(shouldAbort), + () -> drainOrAbortHttpStream(shouldAbort, reason, remaining, requestObject, inputStream)); + } catch (IOException e) { + // this is only here because invokeTrackingDuration() has it in its + // signature + return shouldAbort; + } + } + + /** + * Drain or abort the inner stream. + * Exceptions are swallowed. + * If a close() is attempted and fails, the operation escalates to + * an abort. + * + * @param shouldAbort force an abort; used if explicitly requested. + * @param reason reason for stream being closed; used in messages + * @param remaining remaining bytes + * @param requestObject http request object + * @param inputStream stream to close. + * @return was the stream aborted? + */ + private boolean drainOrAbortHttpStream( + boolean shouldAbort, + final String reason, + final long remaining, + final S3Object requestObject, + final InputStream inputStream) { + + if (!shouldAbort && remaining > 0) { + try { + long drained = 0; + byte[] buffer = new byte[DRAIN_BUFFER_SIZE]; + while (true) { + final int count = inputStream.read(buffer); + if (count < 0) { + // no more data is left + break; + } + drained += count; + } + LOG.debug("Drained stream of {} bytes", drained); + } catch (Exception e) { + // exception escalates to an abort + LOG.debug("When closing {} stream for {}, will abort the stream", uri, reason, e); + shouldAbort = true; + } + } + cleanupWithLogger(LOG, inputStream); + cleanupWithLogger(LOG, requestObject); + streamStatistics.streamClose(shouldAbort, remaining); + + LOG.debug("Stream {} {}: {}; remaining={}", uri, (shouldAbort ? "aborted" : "closed"), reason, + remaining); + return shouldAbort; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3InMemoryInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3InMemoryInputStream.java new file mode 100644 index 0000000000000..c97cf38669b29 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3InMemoryInputStream.java @@ -0,0 +1,98 @@ +/* + * 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.read; + +import java.io.IOException; +import java.nio.ByteBuffer; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.common.BufferData; +import org.apache.hadoop.fs.s3a.S3AInputStream; +import org.apache.hadoop.fs.s3a.S3AReadOpContext; +import org.apache.hadoop.fs.s3a.S3ObjectAttributes; +import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; + +/** + * 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. + */ +public class S3InMemoryInputStream extends S3InputStream { + private static final Logger LOG = LoggerFactory.getLogger(S3InMemoryInputStream.class); + + private ByteBuffer buffer; + + /** + * Initializes a new instance of the {@code S3InMemoryInputStream} class. + * + * @param context read-specific operation context. + * @param s3Attributes attributes of the S3 object being read. + * @param client callbacks used for interacting with the underlying S3 client. + * @param streamStatistics statistics for this stream. + * + * @throws IllegalArgumentException if context is null. + * @throws IllegalArgumentException if s3Attributes is null. + * @throws IllegalArgumentException if client is null. + */ + public S3InMemoryInputStream( + S3AReadOpContext context, + S3ObjectAttributes s3Attributes, + S3AInputStream.InputStreamCallbacks client, + S3AInputStreamStatistics streamStatistics) { + super(context, s3Attributes, client, streamStatistics); + int fileSize = (int) s3Attributes.getLen(); + this.buffer = ByteBuffer.allocate(fileSize); + LOG.debug("Created in-memory input stream for {} (size = {})", this.getName(), fileSize); + } + + /** + * 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. + * It returns false on reaching the end of the stream. + * + * @return true if at least one such buffer is available for reading, false otherwise. + */ + @Override + protected boolean ensureCurrentBuffer() throws IOException { + if (this.isClosed()) { + return false; + } + + if (this.getBlockData().getFileSize() == 0) { + return false; + } + + if (!this.getFilePosition().isValid()) { + this.buffer.clear(); + int numBytesRead = this.getReader().read(buffer, 0, this.buffer.capacity()); + if (numBytesRead <= 0) { + return false; + } + BufferData data = new BufferData(0, buffer); + this.getFilePosition().setData(data, 0, this.getSeekTargetPos()); + } + + return this.getFilePosition().buffer().hasRemaining(); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3InputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3InputStream.java new file mode 100644 index 0000000000000..bbc9008c73ae4 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3InputStream.java @@ -0,0 +1,459 @@ +/* + * 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.read; + +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.CanSetReadahead; +import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.fs.StreamCapabilities; +import org.apache.hadoop.fs.common.BlockData; +import org.apache.hadoop.fs.common.FilePosition; +import org.apache.hadoop.fs.common.Validate; +import org.apache.hadoop.fs.s3a.S3AInputPolicy; +import org.apache.hadoop.fs.s3a.S3AInputStream; +import org.apache.hadoop.fs.s3a.S3AReadOpContext; +import org.apache.hadoop.fs.s3a.S3ObjectAttributes; +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.IOStatisticsSource; + +import static java.util.Objects.requireNonNull; + +/** + * Provides an {@link InputStream} that allows reading from an S3 file. + */ +public abstract class S3InputStream + extends InputStream + implements CanSetReadahead, StreamCapabilities, IOStatisticsSource { + + private static final Logger LOG = LoggerFactory.getLogger(S3InputStream.class); + + // The S3 file read by this instance. + private S3File s3File; + + // Reading of S3 file takes place through this reader. + private S3Reader reader; + + // Name of this stream. Used only for logging. + private final String name; + + // Indicates whether the stream has been closed. + private volatile boolean closed; + + // Current position within the file. + private FilePosition fpos; + + // The target of the most recent seek operation. + private long seekTargetPos; + + // Information about each block of the mapped S3 file. + private BlockData blockData; + + // Read-specific operation context. + private S3AReadOpContext context; + + // Attributes of the S3 object being read. + private S3ObjectAttributes s3Attributes; + + // Callbacks used for interacting with the underlying S3 client. + private S3AInputStream.InputStreamCallbacks client; + + // Used for reporting input stream access statistics. + private final S3AInputStreamStatistics streamStatistics; + + private S3AInputPolicy inputPolicy; + private final ChangeTracker changeTracker; + private final IOStatistics ioStatistics; + + /** + * Initializes a new instance of the {@code S3InputStream} class. + * + * @param context read-specific operation context. + * @param s3Attributes attributes of the S3 object being read. + * @param client callbacks used for interacting with the underlying S3 client. + * @param streamStatistics statistics for this stream. + * + * @throws IllegalArgumentException if context is null. + * @throws IllegalArgumentException if s3Attributes is null. + * @throws IllegalArgumentException if client is null. + */ + public S3InputStream( + S3AReadOpContext context, + S3ObjectAttributes s3Attributes, + S3AInputStream.InputStreamCallbacks client, + S3AInputStreamStatistics streamStatistics) { + + this.context = requireNonNull(context); + this.s3Attributes = requireNonNull(s3Attributes); + this.client = requireNonNull(client); + this.streamStatistics = requireNonNull(streamStatistics); + this.ioStatistics = streamStatistics.getIOStatistics(); + this.name = S3File.getPath(s3Attributes); + this.changeTracker = new ChangeTracker( + this.name, + context.getChangeDetectionPolicy(), + this.streamStatistics.getChangeTrackerStatistics(), + s3Attributes); + + setInputPolicy(context.getInputPolicy()); + setReadahead(context.getReadahead()); + + long fileSize = s3Attributes.getLen(); + int bufferSize = context.getPrefetchBlockSize(); + + this.blockData = new BlockData(fileSize, bufferSize); + this.fpos = new FilePosition(fileSize, bufferSize); + this.s3File = this.getS3File(); + this.reader = new S3Reader(this.s3File); + + this.seekTargetPos = 0; + } + + /** + * Gets the internal IO statistics. + * + * @return the internal IO statistics. + */ + @Override + public IOStatistics getIOStatistics() { + return this.ioStatistics; + } + + /** + * Access the input stream statistics. + * This is for internal testing and may be removed without warning. + * @return the statistics for this input stream + */ + @InterfaceAudience.Private + @InterfaceStability.Unstable + public S3AInputStreamStatistics getS3AStreamStatistics() { + return this.streamStatistics; + } + + /** + * Sets the number of bytes to read ahead each time. + * + * @param readahead the number of bytes to read ahead each time.. + */ + @Override + public synchronized void setReadahead(Long readahead) { + // We support read head by prefetching therefore we ignore the supplied value. + if (readahead != null) { + Validate.checkNotNegative(readahead, "readahead"); + } + } + + /** + * Indicates whether the given {@code capability} is supported by this stream. + * + * @param capability the capability to check. + * @return true if the given {@code capability} is supported by this stream, false otherwise. + */ + @Override + public boolean hasCapability(String capability) { + return capability.equalsIgnoreCase(StreamCapabilities.IOSTATISTICS) + || capability.equalsIgnoreCase(StreamCapabilities.READAHEAD); + } + + /** + * Set/update the input policy of the stream. + * This updates the stream statistics. + * @param inputPolicy new input policy. + */ + private void setInputPolicy(S3AInputPolicy inputPolicy) { + this.inputPolicy = inputPolicy; + streamStatistics.inputPolicySet(inputPolicy.ordinal()); + } + + /** + * Returns the number of bytes that can read from this stream without blocking. + */ + @Override + public int available() throws IOException { + this.throwIfClosed(); + + if (!ensureCurrentBuffer()) { + return 0; + } + + return this.fpos.buffer().remaining(); + } + + /** + * Gets the current position. + * + * @return the current position. + * @throws IOException if there is an IO error during this operation. + */ + public long getPos() throws IOException { + this.throwIfClosed(); + + if (this.fpos.isValid()) { + return this.fpos.absolute(); + } else { + return this.seekTargetPos; + } + } + + /** + * Moves the current read position so that the next read will occur at {@code pos}. + * + * @param pos the absolute position to seek to. + * @throws IOException if there an error during this operation. + * + * @throws IllegalArgumentException if pos is outside of the range [0, file size]. + */ + public void seek(long pos) throws IOException { + this.throwIfClosed(); + this.throwIfInvalidSeek(pos); + + if (!this.fpos.setAbsolute(pos)) { + this.fpos.invalidate(); + this.seekTargetPos = pos; + } + } + + /** + * 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. + * It returns false on reaching the end of the stream. + * + * @return true if at least one such buffer is available for reading, false otherwise. + * @throws IOException if there is an IO error during this operation. + */ + protected abstract boolean ensureCurrentBuffer() throws IOException; + + @Override + public int read() throws IOException { + this.throwIfClosed(); + + if (this.s3File.size() == 0 || this.seekTargetPos >= this.s3File.size()) { + return -1; + } + + if (!ensureCurrentBuffer()) { + return -1; + } + + this.incrementBytesRead(1); + + return this.fpos.buffer().get() & 0xff; + } + + /** + * Reads bytes from this stream and copies them into + * the given {@code buffer} starting at the beginning (offset 0). + * Returns the number of bytes actually copied in to the given buffer. + * + * @param buffer the buffer to copy data into. + * @return the number of bytes actually copied in to the given buffer. + * @throws IOException if there is an IO error during this operation. + */ + @Override + public int read(byte[] buffer) throws IOException { + return this.read(buffer, 0, buffer.length); + } + + /** + * Reads up to {@code len} bytes from this stream and copies them into + * the given {@code buffer} starting at the given {@code offset}. + * Returns the number of bytes actually copied in to the given buffer. + * + * @param buffer the buffer to copy data into. + * @param offset data is copied starting at this offset. + * @param len max number of bytes to copy. + * @return the number of bytes actually copied in to the given buffer. + * @throws IOException if there is an IO error during this operation. + */ + @Override + public int read(byte[] buffer, int offset, int len) throws IOException { + this.throwIfClosed(); + + if (len == 0) { + return 0; + } + + if (this.s3File.size() == 0 || this.seekTargetPos >= this.s3File.size()) { + return -1; + } + + if (!ensureCurrentBuffer()) { + return -1; + } + + int numBytesRead = 0; + int numBytesRemaining = len; + + while (numBytesRemaining > 0) { + if (!ensureCurrentBuffer()) { + break; + } + + ByteBuffer buf = this.fpos.buffer(); + int bytesToRead = Math.min(numBytesRemaining, buf.remaining()); + buf.get(buffer, offset, bytesToRead); + this.incrementBytesRead(bytesToRead); + offset += bytesToRead; + numBytesRemaining -= bytesToRead; + numBytesRead += bytesToRead; + } + + return numBytesRead; + } + + protected S3File getFile() { + return this.s3File; + } + + protected S3Reader getReader() { + return this.reader; + } + + protected S3ObjectAttributes getS3ObjectAttributes() { + return this.s3Attributes; + } + + protected FilePosition getFilePosition() { + return this.fpos; + } + + protected String getName() { + return this.name; + } + + protected boolean isClosed() { + return this.closed; + } + + protected long getSeekTargetPos() { + return this.seekTargetPos; + } + + protected void setSeekTargetPos(long pos) { + this.seekTargetPos = pos; + } + + protected BlockData getBlockData() { + return this.blockData; + } + + protected S3AReadOpContext getContext() { + return this.context; + } + + private void incrementBytesRead(int bytesRead) { + if (bytesRead > 0) { + this.streamStatistics.bytesRead(bytesRead); + if (this.getContext().getStats() != null) { + this.getContext().getStats().incrementBytesRead(bytesRead); + } + this.fpos.incrementBytesRead(bytesRead); + } + } + + protected S3File getS3File() { + return new S3File( + this.context, + this.s3Attributes, + this.client, + this.streamStatistics, + this.changeTracker + ); + } + + protected String getOffsetStr(long offset) { + int blockNumber = -1; + + if (this.blockData.isValidOffset(offset)) { + blockNumber = this.blockData.getBlockNumber(offset); + } + + return String.format("%d:%d", blockNumber, offset); + } + + /** + * Closes this stream and releases all acquired resources. + * + * @throws IOException if there is an IO error during this operation. + */ + @Override + public void close() throws IOException { + if (this.closed) { + return; + } + this.closed = true; + + this.blockData = null; + this.reader.close(); + this.reader = null; + this.s3File = null; + this.fpos.invalidate(); + try { + this.client.close(); + } finally { + this.streamStatistics.close(); + } + this.client = null; + } + + @Override + public boolean markSupported() { + return false; + } + + protected void throwIfClosed() throws IOException { + if (this.closed) { + throw new IOException(this.name + ": " + FSExceptionMessages.STREAM_IS_CLOSED); + } + } + + protected void throwIfInvalidSeek(long pos) throws EOFException { + if (pos < 0) { + throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK + " " + pos); + } + } + + // Unsupported functions. + + @Override + public void mark(int readlimit) { + throw new UnsupportedOperationException("mark not supported"); + } + + @Override + public void reset() { + throw new UnsupportedOperationException("reset not supported"); + } + + @Override + public long skip(long n) { + throw new UnsupportedOperationException("skip not supported"); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3PrefetchingInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3PrefetchingInputStream.java new file mode 100644 index 0000000000000..66dd7c2a37547 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3PrefetchingInputStream.java @@ -0,0 +1,243 @@ +/* + * 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.read; + +import java.io.IOException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.CanSetReadahead; +import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.fs.FSInputStream; +import org.apache.hadoop.fs.StreamCapabilities; +import org.apache.hadoop.fs.common.Validate; +import org.apache.hadoop.fs.s3a.S3AInputStream; +import org.apache.hadoop.fs.s3a.S3AReadOpContext; +import org.apache.hadoop.fs.s3a.S3ObjectAttributes; +import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; + +/** + * Enhanced {@code InputStream} for reading from S3. + * + * This implementation provides improved read throughput by asynchronously prefetching + * blocks of configurable size from the underlying S3 file. + */ +public class S3PrefetchingInputStream + extends FSInputStream + implements CanSetReadahead, StreamCapabilities, IOStatisticsSource { + private static final Logger LOG = LoggerFactory.getLogger(S3PrefetchingInputStream.class); + + // Underlying input stream used for reading S3 file. + private S3InputStream inputStream; + + /** + * Initializes a new instance of the {@code S3PrefetchingInputStream} class. + * + * @param context read-specific operation context. + * @param s3Attributes attributes of the S3 object being read. + * @param client callbacks used for interacting with the underlying S3 client. + * @param streamStatistics statistics for this stream. + * + * @throws IllegalArgumentException if context is null. + * @throws IllegalArgumentException if s3Attributes is null. + * @throws IllegalArgumentException if client is null. + */ + public S3PrefetchingInputStream( + S3AReadOpContext context, + S3ObjectAttributes s3Attributes, + S3AInputStream.InputStreamCallbacks client, + S3AInputStreamStatistics streamStatistics) { + + Validate.checkNotNull(context, "context"); + Validate.checkNotNull(s3Attributes, "s3Attributes"); + Validate.checkNotNullAndNotEmpty(s3Attributes.getBucket(), "s3Attributes.getBucket()"); + Validate.checkNotNullAndNotEmpty(s3Attributes.getKey(), "s3Attributes.getKey()"); + Validate.checkNotNegative(s3Attributes.getLen(), "s3Attributes.getLen()"); + Validate.checkNotNull(client, "client"); + Validate.checkNotNull(streamStatistics, "streamStatistics"); + + long fileSize = s3Attributes.getLen(); + if (fileSize <= context.getPrefetchBlockSize()) { + this.inputStream = new S3InMemoryInputStream(context, s3Attributes, client, streamStatistics); + } else { + this.inputStream = new S3CachingInputStream(context, s3Attributes, client, streamStatistics); + } + } + + /** + * Returns the number of bytes available for reading without blocking. + * + * @return the number of bytes available for reading without blocking. + * @throws IOException if there is an IO error during this operation. + */ + @Override + public synchronized int available() throws IOException { + this.throwIfClosed(); + return this.inputStream.available(); + } + + /** + * Gets the current position. + * + * @return the current position. + * @throws IOException if there is an IO error during this operation. + */ + @Override + public synchronized long getPos() throws IOException { + return this.isClosed() ? 0 : this.inputStream.getPos(); + } + + /** + * Reads and returns one byte from this stream. + * + * @return the next byte from this stream. + * @throws IOException if there is an IO error during this operation. + */ + @Override + public synchronized int read() throws IOException { + this.throwIfClosed(); + return this.inputStream.read(); + } + + /** + * Reads up to {@code len} bytes from this stream and copies them into + * the given {@code buffer} starting at the given {@code offset}. + * Returns the number of bytes actually copied in to the given buffer. + * + * @param buffer the buffer to copy data into. + * @param offset data is copied starting at this offset. + * @param len max number of bytes to copy. + * @return the number of bytes actually copied in to the given buffer. + * @throws IOException if there is an IO error during this operation. + */ + @Override + public synchronized int read(byte[] buffer, int offset, int len) throws IOException { + this.throwIfClosed(); + return this.inputStream.read(buffer, offset, len); + } + + /** + * Closes this stream and releases all acquired resources. + * + * @throws IOException if there is an IO error during this operation. + */ + @Override + public synchronized void close() throws IOException { + if (this.inputStream != null) { + this.inputStream.close(); + this.inputStream = null; + super.close(); + } + } + + /** + * Updates internal data such that the next read will take place at the given {@code pos}. + * + * @param pos new read position. + * @throws IOException if there is an IO error during this operation. + */ + @Override + public synchronized void seek(long pos) throws IOException { + this.throwIfClosed(); + this.inputStream.seek(pos); + } + + /** + * Sets the number of bytes to read ahead each time. + * + * @param readahead the number of bytes to read ahead each time.. + */ + @Override + public synchronized void setReadahead(Long readahead) { + if (!this.isClosed()) { + this.inputStream.setReadahead(readahead); + } + } + + /** + * Indicates whether the given {@code capability} is supported by this stream. + * + * @param capability the capability to check. + * @return true if the given {@code capability} is supported by this stream, false otherwise. + */ + @Override + public boolean hasCapability(String capability) { + if (!this.isClosed()) { + return this.inputStream.hasCapability(capability); + } + + return false; + } + + /** + * Access the input stream statistics. + * This is for internal testing and may be removed without warning. + * @return the statistics for this input stream + */ + @InterfaceAudience.Private + @InterfaceStability.Unstable + public S3AInputStreamStatistics getS3AStreamStatistics() { + if (this.isClosed()) { + return null; + } + return this.inputStream.getS3AStreamStatistics(); + } + + /** + * Gets the internal IO statistics. + * + * @return the internal IO statistics. + */ + @Override + public IOStatistics getIOStatistics() { + if (this.isClosed()) { + return null; + } + return this.inputStream.getIOStatistics(); + } + + protected boolean isClosed() { + return this.inputStream == null; + } + + protected void throwIfClosed() throws IOException { + if (this.isClosed()) { + throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED); + } + } + + // Unsupported functions. + + @Override + public boolean seekToNewSource(long targetPos) throws IOException { + this.throwIfClosed(); + return false; + } + + @Override + public boolean markSupported() { + return false; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3Reader.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3Reader.java new file mode 100644 index 0000000000000..19ab4f6961d8d --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3Reader.java @@ -0,0 +1,169 @@ +/* + * 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.read; + +import java.io.Closeable; +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.net.SocketTimeoutException; +import java.nio.ByteBuffer; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.common.Validate; +import org.apache.hadoop.fs.s3a.Invoker; +import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; + +import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_REMOTE_BLOCK_READ; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfOperation; + +/** + * Provides functionality to read S3 file one block at a time. + */ +public class S3Reader implements Closeable { + private static final Logger LOG = LoggerFactory.getLogger(S3Reader.class); + + // We read from the underlying input stream in blocks of this size. + private static final int READ_BUFFER_SIZE = 64 * 1024; + + // The S3 file to read. + private final S3File s3File; + + // Set to true by close(). + private volatile boolean closed; + + private final S3AInputStreamStatistics streamStatistics; + + /** + * Constructs an instance of {@link S3Reader}. + * + * @param s3File The S3 file to read. + * + * @throws IllegalArgumentException if s3File is null. + */ + public S3Reader(S3File s3File) { + Validate.checkNotNull(s3File, "s3File"); + + this.s3File = s3File; + this.streamStatistics = this.s3File.getStatistics(); + } + + /** + * Stars reading at {@code offset} and reads upto {@code size} bytes into {@code buffer}. + * + * @param buffer the buffer into which data is returned + * @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. + * @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. + */ + public int read(ByteBuffer buffer, long offset, int size) throws IOException { + Validate.checkNotNull(buffer, "buffer"); + Validate.checkWithinRange(offset, "offset", 0, this.s3File.size()); + Validate.checkPositiveInteger(size, "size"); + + if (this.closed) { + return -1; + } + + int reqSize = (int) Math.min(size, this.s3File.size() - offset); + return readOneBlockWithRetries(buffer, offset, reqSize); + } + + @Override + public void close() { + this.closed = true; + } + + private int readOneBlockWithRetries(ByteBuffer buffer, long offset, int size) + throws IOException { + + this.streamStatistics.readOperationStarted(offset, size); + Invoker invoker = this.s3File.getReadInvoker(); + + int invokerResponse = invoker.retry("read", this.s3File.getPath(), true, + trackDurationOfOperation(streamStatistics, STREAM_READ_REMOTE_BLOCK_READ, () -> { + try { + this.readOneBlock(buffer, offset, size); + } catch (EOFException e) { + // the base implementation swallows EOFs. + return -1; + } catch (SocketTimeoutException e) { + throw e; + } catch (IOException e) { + this.s3File.getStatistics().readException(); + throw e; + } + return 0; + })); + + int numBytesRead = buffer.position(); + buffer.limit(numBytesRead); + this.s3File.getStatistics().readOperationCompleted(size, numBytesRead); + + if (invokerResponse < 0) { + return invokerResponse; + } else { + return numBytesRead; + } + } + + private void readOneBlock(ByteBuffer buffer, long offset, int size) throws IOException { + int readSize = Math.min(size, buffer.remaining()); + if (readSize == 0) { + return; + } + + InputStream inputStream = s3File.openForRead(offset, readSize); + int numRemainingBytes = readSize; + byte[] bytes = new byte[READ_BUFFER_SIZE]; + + int numBytesToRead; + int numBytes; + + try { + do { + numBytesToRead = Math.min(READ_BUFFER_SIZE, numRemainingBytes); + numBytes = inputStream.read(bytes, 0, numBytesToRead); + if (numBytes < 0) { + String message = String.format( + "Unexpected end of stream: buffer[%d], readSize = %d, numRemainingBytes = %d", + buffer.capacity(), readSize, numRemainingBytes); + throw new EOFException(message); + } + + if (numBytes > 0) { + buffer.put(bytes, 0, numBytes); + numRemainingBytes -= numBytes; + } + } + while (!this.closed && (numRemainingBytes > 0)); + } finally { + s3File.close(inputStream, numRemainingBytes); + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/package-info.java new file mode 100644 index 0000000000000..b255537b40a02 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/package-info.java @@ -0,0 +1,28 @@ +/* + * 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. + */ + +/** + * High performance s3 input stream which reads in + * blocks and can cache blocks in the local filesystem. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.fs.s3a.read; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AInputStreamStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AInputStreamStatistics.java index 41a8f253159ff..355e550d875bd 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AInputStreamStatistics.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AInputStreamStatistics.java @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.s3a.statistics; +import org.apache.hadoop.fs.common.PrefetchingStatistics; import org.apache.hadoop.fs.statistics.DurationTracker; /** @@ -26,7 +27,7 @@ * It also contains getters for tests. */ public interface S3AInputStreamStatistics extends AutoCloseable, - S3AStatisticInterface { + S3AStatisticInterface, PrefetchingStatistics { /** * Seek backwards, incrementing the seek and backward seek counters. 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 cea8be7f10e40..d10b6484175b1 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 @@ -221,6 +221,41 @@ public void unbuffered() { } + @Override + public DurationTracker prefetchOperationStarted() { + return stubDurationTracker(); + } + + @Override + public void prefetchOperationCompleted() { + + } + + @Override + public void blockAddedToFileCache() { + + } + + @Override + public void blockRemovedFromFileCache() { + + } + + @Override + public void executorAcquired(Duration timeInQueue) { + + } + + @Override + public void memoryAllocated(int size) { + + } + + @Override + public void memoryFreed(int size) { + + } + /** * Return an IO statistics instance. * @return an empty IO statistics instance. @@ -354,6 +389,7 @@ public DurationTracker initiateGetRequest() { public DurationTracker initiateInnerStreamClose(final boolean abort) { return stubDurationTracker(); } + } /** diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md index 7c0a49f8fbeda..4381a5d66d3f7 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md @@ -47,6 +47,7 @@ full details. * [Auditing](./auditing.html). * [Auditing Architecture](./auditing_architecture.html). * [Testing](./testing.html) +* [Prefetching](./prefetching.html) ## Overview @@ -1090,6 +1091,29 @@ options are covered in [Testing](./testing.md). + + fs.s3a.prefetch.enabled + false + + Enables prefetching and caching when reading from input stream. + + + + + fs.s3a.prefetch.block.size + 8MB + + The size of a single prefetched block of data. + + + + + fs.s3a.prefetch.block.count + 8 + + Maximum number of blocks prefetched concurrently at any given time. + + ``` ## Retry and Recovery 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 new file mode 100644 index 0000000000000..9b4e888d604ca --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/prefetching.md @@ -0,0 +1,192 @@ + + +# S3A Prefetching + +This document explains the `S3PrefetchingInputStream` and the various components it uses. + +This input stream implements prefetching and caching to improve read performance of the input +stream. +A high level overview of this feature was published in +[Pinterest Engineering's blog post titled "Improving efficiency and reducing runtime using S3 read optimization"](https://medium.com/pinterest-engineering/improving-efficiency-and-reducing-runtime-using-s3-read-optimization-b31da4b60fa0). + +With prefetching, the input stream divides the remote file into blocks of a fixed size, associates +buffers to these blocks and then reads data into these buffers asynchronously. +It also potentially caches these blocks. + +### Basic Concepts + +* **Remote File**: A binary blob of data stored on some storage device. +* **Block File**: Local file containing a block of the remote file. +* **Block**: A file is divided into a number of blocks. +The size of the first n-1 blocks is same, and the size of the last block may be same or smaller. +* **Block based reading**: The granularity of read is one block. +That is, either an entire block is read and returned or none at all. +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` | + +### Key Components + +`S3PrefetchingInputStream` - When prefetching is enabled, S3AFileSystem will return an instance of +this class as the input stream. +Depending on the remote file size, it will either use +the `S3InMemoryInputStream` or the `S3CachingInputStream` as the underlying input stream. + +`S3InMemoryInputStream` - Underlying input stream used when the remote file size < configured block +size. +Will read the entire remote file into memory. + +`S3CachingInputStream` - Underlying input stream used when remote file size > configured block size. +Uses asynchronous prefetching of blocks and caching to improve performance. + +`BlockData` - Holds information about the blocks in a remote file, such as: + +* Number of blocks in the remote file +* Block size +* State of each block (initially all blocks have state *NOT_READY*). +Other states are: Queued, Ready, Cached. + +`BufferData` - Holds the buffer and additional information about it such as: + +* The block number this buffer is for +* State of the buffer (Unknown, Blank, Prefetching, Caching, Ready, Done). +Initial state of a buffer is blank. + +`CachingBlockManager` - Implements reading data into the buffer, prefetching and caching. + +`BufferPool` - Manages a fixed sized pool of buffers. +It's used by `CachingBlockManager` to acquire buffers. + +`S3File` - Implements operations to interact with S3 such as opening and closing the input stream to +the remote file in S3. + +`S3Reader` - Implements reading from the stream opened by `S3File`. +Reads from this input stream in blocks of 64KB. + +`FilePosition` - Provides functionality related to tracking the position in the file. +Also gives access to the current buffer in use. + +`SingleFilePerBlockCache` - Responsible for caching blocks to the local file system. +Each cache block is stored on the local disk as a separate block file. + +### Operation + +#### S3InMemoryInputStream + +For a remote file with size 5MB, and block size = 8MB, since file size is less than the block size, +the `S3InMemoryInputStream` will be used. + +If the caller makes the following read calls: + +``` +in.read(buffer, 0, 3MB); +in.read(buffer, 0, 2MB); +``` + +When the first read is issued, there is no buffer in use yet. +The `S3InMemoryInputStream` gets the data in this remote file by calling the `ensureCurrentBuffer()` +method, which ensures that a buffer with data is available to be read from. + +The `ensureCurrentBuffer()` then: + +* Reads data into a buffer by calling `S3Reader.read(ByteBuffer buffer, long offset, int size)`. +* `S3Reader` uses `S3File` to open an input stream to the remote file in S3 by making + a `getObject()` request with range as `(0, filesize)`. +* The `S3Reader` reads the entire remote file into the provided buffer, and once reading is complete + closes the S3 stream and frees all underlying resources. +* Now the entire remote file is in a buffer, set this data in `FilePosition` so it can be accessed + by the input stream. + +The read operation now just gets the required bytes from the buffer in `FilePosition`. + +When the second read is issued, there is already a valid buffer which can be used. +Don't do anything else, just read the required bytes from this buffer. + +#### S3CachingInputStream + +If there is a remote file with size 40MB and block size = 8MB, the `S3CachingInputStream` will be +used. + +##### Sequential Reads + +If the caller makes the following calls: + +``` +in.read(buffer, 0, 5MB) +in.read(buffer, 0, 8MB) +``` + +For the first read call, there is no valid buffer yet. +`ensureCurrentBuffer()` is called, and for the first `read()`, prefetch count is set as 1. + +The current block (block 0) is read synchronously, while the blocks to be prefetched (block 1) is +read asynchronously. + +The `CachingBlockManager` is responsible for getting buffers from the buffer pool and reading data +into them. This process of acquiring the buffer pool works as follows: + +* The buffer pool keeps a map of allocated buffers and a pool of available buffers. +The size of this pool is = prefetch block count + 1. +If the prefetch block count is 8, the buffer pool has a size of 9. +* If the pool is not yet at capacity, create a new buffer and add it to the pool. +* If it's at capacity, check if any buffers with state = done can be released. +Releasing a buffer means removing it from allocated and returning it back to the pool of available +buffers. +* If there are no buffers with state = done currently then nothing will be released, so retry the + above step at a fixed interval a few times till a buffer becomes available. +* If after multiple retries there are still no available buffers, release a buffer in the ready state. +The buffer for the block furthest from the current block is released. + +Once a buffer has been acquired by `CachingBlockManager`, if the buffer is in a *READY* state, it is +returned. +This means that data was already read into this buffer asynchronously by a prefetch. +If it's state is *BLANK* then data is read into it using +`S3Reader.read(ByteBuffer buffer, long offset, int size).` + +For the second read call, `in.read(buffer, 0, 8MB)`, since the block sizes are of 8MB and only 5MB +of block 0 has been read so far, 3MB of the required data will be read from the current block 0. +Once all data has been read from this block, `S3CachingInputStream` requests the next block ( +block 1), which will already have been prefetched and so it can just start reading from it. +Also, while reading from block 1 it will also issue prefetch requests for the next blocks. +The number of blocks to be prefetched is determined by `fs.s3a.prefetch.block.count`. + +##### Random Reads + +If the caller makes the following calls: + +``` +in.read(buffer, 0, 5MB) +in.seek(10MB) +in.read(buffer, 0, 4MB) +in.seek(2MB) +in.read(buffer, 0, 4MB) +``` + +The `CachingInputStream` also caches prefetched blocks. +This happens when a `seek()` is issued for outside the current block and the current block still has +not been fully read. + +For the above read sequence, when the `seek(10MB)` call is issued, block 0 has not been read +completely so cache it as the caller will probably want to read from it again. + +When `seek(2MB)` is called, the position is back inside block 0. +The next read can now be satisfied from the locally cached block file, which is typically orders of +magnitude faster than a network based read. \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/EmptyPrefetchingStatistics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/EmptyPrefetchingStatistics.java new file mode 100644 index 0000000000000..ce13d2d992936 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/EmptyPrefetchingStatistics.java @@ -0,0 +1,76 @@ + /* + * 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.common; + +import java.time.Duration; + +import org.apache.hadoop.fs.statistics.DurationTracker; + +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.stubDurationTracker; + +public final class EmptyPrefetchingStatistics implements PrefetchingStatistics { + + private static final EmptyPrefetchingStatistics EMPTY_PREFETCHING_STATISTICS = + new EmptyPrefetchingStatistics(); + + private EmptyPrefetchingStatistics() { + } + + public static EmptyPrefetchingStatistics getInstance() { + return EMPTY_PREFETCHING_STATISTICS; + } + + @Override + public DurationTracker prefetchOperationStarted() { + return stubDurationTracker(); + } + + @Override + public void blockAddedToFileCache() { + + } + + @Override + public void blockRemovedFromFileCache() { + + } + + @Override + public void prefetchOperationCompleted() { + + } + + @Override + public void executorAcquired(Duration timeInQueue) { + + } + + @Override + public void memoryAllocated(int size) { + + } + + @Override + public void memoryFreed(int size) { + + } + +} + diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/ExceptionAsserts.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/ExceptionAsserts.java new file mode 100644 index 0000000000000..96b7acdcdb8e4 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/ExceptionAsserts.java @@ -0,0 +1,60 @@ +/* + * 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.common; + +import org.apache.hadoop.test.LambdaTestUtils; + +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +public final class ExceptionAsserts { + private ExceptionAsserts() {} + + + /** + * Asserts that the given code throws an exception of the given type + * and that the exception message contains the given sub-message. + * + * Usage: + * + * ExceptionAsserts.assertThrows( + * IllegalArgumentException.class, + * "'nullArg' must not be null", + * () -> Preconditions.checkNotNull(null, "nullArg")); + * + * Note: JUnit 5 has similar functionality but it will be a long time before + * we move to that framework because of significant differences and lack of + * backward compatibility for some JUnit rules. + */ + public static void assertThrows( + Class expectedExceptionClass, + String partialMessage, + LambdaTestUtils.VoidCallable code) throws Exception { + + + intercept(expectedExceptionClass, partialMessage, code); + + } + + public static void assertThrows( + Class expectedExceptionClass, + LambdaTestUtils.VoidCallable code) throws Exception { + assertThrows(expectedExceptionClass, null, code); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/SampleDataForTests.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/SampleDataForTests.java new file mode 100644 index 0000000000000..97c2e0b765912 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/SampleDataForTests.java @@ -0,0 +1,57 @@ +/* + * 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.common; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * Frequently used test data items. + */ +public final class SampleDataForTests { + private SampleDataForTests() {} + + + // Array data. + public static final Object[] NULL_ARRAY = null; + public static final Object[] EMPTY_ARRAY = new Object[0]; + public static final Object[] NON_EMPTY_ARRAY = new Object[1]; + + public static final byte[] NULL_BYTE_ARRAY = null; + public static final byte[] EMPTY_BYTE_ARRAY = new byte[0]; + public static final byte[] NON_EMPTY_BYTE_ARRAY = new byte[1]; + + public static final short[] NULL_SHORT_ARRAY = null; + public static final short[] EMPTY_SHORT_ARRAY = new short[0]; + public static final short[] NON_EMPTY_SHORT_ARRAY = new short[1]; + + public static final int[] NULL_INT_ARRAY = null; + public static final int[] EMPTY_INT_ARRAY = new int[0]; + public static final int[] NON_EMPTY_INT_ARRAY = new int[1]; + + public static final long[] NULL_LONG_ARRAY = null; + public static final long[] EMPTY_LONG_ARRAY = new long[0]; + public static final long[] NON_EMPTY_LONG_ARRAY = new long[1]; + + public static final List NULL_LIST = null; + public static final List EMPTY_LIST = new ArrayList(); + public static final List VALID_LIST = Arrays.asList(new Object[1]); +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBlockCache.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBlockCache.java new file mode 100644 index 0000000000000..c402673a49d1e --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBlockCache.java @@ -0,0 +1,99 @@ +/* + * 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.common; + +import java.nio.ByteBuffer; + +import org.junit.Test; + +import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext; +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNotSame; +import static org.junit.Assert.assertTrue; + +public class TestBlockCache extends AbstractHadoopTestBase { + + private static final int BUFFER_SIZE = 16; + + @Test + public void testArgChecks() throws Exception { + // Should not throw. + BlockCache cache = + new SingleFilePerBlockCache(EmptyPrefetchingStatistics.getInstance()); + + ByteBuffer buffer = ByteBuffer.allocate(16); + + // Verify it throws correctly. + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'buffer' must not be null", + () -> cache.put(42, null)); + + ExceptionAsserts.assertThrows( + NullPointerException.class, + () -> new SingleFilePerBlockCache(null)); + } + + + @Test + public void testPutAndGet() throws Exception { + BlockCache cache = + new SingleFilePerBlockCache(new EmptyS3AStatisticsContext().newInputStreamStatistics()); + + ByteBuffer buffer1 = ByteBuffer.allocate(BUFFER_SIZE); + for (byte i = 0; i < BUFFER_SIZE; i++) { + buffer1.put(i); + } + + assertEquals(0, cache.size()); + assertFalse(cache.containsBlock(0)); + cache.put(0, buffer1); + assertEquals(1, cache.size()); + assertTrue(cache.containsBlock(0)); + ByteBuffer buffer2 = ByteBuffer.allocate(BUFFER_SIZE); + cache.get(0, buffer2); + assertNotSame(buffer1, buffer2); + assertBuffersEqual(buffer1, buffer2); + + assertEquals(1, cache.size()); + assertFalse(cache.containsBlock(1)); + cache.put(1, buffer1); + assertEquals(2, cache.size()); + assertTrue(cache.containsBlock(1)); + ByteBuffer buffer3 = ByteBuffer.allocate(BUFFER_SIZE); + cache.get(1, buffer3); + assertNotSame(buffer1, buffer3); + assertBuffersEqual(buffer1, buffer3); + } + + private void assertBuffersEqual(ByteBuffer buffer1, ByteBuffer buffer2) { + assertNotNull(buffer1); + assertNotNull(buffer2); + assertEquals(buffer1.limit(), buffer2.limit()); + assertEquals(BUFFER_SIZE, buffer1.limit()); + for (int i = 0; i < BUFFER_SIZE; i++) { + assertEquals(buffer1.get(i), buffer2.get(i)); + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBlockData.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBlockData.java new file mode 100644 index 0000000000000..dd8c9fb3c7e8d --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBlockData.java @@ -0,0 +1,158 @@ +/* + * 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.common; + +import org.junit.Test; + +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class TestBlockData extends AbstractHadoopTestBase { + + @Test + public void testArgChecks() throws Exception { + // Should not throw. + new BlockData(10, 5); + new BlockData(5, 10); + new BlockData(0, 10); + + // Verify it throws correctly. + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'fileSize' must not be negative", + () -> new BlockData(-1, 2)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'blockSize' must be a positive integer", + () -> new BlockData(10, 0)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'blockSize' must be a positive integer", + () -> new BlockData(10, -2)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'blockNumber' (-1) must be within the range [0, 3]", + () -> new BlockData(10, 3).isLastBlock(-1)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'blockNumber' (11) must be within the range [0, 3]", + () -> new BlockData(10, 3).isLastBlock(11)); + } + + @Test + public void testComputedFields() throws Exception { + testComputedFieldsHelper(0, 10); + testComputedFieldsHelper(1, 10); + testComputedFieldsHelper(10, 1); + testComputedFieldsHelper(10, 2); + testComputedFieldsHelper(10, 3); + } + + private void testComputedFieldsHelper(long fileSize, int blockSize) throws Exception { + BlockData bd = new BlockData(fileSize, blockSize); + + if (fileSize == 0) { + assertFalse(bd.isLastBlock(0)); + assertFalse(bd.isLastBlock(1)); + assertFalse(bd.isValidOffset(0)); + assertEquals(0, bd.getSize(0)); + assertEquals("", bd.getStateString()); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'offset' (0) must be within the range [0, -1]", + () -> bd.getBlockNumber(0)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'blockNumber' (0) must be within the range [0, -1]", + () -> bd.getStartOffset(0)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'offset' (0) must be within the range [0, -1]", + () -> bd.getRelativeOffset(0, 0)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'blockNumber' (0) must be within the range [0, -1]", + () -> bd.getState(0)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'blockNumber' (0) must be within the range [0, -1]", + () -> bd.setState(0, BlockData.State.READY)); + + return; + } + + assertEquals(fileSize, bd.getFileSize()); + assertEquals(blockSize, bd.getBlockSize()); + + int expectedNumBlocks = (int) (fileSize / blockSize); + if (fileSize % blockSize > 0) { + expectedNumBlocks++; + } + assertEquals(expectedNumBlocks, bd.getNumBlocks()); + + int lastBlockNumber = expectedNumBlocks - 1; + for (int b = 0; b < lastBlockNumber; b++) { + assertFalse(bd.isLastBlock(b)); + assertEquals(blockSize, bd.getSize(b)); + } + assertTrue(bd.isLastBlock(lastBlockNumber)); + int lastBlockSize = (int) (fileSize - blockSize * (expectedNumBlocks - 1)); + assertEquals(lastBlockSize, bd.getSize(lastBlockNumber)); + + // Offset related methods. + for (long offset = 0; offset < fileSize; offset++) { + int expectedBlockNumber = (int) (offset / blockSize); + assertEquals(expectedBlockNumber, bd.getBlockNumber(offset)); + + for (int b = 0; b < expectedNumBlocks - 1; b++) { + long expectedStartOffset = b * blockSize; + assertEquals(expectedStartOffset, bd.getStartOffset(b)); + + int expectedRelativeOffset = (int) (offset - expectedStartOffset); + assertEquals(expectedRelativeOffset, bd.getRelativeOffset(b, offset)); + } + } + + + // State methods. + for (int b = 0; b < expectedNumBlocks; b++) { + assertEquals(b * blockSize, bd.getStartOffset(b)); + assertEquals(BlockData.State.NOT_READY, bd.getState(b)); + bd.setState(b, BlockData.State.QUEUED); + assertEquals(BlockData.State.QUEUED, bd.getState(b)); + bd.setState(b, BlockData.State.READY); + assertEquals(BlockData.State.READY, bd.getState(b)); + bd.setState(b, BlockData.State.CACHED); + assertEquals(BlockData.State.CACHED, bd.getState(b)); + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBlockOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBlockOperations.java new file mode 100644 index 0000000000000..da46db1e550f5 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBlockOperations.java @@ -0,0 +1,105 @@ +/* + * 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.common; + +import java.lang.reflect.Method; + +import org.junit.Test; + +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.junit.Assert.assertTrue; + +public class TestBlockOperations extends AbstractHadoopTestBase { + + @Test + public void testArgChecks() throws Exception { + // Should not throw. + BlockOperations ops = new BlockOperations(); + + // Verify it throws correctly. + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'blockNumber' must not be negative", + () -> ops.getPrefetched(-1)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'blockNumber' must not be negative", + () -> ops.getCached(-1)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'blockNumber' must not be negative", + () -> ops.getRead(-1)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'blockNumber' must not be negative", + () -> ops.release(-1)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'blockNumber' must not be negative", + () -> ops.requestPrefetch(-1)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'blockNumber' must not be negative", + () -> ops.requestCaching(-1)); + } + + @Test + public void testGetSummary() throws Exception { + verifySummary("getPrefetched", "GP"); + verifySummary("getCached", "GC"); + verifySummary("getRead", "GR"); + verifySummary("release", "RL"); + verifySummary("requestPrefetch", "RP"); + verifySummary("prefetch", "PF"); + verifySummary("requestCaching", "RC"); + verifySummary("addToCache", "C+"); + + verifySummaryNoArg("cancelPrefetches", "CP"); + verifySummaryNoArg("close", "CX"); + } + + private void verifySummary(String methodName, String shortName) throws Exception { + int blockNumber = 42; + BlockOperations ops = new BlockOperations(); + Method method = ops.getClass().getDeclaredMethod(methodName, int.class); + BlockOperations.Operation op = (BlockOperations.Operation) method.invoke(ops, blockNumber); + ops.end(op); + String summary = ops.getSummary(false); + String opSummary = String.format("%s(%d)", shortName, blockNumber); + String expectedSummary = String.format("%s;E%s;", opSummary, opSummary); + assertTrue(summary.startsWith(expectedSummary)); + } + + private void verifySummaryNoArg(String methodName, String shortName) throws Exception { + BlockOperations ops = new BlockOperations(); + Method method = ops.getClass().getDeclaredMethod(methodName); + BlockOperations.Operation op = (BlockOperations.Operation) method.invoke(ops); + ops.end(op); + String summary = ops.getSummary(false); + String expectedSummary = String.format("%s;E%s;", shortName, shortName); + assertTrue(summary.startsWith(expectedSummary)); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBoundedResourcePool.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBoundedResourcePool.java new file mode 100644 index 0000000000000..154e8464a49cd --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBoundedResourcePool.java @@ -0,0 +1,148 @@ +/* + * 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.common; + +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.IdentityHashMap; +import java.util.Set; + +import org.junit.Test; + +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; + +public class TestBoundedResourcePool extends AbstractHadoopTestBase { + + static class BufferPool extends BoundedResourcePool { + BufferPool(int size) { + super(size); + } + + @Override + protected ByteBuffer createNew() { + return ByteBuffer.allocate(10); + } + } + + @Test + public void testArgChecks() throws Exception { + + // Should not throw. + BufferPool pool = new BufferPool(5); + + // Verify it throws correctly. + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'size' must be a positive integer", + () -> new BufferPool(-1)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'size' must be a positive integer", + () -> new BufferPool(0)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'item' must not be null", + () -> pool.release(null)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "This item is not a part of this pool", + () -> pool.release(ByteBuffer.allocate(4))); + } + + @Test + public void testAcquireReleaseSingle() { + final int numBuffers = 5; + BufferPool pool = new BufferPool(numBuffers); + + assertEquals(0, pool.numCreated()); + assertEquals(numBuffers, pool.numAvailable()); + + ByteBuffer buffer1 = pool.acquire(); + assertNotNull(buffer1); + assertEquals(1, pool.numCreated()); + assertEquals(numBuffers - 1, pool.numAvailable()); + + // Release and immediately reacquire => should not end up creating new buffer. + pool.release(buffer1); + assertEquals(1, pool.numCreated()); + + ByteBuffer buffer2 = pool.acquire(); + assertNotNull(buffer2); + assertSame(buffer1, buffer2); + assertEquals(1, pool.numCreated()); + } + + @Test + public void testAcquireReleaseMultiple() { + final int numBuffers = 5; + BufferPool pool = new BufferPool(numBuffers); + Set buffers = + Collections.newSetFromMap(new IdentityHashMap()); + + assertEquals(0, pool.numCreated()); + + // Acquire all one by one. + for (int i = 0; i < numBuffers; i++) { + assertEquals(numBuffers - i, pool.numAvailable()); + ByteBuffer buffer = pool.acquire(); + assertNotNull(buffer); + assertFalse(buffers.contains(buffer)); + buffers.add(buffer); + assertEquals(i + 1, pool.numCreated()); + } + + assertEquals(numBuffers, pool.numCreated()); + assertEquals(0, pool.numAvailable()); + + int releaseCount = 0; + + // Release all one by one. + for (ByteBuffer buffer : buffers) { + assertEquals(releaseCount, pool.numAvailable()); + releaseCount++; + pool.release(buffer); + assertEquals(releaseCount, pool.numAvailable()); + + // Releasing the same buffer again should not have any ill effect. + pool.release(buffer); + assertEquals(releaseCount, pool.numAvailable()); + pool.release(buffer); + assertEquals(releaseCount, pool.numAvailable()); + } + + // Acquire all one by one again to ensure that they are the same ones we got earlier. + for (int i = 0; i < numBuffers; i++) { + ByteBuffer buffer = pool.acquire(); + assertTrue(buffers.contains(buffer)); + } + + assertEquals(numBuffers, pool.numCreated()); + assertEquals(0, pool.numAvailable()); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBufferData.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBufferData.java new file mode 100644 index 0000000000000..c4699d11540ee --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBufferData.java @@ -0,0 +1,249 @@ +/* + * 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.common; + +import java.nio.ByteBuffer; +import java.nio.ReadOnlyBufferException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +import org.junit.Test; + +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNotSame; +import static org.junit.Assert.assertSame; + +public class TestBufferData extends AbstractHadoopTestBase { + + @Test + public void testArgChecks() throws Exception { + // Should not throw. + ByteBuffer buffer = ByteBuffer.allocate(1); + BufferData data = new BufferData(1, buffer); + + // Verify it throws correctly. + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'blockNumber' must not be negative", + () -> new BufferData(-1, buffer)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'buffer' must not be null", + () -> new BufferData(1, null)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'actionFuture' must not be null", + () -> data.setPrefetch(null)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'actionFuture' must not be null", + () -> data.setCaching(null)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'states' must not be null", + () -> data.throwIfStateIncorrect((BufferData.State[]) null)); + + ExceptionAsserts.assertThrows( + IllegalStateException.class, + "Expected buffer state to be 'READY or CACHING' but found", + () -> data.throwIfStateIncorrect(BufferData.State.READY, BufferData.State.CACHING)); + } + + @Test + public void testValidStateUpdates() { + ByteBuffer buffer = ByteBuffer.allocate(1); + BufferData data = new BufferData(1, buffer); + + assertEquals(BufferData.State.BLANK, data.getState()); + + CompletableFuture actionFuture = new CompletableFuture<>(); + actionFuture.complete(null); + data.setPrefetch(actionFuture); + assertEquals(BufferData.State.PREFETCHING, data.getState()); + assertNotNull(data.getActionFuture()); + assertSame(actionFuture, data.getActionFuture()); + + CompletableFuture actionFuture2 = new CompletableFuture<>(); + data.setCaching(actionFuture2); + assertEquals(BufferData.State.CACHING, data.getState()); + assertNotNull(data.getActionFuture()); + assertSame(actionFuture2, data.getActionFuture()); + assertNotSame(actionFuture, actionFuture2); + + List states = Arrays.asList( + BufferData.State.BLANK, + BufferData.State.PREFETCHING, + BufferData.State.CACHING, + BufferData.State.READY + ); + + BufferData data2 = new BufferData(1, buffer); + BufferData.State prevState = null; + for (BufferData.State state : states) { + if (prevState != null) { + assertEquals(prevState, data2.getState()); + data2.updateState(state, prevState); + assertEquals(state, data2.getState()); + } + prevState = state; + } + } + + @Test + public void testInvalidStateUpdates() throws Exception { + CompletableFuture actionFuture = new CompletableFuture<>(); + actionFuture.complete(null); + testInvalidStateUpdatesHelper( + (d) -> d.setPrefetch(actionFuture), + BufferData.State.BLANK, + BufferData.State.READY); + + testInvalidStateUpdatesHelper( + (d) -> d.setCaching(actionFuture), + BufferData.State.PREFETCHING, + BufferData.State.READY); + } + + @Test + public void testSetReady() throws Exception { + byte[] bytes1 = new byte[5]; + initBytes(bytes1); + + ByteBuffer buffer = ByteBuffer.allocate(10); + buffer.put(bytes1); + buffer.limit(bytes1.length); + BufferData data = new BufferData(1, buffer); + assertNotEquals(BufferData.State.READY, data.getState()); + assertEquals(0, data.getChecksum()); + + data.setReady(BufferData.State.BLANK); + assertEquals(BufferData.State.READY, data.getState()); + assertNotEquals(0, data.getChecksum()); + + // Verify that buffer cannot be modified once in READY state. + ExceptionAsserts.assertThrows( + ReadOnlyBufferException.class, + null, + () -> data.getBuffer().put(bytes1)); + + // Verify that buffer cannot be set to READY state more than once. + ExceptionAsserts.assertThrows( + IllegalStateException.class, + "Checksum cannot be changed once set", + () -> data.setReady(BufferData.State.BLANK)); + + // Verify that we detect post READY buffer modification. + buffer.array()[2] = (byte) 42; + ExceptionAsserts.assertThrows( + IllegalStateException.class, + "checksum changed after setReady()", + () -> data.setDone()); + } + + @Test + public void testChecksum() { + byte[] bytes1 = new byte[5]; + byte[] bytes2 = new byte[10]; + + initBytes(bytes1); + initBytes(bytes2); + + ByteBuffer buffer1 = ByteBuffer.wrap(bytes1); + ByteBuffer buffer2 = ByteBuffer.wrap(bytes2); + buffer2.limit(bytes1.length); + + long checksum1 = BufferData.getChecksum(buffer1); + long checksum2 = BufferData.getChecksum(buffer2); + + assertEquals(checksum1, checksum2); + } + + private void initBytes(byte[] bytes) { + for (int i = 0; i < bytes.length; i++) { + bytes[i] = (byte) i; + } + } + + @FunctionalInterface + public interface StateChanger { + void run(BufferData data) throws Exception; + } + + private void testInvalidStateUpdatesHelper( + StateChanger changeState, + BufferData.State... validFromState) throws Exception { + + ByteBuffer buffer = ByteBuffer.allocate(1); + BufferData data = new BufferData(1, buffer); + data.updateState(validFromState[0], BufferData.State.BLANK); + List states = this.getStatesExcept(validFromState); + BufferData.State prevState = validFromState[0]; + String expectedMessage = + String.format("Expected buffer state to be '%s", validFromState[0]); + for (BufferData.State s : states) { + data.updateState(s, prevState); + + ExceptionAsserts.assertThrows( + IllegalStateException.class, + expectedMessage, + () -> changeState.run(data)); + + assertEquals(s, data.getState()); + prevState = s; + } + } + + static final List ALL_STATES = Arrays.asList( + BufferData.State.UNKNOWN, + BufferData.State.BLANK, + BufferData.State.PREFETCHING, + BufferData.State.CACHING, + BufferData.State.READY + ); + + private List getStatesExcept(BufferData.State... states) { + + List result = new ArrayList<>(); + for (BufferData.State s : ALL_STATES) { + boolean found = false; + for (BufferData.State ss : states) { + if (s == ss) { + found = true; + } + } + + if (!found) { + result.add(s); + } + } + + return result; + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBufferPool.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBufferPool.java new file mode 100644 index 0000000000000..c9134f1e2511e --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBufferPool.java @@ -0,0 +1,163 @@ +/* + * 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.common; + +import org.junit.Test; + +import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; +import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext; +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertSame; + +public class TestBufferPool extends AbstractHadoopTestBase { + + private static final int POOL_SIZE = 2; + private static final int BUFFER_SIZE = 10; + private final S3AInputStreamStatistics s3AInputStreamStatistics = + new EmptyS3AStatisticsContext().newInputStreamStatistics(); + + @Test + public void testArgChecks() throws Exception { + // Should not throw. + BufferPool pool = new BufferPool(POOL_SIZE, BUFFER_SIZE, s3AInputStreamStatistics); + + // Verify it throws correctly. + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'size' must be a positive integer", + () -> new BufferPool(0, 10, s3AInputStreamStatistics)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'size' must be a positive integer", + () -> new BufferPool(-1, 10, s3AInputStreamStatistics)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'bufferSize' must be a positive integer", + () -> new BufferPool(10, 0, s3AInputStreamStatistics)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'bufferSize' must be a positive integer", + () -> new BufferPool(1, -10, s3AInputStreamStatistics)); + + ExceptionAsserts.assertThrows( + NullPointerException.class, + () -> new BufferPool(1, 10, null)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'blockNumber' must not be negative", + () -> pool.acquire(-1)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'blockNumber' must not be negative", + () -> pool.tryAcquire(-1)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'data' must not be null", + () -> pool.release((BufferData) null)); + } + + @Test + public void testGetAndRelease() { + BufferPool pool = new BufferPool(POOL_SIZE, BUFFER_SIZE, s3AInputStreamStatistics); + assertInitialState(pool, POOL_SIZE); + + int count = 0; + for (BufferData data : pool.getAll()) { + count++; + } + assertEquals(0, count); + + BufferData data1 = this.acquire(pool, 1); + BufferData data2 = this.acquire(pool, 2); + BufferData data3 = pool.tryAcquire(3); + assertNull(data3); + + count = 0; + for (BufferData data : pool.getAll()) { + count++; + } + assertEquals(2, count); + + assertEquals(2, pool.numCreated()); + assertEquals(0, pool.numAvailable()); + + data1.updateState(BufferData.State.READY, BufferData.State.BLANK); + pool.release(data1); + + assertEquals(2, pool.numCreated()); + assertEquals(1, pool.numAvailable()); + + data2.updateState(BufferData.State.READY, BufferData.State.BLANK); + pool.release(data2); + + assertEquals(2, pool.numCreated()); + assertEquals(2, pool.numAvailable()); + } + + @Test + public void testRelease() throws Exception { + testReleaseHelper(BufferData.State.BLANK, true); + testReleaseHelper(BufferData.State.PREFETCHING, true); + testReleaseHelper(BufferData.State.CACHING, true); + testReleaseHelper(BufferData.State.READY, false); + } + + private void testReleaseHelper(BufferData.State stateBeforeRelease, boolean expectThrow) + throws Exception { + + BufferPool pool = new BufferPool(POOL_SIZE, BUFFER_SIZE, s3AInputStreamStatistics); + assertInitialState(pool, POOL_SIZE); + + BufferData data = this.acquire(pool, 1); + data.updateState(stateBeforeRelease, BufferData.State.BLANK); + + if (expectThrow) { + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "Unable to release buffer", + () -> pool.release(data)); + } else { + pool.release(data); + } + } + + private BufferData acquire(BufferPool pool, int blockNumber) { + BufferData data = pool.acquire(blockNumber); + assertNotNull(data); + assertSame(data, pool.acquire(blockNumber)); + assertEquals(blockNumber, data.getBlockNumber()); + return data; + } + + private void assertInitialState(BufferPool pool, int poolSize) { + assertEquals(poolSize, pool.numAvailable()); + assertEquals(0, pool.numCreated()); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestExecutorServiceFuturePool.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestExecutorServiceFuturePool.java new file mode 100644 index 0000000000000..00055a9ea1ef4 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestExecutorServiceFuturePool.java @@ -0,0 +1,92 @@ +/* + * 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.common; + +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import org.apache.hadoop.test.AbstractHadoopTestBase; +import org.apache.hadoop.test.LambdaTestUtils; + +import static org.junit.Assert.assertTrue; + +public class TestExecutorServiceFuturePool extends AbstractHadoopTestBase { + + private ExecutorService executorService; + + @Before + public void setUp() { + executorService = Executors.newFixedThreadPool(3); + } + + @After + public void tearDown() { + if (executorService != null) { + executorService.shutdownNow(); + } + } + + @Test + public void testRunnableSucceeds() throws Exception { + ExecutorServiceFuturePool futurePool = new ExecutorServiceFuturePool(executorService); + final AtomicBoolean atomicBoolean = new AtomicBoolean(false); + Future future = futurePool.executeRunnable(() -> atomicBoolean.set(true)); + future.get(30, TimeUnit.SECONDS); + assertTrue("atomicBoolean set to true?", atomicBoolean.get()); + } + + @Test + public void testSupplierSucceeds() throws Exception { + ExecutorServiceFuturePool futurePool = new ExecutorServiceFuturePool(executorService); + final AtomicBoolean atomicBoolean = new AtomicBoolean(false); + Future future = futurePool.executeFunction(() -> { + atomicBoolean.set(true); + return null; + }); + future.get(30, TimeUnit.SECONDS); + assertTrue("atomicBoolean set to true?", atomicBoolean.get()); + } + + @Test + public void testRunnableFails() throws Exception { + ExecutorServiceFuturePool futurePool = new ExecutorServiceFuturePool(executorService); + Future future = futurePool.executeRunnable(() -> { + throw new IllegalStateException("deliberate"); + }); + LambdaTestUtils.intercept(ExecutionException.class, () -> future.get(30, TimeUnit.SECONDS)); + } + + @Test + public void testSupplierFails() throws Exception { + ExecutorServiceFuturePool futurePool = new ExecutorServiceFuturePool(executorService); + Future future = futurePool.executeFunction(() -> { + throw new IllegalStateException("deliberate"); + }); + LambdaTestUtils.intercept(ExecutionException.class, () -> future.get(30, TimeUnit.SECONDS)); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestFilePosition.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestFilePosition.java new file mode 100644 index 0000000000000..a1b4ae610a170 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestFilePosition.java @@ -0,0 +1,216 @@ +/* + * 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.common; + +import java.nio.ByteBuffer; + +import org.junit.Test; + +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class TestFilePosition extends AbstractHadoopTestBase { + + @Test + public void testArgChecks() throws Exception { + ByteBuffer buffer = ByteBuffer.allocate(10); + BufferData data = new BufferData(0, buffer); + + // Should not throw. + new FilePosition(0, 0); + new FilePosition(0, 5); + new FilePosition(10, 5); + new FilePosition(5, 10); + new FilePosition(10, 5).setData(data, 3, 4); + + // Verify it throws correctly. + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'fileSize' must not be negative", + () -> new FilePosition(-1, 2)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'blockSize' must be a positive integer", + () -> new FilePosition(1, 0)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'blockSize' must be a positive integer", + () -> new FilePosition(1, -1)); + + FilePosition pos = new FilePosition(10, 3); + + // Verify that we cannot obtain buffer properties without setting buffer. + ExceptionAsserts.assertThrows( + IllegalStateException.class, + "'buffer' must not be null", + () -> pos.buffer()); + + ExceptionAsserts.assertThrows( + IllegalStateException.class, + "'buffer' must not be null", + () -> pos.absolute()); + + ExceptionAsserts.assertThrows( + IllegalStateException.class, + "'buffer' must not be null", + () -> pos.isWithinCurrentBuffer(2)); + + ExceptionAsserts.assertThrows( + IllegalStateException.class, + "'buffer' must not be null", + () -> pos.blockNumber()); + + ExceptionAsserts.assertThrows( + IllegalStateException.class, + "'buffer' must not be null", + () -> pos.isLastBlock()); + + ExceptionAsserts.assertThrows( + IllegalStateException.class, + "'buffer' must not be null", + () -> pos.bufferFullyRead()); + + // Verify that we cannot set invalid buffer parameters. + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'bufferData' must not be null", + () -> pos.setData(null, 4, 4)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'startOffset' must not be negative", + () -> pos.setData(data, -4, 4)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'readOffset' must not be negative", + () -> pos.setData(data, 4, -4)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'readOffset' must not be negative", + () -> pos.setData(data, 4, -4)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'readOffset' (15) must be within the range [4, 13]", + () -> pos.setData(data, 4, 15)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'readOffset' (3) must be within the range [4, 13]", + () -> pos.setData(data, 4, 3)); + } + + @Test + public void testValidity() { + int bufferSize = 8; + long fileSize = 100; + long bufferStartOffset = 7; + long readStartOffset = 9; + + ByteBuffer buffer = ByteBuffer.allocate(bufferSize); + BufferData data = new BufferData(0, buffer); + FilePosition pos = new FilePosition(fileSize, bufferSize); + + assertFalse(pos.isValid()); + pos.setData(data, bufferStartOffset, readStartOffset); + assertTrue(pos.isValid()); + + pos.invalidate(); + assertFalse(pos.isValid()); + } + + @Test + public void testOffsets() { + int bufferSize = 8; + long fileSize = 100; + long bufferStartOffset = 7; + long readStartOffset = 9; + + ByteBuffer buffer = ByteBuffer.allocate(bufferSize); + BufferData data = new BufferData(0, buffer); + FilePosition pos = new FilePosition(fileSize, bufferSize); + pos.setData(data, bufferStartOffset, readStartOffset); + assertTrue(pos.isValid()); + + assertEquals(readStartOffset, pos.absolute()); + assertEquals(readStartOffset - bufferStartOffset, pos.relative()); + assertTrue(pos.isWithinCurrentBuffer(8)); + assertFalse(pos.isWithinCurrentBuffer(6)); + assertFalse(pos.isWithinCurrentBuffer(1)); + + int expectedBlockNumber = (int) (bufferStartOffset / bufferSize); + assertEquals(expectedBlockNumber, pos.blockNumber()); + assertFalse(pos.isLastBlock()); + + pos.setData(data, fileSize - 3, fileSize - 2); + assertTrue(pos.isLastBlock()); + } + + @Test + public void testBufferStats() { + int bufferSize = 8; + long fileSize = 100; + long bufferStartOffset = 7; + long readStartOffset = 9; + + ByteBuffer buffer = ByteBuffer.allocate(bufferSize); + BufferData data = new BufferData(0, buffer); + FilePosition pos = new FilePosition(fileSize, bufferSize); + pos.setData(data, bufferStartOffset, readStartOffset); + assertTrue(pos.isValid()); + assertEquals(bufferStartOffset, pos.bufferStartOffset()); + + assertEquals(0, pos.numBytesRead()); + assertEquals(0, pos.numSingleByteReads()); + assertEquals(0, pos.numBufferReads()); + + pos.incrementBytesRead(1); + pos.incrementBytesRead(1); + pos.incrementBytesRead(1); + pos.incrementBytesRead(5); + pos.incrementBytesRead(51); + + assertEquals(59, pos.numBytesRead()); + assertEquals(3, pos.numSingleByteReads()); + assertEquals(2, pos.numBufferReads()); + + assertFalse(pos.bufferFullyRead()); + + pos.setData(data, bufferStartOffset, bufferStartOffset); + assertTrue(pos.isValid()); + + assertEquals(0, pos.numBytesRead()); + assertEquals(0, pos.numSingleByteReads()); + assertEquals(0, pos.numBufferReads()); + + for (int i = 0; i < bufferSize; i++) { + pos.buffer().get(); + pos.incrementBytesRead(1); + } + assertTrue(pos.bufferFullyRead()); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestRetryer.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestRetryer.java new file mode 100644 index 0000000000000..7220a7ec8fc40 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestRetryer.java @@ -0,0 +1,81 @@ +/* + * 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.common; + +import org.junit.Test; + +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class TestRetryer extends AbstractHadoopTestBase { + + @Test + public void testArgChecks() throws Exception { + // Should not throw. + new Retryer(10, 50, 500); + + // Verify it throws correctly. + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'perRetryDelay' must be a positive integer", + () -> new Retryer(-1, 50, 500)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'perRetryDelay' must be a positive integer", + () -> new Retryer(0, 50, 500)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'maxDelay' (5) must be greater than 'perRetryDelay' (10)", + () -> new Retryer(10, 5, 500)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'statusUpdateInterval' must be a positive integer", + () -> new Retryer(10, 50, -1)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'statusUpdateInterval' must be a positive integer", + () -> new Retryer(10, 50, 0)); + } + + @Test + public void testRetry() { + int perRetryDelay = 1; + int statusUpdateInterval = 3; + int maxDelay = 10; + + Retryer retryer = new Retryer(perRetryDelay, maxDelay, statusUpdateInterval); + for (int t = 1; t <= maxDelay; t++) { + assertTrue(retryer.continueRetry()); + if (t % statusUpdateInterval == 0) { + assertTrue(retryer.updateStatus()); + } else { + assertFalse(retryer.updateStatus()); + } + } + + assertFalse(retryer.continueRetry()); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestValidate.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestValidate.java new file mode 100644 index 0000000000000..bffbde417b582 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestValidate.java @@ -0,0 +1,322 @@ +/* + * 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.common; + +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Arrays; + +import org.junit.Test; + +import org.apache.hadoop.test.AbstractHadoopTestBase; + +public class TestValidate extends AbstractHadoopTestBase { + @Test + public void testCheckNotNull() throws Exception { + String nonNullArg = "nonNullArg"; + String nullArg = null; + + // Should not throw. + Validate.checkNotNull(nonNullArg, "nonNullArg"); + + // Verify it throws. + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'nullArg' must not be null", + () -> Validate.checkNotNull(nullArg, "nullArg")); + } + + @Test + public void testCheckPositiveInteger() throws Exception { + int positiveArg = 1; + int zero = 0; + int negativeArg = -1; + + // Should not throw. + Validate.checkPositiveInteger(positiveArg, "positiveArg"); + + // Verify it throws. + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'negativeArg' must be a positive integer", + () -> Validate.checkPositiveInteger(negativeArg, "negativeArg")); + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'zero' must be a positive integer", + () -> Validate.checkPositiveInteger(zero, "zero")); + } + + @Test + public void testCheckNotNegative() throws Exception { + int positiveArg = 1; + int zero = 0; + int negativeArg = -1; + + // Should not throw. + Validate.checkNotNegative(zero, "zeroArg"); + Validate.checkNotNegative(positiveArg, "positiveArg"); + + // Verify it throws. + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'negativeArg' must not be negative", + () -> Validate.checkNotNegative(negativeArg, "negativeArg")); + } + + @Test + public void testCheckRequired() throws Exception { + // Should not throw. + Validate.checkRequired(true, "arg"); + + // Verify it throws. + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'arg' is required", + () -> Validate.checkRequired(false, "arg")); + } + + @Test + public void testCheckValid() throws Exception { + // Should not throw. + Validate.checkValid(true, "arg"); + + // Verify it throws. + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'arg' is invalid", + () -> Validate.checkValid(false, "arg")); + } + + @Test + public void testCheckValidWithValues() throws Exception { + String validValues = "foo, bar"; + + // Should not throw. + Validate.checkValid(true, "arg", validValues); + + // Verify it throws. + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'arg' is invalid. Valid values are: foo, bar", + () -> Validate.checkValid(false, "arg", validValues)); + } + + @Test + public void testCheckNotNullAndNotEmpty() throws Exception { + // Should not throw. + Validate.checkNotNullAndNotEmpty(SampleDataForTests.NON_EMPTY_ARRAY, "array"); + Validate.checkNotNullAndNotEmpty(SampleDataForTests.NON_EMPTY_BYTE_ARRAY, "array"); + Validate.checkNotNullAndNotEmpty(SampleDataForTests.NON_EMPTY_SHORT_ARRAY, "array"); + Validate.checkNotNullAndNotEmpty(SampleDataForTests.NON_EMPTY_INT_ARRAY, "array"); + Validate.checkNotNullAndNotEmpty(SampleDataForTests.NON_EMPTY_LONG_ARRAY, "array"); + + // Verify it throws. + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'string' must not be empty", + () -> Validate.checkNotNullAndNotEmpty("", "string")); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'array' must not be null", + () -> Validate.checkNotNullAndNotEmpty(SampleDataForTests.NULL_ARRAY, "array")); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'array' must have at least one element", + () -> Validate.checkNotNullAndNotEmpty(SampleDataForTests.EMPTY_ARRAY, "array")); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'array' must not be null", + () -> Validate.checkNotNullAndNotEmpty(SampleDataForTests.NULL_BYTE_ARRAY, "array")); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'array' must have at least one element", + () -> Validate.checkNotNullAndNotEmpty(SampleDataForTests.EMPTY_BYTE_ARRAY, "array")); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'array' must not be null", + () -> Validate.checkNotNullAndNotEmpty(SampleDataForTests.NULL_SHORT_ARRAY, "array")); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'array' must have at least one element", + () -> Validate.checkNotNullAndNotEmpty(SampleDataForTests.EMPTY_SHORT_ARRAY, "array")); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'array' must not be null", + () -> Validate.checkNotNullAndNotEmpty(SampleDataForTests.NULL_INT_ARRAY, "array")); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'array' must have at least one element", + () -> Validate.checkNotNullAndNotEmpty(SampleDataForTests.EMPTY_INT_ARRAY, "array")); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'array' must not be null", + () -> Validate.checkNotNullAndNotEmpty(SampleDataForTests.NULL_LONG_ARRAY, "array")); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'array' must have at least one element", + () -> Validate.checkNotNullAndNotEmpty(SampleDataForTests.EMPTY_LONG_ARRAY, "array")); + } + + @Test + public void testCheckListNotNullAndNotEmpty() throws Exception { + // Should not throw. + Validate.checkNotNullAndNotEmpty(SampleDataForTests.VALID_LIST, "list"); + + // Verify it throws. + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'list' must not be null", + () -> Validate.checkNotNullAndNotEmpty(SampleDataForTests.NULL_LIST, "list")); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'list' must have at least one element", + () -> Validate.checkNotNullAndNotEmpty(SampleDataForTests.EMPTY_LIST, "list")); + } + + @Test + public void testCheckNotNullAndNumberOfElements() throws Exception { + // Should not throw. + Validate.checkNotNullAndNumberOfElements(Arrays.asList(1, 2, 3), 3, "arg"); + + // Verify it throws. + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'arg' must not be null", + () -> Validate.checkNotNullAndNumberOfElements(null, 3, "arg") + ); + + // Verify it throws. + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "Number of elements in 'arg' must be exactly 3, 2 given.", + () -> Validate.checkNotNullAndNumberOfElements(Arrays.asList(1, 2), 3, "arg") + ); + } + + @Test + public void testCheckValuesEqual() throws Exception { + // Should not throw. + Validate.checkValuesEqual(1, "arg1", 1, "arg2"); + + // Verify it throws. + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'arg1' (1) must equal 'arg2' (2)", + () -> Validate.checkValuesEqual(1, "arg1", 2, "arg2")); + } + + @Test + public void testCheckIntegerMultiple() throws Exception { + // Should not throw. + Validate.checkIntegerMultiple(10, "arg1", 5, "arg2"); + + // Verify it throws. + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'arg1' (10) must be an integer multiple of 'arg2' (3)", + () -> Validate.checkIntegerMultiple(10, "arg1", 3, "arg2")); + } + + @Test + public void testCheckGreater() throws Exception { + // Should not throw. + Validate.checkGreater(10, "arg1", 5, "arg2"); + + // Verify it throws. + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'arg1' (5) must be greater than 'arg2' (10)", + () -> Validate.checkGreater(5, "arg1", 10, "arg2")); + } + + @Test + public void testCheckGreaterOrEqual() throws Exception { + // Should not throw. + Validate.checkGreaterOrEqual(10, "arg1", 5, "arg2"); + + // Verify it throws. + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'arg1' (5) must be greater than or equal to 'arg2' (10)", + () -> Validate.checkGreaterOrEqual(5, "arg1", 10, "arg2")); + } + + @Test + public void testCheckWithinRange() throws Exception { + // Should not throw. + Validate.checkWithinRange(10, "arg", 5, 15); + Validate.checkWithinRange(10.0, "arg", 5.0, 15.0); + + // Verify it throws. + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'arg' (5) must be within the range [10, 20]", + () -> Validate.checkWithinRange(5, "arg", 10, 20)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'arg' (5.0) must be within the range [10.0, 20.0]", + () -> Validate.checkWithinRange(5.0, "arg", 10.0, 20.0)); + } + + @Test + public void testCheckPathExists() throws Exception { + Path tempFile = Files.createTempFile("foo", "bar"); + Path tempDir = tempFile.getParent(); + Path notFound = Paths.get(""); + + // Should not throw. + Validate.checkPathExists(tempFile, "tempFile"); + Validate.checkPathExists(tempDir, "tempDir"); + + // Verify it throws. + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'nullArg' must not be null", + () -> Validate.checkPathExists(null, "nullArg")); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "Path notFound () does not exist", + () -> Validate.checkPathExists(notFound, "notFound")); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "must point to a directory", + () -> Validate.checkPathExistsAsDir(tempFile, "tempFile")); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "must point to a file", + () -> Validate.checkPathExistsAsFile(tempDir, "tempDir")); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java index 9a818d037e4c0..aab66dad86082 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java @@ -31,6 +31,7 @@ import org.apache.hadoop.fs.statistics.StreamStatisticNames; import static org.apache.hadoop.fs.s3a.Constants.ALLOW_REQUESTER_PAYS; +import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY; import static org.apache.hadoop.fs.s3a.Constants.S3A_BUCKET_PROBE; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -78,11 +79,16 @@ public void testRequesterPaysOptionSuccess() throws Throwable { inputStream.seek(0); inputStream.readByte(); - // Verify > 1 call was made, so we're sure it is correctly configured for each request - IOStatisticAssertions - .assertThatStatisticCounter(inputStream.getIOStatistics(), - StreamStatisticNames.STREAM_READ_OPENED) - .isGreaterThan(1); + if (conf.getBoolean(PREFETCH_ENABLED_KEY, true)) { + // For S3PrefetchingInputStream, verify a call was made + IOStatisticAssertions.assertThatStatisticCounter(inputStream.getIOStatistics(), + StreamStatisticNames.STREAM_READ_OPENED).isEqualTo(1); + } else { + // For S3AInputStream, verify > 1 call was made, + // so we're sure it is correctly configured for each request + IOStatisticAssertions.assertThatStatisticCounter(inputStream.getIOStatistics(), + StreamStatisticNames.STREAM_READ_OPENED).isGreaterThan(1); + } // Check list calls work without error fs.listFiles(requesterPaysPath.getParent(), false); 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 3d7ee0882efa4..3a2d1b1b09a49 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 @@ -20,6 +20,7 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; import org.apache.hadoop.fs.statistics.IOStatistics; @@ -33,6 +34,7 @@ import java.io.IOException; +import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; 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; @@ -72,6 +74,7 @@ public void testUnbuffer() throws IOException { IOStatisticsSnapshot iostats = new IOStatisticsSnapshot(); // 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); @@ -138,6 +141,7 @@ public void testUnbufferStreamStatistics() throws IOException { Object streamStatsStr; try { inputStream = fs.open(dest); + skipIfCannotUnbuffer(inputStream); streamStatsStr = demandStringifyIOStatisticsSource(inputStream); LOG.info("initial stream statistics {}", streamStatsStr); @@ -192,6 +196,12 @@ private boolean isObjectStreamOpen(FSDataInputStream inputStream) { return ((S3AInputStream) inputStream.getWrappedStream()).isObjectStreamOpen(); } + private void skipIfCannotUnbuffer(FSDataInputStream inputStream) { + if (!inputStream.hasCapability(StreamCapabilities.UNBUFFER)) { + skip("input stream does not support unbuffer"); + } + } + /** * Read the specified number of bytes from the given * {@link FSDataInputStream} and assert that diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3PrefetchingInputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3PrefetchingInputStream.java new file mode 100644 index 0000000000000..f46e93e1084b9 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3PrefetchingInputStream.java @@ -0,0 +1,189 @@ +/* + * 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 java.net.URI; + +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.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.StoreStatisticNames; +import org.apache.hadoop.fs.statistics.StreamStatisticNames; + +import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_DEFAULT_SIZE; +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.verifyStatisticCounterValue; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticGaugeValue; +import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; + +/** + * Test the prefetching input stream, validates that the underlying S3CachingInputStream and + * S3InMemoryInputStream are working as expected. + */ +public class ITestS3PrefetchingInputStream extends AbstractS3ACostTest { + + public ITestS3PrefetchingInputStream() { + super(true); + } + + private static final Logger LOG = + LoggerFactory.getLogger(ITestS3PrefetchingInputStream.class); + + private static final int S_1K = 1024; + private static final int S_1M = S_1K * S_1K; + // Path for file which should have length > block size so S3CachingInputStream is used + private Path largeFile; + private FileSystem largeFileFS; + private int numBlocks; + private int blockSize; + private long largeFileSize; + // Size should be < block size so S3InMemoryInputStream is used + private static final int SMALL_FILE_SIZE = S_1K * 16; + + + @Override + public Configuration createConfiguration() { + Configuration conf = super.createConfiguration(); + S3ATestUtils.removeBaseAndBucketOverrides(conf, PREFETCH_ENABLED_KEY); + conf.setBoolean(PREFETCH_ENABLED_KEY, true); + return conf; + } + + @Override + public void teardown() throws Exception { + super.teardown(); + cleanupWithLogger(LOG, largeFileFS); + largeFileFS = null; + } + + private void openFS() throws Exception { + Configuration conf = getConfiguration(); + + largeFile = new Path(DEFAULT_CSVTEST_FILE); + blockSize = conf.getInt(PREFETCH_BLOCK_SIZE_KEY, PREFETCH_BLOCK_DEFAULT_SIZE); + largeFileFS = new S3AFileSystem(); + largeFileFS.initialize(new URI(DEFAULT_CSVTEST_FILE), getConfiguration()); + FileStatus fileStatus = largeFileFS.getFileStatus(largeFile); + largeFileSize = fileStatus.getLen(); + numBlocks = calculateNumBlocks(largeFileSize, blockSize); + } + + 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 S3CachingInputStream"); + IOStatistics ioStats; + openFS(); + + try (FSDataInputStream in = largeFileFS.open(largeFile)) { + 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, StreamStatisticNames.STREAM_READ_BLOCKS_IN_FILE_CACHE, + 0); + } + + // Assert that first block is read synchronously, following blocks are prefetched + verifyStatisticCounterValue(ioStats, StreamStatisticNames.STREAM_READ_PREFETCH_OPERATIONS, + numBlocks - 1); + verifyStatisticCounterValue(ioStats, StoreStatisticNames.ACTION_HTTP_GET_REQUEST, numBlocks); + verifyStatisticCounterValue(ioStats, StreamStatisticNames.STREAM_READ_OPENED, numBlocks); + } + // Verify that once stream is closed, all memory is freed + verifyStatisticGaugeValue(ioStats, StreamStatisticNames.STREAM_READ_ACTIVE_MEMORY_IN_USE, 0); + } + + @Test + public void testRandomReadLargeFile() throws Throwable { + describe("random read on a large file, uses S3CachingInputStream"); + IOStatistics ioStats; + openFS(); + + try (FSDataInputStream in = largeFileFS.open(largeFile)) { + ioStats = in.getIOStatistics(); + + byte[] buffer = new byte[blockSize]; + + // Don't read the block completely so it gets cached on seek + in.read(buffer, 0, blockSize - S_1K * 10); + in.seek(blockSize + S_1K * 10); + // Backwards seek, will use cached block + in.seek(S_1K * 5); + in.read(); + + verifyStatisticCounterValue(ioStats, StoreStatisticNames.ACTION_HTTP_GET_REQUEST, 2); + verifyStatisticCounterValue(ioStats, StreamStatisticNames.STREAM_READ_OPENED, 2); + verifyStatisticCounterValue(ioStats, StreamStatisticNames.STREAM_READ_PREFETCH_OPERATIONS, 1); + // block 0 is cached when we seek to block 1, block 1 is cached as it is being prefetched + // when we seek out of block 0, see cancelPrefetches() + verifyStatisticGaugeValue(ioStats, StreamStatisticNames.STREAM_READ_BLOCKS_IN_FILE_CACHE, 2); + } + verifyStatisticGaugeValue(ioStats, StreamStatisticNames.STREAM_READ_BLOCKS_IN_FILE_CACHE, 0); + verifyStatisticGaugeValue(ioStats, StreamStatisticNames.STREAM_READ_ACTIVE_MEMORY_IN_USE, 0); + } + + @Test + public void testRandomReadSmallFile() throws Throwable { + describe("random read on a small file, uses S3InMemoryInputStream"); + + 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 * 4); + in.seek(S_1K * 12); + in.read(buffer, 0, S_1K * 4); + + verifyStatisticCounterValue(ioStats, StoreStatisticNames.ACTION_HTTP_GET_REQUEST, 1); + verifyStatisticCounterValue(ioStats, StreamStatisticNames.STREAM_READ_OPENED, 1); + verifyStatisticCounterValue(ioStats, StreamStatisticNames.STREAM_READ_PREFETCH_OPERATIONS, 0); + // The buffer pool is not used + verifyStatisticGaugeValue(ioStats, StreamStatisticNames.STREAM_READ_ACTIVE_MEMORY_IN_USE, 0); + } + } + +} 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 6162ed13123e1..c35a03c16646c 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 @@ -40,6 +40,7 @@ import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; import org.apache.hadoop.fs.s3a.impl.StoreContext; import org.apache.hadoop.fs.s3a.impl.StoreContextBuilder; +import org.apache.hadoop.fs.s3a.read.S3PrefetchingInputStream; import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics; import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext; @@ -1468,8 +1469,16 @@ public static void skipIfEncryptionNotSet(Configuration configuration, * @return the statistics for the inner stream */ public static S3AInputStreamStatistics getInputStreamStatistics( - FSDataInputStream in) { - return getS3AInputStream(in).getS3AStreamStatistics(); + FSDataInputStream in) { + + InputStream inner = in.getWrappedStream(); + if (inner instanceof S3AInputStream) { + return ((S3AInputStream) inner).getS3AStreamStatistics(); + } else if (inner instanceof S3PrefetchingInputStream) { + return ((S3PrefetchingInputStream) inner).getS3AStreamStatistics(); + } else { + throw new AssertionError("Not an S3AInputStream or S3PrefetchingInputStream: " + inner); + } } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AUnbuffer.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AUnbuffer.java index 204f1aa09394a..0e105c25c3a45 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AUnbuffer.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AUnbuffer.java @@ -34,8 +34,8 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -74,6 +74,6 @@ public void testUnbuffer() throws IOException { stream.unbuffer(); // Verify that unbuffer closed the object stream - verify(objectStream, times(1)).close(); + verify(objectStream, atLeast(1)).close(); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/Fakes.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/Fakes.java new file mode 100644 index 0000000000000..d2a045e335eed --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/Fakes.java @@ -0,0 +1,382 @@ +/* + * 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.read; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.net.URI; +import java.nio.ByteBuffer; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; + +import com.amazonaws.services.s3.model.GetObjectRequest; +import com.amazonaws.services.s3.model.ObjectMetadata; +import com.amazonaws.services.s3.model.S3Object; +import com.amazonaws.services.s3.model.S3ObjectInputStream; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.common.BlockCache; +import org.apache.hadoop.fs.common.BlockData; +import org.apache.hadoop.fs.common.ExecutorServiceFuturePool; +import org.apache.hadoop.fs.common.SingleFilePerBlockCache; +import org.apache.hadoop.fs.common.Validate; +import org.apache.hadoop.fs.s3a.Invoker; +import org.apache.hadoop.fs.s3a.S3AEncryptionMethods; +import org.apache.hadoop.fs.s3a.S3AFileStatus; +import org.apache.hadoop.fs.s3a.S3AInputPolicy; +import org.apache.hadoop.fs.s3a.S3AInputStream; +import org.apache.hadoop.fs.s3a.S3AReadOpContext; +import org.apache.hadoop.fs.s3a.S3ObjectAttributes; +import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; +import org.apache.hadoop.fs.s3a.impl.ChangeTracker; +import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; +import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; +import org.apache.hadoop.fs.s3a.statistics.impl.CountingChangeTracker; +import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext; +import org.apache.hadoop.io.retry.RetryPolicies; +import org.apache.hadoop.io.retry.RetryPolicy; +import org.apache.hadoop.util.functional.CallableRaisingIOE; + +/** + * Provides 'fake' implementations of S3InputStream variants. + * + * These implementations avoid accessing the following real resources: + * -- S3 store + * -- local filesystem + * + * This arrangement allows thorough multi-threaded testing of those + * implementations without accessing external resources. It also helps + * avoid test flakiness introduced by external factors. + */ +public final class Fakes { + + private Fakes() {} + + public static final String E_TAG = "eTag"; + public static final String OWNER = "owner"; + public static final String VERSION_ID = "v1"; + public static final long MODIFICATION_TIME = 0L; + public static final ChangeDetectionPolicy CHANGE_POLICY = + ChangeDetectionPolicy.createPolicy( + ChangeDetectionPolicy.Mode.None, + ChangeDetectionPolicy.Source.None, + false); + + public static S3AFileStatus createFileStatus(String key, long fileSize) { + org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(key); + long blockSize = fileSize; + return new S3AFileStatus( + fileSize, MODIFICATION_TIME, path, blockSize, OWNER, E_TAG, VERSION_ID); + } + + public static S3ObjectAttributes createObjectAttributes( + String bucket, + String key, + long fileSize) { + + org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(key); + String encryptionKey = ""; + + return new S3ObjectAttributes( + bucket, + path, + key, + S3AEncryptionMethods.NONE, + encryptionKey, + E_TAG, + VERSION_ID, + fileSize); + } + + public static S3AReadOpContext createReadContext( + ExecutorServiceFuturePool futurePool, + String key, + int fileSize, + int prefetchBlockSize, + int prefetchBlockCount) { + + S3AFileStatus fileStatus = createFileStatus(key, fileSize); + org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(key); + FileSystem.Statistics statistics = new FileSystem.Statistics("s3a"); + S3AStatisticsContext statisticsContext = new EmptyS3AStatisticsContext(); + RetryPolicy retryPolicy = + RetryPolicies.retryUpToMaximumCountWithFixedSleep(3, 10, TimeUnit.MILLISECONDS); + + return new S3AReadOpContext( + path, + new Invoker(retryPolicy, Invoker.LOG_EVENT), + statistics, + statisticsContext, + fileStatus, + futurePool, + prefetchBlockSize, + prefetchBlockCount) + .withChangeDetectionPolicy( + ChangeDetectionPolicy.createPolicy(ChangeDetectionPolicy.Mode.None, + ChangeDetectionPolicy.Source.ETag, false)) + .withInputPolicy(S3AInputPolicy.Normal); + } + + public static URI createUri(String bucket, String key) { + return URI.create(String.format("s3a://%s/%s", bucket, key)); + } + + public static ChangeTracker createChangeTracker( + String bucket, + String key, + long fileSize) { + + return new ChangeTracker( + createUri(bucket, key).toString(), + CHANGE_POLICY, + new CountingChangeTracker(), + createObjectAttributes(bucket, key, fileSize)); + } + + public static S3ObjectInputStream createS3ObjectInputStream(byte[] buffer) { + return new S3ObjectInputStream(new ByteArrayInputStream(buffer), null); + } + + public static S3AInputStream.InputStreamCallbacks createInputStreamCallbacks( + String bucket, + String key) { + + S3Object object = new S3Object() { + @Override + public S3ObjectInputStream getObjectContent() { + return createS3ObjectInputStream(new byte[8]); + } + + @Override + public ObjectMetadata getObjectMetadata() { + ObjectMetadata metadata = new ObjectMetadata(); + metadata.setHeader("ETag", E_TAG); + return metadata; + } + }; + + return new S3AInputStream.InputStreamCallbacks() { + @Override + public S3Object getObject(GetObjectRequest request) { + return object; + } + + @Override + public CompletableFuture submit(CallableRaisingIOE operation) { + return null; + } + + @Override + public GetObjectRequest newGetRequest(String key) { + return new GetObjectRequest(bucket, key); + } + + @Override + public void close() { + } + }; + } + + + public static S3InputStream createInputStream( + Class clazz, + ExecutorServiceFuturePool futurePool, + String bucket, + String key, + int fileSize, + int prefetchBlockSize, + int prefetchBlockCount) { + + org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(key); + + S3AFileStatus fileStatus = createFileStatus(key, fileSize); + S3ObjectAttributes s3ObjectAttributes = createObjectAttributes(bucket, key, fileSize); + S3AReadOpContext s3AReadOpContext = createReadContext( + futurePool, + key, + fileSize, + prefetchBlockSize, + prefetchBlockCount); + + S3AInputStream.InputStreamCallbacks callbacks = createInputStreamCallbacks(bucket, key); + S3AInputStreamStatistics stats = + s3AReadOpContext.getS3AStatisticsContext().newInputStreamStatistics(); + + if (clazz == TestS3InMemoryInputStream.class) { + return new TestS3InMemoryInputStream(s3AReadOpContext, s3ObjectAttributes, callbacks, stats); + } else if (clazz == TestS3CachingInputStream.class) { + return new TestS3CachingInputStream(s3AReadOpContext, s3ObjectAttributes, callbacks, stats); + } + + throw new RuntimeException("Unsupported class: " + clazz); + } + + public static TestS3InMemoryInputStream createS3InMemoryInputStream( + ExecutorServiceFuturePool futurePool, + String bucket, + String key, + int fileSize) { + + return (TestS3InMemoryInputStream) createInputStream( + TestS3InMemoryInputStream.class, futurePool, bucket, key, fileSize, 1, 1); + } + + public static TestS3CachingInputStream createS3CachingInputStream( + ExecutorServiceFuturePool futurePool, + String bucket, + String key, + int fileSize, + int prefetchBlockSize, + int prefetchBlockCount) { + + return (TestS3CachingInputStream) createInputStream( + TestS3CachingInputStream.class, + futurePool, + bucket, + key, + fileSize, + prefetchBlockSize, + prefetchBlockCount); + } + + public static class TestS3InMemoryInputStream extends S3InMemoryInputStream { + public TestS3InMemoryInputStream( + S3AReadOpContext context, + S3ObjectAttributes s3Attributes, + S3AInputStream.InputStreamCallbacks client, + S3AInputStreamStatistics streamStatistics) { + super(context, s3Attributes, client, streamStatistics); + } + + @Override + protected S3File getS3File() { + randomDelay(200); + return new MockS3File((int) this.getS3ObjectAttributes().getLen(), false); + } + } + + public static class TestS3FilePerBlockCache extends SingleFilePerBlockCache { + private final Map files; + private final int readDelay; + private final int writeDelay; + + public TestS3FilePerBlockCache(int readDelay, int writeDelay) { + super(new EmptyS3AStatisticsContext().newInputStreamStatistics()); + this.files = new ConcurrentHashMap<>(); + this.readDelay = readDelay; + this.writeDelay = writeDelay; + } + + @Override + protected int readFile(Path path, ByteBuffer buffer) { + byte[] source = this.files.get(path); + randomDelay(this.readDelay); + buffer.put(source); + return source.length; + } + + @Override + protected void writeFile(Path path, ByteBuffer buffer) throws IOException { + Validate.checkPositiveInteger(buffer.limit(), "buffer.limit()"); + byte[] dest = new byte[buffer.limit()]; + randomDelay(this.writeDelay); + buffer.rewind(); + buffer.get(dest); + this.files.put(path, dest); + } + + private long fileCount = 0; + + @Override + protected Path getCacheFilePath() throws IOException { + fileCount++; + return Paths.get(Long.toString(fileCount)); + } + + @Override + public void close() throws IOException { + this.files.clear(); + } + } + + private static final Random RANDOM = new Random(); + + private static void randomDelay(int delay) { + try { + Thread.sleep(RANDOM.nextInt(delay)); + } catch (InterruptedException e) { + + } + } + + public static class TestS3CachingBlockManager extends S3CachingBlockManager { + public TestS3CachingBlockManager( + ExecutorServiceFuturePool futurePool, + S3Reader reader, + BlockData blockData, + int bufferPoolSize) { + super(futurePool, reader, blockData, bufferPoolSize, + new EmptyS3AStatisticsContext().newInputStreamStatistics()); + } + + @Override + public int read(ByteBuffer buffer, long offset, int size) throws IOException { + randomDelay(100); + return this.getReader().read(buffer, offset, size); + } + + @Override + protected BlockCache createCache() { + final int readDelayMs = 50; + final int writeDelayMs = 200; + return new TestS3FilePerBlockCache(readDelayMs, writeDelayMs); + } + } + + public static class TestS3CachingInputStream extends S3CachingInputStream { + public TestS3CachingInputStream( + S3AReadOpContext context, + S3ObjectAttributes s3Attributes, + S3AInputStream.InputStreamCallbacks client, + S3AInputStreamStatistics streamStatistics) { + super(context, s3Attributes, client, streamStatistics); + } + + @Override + protected S3File getS3File() { + randomDelay(200); + return new MockS3File((int) this.getS3ObjectAttributes().getLen(), false); + } + + @Override + protected S3CachingBlockManager createBlockManager( + ExecutorServiceFuturePool futurePool, + S3Reader reader, + BlockData blockData, + int bufferPoolSize) { + return new TestS3CachingBlockManager(futurePool, reader, blockData, bufferPoolSize); + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/MockS3File.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/MockS3File.java new file mode 100644 index 0000000000000..82b7a10d40c18 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/MockS3File.java @@ -0,0 +1,112 @@ +/* + * 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.read; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.concurrent.CompletableFuture; + +import com.amazonaws.services.s3.model.GetObjectRequest; +import com.amazonaws.services.s3.model.S3Object; + +import org.apache.hadoop.fs.common.Validate; +import org.apache.hadoop.fs.s3a.S3AInputStream; +import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext; +import org.apache.hadoop.util.functional.CallableRaisingIOE; + +/** + * A mock s3 file with some fault injection. + */ +class MockS3File extends S3File { + private byte[] contents; + + // If true, throws IOException on open request just once. + // That allows test code to validate behavior related to retries. + private boolean throwExceptionOnOpen; + + private static final String BUCKET = "bucket"; + private static final String KEY = "key"; + + MockS3File(int size) { + this(size, false); + } + + MockS3File(int size, boolean throwExceptionOnOpen) { + super( + Fakes.createReadContext(null, KEY, size, 1, 1), + Fakes.createObjectAttributes(BUCKET, KEY, size), + Fakes.createInputStreamCallbacks(BUCKET, KEY), + new EmptyS3AStatisticsContext().EMPTY_INPUT_STREAM_STATISTICS, + Fakes.createChangeTracker(BUCKET, KEY, size) + ); + + this.throwExceptionOnOpen = throwExceptionOnOpen; + this.contents = new byte[size]; + for (int b = 0; b < size; b++) { + this.contents[b] = byteAtOffset(b); + } + } + + @Override + public InputStream openForRead(long offset, int size) throws IOException { + Validate.checkLessOrEqual(offset, "offset", size(), "size()"); + Validate.checkLessOrEqual(size, "size", size() - offset, "size() - offset"); + + if (this.throwExceptionOnOpen) { + this.throwExceptionOnOpen = false; + throw new IOException("Throwing because throwExceptionOnOpen is true "); + } + int bufSize = (int) Math.min(size, size() - offset); + return new ByteArrayInputStream(contents, (int) offset, bufSize); + } + + @Override + public void close(InputStream inputStream, int numRemainingBytes) { + // do nothing since we do not use a real S3 stream. + } + + public static byte byteAtOffset(int offset) { + return (byte) (offset % 128); + } + + public static S3AInputStream.InputStreamCallbacks createClient(String bucketName) { + return new S3AInputStream.InputStreamCallbacks() { + @Override + public S3Object getObject(GetObjectRequest request) { + return null; + } + + @Override + public CompletableFuture submit(CallableRaisingIOE operation) { + return null; + } + + @Override + public GetObjectRequest newGetRequest(String key) { + return new GetObjectRequest(bucketName, key); + } + + @Override + public void close() { + } + }; + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3BlockManager.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3BlockManager.java new file mode 100644 index 0000000000000..eb3b700f2800e --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3BlockManager.java @@ -0,0 +1,86 @@ +/* + * 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.read; + +import java.io.IOException; +import java.nio.ByteBuffer; + +import org.junit.Test; + +import org.apache.hadoop.fs.common.BlockData; +import org.apache.hadoop.fs.common.BufferData; +import org.apache.hadoop.fs.common.ExceptionAsserts; +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.junit.Assert.assertEquals; + +public class TestS3BlockManager extends AbstractHadoopTestBase { + + static final int FILE_SIZE = 12; + static final int BLOCK_SIZE = 3; + + @Test + public void testArgChecks() throws Exception { + BlockData blockData = new BlockData(FILE_SIZE, BLOCK_SIZE); + MockS3File s3File = new MockS3File(FILE_SIZE, false); + S3Reader reader = new S3Reader(s3File); + + // Should not throw. + new S3BlockManager(reader, blockData); + + // Verify it throws correctly. + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'reader' must not be null", + () -> new S3BlockManager(null, blockData)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'blockData' must not be null", + () -> new S3BlockManager(reader, null)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'blockNumber' must not be negative", + () -> new S3BlockManager(reader, blockData).get(-1)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'data' must not be null", + () -> new S3BlockManager(reader, blockData).release(null)); + } + + @Test + public void testGet() throws IOException { + BlockData blockData = new BlockData(FILE_SIZE, BLOCK_SIZE); + MockS3File s3File = new MockS3File(FILE_SIZE, false); + S3Reader reader = new S3Reader(s3File); + S3BlockManager blockManager = new S3BlockManager(reader, blockData); + + for (int b = 0; b < blockData.getNumBlocks(); b++) { + BufferData data = blockManager.get(b); + ByteBuffer buffer = data.getBuffer(); + long startOffset = blockData.getStartOffset(b); + for (int i = 0; i < BLOCK_SIZE; i++) { + assertEquals(startOffset + i, buffer.get()); + } + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3CachingBlockManager.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3CachingBlockManager.java new file mode 100644 index 0000000000000..a9ebae276f3ff --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3CachingBlockManager.java @@ -0,0 +1,337 @@ +/* + * 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.read; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import org.junit.Test; + +import org.apache.hadoop.fs.common.BlockData; +import org.apache.hadoop.fs.common.BufferData; +import org.apache.hadoop.fs.common.ExceptionAsserts; +import org.apache.hadoop.fs.common.ExecutorServiceFuturePool; +import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; +import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext; +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.junit.Assert.assertEquals; + +public class TestS3CachingBlockManager extends AbstractHadoopTestBase { + 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 final ExecutorServiceFuturePool futurePool = new ExecutorServiceFuturePool(threadPool); + private final S3AInputStreamStatistics streamStatistics = + new EmptyS3AStatisticsContext().newInputStreamStatistics(); + + private final BlockData blockData = new BlockData(FILE_SIZE, BLOCK_SIZE); + + @Test + public void testArgChecks() throws Exception { + MockS3File s3File = new MockS3File(FILE_SIZE, false); + S3Reader reader = new S3Reader(s3File); + + // Should not throw. + S3CachingBlockManager blockManager = + new S3CachingBlockManager(futurePool, reader, blockData, POOL_SIZE, streamStatistics); + + // Verify it throws correctly. + ExceptionAsserts.assertThrows( + NullPointerException.class, + () -> new S3CachingBlockManager(null, reader, blockData, POOL_SIZE, streamStatistics)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'reader' must not be null", + () -> new S3CachingBlockManager(futurePool, null, blockData, POOL_SIZE, streamStatistics)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'blockData' must not be null", + () -> new S3CachingBlockManager(futurePool, reader, null, POOL_SIZE, streamStatistics)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'bufferPoolSize' must be a positive integer", + () -> new S3CachingBlockManager(futurePool, reader, blockData, 0, streamStatistics)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'bufferPoolSize' must be a positive integer", + () -> new S3CachingBlockManager(futurePool, reader, blockData, -1, streamStatistics)); + + ExceptionAsserts.assertThrows(NullPointerException.class, + () -> new S3CachingBlockManager(futurePool, reader, blockData, POOL_SIZE, null)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'blockNumber' must not be negative", + () -> blockManager.get(-1)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'data' must not be null", + () -> blockManager.release(null)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'blockNumber' must not be negative", + () -> blockManager.requestPrefetch(-1)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'data' must not be null", + () -> blockManager.requestCaching(null)); + } + + /** + * Extends S3CachingBlockManager so that we can inject asynchronous failures. + */ + static class TestBlockManager extends S3CachingBlockManager { + TestBlockManager( + ExecutorServiceFuturePool futurePool, + S3Reader reader, + BlockData blockData, + int bufferPoolSize, + S3AInputStreamStatistics streamStatistics) { + super(futurePool, reader, blockData, bufferPoolSize, streamStatistics); + } + + // If true, forces the next read operation to fail. + // Resets itself to false after one failure. + private boolean forceNextReadToFail; + + @Override + public int read(ByteBuffer buffer, long offset, int size) throws IOException { + if (forceNextReadToFail) { + forceNextReadToFail = false; + throw new RuntimeException("foo"); + } else { + return super.read(buffer, offset, size); + } + } + + // If true, forces the next cache-put operation to fail. + // Resets itself to false after one failure. + private boolean forceNextCachePutToFail; + + @Override + protected void cachePut(int blockNumber, ByteBuffer buffer) throws IOException { + if (forceNextCachePutToFail) { + forceNextCachePutToFail = false; + throw new RuntimeException("bar"); + } else { + super.cachePut(blockNumber, buffer); + } + } + } + + // @Ignore + @Test + public void testGet() throws Exception { + testGetHelper(false); + } + + // @Ignore + @Test + public void testGetFailure() throws Exception { + testGetHelper(true); + } + + private void testGetHelper(boolean forceReadFailure) throws Exception { + MockS3File s3File = new MockS3File(FILE_SIZE, true); + S3Reader reader = new S3Reader(s3File); + TestBlockManager blockManager = + new TestBlockManager(futurePool, reader, blockData, POOL_SIZE, streamStatistics); + + for (int b = 0; b < blockData.getNumBlocks(); b++) { + // We simulate caching failure for all even numbered blocks. + boolean forceFailure = forceReadFailure && (b % 2 == 0); + + BufferData data = null; + + if (forceFailure) { + blockManager.forceNextReadToFail = true; + + ExceptionAsserts.assertThrows( + RuntimeException.class, + "foo", + () -> blockManager.get(3)); + } else { + data = blockManager.get(b); + + long startOffset = blockData.getStartOffset(b); + for (int i = 0; i < blockData.getSize(b); i++) { + assertEquals(startOffset + i, data.getBuffer().get()); + } + + blockManager.release(data); + } + + assertEquals(POOL_SIZE, blockManager.numAvailable()); + } + } + + // @Ignore + @Test + public void testPrefetch() throws IOException, InterruptedException { + testPrefetchHelper(false); + } + + // @Ignore + @Test + public void testPrefetchFailure() throws IOException, InterruptedException { + testPrefetchHelper(true); + } + + private void testPrefetchHelper(boolean forcePrefetchFailure) + throws IOException, InterruptedException { + MockS3File s3File = new MockS3File(FILE_SIZE, false); + S3Reader reader = new S3Reader(s3File); + TestBlockManager blockManager = + new TestBlockManager(futurePool, reader, blockData, POOL_SIZE, streamStatistics); + assertInitialState(blockManager); + + int expectedNumErrors = 0; + int expectedNumSuccesses = 0; + + for (int b = 0; b < POOL_SIZE; b++) { + // We simulate caching failure for all odd numbered blocks. + boolean forceFailure = forcePrefetchFailure && (b % 2 == 1); + if (forceFailure) { + expectedNumErrors++; + blockManager.forceNextReadToFail = true; + } else { + expectedNumSuccesses++; + } + blockManager.requestPrefetch(b); + } + + assertEquals(0, blockManager.numCached()); + + blockManager.cancelPrefetches(); + waitForCaching(blockManager, expectedNumSuccesses); + assertEquals(expectedNumErrors, this.totalErrors(blockManager)); + assertEquals(expectedNumSuccesses, blockManager.numCached()); + } + + // @Ignore + @Test + public void testCachingOfPrefetched() throws IOException, InterruptedException { + MockS3File s3File = new MockS3File(FILE_SIZE, false); + S3Reader reader = new S3Reader(s3File); + S3CachingBlockManager blockManager = + new S3CachingBlockManager(futurePool, reader, blockData, POOL_SIZE, streamStatistics); + assertInitialState(blockManager); + + for (int b = 0; b < blockData.getNumBlocks(); b++) { + blockManager.requestPrefetch(b); + BufferData data = blockManager.get(b); + blockManager.requestCaching(data); + } + + waitForCaching(blockManager, blockData.getNumBlocks()); + assertEquals(blockData.getNumBlocks(), blockManager.numCached()); + assertEquals(0, this.totalErrors(blockManager)); + } + + // @Ignore + @Test + public void testCachingOfGet() throws IOException, InterruptedException { + testCachingOfGetHelper(false); + } + + // @Ignore + @Test + public void testCachingFailureOfGet() throws IOException, InterruptedException { + testCachingOfGetHelper(true); + } + + public void testCachingOfGetHelper(boolean forceCachingFailure) + throws IOException, InterruptedException { + MockS3File s3File = new MockS3File(FILE_SIZE, false); + S3Reader reader = new S3Reader(s3File); + TestBlockManager blockManager = + new TestBlockManager(futurePool, reader, blockData, POOL_SIZE, streamStatistics); + assertInitialState(blockManager); + + int expectedNumErrors = 0; + int expectedNumSuccesses = 0; + + for (int b = 0; b < blockData.getNumBlocks(); b++) { + // We simulate caching failure for all odd numbered blocks. + boolean forceFailure = forceCachingFailure && (b % 2 == 1); + if (forceFailure) { + expectedNumErrors++; + } else { + expectedNumSuccesses++; + } + + BufferData data = blockManager.get(b); + if (forceFailure) { + blockManager.forceNextCachePutToFail = true; + } + + blockManager.requestCaching(data); + waitForCaching(blockManager, expectedNumSuccesses); + assertEquals(expectedNumSuccesses, blockManager.numCached()); + + if (forceCachingFailure) { + assertEquals(expectedNumErrors, this.totalErrors(blockManager)); + } else { + assertEquals(0, this.totalErrors(blockManager)); + } + } + } + + private void waitForCaching( + S3CachingBlockManager blockManager, + int expectedCount) + throws InterruptedException { + // Wait for async cache operation to be over. + int numTrys = 0; + int count; + do { + Thread.sleep(100); + count = blockManager.numCached(); + numTrys++; + if (numTrys > 600) { + String message = String.format( + "waitForCaching: expected: %d, actual: %d, read errors: %d, caching errors: %d", + expectedCount, count, blockManager.numReadErrors(), blockManager.numCachingErrors()); + throw new IllegalStateException(message); + } + } + while (count < expectedCount); + } + + private int totalErrors(S3CachingBlockManager blockManager) { + return blockManager.numCachingErrors() + blockManager.numReadErrors(); + } + + private void assertInitialState(S3CachingBlockManager blockManager) { + assertEquals(0, blockManager.numCached()); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3File.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3File.java new file mode 100644 index 0000000000000..9f63ea0a889fd --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3File.java @@ -0,0 +1,77 @@ +/* + * 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.read; + +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import org.junit.Test; + +import org.apache.hadoop.fs.common.ExceptionAsserts; +import org.apache.hadoop.fs.common.ExecutorServiceFuturePool; +import org.apache.hadoop.fs.s3a.S3AInputStream; +import org.apache.hadoop.fs.s3a.S3AReadOpContext; +import org.apache.hadoop.fs.s3a.S3ObjectAttributes; +import org.apache.hadoop.fs.s3a.impl.ChangeTracker; +import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; +import org.apache.hadoop.test.AbstractHadoopTestBase; + +public class TestS3File extends AbstractHadoopTestBase { + private final ExecutorService threadPool = Executors.newFixedThreadPool(1); + private final ExecutorServiceFuturePool futurePool = new ExecutorServiceFuturePool(threadPool); + private final S3AInputStream.InputStreamCallbacks client = MockS3File.createClient("bucket"); + + @Test + public void testArgChecks() throws Exception { + S3AReadOpContext readContext = Fakes.createReadContext(futurePool, "key", 10, 10, 1); + S3ObjectAttributes attrs = Fakes.createObjectAttributes("bucket", "key", 10); + S3AInputStreamStatistics stats = + readContext.getS3AStatisticsContext().newInputStreamStatistics(); + ChangeTracker changeTracker = Fakes.createChangeTracker("bucket", "key", 10); + + // Should not throw. + new S3File(readContext, attrs, client, stats, changeTracker); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'context' must not be null", + () -> new S3File(null, attrs, client, stats, changeTracker)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'s3Attributes' must not be null", + () -> new S3File(readContext, null, client, stats, changeTracker)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'client' must not be null", + () -> new S3File(readContext, attrs, null, stats, changeTracker)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'streamStatistics' must not be null", + () -> new S3File(readContext, attrs, client, null, changeTracker)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'changeTracker' must not be null", + () -> new S3File(readContext, attrs, client, stats, null)); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3InputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3InputStream.java new file mode 100644 index 0000000000000..cf3ad400afe80 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3InputStream.java @@ -0,0 +1,251 @@ +/* + * 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.read; + +import java.io.EOFException; +import java.io.IOException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import org.junit.Test; + +import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.fs.common.ExceptionAsserts; +import org.apache.hadoop.fs.common.ExecutorServiceFuturePool; +import org.apache.hadoop.fs.s3a.S3AInputStream; +import org.apache.hadoop.fs.s3a.S3AReadOpContext; +import org.apache.hadoop.fs.s3a.S3ObjectAttributes; +import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.junit.Assert.assertEquals; + +/** + * Applies the same set of tests to both S3CachingInputStream and S3InMemoryInputStream. + */ +public class TestS3InputStream extends AbstractHadoopTestBase { + + private static final int FILE_SIZE = 10; + + private final ExecutorService threadPool = Executors.newFixedThreadPool(4); + private final ExecutorServiceFuturePool futurePool = new ExecutorServiceFuturePool(threadPool); + private final S3AInputStream.InputStreamCallbacks client = MockS3File.createClient("bucket"); + + @Test + public void testArgChecks() throws Exception { + S3AReadOpContext readContext = Fakes.createReadContext(futurePool, "key", 10, 10, 1); + S3ObjectAttributes attrs = Fakes.createObjectAttributes("bucket", "key", 10); + S3AInputStreamStatistics stats = + readContext.getS3AStatisticsContext().newInputStreamStatistics(); + + // Should not throw. + new S3CachingInputStream(readContext, attrs, client, stats); + + ExceptionAsserts.assertThrows( + NullPointerException.class, + () -> new S3CachingInputStream(null, attrs, client, stats)); + + ExceptionAsserts.assertThrows( + NullPointerException.class, + () -> new S3CachingInputStream(readContext, null, client, stats)); + + ExceptionAsserts.assertThrows( + NullPointerException.class, + () -> new S3CachingInputStream(readContext, attrs, null, stats)); + + ExceptionAsserts.assertThrows( + NullPointerException.class, + () -> new S3CachingInputStream(readContext, attrs, client, null)); + } + + @Test + public void testRead0SizedFile() throws Exception { + S3InputStream inputStream = + Fakes.createS3InMemoryInputStream(futurePool, "bucket", "key", 0); + testRead0SizedFileHelper(inputStream, 9); + + inputStream = Fakes.createS3CachingInputStream(futurePool, "bucket", "key", 0, 5, 2); + testRead0SizedFileHelper(inputStream, 5); + } + + private void testRead0SizedFileHelper(S3InputStream inputStream, int bufferSize) + throws Exception { + assertEquals(0, inputStream.available()); + assertEquals(-1, inputStream.read()); + assertEquals(-1, inputStream.read()); + + byte[] buffer = new byte[2]; + assertEquals(-1, inputStream.read(buffer)); + assertEquals(-1, inputStream.read()); + } + + @Test + public void testRead() throws Exception { + S3InputStream inputStream = + Fakes.createS3InMemoryInputStream(futurePool, "bucket", "key", FILE_SIZE); + testReadHelper(inputStream, FILE_SIZE); + + inputStream = + Fakes.createS3CachingInputStream(futurePool, "bucket", "key", FILE_SIZE, 5, 2); + testReadHelper(inputStream, 5); + } + + private void testReadHelper(S3InputStream inputStream, int bufferSize) throws Exception { + assertEquals(bufferSize, inputStream.available()); + assertEquals(0, inputStream.read()); + assertEquals(1, inputStream.read()); + + byte[] buffer = new byte[2]; + assertEquals(2, inputStream.read(buffer)); + assertEquals(2, buffer[0]); + assertEquals(3, buffer[1]); + + assertEquals(4, inputStream.read()); + + buffer = new byte[10]; + int curPos = (int) inputStream.getPos(); + int expectedRemainingBytes = (int) (FILE_SIZE - curPos); + int readStartOffset = 2; + assertEquals( + expectedRemainingBytes, + inputStream.read(buffer, readStartOffset, expectedRemainingBytes)); + + for (int i = 0; i < expectedRemainingBytes; i++) { + assertEquals(curPos + i, buffer[readStartOffset + i]); + } + + assertEquals(-1, inputStream.read()); + Thread.sleep(100); + assertEquals(-1, inputStream.read()); + assertEquals(-1, inputStream.read()); + assertEquals(-1, inputStream.read(buffer)); + assertEquals(-1, inputStream.read(buffer, 1, 3)); + } + + @Test + public void testSeek() throws Exception { + S3InputStream inputStream; + inputStream = Fakes.createS3InMemoryInputStream(futurePool, "bucket", "key", 9); + testSeekHelper(inputStream, 9, 9); + + inputStream = Fakes.createS3CachingInputStream(futurePool, "bucket", "key", 9, 5, 1); + testSeekHelper(inputStream, 5, 9); + } + + private void testSeekHelper(S3InputStream inputStream, int bufferSize, int fileSize) + throws Exception { + assertEquals(0, inputStream.getPos()); + inputStream.seek(7); + assertEquals(7, inputStream.getPos()); + inputStream.seek(0); + + assertEquals(bufferSize, inputStream.available()); + for (int i = 0; i < fileSize; i++) { + assertEquals(i, inputStream.read()); + } + + for (int i = 0; i < fileSize; i++) { + inputStream.seek(i); + for (int j = i; j < fileSize; j++) { + assertEquals(j, inputStream.read()); + } + } + + // Test invalid seeks. + ExceptionAsserts.assertThrows( + EOFException.class, + FSExceptionMessages.NEGATIVE_SEEK, + () -> inputStream.seek(-1)); + } + + @Test + public void testRandomSeek() throws Exception { + S3InputStream inputStream; + inputStream = Fakes.createS3InMemoryInputStream(futurePool, "bucket", "key", 9); + testRandomSeekHelper(inputStream, 9, 9); + + inputStream = Fakes.createS3CachingInputStream(futurePool, "bucket", "key", 9, 5, 1); + testRandomSeekHelper(inputStream, 5, 9); + } + + private void testRandomSeekHelper(S3InputStream inputStream, int bufferSize, int fileSize) + throws Exception { + assertEquals(0, inputStream.getPos()); + inputStream.seek(7); + assertEquals(7, inputStream.getPos()); + inputStream.seek(0); + + assertEquals(bufferSize, inputStream.available()); + for (int i = 0; i < fileSize; i++) { + assertEquals(i, inputStream.read()); + } + + for (int i = 0; i < fileSize; i++) { + inputStream.seek(i); + for (int j = i; j < fileSize; j++) { + assertEquals(j, inputStream.read()); + } + + int seekFromEndPos = fileSize - i - 1; + inputStream.seek(seekFromEndPos); + for (int j = seekFromEndPos; j < fileSize; j++) { + assertEquals(j, inputStream.read()); + } + } + } + + @Test + public void testClose() throws Exception { + S3InputStream inputStream = + Fakes.createS3InMemoryInputStream(futurePool, "bucket", "key", 9); + testCloseHelper(inputStream, 9); + + inputStream = + Fakes.createS3CachingInputStream(futurePool, "bucket", "key", 9, 5, 3); + testCloseHelper(inputStream, 5); + } + + private void testCloseHelper(S3InputStream inputStream, int bufferSize) throws Exception { + assertEquals(bufferSize, inputStream.available()); + assertEquals(0, inputStream.read()); + assertEquals(1, inputStream.read()); + + inputStream.close(); + + ExceptionAsserts.assertThrows( + IOException.class, + FSExceptionMessages.STREAM_IS_CLOSED, + () -> inputStream.available()); + + ExceptionAsserts.assertThrows( + IOException.class, + FSExceptionMessages.STREAM_IS_CLOSED, + () -> inputStream.read()); + + byte[] buffer = new byte[10]; + ExceptionAsserts.assertThrows( + IOException.class, + FSExceptionMessages.STREAM_IS_CLOSED, + () -> inputStream.read(buffer)); + + // Verify a second close() does not throw. + inputStream.close(); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3Reader.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3Reader.java new file mode 100644 index 0000000000000..10e5e29da2d3c --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3Reader.java @@ -0,0 +1,106 @@ +/* + * 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.read; + +import java.nio.ByteBuffer; + +import org.junit.Test; + +import org.apache.hadoop.fs.common.ExceptionAsserts; +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.junit.Assert.assertEquals; + +public class TestS3Reader extends AbstractHadoopTestBase { + + private static final int FILE_SIZE = 9; + private static final int BUFFER_SIZE = 2; + private final S3File s3File = new MockS3File(FILE_SIZE, false); + + @Test + public void testArgChecks() throws Exception { + // Should not throw. + S3Reader reader = new S3Reader(s3File); + + // Verify it throws correctly. + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'s3File' must not be null", + () -> new S3Reader(null)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'buffer' must not be null", + () -> reader.read(null, 10, 2)); + + ByteBuffer buffer = ByteBuffer.allocate(BUFFER_SIZE); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'offset' (-1) must be within the range [0, 9]", + () -> reader.read(buffer, -1, 2)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'offset' (11) must be within the range [0, 9]", + () -> reader.read(buffer, 11, 2)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'size' must be a positive integer", + () -> reader.read(buffer, 1, 0)); + + ExceptionAsserts.assertThrows( + IllegalArgumentException.class, + "'size' must be a positive integer", + () -> reader.read(buffer, 1, -1)); + } + + @Test + public void testGetWithOffset() throws Exception { + for (int i = 0; i < FILE_SIZE; i++) { + testGetHelper(false, i); // no retry + testGetHelper(true, i); // with retry + } + } + + private void testGetHelper(boolean testWithRetry, long startOffset) + throws Exception { + int numBlocks = 0; + ByteBuffer buffer; + S3Reader reader = new S3Reader(new MockS3File(FILE_SIZE, testWithRetry)); + int remainingSize = FILE_SIZE - (int) startOffset; + for (int bufferSize = 0; bufferSize <= FILE_SIZE + 1; bufferSize++) { + buffer = ByteBuffer.allocate(bufferSize); + for (int readSize = 1; readSize <= FILE_SIZE; readSize++) { + buffer.clear(); + int numBytesRead = reader.read(buffer, startOffset, readSize); + int expectedNumBytesRead = Math.min(readSize, remainingSize); + expectedNumBytesRead = Math.min(bufferSize, expectedNumBytesRead); + assertEquals(expectedNumBytesRead, numBytesRead); + + byte[] bytes = buffer.array(); + for (int i = 0; i< expectedNumBytesRead; i++) { + assertEquals(startOffset + i, bytes[i]); + } + } + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java index b8195cb9964fa..eea70ced13c92 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3AInputPolicy; import org.apache.hadoop.fs.s3a.S3AInputStream; +import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; @@ -94,6 +95,14 @@ public class ITestS3AInputStreamPerformance extends S3AScaleTestBase { private boolean testDataAvailable = true; private String assumptionMessage = "test file"; + @Override + protected Configuration createScaleConfiguration() { + Configuration conf = super.createScaleConfiguration(); + S3ATestUtils.removeBaseAndBucketOverrides(conf, PREFETCH_ENABLED_KEY); + conf.setBoolean(PREFETCH_ENABLED_KEY, false); + return conf; + } + /** * Open the FS and the test data. The input stream is always set up here. * @throws IOException IO Problems.