[
https://issues.apache.org/jira/browse/HADOOP-1470?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12508272
]
Doug Cutting commented on HADOOP-1470:
--------------------------------------
It sounds like we're reaching convergence. The InputChecker abstract methods
might be something like:
{code}
/** Return position of beginning of chunk containing pos. */
long getChunkPosition(long pos);
/** Return the maximum length of chunks in this file. */
int getChunkSize();
/** Return the length of each checksum. */
int getChecksumSize();
/** Read a full chunk and its checksum at the specified position, returning its
length. */
int readChunk(long position, byte[] buffer, int start, int length, byte[]
checksum);
{code}
Is that sufficient that an input stream can be fully implemented w/o overriding
other methods?
Note that I've added the checksum into readChunk() to simplify synchronization
and pread. Does that look reasonable?
> 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.