瀏覽代碼

Merge r1462698 through r1464807 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2802@1464815 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 12 年之前
父節點
當前提交
07d7d92179
共有 97 個文件被更改,包括 5603 次插入539 次删除
  1. 8 2
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 4 20
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
  3. 0 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Path.java
  4. 65 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CodecPool.java
  5. 6 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopologyWithNodeGroup.java
  6. 1 1
      hadoop-common-project/hadoop-common/src/site/apt/ClusterSetup.apt.vm
  7. 6 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
  8. 0 36
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java
  9. 70 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/TestCodecPool.java
  10. 38 13
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetworkTopologyWithNodeGroup.java
  11. 18 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  12. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  13. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
  14. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
  15. 33 16
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
  16. 2 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  17. 18 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java
  18. 50 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
  19. 28 0
      hadoop-mapreduce-project/CHANGES.txt
  20. 22 10
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
  21. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/JobStateInternal.java
  22. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/JobEventType.java
  23. 82 11
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
  24. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/local/LocalContainerAllocator.java
  25. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
  26. 99 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestStagingCleanup.java
  27. 62 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java
  28. 10 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/impl/pb/client/MRClientProtocolPBClientImpl.java
  29. 0 14
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java
  30. 5 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
  31. 27 21
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
  32. 102 17
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java
  33. 33 0
      hadoop-tools/hadoop-gridmix/dev-support/findbugs-exclude.xml
  34. 24 0
      hadoop-tools/hadoop-gridmix/pom.xml
  35. 8 2
      hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/Gridmix.java
  36. 7 13
      hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/JobMonitor.java
  37. 4 0
      hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/SerialJobFactory.java
  38. 4 4
      hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/SleepJob.java
  39. 3 2
      hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/emulators/resourceusage/TotalHeapUsageEmulatorPlugin.java
  40. 384 0
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/CommonJobTest.java
  41. 2 2
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/DebugJobFactory.java
  42. 2 2
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/DebugJobProducer.java
  43. 59 36
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/GridmixTestUtils.java
  44. 430 0
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestDistCacheEmulation.java
  45. 989 0
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestGridMixClasses.java
  46. 202 0
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestGridmixSubmission.java
  47. 5 6
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestGridmixSummary.java
  48. 81 0
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestLoadJob.java
  49. 142 0
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestSleepJob.java
  50. 414 0
      hadoop-tools/hadoop-gridmix/src/test/resources/data/wordcount.json
  51. 828 0
      hadoop-tools/hadoop-gridmix/src/test/resources/data/wordcount2.json
  52. 2 2
      hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamJob.java
  53. 12 0
      hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestStreamingOutputKeyValueTypes.java
  54. 0 1
      hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TrApp.java
  55. 40 3
      hadoop-yarn-project/CHANGES.txt
  56. 14 0
      hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
  57. 0 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java
  58. 0 24
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java
  59. 1 79
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerPBImpl.java
  60. 1 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
  61. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
  62. 5 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/main/java/org/apache/hadoop/yarn/applications/unmanagedamlauncher/UnmanagedAMLauncher.java
  63. 9 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/RMAdminProtocolPBClientImpl.java
  64. 18 16
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  65. 0 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/BuilderUtils.java
  66. 45 14
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
  67. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestYarnVersionInfo.java
  68. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java
  69. 12 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java
  70. 58 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/YarnServerBuilderUtils.java
  71. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
  72. 5 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
  73. 30 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
  74. 11 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/api/impl/pb/client/LocalizationProtocolPBClientImpl.java
  75. 151 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalCacheDirectoryManager.java
  76. 6 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTracker.java
  77. 128 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java
  78. 41 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
  79. 4 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/MockNodeStatusUpdater.java
  80. 251 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
  81. 112 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalCacheDirectoryManager.java
  82. 117 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalResourcesTrackerImpl.java
  83. 3 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceRetention.java
  84. 15 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
  85. 5 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
  86. 1 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
  87. 1 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
  88. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
  89. 3 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java
  90. 13 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java
  91. 3 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestFifoScheduler.java
  92. 23 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
  93. 9 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
  94. 0 22
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java
  95. 36 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
  96. 2 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
  97. 23 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/FairScheduler.apt.vm

+ 8 - 2
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -165,6 +165,9 @@ Trunk (Unreleased)
 
   BUG FIXES
 
+    HADOOP-9451. Fault single-layer config if node group topology is enabled.
+    (Junping Du via llu)
+
     HADOOP-8419. Fixed GzipCode NPE reset for IBM JDK. (Yu Li via eyang)
 
     HADOOP-9041. FsUrlStreamHandlerFactory could cause an infinite loop in
@@ -536,6 +539,9 @@ Release 2.0.5-beta - UNRELEASED
 
     HADOOP-9358. "Auth failed" log should include exception string (todd)
 
+    HADOOP-9401. CodecPool: Add counters for number of (de)compressors 
+    leased out. (kkambatl via tucu)
+
   OPTIMIZATIONS
 
     HADOOP-9150. Avoid unnecessary DNS resolution attempts for logical URIs
@@ -604,8 +610,8 @@ Release 2.0.5-beta - UNRELEASED
     HADOOP-9125. LdapGroupsMapping threw CommunicationException after some
     idle time. (Kai Zheng via atm)
 
-    HADOOP-9357. Fallback to default authority if not specified in FileContext.
-    (Andrew Wang via eli)
+    HADOOP-9429. TestConfiguration fails with IBM JAVA. (Amir Sanjar via
+    suresh)
 
 Release 2.0.4-alpha - UNRELEASED
 

+ 4 - 20
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java

@@ -244,33 +244,17 @@ public final class FileContext {
   }
  
   /* 
-   * Resolve a relative path passed from the user.
-   * 
-   * Relative paths are resolved against the current working directory
-   * (e.g. "foo/bar" becomes "/<workingDir>/foo/bar").
-   * Fully-qualified URIs (e.g. "hdfs://nn:p/foo/bar") and slash-relative paths
+   * Remove relative part - return "absolute":
+   * If input is relative path ("foo/bar") add wd: ie "/<workingDir>/foo/bar"
+   * A fully qualified uri ("hdfs://nn:p/foo/bar") or a slash-relative path
    * ("/foo/bar") are returned unchanged.
    * 
-   * Additionally, we fix malformed URIs that specify a scheme but not an 
-   * authority (e.g. "hdfs:///foo/bar"). Per RFC 2395, we remove the scheme
-   * if it matches the default FS, and let the default FS add in the default
-   * scheme and authority later (see {@link #AbstractFileSystem#checkPath}).
-   * 
    * Applications that use FileContext should use #makeQualified() since
-   * they really want a fully-qualified URI.
+   * they really want a fully qualified URI.
    * Hence this method is not called makeAbsolute() and 
    * has been deliberately declared private.
    */
   private Path fixRelativePart(Path p) {
-    // Per RFC 2396 5.2, drop schema if there is a scheme but no authority.
-    if (p.hasSchemeAndNoAuthority()) {
-      String scheme = p.toUri().getScheme();
-      if (scheme.equalsIgnoreCase(defaultFS.getUri().getScheme())) {
-        p = new Path(p.toUri().getSchemeSpecificPart());
-      }
-    }
-    // Absolute paths are unchanged. Relative paths are resolved against the
-    // current working directory.
     if (p.isUriPathAbsolute()) {
       return p;
     } else {

+ 0 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Path.java

@@ -256,10 +256,6 @@ public class Path implements Comparable {
     return  (isUriPathAbsolute() && 
         uri.getScheme() == null && uri.getAuthority() == null);
   }
-
-  public boolean hasSchemeAndNoAuthority() {
-    return uri.getScheme() != null && uri.getAuthority() == null;
-  }
   
   /**
    *  True if the path component (i.e. directory) of this URI is absolute.

+ 65 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CodecPool.java

@@ -21,6 +21,7 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -29,6 +30,10 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.ReflectionUtils;
 
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
 /**
  * A global compressor/decompressor pool used to save and reuse 
  * (possibly native) compression/decompression codecs.
@@ -52,6 +57,29 @@ public class CodecPool {
   private static final Map<Class<Decompressor>, List<Decompressor>> decompressorPool = 
     new HashMap<Class<Decompressor>, List<Decompressor>>();
 
+  private static <T> LoadingCache<Class<T>, AtomicInteger> createCache(
+      Class<T> klass) {
+    return CacheBuilder.newBuilder().build(
+        new CacheLoader<Class<T>, AtomicInteger>() {
+          @Override
+          public AtomicInteger load(Class<T> key) throws Exception {
+            return new AtomicInteger();
+          }
+        });
+  }
+
+  /**
+   * Map to track the number of leased compressors
+   */
+  private static final LoadingCache<Class<Compressor>, AtomicInteger> compressorCounts =
+      createCache(Compressor.class);
+
+   /**
+   * Map to tracks the number of leased decompressors
+   */
+  private static final LoadingCache<Class<Decompressor>, AtomicInteger> decompressorCounts =
+      createCache(Decompressor.class);
+
   private static <T> T borrow(Map<Class<T>, List<T>> pool,
                              Class<? extends T> codecClass) {
     T codec = null;
@@ -90,6 +118,21 @@ public class CodecPool {
     }
   }
   
+  @SuppressWarnings("unchecked")
+  private static <T> int getLeaseCount(
+      LoadingCache<Class<T>, AtomicInteger> usageCounts,
+      Class<? extends T> codecClass) {
+    return usageCounts.getUnchecked((Class<T>) codecClass).get();
+  }
+
+  private static <T> void updateLeaseCount(
+      LoadingCache<Class<T>, AtomicInteger> usageCounts, T codec, int delta) {
+    if (codec != null) {
+      Class<T> codecClass = ReflectionUtils.getClass(codec);
+      usageCounts.getUnchecked(codecClass).addAndGet(delta);
+    }
+  }
+
   /**
    * Get a {@link Compressor} for the given {@link CompressionCodec} from the 
    * pool or a new one.
@@ -111,6 +154,7 @@ public class CodecPool {
         LOG.debug("Got recycled compressor");
       }
     }
+    updateLeaseCount(compressorCounts, compressor, 1);
     return compressor;
   }
   
@@ -137,6 +181,7 @@ public class CodecPool {
         LOG.debug("Got recycled decompressor");
       }
     }
+    updateLeaseCount(decompressorCounts, decompressor, 1);
     return decompressor;
   }
   
@@ -155,6 +200,7 @@ public class CodecPool {
     }
     compressor.reset();
     payback(compressorPool, compressor);
+    updateLeaseCount(compressorCounts, compressor, -1);
   }
   
   /**
@@ -173,5 +219,24 @@ public class CodecPool {
     }
     decompressor.reset();
     payback(decompressorPool, decompressor);
+    updateLeaseCount(decompressorCounts, decompressor, -1);
+  }
+
+  /**
+   * Return the number of leased {@link Compressor}s for this
+   * {@link CompressionCodec}
+   */
+  public static int getLeasedCompressorsCount(CompressionCodec codec) {
+    return (codec == null) ? 0 : getLeaseCount(compressorCounts,
+        codec.getCompressorType());
+  }
+
+  /**
+   * Return the number of leased {@link Decompressor}s for this
+   * {@link CompressionCodec}
+   */
+  public static int getLeasedDecompressorsCount(CompressionCodec codec) {
+    return (codec == null) ? 0 : getLeaseCount(decompressorCounts,
+        codec.getDecompressorType());
   }
 }

+ 6 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopologyWithNodeGroup.java

@@ -191,7 +191,12 @@ public class NetworkTopologyWithNodeGroup extends NetworkTopology {
       }
       rack = getNode(nodeGroup.getNetworkLocation());
 
-      if (rack != null && !(rack instanceof InnerNode)) {
+      // rack should be an innerNode and with parent.
+      // note: rack's null parent case is: node's topology only has one layer, 
+      //       so rack is recognized as "/" and no parent. 
+      // This will be recognized as a node with fault topology.
+      if (rack != null && 
+          (!(rack instanceof InnerNode) || rack.getParent() == null)) {
         throw new IllegalArgumentException("Unexpected data node " 
             + node.toString() 
             + " at an illegal network location");

+ 1 - 1
hadoop-common-project/hadoop-common/src/site/apt/ClusterSetup.apt.vm

@@ -249,7 +249,7 @@ Hadoop MapReduce Next Generation - Cluster Setup
 *-------------------------+-------------------------+------------------------+
 | <<<yarn.resourcemanager.scheduler.class>>> | | |
 | | <<<ResourceManager>>> Scheduler class. | |
-| | | <<<CapacityScheduler>>> (recommended) or <<<FifoScheduler>>> |
+| | | <<<CapacityScheduler>>> (recommended), <<<FairScheduler>>> (also recommended), or <<<FifoScheduler>>> |
 *-------------------------+-------------------------+------------------------+
 | <<<yarn.scheduler.minimum-allocation-mb>>> | | |
 | | Minimum limit of memory to allocate to each container request at the <<<Resource Manager>>>. | |

+ 6 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java

@@ -51,6 +51,10 @@ public class TestConfiguration extends TestCase {
   final static String CONFIG = new File("./test-config.xml").getAbsolutePath();
   final static String CONFIG2 = new File("./test-config2.xml").getAbsolutePath();
   final static Random RAN = new Random();
+  final static boolean IBMJAVA = System.getProperty("java.vendor").contains("IBM"); 
+  final static String XMLHEADER = 
+            IBMJAVA?"<?xml version=\"1.0\" encoding=\"UTF-8\"?><configuration>":
+  "<?xml version=\"1.0\" encoding=\"UTF-8\" standalone=\"no\"?><configuration>";
 
   @Override
   protected void setUp() throws Exception {
@@ -327,8 +331,8 @@ public class TestConfiguration extends TestCase {
     ByteArrayOutputStream baos = new ByteArrayOutputStream(); 
     conf.writeXml(baos);
     String result = baos.toString();
-    assertTrue("Result has proper header", result.startsWith(
-        "<?xml version=\"1.0\" encoding=\"UTF-8\" standalone=\"no\"?><configuration>"));
+    assertTrue("Result has proper header", result.startsWith(XMLHEADER));
+	  
     assertTrue("Result has proper footer", result.endsWith("</configuration>"));
   }
   

+ 0 - 36
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java

@@ -21,8 +21,6 @@ package org.apache.hadoop.fs;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
 import java.util.EnumSet;
 
 import org.apache.hadoop.HadoopIllegalArgumentException;
@@ -1166,40 +1164,6 @@ public abstract class FileContextMainOperationsBaseTest  {
       Assert.assertEquals(fc.getFileStatus(file), fc.getFileLinkStatus(file));
     }
   }
-
-  /**
-   * Test that URIs with a scheme, no authority, and absolute path component
-   * resolve with the authority of the default FS.
-   */
-  @Test(timeout=30000)
-  public void testAbsolutePathSchemeNoAuthority() throws IOException,
-      URISyntaxException {
-    Path file = getTestRootPath(fc, "test/file");
-    createFile(file);
-    URI uri = file.toUri();
-    URI noAuthorityUri = new URI(uri.getScheme(), null, uri.getPath(),
-        uri.getQuery(), uri.getFragment());
-    Path noAuthority = new Path(noAuthorityUri);
-    Assert.assertEquals(fc.getFileStatus(file), fc.getFileStatus(noAuthority));
-  }
-
-  /**
-   * Test that URIs with a scheme, no authority, and relative path component
-   * resolve with the authority of the default FS.
-   */
-  @Test(timeout=30000)
-  public void testRelativePathSchemeNoAuthority() throws IOException,
-      URISyntaxException {
-    Path workDir = new Path(getAbsoluteTestRootPath(fc), new Path("test"));
-    fc.setWorkingDirectory(workDir);
-    Path file = new Path(workDir, "file");
-    createFile(file);
-    URI uri = file.toUri();
-    URI noAuthorityUri = new URI(uri.getScheme() + ":file");
-    System.out.println(noAuthorityUri);
-    Path noAuthority = new Path(noAuthorityUri);
-    Assert.assertEquals(fc.getFileStatus(file), fc.getFileStatus(noAuthority));
-  }
   
   protected void createFile(Path path) throws IOException {
     FSDataOutputStream out = fc.create(path, EnumSet.of(CREATE),

+ 70 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/TestCodecPool.java

@@ -0,0 +1,70 @@
+/**
+ * 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.io.compress;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestCodecPool {
+  private final String LEASE_COUNT_ERR =
+      "Incorrect number of leased (de)compressors";
+  DefaultCodec codec;
+
+  @Before
+  public void setup() {
+    this.codec = new DefaultCodec();
+    this.codec.setConf(new Configuration());
+  }
+
+  @Test(timeout = 1000)
+  public void testCompressorPoolCounts() {
+    // Get two compressors and return them
+    Compressor comp1 = CodecPool.getCompressor(codec);
+    Compressor comp2 = CodecPool.getCompressor(codec);
+    assertEquals(LEASE_COUNT_ERR, 2,
+        CodecPool.getLeasedCompressorsCount(codec));
+
+    CodecPool.returnCompressor(comp2);
+    assertEquals(LEASE_COUNT_ERR, 1,
+        CodecPool.getLeasedCompressorsCount(codec));
+
+    CodecPool.returnCompressor(comp1);
+    assertEquals(LEASE_COUNT_ERR, 0,
+        CodecPool.getLeasedCompressorsCount(codec));
+  }
+
+  @Test(timeout = 1000)
+  public void testDecompressorPoolCounts() {
+    // Get two decompressors and return them
+    Decompressor decomp1 = CodecPool.getDecompressor(codec);
+    Decompressor decomp2 = CodecPool.getDecompressor(codec);
+    assertEquals(LEASE_COUNT_ERR, 2,
+        CodecPool.getLeasedDecompressorsCount(codec));
+
+    CodecPool.returnDecompressor(decomp2);
+    assertEquals(LEASE_COUNT_ERR, 1,
+        CodecPool.getLeasedDecompressorsCount(codec));
+
+    CodecPool.returnDecompressor(decomp1);
+    assertEquals(LEASE_COUNT_ERR, 0,
+        CodecPool.getLeasedDecompressorsCount(codec));
+  }
+}

+ 38 - 13
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopologyWithNodeGroup.java → hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetworkTopologyWithNodeGroup.java

@@ -20,30 +20,31 @@ package org.apache.hadoop.net;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.junit.Test;
 
 public class TestNetworkTopologyWithNodeGroup {
   private final static NetworkTopologyWithNodeGroup cluster = new 
       NetworkTopologyWithNodeGroup();
 
-  private final static DatanodeDescriptor dataNodes[] = new DatanodeDescriptor[] {
-      DFSTestUtil.getDatanodeDescriptor("1.1.1.1", "/d1/r1/s1"),
-      DFSTestUtil.getDatanodeDescriptor("2.2.2.2", "/d1/r1/s1"),
-      DFSTestUtil.getDatanodeDescriptor("3.3.3.3", "/d1/r1/s2"),
-      DFSTestUtil.getDatanodeDescriptor("4.4.4.4", "/d1/r2/s3"),
-      DFSTestUtil.getDatanodeDescriptor("5.5.5.5", "/d1/r2/s3"),
-      DFSTestUtil.getDatanodeDescriptor("6.6.6.6", "/d1/r2/s4"),
-      DFSTestUtil.getDatanodeDescriptor("7.7.7.7", "/d2/r3/s5"),
-      DFSTestUtil.getDatanodeDescriptor("8.8.8.8", "/d2/r3/s6")
+  private final static NodeBase dataNodes[] = new NodeBase[] {
+      new NodeBase("h1", "/d1/r1/s1"),
+      new NodeBase("h2", "/d1/r1/s1"),
+      new NodeBase("h3", "/d1/r1/s2"),
+      new NodeBase("h4", "/d1/r2/s3"),
+      new NodeBase("h5", "/d1/r2/s3"),
+      new NodeBase("h6", "/d1/r2/s4"),
+      new NodeBase("h7", "/d2/r3/s5"),
+      new NodeBase("h8", "/d2/r3/s6")
   };
 
   private final static NodeBase computeNode = new NodeBase("/d1/r1/s1/h9");
+  
+  private final static NodeBase rackOnlyNode = new NodeBase("h10", "/r2");
 
   static {
     for(int i=0; i<dataNodes.length; i++) {
@@ -96,7 +97,7 @@ public class TestNetworkTopologyWithNodeGroup {
 
   @Test
   public void testPseudoSortByDistance() throws Exception {
-    DatanodeDescriptor[] testNodes = new DatanodeDescriptor[4];
+    NodeBase[] testNodes = new NodeBase[4];
 
     // array contains both local node, local node group & local rack node
     testNodes[0] = dataNodes[1];
@@ -147,7 +148,7 @@ public class TestNetworkTopologyWithNodeGroup {
   private Map<Node, Integer> pickNodesAtRandom(int numNodes,
       String excludedScope) {
     Map<Node, Integer> frequency = new HashMap<Node, Integer>();
-    for (DatanodeDescriptor dnd : dataNodes) {
+    for (NodeBase dnd : dataNodes) {
       frequency.put(dnd, 0);
     }
 
@@ -161,6 +162,12 @@ public class TestNetworkTopologyWithNodeGroup {
   /**
    * This test checks that chooseRandom works for an excluded node.
    */
+  /**
+   * Test replica placement policy in case last node is invalid.
+   * We create 6 nodes but the last node is in fault topology (with rack info),
+   * so cannot be added to cluster. We should test proper exception is thrown in 
+   * adding node but shouldn't affect the cluster.
+   */
   @Test
   public void testChooseRandomExcludedNode() {
     String scope = "~" + NodeBase.getPath(dataNodes[0]);
@@ -171,5 +178,23 @@ public class TestNetworkTopologyWithNodeGroup {
       assertTrue(frequency.get(key) > 0 || key == dataNodes[0]);
     }
   }
+  
+  /**
+   * This test checks that adding a node with invalid topology will be failed 
+   * with an exception to show topology is invalid.
+   */
+  @Test
+  public void testAddNodeWithInvalidTopology() {
+    // The last node is a node with invalid topology
+    try {
+      cluster.add(rackOnlyNode);
+      fail("Exception should be thrown, so we should not have reached here.");
+    } catch (Exception e) {
+      if (!(e instanceof IllegalArgumentException)) {
+        fail("Expecting IllegalArgumentException, but caught:" + e);
+      }
+      assertTrue(e.getMessage().contains("illegal network location"));
+    }
+  }
 
 }

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

@@ -28,6 +28,9 @@ Trunk (Unreleased)
 
   IMPROVEMENTS
 
+    HDFS-4665. Move TestNetworkTopologyWithNodeGroup to common.
+    (Junping Du via llu)
+
     HDFS-1620. Rename HdfsConstants -> HdfsServerConstants, FSConstants ->
                HdfsConstants. (Harsh J Chouraria via atm)
 
@@ -343,6 +346,9 @@ Trunk (Unreleased)
 
     HDFS-4604. TestJournalNode fails on Windows. (Ivan Mitic via suresh)
 
+    HDFS-4625. Make TestNNWithQJM#testNewNamenodeTakesOverWriter work on
+    Windows. (Ivan Mitic via suresh)
+
 Release 2.0.5-beta - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -460,6 +466,14 @@ Release 2.0.5-beta - UNRELEASED
     HDFS-4598. Fix the default value of ConcatSourcesParam and the WebHDFS doc.
     (szetszwo)
 
+    HDFS-4655. DNA_FINALIZE is logged as being an unknown command by the DN
+    when received from the standby NN. (atm)
+
+    HDFS-4656. DN heartbeat loop can be briefly tight. (atm)
+
+    HDFS-4658. Standby NN will log that it has received a block report "after
+    becoming active" (atm)
+
 Release 2.0.4-alpha - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -2477,6 +2491,10 @@ Release 0.23.7 - UNRELEASED
     HDFS-4581. checkDiskError should not be called on network errors (Rohit
     Kochar via kihwal)
 
+    HDFS-4649. Webhdfs cannot list large directories (daryn via kihwal)
+
+    HDFS-4548. Webhdfs doesn't renegotiate SPNEGO token (daryn via kihwal)
+
 Release 0.23.6 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -1569,8 +1569,8 @@ public class BlockManager {
       node.receivedBlockReport();
       if (staleBefore && !node.areBlockContentsStale()) {
         LOG.info("BLOCK* processReport: Received first block report from "
-            + node + " after becoming active. Its block contents are no longer"
-            + " considered stale");
+            + node + " after starting up or becoming active. Its block "
+            + "contents are no longer considered stale");
         rescanPostponedMisreplicatedBlocks();
       }
       

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java

@@ -612,6 +612,7 @@ class BPOfferService {
     case DatanodeProtocol.DNA_TRANSFER:
     case DatanodeProtocol.DNA_INVALIDATE:
     case DatanodeProtocol.DNA_SHUTDOWN:
+    case DatanodeProtocol.DNA_FINALIZE:
     case DatanodeProtocol.DNA_RECOVERBLOCK:
     case DatanodeProtocol.DNA_BALANCERBANDWIDTHUPDATE:
       LOG.warn("Got a command from standby NN - ignoring command:" + cmd.getAction());

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java

@@ -512,7 +512,7 @@ class BPServiceActor implements Runnable {
         //
         // Every so often, send heartbeat or block-report
         //
-        if (startTime - lastHeartbeat > dnConf.heartBeatInterval) {
+        if (startTime - lastHeartbeat >= dnConf.heartBeatInterval) {
           //
           // All heartbeat messages include following info:
           // -- Datanode name

+ 33 - 16
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java

@@ -718,9 +718,15 @@ public class NamenodeWebHdfsMethods {
   
   private static StreamingOutput getListingStream(final NamenodeProtocols np, 
       final String p) throws IOException {
-    final DirectoryListing first = getDirectoryListing(np, p,
+    // allows exceptions like FNF or ACE to prevent http response of 200 for
+    // a failure since we can't (currently) return error responses in the
+    // middle of a streaming operation
+    final DirectoryListing firstDirList = getDirectoryListing(np, p,
         HdfsFileStatus.EMPTY_NAME);
 
+    // must save ugi because the streaming object will be executed outside
+    // the remote user's ugi
+    final UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
     return new StreamingOutput() {
       @Override
       public void write(final OutputStream outstream) throws IOException {
@@ -729,21 +735,32 @@ public class NamenodeWebHdfsMethods {
         out.println("{\"" + FileStatus.class.getSimpleName() + "es\":{\""
             + FileStatus.class.getSimpleName() + "\":[");
 
-        final HdfsFileStatus[] partial = first.getPartialListing();
-        if (partial.length > 0) {
-          out.print(JsonUtil.toJsonString(partial[0], false));
-        }
-        for(int i = 1; i < partial.length; i++) {
-          out.println(',');
-          out.print(JsonUtil.toJsonString(partial[i], false));
-        }
-
-        for(DirectoryListing curr = first; curr.hasMore(); ) { 
-          curr = getDirectoryListing(np, p, curr.getLastName());
-          for(HdfsFileStatus s : curr.getPartialListing()) {
-            out.println(',');
-            out.print(JsonUtil.toJsonString(s, false));
-          }
+        try {
+          // restore remote user's ugi
+          ugi.doAs(new PrivilegedExceptionAction<Void>() {
+            @Override
+            public Void run() throws IOException {
+              long n = 0;
+              for (DirectoryListing dirList = firstDirList; ;
+                   dirList = getDirectoryListing(np, p, dirList.getLastName())
+              ) {
+                // send each segment of the directory listing
+                for (HdfsFileStatus s : dirList.getPartialListing()) {
+                  if (n++ > 0) {
+                    out.println(',');
+                  }
+                  out.print(JsonUtil.toJsonString(s, false));
+                }
+                // stop if last segment
+                if (!dirList.hasMore()) {
+                  break;
+                }
+              }
+              return null;
+            }
+          });
+        } catch (InterruptedException e) {
+          throw new IOException(e);
         }
         
         out.println();

+ 2 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -156,7 +156,6 @@ public class WebHdfsFileSystem extends FileSystem
   private URI uri;
   private boolean hasInitedToken;
   private Token<?> delegationToken;
-  private final AuthenticatedURL.Token authToken = new AuthenticatedURL.Token();
   private RetryPolicy retryPolicy = null;
   private Path workingDir;
 
@@ -481,6 +480,8 @@ public class WebHdfsFileSystem extends FileSystem
       try {
         if (op.getRequireAuth()) {
           LOG.debug("open AuthenticatedURL connection");
+          UserGroupInformation.getCurrentUser().checkTGTAndReloginFromKeytab();
+          final AuthenticatedURL.Token authToken = new AuthenticatedURL.Token();
           conn = new AuthenticatedURL(AUTH).openConnection(url, authToken);
         } else {
           LOG.debug("open URL connection");
@@ -1006,20 +1007,12 @@ public class WebHdfsFileSystem extends FileSystem
     @Override
     public long renew(final Token<?> token, final Configuration conf
         ) throws IOException, InterruptedException {
-      final UserGroupInformation ugi = UserGroupInformation.getLoginUser();
-      // update the kerberos credentials, if they are coming from a keytab
-      ugi.reloginFromKeytab();
-
       return getWebHdfs(token, conf).renewDelegationToken(token);
     }
   
     @Override
     public void cancel(final Token<?> token, final Configuration conf
         ) throws IOException, InterruptedException {
-      final UserGroupInformation ugi = UserGroupInformation.getLoginUser();
-      // update the kerberos credentials, if they are coming from a keytab
-      ugi.checkTGTAndReloginFromKeytab();
-
       getWebHdfs(token, conf).cancelDelegationToken(token);
     }
   }

+ 18 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java

@@ -98,9 +98,6 @@ public class TestNNWithQJM {
 
   @Test (timeout = 30000)
   public void testNewNamenodeTakesOverWriter() throws Exception {
-    // Skip the test on Windows. See HDFS-4584.
-    assumeTrue(!Path.WINDOWS);
-
     File nn1Dir = new File(
         MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image-nn1");
     File nn2Dir = new File(
@@ -110,23 +107,37 @@ public class TestNNWithQJM {
         nn1Dir.getAbsolutePath());
     conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
         mjc.getQuorumJournalURI("myjournal").toString());
-    
+
+    // Start the cluster once to generate the dfs dirs
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
       .numDataNodes(0)
       .manageNameDfsDirs(false)
       .checkExitOnShutdown(false)
       .build();
 
+    // Shutdown the cluster before making a copy of the namenode dir
+    // to release all file locks, otherwise, the copy will fail on
+    // some platforms.
+    cluster.shutdown();
+
     try {
-      cluster.getFileSystem().mkdirs(TEST_PATH);
-      
       // Start a second NN pointed to the same quorum.
       // We need to copy the image dir from the first NN -- or else
       // the new NN will just be rejected because of Namespace mismatch.
       FileUtil.fullyDelete(nn2Dir);
       FileUtil.copy(nn1Dir, FileSystem.getLocal(conf).getRaw(),
           new Path(nn2Dir.getAbsolutePath()), false, conf);
-      
+
+      // Start the cluster again
+      cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(0)
+        .format(false)
+        .manageNameDfsDirs(false)
+        .checkExitOnShutdown(false)
+        .build();
+
+      cluster.getFileSystem().mkdirs(TEST_PATH);
+
       Configuration conf2 = new Configuration();
       conf2.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
           nn2Dir.getAbsolutePath());

+ 50 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java

@@ -19,6 +19,8 @@
 package org.apache.hadoop.hdfs.web;
 
 import java.io.IOException;
+import java.net.URISyntaxException;
+import java.security.PrivilegedExceptionAction;
 import java.util.Random;
 
 import org.apache.commons.logging.Log;
@@ -29,9 +31,13 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.TestDFSClientRetries;
 import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.log4j.Level;
 import org.junit.Assert;
 import org.junit.Test;
@@ -208,4 +214,48 @@ public class TestWebHDFS {
     final Configuration conf = WebHdfsTestUtil.createConf();
     TestDFSClientRetries.namenodeRestartTest(conf, true);
   }
+  
+  @Test(timeout=300000)
+  public void testLargeDirectory() throws Exception {
+    final Configuration conf = WebHdfsTestUtil.createConf();
+    final int listLimit = 2;
+    // force small chunking of directory listing
+    conf.setInt(DFSConfigKeys.DFS_LIST_LIMIT, listLimit);
+    // force paths to be only owner-accessible to ensure ugi isn't changing
+    // during listStatus
+    FsPermission.setUMask(conf, new FsPermission((short)0077));
+    
+    final MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
+    try {
+      cluster.waitActive();
+      WebHdfsTestUtil.getWebHdfsFileSystem(conf).setPermission(
+          new Path("/"),
+          new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL));
+
+      // trick the NN into not believing it's not the superuser so we can
+      // tell if the correct user is used by listStatus
+      UserGroupInformation.setLoginUser(
+          UserGroupInformation.createUserForTesting(
+              "not-superuser", new String[]{"not-supergroup"}));
+
+      UserGroupInformation.createUserForTesting("me", new String[]{"my-group"})
+        .doAs(new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws IOException, URISyntaxException {
+            FileSystem fs = WebHdfsTestUtil.getWebHdfsFileSystem(conf);
+            Path d = new Path("/my-dir");
+            Assert.assertTrue(fs.mkdirs(d));
+            for (int i=0; i < listLimit*3; i++) {
+              Path p = new Path(d, "file-"+i);
+              Assert.assertTrue(fs.createNewFile(p));
+            }
+            Assert.assertEquals(listLimit*3, fs.listStatus(d).length);
+            return null;
+          }
+        });
+    } finally {
+      cluster.shutdown();
+    }
+  }
 }

+ 28 - 0
hadoop-mapreduce-project/CHANGES.txt

@@ -158,6 +158,26 @@ Trunk (Unreleased)
     MAPREDUCE-5006. Fix failing streaming tests due to MAPREDUCE-4994.
     (Sandy Ryza via tomwhite)
 
+Release 2.0.5-alpha - UNRELEASED
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
+  BUG FIXES
+
+    MAPREDUCE-5113. Streaming input/output types are ignored with java 
+    mapper/reducer. (sandyr via tucu)
+
+    MAPREDUCE-5098. Fix findbugs warnings in gridmix. (kkambatl via tucu)
+
+    MAPREDUCE-5086. MR app master deletes staging dir when sent a reboot
+    command from the RM. (Jian He via jlowe)
+
 Release 2.0.4-beta - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -195,6 +215,13 @@ Release 2.0.4-beta - UNRELEASED
     MAPREDUCE-5008. Merger progress miscounts with respect to EOF_MARKER.
     (Sandy Ryza via tomwhite)
 
+    MAPREDUCE-5117. Changed MRClientProtocolPBClientImpl to be closeable and thus
+    fix failures in renewal of HistoryServer's delegations tokens. (Siddharth
+    Seth via vinodkv)
+
+    MAPREDUCE-5088. MR Client gets an renewer token exception while Oozie is
+    submitting a job (Daryn Sharp via cos)
+
 Release 2.0.3-alpha - 2013-02-06 
 
   INCOMPATIBLE CHANGES
@@ -744,6 +771,7 @@ Release 0.23.7 - UNRELEASED
     MAPREDUCE-5009. Killing the Task Attempt slated for commit does not clear
     the value from the Task commitAttempt member (Robert Parker via jeagles)
 
+    MAPREDUCE-4991. coverage for gridmix (Aleksey Gorshkov via tgraves)
 
 Release 0.23.6 - UNRELEASED
 

+ 22 - 10
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java

@@ -549,8 +549,14 @@ public class MRAppMaster extends CompositeService {
     }
 
     try {
-      //We are finishing cleanly so this is the last retry
-      isLastAMRetry = true;
+      //if isLastAMRetry comes as true, should never set it to false
+      if ( !isLastAMRetry){
+        if (((JobImpl)job).getInternalState() != JobStateInternal.REBOOT) {
+          LOG.info("We are finishing cleanly so this is the last retry");
+          isLastAMRetry = true;
+        }
+      }
+      notifyIsLastAMRetry(isLastAMRetry);
       // Stop all services
       // This will also send the final report to the ResourceManager
       LOG.info("Calling stop for all the services");
@@ -1272,19 +1278,25 @@ public class MRAppMaster extends CompositeService {
       // that they don't take too long in shutting down
       if(appMaster.containerAllocator instanceof ContainerAllocatorRouter) {
         ((ContainerAllocatorRouter) appMaster.containerAllocator)
-        .setSignalled(true);
-        ((ContainerAllocatorRouter) appMaster.containerAllocator)
-        .setShouldUnregister(appMaster.isLastAMRetry);
-      }
-      
-      if(appMaster.jobHistoryEventHandler != null) {
-        appMaster.jobHistoryEventHandler
-          .setForcejobCompletion(appMaster.isLastAMRetry);
+          .setSignalled(true);
       }
+      appMaster.notifyIsLastAMRetry(appMaster.isLastAMRetry);
       appMaster.stop();
     }
   }
 
+  public void notifyIsLastAMRetry(boolean isLastAMRetry){
+    if(containerAllocator instanceof ContainerAllocatorRouter) {
+      LOG.info("Notify RMCommunicator isAMLastRetry: " + isLastAMRetry);
+      ((ContainerAllocatorRouter) containerAllocator)
+        .setShouldUnregister(isLastAMRetry);
+    }
+    if(jobHistoryEventHandler != null) {
+      LOG.info("Notify JHEH isAMLastRetry: " + isLastAMRetry);
+      jobHistoryEventHandler.setForcejobCompletion(isLastAMRetry);
+    }
+  }
+
   protected static void initAndStartAppMaster(final MRAppMaster appMaster,
       final YarnConfiguration conf, String jobUserName) throws IOException,
       InterruptedException {

+ 2 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/JobStateInternal.java

@@ -30,5 +30,6 @@ public enum JobStateInternal {
   KILL_WAIT,
   KILL_ABORT,
   KILLED,
-  ERROR
+  ERROR,
+  REBOOT
 }

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/JobEventType.java

@@ -54,6 +54,6 @@ public enum JobEventType {
   JOB_TASK_ATTEMPT_FETCH_FAILURE,
   
   //Producer:RMContainerAllocator
-  JOB_UPDATED_NODES
-  
+  JOB_UPDATED_NODES,
+  JOB_AM_REBOOT
 }

+ 82 - 11
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java

@@ -215,6 +215,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
       DIAGNOSTIC_UPDATE_TRANSITION = new DiagnosticsUpdateTransition();
   private static final InternalErrorTransition
       INTERNAL_ERROR_TRANSITION = new InternalErrorTransition();
+  private static final InternalRebootTransition
+      INTERNAL_REBOOT_TRANSITION = new InternalRebootTransition();
   private static final TaskAttemptCompletedEventTransition
       TASK_ATTEMPT_COMPLETED_EVENT_TRANSITION =
           new TaskAttemptCompletedEventTransition();
@@ -246,6 +248,9 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
           .addTransition(JobStateInternal.NEW, JobStateInternal.ERROR,
               JobEventType.INTERNAL_ERROR,
               INTERNAL_ERROR_TRANSITION)
+          .addTransition(JobStateInternal.NEW, JobStateInternal.REBOOT,
+              JobEventType.JOB_AM_REBOOT,
+              INTERNAL_REBOOT_TRANSITION)
           // Ignore-able events
           .addTransition(JobStateInternal.NEW, JobStateInternal.NEW,
               JobEventType.JOB_UPDATED_NODES)
@@ -265,6 +270,9 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
           .addTransition(JobStateInternal.INITED, JobStateInternal.ERROR,
               JobEventType.INTERNAL_ERROR,
               INTERNAL_ERROR_TRANSITION)
+          .addTransition(JobStateInternal.INITED, JobStateInternal.REBOOT,
+              JobEventType.JOB_AM_REBOOT,
+              INTERNAL_REBOOT_TRANSITION)
           // Ignore-able events
           .addTransition(JobStateInternal.INITED, JobStateInternal.INITED,
               JobEventType.JOB_UPDATED_NODES)
@@ -287,6 +295,9 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
           .addTransition(JobStateInternal.SETUP, JobStateInternal.ERROR,
               JobEventType.INTERNAL_ERROR,
               INTERNAL_ERROR_TRANSITION)
+          .addTransition(JobStateInternal.SETUP, JobStateInternal.REBOOT,
+              JobEventType.JOB_AM_REBOOT,
+              INTERNAL_REBOOT_TRANSITION)
           // Ignore-able events
           .addTransition(JobStateInternal.SETUP, JobStateInternal.SETUP,
               JobEventType.JOB_UPDATED_NODES)
@@ -327,6 +338,9 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
               JobStateInternal.RUNNING,
               JobStateInternal.ERROR, JobEventType.INTERNAL_ERROR,
               INTERNAL_ERROR_TRANSITION)
+          .addTransition(JobStateInternal.RUNNING, JobStateInternal.REBOOT,
+              JobEventType.JOB_AM_REBOOT,
+              INTERNAL_REBOOT_TRANSITION)
 
           // Transitions from KILL_WAIT state.
           .addTransition
@@ -352,7 +366,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
               EnumSet.of(JobEventType.JOB_KILL,
                   JobEventType.JOB_UPDATED_NODES,
                   JobEventType.JOB_MAP_TASK_RESCHEDULED,
-                  JobEventType.JOB_TASK_ATTEMPT_FETCH_FAILURE))
+                  JobEventType.JOB_TASK_ATTEMPT_FETCH_FAILURE,
+                  JobEventType.JOB_AM_REBOOT))
 
           // Transitions from COMMITTING state
           .addTransition(JobStateInternal.COMMITTING,
@@ -377,7 +392,10 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
           .addTransition(JobStateInternal.COMMITTING,
               JobStateInternal.ERROR, JobEventType.INTERNAL_ERROR,
               INTERNAL_ERROR_TRANSITION)
-          // Ignore-able events
+          .addTransition(JobStateInternal.COMMITTING, JobStateInternal.REBOOT,
+              JobEventType.JOB_AM_REBOOT,
+              INTERNAL_REBOOT_TRANSITION)
+              // Ignore-able events
           .addTransition(JobStateInternal.COMMITTING,
               JobStateInternal.COMMITTING,
               EnumSet.of(JobEventType.JOB_UPDATED_NODES,
@@ -397,7 +415,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
           .addTransition(JobStateInternal.SUCCEEDED, JobStateInternal.SUCCEEDED,
               EnumSet.of(JobEventType.JOB_KILL, 
                   JobEventType.JOB_UPDATED_NODES,
-                  JobEventType.JOB_TASK_ATTEMPT_FETCH_FAILURE))
+                  JobEventType.JOB_TASK_ATTEMPT_FETCH_FAILURE,
+                  JobEventType.JOB_AM_REBOOT))
 
           // Transitions from FAIL_ABORT state
           .addTransition(JobStateInternal.FAIL_ABORT,
@@ -425,7 +444,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
                   JobEventType.JOB_MAP_TASK_RESCHEDULED,
                   JobEventType.JOB_TASK_ATTEMPT_FETCH_FAILURE,
                   JobEventType.JOB_COMMIT_COMPLETED,
-                  JobEventType.JOB_COMMIT_FAILED))
+                  JobEventType.JOB_COMMIT_FAILED,
+                  JobEventType.JOB_AM_REBOOT))
 
           // Transitions from KILL_ABORT state
           .addTransition(JobStateInternal.KILL_ABORT,
@@ -452,7 +472,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
                   JobEventType.JOB_SETUP_COMPLETED,
                   JobEventType.JOB_SETUP_FAILED,
                   JobEventType.JOB_COMMIT_COMPLETED,
-                  JobEventType.JOB_COMMIT_FAILED))
+                  JobEventType.JOB_COMMIT_FAILED,
+                  JobEventType.JOB_AM_REBOOT))
 
           // Transitions from FAILED state
           .addTransition(JobStateInternal.FAILED, JobStateInternal.FAILED,
@@ -476,7 +497,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
                   JobEventType.JOB_SETUP_FAILED,
                   JobEventType.JOB_COMMIT_COMPLETED,
                   JobEventType.JOB_COMMIT_FAILED,
-                  JobEventType.JOB_ABORT_COMPLETED))
+                  JobEventType.JOB_ABORT_COMPLETED,
+                  JobEventType.JOB_AM_REBOOT))
 
           // Transitions from KILLED state
           .addTransition(JobStateInternal.KILLED, JobStateInternal.KILLED,
@@ -498,7 +520,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
                   JobEventType.JOB_SETUP_FAILED,
                   JobEventType.JOB_COMMIT_COMPLETED,
                   JobEventType.JOB_COMMIT_FAILED,
-                  JobEventType.JOB_ABORT_COMPLETED))
+                  JobEventType.JOB_ABORT_COMPLETED,
+                  JobEventType.JOB_AM_REBOOT))
 
           // No transitions from INTERNAL_ERROR state. Ignore all.
           .addTransition(
@@ -517,9 +540,33 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
                   JobEventType.JOB_COMMIT_COMPLETED,
                   JobEventType.JOB_COMMIT_FAILED,
                   JobEventType.JOB_ABORT_COMPLETED,
-                  JobEventType.INTERNAL_ERROR))
+                  JobEventType.INTERNAL_ERROR,
+                  JobEventType.JOB_AM_REBOOT))
           .addTransition(JobStateInternal.ERROR, JobStateInternal.ERROR,
               JobEventType.JOB_COUNTER_UPDATE, COUNTER_UPDATE_TRANSITION)
