Hairong Kuang commented on HADOOP-928:
--------------------------------------
Yes, Doug. I think what you said about a small outer buffer and a big inner buffer makes sense. Personally I feel that for checksumming, the idea buffer size should be bytesPerSum. But I am not sure of the cost of data copying. After both this patch and the rack-aware performance patch are committed, I plan to work on some benchmarks on file I/O. I will experiment with two buffer size strategies then.
For the sum file buffer, I set its size to be the same as to the outer buffer. My intention is to use the same buffer size to read both data and checksums. Another option is to set the sum buffer size to be (outer buffer size/bytesPerSum), or as you suggested, set it to be the default buffer size. Please let me know what you think.
I also checked ChecksumFileSystem#create. Yes, the data stream should use a small buffer size. I will make the change.
make checksums optional per FileSystem
--------------------------------------
Key: HADOOP-928
URL: https://issues.apache.org/jira/browse/HADOOP-928
Project: Hadoop
Issue Type: Improvement
Components: fs
Reporter: Doug Cutting
Assigned To: Hairong Kuang
Attachments: checksum.patch, checksum1.patch, checksum2.patch
Checksumming is currently built into the base FileSystem class. It should instead be optional, with each FileSystem implementation electing whether to use the Hadoop-provided checksum system, or to disable it, or to implement its own custom checksum system.
To implement this, a ChecksumFileSystem implementation can be provided that wraps another FileSystem implementation, implementing checksums as in Hadoop's current mandatory implementation (i.e., as a separate crc file per file that's elided from directory listings). The 'raw' FileSystem methods would be removed. FSDataInputStream and FSDataOutputStream would be made interfaces.
----------------------------------------
Key: HADOOP-928
URL: https://issues.apache.org/jira/browse/HADOOP-928
Project: Hadoop
Issue Type: Improvement
Components: fs
Reporter: Doug Cutting
Assigned To: Hairong Kuang
Attachments: checksum.patch, checksum1.patch, checksum2.patch
Checksumming is currently built into the base FileSystem class. It should instead be optional, with each FileSystem implementation electing whether to use the Hadoop-provided checksum system, or to disable it, or to implement its own custom checksum system.
To implement this, a ChecksumFileSystem implementation can be provided that wraps another FileSystem implementation, implementing checksums as in Hadoop's current mandatory implementation (i.e., as a separate crc file per file that's elided from directory listings). The 'raw' FileSystem methods would be removed. FSDataInputStream and FSDataOutputStream would be made interfaces.
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.