From 5af02c5881f01f9cc89f3ea73dba4f643b616cab Mon Sep 17 00:00:00 2001 From: Anuj Modi Date: Mon, 13 Nov 2023 03:27:45 -0800 Subject: [PATCH 1/8] Code Changes to enable footer optimization with new buffer size --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 9 +++++ .../fs/azurebfs/AzureBlobFileSystemStore.java | 1 + .../azurebfs/constants/ConfigurationKeys.java | 13 +++++++ .../constants/FileSystemConfigurations.java | 1 + .../fs/azurebfs/services/AbfsInputStream.java | 11 +++++- .../services/AbfsInputStreamContext.java | 11 ++++++ .../ITestAbfsInputStreamReadFooter.java | 37 +++++++++++-------- 7 files changed, 65 insertions(+), 18 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 0bcb97a84969a..5bd935e516b54 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -117,6 +117,11 @@ public class AbfsConfiguration{ DefaultValue = DEFAULT_OPTIMIZE_FOOTER_READ) private boolean optimizeFooterRead; + @IntegerConfigurationValidatorAnnotation( + ConfigurationKey = AZURE_FOOTER_READ_BUFFER_SIZE, + DefaultValue = DEFAULT_FOOTER_READ_BUFFER_SIZE) + private int footerReadBufferSize; + @BooleanConfigurationValidatorAnnotation( ConfigurationKey = FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED, DefaultValue = DEFAULT_FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED) @@ -643,6 +648,10 @@ public boolean optimizeFooterRead() { return this.optimizeFooterRead; } + public int getFooterReadBufferSize() { + return this.footerReadBufferSize; + } + public int getReadBufferSize() { return this.readBufferSize; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 4b356ceef06db..b998d4f0b8163 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -818,6 +818,7 @@ private AbfsInputStreamContext populateAbfsInputStreamContext( .isReadAheadEnabled(abfsConfiguration.isReadAheadEnabled()) .withReadSmallFilesCompletely(abfsConfiguration.readSmallFilesCompletely()) .withOptimizeFooterRead(abfsConfiguration.optimizeFooterRead()) + .withFooterReadBufferSize(abfsConfiguration.getFooterReadBufferSize()) .withReadAheadRange(abfsConfiguration.getReadAheadRange()) .withStreamStatistics(new AbfsInputStreamStatisticsImpl()) .withShouldReadBufferSizeAlways( diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java index 461e43a9f7e75..bd66d7266a30d 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java @@ -104,7 +104,20 @@ public final class ConfigurationKeys { public static final String AZURE_ENABLE_SMALL_WRITE_OPTIMIZATION = "fs.azure.write.enableappendwithflush"; public static final String AZURE_READ_BUFFER_SIZE = "fs.azure.read.request.size"; public static final String AZURE_READ_SMALL_FILES_COMPLETELY = "fs.azure.read.smallfilescompletely"; + /** + * When parquet files are read, first few read are metadata reads before reading the actual data. + * First the read is done of last 8 bytes of parquet file to get the postion of metadta and next read + * is done for reading that metadata. With this optimizations these two reads can be combined into 1. + * Value: {@value} + */ public static final String AZURE_READ_OPTIMIZE_FOOTER_READ = "fs.azure.read.optimizefooterread"; + /** + * In case of footer reads it was not required to read full buffer size. + * Most of the metadata information required was within 256KB and it will be more performant to read lesser. + * This config is used to define how much footer length the user wants to read. + * Value: {@value} + */ + public static final String AZURE_FOOTER_READ_BUFFER_SIZE = "fs.azure.footer.read.request.size"; /** * Read ahead range parameter which can be set by user. diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java index 00fc4a6a3db77..1a0ee626f1834 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java @@ -60,6 +60,7 @@ public final class FileSystemConfigurations { public static final int DEFAULT_READ_BUFFER_SIZE = 4 * ONE_MB; // 4 MB public static final boolean DEFAULT_READ_SMALL_FILES_COMPLETELY = false; public static final boolean DEFAULT_OPTIMIZE_FOOTER_READ = false; + public static final int DEFAULT_FOOTER_READ_BUFFER_SIZE = 512 * ONE_KB; public static final boolean DEFAULT_ALWAYS_READ_BUFFER_SIZE = false; public static final int DEFAULT_READ_AHEAD_BLOCK_SIZE = 4 * ONE_MB; public static final int DEFAULT_READ_AHEAD_RANGE = 64 * ONE_KB; // 64 KB diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index 86442dac9aaf7..6d3bcaba95e70 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -70,6 +70,7 @@ public class AbfsInputStream extends FSInputStream implements CanUnbuffer, private final String path; private final long contentLength; private final int bufferSize; // default buffer size + private final int footerReadSize; // default buffer size to read when reading footer private final int readAheadQueueDepth; // initialized in constructor private final String eTag; // eTag of the path when InputStream are created private final boolean tolerateOobAppends; // whether tolerate Oob Appends @@ -138,6 +139,7 @@ public AbfsInputStream( this.path = path; this.contentLength = contentLength; this.bufferSize = abfsInputStreamContext.getReadBufferSize(); + this.footerReadSize = abfsInputStreamContext.getFooterReadBufferSize(); this.readAheadQueueDepth = abfsInputStreamContext.getReadAheadQueueDepth(); this.tolerateOobAppends = abfsInputStreamContext.isTolerateOobAppends(); this.eTag = eTag; @@ -370,10 +372,10 @@ private int readLastBlock(final byte[] b, final int off, final int len) // data need to be copied to user buffer from index bCursor, // AbfsInutStream buffer is going to contain data from last block start. In // that case bCursor will be set to fCursor - lastBlockStart - long lastBlockStart = max(0, contentLength - bufferSize); + long lastBlockStart = max(0, contentLength - footerReadSize); bCursor = (int) (fCursor - lastBlockStart); // 0 if contentlength is < buffersize - long actualLenToRead = min(bufferSize, contentLength); + long actualLenToRead = min(footerReadSize, contentLength); return optimisedRead(b, off, len, lastBlockStart, actualLenToRead); } @@ -812,6 +814,11 @@ public int getBufferSize() { return bufferSize; } + @VisibleForTesting + public int getFooterReadBufferSize() { + return footerReadSize; + } + @VisibleForTesting public int getReadAheadQueueDepth() { return readAheadQueueDepth; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java index b78a899340f87..1e699ce9baf98 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java @@ -51,6 +51,8 @@ public class AbfsInputStreamContext extends AbfsStreamContext { private boolean optimizeFooterRead; + private int footerReadBufferSize; + private boolean bufferedPreadDisabled; /** A BackReference to the FS instance that created this OutputStream. */ @@ -109,6 +111,11 @@ public AbfsInputStreamContext withOptimizeFooterRead( return this; } + public AbfsInputStreamContext withFooterReadBufferSize(final int footerReadBufferSize) { + this.footerReadBufferSize = footerReadBufferSize; + return this; + } + public AbfsInputStreamContext withShouldReadBufferSizeAlways( final boolean alwaysReadBufferSize) { this.alwaysReadBufferSize = alwaysReadBufferSize; @@ -180,6 +187,10 @@ public boolean optimizeFooterRead() { return this.optimizeFooterRead; } + public int getFooterReadBufferSize() { + return footerReadBufferSize; + } + public boolean shouldReadBufferSizeAlways() { return alwaysReadBufferSize; } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java index cb3eaffe029c9..a0f15723e1a98 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java @@ -153,8 +153,12 @@ public void testSeekToEndAndReadWithConfFalse() throws Exception { private void testSeekAndReadWithConf(boolean optimizeFooterRead, SeekTo seekTo) throws Exception { - for (int i = 2; i <= 6; i++) { - int fileSize = i * ONE_MB; + // Running the test for file sizes ranging from 256 KB to 8 MB/ + // This will cover files less than footer read buffer size, + // Files between footer read buffer and read buffer size + // Files bigger than read buffer size + for (int i = 0; i <= 5; i++) { + int fileSize = (int)Math.pow(2, i) * 256 * ONE_KB; final AzureBlobFileSystem fs = getFileSystem(optimizeFooterRead, fileSize); String fileName = methodName.getMethodName() + i; @@ -190,7 +194,8 @@ private void seekReadAndTest(final FileSystem fs, final Path testFilePath, try (FSDataInputStream iStream = fs.open(testFilePath)) { AbfsInputStream abfsInputStream = (AbfsInputStream) iStream .getWrappedStream(); - long bufferSize = abfsInputStream.getBufferSize(); + long footerReadBufferSize = abfsInputStream.getFooterReadBufferSize(); + long readBufferSize = abfsInputStream.getBufferSize(); seek(iStream, seekPos); byte[] buffer = new byte[length]; long bytesRead = iStream.read(buffer, 0, length); @@ -209,22 +214,22 @@ private void seekReadAndTest(final FileSystem fs, final Path testFilePath, long expectedBCurson; long expectedFCursor; if (optimizationOn) { - if (actualContentLength <= bufferSize) { + if (actualContentLength <= footerReadBufferSize) { expectedLimit = actualContentLength; expectedBCurson = seekPos + actualLength; } else { - expectedLimit = bufferSize; - long lastBlockStart = max(0, actualContentLength - bufferSize); + expectedLimit = footerReadBufferSize; + long lastBlockStart = max(0, actualContentLength - footerReadBufferSize); expectedBCurson = seekPos - lastBlockStart + actualLength; } expectedFCursor = actualContentLength; } else { - if (seekPos + bufferSize < actualContentLength) { - expectedLimit = bufferSize; - expectedFCursor = bufferSize; + if (seekPos + readBufferSize < actualContentLength) { + expectedLimit = readBufferSize; + expectedFCursor = readBufferSize; } else { expectedLimit = actualContentLength - seekPos; - expectedFCursor = min(seekPos + bufferSize, actualContentLength); + expectedFCursor = min(seekPos + readBufferSize, actualContentLength); } expectedBCurson = actualLength; } @@ -239,7 +244,7 @@ private void seekReadAndTest(final FileSystem fs, final Path testFilePath, // Verify data read to AbfsInputStream buffer int from = seekPos; if (optimizationOn) { - from = (int) max(0, actualContentLength - bufferSize); + from = (int) max(0, actualContentLength - footerReadBufferSize); } assertContentReadCorrectly(fileContent, from, (int) abfsInputStream.getLimit(), abfsInputStream.getBuffer(), testFilePath); @@ -249,8 +254,8 @@ private void seekReadAndTest(final FileSystem fs, final Path testFilePath, @Test public void testPartialReadWithNoData() throws Exception { - for (int i = 2; i <= 6; i++) { - int fileSize = i * ONE_MB; + for (int i = 0; i <= 5; i++) { + int fileSize = (int)Math.pow(2, i) * 256 * ONE_KB; final AzureBlobFileSystem fs = getFileSystem(true, fileSize); String fileName = methodName.getMethodName() + i; byte[] fileContent = getRandomBytesArray(fileSize); @@ -292,8 +297,8 @@ private void testPartialReadWithNoData(final FileSystem fs, @Test public void testPartialReadWithSomeDat() throws Exception { - for (int i = 3; i <= 6; i++) { - int fileSize = i * ONE_MB; + for (int i = 0; i <= 5; i++) { + int fileSize = (int)Math.pow(2, i) * 256 * ONE_KB; final AzureBlobFileSystem fs = getFileSystem(true, fileSize); String fileName = methodName.getMethodName() + i; byte[] fileContent = getRandomBytesArray(fileSize); @@ -317,7 +322,7 @@ private void testPartialReadWithSomeDat(final FileSystem fs, // second readRemote returns data till the last 2 bytes int someDataLength = 2; int secondReturnSize = - min(fileContent.length, abfsInputStream.getBufferSize()) - 10 + min(fileContent.length, abfsInputStream.getFooterReadBufferSize()) - 10 - someDataLength; doReturn(10).doReturn(secondReturnSize).doCallRealMethod() .when(abfsInputStream) From 14214f062717f40c41a937222fe9c1326cb6b310 Mon Sep 17 00:00:00 2001 From: Anuj Modi Date: Mon, 13 Nov 2023 04:13:16 -0800 Subject: [PATCH 2/8] Modified Tests --- .../ITestAbfsInputStreamReadFooter.java | 21 +++++++++---------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java index a0f15723e1a98..3adf2cbaa5ad1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java @@ -41,7 +41,6 @@ import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CONNECTIONS_MADE; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_KB; -import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB; public class ITestAbfsInputStreamReadFooter extends ITestAbfsInputStream { @@ -64,8 +63,8 @@ public void testMultipleServerCallsAreMadeWhenTheConfIsFalse() private void testNumBackendCalls(boolean optimizeFooterRead) throws Exception { - for (int i = 1; i <= 4; i++) { - int fileSize = i * ONE_MB; + for (int i = 0; i <= 5; i++) { + int fileSize = (int)Math.pow(2, i) * 256 * ONE_KB; final AzureBlobFileSystem fs = getFileSystem(optimizeFooterRead, fileSize); String fileName = methodName.getMethodName() + i; @@ -211,16 +210,16 @@ private void seekReadAndTest(final FileSystem fs, final Path testFilePath, actualLength = length - delta; } long expectedLimit; - long expectedBCurson; + long expectedBCursor; long expectedFCursor; if (optimizationOn) { if (actualContentLength <= footerReadBufferSize) { expectedLimit = actualContentLength; - expectedBCurson = seekPos + actualLength; + expectedBCursor = seekPos + actualLength; } else { expectedLimit = footerReadBufferSize; long lastBlockStart = max(0, actualContentLength - footerReadBufferSize); - expectedBCurson = seekPos - lastBlockStart + actualLength; + expectedBCursor = seekPos - lastBlockStart + actualLength; } expectedFCursor = actualContentLength; } else { @@ -231,13 +230,13 @@ private void seekReadAndTest(final FileSystem fs, final Path testFilePath, expectedLimit = actualContentLength - seekPos; expectedFCursor = min(seekPos + readBufferSize, actualContentLength); } - expectedBCurson = actualLength; + expectedBCursor = actualLength; } assertEquals(expectedFCursor, abfsInputStream.getFCursor()); assertEquals(expectedFCursor, abfsInputStream.getFCursorAfterLastRead()); assertEquals(expectedLimit, abfsInputStream.getLimit()); - assertEquals(expectedBCurson, abfsInputStream.getBCursor()); + assertEquals(expectedBCursor, abfsInputStream.getBCursor()); assertEquals(actualLength, bytesRead); // Verify user-content read assertContentReadCorrectly(fileContent, seekPos, (int) actualLength, buffer, testFilePath); @@ -295,7 +294,7 @@ private void testPartialReadWithNoData(final FileSystem fs, } @Test - public void testPartialReadWithSomeDat() + public void testPartialReadWithSomeData() throws Exception { for (int i = 0; i <= 5; i++) { int fileSize = (int)Math.pow(2, i) * 256 * ONE_KB; @@ -303,13 +302,13 @@ public void testPartialReadWithSomeDat() String fileName = methodName.getMethodName() + i; byte[] fileContent = getRandomBytesArray(fileSize); Path testFilePath = createFileWithContent(fs, fileName, fileContent); - testPartialReadWithSomeDat(fs, testFilePath, + testPartialReadWithSomeData(fs, testFilePath, fileSize - AbfsInputStream.FOOTER_SIZE, AbfsInputStream.FOOTER_SIZE, fileContent); } } - private void testPartialReadWithSomeDat(final FileSystem fs, + private void testPartialReadWithSomeData(final FileSystem fs, final Path testFilePath, final int seekPos, final int length, final byte[] fileContent) throws IOException, NoSuchFieldException, IllegalAccessException { From e438b9446bfaa4cdbbbb602fa1c5bc5c83f5b9f5 Mon Sep 17 00:00:00 2001 From: Anuj Modi Date: Mon, 13 Nov 2023 09:10:27 -0800 Subject: [PATCH 3/8] PR Checks --- .../fs/azurebfs/constants/FileSystemConfigurations.java | 2 +- .../hadoop/fs/azurebfs/services/AbfsInputStream.java | 3 ++- .../hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java | 1 + .../hadoop/fs/azurebfs/services/ITestAbfsInputStream.java | 2 ++ .../azurebfs/services/ITestAbfsInputStreamReadFooter.java | 8 ++++---- 5 files changed, 10 insertions(+), 6 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java index 1a0ee626f1834..b3825b4c53ec1 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java @@ -59,7 +59,7 @@ public final class FileSystemConfigurations { public static final boolean DEFAULT_AZURE_ENABLE_SMALL_WRITE_OPTIMIZATION = false; public static final int DEFAULT_READ_BUFFER_SIZE = 4 * ONE_MB; // 4 MB public static final boolean DEFAULT_READ_SMALL_FILES_COMPLETELY = false; - public static final boolean DEFAULT_OPTIMIZE_FOOTER_READ = false; + public static final boolean DEFAULT_OPTIMIZE_FOOTER_READ = true; public static final int DEFAULT_FOOTER_READ_BUFFER_SIZE = 512 * ONE_KB; public static final boolean DEFAULT_ALWAYS_READ_BUFFER_SIZE = false; public static final int DEFAULT_READ_AHEAD_BLOCK_SIZE = 4 * ONE_MB; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index 6d3bcaba95e70..dcdf8d7ea17d6 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -360,6 +360,7 @@ private int readFileCompletely(final byte[] b, final int off, final int len) return optimisedRead(b, off, len, 0, contentLength); } + // To do footer read of files when enabled private int readLastBlock(final byte[] b, final int off, final int len) throws IOException { if (len == 0) { @@ -815,7 +816,7 @@ public int getBufferSize() { } @VisibleForTesting - public int getFooterReadBufferSize() { + protected int getFooterReadBufferSize() { return footerReadSize; } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java index beada775ae87b..bc420c6a1f8cd 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java @@ -84,6 +84,7 @@ private void testReadWriteAndSeek(int bufferSize) throws Exception { abfsConfiguration.setWriteBufferSize(bufferSize); abfsConfiguration.setReadBufferSize(bufferSize); abfsConfiguration.setReadAheadEnabled(readaheadEnabled); + abfsConfiguration.setOptimizeFooterRead(false); final byte[] b = new byte[2 * bufferSize]; new Random().nextBytes(b); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStream.java index 2ac58fbcb1668..11b14162eb2f9 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStream.java @@ -169,6 +169,8 @@ protected AzureBlobFileSystem getFileSystem(boolean readSmallFilesCompletely) final AzureBlobFileSystem fs = getFileSystem(); getAbfsStore(fs).getAbfsConfiguration() .setReadSmallFilesCompletely(readSmallFilesCompletely); + getAbfsStore(fs).getAbfsConfiguration() + .setOptimizeFooterRead(false); return fs; } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java index 3adf2cbaa5ad1..fbf469bbcf7d9 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java @@ -64,7 +64,7 @@ public void testMultipleServerCallsAreMadeWhenTheConfIsFalse() private void testNumBackendCalls(boolean optimizeFooterRead) throws Exception { for (int i = 0; i <= 5; i++) { - int fileSize = (int)Math.pow(2, i) * 256 * ONE_KB; + int fileSize = (int) Math.pow(2, i) * 256 * ONE_KB; final AzureBlobFileSystem fs = getFileSystem(optimizeFooterRead, fileSize); String fileName = methodName.getMethodName() + i; @@ -157,7 +157,7 @@ private void testSeekAndReadWithConf(boolean optimizeFooterRead, // Files between footer read buffer and read buffer size // Files bigger than read buffer size for (int i = 0; i <= 5; i++) { - int fileSize = (int)Math.pow(2, i) * 256 * ONE_KB; + int fileSize = (int) Math.pow(2, i) * 256 * ONE_KB; final AzureBlobFileSystem fs = getFileSystem(optimizeFooterRead, fileSize); String fileName = methodName.getMethodName() + i; @@ -254,7 +254,7 @@ private void seekReadAndTest(final FileSystem fs, final Path testFilePath, public void testPartialReadWithNoData() throws Exception { for (int i = 0; i <= 5; i++) { - int fileSize = (int)Math.pow(2, i) * 256 * ONE_KB; + int fileSize = (int) Math.pow(2, i) * 256 * ONE_KB; final AzureBlobFileSystem fs = getFileSystem(true, fileSize); String fileName = methodName.getMethodName() + i; byte[] fileContent = getRandomBytesArray(fileSize); @@ -297,7 +297,7 @@ private void testPartialReadWithNoData(final FileSystem fs, public void testPartialReadWithSomeData() throws Exception { for (int i = 0; i <= 5; i++) { - int fileSize = (int)Math.pow(2, i) * 256 * ONE_KB; + int fileSize = (int) Math.pow(2, i) * 256 * ONE_KB; final AzureBlobFileSystem fs = getFileSystem(true, fileSize); String fileName = methodName.getMethodName() + i; byte[] fileContent = getRandomBytesArray(fileSize); From c0c131932b9a06b4ccbaeb19a8e23da87f383065 Mon Sep 17 00:00:00 2001 From: Anuj Modi Date: Tue, 21 Nov 2023 02:36:58 -0800 Subject: [PATCH 4/8] Addressing Comments --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 5 + .../azurebfs/constants/ConfigurationKeys.java | 10 +- .../fs/azurebfs/services/AbfsInputStream.java | 2 +- .../ITestAbfsInputStreamReadFooter.java | 162 ++++++++++-------- 4 files changed, 105 insertions(+), 74 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 5bd935e516b54..8989f99db8da3 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -1147,6 +1147,11 @@ public void setOptimizeFooterRead(boolean optimizeFooterRead) { this.optimizeFooterRead = optimizeFooterRead; } + @VisibleForTesting + public void setFooterReadBufferSize(int footerReadBufferSize) { + this.footerReadBufferSize = footerReadBufferSize; + } + @VisibleForTesting public void setEnableAbfsListIterator(boolean enableAbfsListIterator) { this.enableAbfsListIterator = enableAbfsListIterator; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java index bd66d7266a30d..2d0738dc7e023 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java @@ -105,15 +105,17 @@ public final class ConfigurationKeys { public static final String AZURE_READ_BUFFER_SIZE = "fs.azure.read.request.size"; public static final String AZURE_READ_SMALL_FILES_COMPLETELY = "fs.azure.read.smallfilescompletely"; /** - * When parquet files are read, first few read are metadata reads before reading the actual data. - * First the read is done of last 8 bytes of parquet file to get the postion of metadta and next read - * is done for reading that metadata. With this optimizations these two reads can be combined into 1. + * When parquet files are read, first few read are metadata reads before + * reading the actual data. First the read is done of last 8 bytes of parquet + * file to get the postion of metadta and next read is done for reading that + * metadata. With this optimization these two reads can be combined into 1. * Value: {@value} */ public static final String AZURE_READ_OPTIMIZE_FOOTER_READ = "fs.azure.read.optimizefooterread"; /** * In case of footer reads it was not required to read full buffer size. - * Most of the metadata information required was within 256KB and it will be more performant to read lesser. + * Most of the metadata information required was within 256 KB and it will be + * more performant to read less. 512 KB is a sweet spot. * This config is used to define how much footer length the user wants to read. * Value: {@value} */ diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index dcdf8d7ea17d6..589dba9db563e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -360,7 +360,7 @@ private int readFileCompletely(final byte[] b, final int off, final int len) return optimisedRead(b, off, len, 0, contentLength); } - // To do footer read of files when enabled + // To do footer read of files when enabled. private int readLastBlock(final byte[] b, final int off, final int len) throws IOException { if (len == 0) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java index fbf469bbcf7d9..923eb9e2acec8 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java @@ -21,6 +21,9 @@ import java.io.IOException; import java.util.Map; +import org.apache.hadoop.fs.FutureDataInputStreamBuilder; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore; +import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; import org.junit.Test; import org.apache.hadoop.fs.FSDataInputStream; @@ -33,6 +36,7 @@ import static java.lang.Math.max; import static java.lang.Math.min; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_FOOTER_READ_BUFFER_SIZE; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyLong; @@ -63,38 +67,46 @@ public void testMultipleServerCallsAreMadeWhenTheConfIsFalse() private void testNumBackendCalls(boolean optimizeFooterRead) throws Exception { - for (int i = 0; i <= 5; i++) { - int fileSize = (int) Math.pow(2, i) * 256 * ONE_KB; - final AzureBlobFileSystem fs = getFileSystem(optimizeFooterRead, - fileSize); - String fileName = methodName.getMethodName() + i; - byte[] fileContent = getRandomBytesArray(fileSize); - Path testFilePath = createFileWithContent(fs, fileName, fileContent); - int length = AbfsInputStream.FOOTER_SIZE; - try (FSDataInputStream iStream = fs.open(testFilePath)) { - byte[] buffer = new byte[length]; - - Map metricMap = getInstrumentationMap(fs); - long requestsMadeBeforeTest = metricMap - .get(CONNECTIONS_MADE.getStatName()); - - iStream.seek(fileSize - 8); - iStream.read(buffer, 0, length); - - iStream.seek(fileSize - (TEN * ONE_KB)); - iStream.read(buffer, 0, length); - - iStream.seek(fileSize - (TWENTY * ONE_KB)); - iStream.read(buffer, 0, length); - - metricMap = getInstrumentationMap(fs); - long requestsMadeAfterTest = metricMap - .get(CONNECTIONS_MADE.getStatName()); - - if (optimizeFooterRead) { - assertEquals(1, requestsMadeAfterTest - requestsMadeBeforeTest); - } else { - assertEquals(3, requestsMadeAfterTest - requestsMadeBeforeTest); + for (int i = 0; i <= 4; i++) { + for (int j = 0; j <= 2; j++) { + int fileSize = (int) Math.pow(2, i) * 256 * ONE_KB; + int footerReadBufferSize = (int) Math.pow(2, j) * 256 * ONE_KB; + final AzureBlobFileSystem fs = getFileSystem(optimizeFooterRead, + fileSize, footerReadBufferSize); + String fileName = methodName.getMethodName() + i; + byte[] fileContent = getRandomBytesArray(fileSize); + Path testFilePath = createFileWithContent(fs, fileName, fileContent); + int length = AbfsInputStream.FOOTER_SIZE; + FutureDataInputStreamBuilder builder = fs.openFile(testFilePath); + builder.opt(ConfigurationKeys.AZURE_FOOTER_READ_BUFFER_SIZE, + footerReadBufferSize); + try (FSDataInputStream iStream = builder.build().get()) { + byte[] buffer = new byte[length]; + + Map metricMap = getInstrumentationMap(fs); + long requestsMadeBeforeTest = metricMap + .get(CONNECTIONS_MADE.getStatName()); + + iStream.seek(fileSize - 8); + iStream.read(buffer, 0, length); + + iStream.seek(fileSize - (TEN * ONE_KB)); + iStream.read(buffer, 0, length); + + iStream.seek(fileSize - (TWENTY * ONE_KB)); + iStream.read(buffer, 0, length); + + metricMap = getInstrumentationMap(fs); + long requestsMadeAfterTest = metricMap + .get(CONNECTIONS_MADE.getStatName()); + + if (optimizeFooterRead) { + assertEquals(1, + requestsMadeAfterTest - requestsMadeBeforeTest); + } else { + assertEquals(3, + requestsMadeAfterTest - requestsMadeBeforeTest); + } } } } @@ -152,19 +164,22 @@ public void testSeekToEndAndReadWithConfFalse() throws Exception { private void testSeekAndReadWithConf(boolean optimizeFooterRead, SeekTo seekTo) throws Exception { - // Running the test for file sizes ranging from 256 KB to 8 MB/ + // Running the test for file sizes ranging from 256 KB to 8 MB // This will cover files less than footer read buffer size, // Files between footer read buffer and read buffer size // Files bigger than read buffer size - for (int i = 0; i <= 5; i++) { - int fileSize = (int) Math.pow(2, i) * 256 * ONE_KB; - final AzureBlobFileSystem fs = getFileSystem(optimizeFooterRead, - fileSize); - String fileName = methodName.getMethodName() + i; - byte[] fileContent = getRandomBytesArray(fileSize); - Path testFilePath = createFileWithContent(fs, fileName, fileContent); - seekReadAndTest(fs, testFilePath, seekPos(seekTo, fileSize), HUNDRED, - fileContent); + for (int i = 0; i <= 4; i++) { + for (int j = 0; j <= 2; j++) { + int fileSize = (int) Math.pow(2, i) * 256 * ONE_KB; + int footerReadBufferSize = (int) Math.pow(2, j) * 256 * ONE_KB; + final AzureBlobFileSystem fs = getFileSystem(optimizeFooterRead, + fileSize, footerReadBufferSize); + String fileName = methodName.getMethodName() + i; + byte[] fileContent = getRandomBytesArray(fileSize); + Path testFilePath = createFileWithContent(fs, fileName, fileContent); + seekReadAndTest(fs, testFilePath, seekPos(seekTo, fileSize), HUNDRED, + fileContent); + } } } @@ -187,10 +202,12 @@ private int seekPos(SeekTo seekTo, int fileSize) { private void seekReadAndTest(final FileSystem fs, final Path testFilePath, final int seekPos, final int length, final byte[] fileContent) - throws IOException, NoSuchFieldException, IllegalAccessException { + throws Exception { AbfsConfiguration conf = getAbfsStore(fs).getAbfsConfiguration(); long actualContentLength = fileContent.length; - try (FSDataInputStream iStream = fs.open(testFilePath)) { + FutureDataInputStreamBuilder builder = fs.openFile(testFilePath); + builder.opt(ConfigurationKeys.AZURE_FOOTER_READ_BUFFER_SIZE, DEFAULT_FOOTER_READ_BUFFER_SIZE); + try (FSDataInputStream iStream = builder.build().get()) { AbfsInputStream abfsInputStream = (AbfsInputStream) iStream .getWrappedStream(); long footerReadBufferSize = abfsInputStream.getFooterReadBufferSize(); @@ -253,15 +270,19 @@ private void seekReadAndTest(final FileSystem fs, final Path testFilePath, @Test public void testPartialReadWithNoData() throws Exception { - for (int i = 0; i <= 5; i++) { - int fileSize = (int) Math.pow(2, i) * 256 * ONE_KB; - final AzureBlobFileSystem fs = getFileSystem(true, fileSize); - String fileName = methodName.getMethodName() + i; - byte[] fileContent = getRandomBytesArray(fileSize); - Path testFilePath = createFileWithContent(fs, fileName, fileContent); - testPartialReadWithNoData(fs, testFilePath, - fileSize - AbfsInputStream.FOOTER_SIZE, AbfsInputStream.FOOTER_SIZE, - fileContent); + for (int i = 0; i <= 4; i++) { + for (int j = 0; j <= 2; j++) { + int fileSize = (int) Math.pow(2, i) * 256 * ONE_KB; + int footerReadBufferSize = (int) Math.pow(2, j) * 256 * ONE_KB; + final AzureBlobFileSystem fs = getFileSystem(true, + fileSize, footerReadBufferSize); + String fileName = methodName.getMethodName() + i; + byte[] fileContent = getRandomBytesArray(fileSize); + Path testFilePath = createFileWithContent(fs, fileName, fileContent); + testPartialReadWithNoData(fs, testFilePath, + fileSize - AbfsInputStream.FOOTER_SIZE, AbfsInputStream.FOOTER_SIZE, + fileContent); + } } } @@ -296,15 +317,19 @@ private void testPartialReadWithNoData(final FileSystem fs, @Test public void testPartialReadWithSomeData() throws Exception { - for (int i = 0; i <= 5; i++) { - int fileSize = (int) Math.pow(2, i) * 256 * ONE_KB; - final AzureBlobFileSystem fs = getFileSystem(true, fileSize); - String fileName = methodName.getMethodName() + i; - byte[] fileContent = getRandomBytesArray(fileSize); - Path testFilePath = createFileWithContent(fs, fileName, fileContent); - testPartialReadWithSomeData(fs, testFilePath, - fileSize - AbfsInputStream.FOOTER_SIZE, AbfsInputStream.FOOTER_SIZE, - fileContent); + for (int i = 0; i <= 4; i++) { + for (int j = 0; j <= 2; j++) { + int fileSize = (int) Math.pow(2, i) * 256 * ONE_KB; + int footerReadBufferSize = (int) Math.pow(2, j) * 256 * ONE_KB; + final AzureBlobFileSystem fs = getFileSystem(true, + fileSize, footerReadBufferSize); + String fileName = methodName.getMethodName() + i; + byte[] fileContent = getRandomBytesArray(fileSize); + Path testFilePath = createFileWithContent(fs, fileName, fileContent); + testPartialReadWithSomeData(fs, testFilePath, + fileSize - AbfsInputStream.FOOTER_SIZE, AbfsInputStream.FOOTER_SIZE, + fileContent); + } } } @@ -347,14 +372,13 @@ private void testPartialReadWithSomeData(final FileSystem fs, } private AzureBlobFileSystem getFileSystem(boolean optimizeFooterRead, - int fileSize) throws IOException { + int fileSize, int footerReadBufferSize) throws IOException { final AzureBlobFileSystem fs = getFileSystem(); - getAbfsStore(fs).getAbfsConfiguration() - .setOptimizeFooterRead(optimizeFooterRead); - if (fileSize <= getAbfsStore(fs).getAbfsConfiguration() - .getReadBufferSize()) { - getAbfsStore(fs).getAbfsConfiguration() - .setReadSmallFilesCompletely(false); + AzureBlobFileSystemStore store = getAbfsStore(fs); + store.getAbfsConfiguration().setOptimizeFooterRead(optimizeFooterRead); + store.getAbfsConfiguration().setFooterReadBufferSize(footerReadBufferSize); + if (fileSize <= store.getAbfsConfiguration().getReadBufferSize()) { + store.getAbfsConfiguration().setReadSmallFilesCompletely(false); } return fs; } From 6bb6bd769809cac2da2fa3e04cdead9f9782e69c Mon Sep 17 00:00:00 2001 From: Anuj Modi Date: Tue, 21 Nov 2023 23:11:07 -0800 Subject: [PATCH 5/8] Enhanced Test Coverage For Footer Related Configs --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 7 ++- .../ITestAbfsInputStreamReadFooter.java | 49 ++++++++++++++----- 2 files changed, 44 insertions(+), 12 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index b998d4f0b8163..a03877668ff73 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -147,8 +147,10 @@ import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.SINGLE_WHITE_SPACE; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.TOKEN_VERSION; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_ABFS_ENDPOINT; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_FOOTER_READ_BUFFER_SIZE; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_BUFFERED_PREAD_DISABLE; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_IDENTITY_TRANSFORM_CLASS; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_FOOTER_READ_BUFFER_SIZE; /** * Provides the bridging logic between Hadoop's abstract filesystem and Azure Storage. @@ -811,6 +813,9 @@ private AbfsInputStreamContext populateAbfsInputStreamContext( boolean bufferedPreadDisabled = options .map(c -> c.getBoolean(FS_AZURE_BUFFERED_PREAD_DISABLE, false)) .orElse(false); + int footerReadBufferSize = options.map(c -> c.getInt( + AZURE_FOOTER_READ_BUFFER_SIZE, DEFAULT_FOOTER_READ_BUFFER_SIZE)) + .orElse(abfsConfiguration.getFooterReadBufferSize()); return new AbfsInputStreamContext(abfsConfiguration.getSasTokenRenewPeriodForStreamsInSeconds()) .withReadBufferSize(abfsConfiguration.getReadBufferSize()) .withReadAheadQueueDepth(abfsConfiguration.getReadAheadQueueDepth()) @@ -818,7 +823,7 @@ private AbfsInputStreamContext populateAbfsInputStreamContext( .isReadAheadEnabled(abfsConfiguration.isReadAheadEnabled()) .withReadSmallFilesCompletely(abfsConfiguration.readSmallFilesCompletely()) .withOptimizeFooterRead(abfsConfiguration.optimizeFooterRead()) - .withFooterReadBufferSize(abfsConfiguration.getFooterReadBufferSize()) + .withFooterReadBufferSize(footerReadBufferSize) .withReadAheadRange(abfsConfiguration.getReadAheadRange()) .withStreamStatistics(new AbfsInputStreamStatisticsImpl()) .withShouldReadBufferSizeAlways( diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java index 923eb9e2acec8..29ebcaa1629a6 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java @@ -24,6 +24,8 @@ import org.apache.hadoop.fs.FutureDataInputStreamBuilder; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore; import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; + +import org.assertj.core.api.Assertions; import org.junit.Test; import org.apache.hadoop.fs.FSDataInputStream; @@ -72,7 +74,7 @@ private void testNumBackendCalls(boolean optimizeFooterRead) int fileSize = (int) Math.pow(2, i) * 256 * ONE_KB; int footerReadBufferSize = (int) Math.pow(2, j) * 256 * ONE_KB; final AzureBlobFileSystem fs = getFileSystem(optimizeFooterRead, - fileSize, footerReadBufferSize); + fileSize); String fileName = methodName.getMethodName() + i; byte[] fileContent = getRandomBytesArray(fileSize); Path testFilePath = createFileWithContent(fs, fileName, fileContent); @@ -81,6 +83,10 @@ private void testNumBackendCalls(boolean optimizeFooterRead) builder.opt(ConfigurationKeys.AZURE_FOOTER_READ_BUFFER_SIZE, footerReadBufferSize); try (FSDataInputStream iStream = builder.build().get()) { + AbfsInputStream abfsInputStream = (AbfsInputStream)iStream.getWrappedStream(); + Assertions.assertThat(abfsInputStream.getFooterReadBufferSize()) + .describedAs("Footer Read Buffer Size Should be same as what set in builder") + .isEqualTo(footerReadBufferSize); byte[] buffer = new byte[length]; Map metricMap = getInstrumentationMap(fs); @@ -164,7 +170,8 @@ public void testSeekToEndAndReadWithConfFalse() throws Exception { private void testSeekAndReadWithConf(boolean optimizeFooterRead, SeekTo seekTo) throws Exception { - // Running the test for file sizes ranging from 256 KB to 8 MB + // Running the test for file sizes ranging from 256 KB to 4 MB with + // Footer Read Buffer size ranging from 256 KB to 1 MB // This will cover files less than footer read buffer size, // Files between footer read buffer and read buffer size // Files bigger than read buffer size @@ -173,12 +180,12 @@ private void testSeekAndReadWithConf(boolean optimizeFooterRead, int fileSize = (int) Math.pow(2, i) * 256 * ONE_KB; int footerReadBufferSize = (int) Math.pow(2, j) * 256 * ONE_KB; final AzureBlobFileSystem fs = getFileSystem(optimizeFooterRead, - fileSize, footerReadBufferSize); + fileSize); String fileName = methodName.getMethodName() + i; byte[] fileContent = getRandomBytesArray(fileSize); Path testFilePath = createFileWithContent(fs, fileName, fileContent); seekReadAndTest(fs, testFilePath, seekPos(seekTo, fileSize), HUNDRED, - fileContent); + fileContent, footerReadBufferSize); } } } @@ -201,16 +208,18 @@ private int seekPos(SeekTo seekTo, int fileSize) { } private void seekReadAndTest(final FileSystem fs, final Path testFilePath, - final int seekPos, final int length, final byte[] fileContent) + final int seekPos, final int length, final byte[] fileContent, int footerReadBufferSize) throws Exception { AbfsConfiguration conf = getAbfsStore(fs).getAbfsConfiguration(); long actualContentLength = fileContent.length; FutureDataInputStreamBuilder builder = fs.openFile(testFilePath); - builder.opt(ConfigurationKeys.AZURE_FOOTER_READ_BUFFER_SIZE, DEFAULT_FOOTER_READ_BUFFER_SIZE); + builder.opt(ConfigurationKeys.AZURE_FOOTER_READ_BUFFER_SIZE, footerReadBufferSize); try (FSDataInputStream iStream = builder.build().get()) { AbfsInputStream abfsInputStream = (AbfsInputStream) iStream .getWrappedStream(); - long footerReadBufferSize = abfsInputStream.getFooterReadBufferSize(); + Assertions.assertThat(abfsInputStream.getFooterReadBufferSize()) + .describedAs("Footer Read Buffer Size Should be same as what set in builder") + .isEqualTo(footerReadBufferSize); long readBufferSize = abfsInputStream.getBufferSize(); seek(iStream, seekPos); byte[] buffer = new byte[length]; @@ -281,19 +290,22 @@ public void testPartialReadWithNoData() Path testFilePath = createFileWithContent(fs, fileName, fileContent); testPartialReadWithNoData(fs, testFilePath, fileSize - AbfsInputStream.FOOTER_SIZE, AbfsInputStream.FOOTER_SIZE, - fileContent); + fileContent, footerReadBufferSize); } } } private void testPartialReadWithNoData(final FileSystem fs, final Path testFilePath, final int seekPos, final int length, - final byte[] fileContent) + final byte[] fileContent, int footerReadBufferSize) throws IOException, NoSuchFieldException, IllegalAccessException { FSDataInputStream iStream = fs.open(testFilePath); try { AbfsInputStream abfsInputStream = (AbfsInputStream) iStream .getWrappedStream(); + Assertions.assertThat(abfsInputStream.getFooterReadBufferSize()) + .describedAs("Footer Read Buffer Size Should be same as what set in builder") + .isEqualTo(footerReadBufferSize); abfsInputStream = spy(abfsInputStream); doReturn(10).doReturn(10).doCallRealMethod().when(abfsInputStream) .readRemote(anyLong(), any(), anyInt(), anyInt(), @@ -328,19 +340,23 @@ public void testPartialReadWithSomeData() Path testFilePath = createFileWithContent(fs, fileName, fileContent); testPartialReadWithSomeData(fs, testFilePath, fileSize - AbfsInputStream.FOOTER_SIZE, AbfsInputStream.FOOTER_SIZE, - fileContent); + fileContent, footerReadBufferSize); } } } private void testPartialReadWithSomeData(final FileSystem fs, final Path testFilePath, final int seekPos, final int length, - final byte[] fileContent) + final byte[] fileContent, final int footerReadBufferSize) throws IOException, NoSuchFieldException, IllegalAccessException { FSDataInputStream iStream = fs.open(testFilePath); try { AbfsInputStream abfsInputStream = (AbfsInputStream) iStream .getWrappedStream(); + Assertions.assertThat(abfsInputStream.getFooterReadBufferSize()) + .describedAs("Footer Read Buffer Size Should be same as what set in builder") + .isEqualTo(footerReadBufferSize); + abfsInputStream = spy(abfsInputStream); // first readRemote, will return first 10 bytes // second readRemote returns data till the last 2 bytes @@ -371,6 +387,17 @@ private void testPartialReadWithSomeData(final FileSystem fs, } } + private AzureBlobFileSystem getFileSystem(boolean optimizeFooterRead, + int fileSize) throws IOException { + final AzureBlobFileSystem fs = getFileSystem(); + AzureBlobFileSystemStore store = getAbfsStore(fs); + store.getAbfsConfiguration().setOptimizeFooterRead(optimizeFooterRead); + if (fileSize <= store.getAbfsConfiguration().getReadBufferSize()) { + store.getAbfsConfiguration().setReadSmallFilesCompletely(false); + } + return fs; + } + private AzureBlobFileSystem getFileSystem(boolean optimizeFooterRead, int fileSize, int footerReadBufferSize) throws IOException { final AzureBlobFileSystem fs = getFileSystem(); From a7597ed0b5e281be9116d4714960f62e4ed0c306 Mon Sep 17 00:00:00 2001 From: Anuj Modi Date: Wed, 22 Nov 2023 04:35:22 -0800 Subject: [PATCH 6/8] Test File Changes --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 2 +- .../ITestAbfsInputStreamReadFooter.java | 145 ++++++++++++------ 2 files changed, 99 insertions(+), 48 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index a03877668ff73..f562c268d7e40 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -814,7 +814,7 @@ private AbfsInputStreamContext populateAbfsInputStreamContext( .map(c -> c.getBoolean(FS_AZURE_BUFFERED_PREAD_DISABLE, false)) .orElse(false); int footerReadBufferSize = options.map(c -> c.getInt( - AZURE_FOOTER_READ_BUFFER_SIZE, DEFAULT_FOOTER_READ_BUFFER_SIZE)) + AZURE_FOOTER_READ_BUFFER_SIZE, abfsConfiguration.getFooterReadBufferSize())) .orElse(abfsConfiguration.getFooterReadBufferSize()); return new AbfsInputStreamContext(abfsConfiguration.getSasTokenRenewPeriodForStreamsInSeconds()) .withReadBufferSize(abfsConfiguration.getReadBufferSize()) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java index 29ebcaa1629a6..17229ca00b4c8 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java @@ -69,24 +69,20 @@ public void testMultipleServerCallsAreMadeWhenTheConfIsFalse() private void testNumBackendCalls(boolean optimizeFooterRead) throws Exception { + int fileIdx = 0; for (int i = 0; i <= 4; i++) { for (int j = 0; j <= 2; j++) { int fileSize = (int) Math.pow(2, i) * 256 * ONE_KB; int footerReadBufferSize = (int) Math.pow(2, j) * 256 * ONE_KB; - final AzureBlobFileSystem fs = getFileSystem(optimizeFooterRead, - fileSize); - String fileName = methodName.getMethodName() + i; - byte[] fileContent = getRandomBytesArray(fileSize); - Path testFilePath = createFileWithContent(fs, fileName, fileContent); + final AzureBlobFileSystem fs = getFileSystem( + optimizeFooterRead, fileSize); + Path testFilePath = createPathAndFileWithContent( + fs, fileIdx++, fileSize); int length = AbfsInputStream.FOOTER_SIZE; - FutureDataInputStreamBuilder builder = fs.openFile(testFilePath); - builder.opt(ConfigurationKeys.AZURE_FOOTER_READ_BUFFER_SIZE, - footerReadBufferSize); + FutureDataInputStreamBuilder builder = getParameterizedBuilder( + testFilePath, fs, footerReadBufferSize); try (FSDataInputStream iStream = builder.build().get()) { - AbfsInputStream abfsInputStream = (AbfsInputStream)iStream.getWrappedStream(); - Assertions.assertThat(abfsInputStream.getFooterReadBufferSize()) - .describedAs("Footer Read Buffer Size Should be same as what set in builder") - .isEqualTo(footerReadBufferSize); + verifyConfigValueInStream(iStream, footerReadBufferSize); byte[] buffer = new byte[length]; Map metricMap = getInstrumentationMap(fs); @@ -175,13 +171,14 @@ private void testSeekAndReadWithConf(boolean optimizeFooterRead, // This will cover files less than footer read buffer size, // Files between footer read buffer and read buffer size // Files bigger than read buffer size + int fileIdx = 0; for (int i = 0; i <= 4; i++) { for (int j = 0; j <= 2; j++) { int fileSize = (int) Math.pow(2, i) * 256 * ONE_KB; int footerReadBufferSize = (int) Math.pow(2, j) * 256 * ONE_KB; - final AzureBlobFileSystem fs = getFileSystem(optimizeFooterRead, - fileSize); - String fileName = methodName.getMethodName() + i; + final AzureBlobFileSystem fs = getFileSystem( + optimizeFooterRead, fileSize); + String fileName = methodName.getMethodName() + fileIdx++; byte[] fileContent = getRandomBytesArray(fileSize); Path testFilePath = createFileWithContent(fs, fileName, fileContent); seekReadAndTest(fs, testFilePath, seekPos(seekTo, fileSize), HUNDRED, @@ -207,19 +204,16 @@ private int seekPos(SeekTo seekTo, int fileSize) { return fileSize - AbfsInputStream.FOOTER_SIZE + 1; } - private void seekReadAndTest(final FileSystem fs, final Path testFilePath, - final int seekPos, final int length, final byte[] fileContent, int footerReadBufferSize) - throws Exception { + private void seekReadAndTest(final AzureBlobFileSystem fs, + final Path testFilePath, final int seekPos, final int length, + final byte[] fileContent, int footerReadBufferSize) throws Exception { AbfsConfiguration conf = getAbfsStore(fs).getAbfsConfiguration(); long actualContentLength = fileContent.length; - FutureDataInputStreamBuilder builder = fs.openFile(testFilePath); - builder.opt(ConfigurationKeys.AZURE_FOOTER_READ_BUFFER_SIZE, footerReadBufferSize); + FutureDataInputStreamBuilder builder = getParameterizedBuilder( + testFilePath, fs, footerReadBufferSize); try (FSDataInputStream iStream = builder.build().get()) { - AbfsInputStream abfsInputStream = (AbfsInputStream) iStream - .getWrappedStream(); - Assertions.assertThat(abfsInputStream.getFooterReadBufferSize()) - .describedAs("Footer Read Buffer Size Should be same as what set in builder") - .isEqualTo(footerReadBufferSize); + AbfsInputStream abfsInputStream = (AbfsInputStream) iStream.getWrappedStream(); + verifyConfigValueInStream(iStream, footerReadBufferSize); long readBufferSize = abfsInputStream.getBufferSize(); seek(iStream, seekPos); byte[] buffer = new byte[length]; @@ -277,15 +271,15 @@ private void seekReadAndTest(final FileSystem fs, final Path testFilePath, } @Test - public void testPartialReadWithNoData() - throws Exception { + public void testPartialReadWithNoData() throws Exception { + int fileIdx = 0; for (int i = 0; i <= 4; i++) { for (int j = 0; j <= 2; j++) { int fileSize = (int) Math.pow(2, i) * 256 * ONE_KB; int footerReadBufferSize = (int) Math.pow(2, j) * 256 * ONE_KB; - final AzureBlobFileSystem fs = getFileSystem(true, - fileSize, footerReadBufferSize); - String fileName = methodName.getMethodName() + i; + final AzureBlobFileSystem fs = getFileSystem( + true, fileSize, footerReadBufferSize); + String fileName = methodName.getMethodName() + fileIdx++; byte[] fileContent = getRandomBytesArray(fileSize); Path testFilePath = createFileWithContent(fs, fileName, fileContent); testPartialReadWithNoData(fs, testFilePath, @@ -297,8 +291,7 @@ public void testPartialReadWithNoData() private void testPartialReadWithNoData(final FileSystem fs, final Path testFilePath, final int seekPos, final int length, - final byte[] fileContent, int footerReadBufferSize) - throws IOException, NoSuchFieldException, IllegalAccessException { + final byte[] fileContent, int footerReadBufferSize) throws IOException { FSDataInputStream iStream = fs.open(testFilePath); try { AbfsInputStream abfsInputStream = (AbfsInputStream) iStream @@ -327,8 +320,7 @@ private void testPartialReadWithNoData(final FileSystem fs, } @Test - public void testPartialReadWithSomeData() - throws Exception { + public void testPartialReadWithSomeData() throws Exception { for (int i = 0; i <= 4; i++) { for (int j = 0; j <= 2; j++) { int fileSize = (int) Math.pow(2, i) * 256 * ONE_KB; @@ -347,17 +339,12 @@ public void testPartialReadWithSomeData() private void testPartialReadWithSomeData(final FileSystem fs, final Path testFilePath, final int seekPos, final int length, - final byte[] fileContent, final int footerReadBufferSize) - throws IOException, NoSuchFieldException, IllegalAccessException { + final byte[] fileContent, final int footerReadBufferSize) throws IOException { FSDataInputStream iStream = fs.open(testFilePath); try { - AbfsInputStream abfsInputStream = (AbfsInputStream) iStream - .getWrappedStream(); - Assertions.assertThat(abfsInputStream.getFooterReadBufferSize()) - .describedAs("Footer Read Buffer Size Should be same as what set in builder") - .isEqualTo(footerReadBufferSize); - - abfsInputStream = spy(abfsInputStream); + verifyConfigValueInStream(iStream, footerReadBufferSize); + AbfsInputStream abfsInputStream = spy((AbfsInputStream) iStream + .getWrappedStream()); // first readRemote, will return first 10 bytes // second readRemote returns data till the last 2 bytes int someDataLength = 2; @@ -387,8 +374,72 @@ private void testPartialReadWithSomeData(final FileSystem fs, } } - private AzureBlobFileSystem getFileSystem(boolean optimizeFooterRead, - int fileSize) throws IOException { + @Test + public void testFooterReadBufferSizeConfiguration() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + Path testFilePath = createPathAndFileWithContent(fs, 0, ONE_KB); + final int footerReadBufferSizeConfig = 4 * ONE_KB; + final int footerReadBufferSizeBuilder = 5 * ONE_KB; + + // Verify that default value is used if nothing is set explicitly + FSDataInputStream iStream = fs.open(testFilePath); + verifyConfigValueInStream(iStream, DEFAULT_FOOTER_READ_BUFFER_SIZE); + + // Verify that value set in config is used if not builder is not used + getAbfsStore(fs).getAbfsConfiguration() + .setFooterReadBufferSize(footerReadBufferSizeConfig); + iStream = fs.open(testFilePath); + verifyConfigValueInStream(iStream, footerReadBufferSizeConfig); + + // Verify that when builder is used value set in parameters is used + FutureDataInputStreamBuilder builder = fs.openFile(testFilePath); + builder.opt(ConfigurationKeys.AZURE_FOOTER_READ_BUFFER_SIZE, + footerReadBufferSizeBuilder); + iStream = builder.build().get(); + verifyConfigValueInStream(iStream, footerReadBufferSizeBuilder); + + // Verify that when builder is used value set in parameters is used + // even if config is set + getAbfsStore(fs).getAbfsConfiguration() + .setFooterReadBufferSize(footerReadBufferSizeConfig); + iStream = builder.build().get(); + verifyConfigValueInStream(iStream, footerReadBufferSizeBuilder); + + // Verify that when the builder is used and parameter in builder is not set, + // the value set in configuration is used + getAbfsStore(fs).getAbfsConfiguration() + .setFooterReadBufferSize(footerReadBufferSizeConfig); + builder = fs.openFile(testFilePath); + iStream = builder.build().get(); + verifyConfigValueInStream(iStream, footerReadBufferSizeConfig); + } + + private void verifyConfigValueInStream(final FSDataInputStream inputStream, + final int expectedValue) { + AbfsInputStream stream = (AbfsInputStream) inputStream.getWrappedStream(); + Assertions.assertThat(stream.getFooterReadBufferSize()) + .describedAs( + "Footer Read Buffer Size Value Is Not As Expected") + .isEqualTo(expectedValue); + } + + private Path createPathAndFileWithContent(final AzureBlobFileSystem fs, + final int fileIdx, final int fileSize) throws Exception { + String fileName = methodName.getMethodName() + fileIdx; + byte[] fileContent = getRandomBytesArray(fileSize); + return createFileWithContent(fs, fileName, fileContent); + } + + private FutureDataInputStreamBuilder getParameterizedBuilder(final Path path, + final AzureBlobFileSystem fs, int footerReadBufferSize) throws Exception { + FutureDataInputStreamBuilder builder = fs.openFile(path); + builder.opt(ConfigurationKeys.AZURE_FOOTER_READ_BUFFER_SIZE, + footerReadBufferSize); + return builder; + } + + private AzureBlobFileSystem getFileSystem(final boolean optimizeFooterRead, + final int fileSize) throws IOException { final AzureBlobFileSystem fs = getFileSystem(); AzureBlobFileSystemStore store = getAbfsStore(fs); store.getAbfsConfiguration().setOptimizeFooterRead(optimizeFooterRead); @@ -398,8 +449,8 @@ private AzureBlobFileSystem getFileSystem(boolean optimizeFooterRead, return fs; } - private AzureBlobFileSystem getFileSystem(boolean optimizeFooterRead, - int fileSize, int footerReadBufferSize) throws IOException { + private AzureBlobFileSystem getFileSystem(final boolean optimizeFooterRead, + final int fileSize, final int footerReadBufferSize) throws IOException { final AzureBlobFileSystem fs = getFileSystem(); AzureBlobFileSystemStore store = getAbfsStore(fs); store.getAbfsConfiguration().setOptimizeFooterRead(optimizeFooterRead); From 2b2ee8e36b820e241b8aa7352df19dc4d03b0058 Mon Sep 17 00:00:00 2001 From: Anuj Modi Date: Thu, 23 Nov 2023 22:18:04 -0800 Subject: [PATCH 7/8] Addressing Comments --- .../ITestAbfsInputStreamReadFooter.java | 80 ++++++++++--------- 1 file changed, 42 insertions(+), 38 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java index 17229ca00b4c8..fc9698e5d44f0 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java @@ -21,9 +21,9 @@ import java.io.IOException; import java.util.Map; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FutureDataInputStreamBuilder; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore; -import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; import org.assertj.core.api.Assertions; import org.junit.Test; @@ -38,6 +38,7 @@ import static java.lang.Math.max; import static java.lang.Math.min; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_FOOTER_READ_BUFFER_SIZE; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_FOOTER_READ_BUFFER_SIZE; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; @@ -376,42 +377,45 @@ private void testPartialReadWithSomeData(final FileSystem fs, @Test public void testFooterReadBufferSizeConfiguration() throws Exception { - final AzureBlobFileSystem fs = getFileSystem(); - Path testFilePath = createPathAndFileWithContent(fs, 0, ONE_KB); - final int footerReadBufferSizeConfig = 4 * ONE_KB; - final int footerReadBufferSizeBuilder = 5 * ONE_KB; - - // Verify that default value is used if nothing is set explicitly - FSDataInputStream iStream = fs.open(testFilePath); - verifyConfigValueInStream(iStream, DEFAULT_FOOTER_READ_BUFFER_SIZE); - - // Verify that value set in config is used if not builder is not used - getAbfsStore(fs).getAbfsConfiguration() - .setFooterReadBufferSize(footerReadBufferSizeConfig); - iStream = fs.open(testFilePath); - verifyConfigValueInStream(iStream, footerReadBufferSizeConfig); - - // Verify that when builder is used value set in parameters is used - FutureDataInputStreamBuilder builder = fs.openFile(testFilePath); - builder.opt(ConfigurationKeys.AZURE_FOOTER_READ_BUFFER_SIZE, - footerReadBufferSizeBuilder); - iStream = builder.build().get(); - verifyConfigValueInStream(iStream, footerReadBufferSizeBuilder); - - // Verify that when builder is used value set in parameters is used - // even if config is set - getAbfsStore(fs).getAbfsConfiguration() - .setFooterReadBufferSize(footerReadBufferSizeConfig); - iStream = builder.build().get(); - verifyConfigValueInStream(iStream, footerReadBufferSizeBuilder); - - // Verify that when the builder is used and parameter in builder is not set, - // the value set in configuration is used - getAbfsStore(fs).getAbfsConfiguration() - .setFooterReadBufferSize(footerReadBufferSizeConfig); - builder = fs.openFile(testFilePath); - iStream = builder.build().get(); - verifyConfigValueInStream(iStream, footerReadBufferSizeConfig); + Configuration config = new Configuration(this.getRawConfiguration()); + config.unset(AZURE_FOOTER_READ_BUFFER_SIZE); + try (AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(config)){ + Path testFilePath = createPathAndFileWithContent(fs, 0, ONE_KB); + final int footerReadBufferSizeConfig = 4 * ONE_KB; + final int footerReadBufferSizeBuilder = 5 * ONE_KB; + + // Verify that default value is used if nothing is set explicitly + FSDataInputStream iStream = fs.open(testFilePath); + verifyConfigValueInStream(iStream, DEFAULT_FOOTER_READ_BUFFER_SIZE); + + // Verify that value set in config is used if not builder is not used + getAbfsStore(fs).getAbfsConfiguration() + .setFooterReadBufferSize(footerReadBufferSizeConfig); + iStream = fs.open(testFilePath); + verifyConfigValueInStream(iStream, footerReadBufferSizeConfig); + + // Verify that when builder is used value set in parameters is used + FutureDataInputStreamBuilder builder = fs.openFile(testFilePath); + builder.opt(AZURE_FOOTER_READ_BUFFER_SIZE, + footerReadBufferSizeBuilder); + iStream = builder.build().get(); + verifyConfigValueInStream(iStream, footerReadBufferSizeBuilder); + + // Verify that when builder is used value set in parameters is used + // even if config is set + getAbfsStore(fs).getAbfsConfiguration() + .setFooterReadBufferSize(footerReadBufferSizeConfig); + iStream = builder.build().get(); + verifyConfigValueInStream(iStream, footerReadBufferSizeBuilder); + + // Verify that when the builder is used and parameter in builder is not set, + // the value set in configuration is used + getAbfsStore(fs).getAbfsConfiguration() + .setFooterReadBufferSize(footerReadBufferSizeConfig); + builder = fs.openFile(testFilePath); + iStream = builder.build().get(); + verifyConfigValueInStream(iStream, footerReadBufferSizeConfig); + } } private void verifyConfigValueInStream(final FSDataInputStream inputStream, @@ -433,7 +437,7 @@ private Path createPathAndFileWithContent(final AzureBlobFileSystem fs, private FutureDataInputStreamBuilder getParameterizedBuilder(final Path path, final AzureBlobFileSystem fs, int footerReadBufferSize) throws Exception { FutureDataInputStreamBuilder builder = fs.openFile(path); - builder.opt(ConfigurationKeys.AZURE_FOOTER_READ_BUFFER_SIZE, + builder.opt(AZURE_FOOTER_READ_BUFFER_SIZE, footerReadBufferSize); return builder; } From 85a4d62229f56c66e9fe87e0f25a4461d328b8b5 Mon Sep 17 00:00:00 2001 From: Anuj Modi Date: Thu, 23 Nov 2023 22:37:30 -0800 Subject: [PATCH 8/8] Fixed Test Case --- .../fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java index fc9698e5d44f0..bf205879cb4a0 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsInputStreamReadFooter.java @@ -388,13 +388,14 @@ public void testFooterReadBufferSizeConfiguration() throws Exception { FSDataInputStream iStream = fs.open(testFilePath); verifyConfigValueInStream(iStream, DEFAULT_FOOTER_READ_BUFFER_SIZE); - // Verify that value set in config is used if not builder is not used + // Verify that value set in config is used if builder is not used getAbfsStore(fs).getAbfsConfiguration() .setFooterReadBufferSize(footerReadBufferSizeConfig); iStream = fs.open(testFilePath); verifyConfigValueInStream(iStream, footerReadBufferSizeConfig); // Verify that when builder is used value set in parameters is used + getAbfsStore(fs).getAbfsConfiguration().unset(AZURE_FOOTER_READ_BUFFER_SIZE); FutureDataInputStreamBuilder builder = fs.openFile(testFilePath); builder.opt(AZURE_FOOTER_READ_BUFFER_SIZE, footerReadBufferSizeBuilder);