hadoop-common-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Doug Cutting (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 Tue, 26 Jun 2007 19:03:26 GMT

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

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

> is it assumed that two readChunk()s on a stream should happen in parallel?

That'd be up to the implementation.  An optimized implementation might cache a file handle
and use it for sequential calls, but if a parallel call comes in, it could create a new file
handle.  It could potentially even keep a pool of file handles.  Or it could just keep a single
file handle and synchronize, seeking whenever the position is not the current position.  That's
probably the implementation we should start with, and optimize it later as needed.  But I
think such optimizations would be FileSystem-specific, not generic.

> Should readChunk change the position of the file descriptor?

The API does not care, since it always passes in the absolute position: there is no file position.
 An implementation should optimize for sequential reads, and perhaps also parallel reads,
but that's a secondary priority.  We may not even have to explicitly optimize for sequential
reads in ChecksumFileSystem, since most implementations of seek already implement that optimization.

> getChecksumSize(): We probabaly do not need this abstract method [...]

Okay.


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


Mime
View raw message