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 to...@apache.org on 2010/06/21 23:48:39 UTC

svn commit: r956712 - in /hadoop/common/branches/branch-0.21: ./ src/java/org/apache/hadoop/fs/ src/java/org/apache/hadoop/fs/ftp/ src/java/org/apache/hadoop/fs/kfs/ src/java/org/apache/hadoop/fs/s3/ src/java/org/apache/hadoop/fs/s3native/

Author: tomwhite
Date: Mon Jun 21 21:48:38 2010
New Revision: 956712

URL: http://svn.apache.org/viewvc?rev=956712&view=rev
Log:
Merge -r 956709:956710 from trunk to branch-0.21. Fixes: HADOOP-6826.

Modified:
    hadoop/common/branches/branch-0.21/CHANGES.txt
    hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/ChecksumFileSystem.java
    hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/FileSystem.java
    hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/FilterFileSystem.java
    hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java
    hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/ftp/FTPFileSystem.java
    hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/kfs/KosmosFileSystem.java
    hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/s3/S3FileSystem.java
    hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java

Modified: hadoop/common/branches/branch-0.21/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.21/CHANGES.txt?rev=956712&r1=956711&r2=956712&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.21/CHANGES.txt (original)
+++ hadoop/common/branches/branch-0.21/CHANGES.txt Mon Jun 21 21:48:38 2010
@@ -1560,6 +1560,9 @@ Release 0.21.0 - Unreleased
     HADOOP-6461.  Webapps aren't located correctly post-split.
     (Todd Lipcon and Steve Loughran via tomwhite)
 
+    HADOOP-6826.  Revert FileSystem create method that takes CreateFlags.
+    (tomwhite)
+
 Release 0.20.3 - Unreleased
 
   NEW FEATURES

Modified: hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/ChecksumFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/ChecksumFileSystem.java?rev=956712&r1=956711&r2=956712&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/ChecksumFileSystem.java (original)
+++ hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/ChecksumFileSystem.java Mon Jun 21 21:48:38 2010
@@ -20,7 +20,6 @@ package org.apache.hadoop.fs;
 
 import java.io.*;
 import java.util.Arrays;
-import java.util.EnumSet;
 import java.util.zip.CRC32;
 
 import org.apache.commons.logging.Log;
