HADOOP-17596. ABFS: Change default Readahead Queue Depth from num(processors) to const (#3106)

* HADOOP-17596. ABFS: Change default Readahead Queue Depth from num(processors) to const (#2795)
. Contributed by Sumangala Patki.

(cherry picked from commit 76d92eb2a2)
This commit is contained in:
sumangala-patki 2021-07-10 15:09:59 +05:30 committed by GitHub
parent 7cb91db575
commit aa6a9cac72
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
3 changed files with 18 additions and 2 deletions

View File

@ -89,7 +89,7 @@ public final class FileSystemConfigurations {
public static final int MIN_LEASE_DURATION = 15;
public static final int MAX_LEASE_DURATION = 60;
public static final int DEFAULT_READ_AHEAD_QUEUE_DEPTH = -1;
public static final int DEFAULT_READ_AHEAD_QUEUE_DEPTH = 2;
public static final boolean DEFAULT_ENABLE_FLUSH = true;
public static final boolean DEFAULT_DISABLE_OUTPUTSTREAM_FLUSH = true;

View File

@ -803,7 +803,7 @@ pattern is detected.
`fs.azure.readaheadqueue.depth`: Sets the readahead queue depth in
AbfsInputStream. In case the set value is negative the read ahead queue depth
will be set as Runtime.getRuntime().availableProcessors(). By default the value
will be -1. To disable readaheads, set this value to 0. If your workload is
will be 2. To disable readaheads, set this value to 0. If your workload is
doing only random reads (non-sequential) or you are seeing throttling, you
may try setting this value to 0.

View File

@ -28,6 +28,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
@ -48,6 +49,7 @@
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.FORWARD_SLASH;
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_READ_AHEAD_QUEUE_DEPTH;
/**
* Unit test AbfsInputStream.
@ -569,6 +571,20 @@ public void testDiffReadRequestSizeAndRAHBlockSize() throws Exception {
testReadAheads(inputStream, FORTY_EIGHT_KB, SIXTEEN_KB);
}
@Test
public void testDefaultReadaheadQueueDepth() throws Exception {
Configuration config = getRawConfiguration();
config.unset(FS_AZURE_READ_AHEAD_QUEUE_DEPTH);
AzureBlobFileSystem fs = getFileSystem(config);
Path testFile = new Path("/testFile");
fs.create(testFile);
FSDataInputStream in = fs.open(testFile);
Assertions.assertThat(
((AbfsInputStream) in.getWrappedStream()).getReadAheadQueueDepth())
.describedAs("readahead queue depth should be set to default value 2")
.isEqualTo(2);
}
private void testReadAheads(AbfsInputStream inputStream,
int readRequestSize,