[
https://issues.apache.org/jira/browse/HADOOP-1470?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12503690
]
Raghu Angadi edited comment on HADOOP-1470 at 6/11/07 10:47 PM:
----------------------------------------------------------------
Let me show what I had in my mind when I proposed "genericChecksum.java" above
on Jun 8th: Main and only aim of it is to share the read and retry loops.
readChunk() and readChecksum are not public interfaces in the sense, map/reduce
never invokes it. FSInputStream public interface does not change.
psuedo code of the class and how it is used. :
{code:title=inputChecker.java}
//First the use case:
FSInputChecker in ChecksumFileSystem would look like this:
class FSInputChecker extends FSInputStream {
InputCheker checker = new InputChecker(this);
public int read(buf, offset, len) {
return checker.read(buf, offset, len);
}
implement readChunk();
implement readChecksum() ;
seek etc slightly modify.
}
class InputChecker {
/// FSInputChecker supports readChunk() and readChecksum() as described in
// my comment
InputChecker(FSInputChecker in);
ReadInfo readInfo;
FSInputStream in;
// contract is same as inputStrem.read().
// following is too simplified. But real implementation will look a bit more
//involved but that meat is shared across the file systems, which I think is
// what this Jira wants.
int read(byte[] userBuf, int offset, int len) {
int bytesRead = 0
while ( bytesRead < len ) {
if ( there is data left in readInfo ) {
copy to userBuf;
} else {
RETRY_LOOP {
in.readChunk().
in.readChecksum();
compare the checksum
If there are mismatches, in.seekToNewSource() etc
}
}
}
resetReadInfo() { readInfo = 0; }
}
{code}
Hope this makes it clear where InputChecker I have in mind fits in. Note that
here FSInputChecker in ChecksumFileSystem and DFSInputStream in DFS, implement
readChunk() and reacChecksum() internally.
edit: reduced the "width" of the code
was:
Let me show what I had in my mind when I proposed "genericChecksum.java" above
on Jun 8th: Main and only aim of it is to share the read and retry loops.
readChunk() and readChecksum are not public interfaces in the sense, map/reduce
never invokes it. FSInputStream public interface does not change.
psuedo code of the class and how it is used. :
{code:title:inputChecker.java}
//First the use case:
FSInputChecker in ChecksumFileSystem would look like this:
class FSInputChecker extends FSInputStream {
InputCheker checker = new InputChecker(this);
public int read(buf, offset, len) {
return checker.read(buf, offset, len);
}
implement readChunk();
implement readChecksum() ;
seek etc slightly modify.
}
class InputChecker {
/// FSInputChecker supports readChunk() and readChecksum() as described in my
comment
InputChecker(FSInputChecker in);
ReadInfo readInfo;
FSInputStream in;
// contract is same as inputStrem.read().
// following is too simplified. But real implementation will look a bit more
involved but that meat is shared across the file systems, which I think is what
this
// Jira wants.
int read(byte[] userBuf, int offset, int len) {
int bytesRead = 0
while ( bytesRead < len ) {
if ( there is data left in readInfo ) {
copy to userBuf;
} else {
RETRY_LOOP {
in.readChunk().
in.readChecksum();
compare the checksum
If there are mismatches, in.seekToNewSource() etc
}
}
}
resetReadInfo() { readInfo = 0; }
}
{code}
Hope this makes it clear where InputChecker I have in mind fits in. Note that
here FSInputChecker in ChecksumFileSystem and DFSInputStream in DFS, implement
readChunk() and reacChecksum() internally.
> 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.