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

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

Sorry, this patch does not apply cleanly to current trunk.  Other patches were 
recently committed that conflict with it.

Do FSInputChecker and FSOutputSummer need to be public?  Aren't they only used 
by ChecksumFileSystem?  Eventually we might want to expose these publicly, if, 
e.g., HDFS's eventual built-in checksum implementation shares code with them, 
but we also might not, since that mechanism might be independent.  So, for now, 
we should probably keep these package-private or even priavate within 
ChecksumFileSystem, the only place they're used.

In ChecksumFileSystem#create(Path, int bufferSize) it looks like two buffers of 
bufferSize are created.  I think only the inner buffer, created by the 
underlying raw filesystem, should be that size, which can be quite large, while 
the outer buffer should be quite small, no larger than bytesPerSum.

Similarly, in ChecksumFileSystem#open(Path, int bufferSize), I think the inner 
buffer should be large, to minimize seeks, system calls, etc., while the outer 
buffer should be quite small, no larger than bytesPerSum.

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

Reply via email to