+
+          // No transitions from AM_REBOOT state. Ignore all.
+          .addTransition(
+              JobStateInternal.REBOOT,
+              JobStateInternal.REBOOT,
+              EnumSet.of(JobEventType.JOB_INIT,
+                  JobEventType.JOB_KILL,
+                  JobEventType.JOB_TASK_COMPLETED,
+                  JobEventType.JOB_TASK_ATTEMPT_COMPLETED,
+                  JobEventType.JOB_MAP_TASK_RESCHEDULED,
+                  JobEventType.JOB_DIAGNOSTIC_UPDATE,
+                  JobEventType.JOB_UPDATED_NODES,
+                  JobEventType.JOB_TASK_ATTEMPT_FETCH_FAILURE,
+                  JobEventType.JOB_SETUP_COMPLETED,
+                  JobEventType.JOB_SETUP_FAILED,
+                  JobEventType.JOB_COMMIT_COMPLETED,
+                  JobEventType.JOB_COMMIT_FAILED,
+                  JobEventType.JOB_ABORT_COMPLETED,
+                  JobEventType.INTERNAL_ERROR,
+                  JobEventType.JOB_AM_REBOOT))
+          .addTransition(JobStateInternal.REBOOT, JobStateInternal.REBOOT,
+              JobEventType.JOB_COUNTER_UPDATE, COUNTER_UPDATE_TRANSITION)
+
           // create the topology tables
           .installTopology();
  
@@ -904,6 +951,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
       return JobState.RUNNING;
     case FAIL_ABORT:
       return JobState.FAILED;
+    case REBOOT:
+      return JobState.ERROR;
     default:
       return JobState.valueOf(smState.name());
     }
@@ -972,6 +1021,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
       case KILLED:
         metrics.killedJob(this);
         break;
+      case REBOOT:
       case ERROR:
       case FAILED:
         metrics.failedJob(this);
@@ -1898,8 +1948,17 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
     }
   }
   
-  private static class InternalErrorTransition implements
+  private static class InternalTerminationTransition implements
       SingleArcTransition<JobImpl, JobEvent> {
+    JobStateInternal terminationState = null;
+    String jobHistoryString = null;
+    public InternalTerminationTransition(JobStateInternal stateInternal,
+        String jobHistoryString) {
+      this.terminationState = stateInternal;
+      //mostly a hack for jbhistoryserver
+      this.jobHistoryString = jobHistoryString;
+    }
+
     @Override
     public void transition(JobImpl job, JobEvent event) {
       //TODO Is this JH event required.
@@ -1907,9 +1966,21 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
       JobUnsuccessfulCompletionEvent failedEvent =
           new JobUnsuccessfulCompletionEvent(job.oldJobId,
               job.finishTime, 0, 0,
-              JobStateInternal.ERROR.toString());
+              jobHistoryString);
       job.eventHandler.handle(new JobHistoryEvent(job.jobId, failedEvent));
-      job.finished(JobStateInternal.ERROR);
+      job.finished(terminationState);
+    }
+  }
+
+  private static class InternalErrorTransition extends InternalTerminationTransition {
+    public InternalErrorTransition(){
+      super(JobStateInternal.ERROR, JobStateInternal.ERROR.toString());
+    }
+  }
+
+  private static class InternalRebootTransition extends InternalTerminationTransition  {
+    public InternalRebootTransition(){
+      super(JobStateInternal.REBOOT, JobStateInternal.ERROR.toString());
     }
   }
 

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/local/LocalContainerAllocator.java

@@ -123,7 +123,7 @@ public class LocalContainerAllocator extends RMCommunicator
       // This can happen if the RM has been restarted. If it is in that state,
       // this application must clean itself up.
       eventHandler.handle(new JobEvent(this.getJob().getID(),
-                                       JobEventType.INTERNAL_ERROR));
+                                       JobEventType.JOB_AM_REBOOT));
       throw new YarnException("Resource Manager doesn't recognize AttemptId: " +
                                this.getContext().getApplicationID());
     }

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java

@@ -574,7 +574,7 @@ public class RMContainerAllocator extends RMContainerRequestor
       // This can happen if the RM has been restarted. If it is in that state,
       // this application must clean itself up.
       eventHandler.handle(new JobEvent(this.getJob().getID(),
-                                       JobEventType.INTERNAL_ERROR));
+                                       JobEventType.JOB_AM_REBOOT));
       throw new YarnException("Resource Manager doesn't recognize AttemptId: " +
                                this.getContext().getApplicationID());
     }

+ 99 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestStagingCleanup.java

@@ -33,7 +33,9 @@ import junit.framework.TestCase;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.JobState;
 import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
@@ -45,6 +47,7 @@ import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocator;
 import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent;
 import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -86,9 +89,68 @@ import org.junit.Test;
      attemptId.setApplicationId(appId);
      JobId jobid = recordFactory.newRecordInstance(JobId.class);
      jobid.setAppId(appId);
-     MRAppMaster appMaster = new TestMRApp(attemptId);
+     ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
+     MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc,
+         JobStateInternal.RUNNING, MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS);
+     appMaster.init(conf);
+     appMaster.start();
+     appMaster.shutDownJob();
+     //test whether notifyIsLastAMRetry called
+     Assert.assertEquals(true, ((TestMRApp)appMaster).getTestIsLastAMRetry());
+     verify(fs).delete(stagingJobPath, true);
+   }
+
+   @Test (timeout = 30000)
+   public void testNoDeletionofStagingOnReboot() throws IOException {
+     conf.set(MRJobConfig.MAPREDUCE_JOB_DIR, stagingJobDir);
+     fs = mock(FileSystem.class);
+     when(fs.delete(any(Path.class),anyBoolean())).thenReturn(true);
+     String user = UserGroupInformation.getCurrentUser().getShortUserName();
+     Path stagingDir = MRApps.getStagingAreaDir(conf, user);
+     when(fs.exists(stagingDir)).thenReturn(true);
+     ApplicationAttemptId attemptId = recordFactory.newRecordInstance(
+         ApplicationAttemptId.class);
+     attemptId.setAttemptId(0);
+     ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class);
+     appId.setClusterTimestamp(System.currentTimeMillis());
+     appId.setId(0);
+     attemptId.setApplicationId(appId);
+     ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
+     MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc,
+         JobStateInternal.REBOOT, 4);
+     appMaster.init(conf);
+     appMaster.start();
+     //shutdown the job, not the lastRetry
+     appMaster.shutDownJob();
+     //test whether notifyIsLastAMRetry called
+     Assert.assertEquals(false, ((TestMRApp)appMaster).getTestIsLastAMRetry());
+     verify(fs, times(0)).delete(stagingJobPath, true);
+   }
+
+   @Test (timeout = 30000)
+   public void testDeletionofStagingOnReboot() throws IOException {
+     conf.set(MRJobConfig.MAPREDUCE_JOB_DIR, stagingJobDir);
+     fs = mock(FileSystem.class);
+     when(fs.delete(any(Path.class),anyBoolean())).thenReturn(true);
+     String user = UserGroupInformation.getCurrentUser().getShortUserName();
+     Path stagingDir = MRApps.getStagingAreaDir(conf, user);
+     when(fs.exists(stagingDir)).thenReturn(true);
+     ApplicationAttemptId attemptId = recordFactory.newRecordInstance(
+         ApplicationAttemptId.class);
+     attemptId.setAttemptId(1);
+     ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class);
+     appId.setClusterTimestamp(System.currentTimeMillis());
+     appId.setId(0);
+     attemptId.setApplicationId(appId);
+     ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
+     MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc,
+         JobStateInternal.REBOOT, MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS);
      appMaster.init(conf);
+     appMaster.start();
+     //shutdown the job, is lastRetry
      appMaster.shutDownJob();
+     //test whether notifyIsLastAMRetry called
+     Assert.assertEquals(true, ((TestMRApp)appMaster).getTestIsLastAMRetry());
      verify(fs).delete(stagingJobPath, true);
    }
    
@@ -151,6 +213,8 @@ import org.junit.Test;
 
    private class TestMRApp extends MRAppMaster {
      ContainerAllocator allocator;
+     boolean testIsLastAMRetry = false;
+     JobStateInternal jobStateInternal;
 
      public TestMRApp(ApplicationAttemptId applicationAttemptId, 
          ContainerAllocator allocator, int maxAppAttempts) {
@@ -160,9 +224,11 @@ import org.junit.Test;
        this.allocator = allocator;
      }
 
-     public TestMRApp(ApplicationAttemptId applicationAttemptId) {
-       this(applicationAttemptId, null,
-           MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS);
+     public TestMRApp(ApplicationAttemptId applicationAttemptId,
+         ContainerAllocator allocator, JobStateInternal jobStateInternal,
+             int maxAppAttempts) {
+       this(applicationAttemptId, allocator, maxAppAttempts);
+       this.jobStateInternal = jobStateInternal;
      }
 
      @Override
@@ -179,6 +245,31 @@ import org.junit.Test;
        return allocator;
      }
 
+     @Override
+     protected Job createJob(Configuration conf, JobStateInternal forcedState,
+         String diagnostic) {
+       JobImpl jobImpl = mock(JobImpl.class);
+       when(jobImpl.getInternalState()).thenReturn(this.jobStateInternal);
+       JobID jobID = JobID.forName("job_1234567890000_0001");
+       JobId jobId = TypeConverter.toYarn(jobID);
+       when(jobImpl.getID()).thenReturn(jobId);
+       ((AppContext) getContext())
+           .getAllJobs().put(jobImpl.getID(), jobImpl);
+       return jobImpl;
+     }
+
+     @Override
+     public void start() {
+       super.start();
+       DefaultMetricsSystem.shutdown();
+     }
+
+     @Override
+     public void notifyIsLastAMRetry(boolean isLastAMRetry){
+       testIsLastAMRetry = isLastAMRetry;
+       super.notifyIsLastAMRetry(isLastAMRetry);
+     }
+
      @Override
      public RMHeartbeatHandler getRMHeartbeatHandler() {
        return getStubbedHeartbeatHandler(getContext());
@@ -197,6 +288,9 @@ import org.junit.Test;
      protected void downloadTokensAndSetupUGI(Configuration conf) {
      }
 
+     public boolean getTestIsLastAMRetry(){
+       return testIsLastAMRetry;
+     }
    }
 
   private final class MRAppTestCleanup extends MRApp {
@@ -288,7 +382,7 @@ import org.junit.Test;
     };
   }
 
