浏览代码

HADOOP-1873. Implement user permissions for Map/Reduce. (Hairong Kuang via shv)

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@611056 13f79535-47bb-0310-9956-ffa450edef68
Konstantin Shvachko 17 年之前
父节点
当前提交
4b57fa6713

+ 6 - 3
CHANGES.txt

@@ -70,6 +70,9 @@ Trunk (unreleased changes)
     This information can be used to develop tools for analysing jobs.
     (Amareshwari Sri Ramadasu via acmurthy)
 
+		HADOOP-1873. Implement user permissions for Map/Reduce framework.
+		(Hairong Kuang via shv)
+
   IMPROVEMENTS
 
     HADOOP-2045.  Change committer list on website to a table, so that
@@ -677,7 +680,7 @@ Release 0.15.0 - 2007-11-2
     (Jim Kellerman via cutting)
 
     HADOOP-1795.  Fix so that jobs can generate output file names with
-    special characters.  (Frédéric Bertin via cutting)
+    special characters.  (Fr??d??ric Bertin via cutting)
 
     HADOOP-1810.  Fix incorrect value type in MRBench (SmallJobs)
     (Devaraj Das via tomwhite)
@@ -3425,7 +3428,7 @@ Release 0.6.0 - 2006-09-08
     on the NameNode.  (Milind Bhandarkar via cutting)
 
 30. HADOOP-424.  Fix MapReduce so that jobs which generate zero splits
-    do not fail.  (Frédéric Bertin via cutting)
+    do not fail.  (Fr??d??ric Bertin via cutting)
 
 31. HADOOP-408.  Adjust some timeouts and remove some others so that
     unit tests run faster.  (cutting)
@@ -3491,7 +3494,7 @@ Release 0.5.0 - 2006-08-04
     StreamSequenceRecordReader.  (Michel Tourn via cutting)
 
  5. HADOOP-358.  Fix a NPE bug in Path.equals().
-    (Frédéric Bertin via cutting)
+    (Fr??d??ric Bertin via cutting)
 
  6. HADOOP-327.  Fix ToolBase to not call System.exit() when
     exceptions are thrown.  (Hairong Kuang via cutting)

+ 44 - 0
src/java/org/apache/hadoop/fs/FileSystem.java

@@ -201,6 +201,50 @@ public abstract class FileSystem extends Configured {
     return path.makeQualified(this);
   }
     
+  /** create a file with the provided permission
+   * The permission of the file is set to be the provided permission as in
+   * setPermission, not permission&~umask
+   * 
+   * It is implemented using two RPCs. It is understood that it is inefficient,
+   * but the implementation is thread-safe. The other option is to change the
+   * value of umask in configuration to be 0, but it is not thread-safe.
+   * 
+   * @param fs file system handle
+   * @param file the name of the file to be created
+   * @param permission the permission of the file
+   * @return an output stream
+   * @throws IOException
+   */
+  public static FSDataOutputStream create(FileSystem fs,
+      Path file, FsPermission permission) throws IOException {
+    // create the file with default permission
+    FSDataOutputStream out = fs.create(file);
+    // set its permission to the supplied one
+    fs.setPermission(file, permission);
+    return out;
+  }
+
+  /** create a directory with the provided permission
+   * The permission of the directory is set to be the provided permission as in
+   * setPermission, not permission&~umask
+   * 
+   * @see #create(FileSystem, Path, FsPermission)
+   * 
+   * @param fs file system handle
+   * @param dir the name of the directory to be created
+   * @param permission the permission of the directory
+   * @return true if the directory creation succeeds; false otherwise
+   * @throws IOException
+   */
+  public static boolean mkdirs(FileSystem fs, Path dir, FsPermission permission)
+  throws IOException {
+    // create the directory using the default permission
+    boolean result = fs.mkdirs(dir);
+    // set its permission to be the supplied one
+    fs.setPermission(dir, permission);
+    return result;
+  }
+
   ///////////////////////////////////////////////////////////////
   // FileSystem
   ///////////////////////////////////////////////////////////////

+ 11 - 0
src/java/org/apache/hadoop/fs/permission/FsPermission.java

