Просмотр исходного кода

HADOOP-2833. Revert bad commit.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/branches/branch-0.16@642477 13f79535-47bb-0310-9956-ffa450edef68
Nigel Daley 17 лет назад
Родитель
Сommit
ef5b13fc98

+ 0 - 3
CHANGES.txt

@@ -64,9 +64,6 @@ 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

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

@@ -56,11 +56,10 @@ import javax.security.auth.login.LoginException;
  *
  ********************************************************/
 class DFSClient implements FSConstants {
-  public static final Log LOG = LogFactory.getLog(DFSClient.class);
+  public static final Log LOG = LogFactory.getLog("org.apache.hadoop.fs.DFSClient");
   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;
@@ -141,13 +140,6 @@ 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) {

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

@@ -96,11 +96,6 @@ 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();
@@ -363,7 +358,6 @@ public class DistributedFileSystem extends FileSystem {
       if(FileNotFoundException.class.getName().equals(re.getClassName())) {
         throw new FileNotFoundException("File does not exist: " + p);
       }
-      throw re;
     }
   }
 
@@ -379,7 +373,6 @@ public class DistributedFileSystem extends FileSystem {
       if(FileNotFoundException.class.getName().equals(re.getClassName())) {
         throw new FileNotFoundException("File does not exist: " + p);
       }
-      throw re;
     }
   }
 }

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

@@ -458,12 +458,13 @@ 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 {
-      ugi = UnixUserGroupInformation.login(job, true);
+      UnixUserGroupInformation.saveToConf(job,
+          UnixUserGroupInformation.UGI_PROPERTY_NAME, UnixUserGroupInformation
+          .login(job));
     } catch (LoginException e) {
-      throw (IOException)(new IOException(
-          "Failed to get the current user's information.").initCause(e));
+      throw new IOException("Failed to get the current user's information: "
+          + e.getMessage());
     }
       
     //
@@ -526,7 +527,8 @@ public class JobClient extends Configured implements MRConstants, Tool  {
     }
 
     // Set the user's name and working directory
-    job.setUser(ugi.getUserName());
+    String user = System.getProperty("user.name");
+    job.setUser(user != null ? user : "Dr Who");
     if (job.getWorkingDirectory() == null) {
       job.setWorkingDirectory(fs.getWorkingDirectory());          
     }

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

@@ -251,12 +251,6 @@ 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.
@@ -264,20 +258,16 @@ 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, boolean save
-      ) throws LoginException {
+  public static UnixUserGroupInformation login(Configuration conf) 
+   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;
   }
   

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

@@ -1,58 +0,0 @@
-/**
- * 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);
-    }
-  }
-}