Doug Cutting commented on HADOOP-928:
-------------------------------------
I'd figured that FSDataInputStream would be the interface that we specify in all public APIs, and that BufferedFSDataInputStream would be the implementation class that's generally used. So the signature for FileSystem#open(Path, int buffersize) would return simply FSDataInputStream, but most implementations would do so by using BufferedFSDataInputStream. Put another way, I don't see any reason to have that method's signature return BufferedFSDataInputStream.
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
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
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.