-  @Test
+  @Test(timeout=20000)
   public void testStagingCleanupOrder() throws Exception {
     MRAppTestCleanup app = new MRAppTestCleanup(1, 1, true,
         this.getClass().getName(), true);

+ 62 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java

@@ -192,6 +192,68 @@ public class TestJobImpl {
     commitHandler.stop();
   }
 
+  @Test(timeout=20000)
+  public void testRebootedDuringSetup() throws Exception{
+    Configuration conf = new Configuration();
+    conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
+    AsyncDispatcher dispatcher = new AsyncDispatcher();
+    dispatcher.init(conf);
+    dispatcher.start();
+    OutputCommitter committer = new StubbedOutputCommitter() {
+      @Override
+      public synchronized void setupJob(JobContext jobContext)
+          throws IOException {
+        while(!Thread.interrupted()){
+          try{
+            wait();
+          }catch (InterruptedException e) {
+          }
+        }
+      }
+    };
+    CommitterEventHandler commitHandler =
+        createCommitterEventHandler(dispatcher, committer);
+    commitHandler.init(conf);
+    commitHandler.start();
+
+    JobImpl job = createStubbedJob(conf, dispatcher, 2);
+    JobId jobId = job.getID();
+    job.handle(new JobEvent(jobId, JobEventType.JOB_INIT));
+    assertJobState(job, JobStateInternal.INITED);
+    job.handle(new JobEvent(jobId, JobEventType.JOB_START));
+    assertJobState(job, JobStateInternal.SETUP);
+
+    job.handle(new JobEvent(job.getID(), JobEventType.JOB_AM_REBOOT));
+    assertJobState(job, JobStateInternal.REBOOT);
+    dispatcher.stop();
+    commitHandler.stop();
+  }
+
+  @Test(timeout=20000)
+  public void testRebootedDuringCommit() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
+    AsyncDispatcher dispatcher = new AsyncDispatcher();
+    dispatcher.init(conf);
+    dispatcher.start();
+    CyclicBarrier syncBarrier = new CyclicBarrier(2);
+    OutputCommitter committer = new WaitingOutputCommitter(syncBarrier, true);
+    CommitterEventHandler commitHandler =
+        createCommitterEventHandler(dispatcher, committer);
+    commitHandler.init(conf);
+    commitHandler.start();
+
+    JobImpl job = createRunningStubbedJob(conf, dispatcher, 2);
+    completeJobTasks(job);
+    assertJobState(job, JobStateInternal.COMMITTING);
+
+    syncBarrier.await();
+    job.handle(new JobEvent(job.getID(), JobEventType.JOB_AM_REBOOT));
+    assertJobState(job, JobStateInternal.REBOOT);
+    dispatcher.stop();
+    commitHandler.stop();
+  }
+
   @Test(timeout=20000)
   public void testKilledDuringSetup() throws Exception {
     Configuration conf = new Configuration();

+ 10 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/impl/pb/client/MRClientProtocolPBClientImpl.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.mapreduce.v2.api.impl.pb.client;
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 
@@ -101,7 +102,8 @@ import org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl;
 
 import com.google.protobuf.ServiceException;
 
-public class MRClientProtocolPBClientImpl implements MRClientProtocol {
+public class MRClientProtocolPBClientImpl implements MRClientProtocol,
+    Closeable {
 
   protected MRClientProtocolPB proxy;
   
@@ -117,6 +119,13 @@ public class MRClientProtocolPBClientImpl implements MRClientProtocol {
     return RPC.getServerAddress(proxy);
   }
 
+  @Override
+  public void close() {
+    if (this.proxy != null) {
+      RPC.stopProxy(this.proxy);
+    }
+  }
+
   @Override
   public GetJobReportResponse getJobReport(GetJobReportRequest request)
       throws YarnRemoteException {

+ 0 - 14
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java

@@ -138,15 +138,6 @@ import org.apache.hadoop.util.ToolRunner;
 public class JobClient extends CLI {
   public static enum TaskStatusFilter { NONE, KILLED, FAILED, SUCCEEDED, ALL }
   private TaskStatusFilter taskOutputFilter = TaskStatusFilter.FAILED; 
-  /* notes that get delegation token was called. Again this is hack for oozie 
-   * to make sure we add history server delegation tokens to the credentials
-   *  for the job. Since the api only allows one delegation token to be returned, 
-   *  we have to add this hack.
-   */
-  private boolean getDelegationTokenCalled = false;
-  /* do we need a HS delegation token for this client */
-  static final String HS_DELEGATION_TOKEN_REQUIRED 
-      = "mapreduce.history.server.delegationtoken.required";
   
   static{
     ConfigUtil.loadResources();
@@ -569,10 +560,6 @@ public class JobClient extends CLI {
     try {
       conf.setBooleanIfUnset("mapred.mapper.new-api", false);
       conf.setBooleanIfUnset("mapred.reducer.new-api", false);
-      if (getDelegationTokenCalled) {
-        conf.setBoolean(HS_DELEGATION_TOKEN_REQUIRED, getDelegationTokenCalled);
-        getDelegationTokenCalled = false;
-      }
       Job job = clientUgi.doAs(new PrivilegedExceptionAction<Job> () {
         @Override
         public Job run() throws IOException, ClassNotFoundException, 
@@ -1173,7 +1160,6 @@ public class JobClient extends CLI {
    */
   public Token<DelegationTokenIdentifier> 
     getDelegationToken(final Text renewer) throws IOException, InterruptedException {
-    getDelegationTokenCalled = true;
     return clientUgi.doAs(new 
         PrivilegedExceptionAction<Token<DelegationTokenIdentifier>>() {
       public Token<DelegationTokenIdentifier> run() throws IOException, 

+ 5 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.mapred;
 
 import java.io.IOException;
+import java.net.InetSocketAddress;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -87,6 +88,10 @@ public class ResourceMgrDelegate extends YarnClientImpl {
     return oldMetrics;
   }
 
+  InetSocketAddress getConnectAddress() {
+    return rmAddress;
+  }
+  
   @SuppressWarnings("rawtypes")
   public Token getDelegationToken(Text renewer) throws IOException,
       InterruptedException {

+ 27 - 21
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java

@@ -60,6 +60,7 @@ import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetDelegationTokenRequ
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.token.Token;
@@ -81,6 +82,7 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.security.client.RMTokenSelector;
 import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.ProtoUtils;
@@ -90,7 +92,7 @@ import com.google.common.annotations.VisibleForTesting;
 /**
  * This class enables the current JobClient (0.22 hadoop) to run on YARN.
  */
-@SuppressWarnings({ "rawtypes", "unchecked" })
+@SuppressWarnings("unchecked")
 public class YARNRunner implements ClientProtocol {
 
   private static final Log LOG = LogFactory.getLog(YARNRunner.class);
@@ -101,14 +103,6 @@ public class YARNRunner implements ClientProtocol {
   private Configuration conf;
   private final FileContext defaultFileContext;
   
-  /* usually is false unless the jobclient get delegation token is 
-   *  called. This is a hack wherein we do return a token from RM 
-   *  on getDelegationtoken but due to the restricted api on jobclient
-   *  we just add a job history DT token when submitting a job.
-   */
-  private static final boolean DEFAULT_HS_DELEGATION_TOKEN_REQUIRED = 
-      false;
-  
   /**
    * Yarn runner incapsulates the client interface of
    * yarn
@@ -185,6 +179,28 @@ public class YARNRunner implements ClientProtocol {
     return resMgrDelegate.getClusterMetrics();
   }
 
+  @VisibleForTesting
+  void addHistoyToken(Credentials ts) throws IOException, InterruptedException {
+    /* check if we have a hsproxy, if not, no need */
+    MRClientProtocol hsProxy = clientCache.getInitializedHSProxy();
+    if (UserGroupInformation.isSecurityEnabled() && (hsProxy != null)) {
+      /*
+       * note that get delegation token was called. Again this is hack for oozie
+       * to make sure we add history server delegation tokens to the credentials
+       */
+      RMTokenSelector tokenSelector = new RMTokenSelector();
+      Text service = SecurityUtil.buildTokenService(resMgrDelegate
+          .getConnectAddress());
+      if (tokenSelector.selectToken(service, ts.getAllTokens()) != null) {
+        Text hsService = SecurityUtil.buildTokenService(hsProxy
+            .getConnectAddress());
+        if (ts.getToken(hsService) == null) {
+          ts.addToken(hsService, getDelegationTokenFromHS(hsProxy));
+        }
+      }
+    }
+  }
+  
   @VisibleForTesting
   Token<?> getDelegationTokenFromHS(MRClientProtocol hsProxy)
       throws IOException, InterruptedException {
@@ -263,18 +279,8 @@ public class YARNRunner implements ClientProtocol {
   public JobStatus submitJob(JobID jobId, String jobSubmitDir, Credentials ts)
   throws IOException, InterruptedException {
     
-    /* check if we have a hsproxy, if not, no need */
-    MRClientProtocol hsProxy = clientCache.getInitializedHSProxy();
-    if (hsProxy != null) {
-      // JobClient will set this flag if getDelegationToken is called, if so, get
-      // the delegation tokens for the HistoryServer also.
-      if (conf.getBoolean(JobClient.HS_DELEGATION_TOKEN_REQUIRED, 
-          DEFAULT_HS_DELEGATION_TOKEN_REQUIRED)) {
-        Token hsDT = getDelegationTokenFromHS(hsProxy);
-        ts.addToken(hsDT.getService(), hsDT);
-      }
-    }
-
+    addHistoyToken(ts);
+    
     // Upload only in security mode: TODO
     Path applicationTokensFile =
         new Path(jobSubmitDir, MRJobConfig.APPLICATION_TOKENS_FILE);

+ 102 - 17
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java

@@ -20,8 +20,10 @@ package org.apache.hadoop.mapred;
 
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
@@ -30,6 +32,7 @@ import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
+import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.security.PrivilegedExceptionAction;
 import java.util.List;
@@ -39,28 +42,24 @@ import junit.framework.TestCase;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.ClientCache;
-import org.apache.hadoop.mapred.ClientServiceDelegate;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Master;
-import org.apache.hadoop.mapred.ResourceMgrDelegate;
-import org.apache.hadoop.mapred.YARNRunner;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.JobPriority;
 import org.apache.hadoop.mapreduce.JobStatus.State;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
+import org.apache.hadoop.mapreduce.v2.api.MRDelegationTokenIdentifier;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetDelegationTokenRequest;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetDelegationTokenResponse;
 import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.yarn.api.ClientRMProtocol;
-import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
@@ -69,21 +68,27 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.DelegationToken;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
-import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
+import org.apache.hadoop.yarn.util.BuilderUtils;
+import org.apache.hadoop.yarn.util.Records;
 import org.apache.log4j.Appender;
 import org.apache.log4j.Layout;
 import org.apache.log4j.Logger;
@@ -146,7 +151,7 @@ public class TestYARNRunner extends TestCase {
    }
 
 
-  @Test
+  @Test(timeout=20000)
   public void testJobKill() throws Exception {
     clientDelegate = mock(ClientServiceDelegate.class);
     when(clientDelegate.getJobStatus(any(JobID.class))).thenReturn(new
@@ -171,7 +176,7 @@ public class TestYARNRunner extends TestCase {
     verify(clientDelegate).killJob(jobId);
   }
 
-  @Test
+  @Test(timeout=20000)
   public void testJobSubmissionFailure() throws Exception {
     when(resourceMgrDelegate.submitApplication(any(ApplicationSubmissionContext.class))).
     thenReturn(appId);
@@ -193,7 +198,7 @@ public class TestYARNRunner extends TestCase {
     }
   }
 
-  @Test
+  @Test(timeout=20000)
   public void testResourceMgrDelegate() throws Exception {
     /* we not want a mock of resource mgr delegate */
     final ClientRMProtocol clientRMProtocol = mock(ClientRMProtocol.class);
@@ -259,8 +264,88 @@ public class TestYARNRunner extends TestCase {
     delegate.getQueueAclsForCurrentUser();
     verify(clientRMProtocol).getQueueUserAcls(any(GetQueueUserAclsInfoRequest.class));
   }
-  
-  @Test
+
+  @Test(timeout=20000)
+  public void testGetHSDelegationToken() throws Exception {
+    try {
+      Configuration conf = new Configuration();
+
+      // Setup mock service
+      InetSocketAddress mockRmAddress = new InetSocketAddress("localhost", 4444);
+      Text rmTokenSevice = SecurityUtil.buildTokenService(mockRmAddress);
+
+      InetSocketAddress mockHsAddress = new InetSocketAddress("localhost", 9200);
+      Text hsTokenSevice = SecurityUtil.buildTokenService(mockHsAddress);
+
+      // Setup mock rm token
+      RMDelegationTokenIdentifier tokenIdentifier = new RMDelegationTokenIdentifier(
+          new Text("owner"), new Text("renewer"), new Text("real"));
+      Token<RMDelegationTokenIdentifier> token = new Token<RMDelegationTokenIdentifier>(
+          new byte[0], new byte[0], tokenIdentifier.getKind(), rmTokenSevice);
+      token.setKind(RMDelegationTokenIdentifier.KIND_NAME);
+
+      // Setup mock history token
+      DelegationToken historyToken = BuilderUtils.newDelegationToken(
+          new byte[0], MRDelegationTokenIdentifier.KIND_NAME.toString(),
+          new byte[0], hsTokenSevice.toString());
+      GetDelegationTokenResponse getDtResponse = Records
+          .newRecord(GetDelegationTokenResponse.class);
+      getDtResponse.setDelegationToken(historyToken);
+
+      // mock services
+      MRClientProtocol mockHsProxy = mock(MRClientProtocol.class);
+      doReturn(mockHsAddress).when(mockHsProxy).getConnectAddress();
+      doReturn(getDtResponse).when(mockHsProxy).getDelegationToken(
+          any(GetDelegationTokenRequest.class));
+
+      ResourceMgrDelegate rmDelegate = mock(ResourceMgrDelegate.class);
+      doReturn(mockRmAddress).when(rmDelegate).getConnectAddress();
+
+      ClientCache clientCache = mock(ClientCache.class);
+      doReturn(mockHsProxy).when(clientCache).getInitializedHSProxy();
+
+      Credentials creds = new Credentials();
+
+      YARNRunner yarnRunner = new YARNRunner(conf, rmDelegate, clientCache);
+
+      // No HS token if no RM token
+      yarnRunner.addHistoyToken(creds);
+      verify(mockHsProxy, times(0)).getDelegationToken(
+          any(GetDelegationTokenRequest.class));
+
+      // No HS token if RM token, but secirity disabled.
+      creds.addToken(new Text("rmdt"), token);
+      yarnRunner.addHistoyToken(creds);
+      verify(mockHsProxy, times(0)).getDelegationToken(
+          any(GetDelegationTokenRequest.class));
+
+      conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION,
+          "kerberos");
+      UserGroupInformation.setConfiguration(conf);
+      creds = new Credentials();
+
+      // No HS token if no RM token, security enabled
+      yarnRunner.addHistoyToken(creds);
+      verify(mockHsProxy, times(0)).getDelegationToken(
+          any(GetDelegationTokenRequest.class));
+
+      // HS token if RM token present, security enabled
+      creds.addToken(new Text("rmdt"), token);
+      yarnRunner.addHistoyToken(creds);
+      verify(mockHsProxy, times(1)).getDelegationToken(
+          any(GetDelegationTokenRequest.class));
+
+      // No additional call to get HS token if RM and HS token present
+      yarnRunner.addHistoyToken(creds);
+      verify(mockHsProxy, times(1)).getDelegationToken(
+          any(GetDelegationTokenRequest.class));
+    } finally {
+      // Back to defaults.
+      UserGroupInformation.setConfiguration(new Configuration());
+    }
+  }
+
+  @Test(timeout=20000)
   public void testHistoryServerToken() throws Exception {
     //Set the master principal in the config
     conf.set(YarnConfiguration.RM_PRINCIPAL,"foo@LOCAL");
@@ -303,7 +388,7 @@ public class TestYARNRunner extends TestCase {
         });
   }
 
-  @Test
+  @Test(timeout=20000)
   public void testAMAdminCommandOpts() throws Exception {
     JobConf jobConf = new JobConf();
     
@@ -366,7 +451,7 @@ public class TestYARNRunner extends TestCase {
       assertTrue("AM admin command opts is after user command opts.", adminIndex < userIndex);
     }
   }
-  @Test
+  @Test(timeout=20000)
   public void testWarnCommandOpts() throws Exception {
     Logger logger = Logger.getLogger(YARNRunner.class);
     

+ 33 - 0
hadoop-tools/hadoop-gridmix/dev-support/findbugs-exclude.xml

@@ -0,0 +1,33 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+   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.
+-->
+<FindBugsFilter>
+
+ <!-- Ignore some irrelevant serialization warnings -->
+  <Match>
+    <Class name="org.apache.hadoop.mapred.gridmix.GridmixRecord$Comparator" />
+    <Bug pattern="SE_COMPARATOR_SHOULD_BE_SERIALIZABLE" />
+  </Match>
+
+  <!-- The called methods actually might throw Exceptions -->
+  <Match>
+    <Class name="org.apache.hadoop.mapred.gridmix.ExecutionSummarizer"/>
+    <Method name="processJobState"/>
+    <Bug pattern="REC_CATCH_EXCEPTION"/>
+    <Bug code="REC"/>
+  </Match>
+</FindBugsFilter>

+ 24 - 0
hadoop-tools/hadoop-gridmix/pom.xml

@@ -91,10 +91,25 @@
       <type>test-jar</type>
       <scope>test</scope>
     </dependency>
+     <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>
     <plugins>
+       <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>findbugs-maven-plugin</artifactId>
+         <configuration>
+          <findbugsXmlOutput>true</findbugsXmlOutput>
+          <xmlOutput>true</xmlOutput>
+          <excludeFilterFile>${basedir}/dev-support/findbugs-exclude.xml</excludeFilterFile>
+          <effort>Max</effort>
+        </configuration>
+      </plugin>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-antrun-plugin</artifactId>
@@ -115,6 +130,15 @@
           </execution>
         </executions>
       </plugin>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <configuration>
+        <excludes>
+        <exclude>src/test/resources/data/*</exclude>
+        </excludes>
+        </configuration>
+      </plugin>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-jar-plugin</artifactId>

+ 8 - 2
hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/Gridmix.java

@@ -36,7 +36,9 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.mapred.gridmix.GenerateData.DataStatistics;
 import org.apache.hadoop.mapred.gridmix.Statistics.JobStats;
 import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
@@ -693,7 +695,7 @@ public class Gridmix extends Configured implements Tool {
     try {
       res = ToolRunner.run(new Configuration(), new Gridmix(argv), argv);
     } finally {
-      System.exit(res);
+      ExitUtil.terminate(res);
     }
   }
 
@@ -800,6 +802,10 @@ public class Gridmix extends Configured implements Tool {
      */
     void abort();
   }
-
+  // it is need for tests
+  protected Summarizer getSummarizer() {
+    return summarizer;
+  }
+  
 }
 

+ 7 - 13
hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/JobMonitor.java

@@ -86,9 +86,7 @@ class JobMonitor implements Gridmix.Component<JobStats> {
    * Add a running job's status to the polling queue.
    */
   public void add(JobStats job) throws InterruptedException {
-    synchronized (runningJobs) {
       runningJobs.put(job);
-    }
   }
 
   /**
@@ -147,12 +145,10 @@ class JobMonitor implements Gridmix.Component<JobStats> {
       boolean shutdown;
       while (true) {
         try {
-          synchronized (runningJobs) {
-            synchronized (mJobs) {
-              graceful = JobMonitor.this.graceful;
-              shutdown = JobMonitor.this.shutdown;
-              runningJobs.drainTo(mJobs);
-            }
+          synchronized (mJobs) {
+            graceful = JobMonitor.this.graceful;
+            shutdown = JobMonitor.this.shutdown;
+            runningJobs.drainTo(mJobs);
           }
 
           // shutdown conditions; either shutdown requested and all jobs
@@ -160,11 +156,9 @@ class JobMonitor implements Gridmix.Component<JobStats> {
           // submitted jobs not in the monitored set
           if (shutdown) {
             if (!graceful) {
-              synchronized (runningJobs) {
-                while (!runningJobs.isEmpty()) {
-                  synchronized (mJobs) {
-                    runningJobs.drainTo(mJobs);
-                  }
+              while (!runningJobs.isEmpty()) {
+                synchronized (mJobs) {
+                  runningJobs.drainTo(mJobs);
                 }
               }
               break;

+ 4 - 0
hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/SerialJobFactory.java

@@ -175,4 +175,8 @@ public class SerialJobFactory extends JobFactory<JobStats> {
     LOG.info(" Starting Serial submission ");
     this.rThread.start();
   }
+  // it is need for test 
+  void setDistCacheEmulator(DistributedCacheEmulator e) {
+    jobCreator.setDistCacheEmulator(e);
+  }
 }

+ 4 - 4
hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/SleepJob.java

@@ -84,7 +84,7 @@ public class SleepJob extends GridmixJob {
       String[] hosts) throws IOException {
     super(conf, submissionMillis, jobdesc, outRoot, ugi, seq);
     this.fakeLocations = numLocations;
-    this.hosts = hosts;
+    this.hosts = hosts.clone();
     this.selector = (fakeLocations > 0)? new Selector(hosts.length, (float) fakeLocations
         / hosts.length, rand.get()) : null;
     this.mapTasksOnly = conf.getBoolean(SLEEPJOB_MAPTASK_ONLY, false);
@@ -289,9 +289,9 @@ public class SleepJob extends GridmixJob {
       this.id = id;
       this.sleepDuration = sleepDuration;
       nSpec = reduceDurations.length;
-      this.reduceDurations = reduceDurations;
+      this.reduceDurations = reduceDurations.clone();
       this.nMaps = nMaps;
-      this.locations = locations;
+      this.locations = locations.clone();
     }
 
     @Override
@@ -349,7 +349,7 @@ public class SleepJob extends GridmixJob {
 
     @Override
     public String[] getLocations() {
-      return locations;
+      return locations.clone();
     }
   }
 

+ 3 - 2
hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/emulators/resourceusage/TotalHeapUsageEmulatorPlugin.java

@@ -93,7 +93,7 @@ implements ResourceUsageEmulatorPlugin {
   
   private static final float DEFAULT_HEAP_LOAD_RATIO = 0.1F;
   
-  public static int ONE_MB = 1024 * 1024;
+  public static final int ONE_MB = 1024 * 1024;
   
   /**
    * Defines the core heap usage emulation algorithm. This engine is expected
@@ -129,7 +129,8 @@ implements ResourceUsageEmulatorPlugin {
   public static class DefaultHeapUsageEmulator 
   implements HeapUsageEmulatorCore {
     // store the unit loads in a list
-    protected static ArrayList<Object> heapSpace = new ArrayList<Object>();
+    protected static final ArrayList<Object> heapSpace =
+        new ArrayList<Object>();
     
     /**
      * Increase heap usage by current process by the given amount.

+ 384 - 0
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/CommonJobTest.java

@@ -0,0 +1,384 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.gridmix;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.IOException;
+import java.text.DecimalFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.mapred.Counters;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobID;
+import org.apache.hadoop.mapred.TaskReport;
+import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.TaskCounter;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.tools.rumen.TaskInfo;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.tools.rumen.JobStory;
+import org.apache.hadoop.util.ToolRunner;
+
+public class CommonJobTest {
+  public static final Log LOG = LogFactory.getLog(Gridmix.class);
+
+  protected static int NJOBS = 2;
+  protected static final long GENDATA = 1; // in megabytes
+  protected static GridmixJobSubmissionPolicy policy = GridmixJobSubmissionPolicy.REPLAY;
+  private static File workspace = new File("target" + File.separator
+          + TestGridmixSubmission.class.getName() + "-test");
+
+  static class DebugGridmix extends Gridmix {
+
+    private JobFactory<?> factory;
+    private TestMonitor monitor;
+
+    @Override
+    protected JobMonitor createJobMonitor(Statistics stats, Configuration conf)
+            throws IOException {
+      monitor = new TestMonitor(3, stats);
+      return monitor;
+    }
+
+    @Override
+    protected JobFactory<?> createJobFactory(JobSubmitter submitter,
+                                             String traceIn, Path scratchDir, Configuration conf,
+                                             CountDownLatch startFlag, UserResolver userResolver) throws IOException {
+      factory = DebugJobFactory.getFactory(submitter, scratchDir, NJOBS, conf,
+              startFlag, userResolver);
+      return factory;
+    }
+
+    public void checkMonitor() throws Exception {
+      monitor.verify(((DebugJobFactory.Debuggable) factory).getSubmitted());
+    }
+  }
+
+  static class TestMonitor extends JobMonitor {
+    private final BlockingQueue<Job> retiredJobs;
+    private final int expected;
+    static final long SLOPBYTES = 1024;
+
+    public TestMonitor(int expected, Statistics stats) {
+      super(3, TimeUnit.SECONDS, stats, 1);
+      this.expected = expected;
+      retiredJobs = new LinkedBlockingQueue<Job>();
+    }
+
+    @Override
+    protected void onSuccess(Job job) {
+      LOG.info(" Job Success " + job);
+      retiredJobs.add(job);
+    }
+
+    @Override
+    protected void onFailure(Job job) {
+      fail("Job failure: " + job);
+    }
+
+    public void verify(ArrayList<JobStory> submitted) throws Exception {
+      assertEquals("Bad job count", expected, retiredJobs.size());
+
+      final ArrayList<Job> succeeded = new ArrayList<Job>();
+      assertEquals("Bad job count", expected, retiredJobs.drainTo(succeeded));
+      final HashMap<String, JobStory> sub = new HashMap<String, JobStory>();
+      for (JobStory spec : submitted) {
+        sub.put(spec.getJobID().toString(), spec);
+      }
+      for (Job job : succeeded) {
+        final String jobName = job.getJobName();
+        Configuration configuration = job.getConfiguration();
+        if (GenerateData.JOB_NAME.equals(jobName)) {
+          RemoteIterator<LocatedFileStatus> rit = GridmixTestUtils.dfs
+                  .listFiles(new Path("/"), true);
+          while (rit.hasNext()) {
+            System.out.println(rit.next().toString());
+          }
+          final Path in = new Path("foo").makeQualified(
+                  GridmixTestUtils.dfs.getUri(),
+                  GridmixTestUtils.dfs.getWorkingDirectory());
+          // data was compressed. All files = compressed test size+ logs= 1000000/2 + logs
+          final ContentSummary generated = GridmixTestUtils.dfs
+                  .getContentSummary(in);
+          assertEquals(550000, generated.getLength(), 10000);
+
+          Counter counter = job.getCounters()
+                  .getGroup("org.apache.hadoop.mapreduce.FileSystemCounter")
+                  .findCounter("HDFS_BYTES_WRITTEN");
+
+          assertEquals(generated.getLength(), counter.getValue());
+
+          continue;
+        } else if (GenerateDistCacheData.JOB_NAME.equals(jobName)) {
+          continue;
+        }
+
+        final String originalJobId = configuration.get(Gridmix.ORIGINAL_JOB_ID);
+        final JobStory spec = sub.get(originalJobId);
+        assertNotNull("No spec for " + jobName, spec);
+        assertNotNull("No counters for " + jobName, job.getCounters());
+        final String originalJobName = spec.getName();
+        System.out.println("originalJobName=" + originalJobName
+                + ";GridmixJobName=" + jobName + ";originalJobID=" + originalJobId);
+        assertTrue("Original job name is wrong.",
+                originalJobName.equals(configuration.get(Gridmix.ORIGINAL_JOB_NAME)));
+
+        // Gridmix job seqNum contains 6 digits
+        int seqNumLength = 6;
+        String jobSeqNum = new DecimalFormat("000000").format(configuration.getInt(
+                GridmixJob.GRIDMIX_JOB_SEQ, -1));
+        // Original job name is of the format MOCKJOB<6 digit sequence number>
+        // because MockJob jobNames are of this format.
+        assertTrue(originalJobName.substring(
+                originalJobName.length() - seqNumLength).equals(jobSeqNum));
+
+        assertTrue("Gridmix job name is not in the expected format.",
+                jobName.equals(GridmixJob.JOB_NAME_PREFIX + jobSeqNum));
+        final FileStatus stat = GridmixTestUtils.dfs.getFileStatus(new Path(
+                GridmixTestUtils.DEST, "" + Integer.valueOf(jobSeqNum)));
+        assertEquals("Wrong owner for " + jobName, spec.getUser(),
+                stat.getOwner());
+        final int nMaps = spec.getNumberMaps();
+        final int nReds = spec.getNumberReduces();
+
+        final JobClient client = new JobClient(
+                GridmixTestUtils.mrvl.getConfig());
+        final TaskReport[] mReports = client.getMapTaskReports(JobID
+                .downgrade(job.getJobID()));
+        assertEquals("Mismatched map count", nMaps, mReports.length);
+        check(TaskType.MAP, spec, mReports, 0, 0, SLOPBYTES, nReds);
+
+        final TaskReport[] rReports = client.getReduceTaskReports(JobID
+                .downgrade(job.getJobID()));
+        assertEquals("Mismatched reduce count", nReds, rReports.length);
+        check(TaskType.REDUCE, spec, rReports, nMaps * SLOPBYTES, 2 * nMaps, 0,
+                0);
+
+      }
+
+    }
+    // Verify if correct job queue is used
+    private void check(final TaskType type, JobStory spec,
+                       final TaskReport[] runTasks, long extraInputBytes,
+                       int extraInputRecords, long extraOutputBytes, int extraOutputRecords)
+            throws Exception {
+
+      long[] runInputRecords = new long[runTasks.length];
+      long[] runInputBytes = new long[runTasks.length];
+      long[] runOutputRecords = new long[runTasks.length];
+      long[] runOutputBytes = new long[runTasks.length];
+      long[] specInputRecords = new long[runTasks.length];
+      long[] specInputBytes = new long[runTasks.length];
+      long[] specOutputRecords = new long[runTasks.length];
+      long[] specOutputBytes = new long[runTasks.length];
+
+      for (int i = 0; i < runTasks.length; ++i) {
+        final TaskInfo specInfo;
+        final Counters counters = runTasks[i].getCounters();
+        switch (type) {
+          case MAP:
+            runInputBytes[i] = counters.findCounter("FileSystemCounters",
+                    "HDFS_BYTES_READ").getValue()
+                    - counters.findCounter(TaskCounter.SPLIT_RAW_BYTES).getValue();
+            runInputRecords[i] = (int) counters.findCounter(
+                    TaskCounter.MAP_INPUT_RECORDS).getValue();
+            runOutputBytes[i] = counters
+                    .findCounter(TaskCounter.MAP_OUTPUT_BYTES).getValue();
+            runOutputRecords[i] = (int) counters.findCounter(
+                    TaskCounter.MAP_OUTPUT_RECORDS).getValue();
+
+            specInfo = spec.getTaskInfo(TaskType.MAP, i);
+            specInputRecords[i] = specInfo.getInputRecords();
+            specInputBytes[i] = specInfo.getInputBytes();
+            specOutputRecords[i] = specInfo.getOutputRecords();
+            specOutputBytes[i] = specInfo.getOutputBytes();
+
+            LOG.info(String.format(type + " SPEC: %9d -> %9d :: %5d -> %5d\n",
+                    specInputBytes[i], specOutputBytes[i], specInputRecords[i],
+                    specOutputRecords[i]));
+            LOG.info(String.format(type + " RUN:  %9d -> %9d :: %5d -> %5d\n",
+                    runInputBytes[i], runOutputBytes[i], runInputRecords[i],
+                    runOutputRecords[i]));
+            break;
+          case REDUCE:
+            runInputBytes[i] = 0;
+            runInputRecords[i] = (int) counters.findCounter(
+                    TaskCounter.REDUCE_INPUT_RECORDS).getValue();
+            runOutputBytes[i] = counters.findCounter("FileSystemCounters",
+                    "HDFS_BYTES_WRITTEN").getValue();
+            runOutputRecords[i] = (int) counters.findCounter(
+                    TaskCounter.REDUCE_OUTPUT_RECORDS).getValue();
+
+            specInfo = spec.getTaskInfo(TaskType.REDUCE, i);
+            // There is no reliable counter for reduce input bytes. The
+            // variable-length encoding of intermediate records and other noise
+            // make this quantity difficult to estimate. The shuffle and spec
+            // input bytes are included in debug output for reference, but are
+            // not checked
+            specInputBytes[i] = 0;
+            specInputRecords[i] = specInfo.getInputRecords();
+            specOutputRecords[i] = specInfo.getOutputRecords();
+            specOutputBytes[i] = specInfo.getOutputBytes();
+            LOG.info(String.format(type + " SPEC: (%9d) -> %9d :: %5d -> %5d\n",
+                    specInfo.getInputBytes(), specOutputBytes[i],
+                    specInputRecords[i], specOutputRecords[i]));
+            LOG.info(String
+                    .format(type + " RUN:  (%9d) -> %9d :: %5d -> %5d\n", counters
+                            .findCounter(TaskCounter.REDUCE_SHUFFLE_BYTES).getValue(),
+                            runOutputBytes[i], runInputRecords[i], runOutputRecords[i]));
+            break;
+          default:
+            fail("Unexpected type: " + type);
+        }
+      }
+
+      // Check input bytes
+      Arrays.sort(specInputBytes);
+      Arrays.sort(runInputBytes);
+      for (int i = 0; i < runTasks.length; ++i) {
+        assertTrue("Mismatched " + type + " input bytes " + specInputBytes[i]
+                + "/" + runInputBytes[i],
+                eqPlusMinus(runInputBytes[i], specInputBytes[i], extraInputBytes));
+      }
+
+      // Check input records
+      Arrays.sort(specInputRecords);
+      Arrays.sort(runInputRecords);
+      for (int i = 0; i < runTasks.length; ++i) {
+        assertTrue(
+                "Mismatched " + type + " input records " + specInputRecords[i]
+                        + "/" + runInputRecords[i],
+                eqPlusMinus(runInputRecords[i], specInputRecords[i],
+                        extraInputRecords));
+      }
+
+      // Check output bytes
+      Arrays.sort(specOutputBytes);
+      Arrays.sort(runOutputBytes);
+      for (int i = 0; i < runTasks.length; ++i) {
+        assertTrue(
+                "Mismatched " + type + " output bytes " + specOutputBytes[i] + "/"
+                        + runOutputBytes[i],
+                eqPlusMinus(runOutputBytes[i], specOutputBytes[i], extraOutputBytes));
+      }
+
+      // Check output records
+      Arrays.sort(specOutputRecords);
+      Arrays.sort(runOutputRecords);
+      for (int i = 0; i < runTasks.length; ++i) {
+        assertTrue(
+                "Mismatched " + type + " output records " + specOutputRecords[i]
+                        + "/" + runOutputRecords[i],
+                eqPlusMinus(runOutputRecords[i], specOutputRecords[i],
+                        extraOutputRecords));
+      }
+
+    }
+
+    private static boolean eqPlusMinus(long a, long b, long x) {
+      final long diff = Math.abs(a - b);
+      return diff <= x;
+    }
+
+  }
+
+  protected void doSubmission(String jobCreatorName, boolean defaultOutputPath)
+          throws Exception {
+    final Path in = new Path("foo").makeQualified(
+            GridmixTestUtils.dfs.getUri(),
+            GridmixTestUtils.dfs.getWorkingDirectory());
+    final Path out = GridmixTestUtils.DEST.makeQualified(
+            GridmixTestUtils.dfs.getUri(),
+            GridmixTestUtils.dfs.getWorkingDirectory());
+    final Path root = new Path(workspace.getAbsolutePath());
+    if (!workspace.exists()) {
+      assertTrue(workspace.mkdirs());
+    }
+    Configuration conf = null;
+
+    try {
+      ArrayList<String> argsList = new ArrayList<String>();
+
+      argsList.add("-D" + FilePool.GRIDMIX_MIN_FILE + "=0");
+      argsList.add("-D" + Gridmix.GRIDMIX_USR_RSV + "="
+              + EchoUserResolver.class.getName());
+      if (jobCreatorName != null) {
+        argsList.add("-D" + JobCreator.GRIDMIX_JOB_TYPE + "=" + jobCreatorName);
+      }
+
+      // Set the config property gridmix.output.directory only if
+      // defaultOutputPath is false. If defaultOutputPath is true, then
+      // let us allow gridmix to use the path foo/gridmix/ as output dir.
+      if (!defaultOutputPath) {
+        argsList.add("-D" + Gridmix.GRIDMIX_OUT_DIR + "=" + out);
+      }
+      argsList.add("-generate");
+      argsList.add(String.valueOf(GENDATA) + "m");
+      argsList.add(in.toString());
+      argsList.add("-"); // ignored by DebugGridmix
+
+      String[] argv = argsList.toArray(new String[argsList.size()]);
+
+      DebugGridmix client = new DebugGridmix();
+      conf = GridmixTestUtils.mrvl.getConfig();
+
+      CompressionEmulationUtil.setCompressionEmulationEnabled(conf, true);
+      conf.setEnum(GridmixJobSubmissionPolicy.JOB_SUBMISSION_POLICY, policy);
+
+      conf.setBoolean(GridmixJob.GRIDMIX_USE_QUEUE_IN_TRACE, true);
+      UserGroupInformation ugi = UserGroupInformation.getLoginUser();
+      conf.set(MRJobConfig.USER_NAME, ugi.getUserName());
+
+      // allow synthetic users to create home directories
+      GridmixTestUtils.dfs.mkdirs(root, new FsPermission((short) 777));
+      GridmixTestUtils.dfs.setPermission(root, new FsPermission((short) 777));
+
+      int res = ToolRunner.run(conf, client, argv);
+      assertEquals("Client exited with nonzero status", 0, res);
+      client.checkMonitor();
+    } catch (Exception e) {
+      e.printStackTrace();
+    } finally {
+      in.getFileSystem(conf).delete(in, true);
+      out.getFileSystem(conf).delete(out, true);
+      root.getFileSystem(conf).delete(root, true);
+    }
+  }
+}

+ 2 - 2
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/DebugJobFactory.java

@@ -29,13 +29,13 @@ import java.util.concurrent.CountDownLatch;
 /**
  * Component generating random job traces for testing on a single node.
  */
-class DebugJobFactory {
+public class DebugJobFactory {
 
   interface Debuggable {
     ArrayList<JobStory> getSubmitted();
   }
 
-  public static JobFactory getFactory(
+  public static JobFactory<?> getFactory(
     JobSubmitter submitter, Path scratch, int numJobs, Configuration conf,
     CountDownLatch startFlag, UserResolver resolver) throws IOException {
     GridmixJobSubmissionPolicy policy = GridmixJobSubmissionPolicy.getPolicy(

+ 2 - 2
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/DebugJobProducer.java

@@ -216,7 +216,7 @@ public class DebugJobProducer implements JobStoryProducer {
      if (user == null) {
        user = String.format("foobar%d", id);
      }
-     GridmixTestUtils.createHomeAndStagingDirectory(user, (JobConf)conf);
+     GridmixTestUtils.createHomeAndStagingDirectory(user, conf);
      return user;
    }
 
@@ -300,7 +300,7 @@ public class DebugJobProducer implements JobStoryProducer {
 
     @Override
     public String getQueueName() {
-      String qName = "q" + ((id % 2) + 1);
+      String qName = "default";
       return qName;
     }
     

+ 59 - 36
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/GridmixTestUtils.java

@@ -4,55 +4,76 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.mapred.MiniMRCluster;
-import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.MiniMRClientCluster;
+import org.apache.hadoop.mapred.MiniMRClientClusterFactory;
+import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.ShellBasedUnixGroupsMapping;
-import org.apache.hadoop.security.Groups;
 
 import java.io.IOException;
 
 /**
- * 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
+ * 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
  * <p/>
  * http://www.apache.org/licenses/LICENSE-2.0
  * <p/>
  * 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.
+ * 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.
  */
 public class GridmixTestUtils {
   private static final Log LOG = LogFactory.getLog(GridmixTestUtils.class);
   static final Path DEST = new Path("/gridmix");
   static FileSystem dfs = null;
   static MiniDFSCluster dfsCluster = null;
-  static MiniMRCluster mrCluster = null;
+  static MiniMRClientCluster mrvl = null;
+  protected static final String GRIDMIX_USE_QUEUE_IN_TRACE = 
+      "gridmix.job-submission.use-queue-in-trace";
+  protected static final String GRIDMIX_DEFAULT_QUEUE = 
+      "gridmix.job-submission.default-queue";
 
-  public static void initCluster() throws IOException {
+  public static void initCluster(Class<?> caller) throws IOException {
     Configuration conf = new Configuration();
-    conf.set("mapred.queue.names", "default,q1,q2");
-    dfsCluster = new MiniDFSCluster(conf, 3, true, null);
+//    conf.set("mapred.queue.names", "default,q1,q2");
+  conf.set("mapred.queue.names", "default");
+    conf.set("yarn.scheduler.capacity.root.queues", "default");
+    conf.set("yarn.scheduler.capacity.root.default.capacity", "100.0");
+    
+    
+    conf.setBoolean(GRIDMIX_USE_QUEUE_IN_TRACE, false);
+    conf.set(GRIDMIX_DEFAULT_QUEUE, "default");
+    
+
+    dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).format(true)
+        .build();// MiniDFSCluster(conf, 3, true, null);
     dfs = dfsCluster.getFileSystem();
     conf.set(JTConfig.JT_RETIREJOBS, "false");
-    mrCluster = new MiniMRCluster(3, dfs.getUri().toString(), 1, null, null, 
-                                  new JobConf(conf));
+    mrvl = MiniMRClientClusterFactory.create(caller, 2, conf);
+    
+    conf = mrvl.getConfig();
+    String[] files = conf.getStrings(MRJobConfig.CACHE_FILES);
+    if (files != null) {
+      String[] timestamps = new String[files.length];
+      for (int i = 0; i < files.length; i++) {
+        timestamps[i] = Long.toString(System.currentTimeMillis());
+      }
+      conf.setStrings(MRJobConfig.CACHE_FILE_TIMESTAMPS, timestamps);
+    }
+    
   }
 
   public static void shutdownCluster() throws IOException {
-    if (mrCluster != null) {
-      mrCluster.shutdown();
+    if (mrvl != null) {
+      mrvl.stop();
     }
     if (dfsCluster != null) {
       dfsCluster.shutdown();
@@ -61,23 +82,25 @@ public class GridmixTestUtils {
 
   /**
    * Methods to generate the home directory for dummy users.
-   *
+   * 
    * @param conf
    */
-  public static void createHomeAndStagingDirectory(String user, JobConf conf) {
+  public static void createHomeAndStagingDirectory(String user,
+      Configuration conf) {
     try {
       FileSystem fs = dfsCluster.getFileSystem();
       String path = "/user/" + user;
       Path homeDirectory = new Path(path);
-      if(fs.exists(homeDirectory)) {
-        fs.delete(homeDirectory,true);
-      }
-      LOG.info("Creating Home directory : " + homeDirectory);
-      fs.mkdirs(homeDirectory);
-      changePermission(user,homeDirectory, fs);
-      Path stagingArea = 
-        new Path(conf.get("mapreduce.jobtracker.staging.root.dir",
-                          "/tmp/hadoop/mapred/staging"));
+      if (!fs.exists(homeDirectory)) {
+        LOG.info("Creating Home directory : " + homeDirectory);
+        fs.mkdirs(homeDirectory);
+        changePermission(user, homeDirectory, fs);
+
+      }    
+      changePermission(user, homeDirectory, fs);
+      Path stagingArea = new Path(
+          conf.get("mapreduce.jobtracker.staging.root.dir",
+              "/tmp/hadoop/mapred/staging"));
       LOG.info("Creating Staging root directory : " + stagingArea);
       fs.mkdirs(stagingArea);
       fs.setPermission(stagingArea, new FsPermission((short) 0777));
@@ -87,7 +110,7 @@ public class GridmixTestUtils {
   }
 
   static void changePermission(String user, Path homeDirectory, FileSystem fs)
-    throws IOException {
+      throws IOException {
     fs.setOwner(homeDirectory, user, "");
   }
 }

+ 430 - 0
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestDistCacheEmulation.java

@@ -0,0 +1,430 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.gridmix;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.MapContext;
+import org.apache.hadoop.mapreduce.MapReduceTestUtil;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.hadoop.mapreduce.task.MapContextImpl;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Validate emulation of distributed cache load in gridmix simulated jobs.
+ * 
+ */
+public class TestDistCacheEmulation {
+
+  private DistributedCacheEmulator dce = null;
+
+  @BeforeClass
+  public static void init() throws IOException {
+    GridmixTestUtils.initCluster(TestDistCacheEmulation.class);
+    File target=new File("target"+File.separator+TestDistCacheEmulation.class.getName());
+    if(!target.exists()){
+      assertTrue(target.mkdirs());
+    }
+    
+  }
+
+  @AfterClass
+  public static void shutDown() throws IOException {
+    GridmixTestUtils.shutdownCluster();
+  }
+
+  /**
+   * Validate the dist cache files generated by GenerateDistCacheData job.
+   * 
+   * @param jobConf
+   *          configuration of GenerateDistCacheData job.
+   * @param sortedFileSizes
+   *          array of sorted distributed cache file sizes
+   * @throws IOException
+   * @throws FileNotFoundException
+   */
+  private void validateDistCacheData(Configuration jobConf,
+      long[] sortedFileSizes) throws FileNotFoundException, IOException {
+    Path distCachePath = dce.getDistributedCacheDir();
+    String filesListFile = jobConf
+        .get(GenerateDistCacheData.GRIDMIX_DISTCACHE_FILE_LIST);
+    FileSystem fs = FileSystem.get(jobConf);
+
+    // Validate the existence of Distributed Cache files list file directly
+    // under distributed cache directory
+    Path listFile = new Path(filesListFile);
+    assertTrue("Path of Distributed Cache files list file is wrong.",
+        distCachePath.equals(listFile.getParent().makeQualified(fs.getUri(), fs.getWorkingDirectory())));
+
+    // Delete the dist cache files list file
+    assertTrue(
+        "Failed to delete distributed Cache files list file " + listFile,
+        fs.delete(listFile,true));
+
+    List<Long> fileSizes = new ArrayList<Long>();
+    for (long size : sortedFileSizes) {
+      fileSizes.add(size);
+    }
+    // validate dist cache files after deleting the 'files list file'
+    validateDistCacheFiles(fileSizes, distCachePath);
+  }
+
+  /**
+   * Validate private/public distributed cache files.
+   * 
+   * @param filesSizesExpected
+   *          list of sizes of expected dist cache files
+   * @param distCacheDir
+   *          the distributed cache dir to be validated
+   * @throws IOException
+   * @throws FileNotFoundException
+   */
+  private void validateDistCacheFiles(List<Long> filesSizesExpected, Path distCacheDir)
+      throws FileNotFoundException, IOException {
+    // RemoteIterator<LocatedFileStatus> iter =
+    FileStatus[] statuses = GridmixTestUtils.dfs.listStatus(distCacheDir);
+    int numFiles = filesSizesExpected.size();
+    assertEquals("Number of files under distributed cache dir is wrong.",
+        numFiles, statuses.length);
+    for (int i = 0; i < numFiles; i++) {
+      FileStatus stat = statuses[i];
+      assertTrue("File size of distributed cache file "
+          + stat.getPath().toUri().getPath() + " is wrong.",
+          filesSizesExpected.remove(stat.getLen()));
+
+      FsPermission perm = stat.getPermission();
+      assertEquals("Wrong permissions for distributed cache file "
+          + stat.getPath().toUri().getPath(), new FsPermission((short) 0644),
+          perm);
+    }
+  }
+
+  /**
+   * Configures 5 HDFS-based dist cache files and 1 local-FS-based dist cache
+   * file in the given Configuration object <code>conf</code>.
+   * 
+   * @param conf
+   *          configuration where dist cache config properties are to be set
+   * @return array of sorted HDFS-based distributed cache file sizes
+   * @throws IOException
+   */
+  private long[] configureDummyDistCacheFiles(Configuration conf)
+      throws IOException {
+    String user = UserGroupInformation.getCurrentUser().getShortUserName();
+    conf.set("user.name", user);
+    
+    // Set some dummy dist cache files in gridmix configuration so that they go
+    // into the configuration of JobStory objects.
+    String[] distCacheFiles = { "hdfs:///tmp/file1.txt",
+        "/tmp/" + user + "/.staging/job_1/file2.txt",
+        "hdfs:///user/user1/file3.txt", "/home/user2/file4.txt",
+        "subdir1/file5.txt", "subdir2/file6.gz" };
+
+    String[] fileSizes = { "400", "2500", "700", "1200", "1500", "500" };
+
+    String[] visibilities = { "true", "false", "false", "true", "true", "false" };
+    String[] timeStamps = { "1234", "2345", "34567", "5434", "125", "134" };
+
+    // DistributedCache.setCacheFiles(fileCaches, conf);
+    conf.setStrings(MRJobConfig.CACHE_FILES, distCacheFiles);
+    conf.setStrings(MRJobConfig.CACHE_FILES_SIZES, fileSizes);
+    conf.setStrings(JobContext.CACHE_FILE_VISIBILITIES, visibilities);
+    conf.setStrings(MRJobConfig.CACHE_FILE_TIMESTAMPS, timeStamps);
+
+    // local FS based dist cache file whose path contains <user>/.staging is
+    // not created on HDFS. So file size 2500 is not added to sortedFileSizes.
+    long[] sortedFileSizes = new long[] { 1500, 1200, 700, 500, 400 };
+    return sortedFileSizes;
+  }
+
+  /**
+   * Runs setupGenerateDistCacheData() on a new DistrbutedCacheEmulator and and
+   * returns the jobConf. Fills the array <code>sortedFileSizes</code> that can
+   * be used for validation. Validation of exit code from
+   * setupGenerateDistCacheData() is done.
+   * 
+   * @param generate
+   *          true if -generate option is specified
+   * @param sortedFileSizes
+   *          sorted HDFS-based distributed cache file sizes
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private Configuration runSetupGenerateDistCacheData(boolean generate,
+      long[] sortedFileSizes) throws IOException, InterruptedException {
+    Configuration conf = new Configuration();
+    long[] fileSizes = configureDummyDistCacheFiles(conf);
+    System.arraycopy(fileSizes, 0, sortedFileSizes, 0, fileSizes.length);
+
+    // Job stories of all 3 jobs will have same dist cache files in their
+    // configurations
+    final int numJobs = 3;
+    DebugJobProducer jobProducer = new DebugJobProducer(numJobs, conf);
+
+    Configuration jobConf = GridmixTestUtils.mrvl.getConfig();
+    Path ioPath = new Path("testSetupGenerateDistCacheData")
+        .makeQualified(GridmixTestUtils.dfs.getUri(),GridmixTestUtils.dfs.getWorkingDirectory());
+    FileSystem fs = FileSystem.get(jobConf);
+    if (fs.exists(ioPath)) {
+      fs.delete(ioPath, true);
+    }
+    FileSystem.mkdirs(fs, ioPath, new FsPermission((short) 0777));
+
+    dce = createDistributedCacheEmulator(jobConf, ioPath, generate);
+    int exitCode = dce.setupGenerateDistCacheData(jobProducer);
+    int expectedExitCode = generate ? 0
+        : Gridmix.MISSING_DIST_CACHE_FILES_ERROR;
+    assertEquals("setupGenerateDistCacheData failed.", expectedExitCode,
+        exitCode);
+
+    // reset back
+    resetDistCacheConfigProperties(jobConf);
+    return jobConf;
+  }
+
+  /**
+   * Reset the config properties related to Distributed Cache in the given job
+   * configuration <code>jobConf</code>.
+   * 
+   * @param jobConf
+   *          job configuration
+   */
+  private void resetDistCacheConfigProperties(Configuration jobConf) {
+    // reset current/latest property names
+    jobConf.setStrings(MRJobConfig.CACHE_FILES, "");
+    jobConf.setStrings(MRJobConfig.CACHE_FILES_SIZES, "");
+    jobConf.setStrings(MRJobConfig.CACHE_FILE_TIMESTAMPS, "");
+    jobConf.setStrings(JobContext.CACHE_FILE_VISIBILITIES, "");
+    // reset old property names
+    jobConf.setStrings("mapred.cache.files", "");
+    jobConf.setStrings("mapred.cache.files.filesizes", "");
+    jobConf.setStrings("mapred.cache.files.visibilities", "");
+    jobConf.setStrings("mapred.cache.files.timestamps", "");
+  }
+
+  /**
+   * Validate GenerateDistCacheData job if it creates dist cache files properly.
+   * 
+   * @throws Exception
+   */
+  @Test (timeout=200000)
+  public void testGenerateDistCacheData() throws Exception {
+    long[] sortedFileSizes = new long[5];
+    Configuration jobConf = runSetupGenerateDistCacheData(true, sortedFileSizes);
+    GridmixJob gridmixJob = new GenerateDistCacheData(jobConf);
+    Job job = gridmixJob.call();
+    assertEquals("Number of reduce tasks in GenerateDistCacheData is not 0.",
+        0, job.getNumReduceTasks());
+    assertTrue("GenerateDistCacheData job failed.",
+        job.waitForCompletion(false));
+    validateDistCacheData(jobConf, sortedFileSizes);
+  }
+
+  /**
+   * Validate setupGenerateDistCacheData by validating <li>permissions of the
+   * distributed cache directories and <li>content of the generated sequence
+   * file. This includes validation of dist cache file paths and their file
+   * sizes.
+   */
+  private void validateSetupGenDC(Configuration jobConf, long[] sortedFileSizes)
+      throws IOException, InterruptedException {
+    // build things needed for validation
+    long sumOfFileSizes = 0;
+    for (int i = 0; i < sortedFileSizes.length; i++) {
+      sumOfFileSizes += sortedFileSizes[i];
+    }
+
+    FileSystem fs = FileSystem.get(jobConf);
+    assertEquals("Number of distributed cache files to be generated is wrong.",
+        sortedFileSizes.length,
+        jobConf.getInt(GenerateDistCacheData.GRIDMIX_DISTCACHE_FILE_COUNT, -1));
+    assertEquals("Total size of dist cache files to be generated is wrong.",
+        sumOfFileSizes,
+        jobConf.getLong(GenerateDistCacheData.GRIDMIX_DISTCACHE_BYTE_COUNT, -1));
+    Path filesListFile = new Path(
+        jobConf.get(GenerateDistCacheData.GRIDMIX_DISTCACHE_FILE_LIST));
+    FileStatus stat = fs.getFileStatus(filesListFile);
+    assertEquals("Wrong permissions of dist Cache files list file "
+        + filesListFile, new FsPermission((short) 0644), stat.getPermission());
+
+    InputSplit split = new FileSplit(filesListFile, 0, stat.getLen(),
+        (String[]) null);
+    TaskAttemptContext taskContext = MapReduceTestUtil
+        .createDummyMapTaskAttemptContext(jobConf);
+    RecordReader<LongWritable, BytesWritable> reader = new GenerateDistCacheData.GenDCDataFormat()
+        .createRecordReader(split, taskContext);
+    MapContext<LongWritable, BytesWritable, NullWritable, BytesWritable> mapContext = new MapContextImpl<LongWritable, BytesWritable, NullWritable, BytesWritable>(
+        jobConf, taskContext.getTaskAttemptID(), reader, null, null,
+        MapReduceTestUtil.createDummyReporter(), split);
+    reader.initialize(split, mapContext);
+
+    // start validating setupGenerateDistCacheData
+    doValidateSetupGenDC(reader, fs, sortedFileSizes);
+  }
+
+  /**
+   * Validate setupGenerateDistCacheData by validating <li>permissions of the
+   * distributed cache directory and <li>content of the generated sequence file.
+   * This includes validation of dist cache file paths and their file sizes.
+   */
+  private void doValidateSetupGenDC(
+      RecordReader<LongWritable, BytesWritable> reader, FileSystem fs,
+      long[] sortedFileSizes) throws IOException, InterruptedException {
+
+    // Validate permissions of dist cache directory
+    Path distCacheDir = dce.getDistributedCacheDir();
+    assertEquals(
+        "Wrong permissions for distributed cache dir " + distCacheDir,
+        fs.getFileStatus(distCacheDir).getPermission().getOtherAction()
+            .and(FsAction.EXECUTE), FsAction.EXECUTE);
+
+    // Validate the content of the sequence file generated by
+    // dce.setupGenerateDistCacheData().
+    LongWritable key = new LongWritable();
+    BytesWritable val = new BytesWritable();
+    for (int i = 0; i < sortedFileSizes.length; i++) {
+      assertTrue("Number of files written to the sequence file by "
+          + "setupGenerateDistCacheData is less than the expected.",
+          reader.nextKeyValue());
+      key = reader.getCurrentKey();
+      val = reader.getCurrentValue();
+      long fileSize = key.get();
+      String file = new String(val.getBytes(), 0, val.getLength());
+
+      // Dist Cache files should be sorted based on file size.
+      assertEquals("Dist cache file size is wrong.", sortedFileSizes[i],
+          fileSize);
+
+      // Validate dist cache file path.
+
+      // parent dir of dist cache file
+      Path parent = new Path(file).getParent().makeQualified(fs.getUri(),fs.getWorkingDirectory());
+      // should exist in dist cache dir
+      assertTrue("Public dist cache file path is wrong.",
+          distCacheDir.equals(parent));
+    }
+  }
+
+  /**
+   * Test if DistributedCacheEmulator's setup of GenerateDistCacheData is
+   * working as expected.
+   * 
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  @Test  (timeout=20000)
+  public void testSetupGenerateDistCacheData() throws IOException,
+      InterruptedException {
+    long[] sortedFileSizes = new long[5];
+    Configuration jobConf = runSetupGenerateDistCacheData(true, sortedFileSizes);
+    validateSetupGenDC(jobConf, sortedFileSizes);
+
+    // Verify if correct exit code is seen when -generate option is missing and
+    // distributed cache files are missing in the expected path.
+    runSetupGenerateDistCacheData(false, sortedFileSizes);
+  }
+
+  /**
+   * Create DistributedCacheEmulator object and do the initialization by calling
+   * init() on it with dummy trace. Also configure the pseudo local FS.
+   */
+  private DistributedCacheEmulator createDistributedCacheEmulator(
+      Configuration conf, Path ioPath, boolean generate) throws IOException {
+    DistributedCacheEmulator dce = new DistributedCacheEmulator(conf, ioPath);
+    JobCreator jobCreator = JobCreator.getPolicy(conf, JobCreator.LOADJOB);
+    jobCreator.setDistCacheEmulator(dce);
+    dce.init("dummytrace", jobCreator, generate);
+    return dce;
+  }
+
+  /**
+   * Test the configuration property for disabling/enabling emulation of
+   * distributed cache load.
+   */
+  @Test  (timeout=2000)
+  public void testDistCacheEmulationConfigurability() throws IOException {
+    Configuration jobConf = GridmixTestUtils.mrvl.getConfig();
+    Path ioPath = new Path("testDistCacheEmulationConfigurability")
+        .makeQualified(GridmixTestUtils.dfs.getUri(),GridmixTestUtils.dfs.getWorkingDirectory());
+    FileSystem fs = FileSystem.get(jobConf);
+    FileSystem.mkdirs(fs, ioPath, new FsPermission((short) 0777));
+
+    // default config
+    dce = createDistributedCacheEmulator(jobConf, ioPath, false);
+    assertTrue("Default configuration of "
+        + DistributedCacheEmulator.GRIDMIX_EMULATE_DISTRIBUTEDCACHE
+        + " is wrong.", dce.shouldEmulateDistCacheLoad());
+
+    // config property set to false
+    jobConf.setBoolean(
+        DistributedCacheEmulator.GRIDMIX_EMULATE_DISTRIBUTEDCACHE, false);
+    dce = createDistributedCacheEmulator(jobConf, ioPath, false);
+    assertFalse("Disabling of emulation of distributed cache load by setting "
+        + DistributedCacheEmulator.GRIDMIX_EMULATE_DISTRIBUTEDCACHE
+        + " to false is not working.", dce.shouldEmulateDistCacheLoad());
+  }
+/** 
+ * test method configureDistCacheFiles
+ * 
+ */
+  @Test  (timeout=2000)
+  public void testDistCacheEmulator() throws Exception {
+
+    Configuration conf = new Configuration();
+    configureDummyDistCacheFiles(conf);
+    File ws = new File("target" + File.separator + this.getClass().getName());
+    Path ioPath = new Path(ws.getAbsolutePath());
+
+    DistributedCacheEmulator dce = new DistributedCacheEmulator(conf, ioPath);
+    JobConf jobConf = new JobConf(conf);
+    jobConf.setUser(UserGroupInformation.getCurrentUser().getShortUserName());
+    File fin=new File("src"+File.separator+"test"+File.separator+"resources"+File.separator+"data"+File.separator+"wordcount.json");
+    dce.init(fin.getAbsolutePath(), JobCreator.LOADJOB, true);
+    dce.configureDistCacheFiles(conf, jobConf);
+    
+    String[] caches=conf.getStrings(MRJobConfig.CACHE_FILES);
+    String[] tmpfiles=conf.getStrings("tmpfiles");
+    // this method should fill caches AND tmpfiles  from MRJobConfig.CACHE_FILES property 
+    assertEquals(6, ((caches==null?0:caches.length)+(tmpfiles==null?0:tmpfiles.length)));
+  }
+}

+ 989 - 0
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestGridMixClasses.java

@@ -0,0 +1,989 @@
+/**
+ * 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.gridmix;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.CustomOutputCommitter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PositionedReadable;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.RawKeyValueIterator;
+import org.apache.hadoop.mapred.gridmix.GridmixKey.Spec;
+import org.apache.hadoop.mapred.gridmix.SleepJob.SleepReducer;
+import org.apache.hadoop.mapred.gridmix.SleepJob.SleepSplit;
+import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.MapContext;
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.ReduceContext;
+import org.apache.hadoop.mapreduce.StatusReporter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.Mapper.Context;
+import org.apache.hadoop.mapreduce.counters.GenericCounter;
+import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit;
+import org.apache.hadoop.mapreduce.lib.map.WrappedMapper;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.reduce.WrappedReducer;
+import org.apache.hadoop.mapreduce.task.MapContextImpl;
+import org.apache.hadoop.mapreduce.task.ReduceContextImpl;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl.DummyReporter;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.tools.rumen.JobStory;
+import org.apache.hadoop.tools.rumen.JobStoryProducer;
+import org.apache.hadoop.tools.rumen.ResourceUsageMetrics;
+import org.apache.hadoop.tools.rumen.ZombieJobProducer;
+import org.apache.hadoop.util.Progress;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.mockito.Mockito.*;
+
+import static org.junit.Assert.*;
+
+public class TestGridMixClasses {
+  private static final Log LOG = LogFactory.getLog(TestGridMixClasses.class);
+
+  /*
+   * simple test LoadSplit (getters,copy, write, read...)
+   */
+  @Test (timeout=1000)
+  public void testLoadSplit() throws Exception {
+
+    LoadSplit test = getLoadSplit();
+
+    ByteArrayOutputStream data = new ByteArrayOutputStream();
+    DataOutputStream out = new DataOutputStream(data);
+    test.write(out);
+    LoadSplit copy = new LoadSplit();
+    copy.readFields(new DataInputStream(new ByteArrayInputStream(data
+            .toByteArray())));
+
+    // data should be the same
+    assertEquals(test.getId(), copy.getId());
+    assertEquals(test.getMapCount(), copy.getMapCount());
+    assertEquals(test.getInputRecords(), copy.getInputRecords());
+
+    assertEquals(test.getOutputBytes()[0], copy.getOutputBytes()[0]);
+    assertEquals(test.getOutputRecords()[0], copy.getOutputRecords()[0]);
+    assertEquals(test.getReduceBytes(0), copy.getReduceBytes(0));
+    assertEquals(test.getReduceRecords(0), copy.getReduceRecords(0));
+    assertEquals(test.getMapResourceUsageMetrics().getCumulativeCpuUsage(),
+            copy.getMapResourceUsageMetrics().getCumulativeCpuUsage());
+    assertEquals(test.getReduceResourceUsageMetrics(0).getCumulativeCpuUsage(),
+            copy.getReduceResourceUsageMetrics(0).getCumulativeCpuUsage());
+
+  }
+
+  /*
+   * simple test GridmixSplit (copy, getters, write, read..)
+   */
+  @Test (timeout=1000)
+  public void testGridmixSplit() throws Exception {
+    Path[] files = {new Path("one"), new Path("two")};
+    long[] start = {1, 2};
+    long[] lengths = {100, 200};
+    String[] locations = {"locOne", "loctwo"};
+
+    CombineFileSplit cfSplit = new CombineFileSplit(files, start, lengths,
+            locations);
+    ResourceUsageMetrics metrics = new ResourceUsageMetrics();
+    metrics.setCumulativeCpuUsage(200);
+
+    double[] reduceBytes = {8.1d, 8.2d};
+    double[] reduceRecords = {9.1d, 9.2d};
+    long[] reduceOutputBytes = {101L, 102L};
+    long[] reduceOutputRecords = {111L, 112L};
+
+    GridmixSplit test = new GridmixSplit(cfSplit, 2, 3, 4L, 5L, 6L, 7L,
+            reduceBytes, reduceRecords, reduceOutputBytes, reduceOutputRecords);
+
+    ByteArrayOutputStream data = new ByteArrayOutputStream();
+    DataOutputStream out = new DataOutputStream(data);
+    test.write(out);
+    GridmixSplit copy = new GridmixSplit();
+    copy.readFields(new DataInputStream(new ByteArrayInputStream(data
+            .toByteArray())));
+
+    // data should be the same
+    assertEquals(test.getId(), copy.getId());
+    assertEquals(test.getMapCount(), copy.getMapCount());
+    assertEquals(test.getInputRecords(), copy.getInputRecords());
+
+    assertEquals(test.getOutputBytes()[0], copy.getOutputBytes()[0]);
+    assertEquals(test.getOutputRecords()[0], copy.getOutputRecords()[0]);
+    assertEquals(test.getReduceBytes(0), copy.getReduceBytes(0));
+    assertEquals(test.getReduceRecords(0), copy.getReduceRecords(0));
+
+  }
+
+  /*
+   * test LoadMapper loadMapper should write to writer record for each reduce
+   */
+  @SuppressWarnings({"rawtypes", "unchecked"})
+  @Test (timeout=10000)
+  public void testLoadMapper() throws Exception {
+
+    Configuration conf = new Configuration();
+    conf.setInt(JobContext.NUM_REDUCES, 2);
+
+    CompressionEmulationUtil.setCompressionEmulationEnabled(conf, true);
+    conf.setBoolean(MRJobConfig.MAP_OUTPUT_COMPRESS, true);
+
+    TaskAttemptID taskId = new TaskAttemptID();
+    RecordReader<NullWritable, GridmixRecord> reader = new FakeRecordReader();
+
+    LoadRecordGkGrWriter writer = new LoadRecordGkGrWriter();
+
+    OutputCommitter committer = new CustomOutputCommitter();
+    StatusReporter reporter = new TaskAttemptContextImpl.DummyReporter();
+    LoadSplit split = getLoadSplit();
+
+    MapContext<NullWritable, GridmixRecord, GridmixKey, GridmixRecord> mapContext = new MapContextImpl<NullWritable, GridmixRecord, GridmixKey, GridmixRecord>(
+            conf, taskId, reader, writer, committer, reporter, split);
+    // context
+    Context ctx = new WrappedMapper<NullWritable, GridmixRecord, GridmixKey, GridmixRecord>()
+            .getMapContext(mapContext);
+
+    reader.initialize(split, ctx);
+    ctx.getConfiguration().setBoolean(MRJobConfig.MAP_OUTPUT_COMPRESS, true);
+    CompressionEmulationUtil.setCompressionEmulationEnabled(
+            ctx.getConfiguration(), true);
+
+    LoadJob.LoadMapper mapper = new LoadJob.LoadMapper();
+    // setup, map, clean
+    mapper.run(ctx);
+
+    Map<GridmixKey, GridmixRecord> data = writer.getData();
+    // check result
+    assertEquals(2, data.size());
+
+  }
+
+  private LoadSplit getLoadSplit() throws Exception {
+
+    Path[] files = {new Path("one"), new Path("two")};
+    long[] start = {1, 2};
+    long[] lengths = {100, 200};
+    String[] locations = {"locOne", "loctwo"};
+
+    CombineFileSplit cfSplit = new CombineFileSplit(files, start, lengths,
+            locations);
+    ResourceUsageMetrics metrics = new ResourceUsageMetrics();
+    metrics.setCumulativeCpuUsage(200);
+    ResourceUsageMetrics[] rMetrics = {metrics};
+
+    double[] reduceBytes = {8.1d, 8.2d};
+    double[] reduceRecords = {9.1d, 9.2d};
+    long[] reduceOutputBytes = {101L, 102L};
+    long[] reduceOutputRecords = {111L, 112L};
+
+    return new LoadSplit(cfSplit, 2, 1, 4L, 5L, 6L, 7L,
+            reduceBytes, reduceRecords, reduceOutputBytes, reduceOutputRecords,
+            metrics, rMetrics);
+  }
+
+  private class FakeRecordLLReader extends
+          RecordReader<LongWritable, LongWritable> {
+
+    int counter = 10;
+
+    @Override
+    public void initialize(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+
+    }
+
+    @Override
+    public boolean nextKeyValue() throws IOException, InterruptedException {
+      counter--;
+      return counter > 0;
+    }
+
+    @Override
+    public LongWritable getCurrentKey() throws IOException,
+            InterruptedException {
+
+      return new LongWritable(counter);
+    }
+
+    @Override
+    public LongWritable getCurrentValue() throws IOException,
+            InterruptedException {
+      return new LongWritable(counter * 10);
+    }
+
+    @Override
+    public float getProgress() throws IOException, InterruptedException {
+      return counter / 10.0f;
+    }
+
+    @Override
+    public void close() throws IOException {
+      // restore data
+      counter = 10;
+    }
+  }
+
+  private class FakeRecordReader extends
+          RecordReader<NullWritable, GridmixRecord> {
+
+    int counter = 10;
+
+    @Override
+    public void initialize(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+
+    }
+
+    @Override
+    public boolean nextKeyValue() throws IOException, InterruptedException {
+      counter--;
+      return counter > 0;
+    }
+
+    @Override
+    public NullWritable getCurrentKey() throws IOException,
+            InterruptedException {
+
+      return NullWritable.get();
+    }
+
+    @Override
+    public GridmixRecord getCurrentValue() throws IOException,
+            InterruptedException {
+      return new GridmixRecord(100, 100L);
+    }
+
+    @Override
+    public float getProgress() throws IOException, InterruptedException {
+      return counter / 10.0f;
+    }
+
+    @Override
+    public void close() throws IOException {
+      // restore data
+      counter = 10;
+    }
+  }
+
+  private class LoadRecordGkGrWriter extends
+          RecordWriter<GridmixKey, GridmixRecord> {
+    private Map<GridmixKey, GridmixRecord> data = new HashMap<GridmixKey, GridmixRecord>();
+
+    @Override
+    public void write(GridmixKey key, GridmixRecord value) throws IOException,
+            InterruptedException {
+      data.put(key, value);
+    }
+
+    @Override
+    public void close(TaskAttemptContext context) throws IOException,
+            InterruptedException {
+    }
+
+    public Map<GridmixKey, GridmixRecord> getData() {
+      return data;
+    }
+
+  }
+
+  private class LoadRecordGkNullWriter extends
+          RecordWriter<GridmixKey, NullWritable> {
+    private Map<GridmixKey, NullWritable> data = new HashMap<GridmixKey, NullWritable>();
+
+    @Override
+    public void write(GridmixKey key, NullWritable value) throws IOException,
+            InterruptedException {
+      data.put(key, value);
+    }
+
+    @Override
+    public void close(TaskAttemptContext context) throws IOException,
+            InterruptedException {
+    }
+
+    public Map<GridmixKey, NullWritable> getData() {
+      return data;
+    }
+
+  }
+
+  private class LoadRecordWriter extends
+          RecordWriter<NullWritable, GridmixRecord> {
+    private Map<NullWritable, GridmixRecord> data = new HashMap<NullWritable, GridmixRecord>();
+
+    @Override
+    public void write(NullWritable key, GridmixRecord value)
+            throws IOException, InterruptedException {
+      data.put(key, value);
+    }
+
+    @Override
+    public void close(TaskAttemptContext context) throws IOException,
+            InterruptedException {
+    }
+
+    public Map<NullWritable, GridmixRecord> getData() {
+      return data;
+    }
+
+  }
+
+  /*
+   * test LoadSortComparator
+   */
+  @Test (timeout=1000)
+  public void testLoadJobLoadSortComparator() throws Exception {
+    LoadJob.LoadSortComparator test = new LoadJob.LoadSortComparator();
+
+    ByteArrayOutputStream data = new ByteArrayOutputStream();
+    DataOutputStream dos = new DataOutputStream(data);
+    WritableUtils.writeVInt(dos, 2);
+    WritableUtils.writeVInt(dos, 1);
+    WritableUtils.writeVInt(dos, 4);
+    WritableUtils.writeVInt(dos, 7);
+    WritableUtils.writeVInt(dos, 4);
+
+    byte[] b1 = data.toByteArray();
+
+    byte[] b2 = data.toByteArray();
+
+    // the same data should be equals
+    assertEquals(0, test.compare(b1, 0, 1, b2, 0, 1));
+    b2[2] = 5;
+    // compare like GridMixKey first byte: shift count -1=4-5
+    assertEquals(-1, test.compare(b1, 0, 1, b2, 0, 1));
+    b2[2] = 2;
+    // compare like GridMixKey first byte: shift count 2=4-2
+    assertEquals(2, test.compare(b1, 0, 1, b2, 0, 1));
+    // compare arrays by first byte witch offset (2-1) because 4==4
+    b2[2] = 4;
+    assertEquals(1, test.compare(b1, 0, 1, b2, 1, 1));
+
+  }
+
+  /*
+   * test SpecGroupingComparator
+   */
+  @Test (timeout=1000)
+  public void testGridmixJobSpecGroupingComparator() throws Exception {
+    GridmixJob.SpecGroupingComparator test = new GridmixJob.SpecGroupingComparator();
+
+    ByteArrayOutputStream data = new ByteArrayOutputStream();
+    DataOutputStream dos = new DataOutputStream(data);
+    WritableUtils.writeVInt(dos, 2);
+    WritableUtils.writeVInt(dos, 1);
+    // 0: REDUCE SPEC
+    WritableUtils.writeVInt(dos, 0);
+    WritableUtils.writeVInt(dos, 7);
+    WritableUtils.writeVInt(dos, 4);
+
+    byte[] b1 = data.toByteArray();
+
+    byte[] b2 = data.toByteArray();
+
+    // the same object should be equals
+    assertEquals(0, test.compare(b1, 0, 1, b2, 0, 1));
+    b2[2] = 1;
+    // for Reduce
+    assertEquals(-1, test.compare(b1, 0, 1, b2, 0, 1));
+    // by Reduce spec
+    b2[2] = 1; // 1: DATA SPEC
+    assertEquals(-1, test.compare(b1, 0, 1, b2, 0, 1));
+    // compare GridmixKey the same objects should be equals
+    assertEquals(0, test.compare(new GridmixKey(GridmixKey.DATA, 100, 2),
+            new GridmixKey(GridmixKey.DATA, 100, 2)));
+    // REDUSE SPEC
+    assertEquals(-1, test.compare(
+            new GridmixKey(GridmixKey.REDUCE_SPEC, 100, 2), new GridmixKey(
+            GridmixKey.DATA, 100, 2)));
+    assertEquals(1, test.compare(new GridmixKey(GridmixKey.DATA, 100, 2),
+            new GridmixKey(GridmixKey.REDUCE_SPEC, 100, 2)));
+    // only DATA
+    assertEquals(2, test.compare(new GridmixKey(GridmixKey.DATA, 102, 2),
+            new GridmixKey(GridmixKey.DATA, 100, 2)));
+
+  }
+
+  /*
+   * test CompareGridmixJob only equals and compare
+   */
+  @Test (timeout=10000)
+  public void testCompareGridmixJob() throws Exception {
+    Configuration conf = new Configuration();
+    Path outRoot = new Path("target");
+    JobStory jobDesc = mock(JobStory.class);
+    when(jobDesc.getName()).thenReturn("JobName");
+    when(jobDesc.getJobConf()).thenReturn(new JobConf(conf));
+    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+    GridmixJob j1 = new LoadJob(conf, 1000L, jobDesc, outRoot, ugi, 0);
+    GridmixJob j2 = new LoadJob(conf, 1000L, jobDesc, outRoot, ugi, 0);
+    GridmixJob j3 = new LoadJob(conf, 1000L, jobDesc, outRoot, ugi, 1);
+    GridmixJob j4 = new LoadJob(conf, 1000L, jobDesc, outRoot, ugi, 1);
+
+    assertTrue(j1.equals(j2));
+    assertEquals(0, j1.compareTo(j2));
+    // Only one parameter matters
+    assertFalse(j1.equals(j3));
+    // compare id and submissionMillis
+    assertEquals(-1, j1.compareTo(j3));
+    assertEquals(-1, j1.compareTo(j4));
+
+  }
+
+  /*
+   * test ReadRecordFactory. should read all data from inputstream
+   */
+  @Test (timeout=1000)
+  public void testReadRecordFactory() throws Exception {
+
+    // RecordFactory factory, InputStream src, Configuration conf
+    RecordFactory rf = new FakeRecordFactory();
+    FakeInputStream input = new FakeInputStream();
+    ReadRecordFactory test = new ReadRecordFactory(rf, input,
+            new Configuration());
+    GridmixKey key = new GridmixKey(GridmixKey.DATA, 100, 2);
+    GridmixRecord val = new GridmixRecord(200, 2);
+    while (test.next(key, val)) {
+
+    }
+    // should be read 10* (GridmixKey.size +GridmixRecord.value)
+    assertEquals(3000, input.getCounter());
+    // should be -1 because all data readed;
+    assertEquals(-1, rf.getProgress(), 0.01);
+
+    test.close();
+  }
+
+  private class FakeRecordFactory extends RecordFactory {
+
+    private int counter = 10;
+
+    @Override
+    public void close() throws IOException {
+
+    }
+
+    @Override
+    public boolean next(GridmixKey key, GridmixRecord val) throws IOException {
+      counter--;
+      return counter >= 0;
+    }
+
+    @Override
+    public float getProgress() throws IOException {
+      return counter;
+    }
+
+  }
+
+  private class FakeInputStream extends InputStream implements Seekable,
+          PositionedReadable {
+    private long counter;
+
+    @Override
+    public int read() throws IOException {
+      return 0;
+    }
+
+    @Override
+    public int read(byte[] b, int off, int len) throws IOException {
+      int realLen = len - off;
+      counter += realLen;
+      for (int i = 0; i < b.length; i++) {
+        b[i] = 0;
+      }
+      return realLen;
+    }
+
+    public long getCounter() {
+      return counter;
+    }
+
+    @Override
+    public void seek(long pos) throws IOException {
+
+    }
+
+    @Override
+    public long getPos() throws IOException {
+      return counter;
+    }
+
+    @Override
+    public boolean seekToNewSource(long targetPos) throws IOException {
+      return false;
+    }
+
+    @Override
+    public int read(long position, byte[] buffer, int offset, int length)
+            throws IOException {
+      return 0;
+    }
+
+    @Override
+    public void readFully(long position, byte[] buffer, int offset, int length)
+            throws IOException {
+
+    }
+
+    @Override
+    public void readFully(long position, byte[] buffer) throws IOException {
+
+    }
+  }
+
+  private class FakeFSDataInputStream extends FSDataInputStream {
+
+    public FakeFSDataInputStream(InputStream in) throws IOException {
+      super(in);
+
+    }
+
+  }
+
+  /*
+   * test LoadRecordReader. It class reads data from some files.
+   */
+  @Test (timeout=1000)
+  public void testLoadJobLoadRecordReader() throws Exception {
+    LoadJob.LoadRecordReader test = new LoadJob.LoadRecordReader();
+    Configuration conf = new Configuration();
+
+    FileSystem fs1 = mock(FileSystem.class);
+    when(fs1.open((Path) anyObject())).thenReturn(
+            new FakeFSDataInputStream(new FakeInputStream()));
+    Path p1 = mock(Path.class);
+    when(p1.getFileSystem((JobConf) anyObject())).thenReturn(fs1);
+
+    FileSystem fs2 = mock(FileSystem.class);
+    when(fs2.open((Path) anyObject())).thenReturn(
+            new FakeFSDataInputStream(new FakeInputStream()));
+    Path p2 = mock(Path.class);
+    when(p2.getFileSystem((JobConf) anyObject())).thenReturn(fs2);
+
+    Path[] paths = {p1, p2};
+
+    long[] start = {0, 0};
+    long[] lengths = {1000, 1000};
+    String[] locations = {"temp1", "temp2"};
+    CombineFileSplit cfsplit = new CombineFileSplit(paths, start, lengths,
+            locations);
+    double[] reduceBytes = {100, 100};
+    double[] reduceRecords = {2, 2};
+    long[] reduceOutputBytes = {500, 500};
+    long[] reduceOutputRecords = {2, 2};
+    ResourceUsageMetrics metrics = new ResourceUsageMetrics();
+    ResourceUsageMetrics[] rMetrics = {new ResourceUsageMetrics(),
+            new ResourceUsageMetrics()};
+    LoadSplit input = new LoadSplit(cfsplit, 2, 3, 1500L, 2L, 3000L, 2L,
+            reduceBytes, reduceRecords, reduceOutputBytes, reduceOutputRecords,
+            metrics, rMetrics);
+    TaskAttemptID taskId = new TaskAttemptID();
+    TaskAttemptContext ctx = new TaskAttemptContextImpl(conf, taskId);
+    test.initialize(input, ctx);
+    GridmixRecord gr = test.getCurrentValue();
+    int counter = 0;
+    while (test.nextKeyValue()) {
+      gr = test.getCurrentValue();
+      if (counter == 0) {
+        // read first file
+        assertEquals(0.5, test.getProgress(), 0.001);
+      } else if (counter == 1) {
+        // read second file
+        assertEquals(1.0, test.getProgress(), 0.001);
+      }
+      //
+      assertEquals(1000, gr.getSize());
+      counter++;
+    }
+    assertEquals(1000, gr.getSize());
+    // Two files have been read
+    assertEquals(2, counter);
+
+    test.close();
+  }
+
+  /*
+   * test LoadReducer
+   */
+
+  @Test (timeout=1000)
+  public void testLoadJobLoadReducer() throws Exception {
+    LoadJob.LoadReducer test = new LoadJob.LoadReducer();
+
+    Configuration conf = new Configuration();
+    conf.setInt(JobContext.NUM_REDUCES, 2);
+    CompressionEmulationUtil.setCompressionEmulationEnabled(conf, true);
+    conf.setBoolean(FileOutputFormat.COMPRESS, true);
+
+    CompressionEmulationUtil.setCompressionEmulationEnabled(conf, true);
+    conf.setBoolean(MRJobConfig.MAP_OUTPUT_COMPRESS, true);
+    TaskAttemptID taskid = new TaskAttemptID();
+
+    RawKeyValueIterator input = new FakeRawKeyValueIterator();
+
+    Counter counter = new GenericCounter();
+    Counter inputValueCounter = new GenericCounter();
+    LoadRecordWriter output = new LoadRecordWriter();
+
+    OutputCommitter committer = new CustomOutputCommitter();
+
+    StatusReporter reporter = new DummyReporter();
+    RawComparator<GridmixKey> comparator = new FakeRawComparator();
+
+    ReduceContext<GridmixKey, GridmixRecord, NullWritable, GridmixRecord> reduceContext = new ReduceContextImpl<GridmixKey, GridmixRecord, NullWritable, GridmixRecord>(
+            conf, taskid, input, counter, inputValueCounter, output, committer,
+            reporter, comparator, GridmixKey.class, GridmixRecord.class);
+    // read for previous data
+    reduceContext.nextKeyValue();
+    org.apache.hadoop.mapreduce.Reducer<GridmixKey, GridmixRecord, NullWritable, GridmixRecord>.Context context = new WrappedReducer<GridmixKey, GridmixRecord, NullWritable, GridmixRecord>()
+            .getReducerContext(reduceContext);
+
+    // test.setup(context);
+    test.run(context);
+    // have been readed 9 records (-1 for previous)
+    assertEquals(9, counter.getValue());
+    assertEquals(10, inputValueCounter.getValue());
+    assertEquals(1, output.getData().size());
+    GridmixRecord record = output.getData().values().iterator()
+            .next();
+
+    assertEquals(1593, record.getSize());
+  }
+
+  protected class FakeRawKeyValueIterator implements RawKeyValueIterator {
+
+    int counter = 10;
+
+    @Override
+    public DataInputBuffer getKey() throws IOException {
+      ByteArrayOutputStream dt = new ByteArrayOutputStream();
+      GridmixKey key = new GridmixKey(GridmixKey.REDUCE_SPEC, 10 * counter, 1L);
+      Spec spec = new Spec();
+      spec.rec_in = counter;
+      spec.rec_out = counter;
+      spec.bytes_out = counter * 100;
+
+      key.setSpec(spec);
+      key.write(new DataOutputStream(dt));
+      DataInputBuffer result = new DataInputBuffer();
+      byte[] b = dt.toByteArray();
+      result.reset(b, 0, b.length);
+      return result;
+    }
+
+    @Override
+    public DataInputBuffer getValue() throws IOException {
+      ByteArrayOutputStream dt = new ByteArrayOutputStream();
+      GridmixRecord key = new GridmixRecord(100, 1);
+      key.write(new DataOutputStream(dt));
+      DataInputBuffer result = new DataInputBuffer();
+      byte[] b = dt.toByteArray();
+      result.reset(b, 0, b.length);
+      return result;
+    }
+
+    @Override
+    public boolean next() throws IOException {
+      counter--;
+      return counter >= 0;
+    }
+
+    @Override
+    public void close() throws IOException {
+
+    }
+
+    @Override
+    public Progress getProgress() {
+      return null;
+    }
+
+  }
+
+  private class FakeRawComparator implements RawComparator<GridmixKey> {
+
+    @Override
+    public int compare(GridmixKey o1, GridmixKey o2) {
+      return o1.compareTo(o2);
+    }
+
+    @Override
+    public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+      if ((l1 - s1) != (l2 - s2)) {
+        return (l1 - s1) - (l2 - s2);
+      }
+      int len = l1 - s1;
+      for (int i = 0; i < len; i++) {
+        if (b1[s1 + i] != b2[s2 + i]) {
+          return b1[s1 + i] - b2[s2 + i];
+        }
+      }
+      return 0;
+    }
+
+  }
+
+  /*
+   * test SerialJobFactory
+   */
+  @Test (timeout=40000)
+  public void testSerialReaderThread() throws Exception {
+
+    Configuration conf = new Configuration();
+    File fin = new File("src" + File.separator + "test" + File.separator
+            + "resources" + File.separator + "data" + File.separator
+            + "wordcount2.json");
+    // read couple jobs from wordcount2.json
+    JobStoryProducer jobProducer = new ZombieJobProducer(new Path(
+            fin.getAbsolutePath()), null, conf);
+    CountDownLatch startFlag = new CountDownLatch(1);
+    UserResolver resolver = new SubmitterUserResolver();
+    FakeJobSubmitter submitter = new FakeJobSubmitter();
+    File ws = new File("target" + File.separator + this.getClass().getName());
+    if (!ws.exists()) {
+      Assert.assertTrue(ws.mkdirs());
+    }
+
+    SerialJobFactory jobFactory = new SerialJobFactory(submitter, jobProducer,
+            new Path(ws.getAbsolutePath()), conf, startFlag, resolver);
+
+    Path ioPath = new Path(ws.getAbsolutePath());
+    jobFactory.setDistCacheEmulator(new DistributedCacheEmulator(conf, ioPath));
+    Thread test = jobFactory.createReaderThread();
+    test.start();
+    Thread.sleep(1000);
+    // SerialReaderThread waits startFlag
+    assertEquals(0, submitter.getJobs().size());
+    // start!
+    startFlag.countDown();
+    while (test.isAlive()) {
+      Thread.sleep(1000);
+      jobFactory.update(null);
+    }
+    // submitter was called twice
+    assertEquals(2, submitter.getJobs().size());
+  }
+
+  private class FakeJobSubmitter extends JobSubmitter {
+    // counter for submitted jobs
+    private List<GridmixJob> jobs = new ArrayList<GridmixJob>();
+
+    public FakeJobSubmitter() {
+      super(null, 1, 1, null, null);
+
+    }
+
+    @Override
+    public void add(GridmixJob job) throws InterruptedException {
+      jobs.add(job);
+    }
+
+    public List<GridmixJob> getJobs() {
+      return jobs;
+    }
+  }
+
+  /*
+   * test SleepMapper
+   */
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  @Test (timeout=10000)
+  public void testSleepMapper() throws Exception {
+    SleepJob.SleepMapper test = new SleepJob.SleepMapper();
+
+    Configuration conf = new Configuration();
+    conf.setInt(JobContext.NUM_REDUCES, 2);
+
+    CompressionEmulationUtil.setCompressionEmulationEnabled(conf, true);
+    conf.setBoolean(MRJobConfig.MAP_OUTPUT_COMPRESS, true);
+    TaskAttemptID taskId = new TaskAttemptID();
+    FakeRecordLLReader reader = new FakeRecordLLReader();
+    LoadRecordGkNullWriter writer = new LoadRecordGkNullWriter();
+    OutputCommitter committer = new CustomOutputCommitter();
+    StatusReporter reporter = new TaskAttemptContextImpl.DummyReporter();
+    SleepSplit split = getSleepSplit();
+    MapContext<LongWritable, LongWritable, GridmixKey, NullWritable> mapcontext = new MapContextImpl<LongWritable, LongWritable, GridmixKey, NullWritable>(
+            conf, taskId, reader, writer, committer, reporter, split);
+    Context context = new WrappedMapper<LongWritable, LongWritable, GridmixKey, NullWritable>()
+            .getMapContext(mapcontext);
+
+    long start = System.currentTimeMillis();
+    LOG.info("start:" + start);
+    LongWritable key = new LongWritable(start + 2000);
+    LongWritable value = new LongWritable(start + 2000);
+    // should slip 2 sec
+    test.map(key, value, context);
+    LOG.info("finish:" + System.currentTimeMillis());
+    assertTrue(System.currentTimeMillis() >= (start + 2000));
+
+    test.cleanup(context);
+    assertEquals(1, writer.getData().size());
+  }
+
+  private SleepSplit getSleepSplit() throws Exception {
+
+    String[] locations = {"locOne", "loctwo"};
+
+    long[] reduceDurations = {101L, 102L};
+
+    return new SleepSplit(0, 2000L, reduceDurations, 2, locations);
+  }
+
+  /*
+   * test SleepReducer
+   */
+  @Test (timeout=1000)
+  public void testSleepReducer() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(JobContext.NUM_REDUCES, 2);
+    CompressionEmulationUtil.setCompressionEmulationEnabled(conf, true);
+    conf.setBoolean(FileOutputFormat.COMPRESS, true);
+
+    CompressionEmulationUtil.setCompressionEmulationEnabled(conf, true);
+    conf.setBoolean(MRJobConfig.MAP_OUTPUT_COMPRESS, true);
+    TaskAttemptID taskId = new TaskAttemptID();
+
+    RawKeyValueIterator input = new FakeRawKeyValueReducerIterator();
+
+    Counter counter = new GenericCounter();
+    Counter inputValueCounter = new GenericCounter();
+    RecordWriter<NullWritable, NullWritable> output = new LoadRecordReduceWriter();
+
+    OutputCommitter committer = new CustomOutputCommitter();
+
+    StatusReporter reporter = new DummyReporter();
+    RawComparator<GridmixKey> comparator = new FakeRawComparator();
+
+    ReduceContext<GridmixKey, NullWritable, NullWritable, NullWritable> reducecontext = new ReduceContextImpl<GridmixKey, NullWritable, NullWritable, NullWritable>(
+            conf, taskId, input, counter, inputValueCounter, output, committer,
+            reporter, comparator, GridmixKey.class, NullWritable.class);
+    org.apache.hadoop.mapreduce.Reducer<GridmixKey, NullWritable, NullWritable, NullWritable>.Context context = new WrappedReducer<GridmixKey, NullWritable, NullWritable, NullWritable>()
+            .getReducerContext(reducecontext);
+
+    SleepReducer test = new SleepReducer();
+    long start = System.currentTimeMillis();
+    test.setup(context);
+    long sleeper = context.getCurrentKey().getReduceOutputBytes();
+    // status has been changed
+    assertEquals("Sleeping... " + sleeper + " ms left", context.getStatus());
+    // should sleep 0.9 sec
+
+    assertTrue(System.currentTimeMillis() >= (start + sleeper));
+    test.cleanup(context);
+    // status has been changed again
+
+    assertEquals("Slept for " + sleeper, context.getStatus());
+
+  }
+
+  private class LoadRecordReduceWriter extends
+          RecordWriter<NullWritable, NullWritable> {
+
+    @Override
+    public void write(NullWritable key, NullWritable value) throws IOException,
+            InterruptedException {
+    }
+
+    @Override
+    public void close(TaskAttemptContext context) throws IOException,
+            InterruptedException {
+    }
+
+  }
+
+  protected class FakeRawKeyValueReducerIterator implements RawKeyValueIterator {
+
+    int counter = 10;
+
+    @Override
+    public DataInputBuffer getKey() throws IOException {
+      ByteArrayOutputStream dt = new ByteArrayOutputStream();
+      GridmixKey key = new GridmixKey(GridmixKey.REDUCE_SPEC, 10 * counter, 1L);
+      Spec spec = new Spec();
+      spec.rec_in = counter;
+      spec.rec_out = counter;
+      spec.bytes_out = counter * 100;
+
+      key.setSpec(spec);
+      key.write(new DataOutputStream(dt));
+      DataInputBuffer result = new DataInputBuffer();
+      byte[] b = dt.toByteArray();
+      result.reset(b, 0, b.length);
+      return result;
+    }
+
+    @Override
+    public DataInputBuffer getValue() throws IOException {
+      ByteArrayOutputStream dt = new ByteArrayOutputStream();
+      NullWritable key = NullWritable.get();
+      key.write(new DataOutputStream(dt));
+      DataInputBuffer result = new DataInputBuffer();
+      byte[] b = dt.toByteArray();
+      result.reset(b, 0, b.length);
+      return result;
+    }
+
+    @Override
+    public boolean next() throws IOException {
+      counter--;
+      return counter >= 0;
+    }
+
+    @Override
+    public void close() throws IOException {
+
+    }
+
+    @Override
+    public Progress getProgress() {
+      return null;
+    }
+
+  }
+}

+ 202 - 0
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestGridmixSubmission.java

@@ -0,0 +1,202 @@
+/**
+ * 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.gridmix;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.tools.rumen.JobStory;
+import org.apache.hadoop.tools.rumen.JobStoryProducer;
+import org.apache.hadoop.util.ExitUtil;
+import org.apache.log4j.Level;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.InputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.zip.GZIPInputStream;
+
+import static org.junit.Assert.*;
+
+public class TestGridmixSubmission extends CommonJobTest {
+  private static File inSpace = new File("src" + File.separator + "test"
+          + File.separator + "resources" + File.separator + "data");
+
+
+  static {
+    ((Log4JLogger) LogFactory.getLog("org.apache.hadoop.mapred.gridmix"))
+            .getLogger().setLevel(Level.DEBUG);
+  }
+
+
+  @BeforeClass
+  public static void init() throws IOException {
+    GridmixTestUtils.initCluster(TestGridmixSubmission.class);
+
+    System.setProperty("src.test.data", inSpace.getAbsolutePath());
+  }
+
+  @AfterClass
+  public static void shutDown() throws IOException {
+    GridmixTestUtils.shutdownCluster();
+  }
+
+  /**
+   * Verifies that the given {@code JobStory} corresponds to the checked-in
+   * WordCount {@code JobStory}. The verification is effected via JUnit
+   * assertions.
+   *
+   * @param js the candidate JobStory.
+   */
+  private void verifyWordCountJobStory(JobStory js) {
+    assertNotNull("Null JobStory", js);
+    String expectedJobStory = "WordCount:johndoe:default:1285322645148:3:1";
+    String actualJobStory = js.getName() + ":" + js.getUser() + ":"
+            + js.getQueueName() + ":" + js.getSubmissionTime() + ":"
+            + js.getNumberMaps() + ":" + js.getNumberReduces();
+    assertEquals("Unexpected JobStory", expectedJobStory, actualJobStory);
+  }
+
+  /**
+   * Expands a file compressed using {@code gzip}.
+   *
+   * @param fs  the {@code FileSystem} corresponding to the given file.
+   * @param in  the path to the compressed file.
+   * @param out the path to the uncompressed output.
+   * @throws Exception if there was an error during the operation.
+   */
+  private void expandGzippedTrace(FileSystem fs, Path in, Path out)
+          throws Exception {
+    byte[] buff = new byte[4096];
+    GZIPInputStream gis = new GZIPInputStream(fs.open(in));
+    FSDataOutputStream fsdOs = fs.create(out);
+    int numRead;
+    while ((numRead = gis.read(buff, 0, buff.length)) != -1) {
+      fsdOs.write(buff, 0, numRead);
+    }
+    gis.close();
+    fsdOs.close();
+  }
+
+  /**
+   * Tests the reading of traces in GridMix3. These traces are generated by
+   * Rumen and are in the JSON format. The traces can optionally be compressed
+   * and uncompressed traces can also be passed to GridMix3 via its standard
+   * input stream. The testing is effected via JUnit assertions.
+   *
+   * @throws Exception if there was an error.
+   */
+  @Test (timeout=20000)
+  public void testTraceReader() throws Exception {
+    Configuration conf = new Configuration();
+    FileSystem lfs = FileSystem.getLocal(conf);
+    Path rootInputDir = new Path(System.getProperty("src.test.data"));
+    rootInputDir = rootInputDir.makeQualified(lfs.getUri(),
+            lfs.getWorkingDirectory());
+    Path rootTempDir = new Path(System.getProperty("test.build.data",
+            System.getProperty("java.io.tmpdir")), "testTraceReader");
+    rootTempDir = rootTempDir.makeQualified(lfs.getUri(),
+            lfs.getWorkingDirectory());
+    Path inputFile = new Path(rootInputDir, "wordcount.json.gz");
+    Path tempFile = new Path(rootTempDir, "gridmix3-wc.json");
+
+    InputStream origStdIn = System.in;
+    InputStream tmpIs = null;
+    try {
+      DebugGridmix dgm = new DebugGridmix();
+      JobStoryProducer jsp = dgm.createJobStoryProducer(inputFile.toString(),
+              conf);
+
+      LOG.info("Verifying JobStory from compressed trace...");
+      verifyWordCountJobStory(jsp.getNextJob());
+
+      expandGzippedTrace(lfs, inputFile, tempFile);
+      jsp = dgm.createJobStoryProducer(tempFile.toString(), conf);
+      LOG.info("Verifying JobStory from uncompressed trace...");
+      verifyWordCountJobStory(jsp.getNextJob());
+
+      tmpIs = lfs.open(tempFile);
+      System.setIn(tmpIs);
+      LOG.info("Verifying JobStory from trace in standard input...");
+      jsp = dgm.createJobStoryProducer("-", conf);
+      verifyWordCountJobStory(jsp.getNextJob());
+    } finally {
+      System.setIn(origStdIn);
+      if (tmpIs != null) {
+        tmpIs.close();
+      }
+      lfs.delete(rootTempDir, true);
+    }
+  }
+
+  @Test (timeout=500000)
+  public void testReplaySubmit() throws Exception {
+    policy = GridmixJobSubmissionPolicy.REPLAY;
+    LOG.info(" Replay started at " + System.currentTimeMillis());
+    doSubmission(null, false);
+    LOG.info(" Replay ended at " + System.currentTimeMillis());
+
+  }
+
+  @Test (timeout=500000)
+  public void testStressSubmit() throws Exception {
+    policy = GridmixJobSubmissionPolicy.STRESS;
+    LOG.info(" Stress started at " + System.currentTimeMillis());
+    doSubmission(null, false);
+    LOG.info(" Stress ended at " + System.currentTimeMillis());
+  }
+
+  // test empty request should be hint message
+  @Test (timeout=100000)
+  public void testMain() throws Exception {
+
+    SecurityManager securityManager = System.getSecurityManager();
+
+    final ByteArrayOutputStream bytes = new ByteArrayOutputStream();
+    final PrintStream out = new PrintStream(bytes);
+    final PrintStream oldOut = System.out;
+    System.setErr(out);
+    ExitUtil.disableSystemExit();
+    try {
+      String[] argv = new String[0];
+      DebugGridmix.main(argv);
+
+    } catch (ExitUtil.ExitException e) {
+      assertEquals("ExitException", e.getMessage());
+      ExitUtil.resetFirstExitException();
+    } finally {
+      System.setErr(oldOut);
+      System.setSecurityManager(securityManager);
+    }
+    String print = bytes.toString();
+    // should be printed tip in std error stream
+    assertTrue(print
+            .contains("Usage: gridmix [-generate <MiB>] [-users URI] [-Dname=value ...] <iopath> <trace>"));
+    assertTrue(print.contains("e.g. gridmix -generate 100m foo -"));
+  }
+
+ 
+}

+ 5 - 6
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestGridmixSummary.java

@@ -133,7 +133,7 @@ public class TestGridmixSummary {
   /**
    * A fake {@link JobFactory}.
    */
-  @SuppressWarnings("unchecked")
+  @SuppressWarnings("rawtypes")
   private static class FakeJobFactory extends JobFactory {
     /**
      * A fake {@link JobStoryProducer} for {@link FakeJobFactory}.
@@ -167,7 +167,7 @@ public class TestGridmixSummary {
    * Test {@link ExecutionSummarizer}.
    */
   @Test
-  @SuppressWarnings("unchecked")
+  @SuppressWarnings({ "unchecked", "rawtypes" })
   public void testExecutionSummarizer() throws IOException {
     Configuration conf = new Configuration();
     
@@ -360,7 +360,6 @@ public class TestGridmixSummary {
    * Test {@link ClusterSummarizer}.
    */
   @Test
-  @SuppressWarnings("deprecation")
   public void testClusterSummarizer() throws IOException {
     ClusterSummarizer cs = new ClusterSummarizer();
     Configuration conf = new Configuration();
@@ -374,13 +373,13 @@ public class TestGridmixSummary {
     assertEquals("JT name mismatch", jt, cs.getJobTrackerInfo());
     assertEquals("NN name mismatch", nn, cs.getNamenodeInfo());
     
-    ClusterStats cstats = ClusterStats.getClusterStats();
+    ClusterStats cStats = ClusterStats.getClusterStats();
     conf.set(JTConfig.JT_IPC_ADDRESS, "local");
     conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "local");
     JobClient jc = new JobClient(conf);
-    cstats.setClusterMetric(jc.getClusterStatus());
+    cStats.setClusterMetric(jc.getClusterStatus());
     
-    cs.update(cstats);
+    cs.update(cStats);
     
     // test
     assertEquals("Cluster summary test failed!", 1, cs.getMaxMapTasks());

+ 81 - 0
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestLoadJob.java

@@ -0,0 +1,81 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.gridmix;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.log4j.Level;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+
+/*
+ Test LoadJob Gridmix sends data to job and after that
+ */
+public class TestLoadJob extends CommonJobTest {
+
+  public static final Log LOG = LogFactory.getLog(Gridmix.class);
+
+  static {
+    ((Log4JLogger) LogFactory.getLog("org.apache.hadoop.mapred.gridmix"))
+            .getLogger().setLevel(Level.DEBUG);
+    ((Log4JLogger) LogFactory.getLog(StressJobFactory.class)).getLogger()
+            .setLevel(Level.DEBUG);
+  }
+
+
+  @BeforeClass
+  public static void init() throws IOException {
+    GridmixTestUtils.initCluster(TestLoadJob.class);
+  }
+
+  @AfterClass
+  public static void shutDown() throws IOException {
+    GridmixTestUtils.shutdownCluster();
+  }
+
+
+  /*
+  * test serial policy  with LoadJob. Task should execute without exceptions
+  */
+  @Test (timeout=500000)
+  public void testSerialSubmit() throws Exception {
+    policy = GridmixJobSubmissionPolicy.SERIAL;
+    LOG.info("Serial started at " + System.currentTimeMillis());
+    doSubmission(JobCreator.LOADJOB.name(), false);
+
+    LOG.info("Serial ended at " + System.currentTimeMillis());
+  }
+
+  /*
+   * test reply policy with LoadJob
+   */
+  @Test  (timeout=500000)
+  public void testReplaySubmit() throws Exception {
+    policy = GridmixJobSubmissionPolicy.REPLAY;
+    LOG.info(" Replay started at " + System.currentTimeMillis());
+    doSubmission(JobCreator.LOADJOB.name(), false);
+
+    LOG.info(" Replay ended at " + System.currentTimeMillis());
+  }
+
+
+}

+ 142 - 0
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestSleepJob.java

@@ -0,0 +1,142 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.gridmix;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.tools.rumen.JobStory;
+import org.apache.log4j.Level;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.junit.Assert.*;
+
+public class TestSleepJob extends CommonJobTest {
+
+  public static final Log LOG = LogFactory.getLog(Gridmix.class);
+
+  static {
+    ((Log4JLogger) LogFactory.getLog("org.apache.hadoop.mapred.gridmix"))
+            .getLogger().setLevel(Level.DEBUG);
+  }
+
+  static GridmixJobSubmissionPolicy policy = GridmixJobSubmissionPolicy.REPLAY;
+
+  @BeforeClass
+  public static void init() throws IOException {
+    GridmixTestUtils.initCluster(TestSleepJob.class);
+  }
+
+  @AfterClass
+  public static void shutDown() throws IOException {
+    GridmixTestUtils.shutdownCluster();
+  }
+
+
+  /*
+  * test RandomLocation
+  */
+  @Test
+  public void testRandomLocation() throws Exception {
+    UserGroupInformation ugi = UserGroupInformation.getLoginUser();
+
+    testRandomLocation(1, 10, ugi);
+    testRandomLocation(2, 10, ugi);
+  }
+
+  @Test
+  public void testMapTasksOnlySleepJobs() throws Exception {
+    Configuration configuration = GridmixTestUtils.mrvl.getConfig();
+
+    DebugJobProducer jobProducer = new DebugJobProducer(5, configuration);
+    configuration.setBoolean(SleepJob.SLEEPJOB_MAPTASK_ONLY, true);
+
+    UserGroupInformation ugi = UserGroupInformation.getLoginUser();
+    JobStory story;
+    int seq = 1;
+    while ((story = jobProducer.getNextJob()) != null) {
+      GridmixJob gridmixJob = JobCreator.SLEEPJOB.createGridmixJob(configuration, 0,
+              story, new Path("ignored"), ugi, seq++);
+      gridmixJob.buildSplits(null);
+      Job job = gridmixJob.call();
+      assertEquals(0, job.getNumReduceTasks());
+    }
+    jobProducer.close();
+    assertEquals(6, seq);
+  }
+
+  // test Serial submit
+  @Test
+  public void testSerialSubmit() throws Exception {
+    // set policy
+    policy = GridmixJobSubmissionPolicy.SERIAL;
+    LOG.info("Serial started at " + System.currentTimeMillis());
+    doSubmission(JobCreator.SLEEPJOB.name(), false);
+    LOG.info("Serial ended at " + System.currentTimeMillis());
+  }
+
+  @Test
+  public void testReplaySubmit() throws Exception {
+    policy = GridmixJobSubmissionPolicy.REPLAY;
+    LOG.info(" Replay started at " + System.currentTimeMillis());
+    doSubmission(JobCreator.SLEEPJOB.name(), false);
+    LOG.info(" Replay ended at " + System.currentTimeMillis());
+  }
+
+  @Test
+  public void testStressSubmit() throws Exception {
+    policy = GridmixJobSubmissionPolicy.STRESS;
+    LOG.info(" Replay started at " + System.currentTimeMillis());
+    doSubmission(JobCreator.SLEEPJOB.name(), false);
+    LOG.info(" Replay ended at " + System.currentTimeMillis());
+  }
+
+  private void testRandomLocation(int locations, int njobs,
+                                  UserGroupInformation ugi) throws Exception {
+    Configuration configuration = new Configuration();
+
+    DebugJobProducer jobProducer = new DebugJobProducer(njobs, configuration);
+    Configuration jconf = GridmixTestUtils.mrvl.getConfig();
+    jconf.setInt(JobCreator.SLEEPJOB_RANDOM_LOCATIONS, locations);
+
+    JobStory story;
+    int seq = 1;
+    while ((story = jobProducer.getNextJob()) != null) {
+      GridmixJob gridmixJob = JobCreator.SLEEPJOB.createGridmixJob(jconf, 0,
+              story, new Path("ignored"), ugi, seq++);
+      gridmixJob.buildSplits(null);
+      List<InputSplit> splits = new SleepJob.SleepInputFormat()
+              .getSplits(gridmixJob.getJob());
+      for (InputSplit split : splits) {
+        assertEquals(locations, split.getLocations().length);
+      }
+    }
+    jobProducer.close();
+  }
+
+}

+ 414 - 0
hadoop-tools/hadoop-gridmix/src/test/resources/data/wordcount.json

@@ -0,0 +1,414 @@
+{
+  "priority" : "NORMAL",
+  "jobID" : "job_201009241532_0001",
+  "user" : "johndoe",
+  "jobName" : "WordCount",
+  "mapTasks" : [ {
+    "startTime" : 1285322651360,
+    "taskID" : "task_201009241532_0001_m_000000",
+    "taskType" : "MAP",
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/default-rack/foo.example.com",
+      "startTime" : 1285322651366,
+      "finishTime" : 1285322658262,
+      "result" : "SUCCESS",
+      "attemptID" : "attempt_201009241532_0001_m_000000_0",
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "hdfsBytesRead" : 704270,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 48266,
+      "mapInputRecords" : 13427,
+      "mapOutputBytes" : 1182333,
+      "mapOutputRecords" : 126063,
+      "combineInputRecords" : 126063,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 6612,
+      "mapInputBytes" : -1
+    } ],
+    "finishTime" : 1285322660778,
+    "preferredLocations" : [ {
+      "layers" : [ "default-rack", "foo.example.com" ]
+    } ],
+    "taskStatus" : "SUCCESS",
+    "inputBytes" : 704270,
+    "inputRecords" : 13427,
+    "outputBytes" : 48266,
+    "outputRecords" : 126063
+  }, {
+    "startTime" : 1285322651361,
+    "taskID" : "task_201009241532_0001_m_000001",
+    "taskType" : "MAP",
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/default-rack/foo.example.com",
+      "startTime" : 1285322651378,
+      "finishTime" : 1285322657906,
+      "result" : "SUCCESS",
+      "attemptID" : "attempt_201009241532_0001_m_000001_0",
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "hdfsBytesRead" : 577214,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 58143,
+      "mapInputRecords" : 13015,
+      "mapOutputBytes" : 985534,
+      "mapOutputRecords" : 108400,
+      "combineInputRecords" : 108400,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 8214,
+      "mapInputBytes" : -1
+    } ],
+    "finishTime" : 1285322660781,
+    "preferredLocations" : [ {
+      "layers" : [ "default-rack", "foo.example.com" ]
+    } ],
+    "taskStatus" : "SUCCESS",
+    "inputBytes" : 577214,
+    "inputRecords" : 13015,
+    "outputBytes" : 58143,
+    "outputRecords" : 108400
+  }, {
+    "startTime" : 1285322660789,
+    "taskID" : "task_201009241532_0001_m_000002",
+    "taskType" : "MAP",
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/default-rack/foo.example.com",
+      "startTime" : 1285322660807,
+      "finishTime" : 1285322664865,
+      "result" : "SUCCESS",
+      "attemptID" : "attempt_201009241532_0001_m_000002_0",
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "hdfsBytesRead" : 163907,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 21510,
+      "mapInputRecords" : 3736,
+      "mapOutputBytes" : 275796,
+      "mapOutputRecords" : 30528,
+      "combineInputRecords" : 30528,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 3040,
+      "mapInputBytes" : -1
+    } ],
+    "finishTime" : 1285322666805,
+    "preferredLocations" : [ {
+      "layers" : [ "default-rack", "foo.example.com" ]
+    } ],
+    "taskStatus" : "SUCCESS",
+    "inputBytes" : 163907,
+    "inputRecords" : 3736,
+    "outputBytes" : 21510,
+    "outputRecords" : 30528
+  } ],
+  "finishTime" : 1285322675837,
+  "reduceTasks" : [ {
+    "startTime" : 1285322660790,
+    "taskID" : "task_201009241532_0001_r_000000",
+    "taskType" : "REDUCE",
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/default-rack/foo.example.com",
+      "startTime" : 1285322660807,
+      "finishTime" : 1285322670759,
+      "result" : "SUCCESS",
+      "attemptID" : "attempt_201009241532_0001_r_000000_0",
+      "shuffleFinished" : 1285322667962,
+      "sortFinished" : 1285322668146,
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 122793,
+      "fileBytesRead" : 111026,
+      "fileBytesWritten" : 111026,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 11713,
+      "reduceInputRecords" : 17866,
+      "reduceShuffleBytes" : 127823,
+      "reduceOutputRecords" : 11713,
+      "spilledRecords" : 17866,
+      "mapInputBytes" : -1
+    } ],
+    "finishTime" : 1285322672821,
+    "preferredLocations" : [ ],
+    "taskStatus" : "SUCCESS",
+    "inputBytes" : 127823,
+    "inputRecords" : 17866,
+    "outputBytes" : 122793,
+    "outputRecords" : 11713
+  } ],
+  "submitTime" : 1285322645148,
+  "launchTime" : 1285322645614,
+  "totalMaps" : 3,
+  "totalReduces" : 1,
+  "otherTasks" : [ {
+    "startTime" : 1285322648294,
+    "taskID" : "task_201009241532_0001_m_000004",
+    "taskType" : "SETUP",
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/default-rack/foo.example.com",
+      "startTime" : 1285322648482,
+      "finishTime" : 1285322649588,
+      "result" : "SUCCESS",
+      "attemptID" : "attempt_201009241532_0001_m_000004_0",
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 0,
+      "mapInputBytes" : -1
+    } ],
+    "finishTime" : 1285322651351,
+    "preferredLocations" : [ ],
+    "taskStatus" : "SUCCESS",
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1
+  }, {
+    "startTime" : 1285322672829,
+    "taskID" : "task_201009241532_0001_m_000003",
+    "taskType" : "CLEANUP",
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/default-rack/foo.example.com",
+      "startTime" : 1285322672838,
+      "finishTime" : 1285322673971,
+      "result" : "SUCCESS",
+      "attemptID" : "attempt_201009241532_0001_m_000003_0",
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 0,
+      "mapInputBytes" : -1
+    } ],
+    "finishTime" : 1285322675835,
+    "preferredLocations" : [ ],
+    "taskStatus" : "SUCCESS",
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1
+  } ],
+  "computonsPerMapInputByte" : -1,
+  "computonsPerMapOutputByte" : -1,
+  "computonsPerReduceInputByte" : -1,
+  "computonsPerReduceOutputByte" : -1,
+  "heapMegabytes" : 1024,
+  "outcome" : "SUCCESS",
+  "jobtype" : "JAVA",
+  "directDependantJobs" : [ ],
+  "successfulMapAttemptCDFs" : [ {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 6896,
+    "minimum" : 4058,
+    "rankings" : [ {
+      "datum" : 4058,
+      "relativeRanking" : 0.05
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.1
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.15
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.2
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.25
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.3
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.35
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.4
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.45
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.5
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.55
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.6
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.65
+    }, {
+      "datum" : 6528,
+      "relativeRanking" : 0.7
+    }, {
+      "datum" : 6528,
+      "relativeRanking" : 0.75
+    }, {
+      "datum" : 6528,
+      "relativeRanking" : 0.8
+    }, {
+      "datum" : 6528,
+      "relativeRanking" : 0.85
+    }, {
+      "datum" : 6528,
+      "relativeRanking" : 0.9
+    }, {
+      "datum" : 6528,
+      "relativeRanking" : 0.95
+    } ],
+    "numberValues" : 3
+  } ],
+  "failedMapAttemptCDFs" : [ {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  } ],
+  "successfulReduceAttemptCDF" : {
+    "maximum" : 9952,
+    "minimum" : 9952,
+    "rankings" : [ {
+      "datum" : 9952,
+      "relativeRanking" : 0.05
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.1
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.15
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.2
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.25
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.3
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.35
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.4
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.45
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.5
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.55
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.6
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.65
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.7
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.75
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.8
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.85
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.9
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.95
+    } ],
+    "numberValues" : 1
+  },
+  "failedReduceAttemptCDF" : {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  },
+  "mapperTriesToSucceed" : [ 1.0 ],
+  "failedMapperFraction" : 0.0,
+  "relativeTime" : 0,
+  "queue" : "default",
+  "clusterMapMB" : -1,
+  "clusterReduceMB" : -1,
+  "jobMapMB" : 1024,
+  "jobReduceMB" : 1024
+}

+ 828 - 0
hadoop-tools/hadoop-gridmix/src/test/resources/data/wordcount2.json

@@ -0,0 +1,828 @@
+{
+  "priority" : "NORMAL",
+  "jobID" : "job_201009241532_0001",
+  "user" : "johndoe",
+  "jobName" : "WordCount",
+  "mapTasks" : [ {
+    "startTime" : 1285322651360,
+    "taskID" : "task_201009241532_0001_m_000000",
+    "taskType" : "MAP",
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/default-rack/foo.example.com",
+      "startTime" : 1285322651366,
+      "finishTime" : 1285322658262,
+      "result" : "SUCCESS",
+      "attemptID" : "attempt_201009241532_0001_m_000000_0",
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "hdfsBytesRead" : 704270,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 48266,
+      "mapInputRecords" : 13427,
+      "mapOutputBytes" : 1182333,
+      "mapOutputRecords" : 126063,
+      "combineInputRecords" : 126063,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 6612,
+      "mapInputBytes" : -1
+    } ],
+    "finishTime" : 1285322660778,
+    "preferredLocations" : [ {
+      "layers" : [ "default-rack", "foo.example.com" ]
+    } ],
+    "taskStatus" : "SUCCESS",
+    "inputBytes" : 704270,
+    "inputRecords" : 13427,
+    "outputBytes" : 48266,
+    "outputRecords" : 126063
+  }, {
+    "startTime" : 1285322651361,
+    "taskID" : "task_201009241532_0001_m_000001",
+    "taskType" : "MAP",
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/default-rack/foo.example.com",
+      "startTime" : 1285322651378,
+      "finishTime" : 1285322657906,
+      "result" : "SUCCESS",
+      "attemptID" : "attempt_201009241532_0001_m_000001_0",
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "hdfsBytesRead" : 577214,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 58143,
+      "mapInputRecords" : 13015,
+      "mapOutputBytes" : 985534,
+      "mapOutputRecords" : 108400,
+      "combineInputRecords" : 108400,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 8214,
+      "mapInputBytes" : -1
+    } ],
+    "finishTime" : 1285322660781,
+    "preferredLocations" : [ {
+      "layers" : [ "default-rack", "foo.example.com" ]
+    } ],
+    "taskStatus" : "SUCCESS",
+    "inputBytes" : 577214,
+    "inputRecords" : 13015,
+    "outputBytes" : 58143,
+    "outputRecords" : 108400
+  }, {
+    "startTime" : 1285322660789,
+    "taskID" : "task_201009241532_0001_m_000002",
+    "taskType" : "MAP",
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/default-rack/foo.example.com",
+      "startTime" : 1285322660807,
+      "finishTime" : 1285322664865,
+      "result" : "SUCCESS",
+      "attemptID" : "attempt_201009241532_0001_m_000002_0",
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "hdfsBytesRead" : 163907,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 21510,
+      "mapInputRecords" : 3736,
+      "mapOutputBytes" : 275796,
+      "mapOutputRecords" : 30528,
+      "combineInputRecords" : 30528,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 3040,
+      "mapInputBytes" : -1
+    } ],
+    "finishTime" : 1285322666805,
+    "preferredLocations" : [ {
+      "layers" : [ "default-rack", "foo.example.com" ]
+    } ],
+    "taskStatus" : "SUCCESS",
+    "inputBytes" : 163907,
+    "inputRecords" : 3736,
+    "outputBytes" : 21510,
+    "outputRecords" : 30528
+  } ],
+  "finishTime" : 1285322675837,
+  "reduceTasks" : [ {
+    "startTime" : 1285322660790,
+    "taskID" : "task_201009241532_0001_r_000000",
+    "taskType" : "REDUCE",
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/default-rack/foo.example.com",
+      "startTime" : 1285322660807,
+      "finishTime" : 1285322670759,
+      "result" : "SUCCESS",
+      "attemptID" : "attempt_201009241532_0001_r_000000_0",
+      "shuffleFinished" : 1285322667962,
+      "sortFinished" : 1285322668146,
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 122793,
+      "fileBytesRead" : 111026,
+      "fileBytesWritten" : 111026,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 11713,
+      "reduceInputRecords" : 17866,
+      "reduceShuffleBytes" : 127823,
+      "reduceOutputRecords" : 11713,
+      "spilledRecords" : 17866,
+      "mapInputBytes" : -1
+    } ],
+    "finishTime" : 1285322672821,
+    "preferredLocations" : [ ],
+    "taskStatus" : "SUCCESS",
+    "inputBytes" : 127823,
+    "inputRecords" : 17866,
+    "outputBytes" : 122793,
+    "outputRecords" : 11713
+  } ],
+  "submitTime" : 1285322645148,
+  "launchTime" : 1285322645614,
+  "totalMaps" : 3,
+  "totalReduces" : 1,
+  "otherTasks" : [ {
+    "startTime" : 1285322648294,
+    "taskID" : "task_201009241532_0001_m_000004",
+    "taskType" : "SETUP",
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/default-rack/foo.example.com",
+      "startTime" : 1285322648482,
+      "finishTime" : 1285322649588,
+      "result" : "SUCCESS",
+      "attemptID" : "attempt_201009241532_0001_m_000004_0",
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 0,
+      "mapInputBytes" : -1
+    } ],
+    "finishTime" : 1285322651351,
+    "preferredLocations" : [ ],
+    "taskStatus" : "SUCCESS",
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1
+  }, {
+    "startTime" : 1285322672829,
+    "taskID" : "task_201009241532_0001_m_000003",
+    "taskType" : "CLEANUP",
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/default-rack/foo.example.com",
+      "startTime" : 1285322672838,
+      "finishTime" : 1285322673971,
+      "result" : "SUCCESS",
+      "attemptID" : "attempt_201009241532_0001_m_000003_0",
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 0,
+      "mapInputBytes" : -1
+    } ],
+    "finishTime" : 1285322675835,
+    "preferredLocations" : [ ],
+    "taskStatus" : "SUCCESS",
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1
+  } ],
+  "computonsPerMapInputByte" : -1,
+  "computonsPerMapOutputByte" : -1,
+  "computonsPerReduceInputByte" : -1,
+  "computonsPerReduceOutputByte" : -1,
+  "heapMegabytes" : 1024,
+  "outcome" : "SUCCESS",
+  "jobtype" : "JAVA",
+  "directDependantJobs" : [ ],
+  "successfulMapAttemptCDFs" : [ {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 6896,
+    "minimum" : 4058,
+    "rankings" : [ {
+      "datum" : 4058,
+      "relativeRanking" : 0.05
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.1
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.15
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.2
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.25
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.3
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.35
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.4
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.45
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.5
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.55
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.6
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.65
+    }, {
+      "datum" : 6528,
+      "relativeRanking" : 0.7
+    }, {
+      "datum" : 6528,
+      "relativeRanking" : 0.75
+    }, {
+      "datum" : 6528,
+      "relativeRanking" : 0.8
+    }, {
+      "datum" : 6528,
+      "relativeRanking" : 0.85
+    }, {
+      "datum" : 6528,
+      "relativeRanking" : 0.9
+    }, {
+      "datum" : 6528,
+      "relativeRanking" : 0.95
+    } ],
+    "numberValues" : 3
+  } ],
+  "failedMapAttemptCDFs" : [ {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  } ],
+  "successfulReduceAttemptCDF" : {
+    "maximum" : 9952,
+    "minimum" : 9952,
+    "rankings" : [ {
+      "datum" : 9952,
+      "relativeRanking" : 0.05
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.1
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.15
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.2
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.25
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.3
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.35
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.4
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.45
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.5
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.55
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.6
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.65
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.7
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.75
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.8
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.85
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.9
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.95
+    } ],
+    "numberValues" : 1
+  },
+  "failedReduceAttemptCDF" : {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  },
+  "mapperTriesToSucceed" : [ 1.0 ],
+  "failedMapperFraction" : 0.0,
+  "relativeTime" : 0,
+  "queue" : "default",
+  "clusterMapMB" : -1,
+  "clusterReduceMB" : -1,
+  "jobMapMB" : 1024,
+  "jobReduceMB" : 1024
+}
+{
+  "priority" : "NORMAL",
+  "jobID" : "job_201009241532_0001",
+  "user" : "johndoe",
+  "jobName" : "WordCount",
+  "mapTasks" : [ {
+    "startTime" : 1285322651360,
+    "taskID" : "task_201009241532_0001_m_000000",
+    "taskType" : "MAP",
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/default-rack/foo.example.com",
+      "startTime" : 1285322651366,
+      "finishTime" : 1285322658262,
+      "result" : "SUCCESS",
+      "attemptID" : "attempt_201009241532_0001_m_000000_0",
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "hdfsBytesRead" : 704270,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 48266,
+      "mapInputRecords" : 13427,
+      "mapOutputBytes" : 1182333,
+      "mapOutputRecords" : 126063,
+      "combineInputRecords" : 126063,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 6612,
+      "mapInputBytes" : -1
+    } ],
+    "finishTime" : 1285322660778,
+    "preferredLocations" : [ {
+      "layers" : [ "default-rack", "foo.example.com" ]
+    } ],
+    "taskStatus" : "SUCCESS",
+    "inputBytes" : 704270,
+    "inputRecords" : 13427,
+    "outputBytes" : 48266,
+    "outputRecords" : 126063
+  }, {
+    "startTime" : 1285322651361,
+    "taskID" : "task_201009241532_0001_m_000001",
+    "taskType" : "MAP",
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/default-rack/foo.example.com",
+      "startTime" : 1285322651378,
+      "finishTime" : 1285322657906,
+      "result" : "SUCCESS",
+      "attemptID" : "attempt_201009241532_0001_m_000001_0",
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "hdfsBytesRead" : 577214,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 58143,
+      "mapInputRecords" : 13015,
+      "mapOutputBytes" : 985534,
+      "mapOutputRecords" : 108400,
+      "combineInputRecords" : 108400,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 8214,
+      "mapInputBytes" : -1
+    } ],
+    "finishTime" : 1285322660781,
+    "preferredLocations" : [ {
+      "layers" : [ "default-rack", "foo.example.com" ]
+    } ],
+    "taskStatus" : "SUCCESS",
+    "inputBytes" : 577214,
+    "inputRecords" : 13015,
+    "outputBytes" : 58143,
+    "outputRecords" : 108400
+  }, {
+    "startTime" : 1285322660789,
+    "taskID" : "task_201009241532_0001_m_000002",
+    "taskType" : "MAP",
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/default-rack/foo.example.com",
+      "startTime" : 1285322660807,
+      "finishTime" : 1285322664865,
+      "result" : "SUCCESS",
+      "attemptID" : "attempt_201009241532_0001_m_000002_0",
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "hdfsBytesRead" : 163907,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 21510,
+      "mapInputRecords" : 3736,
+      "mapOutputBytes" : 275796,
+      "mapOutputRecords" : 30528,
+      "combineInputRecords" : 30528,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 3040,
+      "mapInputBytes" : -1
+    } ],
+    "finishTime" : 1285322666805,
+    "preferredLocations" : [ {
+      "layers" : [ "default-rack", "foo.example.com" ]
+    } ],
+    "taskStatus" : "SUCCESS",
+    "inputBytes" : 163907,
+    "inputRecords" : 3736,
+    "outputBytes" : 21510,
+    "outputRecords" : 30528
+  } ],
+  "finishTime" : 1285322675837,
+  "reduceTasks" : [ {
+    "startTime" : 1285322660790,
+    "taskID" : "task_201009241532_0001_r_000000",
+    "taskType" : "REDUCE",
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/default-rack/foo.example.com",
+      "startTime" : 1285322660807,
+      "finishTime" : 1285322670759,
+      "result" : "SUCCESS",
+      "attemptID" : "attempt_201009241532_0001_r_000000_0",
+      "shuffleFinished" : 1285322667962,
+      "sortFinished" : 1285322668146,
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 122793,
+      "fileBytesRead" : 111026,
+      "fileBytesWritten" : 111026,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 11713,
+      "reduceInputRecords" : 17866,
+      "reduceShuffleBytes" : 127823,
+      "reduceOutputRecords" : 11713,
+      "spilledRecords" : 17866,
+      "mapInputBytes" : -1
+    } ],
+    "finishTime" : 1285322672821,
+    "preferredLocations" : [ ],
+    "taskStatus" : "SUCCESS",
+    "inputBytes" : 127823,
+    "inputRecords" : 17866,
+    "outputBytes" : 122793,
+    "outputRecords" : 11713
+  } ],
+  "submitTime" : 1285322645148,
+  "launchTime" : 1285322645614,
+  "totalMaps" : 3,
+  "totalReduces" : 1,
+  "otherTasks" : [ {
+    "startTime" : 1285322648294,
+    "taskID" : "task_201009241532_0001_m_000004",
+    "taskType" : "SETUP",
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/default-rack/foo.example.com",
+      "startTime" : 1285322648482,
+      "finishTime" : 1285322649588,
+      "result" : "SUCCESS",
+      "attemptID" : "attempt_201009241532_0001_m_000004_0",
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 0,
+      "mapInputBytes" : -1
+    } ],
+    "finishTime" : 1285322651351,
+    "preferredLocations" : [ ],
+    "taskStatus" : "SUCCESS",
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1
+  }, {
+    "startTime" : 1285322672829,
+    "taskID" : "task_201009241532_0001_m_000003",
+    "taskType" : "CLEANUP",
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/default-rack/foo.example.com",
+      "startTime" : 1285322672838,
+      "finishTime" : 1285322673971,
+      "result" : "SUCCESS",
+      "attemptID" : "attempt_201009241532_0001_m_000003_0",
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 0,
+      "mapInputBytes" : -1
+    } ],
+    "finishTime" : 1285322675835,
+    "preferredLocations" : [ ],
+    "taskStatus" : "SUCCESS",
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1
+  } ],
+  "computonsPerMapInputByte" : -1,
+  "computonsPerMapOutputByte" : -1,
+  "computonsPerReduceInputByte" : -1,
+  "computonsPerReduceOutputByte" : -1,
+  "heapMegabytes" : 1024,
+  "outcome" : "SUCCESS",
+  "jobtype" : "JAVA",
+  "directDependantJobs" : [ ],
+  "successfulMapAttemptCDFs" : [ {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 6896,
+    "minimum" : 4058,
+    "rankings" : [ {
+      "datum" : 4058,
+      "relativeRanking" : 0.05
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.1
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.15
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.2
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.25
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.3
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.35
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.4
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.45
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.5
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.55
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.6
+    }, {
+      "datum" : 4058,
+      "relativeRanking" : 0.65
+    }, {
+      "datum" : 6528,
+      "relativeRanking" : 0.7
+    }, {
+      "datum" : 6528,
+      "relativeRanking" : 0.75
+    }, {
+      "datum" : 6528,
+      "relativeRanking" : 0.8
+    }, {
+      "datum" : 6528,
+      "relativeRanking" : 0.85
+    }, {
+      "datum" : 6528,
+      "relativeRanking" : 0.9
+    }, {
+      "datum" : 6528,
+      "relativeRanking" : 0.95
+    } ],
+    "numberValues" : 3
+  } ],
+  "failedMapAttemptCDFs" : [ {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  } ],
+  "successfulReduceAttemptCDF" : {
+    "maximum" : 9952,
+    "minimum" : 9952,
+    "rankings" : [ {
+      "datum" : 9952,
+      "relativeRanking" : 0.05
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.1
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.15
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.2
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.25
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.3
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.35
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.4
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.45
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.5
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.55
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.6
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.65
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.7
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.75
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.8
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.85
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.9
+    }, {
+      "datum" : 9952,
+      "relativeRanking" : 0.95
+    } ],
+    "numberValues" : 1
+  },
+  "failedReduceAttemptCDF" : {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  },
+  "mapperTriesToSucceed" : [ 1.0 ],
+  "failedMapperFraction" : 0.0,
+  "relativeTime" : 0,
+  "queue" : "default",
+  "clusterMapMB" : -1,
+  "clusterReduceMB" : -1,
+  "jobMapMB" : 1024,
+  "jobReduceMB" : 1024
+}

