瀏覽代碼

HADOOP-7817. RawLocalFileSystem.append() should give FSDataOutputStream with accurate .getPos() (Contributed by kanaka kumar avvaru)

(cherry picked from commit 48ca23def1d1c28448a65238814070e79c8f4c4e)
(cherry picked from commit d21bc811d82f685b0a1338bc513d9a925d305a17)

(cherry picked from commit 62da8f6fa688b503ed124313bc2f6f664175d225)

 Conflicts:
	hadoop-common-project/hadoop-common/CHANGES.txt
        hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
Vinayakumar B 10 年之前
父節點
當前提交
783c99d292

+ 3 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -30,6 +30,9 @@ Release 2.6.5 - UNRELEASED
     HADOOP-12958. PhantomReference for filesystem statistics can trigger OOM
     (Sangjin Lee via jlowe)
 
+    HADOOP-7817. RawLocalFileSystem.append() should give FSDataOutputStream
+    with accurate .getPos() (kanaka kumar avvaru via vinayakumarb)
+
 Release 2.6.4 - 2016-02-11
 
   INCOMPATIBLE CHANGES

+ 4 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java

@@ -243,11 +243,13 @@ public class RawLocalFileSystem extends FileSystem {
     if (!exists(f)) {
       throw new FileNotFoundException("File " + f + " not found");
     }
-    if (getFileStatus(f).isDirectory()) {
+    FileStatus status = getFileStatus(f);
+    if (status.isDirectory()) {
       throw new IOException("Cannot append to a diretory (=" + f + " )");
     }
     return new FSDataOutputStream(new BufferedOutputStream(
-        new LocalFSFileOutputStream(f, true), bufferSize), statistics);
+        new LocalFSFileOutputStream(f, true), bufferSize), statistics,
+        status.getLen());
   }
 
   @Override

+ 34 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java

@@ -560,6 +560,40 @@ public class TestLocalFileSystem {
         resolved);
   }
 
+  @Test
+  public void testAppendSetsPosCorrectly() throws Exception {
+    FileSystem fs = fileSys.getRawFileSystem();
+    Path file = new Path(TEST_ROOT_DIR, "test-append");
+
+    fs.delete(file, true);
+    FSDataOutputStream out = fs.create(file);
+
+    try {
+      out.write("text1".getBytes());
+    } finally {
+      out.close();
+    }
+
+    // Verify the position
+    out = fs.append(file);
+    try {
+      assertEquals(5, out.getPos());
+      out.write("text2".getBytes());
+    } finally {
+      out.close();
+    }
+
+    // Verify the content
+    FSDataInputStream in = fs.open(file);
+    try {
+      byte[] buf = new byte[in.available()];
+      in.readFully(buf);
+      assertEquals("text1text2", new String(buf));
+    } finally {
+      in.close();
+    }
+  }
+
   @Test
   public void testFileStatusPipeFile() throws Exception {
     RawLocalFileSystem origFs = new RawLocalFileSystem();