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-928) make checksums optional per FileSystem
Date Thu, 22 Feb 2007 20:43:05 GMT

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

Doug Cutting commented on HADOOP-928:

> return a (unbuffered) FSDataInputStream for open(f) and a BufferedFSDataInputStream for
open(f, bufferSize)

I'm not convinced we should ever return an unbuffered stream.  I think the open(f) call should
return a stream with the default buffer size.  Note that writes that are larger than the buffer
bypass the buffer, so applications that layer their own buffers on top don't pay a penalty
when using a buffered stream.  They might instead however use open(f, bufferSize), specifying
a small buffer size, to save memory.

Note also that a RAM-based FileSystem might disable buffering altogether--even when a bufferSize
is specified.

> make checksums optional per FileSystem
> --------------------------------------
>                 Key: HADOOP-928
>                 URL: https://issues.apache.org/jira/browse/HADOOP-928
>             Project: Hadoop
>          Issue Type: Improvement
>          Components: fs
>            Reporter: Doug Cutting
>         Assigned To: Hairong Kuang
>         Attachments: checksum.patch
> Checksumming is currently built into the base FileSystem class.  It should instead be
optional, with each FileSystem implementation electing whether to use the Hadoop-provided
checksum system, or to disable it, or to implement its own custom checksum system.
> To implement this, a ChecksumFileSystem implementation can be provided that wraps another
FileSystem implementation, implementing checksums as in Hadoop's current mandatory implementation
(i.e., as a separate crc file per file that's elided from directory listings).  The 'raw'
FileSystem methods would be removed.  FSDataInputStream and FSDataOutputStream would be made

This message is automatically generated by JIRA.
You can reply to this email to add a comment to the issue online.

View raw message