+ 2 - 2
hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamJob.java

@@ -875,7 +875,7 @@ public class StreamJob implements Tool {
         IdentifierResolver.TEXT_ID));
     jobConf_.setClass("stream.map.output.reader.class",
       idResolver.getOutputReaderClass(), OutputReader.class);
-    if (isMapperACommand) {
+    if (isMapperACommand || jobConf_.get("stream.map.output") != null) {
       // if mapper is a command, then map output key/value classes come from the
       // idResolver
       jobConf_.setMapOutputKeyClass(idResolver.getOutputKeyClass());
@@ -891,7 +891,7 @@ public class StreamJob implements Tool {
         IdentifierResolver.TEXT_ID));
     jobConf_.setClass("stream.reduce.output.reader.class",
       idResolver.getOutputReaderClass(), OutputReader.class);
-    if (isReducerACommand) {
+    if (isReducerACommand || jobConf_.get("stream.reduce.output") != null) {
       // if reducer is a command, then output key/value classes come from the
       // idResolver
       jobConf_.setOutputKeyClass(idResolver.getOutputKeyClass());

+ 12 - 0
hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestStreamingOutputKeyValueTypes.java

@@ -24,6 +24,7 @@ import org.apache.hadoop.mapred.MapReduceBase;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reducer;
 import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.junit.Before;
 import org.junit.Test;
@@ -177,6 +178,17 @@ public class TestStreamingOutputKeyValueTypes extends TestStreaming {
     args.add("0");
     super.testCommandLine();
   }
+  
+  @Test
+  public void testDefaultToIdentityReducer() throws Exception {
+    args.add("-mapper");args.add(map);
+    args.add("-jobconf");
+    args.add("mapreduce.task.files.preserve.failedtasks=true");
+    args.add("-jobconf");
+    args.add("stream.tmpdir="+System.getProperty("test.build.data","/tmp"));
+    args.add("-inputformat");args.add(TextInputFormat.class.getName());
+    super.testCommandLine();
+  }
 
   @Override
   @Test

+ 0 - 1
hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TrApp.java

@@ -43,7 +43,6 @@ public class TrApp
     // Note the dots translated to underscore: 
     // property names have been escaped in PipeMapRed.safeEnvVarName()
     expectDefined("mapreduce_cluster_local_dir");
-    expect("mapred_output_format_class", "org.apache.hadoop.mapred.TextOutputFormat");
     expect("mapreduce_map_output_key_class", "org.apache.hadoop.io.Text");
     expect("mapreduce_map_output_value_class", "org.apache.hadoop.io.Text");
 

+ 40 - 3
hadoop-yarn-project/CHANGES.txt

@@ -54,6 +54,9 @@ Trunk - Unreleased
 
     YARN-359. Fixing commands for container signalling in Windows. (Chris Nauroth
     via vinodkv)
+    
+    YARN-524 TestYarnVersionInfo failing if generated properties doesn't
+    include an SVN URL(stevel)
 
 Release 2.0.5-beta - UNRELEASED
 
@@ -66,6 +69,9 @@ Release 2.0.5-beta - UNRELEASED
 
     YARN-440. Flatten RegisterNodeManagerResponse. (Xuan Gong via sseth)
 
+    YARN-536. Removed the unused objects ContainerStatus and ContainerStatus from
+    Container which also don't belong to the container. (Xuan Gong via vinodkv)
+
   NEW FEATURES
 
   IMPROVEMENTS
@@ -103,6 +109,20 @@ Release 2.0.5-beta - UNRELEASED
     YARN-450. Define value for * in the scheduling protocol (Zhijie Shen via
     bikas)
 
+    YARN-475. Remove a unused constant in the public API -
+    ApplicationConstants.AM_APP_ATTEMPT_ID_ENV. (Hitesh Shah via vinodkv)
+
+    YARN-309. Changed NodeManager to obtain heart-beat interval from the
+    ResourceManager. (Xuan Gong via vinodkv)
+
+    YARN-447. Move ApplicationComparator in CapacityScheduler to use comparator
+    in ApplicationId. (Nemon Lou via vinodkv)
+
+    YARN-381. Improve fair scheduler docs. (Sandy Ryza via tomwhite)
+
+    YARN-458. YARN daemon addresses must be placed in many different configs. 
+    (sandyr via tucu)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -134,9 +154,6 @@ Release 2.0.5-beta - UNRELEASED
     YARN-485. TestProcfsProcessTree#testProcessTree() doesn't wait long enough 
     for the process to die. (kkambatl via tucu)
  
-    YARN-470. Support a way to disable resource monitoring on the NodeManager.
-    (Siddharth Seth via hitesh)
-    
     YARN-71. Fix the NodeManager to clean up local-dirs on restart.
     (Xuan Gong via sseth)
 
@@ -161,6 +178,23 @@ Release 2.0.5-beta - UNRELEASED
     YARN-515. Node Manager not getting the master key. (Robert Joseph Evans
     via jlowe)
 
+    YARN-382. SchedulerUtils improve way normalizeRequest sets the resource
+    capabilities. (Zhijie Shen via bikas)
+
+    YARN-467. Modify public distributed cache to localize files such that no
+    local directory hits unix file count limits and thus prevent job failures.
+    (Omkar Vinit Joshi via vinodkv)
+
+    YARN-101. Fix NodeManager heartbeat processing to not lose track of completed
+    containers in case of dropped heartbeats. (Xuan Gong via vinodkv)
+
+    YARN-538. RM address DNS lookup can cause unnecessary slowness on every JHS 
+    page load. (sandyr via tucu)
+
+    YARN-532. Change RMAdmin and Localization client protocol PB implementations
+    to implement closeable so that they can be stopped when needed via
+    RPC.stopProxy(). (Siddharth Seth via vinodkv)
+
 Release 2.0.4-alpha - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -176,6 +210,9 @@ Release 2.0.4-alpha - UNRELEASED
     YARN-429. capacity-scheduler config missing from yarn-test artifact.
     (sseth via hitesh)
 
+    YARN-470. Support a way to disable resource monitoring on the NodeManager.
+    (Siddharth Seth via hitesh)
+    
 Release 2.0.3-alpha - 2013-02-06 
 
   INCOMPATIBLE CHANGES

+ 14 - 0
hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml

@@ -256,4 +256,18 @@
     <Bug pattern="IS2_INCONSISTENT_SYNC" />
   </Match>
 
+  <!-- Null pointer exception needs to be ignored here as this is never going to occur. -->
+  <Match>
+    <Class name="org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.LocalResourcesTrackerImpl" />
+    <Method name="decrementFileCountForLocalCacheDirectory" />
+    <Bug pattern="NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE" />
+  </Match>
+
+  <!-- Null pointer exception needs to be ignored here as this is never going to occur. -->
+  <Match>
+    <Class name="org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.LocalResourcesTrackerImpl" />
+    <Method name="getPathForLocalization" />
+    <Bug pattern="NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE" />
+  </Match>
+
 </FindBugsFilter>

+ 0 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java

@@ -42,12 +42,6 @@ public interface ApplicationConstants {
    * only
    */
   public static final String AM_CONTAINER_ID_ENV = "AM_CONTAINER_ID";
-  
-  /**
-   * The environment variable for APPLICATION_ATTEMPT_ID. Set in AppMaster
-   * environment only
-   */
-  public static final String AM_APP_ATTEMPT_ID_ENV = "AM_APP_ATTEMPT_ID";
 
   /**
    * The environment variable for the NM_HOST. Set in the AppMaster environment

+ 0 - 24
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java

@@ -124,18 +124,6 @@ public interface Container extends Comparable<Container> {
   @Unstable
   void setPriority(Priority priority);
   
-  /**
-   * Get the current <code>ContainerState</code> of the container.
-   * @return current <code>ContainerState</code> of the container
-   */
-  @Public
-  @Stable
-  ContainerState getState();
-  
-  @Private
-  @Unstable
-  void setState(ContainerState state);
-  
   /**
    * Get the <code>ContainerToken</code> for the container.
    * @return <code>ContainerToken</code> for the container
@@ -147,16 +135,4 @@ public interface Container extends Comparable<Container> {
   @Private
   @Unstable
   void setContainerToken(ContainerToken containerToken);
-  
-  /**
-   * Get the <code>ContainerStatus</code> of the container.
-   * @return <code>ContainerStatus</code> of the container
-   */
-  @Public
-  @Stable
-  ContainerStatus getContainerStatus();
-  
-  @Private
-  @Unstable
-  void setContainerStatus(ContainerStatus containerStatus);
 }

+ 1 - 79
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerPBImpl.java

@@ -21,8 +21,6 @@ package org.apache.hadoop.yarn.api.records.impl.pb;
 import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerState;
-import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.ContainerToken;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
@@ -31,12 +29,9 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProtoOrBuilder;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStateProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
-import org.apache.hadoop.yarn.util.ProtoUtils;
     
 public class ContainerPBImpl extends ProtoBase<ContainerProto> implements Container {
 
@@ -49,7 +44,6 @@ public class ContainerPBImpl extends ProtoBase<ContainerProto> implements Contai
   private Resource resource = null;
   private Priority priority = null;
   private ContainerToken containerToken = null;
-  private ContainerStatus containerStatus = null;
   
   public ContainerPBImpl() {
     builder = ContainerProto.newBuilder();
@@ -94,11 +88,6 @@ public class ContainerPBImpl extends ProtoBase<ContainerProto> implements Contai
             builder.getContainerToken())) {
       builder.setContainerToken(convertToProtoFormat(this.containerToken));
     }
-    if (this.containerStatus != null
-        && !((ContainerStatusPBImpl) this.containerStatus).getProto().equals(
-            builder.getContainerStatus())) {
-      builder.setContainerStatus(convertToProtoFormat(this.containerStatus));
-    }
   }
 
   private void mergeLocalToProto() {
@@ -115,26 +104,7 @@ public class ContainerPBImpl extends ProtoBase<ContainerProto> implements Contai
     }
     viaProto = false;
   }
-    
-  
-  @Override
-  public ContainerState getState() {
-    ContainerProtoOrBuilder p = viaProto ? proto : builder;
-    if (!p.hasState()) {
-      return null;
-    }
-    return convertFromProtoFormat(p.getState());
-  }
 
-  @Override
-  public void setState(ContainerState state) {
-    maybeInitBuilder();
-    if (state == null) {
-      builder.clearState();
-      return;
-    }
-    builder.setState(convertToProtoFormat(state));
-  }
   @Override
   public ContainerId getId() {
     ContainerProtoOrBuilder p = viaProto ? proto : builder;
@@ -260,35 +230,6 @@ public class ContainerPBImpl extends ProtoBase<ContainerProto> implements Contai
     this.containerToken = containerToken;
   }
 
-  @Override
-  public ContainerStatus getContainerStatus() {
-    ContainerProtoOrBuilder p = viaProto ? proto : builder;
-    if (this.containerStatus != null) {
-      return this.containerStatus;
-    }
-    if (!p.hasContainerStatus()) {
-      return null;
-    }
-    this.containerStatus = convertFromProtoFormat(p.getContainerStatus());
-    return this.containerStatus;
-  }
-
-  @Override
-  public void setContainerStatus(ContainerStatus containerStatus) {
-    maybeInitBuilder();
-    if (containerStatus == null) 
-      builder.clearContainerStatus();
-    this.containerStatus = containerStatus;
-  }
-
-  private ContainerStateProto convertToProtoFormat(ContainerState e) {
-    return ProtoUtils.convertToProtoFormat(e);
-  }
-
-  private ContainerState convertFromProtoFormat(ContainerStateProto e) {
-    return ProtoUtils.convertFromProtoFormat(e);
-  }
-
   private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
     return new ContainerIdPBImpl(p);
   }
