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 dh...@apache.org on 2007/12/04 15:25:07 UTC

svn commit: r600964 - in /lucene/hadoop/trunk: ./ src/java/org/apache/hadoop/dfs/ src/java/org/apache/hadoop/fs/ src/java/org/apache/hadoop/fs/kfs/ src/java/org/apache/hadoop/fs/permission/ src/java/org/apache/hadoop/fs/s3/ src/test/org/apache/hadoop/fs/

Author: dhruba
Date: Tue Dec  4 06:25:05 2007
New Revision: 600964

URL: http://svn.apache.org/viewvc?rev=600964&view=rev
Log:
HADOOP-2288.  Enhance FileSystem API to support access control.
(Tsz Wo (Nicholas), SZE via dhruba)


Added:
    lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/
    lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/AccessControlException.java   (with props)
    lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/FsAction.java   (with props)
    lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/FsPermission.java   (with props)
    lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/PermissionStatus.java   (with props)
    lucene/hadoop/trunk/src/test/org/apache/hadoop/fs/TestLocalFileSystemPermission.java   (with props)
Modified:
    lucene/hadoop/trunk/CHANGES.txt
    lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DistributedFileSystem.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/HftpFileSystem.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FilterFileSystem.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/InMemoryFileSystem.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/ShellCommand.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/kfs/KosmosFileSystem.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/s3/S3FileSystem.java

Modified: lucene/hadoop/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?rev=600964&r1=600963&r2=600964&view=diff
==============================================================================
--- lucene/hadoop/trunk/CHANGES.txt (original)
+++ lucene/hadoop/trunk/CHANGES.txt Tue Dec  4 06:25:05 2007
@@ -20,6 +20,9 @@
     This also fixes a configuration bug in AggregateWordCount, so that the
     job now works.  (enis)
 
+    HADOOP-2288.  Enhance FileSystem API to support access control.
+    (Tsz Wo (Nicholas), SZE via dhruba)
+    
   NEW FEATURES
 
     HADOOP-1857.  Ability to run a script when a task fails to capture stack

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DistributedFileSystem.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DistributedFileSystem.java?rev=600964&r1=600963&r2=600964&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DistributedFileSystem.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DistributedFileSystem.java Tue Dec  4 06:25:05 2007
@@ -21,7 +21,7 @@
 import java.io.*;
 import java.net.*;
 
-import org.apache.hadoop.io.*;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.dfs.FSConstants.DatanodeReportType;
@@ -114,7 +114,8 @@
     return new DFSClient.DFSDataInputStream(dfs.open(getPathName(f),bufferSize));
   }
 
