[ https://issues.apache.org/jira/browse/HADOOP-1470?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12508021 ]
Doug Cutting commented on HADOOP-1470: -------------------------------------- I'm trying to understand the FSInputChecker and FSOutputChecker APIs, and finding it a bit tricky. 1. FSInputChecker and FSOutputChecker should have constructors that initialize fields if possible, rather than relying on subclass constructors to do this. This would make it clearer what subclasses are responsible for and what FSInputChecker does. 2. All the fields of FSInputChecker are 'protected' rather than private, yet I don't think all of them are actually used by subclasses. It'd make the contract of this class clearer if fewer fields were private. Also, protected fields in a public class should have javadoc describing how they're used. Could 'count-pos' be provided via a method? 3. ChecksumFSInputChecker#read(long,byte[],int,int) looks like it moves the file pointer. Or am I misreading? 4. Lots of the stuff handled in ChecksumFSInputChecker (like optimizing for stuff that's already in the buffer) seems like it should be handled by the generic code. Overall, the abstraction boundary seems a bit fuzzy. Ideally implementations should only have to implement readChunk and readChecksum, but in fact they must implement much more (seek & skip, which are non-trivial). What if the abstraction were changed so that pread semantics were in effect for readChunk: an absolute file position is always passed? Then the implementation classes could check their current position and seek as needed. The generic code could, to make things efficient, arrange to always read chunks sequentially, so that most of the time the implementation wouldn't have to actually seek. Perhaps then all of seek() and skip() could be handled in generic code? > 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 > > Attachments: GenericChecksum.patch, genericChecksum.patch, > InputChecker-01.java > > > 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.