Pārlūkot izejas kodu

HADOOP-2833. Merge to 0.16 branch for 0.16.2 release. This fixes a cache consistency problem in 0.16.2

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/branches/branch-0.16@642478 13f79535-47bb-0310-9956-ffa450edef68
Nigel Daley 17 gadi atpakaļ
vecāks
revīzija
5011ba27a3

+ 3 - 0
CHANGES.txt

@@ -64,6 +64,9 @@ Release 0.16.2 - 2008-04-02
     between the RecordReader and the map threads. (Alejandro Abdelnur via
     omalley)
 
+    HADOOP-2833. Do not use "Dr. Who" as the default user in JobClient.
+    A valid user name is required. (Tsz Wo (Nicholas), SZE via rangadi)
+
 Release 0.16.1 - 2008-03-13
 
   INCOMPATIBLE CHANGES

+ 9 - 1
src/java/org/apache/hadoop/dfs/DFSClient.java

@@ -56,10 +56,11 @@ import javax.security.auth.login.LoginException;
  *
  ********************************************************/
 class DFSClient implements FSConstants {
-  public static final Log LOG = LogFactory.getLog("org.apache.hadoop.fs.DFSClient");
+  public static final Log LOG = LogFactory.getLog(DFSClient.class);
   static final int MAX_BLOCK_ACQUIRE_FAILURES = 3;
   private static final int TCP_WINDOW_SIZE = 128 * 1024; // 128 KB
   ClientProtocol namenode;
+  final UnixUserGroupInformation ugi;
   volatile boolean clientRunning = true;
   Random r = new Random();
   String clientName;
@@ -140,6 +141,13 @@ class DFSClient implements FSConstants {
     this.socketTimeout = conf.getInt("dfs.socket.timeout", 
                                      FSConstants.READ_TIMEOUT);
     this.socketFactory = NetUtils.getSocketFactory(conf, ClientProtocol.class);
+
+    try {
+      this.ugi = UnixUserGroupInformation.login(conf, true);
+    } catch (LoginException e) {
+      throw (IOException)(new IOException().initCause(e));
+    }
+
     this.namenode = createNamenode(nameNodeAddr, conf);
     String taskId = conf.get("mapred.task.id");
     if (taskId != null) {

+ 5 - 0
src/java/org/apache/hadoop/dfs/DistributedFileSystem.java

@@ -96,6 +96,11 @@ public class DistributedFileSystem extends FileSystem {
     workingDir = makeAbsolute(dir);
   }
 
+  /** {@inheritDoc} */
+  public Path getHomeDirectory() {
+    return new Path("/user/" + dfs.ugi.getUserName()).makeQualified(this);
+  }
+
   private String getPathName(Path file) {
     checkPath(file);
     String result = makeAbsolute(file).toUri().getPath();

+ 5 - 7
src/java/org/apache/hadoop/mapred/JobClient.java

@@ -458,13 +458,12 @@ public class JobClient extends Configured implements MRConstants, Tool  {
      * set this user's id in job configuration, so later job files can be
      * accessed using this user's id
      */
+    UnixUserGroupInformation ugi = null;
     try {
-      UnixUserGroupInformation.saveToConf(job,
-          UnixUserGroupInformation.UGI_PROPERTY_NAME, UnixUserGroupInformation
-          .login(job));
+      ugi = UnixUserGroupInformation.login(job, true);
     } catch (LoginException e) {
-      throw new IOException("Failed to get the current user's information: "
-          + e.getMessage());
+      throw (IOException)(new IOException(
+          "Failed to get the current user's information.").initCause(e));
     }
       
     //
@@ -527,8 +526,7 @@ public class JobClient extends Configured implements MRConstants, Tool  {
     }
 
     // Set the user's name and working directory
-    String user = System.getProperty("user.name");
-    job.setUser(user != null ? user : "Dr Who");
+    job.setUser(ugi.getUserName());
     if (job.getWorkingDirectory() == null) {
       job.setWorkingDirectory(fs.getWorkingDirectory());          
     }

+ 13 - 3
src/java/org/apache/hadoop/security/UnixUserGroupInformation.java

@@ -251,6 +251,12 @@ public class UnixUserGroupInformation extends UserGroupInformation {
     }
   }
 
+  /** Equivalent to login(conf, false). */
+  public static UnixUserGroupInformation login(Configuration conf)
+    throws LoginException {
+    return login(conf, false);
+  }
+  
   /** Get a user's name & its group names from the given configuration; 
    * If it is not defined in the configuration, get the current user's
    * information from Unix.
@@ -258,16 +264,20 @@ public class UnixUserGroupInformation extends UserGroupInformation {
    * the UGI map.
    * 
    *  @param conf either a job configuration or client's configuration
+   *  @param save saving it to conf?
    *  @return UnixUserGroupInformation a user/group information
    *  @exception LoginException if not able to get the user/group information
    */
-  public static UnixUserGroupInformation login(Configuration conf) 
-   throws LoginException {
+  public static UnixUserGroupInformation login(Configuration conf, boolean save
+      ) throws LoginException {
     UnixUserGroupInformation ugi = readFromConf(conf, UGI_PROPERTY_NAME);
     if (ugi == null) {
       ugi = login();
       LOG.debug("Unix Login: " + ugi);
-    } 
+      if (save) {
+        saveToConf(conf, UGI_PROPERTY_NAME, ugi);
+      }
+    }
     return ugi;
   }
   

+ 58 - 0
src/test/org/apache/hadoop/mapred/TestSubmitJob.java

@@ -0,0 +1,58 @@
+/**
+ * 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.mapred;
+
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.security.UnixUserGroupInformation;
+
+public class TestSubmitJob extends junit.framework.TestCase 
+{
+  static final Path TEST_ROOT_DIR = new Path(
+      System.getProperty("test.build.data", "/tmp")
+      + Path.SEPARATOR + TestSubmitJob.class.getSimpleName());
+  static final JobConf conf = new JobConf(TestSubmitJob.class);
+
+  static JobConf createJobConf(FileSystem fs) throws Exception {
+    fs.delete(TEST_ROOT_DIR);
+    Path input = new Path(TEST_ROOT_DIR, "input");
+    if (!fs.mkdirs(input)) {
+      throw new Exception("Cannot create " + input);
+    }
+    conf.setInputPath(input);
+    conf.setOutputPath(new Path(TEST_ROOT_DIR, "output"));
+    return conf;    
+  }
+  
+  public void testSubmitJobUsername() throws Exception {
+    FileSystem.LOG.info("TEST_ROOT_DIR=" + TEST_ROOT_DIR);
+    FileSystem fs = FileSystem.get(conf);
+    try {
+      UnixUserGroupInformation ugi = UnixUserGroupInformation.login(conf);
+  
+      JobConf conf = createJobConf(fs);
+      JobClient jc = new JobClient(conf);
+      assertEquals(null, conf.getUser());
+      RunningJob rj = jc.submitJob(conf);
+      assertEquals(ugi.getUserName(), conf.getUser());
+      for(; !rj.isComplete(); Thread.sleep(500));
+    }
+    finally {
+      FileUtil.fullyDelete(fs, TEST_ROOT_DIR);
+    }
+  }
+}