[ 
https://issues.apache.org/jira/browse/HADOOP-928?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12468453
 ] 

Hairong Kuang commented on HADOOP-928:
--------------------------------------

Yes, all you said makes sense. But I'd like to make two changes to your 
original proposal:
1. keep all the raw methods in the FileSystem interface.
2. Remove Summer from FSDataOutputStream and make it an interface; remove 
Checker from FSDataInputStream and make it an interface. So 
DistributedFileSystem can have its own checksum implementation.

So FileSystem, ChecksumFileSystem, DistributedFileSystem/LocalFileSystem would 
be like the following:
1. FileSystem
For all the raw methods:
* openRaw returns an FSInputStream specific to each file system
* createRaw returns an FSOutputStream specific to each file system
* deleteRaw delete the given file
and etc.
For all the non-raw methods,
* open return a buffered, position-cached dataInputStream;
* create returns a buffered, position cached DataOutputStream;
* delete is the same as deleteRaw;
and etc.

2. ChecksumFileSystem
* open returns a buffered, position-cached, checksumed DataInputStream;
* create returns a buffered, position-cached, checksumed dataOutputStream;
* delete removes the given file and its checksum file
and etc.

3. DistributedFileSystem and LocalFileSystem extend ChecksumFileSystem and 
implement all the raw methods.

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

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.

Reply via email to