Doug Cutting commented on HADOOP-928:
-------------------------------------
It's hard to for me to see how to remove createRaw and openRaw from FileSystem.
My idea was that each FileSystem would now be responsible for constructing the FSDataOutputStream, a buffered, re-positionable DataOutputStream. Most would implement this by having an internal classes that implement only FSInputStream and FSOutputStream, then their open() and create() implementations will wrap returned values in a standard implementation, something like BufferedFSDataOutputStream and BufferedFSDataInputStream.ChecksumFileSystem would layer on top of this another buffer, and would pass down a small, fixed buffer size to the filesystem it wraps. Most calls to the underlying stream will be to read and write chunks larger than it's buffer and would hence bypass that buffer.
Does that make sense?
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
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
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.