瀏覽代碼

Merge r1244222 through r1245749 from 0.23.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23-PB@1245750 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 13 年之前
父節點
當前提交
bd3c2b26ec
共有 40 個文件被更改,包括 606 次插入114 次删除
  1. 15 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 30 6
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
  3. 5 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java
  4. 9 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java
  5. 0 7
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalFileSystem.java
  6. 10 5
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java
  7. 7 0
      hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
  8. 123 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java
  9. 12 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  10. 55 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
  11. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  12. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HdfsConfiguration.java
  13. 15 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java
  14. 14 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  15. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  16. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
  17. 0 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAppendDifferentChecksum.java
  18. 46 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestShortCircuitLocalRead.java
  19. 4 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLargeDirectoryDelete.java
  20. 19 1
      hadoop-mapreduce-project/CHANGES.txt
  21. 4 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java
  22. 52 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java
  23. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapred/TestMRWithDistributedCache.java
  24. 17 22
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/security/TokenCache.java
  25. 89 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/security/TestTokenCache.java
  26. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java
  27. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
  28. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/JHLogAnalyzer.java
  29. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/io/FileBench.java
  30. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineFileInputFormat.java
  31. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestConcatenatedCompressedInput.java
  32. 9 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRChildTask.java
  33. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTextInputFormat.java
  34. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapCollection.java
  35. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestFileInputFormat.java
  36. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRKeyValueTextInputFormat.java
  37. 29 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
  38. 1 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
  39. 2 2
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ClusterSetup.apt.vm
  40. 19 0
      hadoop-project/pom.xml

+ 15 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -80,7 +80,12 @@ Release 0.23.2 - UNRELEASED
     HADOOP-8071. Avoid an extra packet in client code when nagling is
     disabled. (todd)
 
+    HADOOP-6502. Improve the performance of Configuration.getClassByName when
+    the class is not found by caching negative results.
+    (sharad, todd via todd)
+
   BUG FIXES
+    HADOOP-8054 NPE with FilterFileSystem (Daryn Sharp via bobby)
 
     HADOOP-8042  When copying a file out of HDFS, modifying it, and uploading
     it back into HDFS, the put fails due to a CRC mismatch
@@ -97,6 +102,16 @@ Release 0.23.2 - UNRELEASED
     Double.MAX_VALUE) to avoid making Ganglia's gmetad core. (Varun Kapoor
     via mattf)
 
+    HADOOP-8074. Small bug in hadoop error message for unknown commands.
+    (Colin Patrick McCabe via eli)
+
+    HADOOP-8082 add hadoop-client and hadoop-minicluster to the 
+    dependency-management section. (tucu)
+
+    HADOOP-8066 The full docs build intermittently fails (abayer via tucu)
+
+    HADOOP-8083 javadoc generation for some modules is not done under target/ (tucu)
+
 Release 0.23.1 - 2012-02-08 
 
   INCOMPATIBLE CHANGES

+ 30 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java

@@ -1146,6 +1146,22 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
    * @throws ClassNotFoundException if the class is not found.
    */
   public Class<?> getClassByName(String name) throws ClassNotFoundException {
+    Class<?> ret = getClassByNameOrNull(name);
+    if (ret == null) {
+      throw new ClassNotFoundException("Class " + name + " not found");
+    }
+    return ret;
+  }
+  
+  /**
+   * Load a class by name, returning null rather than throwing an exception
+   * if it couldn't be loaded. This is to avoid the overhead of creating
+   * an exception.
+   * 
+   * @param name the class name
+   * @return the class object, or null if it could not be found.
+   */
+  public Class<?> getClassByNameOrNull(String name) {
     Map<String, Class<?>> map;
     
     synchronized (CACHE_CLASSES) {
@@ -1157,12 +1173,20 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
       }
     }
 
