Issue Details (XML | Word | Printable)

Key: HADOOP-3250
Type: New Feature New Feature
Status: Closed Closed
Resolution: Fixed
Priority: Major Major
Assignee: Tsz Wo (Nicholas), SZE
Reporter: dhruba borthakur
Votes: 0
Watchers: 2
Operations

If you were logged in you would be able to see more operations.
Hadoop Common

Extend FileSystem API to allow appending to files

Created: 14/Apr/08 07:57 PM   Updated: 06/Oct/08 10:09 PM
Return to search
Component/s: fs
Affects Version/s: None
Fix Version/s: 0.18.0

Time Tracking:
Not Specified

File Attachments:
  Size
Text File Licensed for inclusion in ASF works 3250_20080529.patch 2008-05-30 12:02 AM Tsz Wo (Nicholas), SZE 7 kB
Text File Licensed for inclusion in ASF works 3250_20080530.patch 2008-05-30 09:07 PM Tsz Wo (Nicholas), SZE 11 kB
Issue Links:
Blocker
 

Hadoop Flags: Reviewed
Resolution Date: 02/Jun/08 07:52 PM


 Description  « Hide
Provide an API to allow applications to append data to pre-existing files.

 All   Comments   Work Log   Change History   Subversion Commits      Sort Order: Ascending order - Click to sort in descending order
dhruba borthakur added a comment - 14/Apr/08 08:05 PM
One option for this API would be a new method in the FileSystem.open() that takes in the pathname to be opened (and permissions, etc) and returns an FSDataOutputStream. The next write into this stream will occur at the end of the file.

Tsz Wo (Nicholas), SZE added a comment - 21/Apr/08 08:59 PM
FileSystem.open currently returns a FSDataInputStream. Clearly, we cannot use an InputStream to do append. I guess we might want to define a new method, say "append", in the FileSystem API.

dhruba borthakur added a comment - 21/Apr/08 09:04 PM
My mistake. I wanted to that we should create another method called FileSystem.create() with an "append" flag.

Tsz Wo (Nicholas), SZE added a comment - 21/Apr/08 09:16 PM
  • Literally, the word "create" does not mean or include append operation.
  • It seems to me that permission, overwrite, replication and blockSize do not make sense in append. This is the current create header.
    public abstract FSDataOutputStream create(Path f,
          FsPermission permission,
          boolean overwrite,
          int bufferSize,
          short replication,
          long blockSize,
          Progressable progress) throws IOException;

Doug Cutting added a comment - 21/Apr/08 09:29 PM
I like the name 'append' for this.
public FSDataOutputStream append(Path f) {
  return append(f, getDefaultBufferSize(), null);
}
public FSDataOutputStream append(Path f, int bufferSize) {
  return append(f, buffersize, null);
}
public abstract FSDataOutputStream append(Path f, int bufferSize, Progressible progress);

dhruba borthakur added a comment - 21/Apr/08 11:33 PM
+1 on Doug's proposal.

If the path exists and is not being written to, then append() will return a OutputStream to the file. If the path does not exist, then append will generate a IOException.

The DFSClient will issue ClientProtocol.append() call to the namenode. If the file was being written to by another writer but it has exceeded the soft-lease period, then the namenode would start a distributed lease recovery. However, it will not make the client wait for the recovery to complete. The namenode will return an AlreadyBeingCreatedException as if the soft-limit has not yet expired.


Tsz Wo (Nicholas), SZE added a comment - 30/May/08 12:02 AM - edited
3250_20080529.patch:
  • Added append API to FileSystem
  • Implemented RawLocalFileSystem
  • throw new IOException("Not supported") for the other FileSystem subclasses
  • need some tests

Doug Cutting added a comment - 30/May/08 04:18 PM
I think you need to add an implementation for FTPFileSystem now too. Otherwise, this looks good to me.

Tsz Wo (Nicholas), SZE added a comment - 30/May/08 09:07 PM
3250_20080530.patch:
  • implemented FTPFileSystem
  • added tests

Hadoop QA added a comment - 01/Jun/08 10:56 AM
+1 overall. Here are the results of testing the latest attachment
http://issues.apache.org/jira/secure/attachment/12383145/3250_20080530.patch
against trunk revision 661918.

+1 @author. The patch does not contain any @author tags.

+1 tests included. The patch appears to include 4 new or modified tests.

+1 javadoc. The javadoc tool did not generate any warning messages.

+1 javac. The applied patch does not increase the total number of javac compiler warnings.

+1 findbugs. The patch does not introduce any new Findbugs warnings.

+1 release audit. The applied patch does not increase the total number of release audit warnings.

+1 core tests. The patch passed core unit tests.

+1 contrib tests. The patch passed contrib unit tests.

Test results: http://hudson.zones.apache.org/hudson/job/Hadoop-Patch/2527/testReport/
Findbugs warnings: http://hudson.zones.apache.org/hudson/job/Hadoop-Patch/2527/artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
Checkstyle results: http://hudson.zones.apache.org/hudson/job/Hadoop-Patch/2527/artifact/trunk/build/test/checkstyle-errors.html
Console output: http://hudson.zones.apache.org/hudson/job/Hadoop-Patch/2527/console

This message is automatically generated.


Doug Cutting added a comment - 02/Jun/08 06:41 PM
+1 This looks good to me!

Chris Douglas added a comment - 02/Jun/08 07:52 PM
I just committed this. Thanks, Nicholas

Hudson added a comment - 03/Jun/08 01:26 PM

Robert Chansler added a comment - 22/Jul/08 05:41 PM
Removed release note so as to not confuse folks about whether 1700 is done.