steveloughran commented on code in PR #3964:
URL: https://github.com/apache/hadoop/pull/3964#discussion_r855311177


##########
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/TestVectoredReadUtils.java:
##########
@@ -207,6 +207,35 @@ public void testSortAndMergeMoreCases() throws Exception {
             VectoredReadUtils.isOrderedDisjoint(outputList, 1, 800));
 
   }
+
+  @Test
+  public void testMaxSizeZeroDisablesMering() throws Exception {
+    List<FileRange> randomRanges = Arrays.asList(
+            new FileRangeImpl(3000, 110),
+            new FileRangeImpl(3000, 100),
+            new FileRangeImpl(2100, 100)
+    );
+
+    List<CombinedFileRange> combinedFileRanges = VectoredReadUtils
+            .sortAndMergeRanges(randomRanges, 1, 1, 0);
+    Assertions.assertThat(combinedFileRanges)

Review Comment:
   could you pull out this near duplication into a method and reuse?



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java:
##########
@@ -137,6 +137,9 @@ public class S3AInputStream extends FSInputStream 
implements  CanSetReadahead,
   private S3AInputPolicy inputPolicy;
   private long readahead = Constants.DEFAULT_READAHEAD_RANGE;
 
+  /** Vectored IO context. */
+  private VectoredIOContext vectoredIOContext;

Review Comment:
   finall?



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java:
##########
@@ -90,6 +98,7 @@ public S3AReadOpContext(
     this.inputPolicy = checkNotNull(inputPolicy);
     this.changeDetectionPolicy = checkNotNull(changeDetectionPolicy);
     this.readahead = readahead;
+    this.vectoredIOContext = vectoredIOContext;

Review Comment:
   checkNotNull()



##########
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md:
##########
@@ -55,6 +55,35 @@ it isn't, and some attempts to preserve the metaphor are 
"aggressively suboptima
 
 To make most efficient use of S3, care is needed.
 
+## <a name="vectoredIO"></a> Improving read performance using Vectored IO
+The S3A FileSystem supports implementation of vectored read api using which
+a client can provide a list of file ranges to read returning a future read
+object associated with each range. For full api specification please see
+[FSDataInputStream](hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md).

Review Comment:
   use a .html suffix
   
   use mvn site:site to generate the site to check links



##########
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md:
##########
@@ -55,6 +55,35 @@ it isn't, and some attempts to preserve the metaphor are 
"aggressively suboptima
 
 To make most efficient use of S3, care is needed.
 
+## <a name="vectoredIO"></a> Improving read performance using Vectored IO
+The S3A FileSystem supports implementation of vectored read api using which
+a client can provide a list of file ranges to read returning a future read
+object associated with each range. For full api specification please see
+[FSDataInputStream](hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md).
+
+The following properties can be configured to optimise vectored reads based
+on the client requirements.
+
+```xml
+<property>
+  <name>fs.s3a.vectored.read.min.seek.size</name>
+  <value>4 * 1024</value>

Review Comment:
   unless multiplication is used in evaluation of the config (is it} you need 
to put a value in.
   
   use .getLongBytes() with a default of "4k" 



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java:
##########
@@ -1066,4 +1066,31 @@ private Constants() {
    * Require that all S3 access is made through Access Points.
    */
   public static final String AWS_S3_ACCESSPOINT_REQUIRED = 
"fs.s3a.accesspoint.required";
+
+  /**
+   * What is the smallest reasonable seek such that we group
+   * ranges together during vectored read operation.
+   * Value : {@value}.
+   */
+  public static final String AWS_S3_VECTOR_READS_MIN_SEEK_SIZE =
+          "fs.s3a.vectored.read.min.seek.size";
+
+  /**
+   * What is the largest merged read size such that we group
+   * ranges together during vectored read.
+   * Setting this value to 0 will disable merging of ranges.
+   * Value : {@value}.
+   */
+  public static final String AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE =
+          "fs.s3a.vectored.read.max.merged.size";
+
+  /**
+   * Default minimum seek during vectored reads : {@value}.

Review Comment:
   say "bytes" just to be clear



##########
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MoreAsserts.java:
##########
@@ -86,4 +86,16 @@ public static <T> void 
assertFutureFailedExceptionally(CompletableFuture<T> futu
                     "completed exceptionally")
             .isTrue();
   }
+
+  /**
+   * Assert two same type of values.
+   * @param actual actual value.
+   * @param expected expected value.
+   * @param message error message to print in case of mismatch.
+   */
+  public static <T> void assertEqual(T actual, T expected, String message) {
+    Assertions.assertThat(actual)
+            .describedAs("Mismatch in " + message)

Review Comment:
   * use a %s
   



##########
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md:
##########
@@ -55,6 +55,35 @@ it isn't, and some attempts to preserve the metaphor are 
"aggressively suboptima
 
 To make most efficient use of S3, care is needed.
 
+## <a name="vectoredIO"></a> Improving read performance using Vectored IO
+The S3A FileSystem supports implementation of vectored read api using which
+a client can provide a list of file ranges to read returning a future read
+object associated with each range. For full api specification please see
+[FSDataInputStream](hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md).
+
+The following properties can be configured to optimise vectored reads based
+on the client requirements.
+
+```xml
+<property>
+  <name>fs.s3a.vectored.read.min.seek.size</name>
+  <value>4 * 1024</value>
+  <description>
+     What is the smallest reasonable seek such that we group
+     ranges together during vectored read operation.
+   </description>
+</property>
+<property>
+<name>fs.s3a.vectored.read.max.merged.size</name>
+<value>1024 * 1024</value>

Review Comment:
   use .getLongBytes() with a default of 1M



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java:
##########
@@ -810,6 +815,28 @@ public void readFully(long position, byte[] buffer, int 
offset, int length)
     }
   }
 
