You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-dev@hadoop.apache.org by "Doug Cutting (JIRA)" <ji...@apache.org> on 2007/01/25 05:18:49 UTC

[jira] Created: (HADOOP-928) make checksums optional per FileSystem

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: Doug Cutting


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.


[jira] Updated: (HADOOP-928) make checksums optional per FileSystem

Posted by "Hairong Kuang (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-928?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Hairong Kuang updated HADOOP-928:
---------------------------------

    Attachment: checksum1.patch

This patch incorporates most of Doug's comments. 

I finally decided to keep the FSInputStream interface, the raw file system input stream. The reason is that every filesystem needs to implement the raw stream interface in order to provide a data input stream, i.e. FSDataInputStream. Currently the default implementation of FSDataInputStream is a wrapper over the raw stream, providing a buffered, seekable, and position readable data stream.

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


[jira] Commented: (HADOOP-928) make checksums optional per FileSystem

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-928?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12472494 ] 

Doug Cutting commented on HADOOP-928:
-------------------------------------

Overall this looks great.  A few minor issues that I see so far are:

I think we should add an abstract FilterFileSystem, that wraps a FileSystem and delegates all methods to the wrapped FileSystem.  This can be used by ChecksumFileSystem and PhasedFileSystem, making these classes a lot more readable.

FileSystem.pathToFile() should check that the uri starts with 'file://', and perhaps be a LocalFileSystem method.  As it stands, it will return a File for an HDFS path, which makes no sense.

Do we still need FSInputStream and FSOutputStream for anything?  Or can these be removed?

This patch indents four spaces per level, rather than the preferred two.

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


[jira] Commented: (HADOOP-928) make checksums optional per FileSystem

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-928?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12468456 ] 

Doug Cutting commented on HADOOP-928:
-------------------------------------

> keep all the raw methods in the FileSystem interface
> DistributedFileSystem and LocalFileSystem extend ChecksumFileSystem and implement all the raw methods

I think we only need to keep the raw methods if you intend to subclass to add checksums.  If instead, ChecksumFileSystem wraps another FileSystem, then one can do something like:

class RawHDFS extends FileSystem {
  public RawHDFS() { ... }
  .. implement open, create, list, etc. ...
}
abstract class ChecksumFS extends FilteredFileSystem {
  public ChecksumFS(FileSystem fs) { super(fs); }
  .. implement open, create, list, etc. ...
}
class ChecksumHDFS extends ChecksumFileSystem {
  public ChecksumHDFS() { super(new RawHDFS()); }
}

Does that make sense?  But perhaps you prefer to subclass for some reason?  I like the wrapper based approach primarily because it eliminates all the 'raw' methods.


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


[jira] Commented: (HADOOP-928) make checksums optional per FileSystem

Posted by "Hairong Kuang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-928?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12468459 ] 

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

Hi Doug! Yes, wrapping eliminates all the raw methods and thus makes the interface much cleaner. Thanks!

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


[jira] Commented: (HADOOP-928) make checksums optional per FileSystem

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-928?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12468425 ] 

Doug Cutting commented on HADOOP-928:
-------------------------------------

> It's hard to for me to see how to remove createRaw and openRaw from FileSystem.

My idea was that each FileSystem would now be responsible for constructing the FSDataOutputStream, a buffered, re-positionable DataOutputStream.  Most would implement this by having an internal classes that implement only FSInputStream and FSOutputStream, then their open() and create() implementations will wrap returned values in a standard implementation, something like BufferedFSDataOutputStream and BufferedFSDataInputStream.

ChecksumFileSystem would layer on top of this another buffer, and would pass down a small, fixed buffer size to the filesystem it wraps.  Most calls to the underlying stream will be to read and write chunks larger than it's buffer and would hence bypass that buffer.

Does that make sense?


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


[jira] Commented: (HADOOP-928) make checksums optional per FileSystem

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-928?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12476235 ] 

