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 su...@apache.org on 2011/04/13 22:55:03 UTC

svn commit: r1091916 - in /hadoop/common/trunk: ./ src/java/org/apache/hadoop/fs/ src/test/core/org/apache/hadoop/fs/

Author: suresh
Date: Wed Apr 13 20:55:02 2011
New Revision: 1091916

URL: http://svn.apache.org/viewvc?rev=1091916&view=rev
Log:
Undoing commit r1091613 for HADOOP-7223. FileContext createFlag combinations are not clearly defined.


Modified:
    hadoop/common/trunk/CHANGES.txt
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/ChecksumFs.java
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/CreateFlag.java
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/FileContext.java
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/FileSystem.java
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java
    hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java

Modified: hadoop/common/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/CHANGES.txt?rev=1091916&r1=1091915&r2=1091916&view=diff
==============================================================================
--- hadoop/common/trunk/CHANGES.txt (original)
+++ hadoop/common/trunk/CHANGES.txt Wed Apr 13 20:55:02 2011
@@ -86,9 +86,6 @@ Trunk (unreleased changes)
 
     HADOOP-7202. Improve shell Command base class.  (Daryn Sharp via szetszwo)
 
-    HADOOP-7223. FileContext createFlag combinations are not clearly defined.
-    (suresh)
-
     HADOOP-7224. Add CommandFactory to shell.  (Daryn Sharp via szetszwo)
 
   OPTIMIZATIONS

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/fs/ChecksumFs.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/ChecksumFs.java?rev=1091916&r1=1091915&r2=1091916&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/ChecksumFs.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/ChecksumFs.java Wed Apr 13 20:55:02 2011
@@ -337,9 +337,8 @@ public abstract class ChecksumFs extends
       int bytesPerSum = fs.getBytesPerSum();
       int sumBufferSize = fs.getSumBufferSize(bytesPerSum, bufferSize);
       this.sums = fs.getRawFs().createInternal(fs.getChecksumFile(file),
-          EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE),
-          absolutePermission, sumBufferSize, replication, blockSize, progress,
-          bytesPerChecksum, createParent);
+          EnumSet.of(CreateFlag.OVERWRITE), absolutePermission, sumBufferSize,
+          replication,  blockSize,  progress, bytesPerChecksum,  createParent);
       sums.write(CHECKSUM_VERSION, 0, CHECKSUM_VERSION.length);
       sums.writeInt(bytesPerSum);
     }

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/fs/CreateFlag.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/CreateFlag.java?rev=1091916&r1=1091915&r2=1091916&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/CreateFlag.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/CreateFlag.java Wed Apr 13 20:55:02 2011
@@ -17,63 +17,49 @@
  */
 package org.apache.hadoop.fs;
 
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.EnumSet;
-
-import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
 /****************************************************************
- * CreateFlag specifies the file create semantic. Users can combine flags like: <br>
- * <code>
+ *CreateFlag specifies the file create semantic. Users can combine flags like:<br>
+ *<code>
  * EnumSet.of(CreateFlag.CREATE, CreateFlag.APPEND)
  * <code>
- * <p>
- * 
- * Use the CreateFlag as follows:
- * <ol>
- * <li> CREATE - to create a file if it does not exist, 
- * else throw FileAlreadyExists.</li>
- * <li> APPEND - to append to a file if it exists, 
- * else throw FileNotFoundException.</li>
- * <li> OVERWRITE - to truncate a file if it exists, 
- * else throw FileNotFoundException.</li>
- * <li> CREATE|APPEND - to create a file if it does not exist, 
- * else append to an existing file.</li>
- * <li> CREATE|OVERWRITE - to create a file if it does not exist, 
- * else overwrite an existing file.</li>
- * </ol>
+ * and pass it to {@link org.apache.hadoop.fs.FileSystem #create(Path f, FsPermission permission,
+ * EnumSet<CreateFlag> flag, int bufferSize, short replication, long blockSize,
+ * Progressable progress)}.
  * 
- * Following combination is not valid and will result in 
- * {@link HadoopIllegalArgumentException}:
+ * <p>
+ * Combine {@link #OVERWRITE} with either {@link #CREATE} 
+ * or {@link #APPEND} does the same as only use 
+ * {@link #OVERWRITE}. <br>
+ * Combine {@link #CREATE} with {@link #APPEND} has the semantic:
  * <ol>
- * <li> APPEND|OVERWRITE</li>
- * <li> CREATE|APPEND|OVERWRITE</li>
+ * <li> create the file if it does not exist;
+ * <li> append the file if it already exists.
  * </ol>
  *****************************************************************/
 @InterfaceAudience.Public
