浏览代码

HDFS-609. Create a file with the append flag does not work in HDFS.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hdfs/trunk@956719 13f79535-47bb-0310-9956-ffa450edef68
Thomas White 15 年之前
父节点
当前提交
3a59a72165

+ 3 - 0
CHANGES.txt

@@ -989,6 +989,9 @@ Release 0.21.0 - Unreleased
 
     HDFS-1000. Updates libhdfs to the new API for UGI (ddas)
 
+    HDFS-609. Create a file with the append flag does not work in HDFS.
+    (tomwhite)
+
 Release 0.20.3 - Unreleased
 
   IMPROVEMENTS

+ 3 - 3
src/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -219,11 +219,12 @@ public class DistributedFileSystem extends FileSystem {
 
   @Override
   public FSDataOutputStream create(Path f, FsPermission permission,
-    EnumSet<CreateFlag> flag, int bufferSize, short replication, long blockSize,
+    boolean overwrite, int bufferSize, short replication, long blockSize,
     Progressable progress) throws IOException {
 
     return new FSDataOutputStream(dfs.create(getPathName(f), permission,
-                   flag, replication, blockSize, progress, bufferSize),
+        overwrite ? EnumSet.of(CreateFlag.OVERWRITE) : EnumSet.of(CreateFlag.CREATE),
+        replication, blockSize, progress, bufferSize),
         statistics);
   }
   