@@ -388,14 +387,14 @@ public abstract class ChecksumFileSystem
   /** {@inheritDoc} */
   @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 {
     Path parent = f.getParent();
     if (parent != null && !mkdirs(parent)) {
       throw new IOException("Mkdirs failed to create " + parent);
     }
     final FSDataOutputStream out = new FSDataOutputStream(
-        new ChecksumFSOutputSummer(this, f, flag.contains(CreateFlag.OVERWRITE), bufferSize, replication,
+        new ChecksumFSOutputSummer(this, f, overwrite, bufferSize, replication,
             blockSize, progress), null);
     if (permission != null) {
       setPermission(f, permission);

Modified: hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/FileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/FileSystem.java?rev=956712&r1=956711&r2=956712&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/FileSystem.java (original)
+++ hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/FileSystem.java Mon Jun 21 21:48:38 2010
@@ -607,15 +607,13 @@ public abstract class FileSystem extends
                                             long blockSize,
                                             Progressable progress
                                             ) throws IOException {
-    return this.create(f, FsPermission.getDefault(), overwrite ? EnumSet
-        .of(CreateFlag.OVERWRITE) : EnumSet.of(CreateFlag.CREATE), bufferSize,
+    return this.create(f, FsPermission.getDefault(), overwrite, bufferSize,
         replication, blockSize, progress);
   }
 
   /**
    * Opens an FSDataOutputStream at the indicated Path with write-progress
    * reporting.
-   * @deprecated Consider using {@link #create(Path, FsPermission, EnumSet, int, short, long, Progressable)} instead.
    * @param f the file name to open
    * @param permission
    * @param overwrite if a file with this name already exists, then if true,
@@ -627,35 +625,14 @@ public abstract class FileSystem extends
    * @throws IOException
    * @see #setPermission(Path, FsPermission)
    */
-  public FSDataOutputStream create(Path f,
+  public abstract FSDataOutputStream create(Path f,
       FsPermission permission,
       boolean overwrite,
       int bufferSize,
       short replication,
       long blockSize,
-      Progressable progress) throws IOException{
-    return create(f, permission, overwrite ? EnumSet.of(CreateFlag.OVERWRITE)
-        : EnumSet.of(CreateFlag.CREATE), bufferSize, replication, blockSize,
-        progress);
-  }
+      Progressable progress) throws IOException;
   
-  /**
-   * Opens an FSDataOutputStream at the indicated Path with write-progress
-   * reporting.
-   * @param f the file name to open.
-   * @param permission - applied against umask
-   * @param flag determines the semantic of this create.
-   * @param bufferSize the size of the buffer to be used.
-   * @param replication required block replication for the file.
-   * @param blockSize
-   * @param progress
-   * @throws IOException
-   * @see #setPermission(Path, FsPermission)
-   * @see CreateFlag
-   */
-  public abstract FSDataOutputStream create(Path f, FsPermission permission,
-      EnumSet<CreateFlag> flag, int bufferSize, short replication, long blockSize,
-      Progressable progress) throws IOException ;
   
   /*.
    * This create has been added to support the FileContext that processes
@@ -674,117 +651,22 @@ public abstract class FileSystem extends
     // nor does the bytesPerChecksum  hence
     // calling the regular create is good enough.
     // FSs that implement permissions should override this.
-    
-    return this.create(f, absolutePermission, flag, bufferSize, replication,
-        blockSize, progress);
-  }
-  
-  
-  /*.
-   * This create has been added to support the FileContext that passes
-   * an absolute permission with (ie umask was already applied) 
-   * This a temporary method added to support the transition from FileSystem
-   * to FileContext for user applications.
-   */
-  @Deprecated
-  protected FSDataOutputStream primitiveCreate(final Path f,
-      final EnumSet<CreateFlag> createFlag,
-      CreateOpts... opts) throws IOException {
-    checkPath(f);
-    int bufferSize = -1;
-    short replication = -1;
-    long blockSize = -1;
-    int bytesPerChecksum = -1;
-    FsPermission permission = null;
-    Progressable progress = null;
-    Boolean createParent = null;
- 
-    for (CreateOpts iOpt : opts) {
-      if (CreateOpts.BlockSize.class.isInstance(iOpt)) {
-        if (blockSize != -1) {
-          throw new IllegalArgumentException("multiple varargs of same kind");
-        }
-        blockSize = ((CreateOpts.BlockSize) iOpt).getValue();
-      } else if (CreateOpts.BufferSize.class.isInstance(iOpt)) {
-        if (bufferSize != -1) {
-          throw new IllegalArgumentException("multiple varargs of same kind");
-        }
-        bufferSize = ((CreateOpts.BufferSize) iOpt).getValue();
-      } else if (CreateOpts.ReplicationFactor.class.isInstance(iOpt)) {
-        if (replication != -1) {
-          throw new IllegalArgumentException("multiple varargs of same kind");
-        }
-        replication = ((CreateOpts.ReplicationFactor) iOpt).getValue();
-      } else if (CreateOpts.BytesPerChecksum.class.isInstance(iOpt)) {
-        if (bytesPerChecksum != -1) {
-          throw new IllegalArgumentException("multiple varargs of same kind");
-        }
-        bytesPerChecksum = ((CreateOpts.BytesPerChecksum) iOpt).getValue();
-      } else if (CreateOpts.Perms.class.isInstance(iOpt)) {
-        if (permission != null) {
-          throw new IllegalArgumentException("multiple varargs of same kind");
-        }
-        permission = ((CreateOpts.Perms) iOpt).getValue();
-      } else if (CreateOpts.Progress.class.isInstance(iOpt)) {
-        if (progress != null) {
-          throw new IllegalArgumentException("multiple varargs of same kind");
-        }
-        progress = ((CreateOpts.Progress) iOpt).getValue();
-      } else if (CreateOpts.CreateParent.class.isInstance(iOpt)) {
-        if (createParent != null) {
-          throw new IllegalArgumentException("multiple varargs of same kind");
-        }
-        createParent = ((CreateOpts.CreateParent) iOpt).getValue();
-      } else {
-        throw new IllegalArgumentException("Unkown CreateOpts of type " +
-            iOpt.getClass().getName());
-      }
-    }
-    if (blockSize % bytesPerChecksum != 0) {
-      throw new IllegalArgumentException(
-          "blockSize should be a multiple of checksumsize");
-    }
-    
-    FsServerDefaults ssDef = getServerDefaults();
-    
-    if (blockSize == -1) {
-      blockSize = ssDef.getBlockSize();
-    }
-    if (bufferSize == -1) {
-      bufferSize = ssDef.getFileBufferSize();
-    }
-    if (replication == -1) {
-      replication = ssDef.getReplication();
-    }
-    if (permission == null) {
-      permission = FsPermission.getDefault();
-    }
-    if (createParent == null) {
-      createParent = false;
-    }
-    
-    // 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 (!createParent) { // parent must exist.
-      // since this.create makes parent dirs automatically
-      // we must throw exception if parent does not exist.
-      final FileStatus stat = getFileStatus(f.getParent());
-      if (stat == null) {
-        throw new FileNotFoundException("Missing parent:" + f);
-      }
-      if (!stat.isDirectory()) {
-        throw new ParentNotDirectoryException("parent is not a dir:" + f);
+    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);
       }
-      // parent does exist - go ahead with create of file.
+    } else {
+      if (flag.contains(CreateFlag.APPEND) && !flag.contains(CreateFlag.CREATE))
+        throw new IOException("File already exists: " + f.toString());
     }
-    return this.create(f, permission, createFlag, bufferSize, replication,
+    
+    return this.create(f, absolutePermission, flag.contains(CreateFlag.OVERWRITE), bufferSize, replication,
         blockSize, progress);
   }
   
-
   /**
    * This version of the mkdirs method assumes that the permission is absolute.
    * It has been added to support the FileContext that processes the permission

Modified: hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/FilterFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/FilterFileSystem.java?rev=956712&r1=956711&r2=956712&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/FilterFileSystem.java (original)
+++ hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/FilterFileSystem.java Mon Jun 21 21:48:38 2010
@@ -111,10 +111,10 @@ public class FilterFileSystem extends Fi
   /** {@inheritDoc} */
   @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 fs.create(f, permission,
-        flag, bufferSize, replication, blockSize, progress);
+        overwrite, bufferSize, replication, blockSize, progress);
   }
 
   /**

Modified: hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java?rev=956712&r1=956711&r2=956712&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java (original)
+++ hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java Mon Jun 21 21:48:38 2010
@@ -244,14 +244,13 @@ public class RawLocalFileSystem extends 
   /** {@inheritDoc} */
   @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 primitiveCreate(f,
-        permission.applyUMask(FsPermission.getUMask(getConf())), flag,
-        bufferSize,  replication,  blockSize,  progress,  -1);
-    
-    
-     
+
+    FSDataOutputStream out = create(f,
+        overwrite, bufferSize, replication, blockSize, progress);
+    setPermission(f, permission);
+    return out;
   }
   
 

