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

Doug Cutting commented on HADOOP-928:
-------------------------------------

> keep all the raw methods in the FileSystem interface
> DistributedFileSystem and LocalFileSystem extend ChecksumFileSystem and 
> implement all the raw methods

I think we only need to keep the raw methods if you intend to subclass to add 
checksums.  If instead, ChecksumFileSystem wraps another FileSystem, then one 
can do something like:

class RawHDFS extends FileSystem {
  public RawHDFS() { ... }
  .. implement open, create, list, etc. ...
}
abstract class ChecksumFS extends FilteredFileSystem {
  public ChecksumFS(FileSystem fs) { super(fs); }
  .. implement open, create, list, etc. ...
}
class ChecksumHDFS extends ChecksumFileSystem {
  public ChecksumHDFS() { super(new RawHDFS()); }
}

Does that make sense?  But perhaps you prefer to subclass for some reason?  I 
like the wrapper based approach primarily because it eliminates 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