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

Sameer Paranjpye commented on HADOOP-1470:
------------------------------------------


> but we also need to minimize the amount of buffered, checksummed data. It's 
> thus preferable to keep the outermost, checksummed buffer small ...

Granted, but changing the readChunk() contract to allow multiple continguous 
chunks to be read at once doesn't impact buffering. The outermost buffer would 
still be small. However, when an application does a large read, the current API 
requires that the data be fetched piecemeal into the InputChecker buffer, 
checksummed and then copied to the application buffer. Allowing multiple 
contiguous chunks to be read simultaneously, would allow the application buffer 
to be passed directly to readChunk() and checksummed in place and thus save a 
copy.

This is a small performance gain, however, a more important reason is that 
constraining readChunk() to 1 chunk at a time makes it difficult to support 
preads.

Implementing efficient support for 'sequential preads' i.e. multiple pread 
invocations hitting contiguous byte ranges is quite tricky. An implementation 
has to be pretty clever in order to efficiently serve several concurrent 
threads all calling pread. It probably needs to keep a pool of file descriptors 
around and have some complicated logic to determine, among other things:
- if, in a pread invocation, the position on any of its available descriptors 
is close to the position in the call, so that buffered data can be used
- if the position in a call isn't a good match for any of the descriptors, then 
whether to seek on an existing descriptor or open a new one. Using an existing 
descriptor might cause buffered data useful for a subsequent call to be thrown 
away

On the other hand, an implementation thats not trying support 'sequential 
preads', can be a lot simpler. It also needs to keep a pool of descriptors. But 
in each call it can pick an arbitrary descriptor, seek, fetch and return the 
data. It would also do no buffering (except on 1 canonical descriptor which 
corresponds to the file position that the implementation reports to clients).

It's not clear that the 'sequential pread' access pattern occurs frequently in 
application contexts. It certainly doesn't in the use cases we've seen so far, 
they almost all have very little locality of reference across pread 
invocations. However, given the readChunk() API above, an application level 
pread would most likely be split into multiple readChunk() calls and require 
the 'sequential pread' access pattern to be supported. It feels like this 
access pattern is being imposed on implementations by the readChunk() API and 
not by application contexts and that is undesirable. If the readChunk() API 
allowed multiple chunks to be read at once, then an application level pread 
would likely result in a single call to readChunk() and would allow for a 
simple implementation.








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

Reply via email to