You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by um...@apache.org on 2012/10/19 21:55:52 UTC

svn commit: r1400256 - in /hadoop/common/branches/branch-1: CHANGES.txt src/core/org/apache/hadoop/fs/FSDataOutputStream.java src/test/org/apache/hadoop/fs/TestLocalFileSystem.java

Author: umamahesh
Date: Fri Oct 19 19:55:51 2012
New Revision: 1400256

URL: http://svn.apache.org/viewvc?rev=1400256&view=rev
Log:
HADOOP-8861. FSDataOutputStream.sync should call flush() if the underlying wrapped stream is not Syncable. Contributed by Amareshwari Sriramadasu.


Modified:
    hadoop/common/branches/branch-1/CHANGES.txt
    hadoop/common/branches/branch-1/src/core/org/apache/hadoop/fs/FSDataOutputStream.java
    hadoop/common/branches/branch-1/src/test/org/apache/hadoop/fs/TestLocalFileSystem.java

Modified: hadoop/common/branches/branch-1/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1/CHANGES.txt?rev=1400256&r1=1400255&r2=1400256&view=diff
==============================================================================
--- hadoop/common/branches/branch-1/CHANGES.txt (original)
+++ hadoop/common/branches/branch-1/CHANGES.txt Fri Oct 19 19:55:51 2012
@@ -263,6 +263,9 @@ Release 1.2.0 - unreleased
     HADOOP-7827. jsp pages missing DOCTYPE tags. (Dave Vronay, Backport by
     Ivan Mitic via suresh)
 
+    HADOOP-8861. FSDataOutputStream.sync should call flush() if the underlying 
+    wrapped stream is not Syncable. (Amareshwari via umamahesh)
+
 Release 1.1.1 - Unreleased
 
   INCOMPATIBLE CHANGES

Modified: hadoop/common/branches/branch-1/src/core/org/apache/hadoop/fs/FSDataOutputStream.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1/src/core/org/apache/hadoop/fs/FSDataOutputStream.java?rev=1400256&r1=1400255&r2=1400256&view=diff
==============================================================================
--- hadoop/common/branches/branch-1/src/core/org/apache/hadoop/fs/FSDataOutputStream.java (original)
+++ hadoop/common/branches/branch-1/src/core/org/apache/hadoop/fs/FSDataOutputStream.java Fri Oct 19 19:55:51 2012
@@ -95,6 +95,8 @@ public class FSDataOutputStream extends 
   public void sync() throws IOException {
     if (wrappedStream instanceof Syncable) {
       ((Syncable)wrappedStream).sync();
+    } else {
+      wrappedStream.flush();
     }
   }
 }

Modified: hadoop/common/branches/branch-1/src/test/org/apache/hadoop/fs/TestLocalFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1/src/test/org/apache/hadoop/fs/TestLocalFileSystem.java?rev=1400256&r1=1400255&r2=1400256&view=diff
==============================================================================
--- hadoop/common/branches/branch-1/src/test/org/apache/hadoop/fs/TestLocalFileSystem.java (original)
+++ hadoop/common/branches/branch-1/src/test/org/apache/hadoop/fs/TestLocalFileSystem.java Fri Oct 19 19:55:51 2012
@@ -111,6 +111,39 @@ public class TestLocalFileSystem extends
     }
   }
 
+  public void testSyncable() throws IOException {
+    Configuration conf = new Configuration();
+    FileSystem fs = FileSystem.getLocal(conf).getRawFileSystem();
+    Path file = new Path(TEST_ROOT_DIR, "syncable");
+    FSDataOutputStream out = fs.create(file);;
+    final int bytesWritten = 1;
+    byte[] expectedBuf = new byte[] {'0', '1', '2', '3'};
+    try {
+      out.write(expectedBuf, 0, 1); 
+      out.sync();
+      verifyFile(fs, file, bytesWritten, expectedBuf);
+      out.write(expectedBuf, bytesWritten, expectedBuf.length-bytesWritten);
+      out.sync();
+      verifyFile(fs, file, expectedBuf.length, expectedBuf);
+    } finally {
+      out.close();
+    }   
+  }
+  
+  private void verifyFile(FileSystem fs, Path file, int bytesToVerify, 
+      byte[] expectedBytes) throws IOException {
+    FSDataInputStream in = fs.open(file);
+    try {
+      byte[] readBuf = new byte[bytesToVerify];
+      in.readFully(readBuf, 0, bytesToVerify);
+      for (int i=0; i<bytesToVerify; i++) {
+        assertEquals(expectedBytes[i], readBuf[i]);
+      }   
+    } finally {
+      in.close();
+    }   
+  }
+
   public void testCopy() throws IOException {
     Configuration conf = new Configuration();
     LocalFileSystem fs = FileSystem.getLocal(conf);