浏览代码

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 76d92eb2a22c71b5fcde88a9b4d2faec81a1cb9f)
sumangala-patki 3 年之前
父节点
当前提交
aa6a9cac72

+ 1 - 1
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java

@@ -89,7 +89,7 @@ public final class FileSystemConfigurations {
   public static final int MIN_LEASE_DURATION = 15;
   public static final int MIN_LEASE_DURATION = 15;
   public static final int MAX_LEASE_DURATION = 60;
   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_ENABLE_FLUSH = true;
   public static final boolean DEFAULT_DISABLE_OUTPUTSTREAM_FLUSH = true;
   public static final boolean DEFAULT_DISABLE_OUTPUTSTREAM_FLUSH = true;

+ 1 - 1
hadoop-tools/hadoop-azure/src/site/markdown/abfs.md

@@ -803,7 +803,7 @@ pattern is detected.
 `fs.azure.readaheadqueue.depth`: Sets the readahead queue depth in
 `fs.azure.readaheadqueue.depth`: Sets the readahead queue depth in
 AbfsInputStream. In case the set value is negative the read ahead queue depth
 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 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
  doing only random reads (non-sequential) or you are seeing throttling, you
   may try setting this value to 0.
   may try setting this value to 0.
 
 

+ 16 - 0
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java

@@ -28,6 +28,7 @@ import org.assertj.core.api.Assertions;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
@@ -48,6 +49,7 @@ import static org.mockito.Mockito.when;
 
 
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 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.AbfsHttpConstants.FORWARD_SLASH;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_READ_AHEAD_QUEUE_DEPTH;
 
 
 /**
 /**
  * Unit test AbfsInputStream.
  * Unit test AbfsInputStream.
@@ -569,6 +571,20 @@ public class TestAbfsInputStream extends
     testReadAheads(inputStream, FORTY_EIGHT_KB, SIXTEEN_KB);
     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,
   private void testReadAheads(AbfsInputStream inputStream,
       int readRequestSize,
       int readRequestSize,