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


##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java:
##########
@@ -400,7 +398,9 @@ private void initiateRead() {
       for(int i = 0; i < ranges.size(); ++i) {
         FileRange range = ranges.get(i);
         buffers[i] = allocateRelease.getBuffer(false, range.getLength());
-        channel.read(buffers[i], range.getOffset(), i, this);
+        final ByteBuffer buffer = buffers[i];

Review Comment:
   Any specific reason for this refactoring and making it final? 



##########
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).
+
+```xml
+<property>
+  <name>fs.file.checksum.verify</name>
+  <value>false</value>
+</property>
+```
+
+(As you would expect, disabling checksum verification means that errors
+reading data may not be detected during the read operation.
+Use with care in production.)
+
+Filesystem instances which spit buffersduring vector read operations
+MUST declare this by returning `true`
+to the path capabilities probe `fs.capability.vectoredio.sliced`,
+and for the open stream in its `hasCapability()` method.
+
+
+The local filesystem will not slice buffers if the checksum file
+of `filename + ".crc"` is not found. This is not declared in the
+filesystem `hasPathCapability(filename, "fs.capability.vectoredio.sliced")`
+call, as no checks for the checksum file are made then.ddddddddddddddddd

Review Comment:
   typo: then.ddddddddddddddddd 



##########
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 checksum verification is disabled, vector reads of date will always 
returne

Review Comment:
   typo: returne



-- 
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: common-issues-unsubscr...@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
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