Doug Cutting commented on HADOOP-928:
-------------------------------------

> the reason that I set the inner buffer very small is to by-pass the inner buffer and hence avoid one more data copy

Yes, that makes sense, thanks for clarifying.  But unless I missed something, in ChecksumFileSystem#create(Path, int bufferSize), the inner and outer buffers are both bufferSize.

Also, a competing concern is that data not sit in buffers too long before it is checksummed.  Since we use many long-lived multi-megabyte buffers when sorting, this is a real concern.  So another strategy might be to use a small outer buffer and a large inner buffer, and assume that the cost of the extra copy is negligible (or at least warranted).  That way data would be checksummed sooner, and memory corruption in the client could be more reliably detected, but it does require an extra copy.  That was the strategy I assumed when I suggested using large inner buffers and small outer buffers.  It's probably worth benchmarking this at some point, although I'd rather not hold up this issue any longer.

So can you please just check whether my analysis of ChecksumFileSystem#create(Path, int bufferSize) above is correct?  Thanks!

> 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, checksum1.patch, checksum2.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.


[jira] Updated: (HADOOP-928) make checksums optional per FileSystem

Posted by "Hairong Kuang (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-928?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Hairong Kuang updated HADOOP-928:
---------------------------------

    Attachment: checksum.patch

This patch includes two major changes to hadoop file systems:
1. separating the raw FileSystem interface from the ChecksumFileSystem interface.
2. separating checksum generation & checksum checking from FSDataOutputStream & FSDataInputStream respectively

Please review and give me feedback on the changes.

I simply removed all the raw methods from the FileSystem interface. Should I keep them and mark them as deprecated for the backward compatibility purpose?

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


[jira] Updated: (HADOOP-928) make checksums optional per FileSystem

Posted by "Hairong Kuang (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-928?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Hairong Kuang updated HADOOP-928:
---------------------------------

    Status: Patch Available  (was: Open)

> 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, checksum1.patch, checksum2.patch, checksum3.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.


[jira] Commented: (HADOOP-928) make checksums optional per FileSystem

Posted by "Hairong Kuang (JIRA)" <ji...@apache.org>.
    [ 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.


[jira] Commented: (HADOOP-928) make checksums optional per FileSystem

Posted by "Hairong Kuang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-928?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12468453 ] 

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

Yes, all you said makes sense. But I'd like to make two changes to your original proposal:
1. keep all the raw methods in the FileSystem interface.
2. Remove Summer from FSDataOutputStream and make it an interface; remove Checker from FSDataInputStream and make it an interface. So DistributedFileSystem can have its own checksum implementation.

So FileSystem, ChecksumFileSystem, DistributedFileSystem/LocalFileSystem would be like the following:
1. FileSystem
For all the raw methods:
* openRaw returns an FSInputStream specific to each file system
* createRaw returns an FSOutputStream specific to each file system
* deleteRaw delete the given file
and etc.
For all the non-raw methods,
* open return a buffered, position-cached dataInputStream;
* create returns a buffered, position cached DataOutputStream;
* delete is the same as deleteRaw;
and etc.

2. ChecksumFileSystem
* open returns a buffered, position-cached, checksumed DataInputStream;
* create returns a buffered, position-cached, checksumed dataOutputStream;
* delete removes the given file and its checksum file
and etc.

3. DistributedFileSystem and LocalFileSystem extend ChecksumFileSystem and implement all the raw methods.

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


[jira] Updated: (HADOOP-928) make checksums optional per FileSystem

Posted by "Hairong Kuang (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-928?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Hairong Kuang updated HADOOP-928:
---------------------------------

    Attachment: checksum3.patch

It sounds good. The new patch reflects the suggestion. Hope that the patch does not break anything. :-)

> 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, checksum1.patch, checksum2.patch, checksum3.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.


