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

Doug Cutting commented on HADOOP-1470:
--------------------------------------

Raghu, I think it is worth trying to keep as much as is reasonable generic, and 
no more.  If it indeed makes things considerably more complex to share the 
code, as you fear, then we probably should not.  But we should first try, no?

I'm not convinced that your ReadBuffer.java really handles all of the cases 
that might arise--it's pseudo code that hasn't been tested yet.  Things 
sometimes have a way of getting more complicated as they're used and debugged.  
It doesn't even show the checksumming, just handling of one exception and 
retry.  It also doesn't support an important property that we desire, where 
checksums are verified as late as possible, so that we catch more memory 
errors, not just disk errors.  So I don't really see how it illustrates the 
point I think you intend, that a non-generic version will be simpler.

I'm also not sure what your analogy is with HADOOP-1425.  In that case some 
folks don't like subclassing, and prefer an interface and static methods.  
That's fine, and we could do that here if you think such a design would be 
cleaner.  But no one is arguing there that we shouldn't share as much logic as 
possible, rather that discussion was about how the logic is shared.

Finally, I think it's okay to throw an exception in the client when the 
configured blocksize is not a multiple of the configured bytesPerSum.  So, if 
we think it will considerably simplify implementation, I don't see a problem 
with adding this restriction.



> Rework FSInputChecker and FSOutputSummer to support checksum code sharing 
> between ChecksumFileSystem and block level crc dfs
> ----------------------------------------------------------------------------------------------------------------------------
>
>                 Key: HADOOP-1470
>                 URL: https://issues.apache.org/jira/browse/HADOOP-1470
>             Project: Hadoop
>          Issue Type: Improvement
>          Components: fs
>    Affects Versions: 0.12.3
>            Reporter: Hairong Kuang
>            Assignee: Hairong Kuang
>             Fix For: 0.14.0
>
>
> Comment from Doug in HADOOP-1134:
> I'd prefer it if the CRC code could be shared with CheckSumFileSystem. In 
> particular, it seems to me that FSInputChecker and FSOutputSummer could be 
> extended to support pluggable sources and sinks for checksums, respectively, 
> and DFSDataInputStream and DFSDataOutputStream could use these. Advantages of 
> this are: (a) single implementation of checksum logic to debug and maintain; 
> (b) keeps checksumming as close to possible to data generation and use. This 
> patch computes checksums after data has been buffered, and validates them 
> before it is buffered. We sometimes use large buffers and would like to guard 
> against in-memory errors. The current checksum code catches a lot of such 
> errors. So we should compute checksums after minimal buffering (just 
> bytesPerChecksum, ideally) and validate them at the last possible moment 
> (e.g., through the use of a small final buffer with a larger buffer behind 
> it). I do not think this will significantly affect performance, and data 
> integrity is a high priority. 

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