@@ -329,14 +270,6 @@ public class ContainerPBImpl extends ProtoBase<ContainerProto> implements Contai
     return ((ContainerTokenPBImpl)t).getProto();
   }
 
-  private ContainerStatusPBImpl convertFromProtoFormat(ContainerStatusProto p) {
-    return new ContainerStatusPBImpl(p);
-  }
-
-  private ContainerStatusProto convertToProtoFormat(ContainerStatus t) {
-    return ((ContainerStatusPBImpl)t).getProto();
-  }
-
   public String toString() {
     StringBuilder sb = new StringBuilder();
     sb.append("Container: [");
@@ -345,9 +278,7 @@ public class ContainerPBImpl extends ProtoBase<ContainerProto> implements Contai
     sb.append("NodeHttpAddress: ").append(getNodeHttpAddress()).append(", ");
     sb.append("Resource: ").append(getResource()).append(", ");
     sb.append("Priority: ").append(getPriority()).append(", ");
-    sb.append("State: ").append(getState()).append(", ");
     sb.append("Token: ").append(getContainerToken()).append(", ");
-    sb.append("Status: ").append(getContainerStatus());
     sb.append("]");
     return sb.toString();
   }
@@ -357,16 +288,7 @@ public class ContainerPBImpl extends ProtoBase<ContainerProto> implements Contai
   public int compareTo(Container other) {
     if (this.getId().compareTo(other.getId()) == 0) {
       if (this.getNodeId().compareTo(other.getNodeId()) == 0) {
-        if (this.getResource().compareTo(other.getResource()) == 0) {
-          if (this.getState().compareTo(other.getState()) == 0) {
-            //ContainerToken
-            return this.getState().compareTo(other.getState());
-          } else {
-            return this.getState().compareTo(other.getState());
-          }
-        } else {
-          return this.getResource().compareTo(other.getResource());
-        }
+        return this.getResource().compareTo(other.getResource());
       } else {
         return this.getNodeId().compareTo(other.getNodeId());
       }

+ 1 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto

@@ -67,9 +67,7 @@ message ContainerProto {
   optional string node_http_address = 3;
   optional ResourceProto resource = 4;
   optional PriorityProto priority = 5;
-  optional ContainerStateProto state = 6;
-  optional hadoop.common.TokenProto container_token = 7;
-  optional ContainerStatusProto container_status = 8;
+  optional hadoop.common.TokenProto container_token = 6;
 }
 
 enum YarnApplicationStateProto {

+ 0 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java

@@ -608,7 +608,6 @@ public class ApplicationMaster {
             + ", containerNode=" + allocatedContainer.getNodeId().getHost()
             + ":" + allocatedContainer.getNodeId().getPort()
             + ", containerNodeURI=" + allocatedContainer.getNodeHttpAddress()
-            + ", containerState" + allocatedContainer.getState()
             + ", containerResourceMemory"
             + allocatedContainer.getResource().getMemory());
         // + ", containerToken"

+ 5 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/main/java/org/apache/hadoop/yarn/applications/unmanagedamlauncher/UnmanagedAMLauncher.java

@@ -57,10 +57,11 @@ import org.apache.hadoop.yarn.util.Records;
  * unmanagedAM is an AM that is not launched and managed by the RM. The client
  * creates a new application on the RM and negotiates a new attempt id. Then it
  * waits for the RM app state to reach be YarnApplicationState.ACCEPTED after
- * which it spawns the AM in another process and passes it the attempt id via
- * env variable ApplicationConstants.AM_APP_ATTEMPT_ID_ENV. The AM can be in any
- * language. The AM can register with the RM using the attempt id and proceed as
- * normal. The client redirects app stdout and stderr to its own stdout and
+ * which it spawns the AM in another process and passes it the container id via
+ * env variable ApplicationConstants.AM_CONTAINER_ID_ENV. The AM can be in any
+ * language. The AM can register with the RM using the attempt id obtained
+ * from the container id and proceed as normal.
+ * The client redirects app stdout and stderr to its own stdout and
  * stderr and waits for the AM process to exit. Then it waits for the RM to
  * report app completion.
  */

+ 9 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/RMAdminProtocolPBClientImpl.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.api.impl.pb.client;
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 
@@ -65,7 +66,7 @@ import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.Refre
 import com.google.protobuf.ServiceException;
 
 
-public class RMAdminProtocolPBClientImpl implements RMAdminProtocol {
+public class RMAdminProtocolPBClientImpl implements RMAdminProtocol, Closeable {
 
   private RMAdminProtocolPB proxy;
   
@@ -77,6 +78,13 @@ public class RMAdminProtocolPBClientImpl implements RMAdminProtocol {
         RMAdminProtocolPB.class, clientVersion, addr, conf);
   }
 
+  @Override
+  public void close() {
+    if (this.proxy != null) {
+      RPC.stopProxy(this.proxy);
+    }
+  }
+
   @Override
   public RefreshQueuesResponse refreshQueues(RefreshQueuesRequest request)
       throws YarnRemoteException {

+ 18 - 16
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -219,6 +219,11 @@ public class YarnConfiguration extends Configuration {
   public static final String DEFAULT_RM_SCHEDULER = 
       "org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler";
 
+  /** RM set next Heartbeat interval for NM */
+  public static final String RM_NM_HEARTBEAT_INTERVAL_MS =
+      RM_PREFIX + "nodemanagers.heartbeat-interval-ms";
+  public static final long DEFAULT_RM_NM_HEARTBEAT_INTERVAL_MS = 1000;
+
   //Delegation token related keys
   public static final String  DELEGATION_KEY_UPDATE_INTERVAL_KEY = 
     RM_PREFIX + "delegation.key.update-interval";
@@ -329,19 +334,21 @@ public class YarnConfiguration extends Configuration {
     NM_PREFIX +  "delete.thread-count";
   public static final int DEFAULT_NM_DELETE_THREAD_COUNT = 4;
   
-  // TODO: Should this instead be dictated by RM?
-  /** Heartbeat interval to RM*/
-  public static final String NM_TO_RM_HEARTBEAT_INTERVAL_MS = 
-    NM_PREFIX + "heartbeat.interval-ms";
-  public static final int DEFAULT_NM_TO_RM_HEARTBEAT_INTERVAL_MS = 1000;
-  
   /** Keytab for NM.*/
   public static final String NM_KEYTAB = NM_PREFIX + "keytab";
   
   /**List of directories to store localized files in.*/
   public static final String NM_LOCAL_DIRS = NM_PREFIX + "local-dirs";
   public static final String DEFAULT_NM_LOCAL_DIRS = "/tmp/nm-local-dir";
-  
+
+  /**
+   * Number of files in each localized directories
+   * Avoid tuning this too low. 
+   */
+  public static final String NM_LOCAL_CACHE_MAX_FILES_PER_DIRECTORY =
+    NM_PREFIX + "local-cache.max-files-per-directory";
+  public static final int DEFAULT_NM_LOCAL_CACHE_MAX_FILES_PER_DIRECTORY = 8192;
+
   /** Address where the localizer IPC is.*/
   public static final String NM_LOCALIZER_ADDRESS =
     NM_PREFIX + "localizer.address";
@@ -710,21 +717,16 @@ public class YarnConfiguration extends Configuration {
   }
   
   public static String getRMWebAppHostAndPort(Configuration conf) {
-    int port = conf.getSocketAddr(
+    InetSocketAddress address = conf.getSocketAddr(
         YarnConfiguration.RM_WEBAPP_ADDRESS,
         YarnConfiguration.DEFAULT_RM_WEBAPP_ADDRESS,
-        YarnConfiguration.DEFAULT_RM_WEBAPP_PORT).getPort();
-    // Use apps manager address to figure out the host for webapp
-    String host = conf.getSocketAddr(
-        YarnConfiguration.RM_ADDRESS,
-        YarnConfiguration.DEFAULT_RM_ADDRESS,
-        YarnConfiguration.DEFAULT_RM_PORT).getHostName();
-    InetSocketAddress address = NetUtils.createSocketAddrForHost(host, port);
+        YarnConfiguration.DEFAULT_RM_WEBAPP_PORT);
+    address = NetUtils.getConnectAddress(address);
     StringBuffer sb = new StringBuffer();
     InetAddress resolved = address.getAddress();
     if (resolved == null || resolved.isAnyLocalAddress() || 
         resolved.isLoopbackAddress()) {
-      String lh = host;
+      String lh = address.getHostName();
       try {
         lh = InetAddress.getLocalHost().getCanonicalHostName();
       } catch (UnknownHostException e) {

+ 0 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/BuilderUtils.java

@@ -246,11 +246,6 @@ public class BuilderUtils {
     container.setNodeHttpAddress(nodeHttpAddress);
     container.setResource(resource);
     container.setPriority(priority);
-    container.setState(ContainerState.NEW);
-    ContainerStatus containerStatus = Records.newRecord(ContainerStatus.class);
-    containerStatus.setContainerId(containerId);
-    containerStatus.setState(ContainerState.NEW);
-    container.setContainerStatus(containerStatus);
     container.setContainerToken(containerToken);
     return container;
   }

+ 45 - 14
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml

@@ -58,10 +58,16 @@
   </property>
   
   <!-- Resource Manager Configs -->
+  <property>
+    <description>The hostname of the RM.</description>
+    <name>yarn.resourcemanager.hostname</name>
+    <value>0.0.0.0</value>
+  </property>    
+  
   <property>
     <description>The address of the applications manager interface in the RM.</description>
     <name>yarn.resourcemanager.address</name>
-    <value>0.0.0.0:8032</value>
+    <value>${yarn.resourcemanager.hostname}:8032</value>
   </property>
 
   <property>
@@ -84,7 +90,7 @@
   <property>
     <description>The address of the scheduler interface.</description>
     <name>yarn.resourcemanager.scheduler.address</name>
-    <value>0.0.0.0:8030</value>
+    <value>${yarn.resourcemanager.hostname}:8030</value>
   </property>
 
   <property>
@@ -96,12 +102,12 @@
   <property>
     <description>The address of the RM web application.</description>
     <name>yarn.resourcemanager.webapp.address</name>
-    <value>0.0.0.0:8088</value>
+    <value>${yarn.resourcemanager.hostname}:8088</value>
   </property>
 
   <property>
     <name>yarn.resourcemanager.resource-tracker.address</name>
-    <value>0.0.0.0:8031</value>
+    <value>${yarn.resourcemanager.hostname}:8031</value>
   </property>
 
   <property>
@@ -119,7 +125,7 @@
   <property>
     <description>The address of the RM admin interface.</description>
     <name>yarn.resourcemanager.admin.address</name>
-    <value>0.0.0.0:8033</value>
+    <value>${yarn.resourcemanager.hostname}:8033</value>
   </property>
 
   <property>
@@ -278,11 +284,23 @@
     <value>86400</value>
   </property>
 
+  <property>
+    <description>The heart-beat interval in milliseconds for every NodeManager in the cluster.</description>
+    <name>yarn.resourcemanager.nodemanagers.heartbeat-interval-ms</name>
+    <value>1000</value>
+  </property>
+
   <!-- Node Manager Configs -->
+  <property>
+    <description>The hostname of the NM.</description>
+    <name>yarn.nodemanager.hostname</name>
+    <value>0.0.0.0</value>
+  </property>
+  
   <property>
     <description>The address of the container manager in the NM.</description>
     <name>yarn.nodemanager.address</name>
-    <value>0.0.0.0:0</value>
+    <value>${yarn.nodemanager.hostname}:0</value>
   </property>
 
   <property>
@@ -336,12 +354,6 @@
     <value>0</value>
   </property>
 
-  <property>
-    <description>Heartbeat interval to RM</description>
-    <name>yarn.nodemanager.heartbeat.interval-ms</name>
-    <value>1000</value>
-  </property>
-
   <property>
     <description>Keytab for NM.</description>
     <name>yarn.nodemanager.keytab</name>
@@ -359,10 +371,29 @@
     <value>${hadoop.tmp.dir}/nm-local-dir</value>
   </property>
 
+  <property>
+    <description>It limits the maximum number of files which will be localized
+      in a single local directory. If the limit is reached then sub-directories
+      will be created and new files will be localized in them. If it is set to
+      a value less than or equal to 36 [which are sub-directories (0-9 and then
+      a-z)] then NodeManager will fail to start. For example; [for public
+      cache] if this is configured with a value of 40 ( 4 files +
+      36 sub-directories) and the local-dir is "/tmp/local-dir1" then it will
+      allow 4 files to be created directly inside "/tmp/local-dir1/filecache".
+      For files that are localized further it will create a sub-directory "0"
+      inside "/tmp/local-dir1/filecache" and will localize files inside it
+      until it becomes full. If a file is removed from a sub-directory that
+      is marked full, then that sub-directory will be used back again to
+      localize files.
+   </description>
+    <name>yarn.nodemanager.local-cache.max-files-per-directory</name>
+    <value>8192</value>
+  </property>
+
   <property>
     <description>Address where the localizer IPC is.</description>
     <name>yarn.nodemanager.localizer.address</name>
-    <value>0.0.0.0:8040</value>
+    <value>${yarn.nodemanager.hostname}:8040</value>
   </property>
 
   <property>
@@ -493,7 +524,7 @@
   <property>
     <description>NM Webapp address.</description>
     <name>yarn.nodemanager.webapp.address</name>
-    <value>0.0.0.0:8042</value>
+    <value>${yarn.nodemanager.hostname}:8042</value>
   </property>
 
   <property>

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestYarnVersionInfo.java

@@ -40,11 +40,11 @@ public class TestYarnVersionInfo extends TestCase {
     // make sure they aren't Unknown
     assertTrue("getVersion returned Unknown", !YarnVersionInfo.getVersion().equals("Unknown"));
     assertTrue("getUser returned Unknown", !YarnVersionInfo.getUser().equals("Unknown"));
-    assertTrue("getUrl returned Unknown", !YarnVersionInfo.getUrl().equals("Unknown"));
     assertTrue("getSrcChecksum returned Unknown", !YarnVersionInfo.getSrcChecksum().equals("Unknown"));
 
     // these could be Unknown if the VersionInfo generated from code not in svn or git
     // so just check that they return something
+    assertNotNull("getUrl returned null", YarnVersionInfo.getUrl());
     assertNotNull("getRevision returned null", YarnVersionInfo.getRevision());
     assertNotNull("getBranch returned null", YarnVersionInfo.getBranch());
 

+ 3 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java

@@ -42,4 +42,7 @@ public interface NodeHeartbeatResponse {
   void addAllContainersToCleanup(List<ContainerId> containers);
   
   void addAllApplicationsToCleanup(List<ApplicationId> applications);
+
+  long getNextHeartBeatInterval();
+  void setNextHeartBeatInterval(long nextHeartBeatInterval);
 }

+ 12 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java

@@ -271,6 +271,18 @@ public class NodeHeartbeatResponsePBImpl extends ProtoBase<NodeHeartbeatResponse
     builder.addAllApplicationsToCleanup(iterable);
   }
 
+  @Override
+  public long getNextHeartBeatInterval() {
+    NodeHeartbeatResponseProtoOrBuilder p = viaProto ? proto : builder;
+    return (p.getNextHeartBeatInterval());
+  }
+
+  @Override
+  public void setNextHeartBeatInterval(long nextHeartBeatInterval) {
+    maybeInitBuilder();
+    builder.setNextHeartBeatInterval(nextHeartBeatInterval);
+  }
+
   private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
     return new ContainerIdPBImpl(p);
   }

+ 58 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/YarnServerBuilderUtils.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.yarn.server.utils;
+
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
+import org.apache.hadoop.yarn.server.api.records.MasterKey;
+import org.apache.hadoop.yarn.server.api.records.NodeAction;
+
+/**
+ * Server Builder utilities to construct various objects.
+ *
+ */
+public class YarnServerBuilderUtils {
+
+  private static final RecordFactory recordFactory = RecordFactoryProvider
+      .getRecordFactory(null);
+
+  public static NodeHeartbeatResponse newNodeHeartbeatResponse(int responseId,
+      NodeAction action, List<ContainerId> containersToCleanUp,
+      List<ApplicationId> applicationsToCleanUp,
+      MasterKey masterKey, long nextHeartbeatInterval) {
+    NodeHeartbeatResponse response = recordFactory
+        .newRecordInstance(NodeHeartbeatResponse.class);
+    response.setResponseId(responseId);
+    response.setNodeAction(action);
+    response.setMasterKey(masterKey);
+    response.setNextHeartBeatInterval(nextHeartbeatInterval);
+    if(containersToCleanUp != null) {
+      response.addAllContainersToCleanup(containersToCleanUp);
+    }
+    if(applicationsToCleanUp != null) {
+      response.addAllApplicationsToCleanup(applicationsToCleanUp);
+    }
+    return response;
+  }
+}

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto

@@ -47,4 +47,5 @@ message NodeHeartbeatResponseProto {
   optional NodeActionProto nodeAction = 3;
   repeated ContainerIdProto containers_to_cleanup = 4;
   repeated ApplicationIdProto applications_to_cleanup = 5;
+  optional int64 nextHeartBeatInterval = 6;
 }

+ 5 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java

@@ -119,6 +119,10 @@ public class NodeManager extends CompositeService
     return new DeletionService(exec);
   }
 
+  protected NMContext createNMContext(NMContainerTokenSecretManager containerTokenSecretManager) {
+    return new NMContext(containerTokenSecretManager);
+  }
+
   protected void doSecureLogin() throws IOException {
     SecurityUtil.login(getConfig(), YarnConfiguration.NM_KEYTAB,
         YarnConfiguration.NM_PRINCIPAL);
@@ -137,7 +141,7 @@ public class NodeManager extends CompositeService
       containerTokenSecretManager = new NMContainerTokenSecretManager(conf);
     }
 
-    this.context = new NMContext(containerTokenSecretManager);
+    this.context = createNMContext(containerTokenSecretManager);
 
     this.aclsManager = new ApplicationACLsManager(conf);
 

+ 30 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java

@@ -71,7 +71,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
   private final Dispatcher dispatcher;
 
   private NodeId nodeId;
-  private long heartBeatInterval;
+  private long nextHeartBeatInterval;
   private ResourceTracker resourceTracker;
   private InetSocketAddress rmAddress;
   private Resource totalResource;
@@ -88,6 +88,10 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
   private final NodeHealthCheckerService healthChecker;
   private final NodeManagerMetrics metrics;
 
+  private boolean previousHeartBeatSucceeded;
+  private List<ContainerStatus> previousContainersStatuses =
+      new ArrayList<ContainerStatus>();
+
   public NodeStatusUpdaterImpl(Context context, Dispatcher dispatcher,
       NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) {
     super(NodeStatusUpdaterImpl.class.getName());
@@ -95,6 +99,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
     this.context = context;
     this.dispatcher = dispatcher;
     this.metrics = metrics;
+    this.previousHeartBeatSucceeded = true;
   }
 
   @Override
@@ -103,9 +108,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
         YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS,
         YarnConfiguration.DEFAULT_RM_RESOURCE_TRACKER_ADDRESS,
         YarnConfiguration.DEFAULT_RM_RESOURCE_TRACKER_PORT);
-    this.heartBeatInterval =
-        conf.getLong(YarnConfiguration.NM_TO_RM_HEARTBEAT_INTERVAL_MS,
-            YarnConfiguration.DEFAULT_NM_TO_RM_HEARTBEAT_INTERVAL_MS);
+
     int memoryMb = 
         conf.getInt(
             YarnConfiguration.NM_PMEM_MB, YarnConfiguration.DEFAULT_NM_PMEM_MB);
@@ -316,8 +319,14 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
     NodeStatus nodeStatus = recordFactory.newRecordInstance(NodeStatus.class);
     nodeStatus.setNodeId(this.nodeId);
 
-    int numActiveContainers = 0;
     List<ContainerStatus> containersStatuses = new ArrayList<ContainerStatus>();
+    if(previousHeartBeatSucceeded) {
+      previousContainersStatuses.clear();
+    } else {
+      containersStatuses.addAll(previousContainersStatuses);
+    }
+
+    int numActiveContainers = 0;
     for (Iterator<Entry<ContainerId, Container>> i =
         this.context.getContainers().entrySet().iterator(); i.hasNext();) {
       Entry<ContainerId, Container> e = i.next();
@@ -332,6 +341,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
       LOG.info("Sending out status for container: " + containerStatus);
 
       if (containerStatus.getState() == ContainerState.COMPLETE) {
+        previousContainersStatuses.add(containerStatus);
         // Remove
         i.remove();
 
@@ -394,9 +404,6 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
         while (!isStopped) {
           // Send heartbeat
           try {
-            synchronized (heartbeatMonitor) {
-              heartbeatMonitor.wait(heartBeatInterval);
-            }
             NodeStatus nodeStatus = getNodeStatus();
             nodeStatus.setResponseId(lastHeartBeatID);
             
@@ -409,7 +416,9 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
             }
             NodeHeartbeatResponse response =
               resourceTracker.nodeHeartbeat(request);
-
+            previousHeartBeatSucceeded = true;
+            //get next heartbeat interval from response
+            nextHeartBeatInterval = response.getNextHeartBeatInterval();
             // See if the master-key has rolled over
             if (isSecurityEnabled()) {
               MasterKey updatedMasterKey = response.getMasterKey();
@@ -453,9 +462,21 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
                   new CMgrCompletedAppsEvent(appsToCleanup));
             }
           } catch (Throwable e) {
+            previousHeartBeatSucceeded = false;
             // TODO Better error handling. Thread can die with the rest of the
             // NM still running.
             LOG.error("Caught exception in status-updater", e);
+          } finally {
+            synchronized (heartbeatMonitor) {
+              nextHeartBeatInterval = nextHeartBeatInterval <= 0 ?
+                  YarnConfiguration.DEFAULT_RM_NM_HEARTBEAT_INTERVAL_MS :
+                    nextHeartBeatInterval;
+              try {
+                heartbeatMonitor.wait(nextHeartBeatInterval);
+              } catch (InterruptedException e) {
+                // Do Nothing
+              }
+            }
           }
         }
       }

+ 11 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/api/impl/pb/client/LocalizationProtocolPBClientImpl.java

@@ -17,6 +17,7 @@
 */
 package org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.client;
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 
@@ -35,7 +36,8 @@ import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb.Loc
 
 import com.google.protobuf.ServiceException;
 
-public class LocalizationProtocolPBClientImpl implements LocalizationProtocol {
+public class LocalizationProtocolPBClientImpl implements LocalizationProtocol,
+    Closeable {
 
   private LocalizationProtocolPB proxy;
   
@@ -44,7 +46,14 @@ public class LocalizationProtocolPBClientImpl implements LocalizationProtocol {
     proxy = (LocalizationProtocolPB)RPC.getProxy(
         LocalizationProtocolPB.class, clientVersion, addr, conf);
   }
-  
+
+  @Override
+  public void close() {
+    if (this.proxy != null) {
+      RPC.stopProxy(this.proxy);
+    }
+  }
+
   @Override
   public LocalizerHeartbeatResponse heartbeat(LocalizerStatus status)
     throws YarnRemoteException {

+ 151 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalCacheDirectoryManager.java

@@ -0,0 +1,151 @@
+/**
+ * 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.yarn.server.nodemanager.containermanager.localizer;
+
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.Queue;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+
+/**
+ * {@link LocalCacheDirectoryManager} is used for managing hierarchical
+ * directories for local cache. It will allow to restrict the number of files in
+ * a directory to
+ * {@link YarnConfiguration#NM_LOCAL_CACHE_MAX_FILES_PER_DIRECTORY} which
+ * includes 36 sub-directories (named from 0 to 9 and a to z). Root directory is
+ * represented by an empty string. It internally maintains a vacant directory
+ * queue. As soon as the file count for the directory reaches its limit; new
+ * files will not be created in it until at least one file is deleted from it.
+ * New sub directories are not created unless a
+ * {@link LocalCacheDirectoryManager#getRelativePathForLocalization()} request
+ * is made and nonFullDirectories are empty.
+ * 
+ * Note : this structure only returns relative localization path but doesn't
+ * create one on disk.
+ */
+public class LocalCacheDirectoryManager {
+
+  private final int perDirectoryFileLimit;
+  // total 36 = a to z plus 0 to 9
+  public static final int DIRECTORIES_PER_LEVEL = 36;
+
+  private Queue<Directory> nonFullDirectories;
+  private HashMap<String, Directory> knownDirectories;
+  private int totalSubDirectories;
+
+  public LocalCacheDirectoryManager(Configuration conf) {
+    totalSubDirectories = 0;
+    Directory rootDir = new Directory(totalSubDirectories);
+    nonFullDirectories = new LinkedList<Directory>();
+    knownDirectories = new HashMap<String, Directory>();
+    knownDirectories.put("", rootDir);
+    nonFullDirectories.add(rootDir);
+    this.perDirectoryFileLimit =
+        conf.getInt(YarnConfiguration.NM_LOCAL_CACHE_MAX_FILES_PER_DIRECTORY,
+          YarnConfiguration.DEFAULT_NM_LOCAL_CACHE_MAX_FILES_PER_DIRECTORY) - 36;
+  }
+
+  /**
+   * This method will return relative path from the first available vacant
+   * directory.
+   * 
+   * @return {@link String} relative path for localization
+   */
+  public synchronized String getRelativePathForLocalization() {
+    if (nonFullDirectories.isEmpty()) {
+      totalSubDirectories++;
+      Directory newDir = new Directory(totalSubDirectories);
+      nonFullDirectories.add(newDir);
+      knownDirectories.put(newDir.getRelativePath(), newDir);
+    }
+    Directory subDir = nonFullDirectories.peek();
+    if (subDir.incrementAndGetCount() >= perDirectoryFileLimit) {
+      nonFullDirectories.remove();
+    }
+    return subDir.getRelativePath();
+  }
+
+  /**
+   * This method will reduce the file count for the directory represented by
+   * path. The root directory of this Local cache directory manager is
+   * represented by an empty string.
+   */
+  public synchronized void decrementFileCountForPath(String relPath) {
+    relPath = relPath == null ? "" : relPath.trim();
+    Directory subDir = knownDirectories.get(relPath);
+    int oldCount = subDir.getCount();
+    if (subDir.decrementAndGetCount() < perDirectoryFileLimit
+        && oldCount >= perDirectoryFileLimit) {
+      nonFullDirectories.add(subDir);
+    }
+  }
+
+  /*
+   * It limits the number of files and sub directories in the directory to the
+   * limit LocalCacheDirectoryManager#perDirectoryFileLimit.
+   */
+  static class Directory {
+
+    private final String relativePath;
+    private int fileCount;
+
+    public Directory(int directoryNo) {
+      fileCount = 0;
+      if (directoryNo == 0) {
+        relativePath = "";
+      } else {
+        String tPath = Integer.toString(directoryNo - 1, DIRECTORIES_PER_LEVEL);
+        StringBuffer sb = new StringBuffer();
+        if (tPath.length() == 1) {
+          sb.append(tPath.charAt(0));
+        } else {
+          // this is done to make sure we also reuse 0th sub directory
+          sb.append(Integer.toString(
+            Integer.parseInt(tPath.substring(0, 1), DIRECTORIES_PER_LEVEL) - 1,
+            DIRECTORIES_PER_LEVEL));
+        }
+        for (int i = 1; i < tPath.length(); i++) {
+          sb.append(Path.SEPARATOR).append(tPath.charAt(i));
+        }
+        relativePath = sb.toString();
+      }
+    }
+
+    public int incrementAndGetCount() {
+      return ++fileCount;
+    }
+
+    public int decrementAndGetCount() {
+      return --fileCount;
+    }
+
+    public String getRelativePath() {
+      return relativePath;
+    }
+
+    public int getCount() {
+      return fileCount;
+    }
+  }
+}

+ 6 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTracker.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer;
 
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
@@ -35,6 +36,11 @@ interface LocalResourcesTracker
 
   boolean remove(LocalizedResource req, DeletionService delService);
 
+  Path getPathForLocalization(LocalResourceRequest req, Path localDirPath);
+
   String getUser();
 
+  // TODO: Remove this in favour of EventHandler.handle
+  void localizationCompleted(LocalResourceRequest req, boolean success);
+
 }

+ 128 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java

@@ -26,12 +26,13 @@ import java.util.regex.Pattern;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ResourceEvent;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ResourceEventType;
+
 
 /**
  * A collection of {@link LocalizedResource}s all of same
@@ -49,17 +50,43 @@ class LocalResourcesTrackerImpl implements LocalResourcesTracker {
   private final String user;
   private final Dispatcher dispatcher;
   private final ConcurrentMap<LocalResourceRequest,LocalizedResource> localrsrc;
+  private Configuration conf;
+  /*
+   * This flag controls whether this resource tracker uses hierarchical
+   * directories or not. For PRIVATE and PUBLIC resource trackers it
+   * will be set whereas for APPLICATION resource tracker it would
+   * be false.
+   */
+  private final boolean useLocalCacheDirectoryManager;
+  private ConcurrentHashMap<Path, LocalCacheDirectoryManager> directoryManagers;
+  /*
+   * It is used to keep track of resource into hierarchical directory
+   * while it is getting downloaded. It is useful for reference counting
+   * in case resource localization fails.
+   */
+  private ConcurrentHashMap<LocalResourceRequest, Path>
+    inProgressLocalResourcesMap;
 
-  public LocalResourcesTrackerImpl(String user, Dispatcher dispatcher) {
+  public LocalResourcesTrackerImpl(String user, Dispatcher dispatcher,
+      boolean useLocalCacheDirectoryManager, Configuration conf) {
     this(user, dispatcher,
-        new ConcurrentHashMap<LocalResourceRequest,LocalizedResource>());
+      new ConcurrentHashMap<LocalResourceRequest, LocalizedResource>(),
+      useLocalCacheDirectoryManager, conf);
   }
 
   LocalResourcesTrackerImpl(String user, Dispatcher dispatcher,
-      ConcurrentMap<LocalResourceRequest,LocalizedResource> localrsrc) {
+      ConcurrentMap<LocalResourceRequest,LocalizedResource> localrsrc,
+      boolean useLocalCacheDirectoryManager, Configuration conf) {
     this.user = user;
     this.dispatcher = dispatcher;
     this.localrsrc = localrsrc;
+    this.useLocalCacheDirectoryManager = useLocalCacheDirectoryManager;
+    if ( this.useLocalCacheDirectoryManager) {
+      directoryManagers = new ConcurrentHashMap<Path, LocalCacheDirectoryManager>();
+      inProgressLocalResourcesMap =
+        new ConcurrentHashMap<LocalResourceRequest, Path>();
+    }
+    this.conf = conf;
   }
 
   @Override
@@ -73,6 +100,7 @@ class LocalResourcesTrackerImpl implements LocalResourcesTracker {
         LOG.info("Resource " + rsrc.getLocalPath()
             + " is missing, localizing it again");
         localrsrc.remove(req);
+        decrementFileCountForLocalCacheDirectory(req, rsrc);
         rsrc = null;
       }
       if (null == rsrc) {
@@ -90,7 +118,52 @@ class LocalResourcesTrackerImpl implements LocalResourcesTracker {
     rsrc.handle(event);
   }
 
-  /**
+  /*
+   * Update the file-count statistics for a local cache-directory.
+   * This will retrieve the localized path for the resource from
+   * 1) inProgressRsrcMap if the resource was under localization and it
+   * failed.
+   * 2) LocalizedResource if the resource is already localized.
+   * From this path it will identify the local directory under which the
+   * resource was localized. Then rest of the path will be used to decrement
+   * file count for the HierarchicalSubDirectory pointing to this relative
+   * path.
+   */
+  private void decrementFileCountForLocalCacheDirectory(LocalResourceRequest req,
+      LocalizedResource rsrc) {
+    if ( useLocalCacheDirectoryManager) {
+      Path rsrcPath = null;
+      if (inProgressLocalResourcesMap.containsKey(req)) {
+        // This happens when localization of a resource fails.
+        rsrcPath = inProgressLocalResourcesMap.remove(req);
+      } else if (rsrc != null && rsrc.getLocalPath() != null) {
+        rsrcPath = rsrc.getLocalPath().getParent().getParent();
+      }
+      if (rsrcPath != null) {
+        Path parentPath = new Path(rsrcPath.toUri().getRawPath());
+        while (!directoryManagers.containsKey(parentPath)) {
+          parentPath = parentPath.getParent();
+          if ( parentPath == null) {
+            return;
+          }
+        }
+        if ( parentPath != null) {
+          String parentDir = parentPath.toUri().getRawPath().toString();
+          LocalCacheDirectoryManager dir = directoryManagers.get(parentPath);
+          String rsrcDir = rsrcPath.toUri().getRawPath(); 
+          if (rsrcDir.equals(parentDir)) {
+            dir.decrementFileCountForPath("");
+          } else {
+            dir.decrementFileCountForPath(
+              rsrcDir.substring(
+              parentDir.length() + 1));
+          }
+        }
+      }
+    }
+  }
+
+/**
    * This module checks if the resource which was localized is already present
    * or not
    * 
@@ -100,7 +173,8 @@ class LocalResourcesTrackerImpl implements LocalResourcesTracker {
   public boolean isResourcePresent(LocalizedResource rsrc) {
     boolean ret = true;
     if (rsrc.getState() == ResourceState.LOCALIZED) {
-      File file = new File(rsrc.getLocalPath().toUri().getRawPath().toString());
+      File file = new File(rsrc.getLocalPath().toUri().getRawPath().
+        toString());
       if (!file.exists()) {
         ret = false;
       }
@@ -133,11 +207,11 @@ class LocalResourcesTrackerImpl implements LocalResourcesTracker {
       if (ResourceState.LOCALIZED.equals(rsrc.getState())) {
         delService.delete(getUser(), getPathToDelete(rsrc.getLocalPath()));
       }
+      decrementFileCountForLocalCacheDirectory(rem.getRequest(), rsrc);
       return true;
     }
   }
 
-
   /**
    * Returns the path up to the random directory component.
    */
@@ -163,4 +237,50 @@ class LocalResourcesTrackerImpl implements LocalResourcesTracker {
   public Iterator<LocalizedResource> iterator() {
     return localrsrc.values().iterator();
   }
-}
+
+  /**
+   * @return {@link Path} absolute path for localization which includes local
+   *         directory path and the relative hierarchical path (if use local
+   *         cache directory manager is enabled)
+   * 
+   * @param {@link LocalResourceRequest} Resource localization request to
+   *        localize the resource.
+   * @param {@link Path} local directory path
+   */
+  @Override
+  public Path
+      getPathForLocalization(LocalResourceRequest req, Path localDirPath) {
+    if (useLocalCacheDirectoryManager && localDirPath != null) {
+
+      if (!directoryManagers.containsKey(localDirPath)) {
+        directoryManagers.putIfAbsent(localDirPath,
+          new LocalCacheDirectoryManager(conf));
+      }
+      LocalCacheDirectoryManager dir = directoryManagers.get(localDirPath);
+
+      Path rPath = localDirPath;
+      String hierarchicalPath = dir.getRelativePathForLocalization();
+      // For most of the scenarios we will get root path only which
+      // is an empty string
+      if (!hierarchicalPath.isEmpty()) {
+        rPath = new Path(localDirPath, hierarchicalPath);
+      }
+      inProgressLocalResourcesMap.put(req, rPath);
+      return rPath;
+    } else {
+      return localDirPath;
+    }
+  }
+
+  @Override
+  public void localizationCompleted(LocalResourceRequest req,
+      boolean success) {
+    if (useLocalCacheDirectoryManager) {
+      if (!success) {
+        decrementFileCountForLocalCacheDirectory(req, null);
+      } else {
+        inProgressLocalResourcesMap.remove(req);
+      }
+    }
+  }
+}

+ 41 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java

@@ -64,6 +64,7 @@ import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.util.DiskChecker;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -130,7 +131,7 @@ public class ResourceLocalizationService extends CompositeService
   private RecordFactory recordFactory;
   private final ScheduledExecutorService cacheCleanup;
 
-  private final LocalResourcesTracker publicRsrc;
+  private LocalResourcesTracker publicRsrc;
 
   private LocalDirsHandlerService dirsHandler;
 
@@ -158,7 +159,6 @@ public class ResourceLocalizationService extends CompositeService
     this.delService = delService;
     this.dirsHandler = dirsHandler;
 
-    this.publicRsrc = new LocalResourcesTrackerImpl(null, dispatcher);
     this.cacheCleanup = new ScheduledThreadPoolExecutor(1,
         new ThreadFactoryBuilder()
           .setNameFormat("ResourceLocalizationService Cache Cleanup")
@@ -173,8 +173,26 @@ public class ResourceLocalizationService extends CompositeService
     }
   }
 
+  private void validateConf(Configuration conf) {
+    int perDirFileLimit =
+        conf.getInt(YarnConfiguration.NM_LOCAL_CACHE_MAX_FILES_PER_DIRECTORY,
+          YarnConfiguration.DEFAULT_NM_LOCAL_CACHE_MAX_FILES_PER_DIRECTORY);
+    if (perDirFileLimit <= 36) {
+      LOG.error(YarnConfiguration.NM_LOCAL_CACHE_MAX_FILES_PER_DIRECTORY
+          + " parameter is configured with very low value.");
+      throw new YarnException(
+        YarnConfiguration.NM_LOCAL_CACHE_MAX_FILES_PER_DIRECTORY
+            + " parameter is configured with a value less than 37.");
+    } else {
+      LOG.info("per directory file limit = " + perDirFileLimit);
+    }
+  }
+
   @Override
   public void init(Configuration conf) {
+    this.validateConf(conf);
+    this.publicRsrc =
+        new LocalResourcesTrackerImpl(null, dispatcher, true, conf);
     this.recordFactory = RecordFactoryProvider.getRecordFactory(conf);
 
     try {
@@ -212,6 +230,7 @@ public class ResourceLocalizationService extends CompositeService
         YarnConfiguration.NM_LOCALIZER_ADDRESS,
         YarnConfiguration.DEFAULT_NM_LOCALIZER_ADDRESS,
         YarnConfiguration.DEFAULT_NM_LOCALIZER_PORT);
+
     localizerTracker = createLocalizerTracker(conf);
     addService(localizerTracker);
     dispatcher.register(LocalizerEventType.class, localizerTracker);
@@ -306,15 +325,17 @@ public class ResourceLocalizationService extends CompositeService
   private void handleInitApplicationResources(Application app) {
     // 0) Create application tracking structs
     String userName = app.getUser();
-    privateRsrc.putIfAbsent(userName,
-        new LocalResourcesTrackerImpl(userName, dispatcher));
-    if (null != appRsrc.putIfAbsent(ConverterUtils.toString(app.getAppId()),
-        new LocalResourcesTrackerImpl(app.getUser(), dispatcher))) {
+    privateRsrc.putIfAbsent(userName, new LocalResourcesTrackerImpl(userName,
+      dispatcher, false, super.getConfig()));
+    if (null != appRsrc.putIfAbsent(
+      ConverterUtils.toString(app.getAppId()),
+      new LocalResourcesTrackerImpl(app.getUser(), dispatcher, false, super
+        .getConfig()))) {
       LOG.warn("Initializing application " + app + " already present");
       assert false; // TODO: FIXME assert doesn't help
                     // ^ The condition is benign. Tests should fail and it
-                    //   should appear in logs, but it's an internal error
-                    //   that should have no effect on applications
+                    // should appear in logs, but it's an internal error
+                    // that should have no effect on applications
     }
     // 1) Signal container init
     //
@@ -620,6 +641,13 @@ public class ResourceLocalizationService extends CompositeService
             Path publicDirDestPath = dirsHandler.getLocalPathForWrite(
                 "." + Path.SEPARATOR + ContainerLocalizer.FILECACHE,
                 ContainerLocalizer.getEstimatedSize(resource), true);
+            Path hierarchicalPath =
+              publicRsrc.getPathForLocalization(key, publicDirDestPath);
+            if (!hierarchicalPath.equals(publicDirDestPath)) {
+              publicDirDestPath = hierarchicalPath;
+              DiskChecker.checkDir(
+                new File(publicDirDestPath.toUri().getPath()));
+            }
             pending.put(queue.submit(new FSDownload(
                 lfs, null, conf, publicDirDestPath, resource, new Random())),
                 request);
@@ -654,19 +682,21 @@ public class ResourceLocalizationService extends CompositeService
               assoc.getResource().handle(
                   new ResourceLocalizedEvent(key,
                     local, FileUtil.getDU(new File(local.toUri()))));
+              publicRsrc.localizationCompleted(key, true);
               synchronized (attempts) {
                 attempts.remove(key);
               }
             } catch (ExecutionException e) {
               LOG.info("Failed to download rsrc " + assoc.getResource(),
                   e.getCause());
+              LocalResourceRequest req = assoc.getResource().getRequest();
               dispatcher.getEventHandler().handle(
                   new ContainerResourceFailedEvent(
                     assoc.getContext().getContainerId(),
-                    assoc.getResource().getRequest(), e.getCause()));
+                    req, e.getCause()));
+              publicRsrc.localizationCompleted(req, false);
               List<LocalizerResourceRequestEvent> reqs;
               synchronized (attempts) {
-                LocalResourceRequest req = assoc.getResource().getRequest();
                 reqs = attempts.get(req);
                 if (null == reqs) {
                   LOG.error("Missing pending list for " + req);
@@ -1003,4 +1033,4 @@ public class ResourceLocalizationService extends CompositeService
     del.delete(null, dirPath, new Path[] {});
   }
 
-}
+}

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/MockNodeStatusUpdater.java

@@ -31,6 +31,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequ
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
 import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
+import org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils;
 
 /**
  * This class allows a node manager to run without without communicating with a
@@ -73,9 +74,9 @@ public class MockNodeStatusUpdater extends NodeStatusUpdaterImpl {
       LOG.info("Got heartbeat number " + heartBeatID);
       nodeStatus.setResponseId(heartBeatID++);
 
-      NodeHeartbeatResponse nhResponse = recordFactory
-          .newRecordInstance(NodeHeartbeatResponse.class);
-      nhResponse.setResponseId(heartBeatID);
+      NodeHeartbeatResponse nhResponse = YarnServerBuilderUtils
+          .newNodeHeartbeatResponse(heartBeatID, null, null,
+              null, null, 1000L);
       return nhResponse;
     }
   }

+ 251 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.nodemanager;
 
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
@@ -29,6 +30,7 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.logging.Log;
@@ -43,6 +45,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -58,12 +61,15 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequ
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
 import org.apache.hadoop.yarn.server.api.records.NodeAction;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
+import org.apache.hadoop.yarn.server.nodemanager.NodeManager.NMContext;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl;
 import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
+import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
+import org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils;
 import org.apache.hadoop.yarn.service.Service;
 import org.apache.hadoop.yarn.service.Service.STATE;
 import org.apache.hadoop.yarn.util.BuilderUtils;
@@ -91,6 +97,8 @@ public class TestNodeStatusUpdater {
   private final Configuration conf = createNMConfig();
   private NodeManager nm;
   protected NodeManager rebootedNodeManager;
+  private boolean containerStatusBackupSuccessfully = true;
+  private List<ContainerStatus> completedContainerStatusList = new ArrayList<ContainerStatus>();
 
   @After
   public void tearDown() {
@@ -214,9 +222,8 @@ public class TestNodeStatusUpdater {
         Assert.assertEquals(2, activeContainers.size());
       }
 
-      NodeHeartbeatResponse nhResponse = recordFactory
-          .newRecordInstance(NodeHeartbeatResponse.class);
-      nhResponse.setResponseId(heartBeatID);
+      NodeHeartbeatResponse nhResponse = YarnServerBuilderUtils.
+          newNodeHeartbeatResponse(heartBeatID, null, null, null, null, 1000L);
       return nhResponse;
     }
   }
@@ -237,6 +244,22 @@ public class TestNodeStatusUpdater {
     }
   }
 
+  private class MyNodeStatusUpdater2 extends NodeStatusUpdaterImpl {
+    public ResourceTracker resourceTracker;
+
+    public MyNodeStatusUpdater2(Context context, Dispatcher dispatcher,
+        NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) {
+      super(context, dispatcher, healthChecker, metrics);
+      resourceTracker = new MyResourceTracker4(context);
+    }
+
+    @Override
+    protected ResourceTracker getRMClient() {
+      return resourceTracker;
+    }
+
+  }
+
   private class MyNodeStatusUpdater3 extends NodeStatusUpdaterImpl {
     public ResourceTracker resourceTracker;
     private Context context;
@@ -325,10 +348,9 @@ public class TestNodeStatusUpdater {
       NodeStatus nodeStatus = request.getNodeStatus();
       nodeStatus.setResponseId(heartBeatID++);
       
-      NodeHeartbeatResponse nhResponse = recordFactory
-      .newRecordInstance(NodeHeartbeatResponse.class);
-      nhResponse.setResponseId(heartBeatID);
-      nhResponse.setNodeAction(heartBeatNodeAction);
+      NodeHeartbeatResponse nhResponse = YarnServerBuilderUtils.
+          newNodeHeartbeatResponse(heartBeatID, heartBeatNodeAction, null,
+              null, null, 1000L);
       return nhResponse;
     }
   }
@@ -361,10 +383,9 @@ public class TestNodeStatusUpdater {
       LOG.info("Got heartBeatId: [" + heartBeatID +"]");
       NodeStatus nodeStatus = request.getNodeStatus();
       nodeStatus.setResponseId(heartBeatID++);
-      NodeHeartbeatResponse nhResponse =
-              recordFactory.newRecordInstance(NodeHeartbeatResponse.class);
-      nhResponse.setResponseId(heartBeatID);
-      nhResponse.setNodeAction(heartBeatNodeAction);
+      NodeHeartbeatResponse nhResponse = YarnServerBuilderUtils.
+          newNodeHeartbeatResponse(heartBeatID, heartBeatNodeAction, null,
+              null, null, 1000L);
 
       if (nodeStatus.getKeepAliveApplications() != null
           && nodeStatus.getKeepAliveApplications().size() > 0) {
@@ -386,6 +407,104 @@ public class TestNodeStatusUpdater {
     }
   }
 
+  private class MyResourceTracker4 implements ResourceTracker {
+
+    public NodeAction registerNodeAction = NodeAction.NORMAL;
+    public NodeAction heartBeatNodeAction = NodeAction.NORMAL;
+    private Context context;
+
+    public MyResourceTracker4(Context context) {
+      this.context = context;
+    }
+
+    @Override
+    public RegisterNodeManagerResponse registerNodeManager(
+        RegisterNodeManagerRequest request) throws YarnRemoteException {
+      RegisterNodeManagerResponse response = recordFactory
+          .newRecordInstance(RegisterNodeManagerResponse.class);
+      response.setNodeAction(registerNodeAction);
+      return response;
+    }
+
+    @Override
+    public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request)
+        throws YarnRemoteException {
+      try {
+        if (heartBeatID == 0) {
+          Assert.assertEquals(request.getNodeStatus().getContainersStatuses()
+              .size(), 0);
+          Assert.assertEquals(context.getContainers().size(), 0);
+        } else if (heartBeatID == 1) {
+          Assert.assertEquals(request.getNodeStatus().getContainersStatuses()
+              .size(), 5);
+          Assert.assertTrue(request.getNodeStatus().getContainersStatuses()
+              .get(0).getState() == ContainerState.RUNNING
+              && request.getNodeStatus().getContainersStatuses().get(0)
+                  .getContainerId().getId() == 1);
+          Assert.assertTrue(request.getNodeStatus().getContainersStatuses()
+              .get(1).getState() == ContainerState.RUNNING
+              && request.getNodeStatus().getContainersStatuses().get(1)
+                  .getContainerId().getId() == 2);
+          Assert.assertTrue(request.getNodeStatus().getContainersStatuses()
+              .get(2).getState() == ContainerState.COMPLETE
+              && request.getNodeStatus().getContainersStatuses().get(2)
+                  .getContainerId().getId() == 3);
+          Assert.assertTrue(request.getNodeStatus().getContainersStatuses()
+              .get(3).getState() == ContainerState.COMPLETE
+              && request.getNodeStatus().getContainersStatuses().get(3)
+                  .getContainerId().getId() == 4);
+          Assert.assertTrue(request.getNodeStatus().getContainersStatuses()
+              .get(4).getState() == ContainerState.RUNNING
+              && request.getNodeStatus().getContainersStatuses().get(4)
+                  .getContainerId().getId() == 5);
+          throw new YarnException("Lost the heartbeat response");
+        } else if (heartBeatID == 2) {
+          Assert.assertEquals(request.getNodeStatus().getContainersStatuses()
+              .size(), 7);
+          Assert.assertTrue(request.getNodeStatus().getContainersStatuses()
+              .get(0).getState() == ContainerState.COMPLETE
+              && request.getNodeStatus().getContainersStatuses().get(0)
+                  .getContainerId().getId() == 3);
+          Assert.assertTrue(request.getNodeStatus().getContainersStatuses()
+              .get(1).getState() == ContainerState.COMPLETE
+              && request.getNodeStatus().getContainersStatuses().get(1)
+                  .getContainerId().getId() == 4);
+          Assert.assertTrue(request.getNodeStatus().getContainersStatuses()
+              .get(2).getState() == ContainerState.RUNNING
+              && request.getNodeStatus().getContainersStatuses().get(2)
+                  .getContainerId().getId() == 1);
+          Assert.assertTrue(request.getNodeStatus().getContainersStatuses()
+              .get(3).getState() == ContainerState.RUNNING
+              && request.getNodeStatus().getContainersStatuses().get(3)
+                  .getContainerId().getId() == 2);
+          Assert.assertTrue(request.getNodeStatus().getContainersStatuses()
+              .get(4).getState() == ContainerState.RUNNING
+              && request.getNodeStatus().getContainersStatuses().get(4)
+                  .getContainerId().getId() == 5);
+          Assert.assertTrue(request.getNodeStatus().getContainersStatuses()
+              .get(5).getState() == ContainerState.RUNNING
+              && request.getNodeStatus().getContainersStatuses().get(5)
+                  .getContainerId().getId() == 6);
+          Assert.assertTrue(request.getNodeStatus().getContainersStatuses()
+              .get(6).getState() == ContainerState.COMPLETE
+              && request.getNodeStatus().getContainersStatuses().get(6)
+                  .getContainerId().getId() == 7);
+        }
+      } catch (AssertionError error) {
+        LOG.info(error);
+        containerStatusBackupSuccessfully = false;
+      } finally {
+        heartBeatID++;
+      }
+      NodeStatus nodeStatus = request.getNodeStatus();
+      nodeStatus.setResponseId(heartBeatID);
+      NodeHeartbeatResponse nhResponse =
+          YarnServerBuilderUtils.newNodeHeartbeatResponse(heartBeatID,
+              heartBeatNodeAction, null, null, null, 1000L);
+      return nhResponse;
+    }
+  }
+
   @Before
   public void clearError() {
     nmStartError = null;
@@ -727,6 +846,127 @@ public class TestNodeStatusUpdater {
     }
   }
 
+  /**
+   * Test completed containerStatus get back up when heart beat lost
+   */
+  @Test(timeout = 20000)
+  public void testCompletedContainerStatusBackup() throws Exception {
+    nm = new NodeManager() {
+      @Override
+      protected NodeStatusUpdater createNodeStatusUpdater(Context context,
+          Dispatcher dispatcher, NodeHealthCheckerService healthChecker) {
+        MyNodeStatusUpdater2 myNodeStatusUpdater =
+            new MyNodeStatusUpdater2(context, dispatcher, healthChecker,
+                metrics);
+        return myNodeStatusUpdater;
+      }
+
+      @Override
+      protected NMContext createNMContext(
+          NMContainerTokenSecretManager containerTokenSecretManager) {
+        return new MyNMContext(containerTokenSecretManager);
+      }
+
+    };
+
+    YarnConfiguration conf = createNMConfig();
+    nm.init(conf);
+    nm.start();
+
+    int waitCount = 0;
+    while (heartBeatID <= 3 && waitCount++ != 20) {
+      Thread.sleep(500);
+    }
+    if(!containerStatusBackupSuccessfully) {
+      Assert.fail("ContainerStatus Backup failed");
+    }
+    nm.stop();
+  }
+
+  private class MyNMContext extends NMContext {
+    ConcurrentMap<ContainerId, Container> containers =
+        new ConcurrentSkipListMap<ContainerId, Container>();
+
+    public MyNMContext(NMContainerTokenSecretManager
+        containerTokenSecretManager) {
+      super(containerTokenSecretManager);
+    }
+
+    @Override
+    public ConcurrentMap<ContainerId, Container> getContainers() {
+      if (heartBeatID == 0) {
+        return containers;
+      } else if (heartBeatID == 1) {
+        ContainerStatus containerStatus1 =
+            createContainerStatus(1, ContainerState.RUNNING);
+        Container container1 = getMockContainer(containerStatus1);
+        containers.put(containerStatus1.getContainerId(), container1);
+
+        ContainerStatus containerStatus2 =
+            createContainerStatus(2, ContainerState.RUNNING);
+        Container container2 = getMockContainer(containerStatus2);
+        containers.put(containerStatus2.getContainerId(), container2);
+
+        ContainerStatus containerStatus3 =
+            createContainerStatus(3, ContainerState.COMPLETE);
+        Container container3 = getMockContainer(containerStatus3);
+        containers.put(containerStatus3.getContainerId(), container3);
+        completedContainerStatusList.add(containerStatus3);
+
+        ContainerStatus containerStatus4 =
+            createContainerStatus(4, ContainerState.COMPLETE);
+        Container container4 = getMockContainer(containerStatus4);
+        containers.put(containerStatus4.getContainerId(), container4);
+        completedContainerStatusList.add(containerStatus4);
+
+        ContainerStatus containerStatus5 =
+            createContainerStatus(5, ContainerState.RUNNING);
+        Container container5 = getMockContainer(containerStatus5);
+        containers.put(containerStatus5.getContainerId(), container5);
+
+        return containers;
+      } else if (heartBeatID == 2) {
+        ContainerStatus containerStatus6 =
+            createContainerStatus(6, ContainerState.RUNNING);
+        Container container6 = getMockContainer(containerStatus6);
+        containers.put(containerStatus6.getContainerId(), container6);
+
+        ContainerStatus containerStatus7 =
+            createContainerStatus(7, ContainerState.COMPLETE);
+        Container container7 = getMockContainer(containerStatus7);
+        containers.put(containerStatus7.getContainerId(), container7);
+        completedContainerStatusList.add(containerStatus7);
+
+        return containers;
+      } else {
+        containers.clear();
+
+        return containers;
+      }
+    }
+
+    private ContainerStatus createContainerStatus(int id,
+        ContainerState containerState) {
+      ApplicationId applicationId =
+          BuilderUtils.newApplicationId(System.currentTimeMillis(), id);
+      ApplicationAttemptId applicationAttemptId =
+          BuilderUtils.newApplicationAttemptId(applicationId, id);
+      ContainerId contaierId =
+          BuilderUtils.newContainerId(applicationAttemptId, id);
+      ContainerStatus containerStatus =
+          BuilderUtils.newContainerStatus(contaierId, containerState,
+              "test_containerStatus: id=" + id + ", containerState: "
+                  + containerState, 0);
+      return containerStatus;
+    }
+
+    private Container getMockContainer(ContainerStatus containerStatus) {
+      Container container = mock(Container.class);
+      when(container.cloneAndGetContainerStatus()).thenReturn(containerStatus);
+      return container;
+    }
+  }
+
   private void verifyNodeStartFailure(String errMessage) {
     YarnConfiguration conf = createNMConfig();
     nm.init(conf);

+ 112 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalCacheDirectoryManager.java

@@ -0,0 +1,112 @@
+/**
+ * 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.yarn.server.nodemanager.containermanager.localizer;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.junit.Test;
+
+public class TestLocalCacheDirectoryManager {
+
+  @Test(timeout = 10000)
+  public void testHierarchicalSubDirectoryCreation() {
+    // setting per directory file limit to 1.
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.set(YarnConfiguration.NM_LOCAL_CACHE_MAX_FILES_PER_DIRECTORY, "37");
+
+    LocalCacheDirectoryManager hDir = new LocalCacheDirectoryManager(conf);
+    // Test root directory path = ""
+    Assert.assertTrue(hDir.getRelativePathForLocalization().isEmpty());
+
+    // Testing path generation from "0" to "0/0/z/z"
+    for (int i = 1; i <= 37 * 36 * 36; i++) {
+      StringBuffer sb = new StringBuffer();
+      String num = Integer.toString(i - 1, 36);
+      if (num.length() == 1) {
+        sb.append(num.charAt(0));
+      } else {
+        sb.append(Integer.toString(
+          Integer.parseInt(num.substring(0, 1), 36) - 1, 36));
+      }
+      for (int j = 1; j < num.length(); j++) {
+        sb.append(Path.SEPARATOR).append(num.charAt(j));
+      }
+      Assert.assertEquals(sb.toString(), hDir.getRelativePathForLocalization());
+    }
+
+    String testPath1 = "4";
+    String testPath2 = "2";
+    /*
+     * Making sure directory "4" and "2" becomes non-full so that they are
+     * reused for future getRelativePathForLocalization() calls in the order
+     * they are freed.
+     */
+    hDir.decrementFileCountForPath(testPath1);
+    hDir.decrementFileCountForPath(testPath2);
+    // After below call directory "4" should become full.
+    Assert.assertEquals(testPath1, hDir.getRelativePathForLocalization());
+    Assert.assertEquals(testPath2, hDir.getRelativePathForLocalization());
+  }
+
+  @Test(timeout = 10000)
+  public void testMinimumPerDirectoryFileLimit() {
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.set(YarnConfiguration.NM_LOCAL_CACHE_MAX_FILES_PER_DIRECTORY, "1");
+    Exception e = null;
+    ResourceLocalizationService service =
+        new ResourceLocalizationService(null, null, null, null);
+    try {
+      service.init(conf);
+    } catch (Exception e1) {
+      e = e1;
+    }
+    Assert.assertNotNull(e);
+    Assert.assertEquals(YarnException.class, e.getClass());
+    Assert.assertEquals(e.getMessage(),
+      YarnConfiguration.NM_LOCAL_CACHE_MAX_FILES_PER_DIRECTORY
+          + " parameter is configured with a value less than 37.");
+
+  }
+
+  @Test(timeout = 1000)
+  public void testDirectoryStateChangeFromFullToNonFull() {
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.set(YarnConfiguration.NM_LOCAL_CACHE_MAX_FILES_PER_DIRECTORY, "40");
+    LocalCacheDirectoryManager dir = new LocalCacheDirectoryManager(conf);
+
+    // checking for first four paths
+    String rootPath = "";
+    String firstSubDir = "0";
+    for (int i = 0; i < 4; i++) {
+      Assert.assertEquals(rootPath, dir.getRelativePathForLocalization());
+    }
+    // Releasing two files from the root directory.
+    dir.decrementFileCountForPath(rootPath);
+    dir.decrementFileCountForPath(rootPath);
+    // Space for two files should be available in root directory.
+    Assert.assertEquals(rootPath, dir.getRelativePathForLocalization());
+    Assert.assertEquals(rootPath, dir.getRelativePathForLocalization());
+    // As no space is now available in root directory so it should be from
+    // first sub directory
+    Assert.assertEquals(firstSubDir, dir.getRelativePathForLocalization());
+  }
+}

+ 117 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalResourcesTrackerImpl.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -50,17 +51,17 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.even
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ResourceRequestEvent;
 import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.junit.Test;
-import org.mortbay.log.Log;
 
 public class TestLocalResourcesTrackerImpl {
 
-  @Test
+  @Test(timeout=10000)
   @SuppressWarnings("unchecked")
   public void test() {
     String user = "testuser";
     DrainDispatcher dispatcher = null;
     try {
-      dispatcher = createDispatcher(new Configuration());
+      Configuration conf = new Configuration();
+      dispatcher = createDispatcher(conf);
       EventHandler<LocalizerEvent> localizerEventHandler =
           mock(EventHandler.class);
       EventHandler<LocalizerEvent> containerEventHandler =
@@ -86,7 +87,8 @@ public class TestLocalResourcesTrackerImpl {
       localrsrc.put(req1, lr1);
       localrsrc.put(req2, lr2);
       LocalResourcesTracker tracker =
-          new LocalResourcesTrackerImpl(user, dispatcher, localrsrc);
+          new LocalResourcesTrackerImpl(user, dispatcher, localrsrc, false,
+            conf);
 
       ResourceEvent req11Event =
           new ResourceRequestEvent(req1, LocalResourceVisibility.PUBLIC, lc1);
@@ -152,13 +154,14 @@ public class TestLocalResourcesTrackerImpl {
     }
   }
 
-  @Test
+  @Test(timeout=10000)
   @SuppressWarnings("unchecked")
   public void testConsistency() {
     String user = "testuser";
     DrainDispatcher dispatcher = null;
     try {
-      dispatcher = createDispatcher(new Configuration());
+      Configuration conf = new Configuration();
+      dispatcher = createDispatcher(conf);
       EventHandler<LocalizerEvent> localizerEventHandler = mock(EventHandler.class);
       EventHandler<LocalizerEvent> containerEventHandler = mock(EventHandler.class);
       dispatcher.register(LocalizerEventType.class, localizerEventHandler);
@@ -172,7 +175,7 @@ public class TestLocalResourcesTrackerImpl {
       ConcurrentMap<LocalResourceRequest, LocalizedResource> localrsrc = new ConcurrentHashMap<LocalResourceRequest, LocalizedResource>();
       localrsrc.put(req1, lr1);
       LocalResourcesTracker tracker = new LocalResourcesTrackerImpl(user,
-          dispatcher, localrsrc);
+          dispatcher, localrsrc, false, conf);
 
       ResourceEvent req11Event = new ResourceRequestEvent(req1,
           LocalResourceVisibility.PUBLIC, lc1);
@@ -221,6 +224,113 @@ public class TestLocalResourcesTrackerImpl {
     }
   }
 
+  @Test(timeout = 100000)
+  @SuppressWarnings("unchecked")
+  public void testHierarchicalLocalCacheDirectories() {
+    String user = "testuser";
+    DrainDispatcher dispatcher = null;
+    try {
+      Configuration conf = new Configuration();
+      // setting per directory file limit to 1.
+      conf.set(YarnConfiguration.NM_LOCAL_CACHE_MAX_FILES_PER_DIRECTORY, "37");
+      dispatcher = createDispatcher(conf);
+
+      EventHandler<LocalizerEvent> localizerEventHandler =
+          mock(EventHandler.class);
+      EventHandler<LocalizerEvent> containerEventHandler =
+          mock(EventHandler.class);
+      dispatcher.register(LocalizerEventType.class, localizerEventHandler);
+      dispatcher.register(ContainerEventType.class, containerEventHandler);
+
+      DeletionService mockDelService = mock(DeletionService.class);
+
+      ConcurrentMap<LocalResourceRequest, LocalizedResource> localrsrc =
+          new ConcurrentHashMap<LocalResourceRequest, LocalizedResource>();
+      LocalResourcesTracker tracker = new LocalResourcesTrackerImpl(user,
+          dispatcher, localrsrc, true, conf);
+
+      // This is a random path. NO File creation will take place at this place.
+      Path localDir = new Path("/tmp");
+
+      // Container 1 needs lr1 resource
+      ContainerId cId1 = BuilderUtils.newContainerId(1, 1, 1, 1);
+      LocalResourceRequest lr1 = createLocalResourceRequest(user, 1, 1,
+          LocalResourceVisibility.PUBLIC);
+      LocalizerContext lc1 = new LocalizerContext(user, cId1, null);
+
+      // Container 1 requests lr1 to be localized
+      ResourceEvent reqEvent1 = new ResourceRequestEvent(lr1,
+          LocalResourceVisibility.PUBLIC, lc1);
+      tracker.handle(reqEvent1);
+
+      // Simulate the process of localization of lr1
+      Path hierarchicalPath1 = tracker.getPathForLocalization(lr1, localDir);
+      // Simulate lr1 getting localized
+      ResourceLocalizedEvent rle =
+          new ResourceLocalizedEvent(lr1,
+              new Path(hierarchicalPath1.toUri().toString() +
+                  Path.SEPARATOR + "file1"), 120);
+      tracker.handle(rle);
+      // Localization successful.
+      tracker.localizationCompleted(lr1, true);
+
+      LocalResourceRequest lr2 = createLocalResourceRequest(user, 3, 3,
+          LocalResourceVisibility.PUBLIC);
+      Path hierarchicalPath2 = tracker.getPathForLocalization(lr2, localDir);
+      // localization failed.
+      tracker.localizationCompleted(lr2, false);
+
+      /*
+       * The path returned for two localization should be different because we
+       * are limiting one file per sub-directory.
+       */
+      Assert.assertNotSame(hierarchicalPath1, hierarchicalPath2);
+
+      LocalResourceRequest lr3 = createLocalResourceRequest(user, 2, 2,
+          LocalResourceVisibility.PUBLIC);
+      ResourceEvent reqEvent3 = new ResourceRequestEvent(lr3,
+          LocalResourceVisibility.PUBLIC, lc1);
+      tracker.handle(reqEvent3);
+      Path hierarchicalPath3 = tracker.getPathForLocalization(lr3, localDir);
+      tracker.localizationCompleted(lr3, true);
+
+      // Verifying that path created is inside the subdirectory
+      Assert.assertEquals(hierarchicalPath3.toUri().toString(),
+          hierarchicalPath1.toUri().toString() + Path.SEPARATOR + "0");
+
+      // Container 1 releases resource lr1
+      ResourceEvent relEvent1 = new ResourceReleaseEvent(lr1, cId1);
+      tracker.handle(relEvent1);
+
+      // Validate the file counts now
+      int resources = 0;
+      Iterator<LocalizedResource> iter = tracker.iterator();
+      while (iter.hasNext()) {
+        iter.next();
+        resources++;
+      }
+      // There should be only two resources lr1 and lr3 now.
+      Assert.assertEquals(2, resources);
+
+      // Now simulate cache cleanup - removes unused resources.
+      iter = tracker.iterator();
+      while (iter.hasNext()) {
+        LocalizedResource rsrc = iter.next();
+        if (rsrc.getRefCount() == 0) {
+          Assert.assertTrue(tracker.remove(rsrc, mockDelService));
+          resources--;
+        }
+      }
+      // lr1 is not used by anyone and will be removed, only lr3 will hang
+      // around
+      Assert.assertEquals(1, resources);
+    } finally {
+      if (dispatcher != null) {
+        dispatcher.stop();
+      }
+    }
+  }
+
   private boolean createdummylocalizefile(Path path) {
     boolean ret = false;
     File file = new File(path.toUri().getRawPath().toString());

+ 3 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceRetention.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
@@ -76,10 +77,11 @@ public class TestResourceRetention {
 
   LocalResourcesTracker createMockTracker(String user, final long rsrcSize,
       long nRsrcs, long timestamp, long tsstep) {
+    Configuration conf = new Configuration();
     ConcurrentMap<LocalResourceRequest,LocalizedResource> trackerResources =
       new ConcurrentHashMap<LocalResourceRequest,LocalizedResource>();
     LocalResourcesTracker ret = spy(new LocalResourcesTrackerImpl(user, null,
-          trackerResources));
+          trackerResources, false, conf));
     for (int i = 0; i < nRsrcs; ++i) {
       final LocalResourceRequest req = new LocalResourceRequest(
           new Path("file:///" + user + "/rsrc" + i), timestamp + i * tsstep,

+ 15 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.PolicyProvider;
+import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -51,6 +52,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeReconnectEvent
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStatusEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider;
+import org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils;
 import org.apache.hadoop.yarn.service.AbstractService;
 import org.apache.hadoop.yarn.util.RackResolver;
 
@@ -67,6 +69,7 @@ public class ResourceTrackerService extends AbstractService implements
   private final NMLivelinessMonitor nmLivelinessMonitor;
   private final RMContainerTokenSecretManager containerTokenSecretManager;
 
+  private long nextHeartBeatInterval;
   private Server server;
   private InetSocketAddress resourceTrackerAddress;
 
@@ -100,6 +103,14 @@ public class ResourceTrackerService extends AbstractService implements
         YarnConfiguration.DEFAULT_RM_RESOURCE_TRACKER_PORT);
 
     RackResolver.init(conf);
+    nextHeartBeatInterval =
+        conf.getLong(YarnConfiguration.RM_NM_HEARTBEAT_INTERVAL_MS,
+            YarnConfiguration.DEFAULT_RM_NM_HEARTBEAT_INTERVAL_MS);
+    if (nextHeartBeatInterval <= 0) {
+      throw new YarnException("Invalid Configuration. "
+          + YarnConfiguration.RM_NM_HEARTBEAT_INTERVAL_MS
+          + " should be larger than 0.");
+    }
     super.init(conf);
   }
 
@@ -223,9 +234,6 @@ public class ResourceTrackerService extends AbstractService implements
           new RMNodeEvent(nodeId, RMNodeEventType.DECOMMISSION));
       return shutDown;
     }
-
-    NodeHeartbeatResponse nodeHeartBeatResponse = recordFactory
-        .newRecordInstance(NodeHeartbeatResponse.class);
     
     // 3. Check if it's a 'fresh' heartbeat i.e. not duplicate heartbeat
     NodeHeartbeatResponse lastNodeHeartbeatResponse = rmNode.getLastNodeHeartBeatResponse();
@@ -246,10 +254,11 @@ public class ResourceTrackerService extends AbstractService implements
     }
 
     // Heartbeat response
-    nodeHeartBeatResponse.setResponseId(lastNodeHeartbeatResponse.getResponseId() + 1);
+    NodeHeartbeatResponse nodeHeartBeatResponse = YarnServerBuilderUtils
+        .newNodeHeartbeatResponse(lastNodeHeartbeatResponse.
+            getResponseId() + 1, NodeAction.NORMAL, null, null, null,
+            nextHeartBeatInterval);
     rmNode.updateNodeHeartbeatResponseForCleanup(nodeHeartBeatResponse);
-    nodeHeartBeatResponse.setNodeAction(NodeAction.NORMAL);
-
     // Check if node's masterKey needs to be updated and if the currentKey has
     // roller over, send it across
     if (isSecurityEnabled()) {

+ 5 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java

@@ -771,6 +771,11 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       // Set the masterContainer
       appAttempt.setMasterContainer(amContainerAllocation.getContainers().get(
                                                                            0));
+      // Updating CLC's resource is no longer necessary once YARN-486 is
+      // completed, because nothing from Container to CLC will be copied into
+      // CLC then.
+      appAttempt.getSubmissionContext().getAMContainerSpec().setResource(
+          appAttempt.getMasterContainer().getResource());
 
       RMStateStore store = appAttempt.rmContext.getStateStore();
       appAttempt.storeAttempt(store);

+ 1 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java

@@ -288,14 +288,9 @@ public class RMContainerImpl implements RMContainer {
     public void transition(RMContainerImpl container, RMContainerEvent event) {
       RMContainerFinishedEvent finishedEvent = (RMContainerFinishedEvent) event;
 
-      // Update container-status for diagnostics. Today we completely
-      // replace it on finish. We may just need to update diagnostics.
-      container.container.setContainerStatus(finishedEvent
-          .getRemoteContainerStatus());
-
       // Inform AppAttempt
       container.eventHandler.handle(new RMAppAttemptContainerFinishedEvent(
-          container.appAttemptId, container.container.getContainerStatus()));
+          container.appAttemptId, finishedEvent.getRemoteContainerStatus()));
     }
   }
 

+ 1 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java

@@ -108,8 +108,7 @@ public class SchedulerUtils {
     Resource normalized = 
         Resources.normalize(
             resourceCalculator, ask.getCapability(), minimumResource);
-    ask.getCapability().setMemory(normalized.getMemory());
-    ask.getCapability().setVirtualCores(normalized.getVirtualCores());
+    ask.setCapability(normalized);
   }
 
 }

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java

@@ -108,7 +108,7 @@ implements ResourceScheduler, CapacitySchedulerContext, Configurable {
     new Comparator<FiCaSchedulerApp>() {
     @Override
     public int compare(FiCaSchedulerApp a1, FiCaSchedulerApp a2) {
-      return a1.getApplicationId().getId() - a2.getApplicationId().getId();
+      return a1.getApplicationId().compareTo(a2.getApplicationId());
     }
   };
 

+ 3 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java

@@ -26,7 +26,6 @@ import java.util.Map;
 
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeHealthStatus;
@@ -71,11 +70,11 @@ public class MockNM {
     this.resourceTracker = resourceTracker;
   }
 
-  public void containerStatus(Container container) throws Exception {
+  public void containerStatus(ContainerStatus containerStatus) throws Exception {
     Map<ApplicationId, List<ContainerStatus>> conts = 
         new HashMap<ApplicationId, List<ContainerStatus>>();
-    conts.put(container.getId().getApplicationAttemptId().getApplicationId(), 
-        Arrays.asList(new ContainerStatus[] { container.getContainerStatus() }));
+    conts.put(containerStatus.getContainerId().getApplicationAttemptId().getApplicationId(),
+        Arrays.asList(new ContainerStatus[] { containerStatus }));
     nodeHeartbeat(conts, true);
   }
 

+ 13 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java

@@ -76,6 +76,9 @@ public class NodeManager implements ContainerManager {
   final Map<ApplicationId, List<Container>> containers = 
     new HashMap<ApplicationId, List<Container>>();
   
+  final Map<Container, ContainerStatus> containerStatusMap =
+      new HashMap<Container, ContainerStatus>();
+
   public NodeManager(String hostName, int containerManagerPort, int httpPort,
       String rackName, Resource capability,
       ResourceTrackerService resourceTrackerService, RMContext rmContext)
@@ -137,7 +140,7 @@ public class NodeManager implements ContainerManager {
     List<ContainerStatus> containerStatuses = new ArrayList<ContainerStatus>();
     for (List<Container> appContainers : containers.values()) {
       for (Container container : appContainers) {
-        containerStatuses.add(container.getContainerStatus());
+        containerStatuses.add(containerStatusMap.get(container));
       }
     }
     return containerStatuses;
@@ -189,8 +192,11 @@ public class NodeManager implements ContainerManager {
             null, null                                 // DKDC - Doesn't matter
             );
 
+    ContainerStatus containerStatus =
+        BuilderUtils.newContainerStatus(container.getId(), ContainerState.NEW,
+            "", -1000);
     applicationContainers.add(container);
-    
+    containerStatusMap.put(container, containerStatus);
     Resources.subtractFrom(available, containerLaunchContext.getResource());
     Resources.addTo(used, containerLaunchContext.getResource());
     
@@ -223,7 +229,9 @@ public class NodeManager implements ContainerManager {
     List<Container> applicationContainers = containers.get(applicationId);
     for (Container c : applicationContainers) {
       if (c.getId().compareTo(containerID) == 0) {
-        c.setState(ContainerState.COMPLETE);
+        ContainerStatus containerStatus = containerStatusMap.get(c);
+        containerStatus.setState(ContainerState.COMPLETE);
+        containerStatusMap.put(c, containerStatus);
       }
     }
     
@@ -277,8 +285,8 @@ public class NodeManager implements ContainerManager {
     }
     GetContainerStatusResponse response = 
         recordFactory.newRecordInstance(GetContainerStatusResponse.class);
-    if (container != null && container.getContainerStatus() != null) {
-      response.setStatus(container.getContainerStatus());
+    if (container != null && containerStatusMap.get(container).getState() != null) {
+      response.setStatus(containerStatusMap.get(container));
     }
     return response;
   }

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestFifoScheduler.java

@@ -139,8 +139,9 @@ public class TestFifoScheduler {
 
     Container c1 = allocated1.get(0);
     Assert.assertEquals(GB, c1.getResource().getMemory());
-    c1.setState(ContainerState.COMPLETE);
-    nm1.containerStatus(c1);
+    ContainerStatus containerStatus = BuilderUtils.newContainerStatus(
+        c1.getId(), ContainerState.COMPLETE, "", 0);
+    nm1.containerStatus(containerStatus);
     int waitCount = 0;
     while (attempt1.getJustFinishedContainers().size() < 1
         && waitCount++ != 20) {

+ 23 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java

@@ -53,6 +53,29 @@ public class TestResourceTrackerService {
   private File hostFile = new File(TEMP_DIR + File.separator + "hostFile.txt");
   private MockRM rm;
 
+  /**
+   * Test RM read NM next heartBeat Interval correctly from Configuration file,
+   * and NM get next heartBeat Interval from RM correctly
+   */
+  @Test (timeout = 5000)
+  public void testGetNextHeartBeatInterval() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(YarnConfiguration.RM_NM_HEARTBEAT_INTERVAL_MS, "4000");
+
+    rm = new MockRM(conf);
+    rm.start();
+
+    MockNM nm1 = rm.registerNode("host1:1234", 5120);
+    MockNM nm2 = rm.registerNode("host2:5678", 10240);
+
+    NodeHeartbeatResponse nodeHeartbeat = nm1.nodeHeartbeat(true);
+    Assert.assertEquals(4000, nodeHeartbeat.getNextHeartBeatInterval());
+
+    NodeHeartbeatResponse nodeHeartbeat2 = nm2.nodeHeartbeat(true);
+    Assert.assertEquals(4000, nodeHeartbeat2.getNextHeartBeatInterval());
+
+  }
+
   /**
    * Decommissioning using a pre-configured include hosts file
    */

+ 9 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java

@@ -202,9 +202,10 @@ public class TestRMAppAttemptTransitions {
     submissionContext = mock(ApplicationSubmissionContext.class);
     when(submissionContext.getUser()).thenReturn(user);
     when(submissionContext.getQueue()).thenReturn(queue);
-    ContainerLaunchContext amContainerSpec = mock(ContainerLaunchContext.class);
-    Resource resource = mock(Resource.class);
-    when(amContainerSpec.getResource()).thenReturn(resource);
+    Resource resource = BuilderUtils.newResource(1536, 1);
+    ContainerLaunchContext amContainerSpec =
+        BuilderUtils.newContainerLaunchContext(null, user, resource, null, null,
+            null, null, null, null);
     when(submissionContext.getAMContainerSpec()).thenReturn(amContainerSpec);
     
     unmanagedAM = false;
@@ -469,8 +470,10 @@ public class TestRMAppAttemptTransitions {
     
     // Mock the allocation of AM container 
     Container container = mock(Container.class);
+    Resource resource = BuilderUtils.newResource(2048, 1);
     when(container.getId()).thenReturn(
         BuilderUtils.newContainerId(applicationAttempt.getAppAttemptId(), 1));
+    when(container.getResource()).thenReturn(resource);
     Allocation allocation = mock(Allocation.class);
     when(allocation.getContainers()).
         thenReturn(Collections.singletonList(container));
@@ -491,6 +494,9 @@ public class TestRMAppAttemptTransitions {
     applicationAttempt.handle(
         new RMAppAttemptStoredEvent(
             applicationAttempt.getAppAttemptId(), null));
+    assertEquals(resource,
+        applicationAttempt.getSubmissionContext()
+        .getAMContainerSpec().getResource());
     
     testAppAttemptAllocatedState(container);
     

+ 0 - 22
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java

@@ -19,7 +19,6 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
@@ -43,35 +42,23 @@ public class TestSchedulerUtils {
 
     // case negative memory
     ask.setCapability(Resources.createResource(-1024));
-    Resource before = ask.getCapability();
     SchedulerUtils.normalizeRequest(ask, resourceCalculator, null, minResource);
-    Resource after = ask.getCapability();
     assertEquals(minMemory, ask.getCapability().getMemory());
-    assertTrue(before == after);
 
     // case zero memory
     ask.setCapability(Resources.createResource(0));
-    before = ask.getCapability();
     SchedulerUtils.normalizeRequest(ask, resourceCalculator, null, minResource);
-    after = ask.getCapability();
     assertEquals(minMemory, ask.getCapability().getMemory());
-    assertTrue(before == after);
 
     // case memory is a multiple of minMemory
     ask.setCapability(Resources.createResource(2 * minMemory));
-    before = ask.getCapability();
     SchedulerUtils.normalizeRequest(ask, resourceCalculator, null, minResource);
-    after = ask.getCapability();
     assertEquals(2 * minMemory, ask.getCapability().getMemory());
-    assertTrue(before == after);
 
     // case memory is not a multiple of minMemory
     ask.setCapability(Resources.createResource(minMemory + 10));
-    before = ask.getCapability();
     SchedulerUtils.normalizeRequest(ask, resourceCalculator, null, minResource);
-    after = ask.getCapability();
     assertEquals(2 * minMemory, ask.getCapability().getMemory());
-    assertTrue(before == after);
 
   }
   
@@ -86,33 +73,24 @@ public class TestSchedulerUtils {
 
     // case negative memory/vcores
     ask.setCapability(Resources.createResource(-1024, -1));
-    Resource before = ask.getCapability();
     SchedulerUtils.normalizeRequest(
         ask, resourceCalculator, clusterResource, minResource);
-    Resource after = ask.getCapability();
     assertEquals(minResource, ask.getCapability());
-    assertTrue(before == after);
 
     // case zero memory/vcores
     ask.setCapability(Resources.createResource(0, 0));
-    before = ask.getCapability();
     SchedulerUtils.normalizeRequest(
         ask, resourceCalculator, clusterResource, minResource);
-    after = ask.getCapability();
     assertEquals(minResource, ask.getCapability());
     assertEquals(1, ask.getCapability().getVirtualCores());
     assertEquals(1024, ask.getCapability().getMemory());
-    assertTrue(before == after);
 
     // case non-zero memory & zero cores
     ask.setCapability(Resources.createResource(1536, 0));
-    before = ask.getCapability();
     SchedulerUtils.normalizeRequest(
         ask, resourceCalculator, clusterResource, minResource);
-    after = ask.getCapability();
     assertEquals(Resources.createResource(2048, 1), ask.getCapability());
     assertEquals(1, ask.getCapability().getVirtualCores());
     assertEquals(2048, ask.getCapability().getMemory());
-    assertTrue(before == after);
   }
 }

+ 36 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java

@@ -19,8 +19,10 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.util.Comparator;
 import java.util.List;
 
 import junit.framework.Assert;
@@ -43,13 +45,19 @@ import org.apache.hadoop.yarn.server.resourcemanager.Task;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
+import org.mockito.Mockito;
+import static org.mockito.Mockito.*;
+
 
 public class TestCapacityScheduler {
   private static final Log LOG = LogFactory.getLog(TestCapacityScheduler.class);
@@ -452,5 +460,33 @@ public class TestCapacityScheduler {
     }
     return result;
   }
+  
+    
+    @Test (timeout = 5000)
+    public void testApplicationComparator()
+    {
+      CapacityScheduler cs = new CapacityScheduler();
+      Comparator<FiCaSchedulerApp> appComparator= cs.getApplicationComparator();
+      ApplicationId id1 = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(ApplicationId.class);
+      id1.setClusterTimestamp(1);
+      id1.setId(1);
+      ApplicationId id2 = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(ApplicationId.class);
+      id2.setClusterTimestamp(1);
+      id2.setId(2);
+      ApplicationId id3 = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(ApplicationId.class);
+      id3.setClusterTimestamp(2);
+      id3.setId(1);
+      //same clusterId
+      FiCaSchedulerApp app1 = Mockito.mock(FiCaSchedulerApp.class);
+      when(app1.getApplicationId()).thenReturn(id1);
+      FiCaSchedulerApp app2 = Mockito.mock(FiCaSchedulerApp.class);
+      when(app2.getApplicationId()).thenReturn(id2);
+      FiCaSchedulerApp app3 = Mockito.mock(FiCaSchedulerApp.class);
+      when(app3.getApplicationId()).thenReturn(id3);
+      assertTrue(appComparator.compare(app1, app2) < 0);
+      //different clusterId
+      assertTrue(appComparator.compare(app1, app3) < 0);
+      assertTrue(appComparator.compare(app2, app3) < 0);
+    }
 
 }

+ 2 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java

@@ -49,6 +49,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaS
 import org.apache.hadoop.yarn.server.resourcemanager.security.ApplicationTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
+import org.apache.hadoop.yarn.util.BuilderUtils;
 
 public class TestUtils {
   private static final Log LOG = LogFactory.getLog(TestUtils.class);
@@ -136,9 +137,7 @@ public class TestUtils {
   
   public static ApplicationAttemptId 
   getMockApplicationAttemptId(int appId, int attemptId) {
-    ApplicationId applicationId = mock(ApplicationId.class);
-    when(applicationId.getClusterTimestamp()).thenReturn(0L);
-    when(applicationId.getId()).thenReturn(appId);
+    ApplicationId applicationId = BuilderUtils.newApplicationId(0l, appId);
     ApplicationAttemptId applicationAttemptId = mock(ApplicationAttemptId.class);  
     when(applicationAttemptId.getApplicationId()).thenReturn(applicationId);
     when(applicationAttemptId.getAttemptId()).thenReturn(attemptId);

+ 23 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/FairScheduler.apt.vm

@@ -85,8 +85,9 @@ Hadoop MapReduce Next Generation - Fair Scheduler
   cause too much intermediate data to be created or too much context-switching.
   Limiting the apps does not cause any subsequently submitted apps to fail, 
   only to wait in the scheduler's queue until some of the user's earlier apps 
-  finish. apps to run from each user/queue are chosen in order of priority and 
-  then submit time, as in the default FIFO scheduler in Hadoop.
+  finish. Apps to run from each user/queue are chosen in the same fair sharing
+  manner, but can alternatively be configured to be chosen in order of submit
+  time, as in the default FIFO scheduler in Hadoop.
 
   Certain add-ons are not yet supported which existed in the original (MR1) 
   Fair Scheduler. Among them, is the use of a custom policies governing 
@@ -142,7 +143,9 @@ Hadoop MapReduce Next Generation - Fair Scheduler
  * <<<yarn.scheduler.fair.sizebasedweight>>>
   
     * Whether to assign shares to individual apps based on their size, rather than
-      providing an equal share to all apps regardless of size. Defaults to false.
+      providing an equal share to all apps regardless of size. When set to true,
+      apps are weighted by the natural logarithm of one plus the app's total
+      requested memory, divided by the natural logarithm of 2. Defaults to false.
 
  * <<<yarn.scheduler.fair.assignmultiple>>>
 
@@ -180,16 +183,29 @@ Allocation file format
  * <<Queue elements>>, which represent queues. Each may contain the following
      properties:
 
-   * minResources: minimum amount of aggregate memory
+   * minResources: minimum MB of aggregate memory the queue expects. If a queue
+     demands resources, and its current allocation is below its configured minimum,
+     it will be assigned available resources before any queue that is not in this
+     situation.  If multiple queues are in this situation, resources go to the
+     queue with the smallest ratio between allocation and minimum. Note that it is
+     possible that a queue that is below its minimum may not immediately get up to
+     its minimum when it submits an application, because already-running jobs may
+     be using those resources.
 
-   * maxResources: maximum amount of aggregate memory
+   * maxResources: maximum MB of aggregate memory a queue is allowed.  A queue
+     will never be assigned a container that would put it over this limit.
 
    * maxRunningApps: limit the number of apps from the queue to run at once
 
-   * weight: to share the cluster non-proportionally with other queues
+   * weight: to share the cluster non-proportionally with other queues. Weights
+     default to 1, and a queue with weight 2 should receive approximately twice
+     as many resources as a queue with the default weight.
 
    * schedulingMode: either "fifo" or "fair" depending on the in-queue scheduling
-     policy desired
+     policy desired. Defaults to "fair". If "fifo", apps with earlier submit
+     times are given preference for containers, but apps submitted later may
+     run concurrently if there is leftover space on the cluster after satisfying
+     the earlier app's requests.
 
    * aclSubmitApps: a list of users that can submit apps to the queue. A (default)
      value of "*" means that any users can submit apps. A queue inherits the ACL of