[jira] Commented: (HADOOP-928) make checksums optional per FileSystem

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
    [ 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 interfaces.

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


[jira] Assigned: (HADOOP-928) make checksums optional per FileSystem

Posted by "Hairong Kuang (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-928?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Hairong Kuang reassigned HADOOP-928:
------------------------------------

    Assignee: Hairong Kuang  (was: Doug Cutting)

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


[jira] Commented: (HADOOP-928) make checksums optional per FileSystem

Posted by "Hairong Kuang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-928?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12474865 ] 

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

How about the following proposal to remove FSInputStream?

1. Replace current FSInputStream with FSDataInputStream.
  * FSDataInputStream extends DataInputStream and implemets Seekable and PositionedReadable.
  * Every file system including ChecksumedFileSystem provides its own specific implementation of FSDataInputStream.
  * Open returns a FSDataInputStream.
2. Replace current FSDataInputStream with BufferedFSDataInputStream.
  * BufferedFSDataInputStream extends FSDataInputStream, providing a buffered and position cached implementation of FSDataInputStream.
  * Open(file, buffersize) returns a BufferedFSDataInputStream.

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


[jira] Commented: (HADOOP-928) make checksums optional per FileSystem

Posted by "Hairong Kuang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-928?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12475133 ] 

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

I did not make it clear. I meant that both open(f) and open(f, bufferSize) return a FSDataInputStream. But most implementations return a (unbuffered) FSDataInputStream for open(f) and a BufferedFSDataInputStream for open(f, bufferSize);

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


[jira] Updated: (HADOOP-928) make checksums optional per FileSystem

Posted by "Hairong Kuang (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-928?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Hairong Kuang updated HADOOP-928:
---------------------------------

    Attachment: checksum2.patch

> 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, checksum1.patch, checksum2.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.


[jira] Commented: (HADOOP-928) make checksums optional per FileSystem

Posted by "Hairong Kuang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-928?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12476049 ] 

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

OK, I will keep FSInputChecker and FSOutputSummer private within ChecksumFileSystem so far.

As for setting bufferSize in ChecksumFileSystem#create(Path, int bufferSize) and ChecksumFileSystem#open(Path, int bufferSize), the reason that I set the inner buffer very small is to by-pass the inner buffer and hence avoid one more data copy.



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


[jira] Updated: (HADOOP-928) make checksums optional per FileSystem

Posted by "Hairong Kuang (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-928?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Hairong Kuang updated HADOOP-928:
---------------------------------

    Comment: was deleted

> 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, checksum1.patch, checksum2.patch, checksum3.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.


[jira] Commented: (HADOOP-928) make checksums optional per FileSystem

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-928?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12476311 ] 

Doug Cutting commented on HADOOP-928:
-------------------------------------

The sum buffer should probably be no smaller than the default and also should be flushed no more frequently than the main buffer.  So that would indicate that it be sized to the larger of io.file.buffer.size and bufferSize/bytesPerSum.  Does that sound right to you?

> 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, checksum1.patch, checksum2.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.


[jira] Updated: (HADOOP-928) make checksums optional per FileSystem

Posted by "Hairong Kuang (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-928?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Hairong Kuang updated HADOOP-928:
---------------------------------

    Attachment:     (was: checksum.patch)

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


[jira] Commented: (HADOOP-928) make checksums optional per FileSystem

Posted by "Hadoop QA (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-928?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12496530 ] 

Hadoop QA commented on HADOOP-928:
----------------------------------

Integrated in Hadoop-Nightly #91 (See http://lucene.zones.apache.org:8080/hudson/job/Hadoop-Nightly/91/)

> 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
>             Fix For: 0.12.0
>
>         Attachments: checksum.patch, checksum1.patch, checksum2.patch, checksum3.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.


[jira] Commented: (HADOOP-928) make checksums optional per FileSystem

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-928?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12476040 ] 

Doug Cutting commented on HADOOP-928:
-------------------------------------

Sorry, this patch does not apply cleanly to current trunk.  Other patches were recently committed that conflict with it.