@@ -69,6 +69,17 @@ public class FsPermission implements Writable {
    */
   public FsPermission(short mode) { fromShort(mode); }
 
+  /**
+   * Copy constructor
+   * 
+   * @param other other permission
+   */
+  public FsPermission(FsPermission other) {
+    this.useraction = other.useraction;
+    this.groupaction = other.groupaction;
+    this.otheraction = other.otheraction;
+  }
+  
   /** Return user {@link FsAction}. */
   public FsAction getUserAction() {return useraction;}
 

+ 32 - 3
src/java/org/apache/hadoop/mapred/JobClient.java

@@ -37,6 +37,8 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Random;
 
+import javax.security.auth.login.LoginException;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -45,6 +47,7 @@ import org.apache.hadoop.filecache.DistributedCache;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.Text;
@@ -56,6 +59,7 @@ import org.apache.hadoop.io.retry.RetryProxy;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.mapred.TaskInProgress;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UnixUserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
@@ -425,6 +429,13 @@ public class JobClient extends Configured implements MRConstants, Tool  {
     return submitJob(job);
   }
     
+  // job files are world-wide readable and owner writable
+  final private static FsPermission JOB_FILE_PERMISSION = 
+    FsPermission.createImmutable((short) 0644); // rw-r--r--
+
+  // system directories are world-wide readable and owner readable
+  final static FsPermission SYSTEM_DIR_PERMISSION =
+    FsPermission.createImmutable((short) 0733); // rwx-wx-wx
    
   /**
    * Submit a job to the MR system.
@@ -440,8 +451,21 @@ public class JobClient extends Configured implements MRConstants, Tool  {
    */
   public RunningJob submitJob(JobConf job) throws FileNotFoundException, 
                                                   InvalidJobConfException, IOException {
+    /*
+     * set this user's id in job configuration, so later job files can be
+     * accessed using this user's id
+     */
+    try {
+      UnixUserGroupInformation.saveToConf(job,
+          UnixUserGroupInformation.UGI_PROPERTY_NAME, UnixUserGroupInformation
+          .login(job));
+    } catch (LoginException e) {
+      throw new IOException("Failed to get the current user's information: "
+          + e.getMessage());
+    }
+      
     //
-    // First figure out what fs the JobTracker is using.  Copy the
+    // Figure out what fs the JobTracker is using.  Copy the
     // job to it, under a temporary name.  This allows DFS to work,
     // and under the local fs also provides UNIX-like object loading 
     // semantics.  (that is, if the job file is deleted right after
@@ -454,6 +478,7 @@ public class JobClient extends Configured implements MRConstants, Tool  {
     FileSystem fs = getFs();
     LOG.debug("default FileSystem: " + fs.getUri());
     fs.delete(submitJobDir);    
+    FileSystem.mkdirs(fs, submitJobDir, new FsPermission(SYSTEM_DIR_PERMISSION));
     Path submitJobFile = new Path(submitJobDir, "job.xml");
     Path submitJarFile = new Path(submitJobDir, "job.jar");
     Path submitSplitFile = new Path(submitJobDir, "job.split");
@@ -492,6 +517,7 @@ public class JobClient extends Configured implements MRConstants, Tool  {
       job.setJar(submitJarFile.toString());
       fs.copyFromLocalFile(new Path(originalJarPath), submitJarFile);
       fs.setReplication(submitJarFile, replication);
+      fs.setPermission(submitJarFile, new FsPermission(JOB_FILE_PERMISSION));
     } else {
       LOG.warn("No job jar file set.  User classes may not be found. "+
                "See JobConf(Class) or JobConf#setJar(String).");
@@ -535,7 +561,8 @@ public class JobClient extends Configured implements MRConstants, Tool  {
       }
     });
     // write the splits to a file for the job tracker
-    FSDataOutputStream out = fs.create(submitSplitFile);
+    FSDataOutputStream out = FileSystem.create(fs,
+        submitSplitFile, new FsPermission(JOB_FILE_PERMISSION));
     try {
       writeSplitsFile(splits, out);
     } finally {
@@ -545,7 +572,9 @@ public class JobClient extends Configured implements MRConstants, Tool  {
     job.setNumMapTasks(splits.length);
         
     // Write job file to JobTracker's fs        
-    out = fs.create(submitJobFile, replication);
+    out = FileSystem.create(fs, submitJobFile,
+        new FsPermission(JOB_FILE_PERMISSION));
+
     try {
       job.write(out);
     } finally {

+ 10 - 1
src/java/org/apache/hadoop/mapred/JobTracker.java

@@ -46,7 +46,10 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.AccessControlException;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RPC.VersionMismatch;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.metrics.MetricsContext;
@@ -677,11 +680,17 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
         // clean up the system dir, which will only work if hdfs is out of 
         // safe mode
         fs.delete(systemDir);
-        if (fs.mkdirs(systemDir)) {
+        if (FileSystem.mkdirs(fs, systemDir, 
+            new FsPermission(JobClient.SYSTEM_DIR_PERMISSION))) {
           break;
         }
         LOG.error("Mkdirs failed to create " + systemDir);
       } catch (IOException ie) {
+        if (ie instanceof RemoteException && 
+            AccessControlException.class.getName().equals(
+                ((RemoteException)ie).getClassName())) {
+          throw ie;
+        }
         LOG.info("problem cleaning system directory: " + systemDir, ie);
       }
       Thread.sleep(SYSTEM_DIR_CLEANUP_RETRY_PERIOD);

+ 9 - 0
src/test/org/apache/hadoop/security/TestPermission.java

@@ -89,6 +89,15 @@ public class TestPermission extends TestCase {
       checkPermission(fs, "/b1", inheritPerm);
       checkPermission(fs, "/b1/b2", inheritPerm);
       checkPermission(fs, "/b1/b2/b3.txt", filePerm);
+      
+      conf.setInt(FsPermission.UMASK_LABEL, 0022);
+      FsPermission permission = 
+        FsPermission.createImmutable((short)0666);
+      FileSystem.mkdirs(fs, new Path("/c1"), new FsPermission(permission));
+      FileSystem.create(fs, new Path("/c1/c2.txt"),
+          new FsPermission(permission));
+      checkPermission(fs, "/c1", permission);
+      checkPermission(fs, "/c1/c2.txt", permission);
     }
     finally {
       try{fs.close();} catch(Exception e) {}