-  public FSDataOutputStream create(Path f, boolean overwrite,
+  public FSDataOutputStream create(Path f, FsPermission permission,
+    boolean overwrite,
     int bufferSize, short replication, long blockSize,
     Progressable progress) throws IOException {
 
@@ -173,7 +174,7 @@
     return stats;
   }
 
-  public boolean mkdirs(Path f) throws IOException {
+  public boolean mkdirs(Path f, FsPermission permission) throws IOException {
     return dfs.mkdirs(getPathName(f));
   }
 

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/HftpFileSystem.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/HftpFileSystem.java?rev=600964&r1=600963&r2=600964&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/HftpFileSystem.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/HftpFileSystem.java Tue Dec  4 06:25:05 2007
@@ -45,6 +45,7 @@
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.util.Progressable;
 
 /** An implementation of a protocol for accessing filesystems over HTTP.
@@ -218,7 +219,8 @@
   public void setWorkingDirectory(Path f) { }
 
   @Override
-  public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize,
+  public FSDataOutputStream create(Path f, FsPermission permission,
+                                   boolean overwrite, int bufferSize,
                                    short replication, long blockSize,
                                    Progressable progress) throws IOException {
     throw new IOException("Not supported");
@@ -235,7 +237,7 @@
   }
 
   @Override
-  public boolean mkdirs(Path f) throws IOException {
+  public boolean mkdirs(Path f, FsPermission permission) throws IOException {
     throw new IOException("Not supported");
   }
 

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java?rev=600964&r1=600963&r2=600964&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java Tue Dec  4 06:25:05 2007
@@ -27,6 +27,7 @@
 import org.apache.hadoop.dfs.*;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.util.*;
+import org.apache.hadoop.fs.permission.FsPermission;
 
 /****************************************************************
  * An abstract base class for a fairly generic filesystem.  It
@@ -378,13 +379,38 @@
    * @param bufferSize the size of the buffer to be used.
    * @param replication required block replication for the file. 
    */
-  public abstract FSDataOutputStream create(Path f, 
+  public FSDataOutputStream create(Path f,
                                             boolean overwrite,
                                             int bufferSize,
                                             short replication,
                                             long blockSize,
                                             Progressable progress
-                                            ) throws IOException;
+                                            ) throws IOException {
+    return this.create(f, FsPermission.getDefault(getConf()),
+        overwrite, bufferSize, replication, blockSize, progress);
+  }
+
+  /**
+   * Opens an FSDataOutputStream at the indicated Path with write-progress
+   * reporting.
+   * @param f the file name to open
+   * @param permission
+   * @param overwrite if a file with this name already exists, then if true,
+   *   the file will be overwritten, and if false an error will be thrown.
+   * @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)
+   */
+  public abstract FSDataOutputStream create(Path f,
+      FsPermission permission,
+      boolean overwrite,
+      int bufferSize,
+      short replication,
+      long blockSize,
+      Progressable progress) throws IOException;
 
   /**
    * Creates the given Path as a brand-new zero-length file.  If
@@ -804,13 +830,21 @@
    * @return the directory pathname
    */
   public abstract Path getWorkingDirectory();
-    
+
+  /**
+   * Call {@link #mkdirs(Path, FsPermission)} with default permission.
+   */
+  public boolean mkdirs(Path f) throws IOException {
+    return mkdirs(f, FsPermission.getDefault(getConf()));
+  }
+
   /**
    * Make the given file and all non-existent parents into
    * directories. Has the semantics of Unix 'mkdir -p'.
    * Existence of the directory hierarchy is not an error.
    */
-  public abstract boolean mkdirs(Path f) throws IOException;
+  public abstract boolean mkdirs(Path f, FsPermission permission
+      ) throws IOException;
 
   /**
    * The src file is on the local disk.  Add it to FS at
@@ -956,7 +990,7 @@
    */
   public short getDefaultReplication() { return 1; }
 
-  /* 
+  /**
    * Return a file status object that represents the
    * file.
    * @param f The path to the file we want information from
@@ -964,4 +998,24 @@
    * @throws IOException see specific implementation
    */
   public abstract FileStatus getFileStatus(Path f) throws IOException;
+
+  /**
+   * Set permission of a path.
+   * @param p
+   * @param permission
+   */
+  public void setPermission(Path p, FsPermission permission
+      ) throws IOException {
+  }
+
+  /**
+   * Set owner of a path (i.e. a file or a directory).
+   * The parameters username and groupname cannot both be null.
+   * @param p The path
+   * @param username If it is null, the original username remains unchanged.
+   * @param groupname If it is null, the original groupname remains unchanged.
+   */
+  public void setOwner(Path p, String username, String groupname
+      ) throws IOException {
+  }
 }

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FilterFileSystem.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FilterFileSystem.java?rev=600964&r1=600963&r2=600964&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FilterFileSystem.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FilterFileSystem.java Tue Dec  4 06:25:05 2007
@@ -22,6 +22,7 @@
 import java.net.URI;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.util.Progressable;
 
 /****************************************************************
@@ -104,23 +105,13 @@
     return fs.open(f, bufferSize);
   }
   
-  /**
-   * Opens an FSDataOutputStream at the indicated Path with write-progress
-   * reporting.
-   * @param f the file name to open
-   * @param overwrite if a file with this name already exists, then if true,
-   *   the file will be overwritten, and if false an error will be thrown.
-   * @param bufferSize the size of the buffer to be used.
-   * @param replication required block replication for the file. 
-   */
-  public FSDataOutputStream create(Path f, 
-                                   boolean overwrite,
-                                   int bufferSize,
-                                   short replication,
-                                   long blockSize,
-                                   Progressable progress
-                                   ) throws IOException {
-    return fs.create(f, overwrite, bufferSize, replication, blockSize, progress);
+  /** {@inheritDoc} */
+  @Override
+  public FSDataOutputStream create(Path f, FsPermission permission,
+      boolean overwrite, int bufferSize, short replication, long blockSize,
+      Progressable progress) throws IOException {
+    return fs.create(f, permission,
+        overwrite, bufferSize, replication, blockSize, progress);
   }
 
   /**
@@ -180,13 +171,10 @@
     return fs.getWorkingDirectory();
   }
   
-  /**
-   * Make the given file and all non-existent parents into directories. Has
-   * the semantics of Unix 'mkdir -p'. Existence of the directory hierarchy is
-   * not an error.
-   */
-  public boolean mkdirs(Path f) throws IOException {
-    return fs.mkdirs(f);
+  /** {@inheritDoc} */
+  @Override
+  public boolean mkdirs(Path f, FsPermission permission) throws IOException {
+    return fs.mkdirs(f, permission);
   }
 
   /**
@@ -260,5 +248,19 @@
   public void close() throws IOException {
     super.close();
     fs.close();
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public void setOwner(Path p, String username, String groupname
+      ) throws IOException {
+    fs.setOwner(p, username, groupname);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public void setPermission(Path p, FsPermission permission
+      ) throws IOException {
+    fs.setPermission(p, permission);
   }
 }

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/InMemoryFileSystem.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/InMemoryFileSystem.java?rev=600964&r1=600963&r2=600964&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/InMemoryFileSystem.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/InMemoryFileSystem.java Tue Dec  4 06:25:05 2007
@@ -23,6 +23,7 @@
 import java.net.URI;
 import java.util.*;
 
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.Progressable;
@@ -187,7 +188,11 @@
       }
     }
   
-    public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize,
+    /**
+     * @param permission Currently ignored.
+     */
+    public FSDataOutputStream create(Path f, FsPermission permission,
+                                     boolean overwrite, int bufferSize,
                                      short replication, long blockSize, Progressable progress)
       throws IOException {
       synchronized (this) {
@@ -274,7 +279,10 @@
       return staticWorkingDir;
     }
 
-    public boolean mkdirs(Path f) throws IOException {
+    /**
+     * @param permission Currently ignored.
+     */
+    public boolean mkdirs(Path f, FsPermission permission) throws IOException {
       return true;
     }
   

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java?rev=600964&r1=600963&r2=600964&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/RawLocalFileSystem.java Tue Dec  4 06:25:05 2007
@@ -25,7 +25,9 @@
 import java.util.*;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.*;
 import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.StringUtils;
 
 /****************************************************************
  * Implement the FileSystem API for the raw local filesystem.
@@ -190,6 +192,17 @@
     return new FSDataOutputStream(
         new BufferedOutputStream(new LocalFSFileOutputStream(f), bufferSize));
   }
+
+  /** {@inheritDoc} */
+  @Override
+  public FSDataOutputStream create(Path f, FsPermission permission,
+      boolean overwrite, int bufferSize, short replication, long blockSize,
+      Progressable progress) throws IOException {
+    FSDataOutputStream out = create(f,
+        overwrite, bufferSize, replication, blockSize, progress);
+    setPermission(f, permission);
+    return out;
+  }
   
   public boolean rename(Path src, Path dst) throws IOException {
     if (useCopyForRename) {
@@ -216,7 +229,8 @@
       return null;
     }
     if (localf.isFile()) {
-      return new FileStatus[] { new RawLocalFileStatus(localf) };
+      return new FileStatus[] {
+          new RawLocalFileStatus(localf, getDefaultBlockSize()) };
     }
 
     String[] names = localf.list();
@@ -240,6 +254,14 @@
     return (parent == null || mkdirs(parent)) &&
       (p2f.mkdir() || p2f.isDirectory());
   }
+
+  /** {@inheritDoc} */
+  @Override
+  public boolean mkdirs(Path f, FsPermission permission) throws IOException {
+    boolean b = mkdirs(f);
+    setPermission(f, permission);
+    return b;
+  }
   
   /**
    * Set the working directory to the given directory.
@@ -330,14 +352,77 @@
     return "LocalFS";
   }
   
-  public FileStatus getFileStatus(Path f) throws IOException {
-    return new RawLocalFileStatus(pathToFile(f));
+  public FileStatus getFileStatus(Path f) {
+    return new RawLocalFileStatus(pathToFile(f), getDefaultBlockSize());
   }
 
-  private class RawLocalFileStatus extends FileStatus {
-    RawLocalFileStatus(File f) throws IOException {
-      super(f.length(), f.isDirectory(), 1, getDefaultBlockSize(),
+  static class RawLocalFileStatus extends FileStatus {
+    private File file;
+    private PermissionStatus permissions;
+
+    RawLocalFileStatus(File f, long defaultBlockSize) {
+      super(f.length(), f.isDirectory(), 1, defaultBlockSize,
             f.lastModified(), new Path(f.toURI().toString()));
+      file = f;
+    }
+
+    PermissionStatus getPermissionStatus() {
+      if (permissions == null) {
+        try {
+          permissions = getPermissionStatus(file);
+        }
+        catch(IOException e) {
+          LOG.debug(StringUtils.stringifyException(e));
+        }
+      }
+      return permissions;
+    }
+
+    private static PermissionStatus getPermissionStatus(File f
+        ) throws IOException {
+      StringTokenizer t = new StringTokenizer(
+          execCommand(f, ShellCommand.getGET_PERMISSION_COMMAND()));
+      //expected format
+      //-rw-------    1 username groupname ...
+      FsPermission p = FsPermission.valueOf(t.nextToken());
+      t.nextToken();
+      return new PermissionStatus(t.nextToken(), t.nextToken(), p);
     }
+  }
+
+  /**
+   * Use the command chown to set owner.
+   */
+  @Override
+  public void setOwner(Path p, String username, String groupname
+      ) throws IOException {
+    if (username == null && groupname == null) {
+      throw new IOException("username == null && groupname == null");
+    }
+
+    //[OWNER][:[GROUP]]
+    String s = (username == null? "": username)
+             + (groupname == null? "": ":" + groupname);
+    execCommand(pathToFile(p), ShellCommand.SET_OWNER_COMMAND, s);
+  }
+
+  /**
+   * Use the command chmod to set permission.
+   */
+  @Override
+  public void setPermission(Path p, FsPermission permission
+      ) throws IOException {
+    execCommand(pathToFile(p), ShellCommand.SET_PERMISSION_COMMAND,
+        String.format("%04o", permission.toShort()));
+  }
+
+  private static String execCommand(File f, String... cmd) throws IOException {
+    String[] args = new String[cmd.length + 1];
+    System.arraycopy(cmd, 0, args, 0, cmd.length);
+    args[cmd.length] = f.getCanonicalPath();
+    LOG.debug("args=" + Arrays.asList(args));
+    String output = ShellCommand.execCommand(args);
+    LOG.debug("output=" + output);
+    return output;
   }
 }

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/ShellCommand.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/ShellCommand.java?rev=600964&r1=600963&r2=600964&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/ShellCommand.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/ShellCommand.java Tue Dec  4 06:25:05 2007
@@ -23,6 +23,17 @@
 
 /** A base class for running a unix command like du or df*/
 abstract public class ShellCommand {
+  /** a Unix command to get a list of groups */
+  public static final String GROUPS_COMMAND = "groups";
+  /** a Unix command to set permission */
+  public static final String SET_PERMISSION_COMMAND = "chmod";
+  /** a Unix command to set owner */
+  public static final String SET_OWNER_COMMAND = "chown";
+  /** Return a Unix command to get permission information. */
+  public static String[] getGET_PERMISSION_COMMAND() {
+    return new String[]{"ls", "-ld"};
+  }
+
   private long    interval;   // refresh interval in msec
   private long    lastTime;   // last time the command was performed
   
@@ -104,7 +115,7 @@
    * Static method to execute a command. Covers most of the simple cases 
    * without requiring the user to implement Command interface.
    */
-  public static String execCommand(String[] cmd) throws IOException {
+  public static String execCommand(String ... cmd) throws IOException {
     SimpleCommandExecutor exec = new SimpleCommandExecutor(cmd);
     exec.run();
     return exec.getReply();

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/kfs/KosmosFileSystem.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/kfs/KosmosFileSystem.java?rev=600964&r1=600963&r2=600964&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/kfs/KosmosFileSystem.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/kfs/KosmosFileSystem.java Tue Dec  4 06:25:05 2007
@@ -30,6 +30,7 @@
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.util.Progressable;
 
 /**
@@ -100,7 +101,8 @@
         return kfsImpl.exists(srep);
     }
 
-    public boolean mkdirs(Path path) throws IOException {
+    public boolean mkdirs(Path path, FsPermission permission
+        ) throws IOException {
 	Path absolute = makeAbsolute(path);
         String srep = absolute.toUri().getPath();
 
@@ -244,7 +246,8 @@
 
     }
 
-    public FSDataOutputStream create(Path file, boolean overwrite, int bufferSize,
+    public FSDataOutputStream create(Path file, FsPermission permission,
+                                     boolean overwrite, int bufferSize,
 				     short replication, long blockSize, Progressable progress)
 	throws IOException {
 

Added: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/AccessControlException.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/AccessControlException.java?rev=600964&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/AccessControlException.java (added)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/AccessControlException.java Tue Dec  4 06:25:05 2007
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.permission;
+
+/**
+ * An exception class for access control related issues.
+ */
+public class AccessControlException extends java.io.IOException {
+  //Required by {@link java.io.Serializable}.
+  private static final long serialVersionUID = 1L;
+
+  /**
+   * Constructs an {@link AccessControlException}
+   * with the specified detail message.
+   * @param s the detail message.
+   */
+  public AccessControlException(String s) {super(s);}
+}
\ No newline at end of file

Propchange: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/AccessControlException.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/AccessControlException.java
------------------------------------------------------------------------------
    svn:keywords = Id Revision HeadURL

Added: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/FsAction.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/FsAction.java?rev=600964&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/FsAction.java (added)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/FsAction.java Tue Dec  4 06:25:05 2007
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.permission;
+
+/**
+ * File system actions, e.g. read, write, etc.
+ */
+public enum FsAction {
+  //POSIX style
+  NONE(0, "---"),
+  EXECUTE(1, "--x"),
+  WRITE(2, "-w-"),
+  WRITE_EXECUTE(3, "-wx"),
+  READ(4, "r--"),
+  READ_EXECUTE(5, "r-x"),
+  READ_WRITE(6, "rw-"),
+  ALL(7, "rwx");
+
+  //constants
+  /** Octal representation */
+  public final int INDEX;
+  /** Symbolic representation */
+  public final String SYMBOL;
+
+  private FsAction(int v, String s) {
+    INDEX = v;
+    SYMBOL = s;
+  }
+
+  /**
+   * Return true if this action implies that action.
+   * @param that
+   */
+  public boolean implies(FsAction that) {
+    if (that != null) {
+      return (this.INDEX & that.INDEX) == that.INDEX;
+    }
+    return false;
+  }
+
+  /** AND operation. */
+  public FsAction and(FsAction that) {
+    return values()[this.INDEX & that.INDEX];
+  }
+  /** OR operation. */
+  public FsAction or(FsAction that) {
+    return values()[this.INDEX | that.INDEX];
+  }
+  /** NOT operation. */
+  public FsAction not() {
+    return values()[7 - INDEX];
+  }
+}
\ No newline at end of file

Propchange: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/FsAction.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/FsAction.java
------------------------------------------------------------------------------
    svn:keywords = Id Revision HeadURL

Added: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/FsPermission.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/FsPermission.java?rev=600964&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/FsPermission.java (added)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/FsPermission.java Tue Dec  4 06:25:05 2007
@@ -0,0 +1,176 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.permission;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.*;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+/**
+ * A class for file/directory permissions.
+ */
+public class FsPermission implements Writable {
+  static final WritableFactory FACTORY = new WritableFactory() {
+    public Writable newInstance() { return new FsPermission(); }
+  };
+  static {                                      // register a ctor
+    WritableFactories.setFactory(FsPermission.class, FACTORY);
+  }
+
+  //POSIX permission style
+  private FsAction useraction = null;
+  private FsAction groupaction = null;
+  private FsAction otheraction = null;
+
+  private FsPermission() {}
+
+  /**
+   * Construct by the given {@link FsAction}.
+   * @param u user action
+   * @param g group action
+   * @param o other action
+   */
+  public FsPermission(FsAction u, FsAction g, FsAction o) {set(u, g, o);}
+
+  /**
+   * Construct by the given mode.
+   * @param mode
+   * @see #toShort()
+   */
+  public FsPermission(short mode) { fromShort(mode); }
+
+  /** Return user {@link FsAction}. */
+  public FsAction getUserAction() {return useraction;}
+
+  /** Return group {@link FsAction}. */
+  public FsAction getGroupAction() {return groupaction;}
+
+  /** Return other {@link FsAction}. */
+  public FsAction getOtherAction() {return otheraction;}
+
+  private void set(FsAction u, FsAction g, FsAction o) {
+    useraction = u;
+    groupaction = g;
+    otheraction = o;
+  }
+  private void fromShort(short n) {
+    FsAction[] v = FsAction.values();
+    set(v[(n >>> 6) & 7], v[(n >>> 3) & 7], v[n & 7]);
+  }
+
+  /** {@inheritDoc} */
+  public void write(DataOutput out) throws IOException {
+    out.writeShort(toShort());
+  }
+
+  /** {@inheritDoc} */
+  public void readFields(DataInput in) throws IOException {
+    fromShort(in.readShort());
+  }
+
+  /**
+   * Create and initialize a {@link FsPermission} from {@link DataInput}.
+   */
+  public static FsPermission read(DataInput in) throws IOException {
+    FsPermission p = new FsPermission();
+    p.readFields(in);
+    return p;
+  }
+
+  /**
+   * Encode the object to a short.
+   */
+  public short toShort() {
+    int s = (useraction.INDEX<<6) | (groupaction.INDEX<<3) | otheraction.INDEX;
+    return (short)s;
+  }
+
+  /** {@inheritDoc} */
+  public boolean equals(Object obj) {
+    if (obj instanceof FsPermission) {
+      FsPermission that = (FsPermission)obj;
+      return this.useraction == that.useraction
+          && this.groupaction == that.groupaction
+          && this.otheraction == that.otheraction;
+    }
+    return false;
+  }
+
+  /** {@inheritDoc} */
+  public int hashCode() {return toShort();}
+
+  /** {@inheritDoc} */
+  public String toString() {
+    return useraction.SYMBOL + groupaction.SYMBOL + otheraction.SYMBOL;
+  }
+
+  /** Apply a umask to this permission and return a new one */
+  public FsPermission applyUMask(FsPermission umask) {
+    return new FsPermission(useraction.and(umask.useraction.not()),
+        groupaction.and(umask.groupaction.not()),
+        otheraction.and(umask.otheraction.not()));
+  }
+
+  /** umask property label */
+  public static final String UMASK_LABEL = "hadoop.dfs.umask";
+  public static final int DEFAULT_UMASK = 0022;
+
+  /** Get the user file creation mask (umask) */
+  public static FsPermission getUMask(Configuration conf) {
+    int umask = DEFAULT_UMASK;
+    if (conf != null) {
+      umask = conf.getInt(UMASK_LABEL, DEFAULT_UMASK);
+    }
+    return new FsPermission((short)umask);
+  }
+  /** Set the user file creation mask (umask) */
+  public static void setUMask(Configuration conf, FsPermission umask) {
+    conf.setInt(UMASK_LABEL, umask.toShort());
+  }
+
+  private static final FsPermission NONE = new FsPermission((short)0);
+
+  /**
+   * Get the default permission from conf.
+   * @param conf
+   */
+  public static FsPermission getDefault(Configuration conf) {
+    return NONE.applyUMask(getUMask(conf));
+  }
+
+  /**
+   * Create a FsPermission from a Unix symbolic permission string
+   * @param unixSymbolicPermission e.g. "-rw-rw-rw-"
+   */
+  public static FsPermission valueOf(String unixSymbolicPermission) {
+    if (unixSymbolicPermission.length() != 10) {
+      throw new IllegalArgumentException("length != 10(unixSymbolicPermission="
+          + unixSymbolicPermission + ")");
+    }
+    int n = 0;
+    for(int i = 1; i < unixSymbolicPermission.length(); i++) {
+      n = n << 1;
+      char c = unixSymbolicPermission.charAt(i);
+      n += c == '-' || c == 'T'? 0: 1;
+    }
+    return new FsPermission((short)n);
+  }
+}
\ No newline at end of file

Propchange: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/FsPermission.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/FsPermission.java
------------------------------------------------------------------------------
    svn:keywords = Id Revision HeadURL

Added: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/PermissionStatus.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/PermissionStatus.java?rev=600964&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/PermissionStatus.java (added)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/PermissionStatus.java Tue Dec  4 06:25:05 2007
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.permission;
+
+import org.apache.hadoop.io.*;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+/**
+ * Store permission related information.
+ */
+public class PermissionStatus implements Writable {
+  static final WritableFactory FACTORY = new WritableFactory() {
+    public Writable newInstance() { return new PermissionStatus(); }
+  };
+  static {                                      // register a ctor
+    WritableFactories.setFactory(PermissionStatus.class, FACTORY);
+  }
+
+  private String username;
+  private String groupname;
+  private FsPermission permission;
+
+  private PermissionStatus() {}
+
+  /** Constructor */
+  public PermissionStatus(String user, String group, FsPermission permission) {
+    username = user;
+    groupname = group;
+    this.permission = permission;
+  }
+
+  /** Return user name */
+  public String getUserName() {return username;}
+
+  /** Return group name */
+  public String getGroupName() {return groupname;}
+
+  /** Return permission */
+  public FsPermission getPermission() {return permission;}
+
+  /** {@inheritDoc} */
+  public void readFields(DataInput in) throws IOException {
+    username = Text.readString(in);
+    groupname = Text.readString(in);
+    permission = FsPermission.read(in);
+  }
+
+  /** {@inheritDoc} */
+  public void write(DataOutput out) throws IOException {
+    Text.writeString(out, username);
+    Text.writeString(out, groupname);
+    permission.write(out);
+  }
+
+  /**
+   * Create and initialize a {@link PermissionStatus} from {@link DataInput}.
+   */
+  public static PermissionStatus read(DataInput in) throws IOException {
+    PermissionStatus p = new PermissionStatus();
+    p.readFields(in);
+    return p;
+  }
+
+  /** {@inheritDoc} */
+  public String toString() {
+    return username + ":" + groupname + ":" + permission;
+  }
+}
\ No newline at end of file

Propchange: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/PermissionStatus.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/permission/PermissionStatus.java
------------------------------------------------------------------------------
    svn:keywords = Id Revision HeadURL

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/s3/S3FileSystem.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/s3/S3FileSystem.java?rev=600964&r1=600963&r2=600964&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/s3/S3FileSystem.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/s3/S3FileSystem.java Tue Dec  4 06:25:05 2007
@@ -33,6 +33,7 @@
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.io.retry.RetryProxy;
@@ -127,8 +128,11 @@
     return store.inodeExists(makeAbsolute(path));
   }
 
+  /**
+   * @param permission Currently ignored.
+   */
   @Override
-  public boolean mkdirs(Path path) throws IOException {
+  public boolean mkdirs(Path path, FsPermission permission) throws IOException {
     Path absolutePath = makeAbsolute(path);
     INode inode = store.retrieveINode(absolutePath);
     if (inode == null) {
@@ -178,9 +182,13 @@
     return ret.toArray(new FileStatus[0]);
   }
 
-  @Override
-  public FSDataOutputStream create(Path file, boolean overwrite, int bufferSize,
-                                   short replication, long blockSize, Progressable progress)
+  /**
+   * @param permission Currently ignored.
+   */
+  @Override
+  public FSDataOutputStream create(Path file, FsPermission permission,
+      boolean overwrite, int bufferSize,
+      short replication, long blockSize, Progressable progress)
     throws IOException {
 
     INode inode = store.retrieveINode(makeAbsolute(file));

Added: lucene/hadoop/trunk/src/test/org/apache/hadoop/fs/TestLocalFileSystemPermission.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/test/org/apache/hadoop/fs/TestLocalFileSystemPermission.java?rev=600964&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/test/org/apache/hadoop/fs/TestLocalFileSystemPermission.java (added)
+++ lucene/hadoop/trunk/src/test/org/apache/hadoop/fs/TestLocalFileSystemPermission.java Tue Dec  4 06:25:05 2007
@@ -0,0 +1,155 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.*;
+import org.apache.hadoop.util.StringUtils;
+
+import java.io.*;
+import java.util.*;
+
+import junit.framework.*;
+
+/**
+ * This class tests the local file system via the FileSystem abstraction.
+ */
+public class TestLocalFileSystemPermission extends TestCase {
+  static final String TEST_PATH_PREFIX = new Path(System.getProperty(
+      "test.build.data", "/tmp")).toString().replace(' ', '_')
+      + "/" + TestLocalFileSystemPermission.class.getSimpleName() + "_";
+
+  {
+    try {
+      ((org.apache.commons.logging.impl.Log4JLogger)FileSystem.LOG).getLogger()
+      .setLevel(org.apache.log4j.Level.DEBUG);
+    }
+    catch(Exception e) {
+      System.out.println("Cannot change log level\n"
+          + StringUtils.stringifyException(e));
+    }
+  }
+
+  private Path writeFile(FileSystem fs, String name) throws IOException {
+    Path f = new Path(TEST_PATH_PREFIX + name);
+    FSDataOutputStream stm = fs.create(f);
+    stm.writeBytes("42\n");
+    stm.close();
+    return f;
+  }
+
+  private void cleanupFile(FileSystem fs, Path name) throws IOException {
+    assertTrue(fs.exists(name));
+    fs.delete(name);
+    assertTrue(!fs.exists(name));
+  }
+
+  /** Test LocalFileSystem.setPermission */
+  public void testLocalFSsetPermission() throws IOException {
+    if (Path.WINDOWS) {
+      System.out.println("Cannot run test for Windows");
+      return;
+    }
+    Configuration conf = new Configuration();
+    LocalFileSystem localfs = FileSystem.getLocal(conf);
+    String filename = "foo";
+    Path f = writeFile(localfs, filename);
+    try {
+      System.out.println(filename + ": " + getPermission(localfs, f));
+    }
+    catch(Exception e) {
+      System.out.println(StringUtils.stringifyException(e));
+      System.out.println("Cannot run test");
+      return;
+    }
+
+    try {
+      // create files and manipulate them.
+      FsPermission all = new FsPermission((short)0777);
+      FsPermission none = new FsPermission((short)0);
+
+      localfs.setPermission(f, none);
+      assertEquals(none, getPermission(localfs, f));
+
+      localfs.setPermission(f, all);
+      assertEquals(all, getPermission(localfs, f));
+    }
+    finally {cleanupFile(localfs, f);}
+  }
+
+  FsPermission getPermission(LocalFileSystem fs, Path p) throws IOException {
+    RawLocalFileSystem.RawLocalFileStatus s
+      = (RawLocalFileSystem.RawLocalFileStatus)fs.getFileStatus(p);
+    return s.getPermissionStatus().getPermission();
+  }
+
+  /** Test LocalFileSystem.setOwner */
+  public void testLocalFSsetOwner() throws IOException {
+    if (Path.WINDOWS) {
+      System.out.println("Cannot run test for Windows");
+      return;
+    }
+
+    Configuration conf = new Configuration();
+    LocalFileSystem localfs = FileSystem.getLocal(conf);
+    String filename = "bar";
+    Path f = writeFile(localfs, filename);
+    List<String> groups = null;
+    try {
+      groups = getGroups();
+      System.out.println(filename + ": " + getPermission(localfs, f));
+    }
+    catch(IOException e) {
+      System.out.println(StringUtils.stringifyException(e));
+      System.out.println("Cannot run test");
+      return;
+    }
+    if (groups == null || groups.size() < 2) {
+      System.out.println("Cannot run test: need at least two groups.  groups="
+          + groups);
+      return;
+    }
+
+    // create files and manipulate them.
+    try {
+      String g0 = groups.get(0);
+      localfs.setOwner(f, null, g0);
+      assertEquals(g0, getGroup(localfs, f));
+
+      String g1 = groups.get(1);
+      localfs.setOwner(f, null, g1);
+      assertEquals(g1, getGroup(localfs, f));
+    }
+    finally {cleanupFile(localfs, f);}
+  }
+
+  static List<String> getGroups() throws IOException {
+    List<String> a = new ArrayList<String>();
+    String s = ShellCommand.execCommand(ShellCommand.GROUPS_COMMAND);
+    for(StringTokenizer t = new StringTokenizer(s); t.hasMoreTokens(); ) {
+      a.add(t.nextToken());
+    }
+    return a;
+  }
+
+  String getGroup(LocalFileSystem fs, Path p) throws IOException {
+    RawLocalFileSystem.RawLocalFileStatus s
+      = (RawLocalFileSystem.RawLocalFileStatus)fs.getFileStatus(p);
+    return s.getPermissionStatus().getGroupName();
+  }
+}

Propchange: lucene/hadoop/trunk/src/test/org/apache/hadoop/fs/TestLocalFileSystemPermission.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/hadoop/trunk/src/test/org/apache/hadoop/fs/TestLocalFileSystemPermission.java
------------------------------------------------------------------------------
    svn:keywords = Id Revision HeadURL