Do FSInputChecker and FSOutputSummer need to be public?  Aren't they only used by ChecksumFileSystem?  Eventually we might want to expose these publicly, if, e.g., HDFS's eventual built-in checksum implementation shares code with them, but we also might not, since that mechanism might be independent.  So, for now, we should probably keep these package-private or even priavate within ChecksumFileSystem, the only place they're used.

In ChecksumFileSystem#create(Path, int bufferSize) it looks like two buffers of bufferSize are created.  I think only the inner buffer, created by the underlying raw filesystem, should be that size, which can be quite large, while the outer buffer should be quite small, no larger than bytesPerSum.

Similarly, in ChecksumFileSystem#open(Path, int bufferSize), I think the inner buffer should be large, to minimize seeks, system calls, etc., while the outer buffer should be quite small, no larger than bytesPerSum.

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


[jira] Commented: (HADOOP-928) make checksums optional per FileSystem

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-928?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12475127 ] 

Doug Cutting commented on HADOOP-928:
-------------------------------------

I'd figured that FSDataInputStream would be the interface that we specify in all public APIs, and that BufferedFSDataInputStream would be the implementation class that's generally used.  So the signature for FileSystem#open(Path, int buffersize) would return simply FSDataInputStream, but most implementations would do so by using BufferedFSDataInputStream.  Put another way, I don't see any reason to have that method's signature return BufferedFSDataInputStream.

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


[jira] Commented: (HADOOP-928) make checksums optional per FileSystem

Posted by "Hairong Kuang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-928?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12468417 ] 

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

> The 'raw' FileSystem methods would be removed.

It's hard to for me to see how to remove createRaw and openRaw from FileSystem. The method "create" returns a buffered, position-cached, and/or checksummed stream, i.e., FSDataOutputStream, while the method "createRaw" returns a raw I/O stream, i.e. FSOutputStream. They seem to serve different purposes no matter the file system is checksummed or not.

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


[jira] Updated: (HADOOP-928) make checksums optional per FileSystem

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-928?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Doug Cutting updated HADOOP-928:
--------------------------------

       Resolution: Fixed
    Fix Version/s: 0.12.0
           Status: Resolved  (was: Patch Available)

I just committed this.  Thanks, Hairong!

> 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
>             Fix For: 0.12.0
>
>         Attachments: checksum.patch, checksum1.patch, checksum2.patch, checksum3.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.


[jira] Updated: (HADOOP-928) make checksums optional per FileSystem

Posted by "Hairong Kuang (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-928?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Hairong Kuang updated HADOOP-928:
---------------------------------

    Attachment: checksum.patch

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


[jira] Commented: (HADOOP-928) make checksums optional per FileSystem

Posted by "Hairong Kuang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-928?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12476306 ] 

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

Yes, Doug. I think what you said about a small outer buffer and a big inner buffer makes sense. Personally I feel that for checksumming, the idea buffer size should be bytesPerSum. But I am not sure of the cost of data copying. After both this patch and the rack-aware performance patch are committed, I plan to work on some benchmarks on file I/O. I will experiment with two buffer size strategies then.

For the sum file buffer, I set its size to be the same as to the outer buffer. My intention is to use the same buffer size to read both data and checksums. Another option is to set the sum buffer size to be (outer buffer size/bytesPerSum), or as you suggested, set it to be the default buffer size. Please let me know what you think.

I also checked ChecksumFileSystem#create. Yes, the data stream should use a small buffer size. I will make the change.

> 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, checksum1.patch, checksum2.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.


[jira] Commented: (HADOOP-928) make checksums optional per FileSystem

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-928?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12476239 ] 

Doug Cutting commented on HADOOP-928:
-------------------------------------

One more thing: the sum file buffer should also be kept small, probably to the default (io.file.buffer.size).  Currently the patch passes the bufferSize parameter there, rather than letting it default.

> 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, checksum1.patch, checksum2.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.