hadoop-common-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Hairong Kuang (JIRA)" <j...@apache.org>
Subject [jira] Commented: (HADOOP-1470) Rework FSInputChecker and FSOutputSummer to support checksum code sharing between ChecksumFileSystem and block level crc dfs
Date Mon, 11 Jun 2007 20:48:27 GMT

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

Hairong Kuang commented on HADOOP-1470:
---------------------------------------

Chatting with Raghu again. Hopefully we can reach a common ground. 
1. We need a method to read a chunk without checksum verification. 
2. I do not like the idea of passing both a local buffer and a user buffer to each read. Instead,
I'd like to just pass a buffer. Either it is a local buffer or a user buffer is up to each
implementation to decide it.
3. For the readChecksum() API, can we let it return a long for now?

So the propsed reading interface is like the following:

abstract class Checker extends FSInputStream {
Checksum sum;

/** read a checksum verified chunk into buf at offset
  * If there is a Checksum error, it retries a different replica
  * If the buf does not have enough space to hold the chunk, throws an IllegalArgumentException
  * returns the number of bytes read; -1 if end of file
  */
int readVerifiedChunk( byte[] buf, int offset );

/** read a chunk into buf at offset.
  * It ends at the a checksum boundary or end of file or at any pos which could be checksum
verified, for example, at a block boundary in block-level-crc dfs
  * data read have not been checksum verified
  * If the buf does not have enough space to hold the chunk, throws an IllegalArgumentException
  * returns the number of bytes read; -1 if end of file
  */
abstract int readChunk( byte[] buf, int offset, int len) throws IOException;

/** read a checksum */
abstract long readChecksum() throws IOException;
}

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


Mime
View raw message