Doug Cutting commented on HADOOP-928:
-------------------------------------
Overall this looks great. A few minor issues that I see so far are:
I think we should add an abstract FilterFileSystem, that wraps a FileSystem and delegates all methods to the wrapped FileSystem. This can be used by ChecksumFileSystem and PhasedFileSystem, making these classes a lot more readable.
FileSystem.pathToFile() should check that the uri starts with 'file://', and perhaps be a LocalFileSystem method. As it stands, it will return a File for an HDFS path, which makes no sense.
Do we still need FSInputStream and FSOutputStream for anything? Or can these be removed?
This patch indents four spaces per level, rather than the preferred two.
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.