[ 
https://issues.apache.org/jira/browse/HADOOP-18296?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=18007927#comment-18007927
 ] 

ASF GitHub Bot commented on HADOOP-18296:
-----------------------------------------

mukund-thakur commented on code in PR #7732:
URL: https://github.com/apache/hadoop/pull/7732#discussion_r2214416106


##########
hadoop-common-project/hadoop-common/src/main/resources/core-default.xml:
##########
@@ -1364,6 +1364,24 @@
   <description>File space usage statistics refresh interval in 
msec.</description>
 </property>
 
+<property>
+  <name>fs.file.checksum.verify</name>
+  <value>true</value>
+  <description>
+    Should data read through the local filesystem (file://) URLs be verified 
aginst
+    the checksums stored in the associated checksum files?
+    Setting this to false skips loading the checksum files, reading data in 
checksum-aligned
+    blocks and verifying checksums. This may improve performance
+    when reading data, though it pushes the responsibility of detecting errors
+    into the file formats themselves, or the underlying storage system.
+    Even when verification is enabled, files without associated checksum files
+    .$FILENAME.crc are never be verified.
+    When fs.file.checksum.verify is false, vector reads of date will always 
return

Review Comment:
   typo : date to data



##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java:
##########
@@ -489,9 +512,27 @@ public void readVectored(final List<? extends FileRange> 
ranges,
       }
     }
 
+    private boolean delegateVectorReadsToInner() {

Review Comment:
   nit: this name seems a bit off to me. Also javadocs. 



##########
hadoop-common-project/hadoop-common/src/main/resources/core-default.xml:
##########
@@ -1364,6 +1364,24 @@
   <description>File space usage statistics refresh interval in 
msec.</description>
 </property>
 
+<property>
+  <name>fs.file.checksum.verify</name>
+  <value>true</value>
+  <description>
+    Should data read through the local filesystem (file://) URLs be verified 
aginst
+    the checksums stored in the associated checksum files?
+    Setting this to false skips loading the checksum files, reading data in 
checksum-aligned
+    blocks and verifying checksums. This may improve performance
+    when reading data, though it pushes the responsibility of detecting errors
+    into the file formats themselves, or the underlying storage system.
+    Even when verification is enabled, files without associated checksum files
+    .$FILENAME.crc are never be verified.

Review Comment:
   nit: typo extra "be"



##########
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractVectoredReadTest.java:
##########
@@ -619,4 +625,61 @@ protected <T extends Throwable> void 
verifyExceptionalVectoredRead(
       });
     }
   }
+
+  @Test
+  public void testBufferSlicing() throws Throwable {
+    describe("Test buffer slicing behavior in vectored IO");
+
+    final int numBuffers = 8;
+    final int bufferSize = S_4K;
+    long offset = 0;
+    final List<FileRange> fileRanges = new ArrayList<>();
+    for (int i = 0; i < numBuffers; i++) {
+      fileRanges.add(FileRange.createFileRange(offset, bufferSize));
+      // increment and add a non-binary-aligned gap, so as to force
+      // offsets to be misaligned with possible page sizes.
+      offset += bufferSize + 4000;
+    }
+    TrackingByteBufferPool trackerPool = 
TrackingByteBufferPool.wrap(getPool());
+    int unknownBuffers = 0;
+    boolean slicing;
+    try (FSDataInputStream in = openVectorFile()) {
+      slicing = in.hasCapability(VECTOREDIO_BUFFERS_SLICED);
+      LOG.info("Slicing is {} for vectored IO with stream {}", slicing, in);
+      in.readVectored(fileRanges, s -> trackerPool.getBuffer(isDirect, s), 
trackerPool::putBuffer);
+
+      // check that all buffers are from the the pool, unless they are sliced.
+      for (FileRange res : fileRanges) {
+        CompletableFuture<ByteBuffer> data = res.getData();
+        ByteBuffer buffer = awaitFuture(data);
+        Assertions.assertThat(buffer)
+            .describedAs("Buffer must not be null")
+            .isNotNull();
+        Assertions.assertThat(slicing || trackerPool.containsBuffer(buffer))
+            .describedAs("Buffer must be from the pool")
+            .isTrue();
+        try {
+          trackerPool.putBuffer(buffer);
+        } catch 
(TrackingByteBufferPool.ReleasingUnallocatedByteBufferException e) {
+          // this can happen if the buffer was sliced, as it is not in the 
pool.
+          if (!slicing) {
+            throw e;
+          }
+          LOG.info("Sliced buffer detected: {}", buffer);
+          unknownBuffers++;
+        }
+      }
+    }
+    try {
+      trackerPool.close();
+    } catch (TrackingByteBufferPool.LeakedByteBufferException e) {
+      if (!slicing) {
+        throw e;
+      }
+      LOG.info("Slicing is enabled; we saw leaked buffers: {} after {}"

Review Comment:
   shouldn't this LOG be outside of the catch block? 



##########
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md:
##########
@@ -665,8 +665,48 @@ support through an explicit `hasCapability()` probe:
 Stream.hasCapability("in:readvectored")
 ```
 
-Given the HADOOP-18296 problem with `ChecksumFileSystem` and direct buffers, 
across all releases,
-it is best to avoid using this API in production with direct buffers.
+#### Buffer Slicing
+
+[HADOOP-18296](https://issues.apache.org/jira/browse/HADOOP-18296),
+_Memory fragmentation in ChecksumFileSystem Vectored IO implementation_
+highlights that `ChecksumFileSystem` (which the default implementation of 
`file://`
+subclasses), may return buffers which are sliced subsets of buffers allocated
+through the `allocate()` function passed in.
+
+This will happen during reads with and without range coalescing.
+
+Checksum verification may be disabled by setting the option
+`fs.file.checksum.verify` to true (Hadoop 3.4.2 and later).

Review Comment:
   typo: option to false.





> Memory fragmentation in ChecksumFileSystem Vectored IO implementation.
> ----------------------------------------------------------------------
>
>                 Key: HADOOP-18296
>                 URL: https://issues.apache.org/jira/browse/HADOOP-18296
>             Project: Hadoop Common
>          Issue Type: Sub-task
>          Components: common
>    Affects Versions: 3.4.0
>            Reporter: Mukund Thakur
>            Assignee: Steve Loughran
>            Priority: Minor
>              Labels: fs, pull-request-available
>
> As we have implemented merging of ranges in the ChecksumFSInputChecker 
> 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).
>  
> Note this only happens for direct byte buffers.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org

Reply via email to