-    Class<?> clazz = map.get(name);
-    if (clazz == null) {
-      clazz = Class.forName(name, true, classLoader);
-      if (clazz != null) {
-        // two putters can race here, but they'll put the same class
-        map.put(name, clazz);
+    Class<?> clazz = null;
+    if (!map.containsKey(name)) {
+      try {
+        clazz = Class.forName(name, true, classLoader);
+      } catch (ClassNotFoundException e) {
+        map.put(name, null); //cache negative that class is not found
+        return null;
+      }
+      // two putters can race here, but they'll put the same class
+      map.put(name, clazz);
+    } else { // check already performed on this class name
+      clazz = map.get(name);
+      if (clazz == null) { // found the negative
+        return null;
       }
     }
 

+ 5 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java

@@ -80,6 +80,11 @@ public class FilterFileSystem extends FileSystem {
    */
   public void initialize(URI name, Configuration conf) throws IOException {
     super.initialize(name, conf);
+    // this is less than ideal, but existing filesystems sometimes neglect
+    // to initialize the embedded filesystem
+    if (fs.getConf() == null) {
+      fs.initialize(name, conf);
+    }
     String scheme = name.getScheme();
     if (!scheme.equals(fs.getUri().getScheme())) {
       swapScheme = scheme;

+ 9 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java

@@ -269,7 +269,15 @@ public class FsShell extends Configured implements Tool {
   
   private void displayError(String cmd, String message) {
     for (String line : message.split("\n")) {
-      System.err.println(cmd.substring(1) + ": " + line);
+      System.err.println(cmd + ": " + line);
+      if (cmd.charAt(0) != '-') {
+        Command instance = null;
+        instance = commandFactory.getInstance("-" + cmd);
+        if (instance != null) {
+          System.err.println("Did you mean -" + cmd + "?  This command " +
+              "begins with a dash.");
+        }
+      }
     }
   }
   

+ 0 - 7
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalFileSystem.java

@@ -48,13 +48,6 @@ public class LocalFileSystem extends ChecksumFileSystem {
     super(rawLocalFileSystem);
   }
     
-  @Override
-  public void initialize(URI uri, Configuration conf) throws IOException {
-    super.initialize(uri, conf);
-    // ctor didn't initialize the filtered fs
-    getRawFileSystem().initialize(uri, conf);
-  }
-  
   /** Convert a path to a File. */
   public File pathToFile(Path path) {
     return ((RawLocalFileSystem)fs).pathToFile(path);

+ 10 - 5
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java

@@ -86,17 +86,22 @@ public class ReflectionUtils {
     //invoke configure on theObject
     try {
       Class<?> jobConfClass = 
-        conf.getClassByName("org.apache.hadoop.mapred.JobConf");
+        conf.getClassByNameOrNull("org.apache.hadoop.mapred.JobConf");
+      if (jobConfClass == null) {
+        return;
+      }
+      
       Class<?> jobConfigurableClass = 
-        conf.getClassByName("org.apache.hadoop.mapred.JobConfigurable");
-       if (jobConfClass.isAssignableFrom(conf.getClass()) &&
+        conf.getClassByNameOrNull("org.apache.hadoop.mapred.JobConfigurable");
+      if (jobConfigurableClass == null) {
+        return;
+      }
+      if (jobConfClass.isAssignableFrom(conf.getClass()) &&
             jobConfigurableClass.isAssignableFrom(theObject.getClass())) {
         Method configureMethod = 
           jobConfigurableClass.getMethod("configure", jobConfClass);
         configureMethod.invoke(theObject, conf);
       }
-    } catch (ClassNotFoundException e) {
-      //JobConf/JobConfigurable not in classpath. no need to configure
     } catch (Exception e) {
       throw new RuntimeException("Error in configuring object", e);
     }

+ 7 - 0
hadoop-common-project/hadoop-common/src/main/resources/core-default.xml

@@ -216,6 +216,13 @@
   determine the host, port, etc. for a filesystem.</description>
 </property>
 
+<property>
+  <name>fs.default.name</name>
+  <value>file:///</value>
+  <description>Deprecated. Use (fs.defaultFS) property
+  instead</description>
+</property>
+
 <property>
   <name>fs.trash.interval</name>
   <value>0</value>

+ 123 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java

@@ -18,24 +18,39 @@
 
 package org.apache.hadoop.fs;
 
+import static org.junit.Assert.*;
+import static org.mockito.Matchers.*;
+import static org.mockito.Mockito.*;
+
 import java.io.IOException;
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
+import java.net.URI;
 import java.util.EnumSet;
 import java.util.Iterator;
 
-import junit.framework.TestCase;
 import org.apache.commons.logging.Log;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.Options.CreateOpts;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Progressable;
+import org.junit.BeforeClass;
+import org.junit.Test;
 
-public class TestFilterFileSystem extends TestCase {
+public class TestFilterFileSystem {
 
   private static final Log LOG = FileSystem.LOG;
+  private static final Configuration conf = new Configuration();
 
+  @BeforeClass
+  public static void setup() {
+    conf.set("fs.flfs.impl", FilterLocalFileSystem.class.getName());
+    conf.setBoolean("fs.flfs.impl.disable.cache", true);
+    conf.setBoolean("fs.file.impl.disable.cache", true);
+  }
+  
   public static class DontCheck {
     public BlockLocation[] getFileBlockLocations(Path p, 
         long start, long len) { return null; }
@@ -153,6 +168,7 @@ public class TestFilterFileSystem extends TestCase {
     
   }
   
+  @Test
   public void testFilterFileSystem() throws Exception {
     for (Method m : FileSystem.class.getDeclaredMethods()) {
       if (Modifier.isStatic(m.getModifiers()))
@@ -176,4 +192,109 @@ public class TestFilterFileSystem extends TestCase {
     }
   }
   
+  @Test
+  public void testFilterEmbedInit() throws Exception {
+    FileSystem mockFs = createMockFs(false); // no conf = need init
+    checkInit(new FilterFileSystem(mockFs), true);
+  }
+
+  @Test
+  public void testFilterEmbedNoInit() throws Exception {
+    FileSystem mockFs = createMockFs(true); // has conf = skip init
+    checkInit(new FilterFileSystem(mockFs), false);
+  }
+
+  @Test
+  public void testLocalEmbedInit() throws Exception {
+    FileSystem mockFs = createMockFs(false); // no conf = need init
+    checkInit(new LocalFileSystem(mockFs), true);
+  }  
+  
+  @Test
+  public void testLocalEmbedNoInit() throws Exception {
+    FileSystem mockFs = createMockFs(true); // has conf = skip init
+    checkInit(new LocalFileSystem(mockFs), false);
+  }
+  
+  private FileSystem createMockFs(boolean useConf) {
+    FileSystem mockFs = mock(FileSystem.class);
+    when(mockFs.getUri()).thenReturn(URI.create("mock:/"));
+    when(mockFs.getConf()).thenReturn(useConf ? conf : null);
+    return mockFs;
+  }
+
+  @Test
+  public void testGetLocalFsSetsConfs() throws Exception {
+    LocalFileSystem lfs = FileSystem.getLocal(conf);
+    checkFsConf(lfs, conf, 2);
+  }
+
+  @Test
+  public void testGetFilterLocalFsSetsConfs() throws Exception {
+    FilterFileSystem flfs =
+        (FilterFileSystem) FileSystem.get(URI.create("flfs:/"), conf);
+    checkFsConf(flfs, conf, 3);
+  }
+
+  @Test
+  public void testInitLocalFsSetsConfs() throws Exception {
+    LocalFileSystem lfs = new LocalFileSystem();
+    checkFsConf(lfs, null, 2);
+    lfs.initialize(lfs.getUri(), conf);
+    checkFsConf(lfs, conf, 2);
+  }
+
+  @Test
+  public void testInitFilterFsSetsEmbedConf() throws Exception {
+    LocalFileSystem lfs = new LocalFileSystem();
+    checkFsConf(lfs, null, 2);
+    FilterFileSystem ffs = new FilterFileSystem(lfs);
+    assertEquals(lfs, ffs.getRawFileSystem());
+    checkFsConf(ffs, null, 3);
+    ffs.initialize(URI.create("filter:/"), conf);
+    checkFsConf(ffs, conf, 3);
+  }
+
+  @Test
+  public void testInitFilterLocalFsSetsEmbedConf() throws Exception {
+    FilterFileSystem flfs = new FilterLocalFileSystem();
+    assertEquals(LocalFileSystem.class, flfs.getRawFileSystem().getClass());
+    checkFsConf(flfs, null, 3);
+    flfs.initialize(URI.create("flfs:/"), conf);
+    checkFsConf(flfs, conf, 3);
+  }
+
+  private void checkInit(FilterFileSystem fs, boolean expectInit)
+      throws Exception {
+    URI uri = URI.create("filter:/");
+    fs.initialize(uri, conf);
+    
+    FileSystem embedFs = fs.getRawFileSystem();
+    if (expectInit) {
+      verify(embedFs, times(1)).initialize(eq(uri), eq(conf));
+    } else {
+      verify(embedFs, times(0)).initialize(any(URI.class), any(Configuration.class));
+    }
+  }
+
+  // check the given fs's conf, and all its filtered filesystems
+  private void checkFsConf(FileSystem fs, Configuration conf, int expectDepth) {
+    int depth = 0;
+    while (true) {
+      depth++; 
+      assertFalse("depth "+depth+">"+expectDepth, depth > expectDepth);
+      assertEquals(conf, fs.getConf());
+      if (!(fs instanceof FilterFileSystem)) {
+        break;
+      }
+      fs = ((FilterFileSystem) fs).getRawFileSystem();
+    }
+    assertEquals(expectDepth, depth);
+  }
+  
+  private static class FilterLocalFileSystem extends FilterFileSystem {
+    FilterLocalFileSystem() {
+      super(new LocalFileSystem());
+    }
+  }
 }

+ 12 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -138,6 +138,9 @@ Release 0.23.2 - UNRELEASED
     HDFS-2931. Switch DataNode's BlockVolumeChoosingPolicy to private-audience.
     (harsh via szetszwo)
 
+    HDFS-2655. BlockReaderLocal#skip performs unnecessary IO. (Brandon Li 
+    via jitendra)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -154,6 +157,15 @@ Release 0.23.2 - UNRELEASED
     HDFS-2815. Namenode sometimes oes not come out of safemode during 
     NN crash + restart. (Uma Maheswara Rao via suresh)
 
+    HDFS-2950. Secondary NN HTTPS address should be listed as a
+    NAMESERVICE_SPECIFIC_KEY. (todd)
+
+    HDFS-2525. Race between BlockPoolSliceScanner and append. (Brandon Li
+    via jitendra)
+
+    HDFS-2938. Recursive delete of a large directory make namenode
+    unresponsive. (Hari Mankude via suresh)
+
 Release 0.23.1 - 2012-02-08 
 
   INCOMPATIBLE CHANGES

+ 55 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java

@@ -369,26 +369,68 @@ class BlockReaderLocal implements BlockReader {
     if (LOG.isDebugEnabled()) {
       LOG.debug("skip " + n);
     }
+    if (n <= 0) {
+      return 0;
+    }
     if (!verifyChecksum) {
       return dataIn.skip(n);
     }
-    // Skip by reading the data so we stay in sync with checksums.
-    // This could be implemented more efficiently in the future to
-    // skip to the beginning of the appropriate checksum chunk
-    // and then only read to the middle of that chunk.
+  
+    // caller made sure newPosition is not beyond EOF.
+    int remaining = dataBuff.remaining();
+    int position = dataBuff.position();
+    int newPosition = position + (int)n;
+  
+    // if the new offset is already read into dataBuff, just reposition
+    if (n <= remaining) {
+      assert offsetFromChunkBoundary == 0;
+      dataBuff.position(newPosition);
+      return n;
+    }
+  
+    // for small gap, read through to keep the data/checksum in sync
+    if (n - remaining <= bytesPerChecksum) {
+      dataBuff.position(position + remaining);
+      if (skipBuf == null) {
+        skipBuf = new byte[bytesPerChecksum];
+      }
+      int ret = read(skipBuf, 0, (int)(n - remaining));
+      return ret;
+    }
+  
+    // optimize for big gap: discard the current buffer, skip to
+    // the beginning of the appropriate checksum chunk and then
+    // read to the middle of that chunk to be in sync with checksums.
+    this.offsetFromChunkBoundary = newPosition % bytesPerChecksum;
+    long toskip = n - remaining - this.offsetFromChunkBoundary;
+  
+    dataBuff.clear();
+    checksumBuff.clear();
+  
+    long dataSkipped = dataIn.skip(toskip);
+    if (dataSkipped != toskip) {
+      throw new IOException("skip error in data input stream");
+    }
+    long checkSumOffset = (dataSkipped / bytesPerChecksum) * checksumSize;
+    if (checkSumOffset > 0) {
+      long skipped = checksumIn.skip(checkSumOffset);
+      if (skipped != checkSumOffset) {
+        throw new IOException("skip error in checksum input stream");
+      }
+    }
+
+    // read into the middle of the chunk
     if (skipBuf == null) {
       skipBuf = new byte[bytesPerChecksum];
     }
-    long nSkipped = 0;
-    while ( nSkipped < n ) {
-      int toSkip = (int)Math.min(n-nSkipped, skipBuf.length);
-      int ret = read(skipBuf, 0, toSkip);
-      if ( ret <= 0 ) {
-        return nSkipped;
-      }
-      nSkipped += ret;
+    assert skipBuf.length == bytesPerChecksum;
+    assert this.offsetFromChunkBoundary < bytesPerChecksum;
+    int ret = read(skipBuf, 0, this.offsetFromChunkBoundary);
+    if (ret == -1) {  // EOS
+      return toskip;
+    } else {
+      return (toskip + ret);
     }
-    return nSkipped;
   }
 
   @Override

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -86,6 +86,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_NAMENODE_SAFEMODE_MIN_DATANODES_DEFAULT = 0;
   public static final String  DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY = "dfs.namenode.secondary.http-address";
   public static final String  DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_DEFAULT = "0.0.0.0:50090";
+  public static final String  DFS_NAMENODE_SECONDARY_HTTPS_PORT_KEY = "dfs.namenode.secondary.https-port";
+  public static final int     DFS_NAMENODE_SECONDARY_HTTPS_PORT_DEFAULT = 50490;
   public static final String  DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_KEY = "dfs.namenode.checkpoint.check.period";
   public static final long    DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_DEFAULT = 60;
   public static final String  DFS_NAMENODE_CHECKPOINT_PERIOD_KEY = "dfs.namenode.checkpoint.period";

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HdfsConfiguration.java

@@ -81,6 +81,7 @@ public class HdfsConfiguration extends Configuration {
     deprecate("dfs.safemode.extension", DFSConfigKeys.DFS_NAMENODE_SAFEMODE_EXTENSION_KEY);
     deprecate("dfs.safemode.threshold.pct", DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY);
     deprecate("dfs.secondary.http.address", DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY);
+    deprecate("dfs.secondary.https.port", DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTPS_PORT_KEY);
     deprecate("dfs.socket.timeout", DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY);
     deprecate("fs.checkpoint.dir", DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY);
     deprecate("fs.checkpoint.edits.dir", DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY);

+ 15 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java

@@ -51,11 +51,8 @@ import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 import org.apache.hadoop.io.IOUtils;
 
 /**
- * Performs two types of scanning:
- * <li> Gets block files from the data directories and reconciles the
- * difference between the blocks on the disk and in memory.</li>
- * <li> Scans the data directories for block files under a block pool
- * and verifies that the files are not corrupt</li>
+ * Scans the block files under a block pool and verifies that the
+ * files are not corrupt.
  * This keeps track of blocks and their last verification times.
  * Currently it does not modify the metadata for block.
  */
@@ -430,6 +427,19 @@ class BlockPoolSliceScanner {
           return;
         }
 
+        // If the block exists, the exception may due to a race with write:
+        // The BlockSender got an old block path in rbw. BlockReceiver removed
+        // the rbw block from rbw to finalized but BlockSender tried to open the
+        // file before BlockReceiver updated the VolumeMap. The state of the
+        // block can be changed again now, so ignore this error here. If there
+        // is a block really deleted by mistake, DirectoryScan should catch it.
+        if (e instanceof FileNotFoundException ) {
+          LOG.info("Verification failed for " + block +
+              ". It may be due to race with write.");
+          deleteBlock(block.getLocalBlock());
+          return;
+        }
+
         LOG.warn((second ? "Second " : "First ") + "Verification failed for "
             + block, e);
         

+ 14 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -1953,15 +1953,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     } finally {
       writeUnlock();
     }
-
-    getEditLog().logSync();
-
-    writeLock();
-    try {
-      removeBlocks(collectedBlocks); // Incremental deletion of blocks
-    } finally {
-      writeUnlock();
-    }
+    getEditLog().logSync(); 
+    removeBlocks(collectedBlocks); // Incremental deletion of blocks
     collectedBlocks.clear();
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* Namesystem.delete: "
@@ -1970,16 +1963,24 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return true;
   }
 
-  /** From the given list, incrementally remove the blocks from blockManager */
+  /** 
+   * From the given list, incrementally remove the blocks from blockManager
+   * Writelock is dropped and reacquired every BLOCK_DELETION_INCREMENT to
+   * ensure that other waiters on the lock can get in. See HDFS-2938
+   */
   private void removeBlocks(List<Block> blocks) {
-    assert hasWriteLock();
     int start = 0;
     int end = 0;
     while (start < blocks.size()) {
       end = BLOCK_DELETION_INCREMENT + start;
       end = end > blocks.size() ? blocks.size() : end;
-      for (int i=start; i<end; i++) {
-        blockManager.removeBlock(blocks.get(i));
+      writeLock();
+      try {
+        for (int i = start; i < end; i++) {
+          blockManager.removeBlock(blocks.get(i));
+        }
+      } finally {
+        writeUnlock();
       }
       start = end;
     }

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -117,6 +117,7 @@ public class NameNode {
     DFS_NAMENODE_HTTPS_ADDRESS_KEY,
     DFS_NAMENODE_KEYTAB_FILE_KEY,
     DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY,
+    DFS_NAMENODE_SECONDARY_HTTPS_PORT_KEY,
     DFS_SECONDARY_NAMENODE_KEYTAB_FILE_KEY,
     DFS_NAMENODE_BACKUP_ADDRESS_KEY,
     DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY,

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java

@@ -266,7 +266,8 @@ public class SecondaryNameNode implements Runnable {
                 Krb5AndCertsSslSocketConnector.KRB5_CIPHER_SUITES.get(0));
             InetSocketAddress secInfoSocAddr = 
               NetUtils.createSocketAddr(infoBindAddress + ":"+ conf.getInt(
-                "dfs.secondary.https.port", 443));
+                DFS_NAMENODE_SECONDARY_HTTPS_PORT_KEY,
+                DFS_NAMENODE_SECONDARY_HTTPS_PORT_DEFAULT));
             imagePort = secInfoSocAddr.getPort();
             infoServer.addSslListener(secInfoSocAddr, conf, false, true);
           }

+ 0 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAppendDifferentChecksum.java

@@ -47,12 +47,6 @@ public class TestAppendDifferentChecksum {
   public static void setupCluster() throws IOException {
     Configuration conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 4096);
-    
-    // disable block scanner, since otherwise this test can trigger
-    // HDFS-2525, which is a different bug than we're trying to unit test
-    // here! When HDFS-2525 is fixed, this can be removed.
-    conf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1);
-
     conf.set("fs.hdfs.impl.disable.cache", "true");
     cluster = new MiniDFSCluster.Builder(conf)
       .numDataNodes(1)

+ 46 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestShortCircuitLocalRead.java

@@ -238,7 +238,53 @@ public class TestShortCircuitLocalRead {
       cluster.shutdown();
     }
   }
+
+  @Test
+  public void testSkipWithVerifyChecksum() throws IOException {
+    int size = blockSize;
+    Configuration conf = new Configuration();
+    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
+    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, false);
+    conf.set(DFSConfigKeys.DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY,
+        UserGroupInformation.getCurrentUser().getShortUserName());
+    if (simulatedStorage) {
+      conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
+    }
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
+        .format(true).build();
+    FileSystem fs = cluster.getFileSystem();
+    try {
+      // check that / exists
+      Path path = new Path("/");
+      assertTrue("/ should be a directory", fs.getFileStatus(path)
+          .isDirectory() == true);
+      
+      byte[] fileData = AppendTestUtil.randomBytes(seed, size*3);
+      // create a new file in home directory. Do not close it.
+      Path file1 = new Path("filelocal.dat");
+      FSDataOutputStream stm = createFile(fs, file1, 1);
   
+      // write to file
+      stm.write(fileData);
+      stm.close();
+      
+      // now test the skip function
+      FSDataInputStream instm = fs.open(file1);
+      byte[] actual = new byte[fileData.length];
+      // read something from the block first, otherwise BlockReaderLocal.skip()
+      // will not be invoked
+      int nread = instm.read(actual, 0, 3);
+      long skipped = 2*size+3;
+      instm.seek(skipped);
+      nread = instm.read(actual, (int)(skipped + nread), 3);
+      instm.close();
+        
+    } finally {
+      fs.close();
+      cluster.shutdown();
+    }
+  }
+     
   /**
    * Test to run benchmarks between shortcircuit read vs regular read with
    * specified number of threads simultaneously reading.

+ 4 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLargeDirectoryDelete.java

@@ -117,8 +117,11 @@ public class TestLargeDirectoryDelete {
           try {
             int blockcount = getBlockCount();
             if (blockcount < TOTAL_BLOCKS && blockcount > 0) {
-              synchronized(mc.getNamesystem()) {
+              mc.getNamesystem().writeLock();
+              try {
                 lockOps++;
+              } finally {
+                mc.getNamesystem().writeUnlock();
               }
               Thread.sleep(1);
             }

+ 19 - 1
hadoop-mapreduce-project/CHANGES.txt

@@ -33,6 +33,11 @@ Release 0.23.2 - UNRELEASED
   NEW FEATURES
 
   IMPROVEMENTS
+    MAPREDUCE-3849. Change TokenCache's reading of the binary token file
+    (Daryn Sharp via bobby)
+
+    MAPREDUCE-3854. Fixed and reenabled tests related to MR child JVM's           
+    environmental variables in TestMiniMRChildTask. (Tom White via vinodkv)
 
   OPTIMIZATIONS
 
@@ -44,6 +49,13 @@ Release 0.23.2 - UNRELEASED
     MAPREDUCE-3852. Test TestLinuxResourceCalculatorPlugin failing. (Thomas 
     Graves via mahadev)
 
+    MAPREDUCE-3864. Fix cluster setup docs for correct SecondaryNameNode
+    HTTPS parameters. (todd)
+
+    MAPREDUCE-3856. Instances of RunningJob class givs incorrect job tracking
+    urls when mutiple jobs are submitted from same client jvm. (Eric Payne via
+    sseth)
+ 
 Release 0.23.1 - 2012-02-08 
 
   NEW FEATURES
@@ -72,7 +84,10 @@ Release 0.23.1 - 2012-02-08
 
     MAPREDUCE-3840.  JobEndNotifier doesn't use the proxyToUse during connecting
     (Ravi Prakash via bobby)
-    
+
+    MAPREDUCE-3736. Variable substitution depth too large for fs.default.name 
+    causes jobs to fail (ahmed via tucu).
+ 
     MAPREDUCE-2733. [Gridmix] Gridmix3 cpu emulation system tests. 
                     (Vinay Thota via amarrk)
 
@@ -789,6 +804,9 @@ Release 0.23.1 - 2012-02-08
     MAPREDUCE-3802. Added test to validate that AM can crash multiple times and
     still can recover successfully after MAPREDUCE-3846. (vinodkv)
 
+    MAPREDUCE-3858. Task attempt failure during commit results in task never completing.
+    (Tom White via mahadev)
+
 Release 0.23.0 - 2011-11-01 
 
   INCOMPATIBLE CHANGES

+ 4 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java

@@ -832,6 +832,9 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
     public TaskState transition(TaskImpl task, TaskEvent event) {
       task.failedAttempts++;
       TaskTAttemptEvent castEvent = (TaskTAttemptEvent) event;
+      if (castEvent.getTaskAttemptID().equals(task.commitAttempt)) {
+        task.commitAttempt = null;
+      }
       TaskAttempt attempt = task.attempts.get(castEvent.getTaskAttemptID());
       if (attempt.getAssignedContainerMgrAddress() != null) {
         //container was assigned
@@ -877,6 +880,7 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
 
     protected void unSucceed(TaskImpl task) {
       ++task.numberUncompletedAttempts;
+      task.commitAttempt = null;
       task.successfulAttempt = null;
     }
   }

+ 52 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java

@@ -18,6 +18,8 @@
 package org.apache.hadoop.mapreduce.v2.app.job.impl;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -261,6 +263,12 @@ public class TestTaskImpl {
     assertTaskRunningState();    
   }
   
+  private void commitTaskAttempt(TaskAttemptId attemptId) {
+    mockTask.handle(new TaskTAttemptEvent(attemptId, 
+        TaskEventType.T_ATTEMPT_COMMIT_PENDING));
+    assertTaskRunningState();    
+  }
+  
   private MockTaskAttemptImpl getLastAttempt() {
     return taskAttempts.get(taskAttempts.size()-1);
   }
@@ -279,32 +287,45 @@ public class TestTaskImpl {
     assertTaskRunningState();  
   }
   
+  private void failRunningTaskAttempt(TaskAttemptId attemptId) {
+    mockTask.handle(new TaskTAttemptEvent(attemptId, 
+        TaskEventType.T_ATTEMPT_FAILED));
+    assertTaskRunningState();
+  }
+  
   /**
    * {@link TaskState#NEW}
    */
   private void assertTaskNewState() {
-    assertEquals(mockTask.getState(), TaskState.NEW);
+    assertEquals(TaskState.NEW, mockTask.getState());
   }
   
   /**
    * {@link TaskState#SCHEDULED}
    */
   private void assertTaskScheduledState() {
-    assertEquals(mockTask.getState(), TaskState.SCHEDULED);
+    assertEquals(TaskState.SCHEDULED, mockTask.getState());
   }
 
   /**
    * {@link TaskState#RUNNING}
    */
   private void assertTaskRunningState() {
-    assertEquals(mockTask.getState(), TaskState.RUNNING);        
+    assertEquals(TaskState.RUNNING, mockTask.getState());
   }
     
   /**
    * {@link TaskState#KILL_WAIT}
    */
   private void assertTaskKillWaitState() {
-    assertEquals(mockTask.getState(), TaskState.KILL_WAIT);
+    assertEquals(TaskState.KILL_WAIT, mockTask.getState());
+  }
+  
+  /**
+   * {@link TaskState#SUCCEEDED}
+   */
+  private void assertTaskSucceededState() {
+    assertEquals(TaskState.SUCCEEDED, mockTask.getState());
   }
   
   @Test
@@ -409,5 +430,32 @@ public class TestTaskImpl {
     assert(mockTask.getProgress() == progress);
         
   }
+  
+  @Test
+  public void testFailureDuringTaskAttemptCommit() {
+    TaskId taskId = getNewTaskID();
+    scheduleTaskAttempt(taskId);
+    launchTaskAttempt(getLastAttempt().getAttemptId());
+    updateLastAttemptState(TaskAttemptState.COMMIT_PENDING);
+    commitTaskAttempt(getLastAttempt().getAttemptId());
+
+    // During the task attempt commit there is an exception which causes
+    // the attempt to fail
+    updateLastAttemptState(TaskAttemptState.FAILED);
+    failRunningTaskAttempt(getLastAttempt().getAttemptId());
+
+    assertEquals(2, taskAttempts.size());
+    updateLastAttemptState(TaskAttemptState.SUCCEEDED);
+    commitTaskAttempt(getLastAttempt().getAttemptId());
+    mockTask.handle(new TaskTAttemptEvent(getLastAttempt().getAttemptId(), 
+        TaskEventType.T_ATTEMPT_SUCCEEDED));
+    
+    assertFalse("First attempt should not commit",
+        mockTask.canCommit(taskAttempts.get(0).getAttemptId()));
+    assertTrue("Second attempt should commit",
+        mockTask.canCommit(getLastAttempt().getAttemptId()));
+
+    assertTaskSucceededState();
+  }
 
 }

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapred/TestMRWithDistributedCache.java

@@ -159,7 +159,7 @@ public class TestMRWithDistributedCache extends TestCase {
   public void testLocalJobRunner() throws Exception {
     Configuration c = new Configuration();
     c.set(JTConfig.JT_IPC_ADDRESS, "local");
-    c.set("fs.default.name", "file:///");
+    c.set("fs.defaultFS", "file:///");
     testWithConf(c);
   }
 

+ 17 - 22
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/security/TokenCache.java

@@ -114,31 +114,10 @@ public class TokenCache {
       throw new IOException(
           "Can't get Master Kerberos principal for use as renewer");
     }
-    boolean readFile = true;
+    mergeBinaryTokens(credentials, conf);
 
     String fsName = fs.getCanonicalServiceName();
     if (TokenCache.getDelegationToken(credentials, fsName) == null) {
-      //TODO: Need to come up with a better place to put
-      //this block of code to do with reading the file
-      if (readFile) {
-        readFile = false;
-        String binaryTokenFilename =
-          conf.get(MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY);
-        if (binaryTokenFilename != null) {
-          Credentials binary;
-          try {
-            binary = Credentials.readTokenStorageFile(
-                new Path("file:///" +  binaryTokenFilename), conf);
-          } catch (IOException e) {
-            throw new RuntimeException(e);
-          }
-          credentials.addAll(binary);
-        }
-        if (TokenCache.getDelegationToken(credentials, fsName) != null) {
-          LOG.debug("DT for " + fsName  + " is already present");
-          return;
-        }
-      }
       List<Token<?>> tokens =
           fs.getDelegationTokens(delegTokenRenewer, credentials);
       if (tokens != null) {
@@ -161,6 +140,22 @@ public class TokenCache {
     }
   }
 
+  private static void mergeBinaryTokens(Credentials creds, Configuration conf) {
+    String binaryTokenFilename =
+        conf.get(MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY);
+    if (binaryTokenFilename != null) {
+      Credentials binary;
+      try {
+        binary = Credentials.readTokenStorageFile(
+            new Path("file:///" +  binaryTokenFilename), conf);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+      // supplement existing tokens with the tokens in the binary file
+      creds.mergeAll(binary);
+    }
+  }
+  
   /**
    * file name used on HDFS for generated job token
    */

+ 89 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/security/TestTokenCache.java

@@ -21,23 +21,27 @@ package org.apache.hadoop.mapreduce.security;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+import java.io.IOException;
 import java.net.URI;
 import java.util.LinkedList;
 import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.Master;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
@@ -162,6 +166,91 @@ public class TestTokenCache {
     return mockFs;
   }
 
+  @Test
+  @SuppressWarnings("deprecation")
+  public void testBinaryCredentials() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(YarnConfiguration.RM_PRINCIPAL, "mapred/host@REALM");
+    String renewer = Master.getMasterPrincipal(conf);
+
+    Path TEST_ROOT_DIR =
+        new Path(System.getProperty("test.build.data","test/build/data"));
+    // ick, but need fq path minus file:/
+    String binaryTokenFile = FileSystem.getLocal(conf).makeQualified(
+        new Path(TEST_ROOT_DIR, "tokenFile")).toUri().getPath();
+
+    FileSystem fs1 = createFileSystemForService("service1");
+    FileSystem fs2 = createFileSystemForService("service2");
+    FileSystem fs3 = createFileSystemForService("service3");
+    
+    // get the tokens for fs1 & fs2 and write out to binary creds file
+    Credentials creds = new Credentials();
+    Token<?> token1 = fs1.getDelegationToken(renewer);
+    Token<?> token2 = fs2.getDelegationToken(renewer);
+    creds.addToken(token1.getService(), token1);
+    creds.addToken(token2.getService(), token2);
+    // wait to set, else the obtain tokens call above will fail with FNF
+    conf.set(MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY, binaryTokenFile);
+    creds.writeTokenStorageFile(new Path(binaryTokenFile), conf);
+    
+    // re-init creds and add a newer token for fs1
+    creds = new Credentials();
+    Token<?> newerToken1 = fs1.getDelegationToken(renewer);
+    assertFalse(newerToken1.equals(token1));
+    creds.addToken(newerToken1.getService(), newerToken1);
+    checkToken(creds, newerToken1);
+    
+    // get token for fs1, see that fs2's token was loaded 
+    TokenCache.obtainTokensForNamenodesInternal(fs1, creds, conf);
+    checkToken(creds, newerToken1, token2);
+    
+    // get token for fs2, nothing should change since already present
+    TokenCache.obtainTokensForNamenodesInternal(fs2, creds, conf);
+    checkToken(creds, newerToken1, token2);
+    
+    // get token for fs3, should only add token for fs3
+    TokenCache.obtainTokensForNamenodesInternal(fs3, creds, conf);
+    Token<?> token3 = creds.getToken(new Text(fs3.getCanonicalServiceName()));
+    assertTrue(token3 != null);
+    checkToken(creds, newerToken1, token2, token3);
+    
+    // be paranoid, check one last time that nothing changes
+    TokenCache.obtainTokensForNamenodesInternal(fs1, creds, conf);
+    TokenCache.obtainTokensForNamenodesInternal(fs2, creds, conf);
+    TokenCache.obtainTokensForNamenodesInternal(fs3, creds, conf);
+    checkToken(creds, newerToken1, token2, token3);
+  }
+
+  private void checkToken(Credentials creds, Token<?> ... tokens) {
+    assertEquals(tokens.length, creds.getAllTokens().size());
+    for (Token<?> token : tokens) {
+      Token<?> credsToken = creds.getToken(token.getService());
+      assertTrue(credsToken != null);
+      assertEquals(token, credsToken);
+    }
+  }
+  
+  @SuppressWarnings("deprecation")
+  private FileSystem createFileSystemForService(final String service)
+      throws IOException {
+    FileSystem mockFs = mock(FileSystem.class);
+    when(mockFs.getCanonicalServiceName()).thenReturn(service);
+    when(mockFs.getDelegationToken(any(String.class))).thenAnswer(
+        new Answer<Token<?>>() {
+          int unique = 0;
+          @Override
+          public Token<?> answer(InvocationOnMock invocation) throws Throwable {
+            Token<?> token = new Token<TokenIdentifier>();
+            token.setService(new Text(service));
+            // use unique value so when we restore from token storage, we can
+            // tell if it's really the same token
+            token.setKind(new Text("token" + unique++));
+            return token;
+          }
+        });
+    return mockFs;
+  }
+
   @Test
   public void testCleanUpTokenReferral() throws Exception {
     Configuration conf = new Configuration();

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java

@@ -81,8 +81,7 @@ public class ClientServiceDelegate {
   private static final Log LOG = LogFactory.getLog(ClientServiceDelegate.class);
 
   // Caches for per-user NotRunningJobs
-  private static HashMap<JobState, HashMap<String, NotRunningJob>> notRunningJobs =
-      new HashMap<JobState, HashMap<String, NotRunningJob>>();
+  private HashMap<JobState, HashMap<String, NotRunningJob>> notRunningJobs;
 
   private final Configuration conf;
   private final JobID jobId;
@@ -108,6 +107,7 @@ public class ClientServiceDelegate {
     this.jobId = jobId;
     this.historyServerProxy = historyServerProxy;
     this.appId = TypeConverter.toYarn(jobId).getAppId();
+    notRunningJobs = new HashMap<JobState, HashMap<String, NotRunningJob>>();
   }
 
   // Get the instance of the NotRunningJob corresponding to the specified

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java

@@ -59,7 +59,7 @@ public class TestNoDefaultsJobConf extends HadoopTestCase {
 
     JobConf conf = new JobConf(false);
 
-    conf.set("fs.default.name", createJobConf().get("fs.default.name"));
+    conf.set("fs.defaultFS", createJobConf().get("fs.defaultFS"));
 
     conf.setJobName("mr");
 
@@ -100,4 +100,4 @@ public class TestNoDefaultsJobConf extends HadoopTestCase {
 
   }
 
-}
+}

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/JHLogAnalyzer.java

@@ -1024,7 +1024,7 @@ public class JHLogAnalyzer {
       if(testFile != null) {
         LOG.info("Start JHLA test ============ ");
         LocalFileSystem lfs = FileSystem.getLocal(conf);
-        conf.set("fs.default.name", "file:///");
+        conf.set("fs.defaultFS", "file:///");
         JHLAMapper map = new JHLAMapper(conf);
         map.parseLogFile(lfs, new Path(testFile), 0L,
                          new LoggingCollector(), Reporter.NULL);

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/io/FileBench.java

@@ -53,7 +53,7 @@ public class FileBench extends Configured implements Tool {
 "unless they are also explicitly included, as in \"-pln -zip\"\n" +
 "Note that CompressionType params only apply to SequenceFiles\n\n" +
 "Useful options to set:\n" +
-"-D fs.default.name=\"file:///\" \\\n" +
+"-D fs.defaultFS=\"file:///\" \\\n" +
 "-D fs.file.impl=org.apache.hadoop.fs.RawLocalFileSystem \\\n" +
 "-D filebench.file.bytes=$((10*1024*1024*1024)) \\\n" +
 "-D filebench.key.words=5 \\\n" +

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineFileInputFormat.java

@@ -41,7 +41,7 @@ public class TestCombineFileInputFormat {
   private static FileSystem localFs = null; 
   static {
     try {
-      defaultConf.set("fs.default.name", "file:///");
+      defaultConf.set("fs.defaultFS", "file:///");
       localFs = FileSystem.getLocal(defaultConf);
     } catch (IOException e) {
       throw new RuntimeException("init failure", e);

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestConcatenatedCompressedInput.java

@@ -69,7 +69,7 @@ public class TestConcatenatedCompressedInput {
 
   static {
     try {
-      defaultConf.set("fs.default.name", "file:///");
+      defaultConf.set("fs.defaultFS", "file:///");
       localFs = FileSystem.getLocal(defaultConf);
     } catch (IOException e) {
       throw new RuntimeException("init failure", e);

+ 9 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRChildTask.java

@@ -219,8 +219,8 @@ public class TestMiniMRChildTask {
       
       // check if X=$X:/abc works for LD_LIBRARY_PATH
       checkEnv("LD_LIBRARY_PATH", "/tmp", "append");
-      // check if X=/tmp works for an already existing parameter
-      checkEnv("HOME", "/tmp", "noappend");
+      // check if X=y works for an already existing parameter
+      checkEnv("LANG", "en_us_8859_1", "noappend");
       // check if X=/tmp for a new env variable
       checkEnv("MY_PATH", "/tmp", "noappend");
       // check if X=$X:/tmp works for a new env var and results into :/tmp
@@ -269,8 +269,8 @@ public class TestMiniMRChildTask {
       
       // check if X=$X:/abc works for LD_LIBRARY_PATH
       checkEnv("LD_LIBRARY_PATH", "/tmp", "append");
-      // check if X=/tmp works for an already existing parameter
-      checkEnv("HOME", "/tmp", "noappend");
+      // check if X=y works for an already existing parameter
+      checkEnv("LANG", "en_us_8859_1", "noappend");
       // check if X=/tmp for a new env variable
       checkEnv("MY_PATH", "/tmp", "noappend");
       // check if X=$X:/tmp works for a new env var and results into :/tmp
@@ -369,7 +369,7 @@ public class TestMiniMRChildTask {
    *   - x=y (x can be a already existing env variable or a new variable)
    *   - x=$x:y (replace $x with the current value of x)
    */
-
+  @Test
   public void testTaskEnv(){
     try {
       JobConf conf = new JobConf(mr.getConfig());
@@ -392,6 +392,7 @@ public class TestMiniMRChildTask {
    *   - x=y (x can be a already existing env variable or a new variable)
    *   - x=$x:y (replace $x with the current value of x)
    */
+  @Test
   public void testTaskOldEnv(){
     try {
       JobConf conf = new JobConf(mr.getConfig());
@@ -415,7 +416,7 @@ public class TestMiniMRChildTask {
               EnvCheckMapper.class, EnvCheckReducer.class);
     // test 
     //  - new SET of new var (MY_PATH)
-    //  - set of old var (HOME)
+    //  - set of old var (LANG)
     //  - append to an old var from modified env (LD_LIBRARY_PATH)
     //  - append to an old var from tt's env (PATH)
     //  - append to a new var (NEW_PATH)
@@ -432,10 +433,10 @@ public class TestMiniMRChildTask {
       mapTaskJavaOpts = reduceTaskJavaOpts = TASK_OPTS_VAL;
     }
     conf.set(mapTaskEnvKey, 
-             "MY_PATH=/tmp,HOME=/tmp,LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/tmp," +
+             "MY_PATH=/tmp,LANG=en_us_8859_1,LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/tmp," +
              "PATH=$PATH:/tmp,NEW_PATH=$NEW_PATH:/tmp");
     conf.set(reduceTaskEnvKey, 
-             "MY_PATH=/tmp,HOME=/tmp,LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/tmp," +
+             "MY_PATH=/tmp,LANG=en_us_8859_1,LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/tmp," +
              "PATH=$PATH:/tmp,NEW_PATH=$NEW_PATH:/tmp");
     conf.set("path", System.getenv("PATH"));
     conf.set(mapTaskJavaOptsKey, mapTaskJavaOpts);

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTextInputFormat.java

@@ -55,7 +55,7 @@ public class TestTextInputFormat {
   private static FileSystem localFs = null; 
   static {
     try {
-      defaultConf.set("fs.default.name", "file:///");
+      defaultConf.set("fs.defaultFS", "file:///");
       localFs = FileSystem.getLocal(defaultConf);
     } catch (IOException e) {
       throw new RuntimeException("init failure", e);

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapCollection.java

@@ -314,7 +314,7 @@ public class TestMapCollection {
     job.setNumReduceTasks(1);
     job.getConfiguration().set(MRConfig.FRAMEWORK_NAME, MRConfig.LOCAL_FRAMEWORK_NAME);
     job.getConfiguration().setInt(MRJobConfig.IO_SORT_FACTOR, 1000);
-    job.getConfiguration().set("fs.default.name", "file:///");
+    job.getConfiguration().set("fs.defaultFS", "file:///");
     job.getConfiguration().setInt("test.mapcollection.num.maps", 1);
     job.setInputFormatClass(FakeIF.class);
     job.setOutputFormatClass(NullOutputFormat.class);

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestFileInputFormat.java

@@ -45,9 +45,9 @@ public class TestFileInputFormat {
   @Test
   public void testAddInputPath() throws IOException {
     final Configuration conf = new Configuration();
-    conf.set("fs.default.name", "s3://abc:xyz@hostname/");
+    conf.set("fs.defaultFS", "s3://abc:xyz@hostname/");
     final Job j = Job.getInstance(conf);
-    j.getConfiguration().set("fs.default.name", "s3://abc:xyz@hostname/");
+    j.getConfiguration().set("fs.defaultFS", "s3://abc:xyz@hostname/");
 
     //setup default fs
     final FileSystem defaultfs = FileSystem.get(conf);

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRKeyValueTextInputFormat.java

@@ -57,7 +57,7 @@ public class TestMRKeyValueTextInputFormat {
   private static FileSystem localFs = null; 
   static {
     try {
-      defaultConf.set("fs.default.name", "file:///");
+      defaultConf.set("fs.defaultFS", "file:///");
       localFs = FileSystem.getLocal(defaultConf);
     } catch (IOException e) {
       throw new RuntimeException("init failure", e);

+ 29 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java

@@ -148,10 +148,15 @@ public class TestMRJobs {
     job.addFileToClassPath(APP_JAR); // The AppMaster jar itself.
     job.setJarByClass(SleepJob.class);
     job.setMaxMapAttempts(1); // speed up failures
-    job.waitForCompletion(true);
+    job.submit();
+    String trackingUrl = job.getTrackingURL();
+    String jobId = job.getJobID().toString();
     boolean succeeded = job.waitForCompletion(true);
     Assert.assertTrue(succeeded);
     Assert.assertEquals(JobStatus.State.SUCCEEDED, job.getJobState());
+    Assert.assertTrue("Tracking URL was " + trackingUrl +
+                      " but didn't Match Job ID " + jobId ,
+          trackingUrl.endsWith(jobId.substring(jobId.lastIndexOf("_")) + "/"));
     verifySleepJobCounters(job);
     verifyTaskProgress(job);
     
@@ -209,9 +214,15 @@ public class TestMRJobs {
     job.addFileToClassPath(APP_JAR); // The AppMaster jar itself.
     job.setJarByClass(RandomTextWriterJob.class);
     job.setMaxMapAttempts(1); // speed up failures
+    job.submit();
+    String trackingUrl = job.getTrackingURL();
+    String jobId = job.getJobID().toString();
     boolean succeeded = job.waitForCompletion(true);
     Assert.assertTrue(succeeded);
     Assert.assertEquals(JobStatus.State.SUCCEEDED, job.getJobState());
+    Assert.assertTrue("Tracking URL was " + trackingUrl +
+                      " but didn't Match Job ID " + jobId ,
+          trackingUrl.endsWith(jobId.substring(jobId.lastIndexOf("_")) + "/"));
     
     // Make sure there are three files in the output-dir
     
@@ -316,9 +327,14 @@ public class TestMRJobs {
         new Path(mrCluster.getTestWorkDir().getAbsolutePath(),
         "failmapper-output"));
     job.addFileToClassPath(APP_JAR); // The AppMaster jar itself.
+    job.submit();
+    String trackingUrl = job.getTrackingURL();
+    String jobId = job.getJobID().toString();
     boolean succeeded = job.waitForCompletion(true);
     Assert.assertFalse(succeeded);
-
+    Assert.assertTrue("Tracking URL was " + trackingUrl +
+                      " but didn't Match Job ID " + jobId ,
+          trackingUrl.endsWith(jobId.substring(jobId.lastIndexOf("_")) + "/"));
     return job;
   }
 
@@ -360,8 +376,14 @@ public class TestMRJobs {
         // //Job with reduces
         // Job job = sleepJob.createJob(3, 2, 10000, 1, 10000, 1);
         job.addFileToClassPath(APP_JAR); // The AppMaster jar itself.
+        job.submit();
+        String trackingUrl = job.getTrackingURL();
+        String jobId = job.getJobID().toString();
         job.waitForCompletion(true);
         Assert.assertEquals(JobStatus.State.SUCCEEDED, job.getJobState());
+        Assert.assertTrue("Tracking URL was " + trackingUrl +
+                          " but didn't Match Job ID " + jobId ,
+          trackingUrl.endsWith(jobId.substring(jobId.lastIndexOf("_")) + "/"));
         return null;
       }
     });
@@ -444,7 +466,12 @@ public class TestMRJobs {
     job.setMaxMapAttempts(1); // speed up failures
 
     job.submit();
+    String trackingUrl = job.getTrackingURL();
+    String jobId = job.getJobID().toString();
     Assert.assertTrue(job.waitForCompletion(false));
+    Assert.assertTrue("Tracking URL was " + trackingUrl +
+                      " but didn't Match Job ID " + jobId ,
+          trackingUrl.endsWith(jobId.substring(jobId.lastIndexOf("_")) + "/"));
   }
 
   private Path createTempFile(String filename, String contents)

+ 1 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml

@@ -457,7 +457,7 @@
 
   <property>
     <name>mapreduce.job.hdfs-servers</name>
-    <value>${fs.default.name}</value>
+    <value>${fs.defaultFS}</value>
   </property>
 
   <!-- WebAppProxy Configuration-->

+ 2 - 2
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ClusterSetup.apt.vm

@@ -738,9 +738,9 @@ KVNO Timestamp         Principal
 *-------------------------+-------------------------+------------------------+
 || Parameter              || Value                  || Notes                 |
 *-------------------------+-------------------------+------------------------+
-| <<<dfs.namenode.secondary.https-address>>> | <c_nn_host_fqdn:50090> | |
+| <<<dfs.namenode.secondary.http-address>>> | <c_nn_host_fqdn:50090> | |
 *-------------------------+-------------------------+------------------------+
-| <<<dfs.secondary.https.port>>> | <50090> | |
+| <<<dfs.namenode.secondary.https-port>>> | <50470> | |
 *-------------------------+-------------------------+------------------------+
 | <<<dfs.namenode.secondary.keytab.file>>> | | | 
 | | </etc/security/keytab/sn.service.keytab> | |

+ 19 - 0
hadoop-project/pom.xml

@@ -239,6 +239,17 @@
         <version>${project.version}</version>
       </dependency>
 
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-client</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-minicluster</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+
       <dependency>
         <groupId>com.google.guava</groupId>
         <artifactId>guava</artifactId>
@@ -733,6 +744,11 @@
     </pluginManagement>
 
     <plugins>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>findbugs-maven-plugin</artifactId>
+      </plugin>
+      
        <plugin>
          <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-antrun-plugin</artifactId>
@@ -954,6 +970,9 @@
                 <goals>
                   <goal>jar</goal>
                 </goals>
+                <configuration>
+                  <destDir>${project.build.directory}</destDir>
+                </configuration>
               </execution>
             </executions>
           </plugin>