Modified: hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/ftp/FTPFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/ftp/FTPFileSystem.java?rev=956712&r1=956711&r2=956712&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/ftp/FTPFileSystem.java (original)
+++ hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/ftp/FTPFileSystem.java Mon Jun 21 21:48:38 2010
@@ -21,7 +21,6 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.net.URI;
-import java.util.EnumSet;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -32,7 +31,6 @@ import org.apache.commons.net.ftp.FTPRep
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -200,28 +198,18 @@ public class FTPFileSystem extends FileS
    */
   @Override
   public FSDataOutputStream create(Path file, FsPermission permission,
-      EnumSet<CreateFlag> flag, int bufferSize, short replication, long blockSize,
+      boolean overwrite, int bufferSize, short replication, long blockSize,
       Progressable progress) throws IOException {
     final FTPClient client = connect();
     Path workDir = new Path(client.printWorkingDirectory());
     Path absolute = makeAbsolute(workDir, file);
-    
-    boolean overwrite = flag.contains(CreateFlag.OVERWRITE);
-    boolean create = flag.contains(CreateFlag.CREATE);
-    boolean append= flag.contains(CreateFlag.APPEND);
-    
     if (exists(client, file)) {
       if (overwrite) {
         delete(client, file);
-      } else if(append){
-        return append(file, bufferSize, progress);
       } else {
         disconnect(client);
         throw new IOException("File already exists: " + file);
       }
-    } else {
-      if(append && !create)
-        throw new FileNotFoundException("File does not exist: "+ file);
     }
     
     Path parent = absolute.getParent();

Modified: hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/kfs/KosmosFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/kfs/KosmosFileSystem.java?rev=956712&r1=956711&r2=956712&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/kfs/KosmosFileSystem.java (original)
+++ hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/kfs/KosmosFileSystem.java Mon Jun 21 21:48:38 2010
@@ -22,13 +22,11 @@ package org.apache.hadoop.fs.kfs;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URI;
-import java.util.EnumSet;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
-import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -191,25 +189,16 @@ public class KosmosFileSystem extends Fi
 
     @Override
     public FSDataOutputStream create(Path file, FsPermission permission,
-                                     EnumSet<CreateFlag> flag, int bufferSize,
+                                     boolean overwrite, int bufferSize,
 				     short replication, long blockSize, Progressable progress)
 	throws IOException {
 
-      boolean overwrite = flag.contains(CreateFlag.OVERWRITE);
-      boolean create = flag.contains(CreateFlag.CREATE);
-      boolean append= flag.contains(CreateFlag.APPEND);
-      
         if (exists(file)) {
             if (overwrite) {
                 delete(file, true);
-            } else if (append){
-             return append(file, bufferSize, progress);
             } else {
                 throw new IOException("File already exists: " + file);
             }
-        } else {
-          if(append && !create)
-            throw new FileNotFoundException("File does not exist: "+ file);
         }
 
 	Path parent = file.getParent();

Modified: hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/s3/S3FileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/s3/S3FileSystem.java?rev=956712&r1=956711&r2=956712&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/s3/S3FileSystem.java (original)
+++ hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/s3/S3FileSystem.java Mon Jun 21 21:48:38 2010
@@ -22,7 +22,6 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URI;
 import java.util.ArrayList;
-import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -31,7 +30,6 @@ import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -207,24 +205,18 @@ public class S3FileSystem extends FileSy
    */
   @Override
   public FSDataOutputStream create(Path file, FsPermission permission,
-      EnumSet<CreateFlag> flag, int bufferSize,
+      boolean overwrite, int bufferSize,
       short replication, long blockSize, Progressable progress)
     throws IOException {
 
     INode inode = store.retrieveINode(makeAbsolute(file));
     if (inode != null) {
-      if (flag.contains(CreateFlag.OVERWRITE)) {
+      if (overwrite) {
         delete(file, true);
-      } else if (flag.contains(CreateFlag.APPEND)){
-        return append(file, bufferSize, progress);
       } else {
         throw new IOException("File already exists: " + file);
       }
     } else {
-      
-      if(flag.contains(CreateFlag.APPEND) && !flag.contains(CreateFlag.CREATE))
-        throw new FileNotFoundException("File does not exist: "+ file);
-      
       Path parent = file.getParent();
       if (parent != null) {
         if (!mkdirs(parent)) {

Modified: hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java?rev=956712&r1=956711&r2=956712&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java (original)
+++ hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java Mon Jun 21 21:48:38 2010
@@ -30,7 +30,6 @@ import java.security.DigestOutputStream;
 import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
 import java.util.ArrayList;
-import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -44,7 +43,6 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BufferedFSInputStream;
-import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSInputStream;
@@ -326,18 +324,11 @@ public class NativeS3FileSystem extends 
   
   @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 {
 
-    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());
+    if (exists(f) && !overwrite) {
+      throw new IOException("File already exists:"+f);
     }
     
     LOG.debug("Creating new file '" + f + "' in S3");