@@ -240,7 +241,6 @@ public class DistributedFileSystem extends FileSystem {
 
   /**
    * Same as create(), except fails if parent directory doesn't already exist.
-   * @see #create(Path, FsPermission, EnumSet, int, short, long, Progressable)
    */
   public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
       EnumSet<CreateFlag> flag, int bufferSize, short replication,

+ 1 - 1
src/java/org/apache/hadoop/hdfs/HftpFileSystem.java

@@ -305,7 +305,7 @@ public class HftpFileSystem extends FileSystem {
 
   @Override
   public FSDataOutputStream create(Path f, FsPermission permission,
-      EnumSet<CreateFlag> flag, int bufferSize, short replication,
+      boolean overwrite, int bufferSize, short replication,
       long blockSize, Progressable progress) throws IOException {
     throw new IOException("Not supported");
   }

+ 1 - 1
src/test/hdfs/org/apache/hadoop/hdfs/TestDFSPermission.java

@@ -174,7 +174,7 @@ public class TestDFSPermission extends TestCase {
     // create the file/directory
     switch (op) {
     case CREATE:
-      FSDataOutputStream out = fs.create(name, permission, EnumSet.of(CreateFlag.OVERWRITE), 
+      FSDataOutputStream out = fs.create(name, permission, true, 
           conf.getInt("io.file.buffer.size", 4096),
           fs.getDefaultReplication(), fs.getDefaultBlockSize(), null);
       out.close();

+ 1 - 1
src/test/hdfs/org/apache/hadoop/hdfs/TestFSInputChecker.java

@@ -56,7 +56,7 @@ public class TestFSInputChecker extends TestCase {
   private void writeFile(FileSystem fileSys, Path name) throws IOException {
     // create and write a file that contains three blocks of data
     FSDataOutputStream stm = fileSys.create(name, new FsPermission((short)0777),
-        EnumSet.of(CreateFlag.OVERWRITE), fileSys.getConf().getInt("io.file.buffer.size", 4096),
+        true, fileSys.getConf().getInt("io.file.buffer.size", 4096),
         NUM_OF_DATANODES, BLOCK_SIZE, null);
     stm.write(expected);
     stm.close();

+ 0 - 93
src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreation.java

@@ -611,91 +611,6 @@ public class TestFileCreation extends junit.framework.TestCase {
     }
   }
   
-  /**
-   * Test file creation with all supported flags.
-   */
-  public void testFileCreationWithFlags() throws IOException {
-    Configuration conf = new HdfsConfiguration();
-    if (simulatedStorage) {
-      conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
-    }
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
-    FileSystem fs = cluster.getFileSystem();
-    Path path = new Path("/" + System.currentTimeMillis()
-        + "-testFileCreationWithFlags");
-    FSDataOutputStream out = null;
-
-    // append to a non-exist file, it should throw an IOException
-    try {
-      IOException expectedException = null;
-      EnumSet<CreateFlag> appendNoFile = EnumSet.of(CreateFlag.APPEND);
-      // this should throw a IOException, because the file does not exist
-      try {
-        out = createFileWithFlag(fs, path, 1, appendNoFile);
-      } catch (IOException e) {
-        expectedException = e;
-      } finally {
-        if (out != null)
-          out.close();
-      }
-      assertTrue(
-          "Append a non-exists file with no create flag should throw an IOException ",
-          expectedException != null);
-
-      // the file already exists, and recreate it with CreateFlag.APPEND,
-      // CreateFlag.CREATE. It will not throw any exception.
-      EnumSet<CreateFlag> appendAndCreate = EnumSet.of(CreateFlag.APPEND,
-          CreateFlag.CREATE);
-      out = createFileWithFlag(fs, path, 1, appendAndCreate);
-      out.close();
-
-      // the file already exists, and recreate it only with CreateFlag.CREATE
-      // flag. it should throw an IOException
-      expectedException = null;
-      EnumSet<CreateFlag> createExistsFile = EnumSet.of(CreateFlag.CREATE);
-      // this should throw a IOException, because the file already exists
-      try {
-        createFileWithFlag(fs, path, 1, createExistsFile);
-      } catch (IOException e) {
-        expectedException = e;
-      }
-      assertTrue(
-          "create a file which already exists should throw an IOException ",
-          expectedException != null);
-
-      // the file exists, recreate it with the flag of CreateFlag.OVERWRITE.
-      EnumSet<CreateFlag> overwriteFile = EnumSet.of(CreateFlag.OVERWRITE);
-      out = createFileWithFlag(fs, path, 1, overwriteFile);
-      out.close();
-
-      // the file exists, recreate it with the flag of CreateFlag.OVERWRITE
-      // together with CreateFlag.CREATE. It has the same effect as only specify
-      // CreateFlag.OVERWRITE.
-      EnumSet<CreateFlag> overwriteWithCreateFile = EnumSet.of(
-          CreateFlag.OVERWRITE, CreateFlag.CREATE);
-      out = createFileWithFlag(fs, path, 1, overwriteWithCreateFile);
-      out.close();
-
-      // the file exists, recreate it with the flag of CreateFlag.OVERWRITE
-      // together with CreateFlag.APPEND. It has the same effect as only specify
-      // CreateFlag.OVERWRITE.
-      EnumSet<CreateFlag> overwriteWithAppendFile = EnumSet.of(
-          CreateFlag.OVERWRITE, CreateFlag.APPEND);
-      out = createFileWithFlag(fs, path, 1, overwriteWithAppendFile);
-      out.close();
-
-      fs.delete(path, true);
-
-      EnumSet<CreateFlag> createNonExistsFile = EnumSet.of(CreateFlag.CREATE,
-          CreateFlag.OVERWRITE);
-      out = createFileWithFlag(fs, path, 1, createNonExistsFile);
-      out.close();
-      fs.delete(path, true);
-    } finally {
-      cluster.shutdown();
-    }
-  }
-  
   /**
    * Test file creation using createNonRecursive().
    */
@@ -789,14 +704,6 @@ public class TestFileCreation extends junit.framework.TestCase {
     return stm;
   }
   
-  // creates a file with the flag api
-  static FSDataOutputStream createFileWithFlag(FileSystem fileSys, Path name, int repl, EnumSet<CreateFlag> flag)
-    throws IOException {
-    System.out.println("createFile: Created " + name + " with " + repl + " replica.");
-    FSDataOutputStream stm = fileSys.create(name, FsPermission.getDefault(), flag, 
-                                            fileSys.getConf().getInt("io.file.buffer.size", 4096),(short)repl, (long)blockSize, null);
-    return stm;
-  }
 
 /**
  * Test that file data becomes available before file is closed.

+ 1 - 1
src/test/hdfs/org/apache/hadoop/security/TestPermission.java

@@ -117,7 +117,7 @@ public class TestPermission extends TestCase {
 
       FsPermission filePerm = new FsPermission((short)0444);
       FSDataOutputStream out = fs.create(new Path("/b1/b2/b3.txt"), filePerm,
-          EnumSet.of(CreateFlag.OVERWRITE), conf.getInt("io.file.buffer.size", 4096),
+          true, conf.getInt("io.file.buffer.size", 4096),
           fs.getDefaultReplication(), fs.getDefaultBlockSize(), null);
       out.write(123);
       out.close();