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

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

Hairong Kuang commented on HADOOP-928:
--------------------------------------

Doug, thanks for your comments. These are great suggestions.

I am looking at the possibility of removing FSInputStream and FSOutputStream. I feel that
it is OK to remove FSOutputStream. But we probably should keep FSInputStream because FSDataInputStream
implictly requires that a raw input stream should be Seekable and PositionReadable. In hadoop,
we either pass in a FSInputStream which extends InputStream and implements Seekable and PositionReadable
or a FSInputChecker which extends FilterInputStream and implements Seekable and PositionReadable.


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

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