+  /**
+   * To override this value set property defined by
+   * {@link Constants#AWS_S3_VECTOR_READS_MIN_SEEK_SIZE} in configuration.
+   *
+   * {@inheritDoc}.
+   */
+  @Override
+  public int minSeekForVectorReads() {
+    return vectoredIOContext.getMinSeekForVectorReads();
+  }
+
+  /**
+   * To override this value set property defined by
+   * {@link Constants#AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE} in 
configuration.

Review Comment:
   remove the detail about having to set the configuration value so that 
if/when the openfile builder lets you do it too, the docs aren't obsolete



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java:
##########
@@ -51,4 +58,46 @@ public void testEOFRanges() throws Exception {
     fileRanges.add(new FileRangeImpl(DATASET_LEN, 100));
     testExceptionalVectoredRead(fs, fileRanges, "EOFException is expected");
   }
+
+  @Test
+  public void testMinSeekAndMaxSizeConfigsPropagation() throws Exception {
+    Configuration conf = getFileSystem().getConf();
+    S3ATestUtils.removeBaseAndBucketOverrides(conf,
+            Constants.AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE,
+            Constants.AWS_S3_VECTOR_READS_MIN_SEEK_SIZE);
+    S3ATestUtils.disableFilesystemCaching(conf);
+    final int configuredMinSeek = 1024;
+    final int configuredMaxSize = 2 * 1024;
+    conf.setInt(Constants.AWS_S3_VECTOR_READS_MIN_SEEK_SIZE, 
configuredMinSeek);
+    conf.setInt(Constants.AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE, 
configuredMaxSize);
+    try (S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) {
+      try (FSDataInputStream fis = fs.open(path(VECTORED_READ_FILE_NAME))) {
+        int newMinSeek = fis.minSeekForVectorReads();
+        int newMaxSize = fis.maxReadSizeForVectorReads();
+        assertEqual(newMinSeek, configuredMinSeek,
+                "configured s3a min seek for vectored reads");
+        assertEqual(newMaxSize, configuredMaxSize,
+                "configured s3a max size for vectored reads");
+      }
+    }
+  }
+
+  @Test
+  public void testMinSeekAndMaxSizeDefaultValues() throws Exception {
+    Configuration conf = getFileSystem().getConf();
+    S3ATestUtils.removeBaseAndBucketOverrides(conf,
+            Constants.AWS_S3_VECTOR_READS_MIN_SEEK_SIZE,
+            Constants.AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE);
+    S3ATestUtils.disableFilesystemCaching(conf);

Review Comment:
   not needed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to