[ 
https://issues.apache.org/jira/browse/HADOOP-18106?focusedWorklogId=781438&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-781438
 ]

ASF GitHub Bot logged work on HADOOP-18106:
-------------------------------------------

                Author: ASF GitHub Bot
            Created on: 15/Jun/22 01:25
            Start Date: 15/Jun/22 01:25
    Worklog Time Spent: 10m 
      Work Description: mukund-thakur commented on code in PR #4427:
URL: https://github.com/apache/hadoop/pull/4427#discussion_r897442780


##########
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractVectoredReadTest.java:
##########
@@ -167,36 +195,85 @@ public void testSomeRangesMergedSomeUnmerged() throws 
Exception {
     try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
       in.readVectored(fileRanges, allocate);
       validateVectoredReadResult(fileRanges, DATASET);
+      returnBuffersToPoolPostRead(fileRanges, pool);
+    }
+  }
+
+  @Test
+  public void testOverlappingRanges() throws Exception {
+    FileSystem fs = getFileSystem();
+    List<FileRange> fileRanges = getSampleOverlappingRanges();
+    try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
+      in.readVectored(fileRanges, allocate);
+      validateVectoredReadResult(fileRanges, DATASET);
+      returnBuffersToPoolPostRead(fileRanges, pool);
     }
   }
 
   @Test
   public void testSameRanges() throws Exception {
+    // Same ranges are special case of overlapping only.
     FileSystem fs = getFileSystem();
+    List<FileRange> fileRanges = getSampleSameRanges();
+    try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
+      in.readVectored(fileRanges, allocate);
+      validateVectoredReadResult(fileRanges, DATASET);
+      returnBuffersToPoolPostRead(fileRanges, pool);
+    }
+  }
+
+  protected List<FileRange> getSampleSameRanges() {
     List<FileRange> fileRanges = new ArrayList<>();
     fileRanges.add(new FileRangeImpl(8*1024, 1000));
     fileRanges.add(new FileRangeImpl(8*1024, 1000));
     fileRanges.add(new FileRangeImpl(8*1024, 1000));
+    return fileRanges;
+  }
+
+  protected List<FileRange> getSampleOverlappingRanges() {
+    List<FileRange> fileRanges = new ArrayList<>();
+    fileRanges.add(FileRange.createFileRange(100, 500));
+    fileRanges.add(FileRange.createFileRange(400, 500));
+    return fileRanges;
+  }
+  protected void validateUnsupportedOperation(FileSystem fs,
+                                            List<? extends FileRange> 
fileRanges)
+          throws Exception {
     CompletableFuture<FSDataInputStream> builder =
             fs.openFile(path(VECTORED_READ_FILE_NAME))
                     .build();
     try (FSDataInputStream in = builder.get()) {
+      LambdaTestUtils.intercept(UnsupportedOperationException.class,
+              () -> in.readVectored(fileRanges, allocate));
+    }
+  }
+
+  @Test
+  public void testSomeRandomNonOverlappingRanges() throws Exception {
+    FileSystem fs = getFileSystem();
+    List<FileRange> fileRanges = new ArrayList<>();
+    fileRanges.add(new FileRangeImpl(500, 100));
+    fileRanges.add(new FileRangeImpl(1000, 200));
+    fileRanges.add(new FileRangeImpl(50, 10));
+    fileRanges.add(new FileRangeImpl(10, 5));
+    try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {

Review Comment:
   Done. Read policy vectored is not introduced yet.





Issue Time Tracking
-------------------

    Worklog Id:     (was: 781438)
    Time Spent: 1h 10m  (was: 1h)

> Handle memory fragmentation in S3 Vectored IO implementation.
> -------------------------------------------------------------
>
>                 Key: HADOOP-18106
>                 URL: https://issues.apache.org/jira/browse/HADOOP-18106
>             Project: Hadoop Common
>          Issue Type: Sub-task
>          Components: fs/s3
>            Reporter: Mukund Thakur
>            Assignee: Mukund Thakur
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 1h 10m
>  Remaining Estimate: 0h
>
> As we have implemented merging of ranges in the S3AInputStream implementation 
> of vectored IO api, it can lead to memory fragmentation. Let me explain by 
> example.
>  
> Suppose client requests for 3 ranges. 
> 0-500, 700-1000 and 1200-1500.
> Now because of merging, all the above ranges will get merged into one and we 
> will allocate a big byte buffer of 0-1500 size but return sliced byte buffers 
> for the desired ranges.
> Now once the client is done reading all the ranges, it will only be able to 
> free the memory for requested ranges and memory of the gaps will never be 
> released for eg here (500-700 and 1000-1200).
>  



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

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

Reply via email to