-@InterfaceStability.Evolving
+@InterfaceStability.Stable
 public enum CreateFlag {
 
   /**
-   * Create a file. See javadoc for more description
+   * create the file if it does not exist, and throw an IOException if it
    * already exists
    */
   CREATE((short) 0x01),
 
   /**
-   * Truncate/overwrite a file. Same as POSIX O_TRUNC. See javadoc for description.
+   * create the file if it does not exist, if it exists, overwrite it.
    */
   OVERWRITE((short) 0x02),
 
   /**
-   * Append to a file. See javadoc for more description.
+   * append to a file, and throw an IOException if it does not exist
    */
   APPEND((short) 0x04);
 
-  private final short mode;
+  private short mode;
 
   private CreateFlag(short mode) {
     this.mode = mode;
@@ -82,38 +68,4 @@ public enum CreateFlag {
   short getMode() {
     return mode;
   }
-  
-  /**
-   * Validate the CreateFlag for create operation
-   * @param path Object representing the path; usually String or {@link Path}
-   * @param pathExists pass true if the path exists in the file system
-   * @param flag set of CreateFlag
-   * @throws IOException on error
-   * @throws HadoopIllegalArgumentException if the CreateFlag is invalid
-   */
-  public static void validate(Object path, boolean pathExists,
-      EnumSet<CreateFlag> flag) throws IOException {
-    if (flag == null || flag.isEmpty()) {
-      throw new HadoopIllegalArgumentException(flag
-          + " does not specify any options");
-    }
-    final boolean append = flag.contains(APPEND);
-    final boolean overwrite = flag.contains(OVERWRITE);
-    
-    // Both append and overwrite is an error
-    if (append && overwrite) {
-      throw new HadoopIllegalArgumentException(
-          flag + "Both append and overwrite options cannot be enabled.");
-    }
-    if (pathExists) {
-      if (!(append || overwrite)) {
-        throw new FileAlreadyExistsException("File already exists: "
-            + path.toString()
-            + ". Append or overwrite option must be specified in " + flag);
-      }
-    } else if (!flag.contains(CREATE)) {
-      throw new FileNotFoundException("Non existing file: " + path.toString()
-          + ". Create option is not specified in " + flag);
-    }
-  }
 }
\ No newline at end of file

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/fs/FileContext.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/FileContext.java?rev=1091916&r1=1091915&r2=1091916&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/FileContext.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/FileContext.java Wed Apr 13 20:55:02 2011
@@ -511,7 +511,7 @@ public final class FileContext {
    * writing into the file.
    * 
    * @param f the file name to open
-   * @param createFlag gives the semantics of create; see {@link CreateFlag}
+   * @param createFlag gives the semantics of create: overwrite, append etc.
    * @param opts file creation options; see {@link Options.CreateOpts}.
    *          <ul>
    *          <li>Progress - to report progress on the operation - default null
@@ -2057,10 +2057,7 @@ public final class FileContext {
         OutputStream out = null;
         try {
           in = open(qSrc);
-          EnumSet<CreateFlag> createFlag = overwrite ? EnumSet.of(
-              CreateFlag.CREATE, CreateFlag.OVERWRITE) : 
-                EnumSet.of(CreateFlag.CREATE);
-          out = create(qDst, createFlag);
+          out = create(qDst, EnumSet.of(CreateFlag.OVERWRITE));
           IOUtils.copyBytes(in, out, conf, true);
         } catch (IOException e) {
           IOUtils.closeStream(out);

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/fs/FileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/FileSystem.java?rev=1091916&r1=1091915&r2=1091916&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/FileSystem.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/FileSystem.java Wed Apr 13 20:55:02 2011
@@ -717,21 +717,24 @@ public abstract class FileSystem extends
      FsPermission absolutePermission, EnumSet<CreateFlag> flag, int bufferSize,
      short replication, long blockSize, Progressable progress,
      int bytesPerChecksum) throws IOException {
-
-    boolean pathExists = exists(f);
-    CreateFlag.validate(f, pathExists, flag);
     
     // Default impl  assumes that permissions do not matter and 
     // nor does the bytesPerChecksum  hence
     // calling the regular create is good enough.
     // FSs that implement permissions should override this.
 
-    if (pathExists && flag.contains(CreateFlag.APPEND)) {
-      return append(f, bufferSize, progress);
+    if (exists(f)) {
+      if (flag.contains(CreateFlag.APPEND)) {
+        return append(f, bufferSize, progress);
+      } else if (!flag.contains(CreateFlag.OVERWRITE)) {
+        throw new IOException("File already exists: " + f);
+      }
+    } else {
+      if (flag.contains(CreateFlag.APPEND) && !flag.contains(CreateFlag.CREATE))
+        throw new IOException("File already exists: " + f.toString());
     }
     
-    return this.create(f, absolutePermission,
-        flag.contains(CreateFlag.OVERWRITE), bufferSize, replication,
+    return this.create(f, absolutePermission, flag.contains(CreateFlag.OVERWRITE), bufferSize, replication,
         blockSize, progress);
   }
   

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java?rev=1091916&r1=1091915&r2=1091916&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java Wed Apr 13 20:55:02 2011
@@ -264,6 +264,28 @@ public class RawLocalFileSystem extends 
     return out;
   }
   
+
+  @Override
+  protected FSDataOutputStream primitiveCreate(Path f,
+      FsPermission absolutePermission, EnumSet<CreateFlag> flag,
+      int bufferSize, short replication, long blockSize, Progressable progress,
+      int bytesPerChecksum) throws IOException {
+    
+    if(flag.contains(CreateFlag.APPEND)){
+      if (!exists(f)){
+        if(flag.contains(CreateFlag.CREATE)) {
+          return create(f, false, bufferSize, replication, blockSize, null);
+        }
+      }
+      return append(f, bufferSize, null);
+    }
+ 
+    FSDataOutputStream out = create(f, flag.contains(CreateFlag.OVERWRITE),
+                                 bufferSize, replication, blockSize, progress);
+    setPermission(f, absolutePermission);
+    return out;
+  }
+
   public boolean rename(Path src, Path dst) throws IOException {
     if (pathToFile(src).renameTo(pathToFile(dst))) {
       return true;

Modified: hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java?rev=1091916&r1=1091915&r2=1091916&view=diff
==============================================================================
--- hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java (original)
+++ hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java Wed Apr 13 20:55:02 2011
@@ -21,8 +21,8 @@ package org.apache.hadoop.fs;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.EnumSet;
+import java.util.Iterator;
 
-import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.fs.Options.CreateOpts;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -32,7 +32,6 @@ import org.junit.Before;
 import org.junit.Test;
 
 import static org.apache.hadoop.fs.FileContextTestHelper.*;
-import static org.apache.hadoop.fs.CreateFlag.*;
 
 /**
  * <p>
@@ -156,7 +155,7 @@ public abstract class FileContextMainOpe
     
     // Now open a file relative to the wd we just set above.
     Path absolutePath = new Path(absoluteDir, "foo");
-    fc.create(absolutePath, EnumSet.of(CREATE)).close();
+    fc.create(absolutePath, EnumSet.of(CreateFlag.CREATE)).close();
     fc.open(new Path("foo")).close();
     
     
@@ -646,7 +645,7 @@ public abstract class FileContextMainOpe
     
     fc.mkdir(path.getParent(), FsPermission.getDefault(), true);
 
-    FSDataOutputStream out = fc.create(path, EnumSet.of(CREATE),
+    FSDataOutputStream out = fc.create(path, EnumSet.of(CreateFlag.CREATE),
         CreateOpts.repFac((short) 1), CreateOpts
             .blockSize(getDefaultBlockSize()));
     out.write(data, 0, len);
@@ -671,93 +670,31 @@ public abstract class FileContextMainOpe
 
   }
   
-  @Test(expected=HadoopIllegalArgumentException.class)
-  public void testNullCreateFlag() throws IOException {
-    Path p = getTestRootPath(fc, "test/file");
-    fc.create(p, null);
-    Assert.fail("Excepted exception not thrown");
-  }
-  
-  @Test(expected=HadoopIllegalArgumentException.class)
-  public void testEmptyCreateFlag() throws IOException {
-    Path p = getTestRootPath(fc, "test/file");
-    fc.create(p, EnumSet.noneOf(CreateFlag.class));
-    Assert.fail("Excepted exception not thrown");
-  }
-  
-  @Test(expected=FileAlreadyExistsException.class)
-  public void testCreateFlagCreateExistingFile() throws IOException {
-    Path p = getTestRootPath(fc, "test/file");
-    createFile(p);
-    fc.create(p, EnumSet.of(CREATE));
-    Assert.fail("Excepted exception not thrown");
-  }
-  
-  @Test(expected=FileNotFoundException.class)
-  public void testCreateFlagOverwriteNonExistingFile() throws IOException {
-    Path p = getTestRootPath(fc, "test/nonExistent");
-    fc.create(p, EnumSet.of(OVERWRITE));
-    Assert.fail("Excepted exception not thrown");
-  }
-  
-  @Test
-  public void testCreateFlagOverwriteExistingFile() throws IOException {
-    Path p = getTestRootPath(fc, "test/file");
-    createFile(p);
-    FSDataOutputStream out = fc.create(p, EnumSet.of(OVERWRITE));
-    writeData(fc, p, out, data, data.length);
-  }
-  
-  @Test(expected=FileNotFoundException.class)
-  public void testCreateFlagAppendNonExistingFile() throws IOException {
-    Path p = getTestRootPath(fc, "test/nonExistent");
-    fc.create(p, EnumSet.of(APPEND));
-    Assert.fail("Excepted exception not thrown");
-  }
-  
-  @Test
-  public void testCreateFlagAppendExistingFile() throws IOException {
-    Path p = getTestRootPath(fc, "test/file");
-    createFile(p);
-    FSDataOutputStream out = fc.create(p, EnumSet.of(APPEND));
-    writeData(fc, p, out, data, 2 * data.length);
-  }
-  
   @Test
-  public void testCreateFlagCreateAppendNonExistingFile() throws IOException {
-    Path p = getTestRootPath(fc, "test/nonExistent");
-    FSDataOutputStream out = fc.create(p, EnumSet.of(CREATE, APPEND));
-    writeData(fc, p, out, data, data.length);
-  }
-  
-  @Test
-  public void testCreateFlagCreateAppendExistingFile() throws IOException {
-    Path p = getTestRootPath(fc, "test/nonExistent");
-    createFile(p);
-    FSDataOutputStream out = fc.create(p, EnumSet.of(CREATE, APPEND));
-    writeData(fc, p, out, data, 2*data.length);
-  }
-  
-  @Test(expected=HadoopIllegalArgumentException.class)
-  public void testCreateFlagAppendOverwrite() throws IOException {
-    Path p = getTestRootPath(fc, "test/nonExistent");
-    fc.create(p, EnumSet.of(APPEND, OVERWRITE));
-    Assert.fail("Excepted exception not thrown");
-  }
-  
-  @Test(expected=HadoopIllegalArgumentException.class)
-  public void testCreateFlagAppendCreateOverwrite() throws IOException {
-    Path p = getTestRootPath(fc, "test/nonExistent");
-    fc.create(p, EnumSet.of(CREATE, APPEND, OVERWRITE));
-    Assert.fail("Excepted exception not thrown");
-  }
-  
-  private static void writeData(FileContext fc, Path p, FSDataOutputStream out,
-      byte[] data, long expectedLen) throws IOException {
+  public void testOverwrite() throws IOException {
+    Path path = getTestRootPath(fc, "test/hadoop/file");
+    
+    fc.mkdir(path.getParent(), FsPermission.getDefault(), true);
+
+    createFile(path);
+    
+    Assert.assertTrue("Exists", exists(fc, path));
+    Assert.assertEquals("Length", data.length, fc.getFileStatus(path).getLen());
+    
+    try {
+      fc.create(path, EnumSet.of(CreateFlag.CREATE));
+      Assert.fail("Should throw IOException.");
+    } catch (IOException e) {
+      // Expected
+    }
+    
+    FSDataOutputStream out = fc.create(path,EnumSet.of(CreateFlag.OVERWRITE));
     out.write(data, 0, data.length);
     out.close();
-    Assert.assertTrue("Exists", exists(fc, p));
-    Assert.assertEquals("Length", expectedLen, fc.getFileStatus(p).getLen());
+    
+    Assert.assertTrue("Exists", exists(fc, path));
+    Assert.assertEquals("Length", data.length, fc.getFileStatus(path).getLen());
+    
   }
   
   @Test
@@ -1120,7 +1057,7 @@ public abstract class FileContextMainOpe
     //HADOOP-4760 according to Closeable#close() closing already-closed 
     //streams should have no effect. 
     Path src = getTestRootPath(fc, "test/hadoop/file");
-    FSDataOutputStream out = fc.create(src, EnumSet.of(CREATE),
+    FSDataOutputStream out = fc.create(src, EnumSet.of(CreateFlag.CREATE),
             Options.CreateOpts.createParent());
     
     out.writeChar('H'); //write some data
@@ -1154,7 +1091,7 @@ public abstract class FileContextMainOpe
   }
   
   protected void createFile(Path path) throws IOException {
-    FSDataOutputStream out = fc.create(path, EnumSet.of(CREATE),
+    FSDataOutputStream out = fc.create(path, EnumSet.of(CreateFlag.CREATE),
         Options.CreateOpts.createParent());
     out.write(data, 0, data.length);
     out.close();