Quellcode durchsuchen

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 vor 12 Jahren
Ursprung
Commit
07d7d92179
97 geänderte Dateien mit 5603 neuen und 539 gelöschten Zeilen
  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
   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-8419. Fixed GzipCode NPE reset for IBM JDK. (Yu Li via eyang)
 
 
     HADOOP-9041. FsUrlStreamHandlerFactory could cause an infinite loop in
     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-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
   OPTIMIZATIONS
 
 
     HADOOP-9150. Avoid unnecessary DNS resolution attempts for logical URIs
     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
     HADOOP-9125. LdapGroupsMapping threw CommunicationException after some
     idle time. (Kai Zheng via atm)
     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
 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.
    * ("/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
    * 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 
    * Hence this method is not called makeAbsolute() and 
    * has been deliberately declared private.
    * has been deliberately declared private.
    */
    */
   private Path fixRelativePart(Path p) {
   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()) {
     if (p.isUriPathAbsolute()) {
       return p;
       return p;
     } else {
     } 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() && 
     return  (isUriPathAbsolute() && 
         uri.getScheme() == null && uri.getAuthority() == null);
         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.
    *  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.HashMap;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 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.conf.Configuration;
 import org.apache.hadoop.util.ReflectionUtils;
 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 
  * A global compressor/decompressor pool used to save and reuse 
  * (possibly native) compression/decompression codecs.
  * (possibly native) compression/decompression codecs.
@@ -52,6 +57,29 @@ public class CodecPool {
   private static final Map<Class<Decompressor>, List<Decompressor>> decompressorPool = 
   private static final Map<Class<Decompressor>, List<Decompressor>> decompressorPool = 
     new HashMap<Class<Decompressor>, List<Decompressor>>();
     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,
   private static <T> T borrow(Map<Class<T>, List<T>> pool,
                              Class<? extends T> codecClass) {
                              Class<? extends T> codecClass) {
     T codec = null;
     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 
    * Get a {@link Compressor} for the given {@link CompressionCodec} from the 
    * pool or a new one.
    * pool or a new one.
@@ -111,6 +154,7 @@ public class CodecPool {
         LOG.debug("Got recycled compressor");
         LOG.debug("Got recycled compressor");
       }
       }
     }
     }
+    updateLeaseCount(compressorCounts, compressor, 1);
     return compressor;
     return compressor;
   }
   }
   
   
@@ -137,6 +181,7 @@ public class CodecPool {
         LOG.debug("Got recycled decompressor");
         LOG.debug("Got recycled decompressor");
       }
       }
     }
     }
+    updateLeaseCount(decompressorCounts, decompressor, 1);
     return decompressor;
     return decompressor;
   }
   }
   
   
@@ -155,6 +200,7 @@ public class CodecPool {
     }
     }
     compressor.reset();
     compressor.reset();
     payback(compressorPool, compressor);
     payback(compressorPool, compressor);
+    updateLeaseCount(compressorCounts, compressor, -1);
   }
   }
   
   
   /**
   /**
@@ -173,5 +219,24 @@ public class CodecPool {
     }
     }
     decompressor.reset();
     decompressor.reset();
     payback(decompressorPool, decompressor);
     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());
       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 " 
         throw new IllegalArgumentException("Unexpected data node " 
             + node.toString() 
             + node.toString() 
             + " at an illegal network location");
             + " 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>>> | | |
 | <<<yarn.resourcemanager.scheduler.class>>> | | |
 | | <<<ResourceManager>>> Scheduler class. | |
 | | <<<ResourceManager>>> Scheduler class. | |
-| | | <<<CapacityScheduler>>> (recommended) or <<<FifoScheduler>>> |
+| | | <<<CapacityScheduler>>> (recommended), <<<FairScheduler>>> (also recommended), or <<<FifoScheduler>>> |
 *-------------------------+-------------------------+------------------------+
 *-------------------------+-------------------------+------------------------+
 | <<<yarn.scheduler.minimum-allocation-mb>>> | | |
 | <<<yarn.scheduler.minimum-allocation-mb>>> | | |
 | | Minimum limit of memory to allocate to each container request at the <<<Resource Manager>>>. | |
 | | 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 CONFIG = new File("./test-config.xml").getAbsolutePath();
   final static String CONFIG2 = new File("./test-config2.xml").getAbsolutePath();
   final static String CONFIG2 = new File("./test-config2.xml").getAbsolutePath();
   final static Random RAN = new Random();
   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
   @Override
   protected void setUp() throws Exception {
   protected void setUp() throws Exception {
@@ -327,8 +331,8 @@ public class TestConfiguration extends TestCase {
     ByteArrayOutputStream baos = new ByteArrayOutputStream(); 
     ByteArrayOutputStream baos = new ByteArrayOutputStream(); 
     conf.writeXml(baos);
     conf.writeXml(baos);
     String result = baos.toString();
     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>"));
     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.File;
 import java.io.FileNotFoundException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
 import java.util.EnumSet;
 import java.util.EnumSet;
 
 
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.HadoopIllegalArgumentException;
@@ -1166,40 +1164,6 @@ public abstract class FileContextMainOperationsBaseTest  {
       Assert.assertEquals(fc.getFileStatus(file), fc.getFileLinkStatus(file));
       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 {
   protected void createFile(Path path) throws IOException {
     FSDataOutputStream out = fc.create(path, EnumSet.of(CREATE),
     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.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map;
 
 
-import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.junit.Test;
 import org.junit.Test;
 
 
 public class TestNetworkTopologyWithNodeGroup {
 public class TestNetworkTopologyWithNodeGroup {
   private final static NetworkTopologyWithNodeGroup cluster = new 
   private final static NetworkTopologyWithNodeGroup cluster = new 
       NetworkTopologyWithNodeGroup();
       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 computeNode = new NodeBase("/d1/r1/s1/h9");
+  
+  private final static NodeBase rackOnlyNode = new NodeBase("h10", "/r2");
 
 
   static {
   static {
     for(int i=0; i<dataNodes.length; i++) {
     for(int i=0; i<dataNodes.length; i++) {
@@ -96,7 +97,7 @@ public class TestNetworkTopologyWithNodeGroup {
 
 
   @Test
   @Test
   public void testPseudoSortByDistance() throws Exception {
   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
     // array contains both local node, local node group & local rack node
     testNodes[0] = dataNodes[1];
     testNodes[0] = dataNodes[1];
@@ -147,7 +148,7 @@ public class TestNetworkTopologyWithNodeGroup {
   private Map<Node, Integer> pickNodesAtRandom(int numNodes,
   private Map<Node, Integer> pickNodesAtRandom(int numNodes,
       String excludedScope) {
       String excludedScope) {
     Map<Node, Integer> frequency = new HashMap<Node, Integer>();
     Map<Node, Integer> frequency = new HashMap<Node, Integer>();
-    for (DatanodeDescriptor dnd : dataNodes) {
+    for (NodeBase dnd : dataNodes) {
       frequency.put(dnd, 0);
       frequency.put(dnd, 0);
     }
     }
 
 
@@ -161,6 +162,12 @@ public class TestNetworkTopologyWithNodeGroup {
   /**
   /**
    * This test checks that chooseRandom works for an excluded node.
    * 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
   @Test
   public void testChooseRandomExcludedNode() {
   public void testChooseRandomExcludedNode() {
     String scope = "~" + NodeBase.getPath(dataNodes[0]);
     String scope = "~" + NodeBase.getPath(dataNodes[0]);
@@ -171,5 +178,23 @@ public class TestNetworkTopologyWithNodeGroup {
       assertTrue(frequency.get(key) > 0 || key == dataNodes[0]);
       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
   IMPROVEMENTS
 
 
+    HDFS-4665. Move TestNetworkTopologyWithNodeGroup to common.
+    (Junping Du via llu)
+
     HDFS-1620. Rename HdfsConstants -> HdfsServerConstants, FSConstants ->
     HDFS-1620. Rename HdfsConstants -> HdfsServerConstants, FSConstants ->
                HdfsConstants. (Harsh J Chouraria via atm)
                HdfsConstants. (Harsh J Chouraria via atm)
 
 
@@ -343,6 +346,9 @@ Trunk (Unreleased)
 
 
     HDFS-4604. TestJournalNode fails on Windows. (Ivan Mitic via suresh)
     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
 Release 2.0.5-beta - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -460,6 +466,14 @@ Release 2.0.5-beta - UNRELEASED
     HDFS-4598. Fix the default value of ConcatSourcesParam and the WebHDFS doc.
     HDFS-4598. Fix the default value of ConcatSourcesParam and the WebHDFS doc.
     (szetszwo)
     (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
 Release 2.0.4-alpha - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -2477,6 +2491,10 @@ Release 0.23.7 - UNRELEASED
     HDFS-4581. checkDiskError should not be called on network errors (Rohit
     HDFS-4581. checkDiskError should not be called on network errors (Rohit
     Kochar via kihwal)
     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
 Release 0.23.6 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   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();
       node.receivedBlockReport();
       if (staleBefore && !node.areBlockContentsStale()) {
       if (staleBefore && !node.areBlockContentsStale()) {
         LOG.info("BLOCK* processReport: Received first block report from "
         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();
         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_TRANSFER:
     case DatanodeProtocol.DNA_INVALIDATE:
     case DatanodeProtocol.DNA_INVALIDATE:
     case DatanodeProtocol.DNA_SHUTDOWN:
     case DatanodeProtocol.DNA_SHUTDOWN:
+    case DatanodeProtocol.DNA_FINALIZE:
     case DatanodeProtocol.DNA_RECOVERBLOCK:
     case DatanodeProtocol.DNA_RECOVERBLOCK:
     case DatanodeProtocol.DNA_BALANCERBANDWIDTHUPDATE:
     case DatanodeProtocol.DNA_BALANCERBANDWIDTHUPDATE:
       LOG.warn("Got a command from standby NN - ignoring command:" + cmd.getAction());
       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
         // Every so often, send heartbeat or block-report
         //
         //
-        if (startTime - lastHeartbeat > dnConf.heartBeatInterval) {
+        if (startTime - lastHeartbeat >= dnConf.heartBeatInterval) {
           //
           //
           // All heartbeat messages include following info:
           // All heartbeat messages include following info:
           // -- Datanode name
           // -- 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, 
   private static StreamingOutput getListingStream(final NamenodeProtocols np, 
       final String p) throws IOException {
       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);
         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() {
     return new StreamingOutput() {
       @Override
       @Override
       public void write(final OutputStream outstream) throws IOException {
       public void write(final OutputStream outstream) throws IOException {
@@ -729,21 +735,32 @@ public class NamenodeWebHdfsMethods {
         out.println("{\"" + FileStatus.class.getSimpleName() + "es\":{\""
         out.println("{\"" + FileStatus.class.getSimpleName() + "es\":{\""
             + FileStatus.class.getSimpleName() + "\":[");
             + 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();
         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 URI uri;
   private boolean hasInitedToken;
   private boolean hasInitedToken;
   private Token<?> delegationToken;
   private Token<?> delegationToken;
-  private final AuthenticatedURL.Token authToken = new AuthenticatedURL.Token();
   private RetryPolicy retryPolicy = null;
   private RetryPolicy retryPolicy = null;
   private Path workingDir;
   private Path workingDir;
 
 
@@ -481,6 +480,8 @@ public class WebHdfsFileSystem extends FileSystem
       try {
       try {
         if (op.getRequireAuth()) {
         if (op.getRequireAuth()) {
           LOG.debug("open AuthenticatedURL connection");
           LOG.debug("open AuthenticatedURL connection");
+          UserGroupInformation.getCurrentUser().checkTGTAndReloginFromKeytab();
+          final AuthenticatedURL.Token authToken = new AuthenticatedURL.Token();
           conn = new AuthenticatedURL(AUTH).openConnection(url, authToken);
           conn = new AuthenticatedURL(AUTH).openConnection(url, authToken);
         } else {
         } else {
           LOG.debug("open URL connection");
           LOG.debug("open URL connection");
@@ -1006,20 +1007,12 @@ public class WebHdfsFileSystem extends FileSystem
     @Override
     @Override
     public long renew(final Token<?> token, final Configuration conf
     public long renew(final Token<?> token, final Configuration conf
         ) throws IOException, InterruptedException {
         ) 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);
       return getWebHdfs(token, conf).renewDelegationToken(token);
     }
     }
   
   
     @Override
     @Override
     public void cancel(final Token<?> token, final Configuration conf
     public void cancel(final Token<?> token, final Configuration conf
         ) throws IOException, InterruptedException {
         ) 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);
       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)
   @Test (timeout = 30000)
   public void testNewNamenodeTakesOverWriter() throws Exception {
   public void testNewNamenodeTakesOverWriter() throws Exception {
-    // Skip the test on Windows. See HDFS-4584.
-    assumeTrue(!Path.WINDOWS);
-
     File nn1Dir = new File(
     File nn1Dir = new File(
         MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image-nn1");
         MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image-nn1");
     File nn2Dir = new File(
     File nn2Dir = new File(
@@ -110,23 +107,37 @@ public class TestNNWithQJM {
         nn1Dir.getAbsolutePath());
         nn1Dir.getAbsolutePath());
     conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
     conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
         mjc.getQuorumJournalURI("myjournal").toString());
         mjc.getQuorumJournalURI("myjournal").toString());
-    
+
+    // Start the cluster once to generate the dfs dirs
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
       .numDataNodes(0)
       .numDataNodes(0)
       .manageNameDfsDirs(false)
       .manageNameDfsDirs(false)
       .checkExitOnShutdown(false)
       .checkExitOnShutdown(false)
       .build();
       .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 {
     try {
-      cluster.getFileSystem().mkdirs(TEST_PATH);
-      
       // Start a second NN pointed to the same quorum.
       // Start a second NN pointed to the same quorum.
       // We need to copy the image dir from the first NN -- or else
       // We need to copy the image dir from the first NN -- or else
       // the new NN will just be rejected because of Namespace mismatch.
       // the new NN will just be rejected because of Namespace mismatch.
       FileUtil.fullyDelete(nn2Dir);
       FileUtil.fullyDelete(nn2Dir);
       FileUtil.copy(nn1Dir, FileSystem.getLocal(conf).getRaw(),
       FileUtil.copy(nn1Dir, FileSystem.getLocal(conf).getRaw(),
           new Path(nn2Dir.getAbsolutePath()), false, conf);
           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();
       Configuration conf2 = new Configuration();
       conf2.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
       conf2.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
           nn2Dir.getAbsolutePath());
           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;
 package org.apache.hadoop.hdfs.web;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.net.URISyntaxException;
+import java.security.PrivilegedExceptionAction;
 import java.util.Random;
 import java.util.Random;
 
 
 import org.apache.commons.logging.Log;
 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.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 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.MiniDFSCluster;
 import org.apache.hadoop.hdfs.TestDFSClientRetries;
 import org.apache.hadoop.hdfs.TestDFSClientRetries;
 import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
 import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.log4j.Level;
 import org.apache.log4j.Level;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.Test;
@@ -208,4 +214,48 @@ public class TestWebHDFS {
     final Configuration conf = WebHdfsTestUtil.createConf();
     final Configuration conf = WebHdfsTestUtil.createConf();
     TestDFSClientRetries.namenodeRestartTest(conf, true);
     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.
     MAPREDUCE-5006. Fix failing streaming tests due to MAPREDUCE-4994.
     (Sandy Ryza via tomwhite)
     (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
 Release 2.0.4-beta - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -195,6 +215,13 @@ Release 2.0.4-beta - UNRELEASED
     MAPREDUCE-5008. Merger progress miscounts with respect to EOF_MARKER.
     MAPREDUCE-5008. Merger progress miscounts with respect to EOF_MARKER.
     (Sandy Ryza via tomwhite)
     (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 
 Release 2.0.3-alpha - 2013-02-06 
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -744,6 +771,7 @@ Release 0.23.7 - UNRELEASED
     MAPREDUCE-5009. Killing the Task Attempt slated for commit does not clear
     MAPREDUCE-5009. Killing the Task Attempt slated for commit does not clear
     the value from the Task commitAttempt member (Robert Parker via jeagles)
     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
 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 {
     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
       // Stop all services
       // This will also send the final report to the ResourceManager
       // This will also send the final report to the ResourceManager
       LOG.info("Calling stop for all the services");
       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
       // that they don't take too long in shutting down
       if(appMaster.containerAllocator instanceof ContainerAllocatorRouter) {
       if(appMaster.containerAllocator instanceof ContainerAllocatorRouter) {
         ((ContainerAllocatorRouter) appMaster.containerAllocator)
         ((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();
       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,
   protected static void initAndStartAppMaster(final MRAppMaster appMaster,
       final YarnConfiguration conf, String jobUserName) throws IOException,
       final YarnConfiguration conf, String jobUserName) throws IOException,
       InterruptedException {
       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_WAIT,
   KILL_ABORT,
   KILL_ABORT,
   KILLED,
   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,
   JOB_TASK_ATTEMPT_FETCH_FAILURE,
   
   
   //Producer:RMContainerAllocator
   //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();
       DIAGNOSTIC_UPDATE_TRANSITION = new DiagnosticsUpdateTransition();
   private static final InternalErrorTransition
   private static final InternalErrorTransition
       INTERNAL_ERROR_TRANSITION = new InternalErrorTransition();
       INTERNAL_ERROR_TRANSITION = new InternalErrorTransition();
+  private static final InternalRebootTransition
+      INTERNAL_REBOOT_TRANSITION = new InternalRebootTransition();
   private static final TaskAttemptCompletedEventTransition
   private static final TaskAttemptCompletedEventTransition
       TASK_ATTEMPT_COMPLETED_EVENT_TRANSITION =
       TASK_ATTEMPT_COMPLETED_EVENT_TRANSITION =
           new TaskAttemptCompletedEventTransition();
           new TaskAttemptCompletedEventTransition();
@@ -246,6 +248,9 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
           .addTransition(JobStateInternal.NEW, JobStateInternal.ERROR,
           .addTransition(JobStateInternal.NEW, JobStateInternal.ERROR,
               JobEventType.INTERNAL_ERROR,
               JobEventType.INTERNAL_ERROR,
               INTERNAL_ERROR_TRANSITION)
               INTERNAL_ERROR_TRANSITION)
+          .addTransition(JobStateInternal.NEW, JobStateInternal.REBOOT,
+              JobEventType.JOB_AM_REBOOT,
+              INTERNAL_REBOOT_TRANSITION)
           // Ignore-able events
           // Ignore-able events
           .addTransition(JobStateInternal.NEW, JobStateInternal.NEW,
           .addTransition(JobStateInternal.NEW, JobStateInternal.NEW,
               JobEventType.JOB_UPDATED_NODES)
               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,
           .addTransition(JobStateInternal.INITED, JobStateInternal.ERROR,
               JobEventType.INTERNAL_ERROR,
               JobEventType.INTERNAL_ERROR,
               INTERNAL_ERROR_TRANSITION)
               INTERNAL_ERROR_TRANSITION)
+          .addTransition(JobStateInternal.INITED, JobStateInternal.REBOOT,
+              JobEventType.JOB_AM_REBOOT,
+              INTERNAL_REBOOT_TRANSITION)
           // Ignore-able events
           // Ignore-able events
           .addTransition(JobStateInternal.INITED, JobStateInternal.INITED,
           .addTransition(JobStateInternal.INITED, JobStateInternal.INITED,
               JobEventType.JOB_UPDATED_NODES)
               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,
           .addTransition(JobStateInternal.SETUP, JobStateInternal.ERROR,
               JobEventType.INTERNAL_ERROR,
               JobEventType.INTERNAL_ERROR,
               INTERNAL_ERROR_TRANSITION)
               INTERNAL_ERROR_TRANSITION)
+          .addTransition(JobStateInternal.SETUP, JobStateInternal.REBOOT,
+              JobEventType.JOB_AM_REBOOT,
+              INTERNAL_REBOOT_TRANSITION)
           // Ignore-able events
           // Ignore-able events
           .addTransition(JobStateInternal.SETUP, JobStateInternal.SETUP,
           .addTransition(JobStateInternal.SETUP, JobStateInternal.SETUP,
               JobEventType.JOB_UPDATED_NODES)
               JobEventType.JOB_UPDATED_NODES)
@@ -327,6 +338,9 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
               JobStateInternal.RUNNING,
               JobStateInternal.RUNNING,
               JobStateInternal.ERROR, JobEventType.INTERNAL_ERROR,
               JobStateInternal.ERROR, JobEventType.INTERNAL_ERROR,
               INTERNAL_ERROR_TRANSITION)
               INTERNAL_ERROR_TRANSITION)
+          .addTransition(JobStateInternal.RUNNING, JobStateInternal.REBOOT,
+              JobEventType.JOB_AM_REBOOT,
+              INTERNAL_REBOOT_TRANSITION)
 
 
           // Transitions from KILL_WAIT state.
           // Transitions from KILL_WAIT state.
           .addTransition
           .addTransition
@@ -352,7 +366,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
               EnumSet.of(JobEventType.JOB_KILL,
               EnumSet.of(JobEventType.JOB_KILL,
                   JobEventType.JOB_UPDATED_NODES,
                   JobEventType.JOB_UPDATED_NODES,
                   JobEventType.JOB_MAP_TASK_RESCHEDULED,
                   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
           // Transitions from COMMITTING state
           .addTransition(JobStateInternal.COMMITTING,
           .addTransition(JobStateInternal.COMMITTING,
@@ -377,7 +392,10 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
           .addTransition(JobStateInternal.COMMITTING,
           .addTransition(JobStateInternal.COMMITTING,
               JobStateInternal.ERROR, JobEventType.INTERNAL_ERROR,
               JobStateInternal.ERROR, JobEventType.INTERNAL_ERROR,
               INTERNAL_ERROR_TRANSITION)
               INTERNAL_ERROR_TRANSITION)
-          // Ignore-able events
+          .addTransition(JobStateInternal.COMMITTING, JobStateInternal.REBOOT,
+              JobEventType.JOB_AM_REBOOT,
+              INTERNAL_REBOOT_TRANSITION)
+              // Ignore-able events
           .addTransition(JobStateInternal.COMMITTING,
           .addTransition(JobStateInternal.COMMITTING,
               JobStateInternal.COMMITTING,
               JobStateInternal.COMMITTING,
               EnumSet.of(JobEventType.JOB_UPDATED_NODES,
               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,
           .addTransition(JobStateInternal.SUCCEEDED, JobStateInternal.SUCCEEDED,
               EnumSet.of(JobEventType.JOB_KILL, 
               EnumSet.of(JobEventType.JOB_KILL, 
                   JobEventType.JOB_UPDATED_NODES,
                   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
           // Transitions from FAIL_ABORT state
           .addTransition(JobStateInternal.FAIL_ABORT,
           .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_MAP_TASK_RESCHEDULED,
                   JobEventType.JOB_TASK_ATTEMPT_FETCH_FAILURE,
                   JobEventType.JOB_TASK_ATTEMPT_FETCH_FAILURE,
                   JobEventType.JOB_COMMIT_COMPLETED,
                   JobEventType.JOB_COMMIT_COMPLETED,
-                  JobEventType.JOB_COMMIT_FAILED))
+                  JobEventType.JOB_COMMIT_FAILED,
+                  JobEventType.JOB_AM_REBOOT))
 
 
           // Transitions from KILL_ABORT state
           // Transitions from KILL_ABORT state
           .addTransition(JobStateInternal.KILL_ABORT,
           .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_COMPLETED,
                   JobEventType.JOB_SETUP_FAILED,
                   JobEventType.JOB_SETUP_FAILED,
                   JobEventType.JOB_COMMIT_COMPLETED,
                   JobEventType.JOB_COMMIT_COMPLETED,
-                  JobEventType.JOB_COMMIT_FAILED))
+                  JobEventType.JOB_COMMIT_FAILED,
+                  JobEventType.JOB_AM_REBOOT))
 
 
           // Transitions from FAILED state
           // Transitions from FAILED state
           .addTransition(JobStateInternal.FAILED, JobStateInternal.FAILED,
           .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_SETUP_FAILED,
                   JobEventType.JOB_COMMIT_COMPLETED,
                   JobEventType.JOB_COMMIT_COMPLETED,
                   JobEventType.JOB_COMMIT_FAILED,
                   JobEventType.JOB_COMMIT_FAILED,
-                  JobEventType.JOB_ABORT_COMPLETED))
+                  JobEventType.JOB_ABORT_COMPLETED,
+                  JobEventType.JOB_AM_REBOOT))
 
 
           // Transitions from KILLED state
           // Transitions from KILLED state
           .addTransition(JobStateInternal.KILLED, JobStateInternal.KILLED,
           .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_SETUP_FAILED,
                   JobEventType.JOB_COMMIT_COMPLETED,
                   JobEventType.JOB_COMMIT_COMPLETED,
                   JobEventType.JOB_COMMIT_FAILED,
                   JobEventType.JOB_COMMIT_FAILED,
-                  JobEventType.JOB_ABORT_COMPLETED))
+                  JobEventType.JOB_ABORT_COMPLETED,
+                  JobEventType.JOB_AM_REBOOT))
 
 
           // No transitions from INTERNAL_ERROR state. Ignore all.
           // No transitions from INTERNAL_ERROR state. Ignore all.
           .addTransition(
           .addTransition(
@@ -517,9 +540,33 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
                   JobEventType.JOB_COMMIT_COMPLETED,
                   JobEventType.JOB_COMMIT_COMPLETED,
                   JobEventType.JOB_COMMIT_FAILED,
                   JobEventType.JOB_COMMIT_FAILED,
                   JobEventType.JOB_ABORT_COMPLETED,
                   JobEventType.JOB_ABORT_COMPLETED,
-                  JobEventType.INTERNAL_ERROR))
+                  JobEventType.INTERNAL_ERROR,
+                  JobEventType.JOB_AM_REBOOT))
           .addTransition(JobStateInternal.ERROR, JobStateInternal.ERROR,
           .addTransition(JobStateInternal.ERROR, JobStateInternal.ERROR,
               JobEventType.JOB_COUNTER_UPDATE, COUNTER_UPDATE_TRANSITION)
               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
           // create the topology tables
           .installTopology();
           .installTopology();
  
  
@@ -904,6 +951,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
       return JobState.RUNNING;
       return JobState.RUNNING;
     case FAIL_ABORT:
     case FAIL_ABORT:
       return JobState.FAILED;
       return JobState.FAILED;
+    case REBOOT:
+      return JobState.ERROR;
     default:
     default:
       return JobState.valueOf(smState.name());
       return JobState.valueOf(smState.name());
     }
     }
@@ -972,6 +1021,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
       case KILLED:
       case KILLED:
         metrics.killedJob(this);
         metrics.killedJob(this);
         break;
         break;
+      case REBOOT:
       case ERROR:
       case ERROR:
       case FAILED:
       case FAILED:
         metrics.failedJob(this);
         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> {
       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
     @Override
     public void transition(JobImpl job, JobEvent event) {
     public void transition(JobImpl job, JobEvent event) {
       //TODO Is this JH event required.
       //TODO Is this JH event required.
@@ -1907,9 +1966,21 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
       JobUnsuccessfulCompletionEvent failedEvent =
       JobUnsuccessfulCompletionEvent failedEvent =
           new JobUnsuccessfulCompletionEvent(job.oldJobId,
           new JobUnsuccessfulCompletionEvent(job.oldJobId,
               job.finishTime, 0, 0,
               job.finishTime, 0, 0,
-              JobStateInternal.ERROR.toString());
+              jobHistoryString);
       job.eventHandler.handle(new JobHistoryEvent(job.jobId, failedEvent));
       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 can happen if the RM has been restarted. If it is in that state,
       // this application must clean itself up.
       // this application must clean itself up.
       eventHandler.handle(new JobEvent(this.getJob().getID(),
       eventHandler.handle(new JobEvent(this.getJob().getID(),
-                                       JobEventType.INTERNAL_ERROR));
+                                       JobEventType.JOB_AM_REBOOT));
       throw new YarnException("Resource Manager doesn't recognize AttemptId: " +
       throw new YarnException("Resource Manager doesn't recognize AttemptId: " +
                                this.getContext().getApplicationID());
                                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 can happen if the RM has been restarted. If it is in that state,
       // this application must clean itself up.
       // this application must clean itself up.
       eventHandler.handle(new JobEvent(this.getJob().getID(),
       eventHandler.handle(new JobEvent(this.getJob().getID(),
-                                       JobEventType.INTERNAL_ERROR));
+                                       JobEventType.JOB_AM_REBOOT));
       throw new YarnException("Resource Manager doesn't recognize AttemptId: " +
       throw new YarnException("Resource Manager doesn't recognize AttemptId: " +
                                this.getContext().getApplicationID());
                                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.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 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.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.JobState;
 import org.apache.hadoop.mapreduce.v2.api.records.JobState;
 import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
 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.ContainerAllocatorEvent;
 import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler;
 import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 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.security.UserGroupInformation;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -86,9 +89,68 @@ import org.junit.Test;
      attemptId.setApplicationId(appId);
      attemptId.setApplicationId(appId);
      JobId jobid = recordFactory.newRecordInstance(JobId.class);
      JobId jobid = recordFactory.newRecordInstance(JobId.class);
      jobid.setAppId(appId);
      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.init(conf);
+     appMaster.start();
+     //shutdown the job, is lastRetry
      appMaster.shutDownJob();
      appMaster.shutDownJob();
+     //test whether notifyIsLastAMRetry called
+     Assert.assertEquals(true, ((TestMRApp)appMaster).getTestIsLastAMRetry());
      verify(fs).delete(stagingJobPath, true);
      verify(fs).delete(stagingJobPath, true);
    }
    }
    
    
@@ -151,6 +213,8 @@ import org.junit.Test;
 
 
    private class TestMRApp extends MRAppMaster {
    private class TestMRApp extends MRAppMaster {
      ContainerAllocator allocator;
      ContainerAllocator allocator;
+     boolean testIsLastAMRetry = false;
+     JobStateInternal jobStateInternal;
 
 
      public TestMRApp(ApplicationAttemptId applicationAttemptId, 
      public TestMRApp(ApplicationAttemptId applicationAttemptId, 
          ContainerAllocator allocator, int maxAppAttempts) {
          ContainerAllocator allocator, int maxAppAttempts) {
@@ -160,9 +224,11 @@ import org.junit.Test;
        this.allocator = allocator;
        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
      @Override
@@ -179,6 +245,31 @@ import org.junit.Test;
        return allocator;
        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
      @Override
      public RMHeartbeatHandler getRMHeartbeatHandler() {
      public RMHeartbeatHandler getRMHeartbeatHandler() {
        return getStubbedHeartbeatHandler(getContext());
        return getStubbedHeartbeatHandler(getContext());
@@ -197,6 +288,9 @@ import org.junit.Test;
      protected void downloadTokensAndSetupUGI(Configuration conf) {
      protected void downloadTokensAndSetupUGI(Configuration conf) {
      }
      }
 
 
+     public boolean getTestIsLastAMRetry(){
+       return testIsLastAMRetry;
+     }
    }
    }
 
 
   private final class MRAppTestCleanup extends MRApp {
   private final class MRAppTestCleanup extends MRApp {
@@ -288,7 +382,7 @@ import org.junit.Test;
     };
     };
   }
   }
 
 
-  @Test
+  @Test(timeout=20000)
   public void testStagingCleanupOrder() throws Exception {
   public void testStagingCleanupOrder() throws Exception {
     MRAppTestCleanup app = new MRAppTestCleanup(1, 1, true,
     MRAppTestCleanup app = new MRAppTestCleanup(1, 1, true,
         this.getClass().getName(), 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();
     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)
   @Test(timeout=20000)
   public void testKilledDuringSetup() throws Exception {
   public void testKilledDuringSetup() throws Exception {
     Configuration conf = new Configuration();
     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;
 package org.apache.hadoop.mapreduce.v2.api.impl.pb.client;
 
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 
 
@@ -101,7 +102,8 @@ import org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl;
 
 
 import com.google.protobuf.ServiceException;
 import com.google.protobuf.ServiceException;
 
 
-public class MRClientProtocolPBClientImpl implements MRClientProtocol {
+public class MRClientProtocolPBClientImpl implements MRClientProtocol,
+    Closeable {
 
 
   protected MRClientProtocolPB proxy;
   protected MRClientProtocolPB proxy;
   
   
@@ -117,6 +119,13 @@ public class MRClientProtocolPBClientImpl implements MRClientProtocol {
     return RPC.getServerAddress(proxy);
     return RPC.getServerAddress(proxy);
   }
   }
 
 
+  @Override
+  public void close() {
+    if (this.proxy != null) {
+      RPC.stopProxy(this.proxy);
+    }
+  }
+
   @Override
   @Override
   public GetJobReportResponse getJobReport(GetJobReportRequest request)
   public GetJobReportResponse getJobReport(GetJobReportRequest request)
       throws YarnRemoteException {
       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 class JobClient extends CLI {
   public static enum TaskStatusFilter { NONE, KILLED, FAILED, SUCCEEDED, ALL }
   public static enum TaskStatusFilter { NONE, KILLED, FAILED, SUCCEEDED, ALL }
   private TaskStatusFilter taskOutputFilter = TaskStatusFilter.FAILED; 
   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{
   static{
     ConfigUtil.loadResources();
     ConfigUtil.loadResources();
@@ -569,10 +560,6 @@ public class JobClient extends CLI {
     try {
     try {
       conf.setBooleanIfUnset("mapred.mapper.new-api", false);
       conf.setBooleanIfUnset("mapred.mapper.new-api", false);
       conf.setBooleanIfUnset("mapred.reducer.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> () {
       Job job = clientUgi.doAs(new PrivilegedExceptionAction<Job> () {
         @Override
         @Override
         public Job run() throws IOException, ClassNotFoundException, 
         public Job run() throws IOException, ClassNotFoundException, 
@@ -1173,7 +1160,6 @@ public class JobClient extends CLI {
    */
    */
   public Token<DelegationTokenIdentifier> 
   public Token<DelegationTokenIdentifier> 
     getDelegationToken(final Text renewer) throws IOException, InterruptedException {
     getDelegationToken(final Text renewer) throws IOException, InterruptedException {
-    getDelegationTokenCalled = true;
     return clientUgi.doAs(new 
     return clientUgi.doAs(new 
         PrivilegedExceptionAction<Token<DelegationTokenIdentifier>>() {
         PrivilegedExceptionAction<Token<DelegationTokenIdentifier>>() {
       public Token<DelegationTokenIdentifier> run() throws IOException, 
       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;
 package org.apache.hadoop.mapred;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.net.InetSocketAddress;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
@@ -87,6 +88,10 @@ public class ResourceMgrDelegate extends YarnClientImpl {
     return oldMetrics;
     return oldMetrics;
   }
   }
 
 
+  InetSocketAddress getConnectAddress() {
+    return rmAddress;
+  }
+  
   @SuppressWarnings("rawtypes")
   @SuppressWarnings("rawtypes")
   public Token getDelegationToken(Text renewer) throws IOException,
   public Token getDelegationToken(Text renewer) throws IOException,
       InterruptedException {
       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.jobhistory.JobHistoryUtils;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.token.Token;
 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.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 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.BuilderUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.ProtoUtils;
 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.
  * This class enables the current JobClient (0.22 hadoop) to run on YARN.
  */
  */
-@SuppressWarnings({ "rawtypes", "unchecked" })
+@SuppressWarnings("unchecked")
 public class YARNRunner implements ClientProtocol {
 public class YARNRunner implements ClientProtocol {
 
 
   private static final Log LOG = LogFactory.getLog(YARNRunner.class);
   private static final Log LOG = LogFactory.getLog(YARNRunner.class);
@@ -101,14 +103,6 @@ public class YARNRunner implements ClientProtocol {
   private Configuration conf;
   private Configuration conf;
   private final FileContext defaultFileContext;
   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 runner incapsulates the client interface of
    * yarn
    * yarn
@@ -185,6 +179,28 @@ public class YARNRunner implements ClientProtocol {
     return resMgrDelegate.getClusterMetrics();
     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
   @VisibleForTesting
   Token<?> getDelegationTokenFromHS(MRClientProtocol hsProxy)
   Token<?> getDelegationTokenFromHS(MRClientProtocol hsProxy)
       throws IOException, InterruptedException {
       throws IOException, InterruptedException {
@@ -263,18 +279,8 @@ public class YARNRunner implements ClientProtocol {
   public JobStatus submitJob(JobID jobId, String jobSubmitDir, Credentials ts)
   public JobStatus submitJob(JobID jobId, String jobSubmitDir, Credentials ts)
   throws IOException, InterruptedException {
   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
     // Upload only in security mode: TODO
     Path applicationTokensFile =
     Path applicationTokensFile =
         new Path(jobSubmitDir, MRJobConfig.APPLICATION_TOKENS_FILE);
         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.Matchers.any;
 import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.when;
 
 
@@ -30,6 +32,7 @@ import java.io.File;
 import java.io.FileOutputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.io.OutputStream;
+import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.nio.ByteBuffer;
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedExceptionAction;
 import java.util.List;
 import java.util.List;
@@ -39,28 +42,24 @@ import junit.framework.TestCase;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Path;
 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.JobID;
 import org.apache.hadoop.mapreduce.JobPriority;
 import org.apache.hadoop.mapreduce.JobPriority;
 import org.apache.hadoop.mapreduce.JobStatus.State;
 import org.apache.hadoop.mapreduce.JobStatus.State;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
 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.GetDelegationTokenRequest;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetDelegationTokenResponse;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetDelegationTokenResponse;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 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.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.GetAllApplicationsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetAllApplicationsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
 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.GetClusterMetricsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesResponse;
 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.GetQueueInfoRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse;
 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.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.DelegationToken;
 import org.apache.hadoop.yarn.api.records.DelegationToken;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 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.YarnApplicationState;
+import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 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.Appender;
 import org.apache.log4j.Layout;
 import org.apache.log4j.Layout;
 import org.apache.log4j.Logger;
 import org.apache.log4j.Logger;
@@ -146,7 +151,7 @@ public class TestYARNRunner extends TestCase {
    }
    }
 
 
 
 
-  @Test
+  @Test(timeout=20000)
   public void testJobKill() throws Exception {
   public void testJobKill() throws Exception {
     clientDelegate = mock(ClientServiceDelegate.class);
     clientDelegate = mock(ClientServiceDelegate.class);
     when(clientDelegate.getJobStatus(any(JobID.class))).thenReturn(new
     when(clientDelegate.getJobStatus(any(JobID.class))).thenReturn(new
@@ -171,7 +176,7 @@ public class TestYARNRunner extends TestCase {
     verify(clientDelegate).killJob(jobId);
     verify(clientDelegate).killJob(jobId);
   }
   }
 
 
-  @Test
+  @Test(timeout=20000)
   public void testJobSubmissionFailure() throws Exception {
   public void testJobSubmissionFailure() throws Exception {
     when(resourceMgrDelegate.submitApplication(any(ApplicationSubmissionContext.class))).
     when(resourceMgrDelegate.submitApplication(any(ApplicationSubmissionContext.class))).
     thenReturn(appId);
     thenReturn(appId);
@@ -193,7 +198,7 @@ public class TestYARNRunner extends TestCase {
     }
     }
   }
   }
 
 
-  @Test
+  @Test(timeout=20000)
   public void testResourceMgrDelegate() throws Exception {
   public void testResourceMgrDelegate() throws Exception {
     /* we not want a mock of resource mgr delegate */
     /* we not want a mock of resource mgr delegate */
     final ClientRMProtocol clientRMProtocol = mock(ClientRMProtocol.class);
     final ClientRMProtocol clientRMProtocol = mock(ClientRMProtocol.class);
@@ -259,8 +264,88 @@ public class TestYARNRunner extends TestCase {
     delegate.getQueueAclsForCurrentUser();
     delegate.getQueueAclsForCurrentUser();
     verify(clientRMProtocol).getQueueUserAcls(any(GetQueueUserAclsInfoRequest.class));
     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 {
   public void testHistoryServerToken() throws Exception {
     //Set the master principal in the config
     //Set the master principal in the config
     conf.set(YarnConfiguration.RM_PRINCIPAL,"foo@LOCAL");
     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 {
   public void testAMAdminCommandOpts() throws Exception {
     JobConf jobConf = new JobConf();
     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);
       assertTrue("AM admin command opts is after user command opts.", adminIndex < userIndex);
     }
     }
   }
   }
-  @Test
+  @Test(timeout=20000)
   public void testWarnCommandOpts() throws Exception {
   public void testWarnCommandOpts() throws Exception {
     Logger logger = Logger.getLogger(YARNRunner.class);
     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>
       <type>test-jar</type>
       <scope>test</scope>
       <scope>test</scope>
     </dependency>
     </dependency>
+     <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
   </dependencies>
 
 
   <build>
   <build>
     <plugins>
     <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>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-antrun-plugin</artifactId>
         <artifactId>maven-antrun-plugin</artifactId>
@@ -115,6 +130,15 @@
           </execution>
           </execution>
         </executions>
         </executions>
       </plugin>
       </plugin>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <configuration>
+        <excludes>
+        <exclude>src/test/resources/data/*</exclude>
+        </excludes>
+        </configuration>
+      </plugin>
       <plugin>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-jar-plugin</artifactId>
         <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.GenerateData.DataStatistics;
 import org.apache.hadoop.mapred.gridmix.Statistics.JobStats;
 import org.apache.hadoop.mapred.gridmix.Statistics.JobStats;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.Tool;
@@ -693,7 +695,7 @@ public class Gridmix extends Configured implements Tool {
     try {
     try {
       res = ToolRunner.run(new Configuration(), new Gridmix(argv), argv);
       res = ToolRunner.run(new Configuration(), new Gridmix(argv), argv);
     } finally {
     } finally {
-      System.exit(res);
+      ExitUtil.terminate(res);
     }
     }
   }
   }
 
 
@@ -800,6 +802,10 @@ public class Gridmix extends Configured implements Tool {
      */
      */
     void abort();
     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.
    * Add a running job's status to the polling queue.
    */
    */
   public void add(JobStats job) throws InterruptedException {
   public void add(JobStats job) throws InterruptedException {
-    synchronized (runningJobs) {
       runningJobs.put(job);
       runningJobs.put(job);
-    }
   }
   }
 
 
   /**
   /**
@@ -147,12 +145,10 @@ class JobMonitor implements Gridmix.Component<JobStats> {
       boolean shutdown;
       boolean shutdown;
       while (true) {
       while (true) {
         try {
         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
           // 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
           // submitted jobs not in the monitored set
           if (shutdown) {
           if (shutdown) {
             if (!graceful) {
             if (!graceful) {
-              synchronized (runningJobs) {
-                while (!runningJobs.isEmpty()) {
-                  synchronized (mJobs) {
-                    runningJobs.drainTo(mJobs);
-                  }
+              while (!runningJobs.isEmpty()) {
+                synchronized (mJobs) {
+                  runningJobs.drainTo(mJobs);
                 }
                 }
               }
               }
               break;
               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 ");
     LOG.info(" Starting Serial submission ");
     this.rThread.start();
     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 {
       String[] hosts) throws IOException {
     super(conf, submissionMillis, jobdesc, outRoot, ugi, seq);
     super(conf, submissionMillis, jobdesc, outRoot, ugi, seq);
     this.fakeLocations = numLocations;
     this.fakeLocations = numLocations;
-    this.hosts = hosts;
+    this.hosts = hosts.clone();
     this.selector = (fakeLocations > 0)? new Selector(hosts.length, (float) fakeLocations
     this.selector = (fakeLocations > 0)? new Selector(hosts.length, (float) fakeLocations
         / hosts.length, rand.get()) : null;
         / hosts.length, rand.get()) : null;
     this.mapTasksOnly = conf.getBoolean(SLEEPJOB_MAPTASK_ONLY, false);
     this.mapTasksOnly = conf.getBoolean(SLEEPJOB_MAPTASK_ONLY, false);
@@ -289,9 +289,9 @@ public class SleepJob extends GridmixJob {
       this.id = id;
       this.id = id;
       this.sleepDuration = sleepDuration;
       this.sleepDuration = sleepDuration;
       nSpec = reduceDurations.length;
       nSpec = reduceDurations.length;
-      this.reduceDurations = reduceDurations;
+      this.reduceDurations = reduceDurations.clone();
       this.nMaps = nMaps;
       this.nMaps = nMaps;
-      this.locations = locations;
+      this.locations = locations.clone();
     }
     }
 
 
     @Override
     @Override
@@ -349,7 +349,7 @@ public class SleepJob extends GridmixJob {
 
 
     @Override
     @Override
     public String[] getLocations() {
     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;
   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
    * Defines the core heap usage emulation algorithm. This engine is expected
@@ -129,7 +129,8 @@ implements ResourceUsageEmulatorPlugin {
   public static class DefaultHeapUsageEmulator 
   public static class DefaultHeapUsageEmulator 
   implements HeapUsageEmulatorCore {
   implements HeapUsageEmulatorCore {
     // store the unit loads in a list
     // 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.
      * 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.
  * Component generating random job traces for testing on a single node.
  */
  */
-class DebugJobFactory {
+public class DebugJobFactory {
 
 
   interface Debuggable {
   interface Debuggable {
     ArrayList<JobStory> getSubmitted();
     ArrayList<JobStory> getSubmitted();
   }
   }
 
 
-  public static JobFactory getFactory(
+  public static JobFactory<?> getFactory(
     JobSubmitter submitter, Path scratch, int numJobs, Configuration conf,
     JobSubmitter submitter, Path scratch, int numJobs, Configuration conf,
     CountDownLatch startFlag, UserResolver resolver) throws IOException {
     CountDownLatch startFlag, UserResolver resolver) throws IOException {
     GridmixJobSubmissionPolicy policy = GridmixJobSubmissionPolicy.getPolicy(
     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) {
      if (user == null) {
        user = String.format("foobar%d", id);
        user = String.format("foobar%d", id);
      }
      }
-     GridmixTestUtils.createHomeAndStagingDirectory(user, (JobConf)conf);
+     GridmixTestUtils.createHomeAndStagingDirectory(user, conf);
      return user;
      return user;
    }
    }
 
 
@@ -300,7 +300,7 @@ public class DebugJobProducer implements JobStoryProducer {
 
 
     @Override
     @Override
     public String getQueueName() {
     public String getQueueName() {
-      String qName = "q" + ((id % 2) + 1);
+      String qName = "default";
       return qName;
       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.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 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.mapreduce.server.jobtracker.JTConfig;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.ShellBasedUnixGroupsMapping;
-import org.apache.hadoop.security.Groups;
 
 
 import java.io.IOException;
 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/>
  * <p/>
  * http://www.apache.org/licenses/LICENSE-2.0
  * http://www.apache.org/licenses/LICENSE-2.0
  * <p/>
  * <p/>
  * Unless required by applicable law or agreed to in writing, software
  * 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 {
 public class GridmixTestUtils {
   private static final Log LOG = LogFactory.getLog(GridmixTestUtils.class);
   private static final Log LOG = LogFactory.getLog(GridmixTestUtils.class);
   static final Path DEST = new Path("/gridmix");
   static final Path DEST = new Path("/gridmix");
   static FileSystem dfs = null;
   static FileSystem dfs = null;
   static MiniDFSCluster dfsCluster = 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();
     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();
     dfs = dfsCluster.getFileSystem();
     conf.set(JTConfig.JT_RETIREJOBS, "false");
     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 {
   public static void shutdownCluster() throws IOException {
-    if (mrCluster != null) {
-      mrCluster.shutdown();
+    if (mrvl != null) {
+      mrvl.stop();
     }
     }
     if (dfsCluster != null) {
     if (dfsCluster != null) {
       dfsCluster.shutdown();
       dfsCluster.shutdown();
@@ -61,23 +82,25 @@ public class GridmixTestUtils {
 
 
   /**
   /**
    * Methods to generate the home directory for dummy users.
    * Methods to generate the home directory for dummy users.
-   *
+   * 
    * @param conf
    * @param conf
    */
    */
-  public static void createHomeAndStagingDirectory(String user, JobConf conf) {
+  public static void createHomeAndStagingDirectory(String user,
+      Configuration conf) {
     try {
     try {
       FileSystem fs = dfsCluster.getFileSystem();
       FileSystem fs = dfsCluster.getFileSystem();
       String path = "/user/" + user;
       String path = "/user/" + user;
       Path homeDirectory = new Path(path);
       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);
       LOG.info("Creating Staging root directory : " + stagingArea);
       fs.mkdirs(stagingArea);
       fs.mkdirs(stagingArea);
       fs.setPermission(stagingArea, new FsPermission((short) 0777));
       fs.setPermission(stagingArea, new FsPermission((short) 0777));
@@ -87,7 +110,7 @@ public class GridmixTestUtils {
   }
   }
 
 
   static void changePermission(String user, Path homeDirectory, FileSystem fs)
   static void changePermission(String user, Path homeDirectory, FileSystem fs)
-    throws IOException {
+      throws IOException {
     fs.setOwner(homeDirectory, user, "");
     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}.
    * A fake {@link JobFactory}.
    */
    */
-  @SuppressWarnings("unchecked")
+  @SuppressWarnings("rawtypes")
   private static class FakeJobFactory extends JobFactory {
   private static class FakeJobFactory extends JobFactory {
     /**
     /**
      * A fake {@link JobStoryProducer} for {@link FakeJobFactory}.
      * A fake {@link JobStoryProducer} for {@link FakeJobFactory}.
@@ -167,7 +167,7 @@ public class TestGridmixSummary {
    * Test {@link ExecutionSummarizer}.
    * Test {@link ExecutionSummarizer}.
    */
    */
   @Test
   @Test
-  @SuppressWarnings("unchecked")
+  @SuppressWarnings({ "unchecked", "rawtypes" })
   public void testExecutionSummarizer() throws IOException {
   public void testExecutionSummarizer() throws IOException {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     
     
@@ -360,7 +360,6 @@ public class TestGridmixSummary {
    * Test {@link ClusterSummarizer}.
    * Test {@link ClusterSummarizer}.
    */
    */
   @Test
   @Test
-  @SuppressWarnings("deprecation")
   public void testClusterSummarizer() throws IOException {
   public void testClusterSummarizer() throws IOException {
     ClusterSummarizer cs = new ClusterSummarizer();
     ClusterSummarizer cs = new ClusterSummarizer();
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
@@ -374,13 +373,13 @@ public class TestGridmixSummary {
     assertEquals("JT name mismatch", jt, cs.getJobTrackerInfo());
     assertEquals("JT name mismatch", jt, cs.getJobTrackerInfo());
     assertEquals("NN name mismatch", nn, cs.getNamenodeInfo());
     assertEquals("NN name mismatch", nn, cs.getNamenodeInfo());
     
     
-    ClusterStats cstats = ClusterStats.getClusterStats();
+    ClusterStats cStats = ClusterStats.getClusterStats();
     conf.set(JTConfig.JT_IPC_ADDRESS, "local");
     conf.set(JTConfig.JT_IPC_ADDRESS, "local");
     conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "local");
     conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "local");
     JobClient jc = new JobClient(conf);
     JobClient jc = new JobClient(conf);
-    cstats.setClusterMetric(jc.getClusterStatus());
+    cStats.setClusterMetric(jc.getClusterStatus());
     
     
-    cs.update(cstats);
+    cs.update(cStats);
     
     
     // test
     // test
     assertEquals("Cluster summary test failed!", 1, cs.getMaxMapTasks());
     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));
         IdentifierResolver.TEXT_ID));
     jobConf_.setClass("stream.map.output.reader.class",
     jobConf_.setClass("stream.map.output.reader.class",
       idResolver.getOutputReaderClass(), OutputReader.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
       // if mapper is a command, then map output key/value classes come from the
       // idResolver
       // idResolver
       jobConf_.setMapOutputKeyClass(idResolver.getOutputKeyClass());
       jobConf_.setMapOutputKeyClass(idResolver.getOutputKeyClass());
@@ -891,7 +891,7 @@ public class StreamJob implements Tool {
         IdentifierResolver.TEXT_ID));
         IdentifierResolver.TEXT_ID));
     jobConf_.setClass("stream.reduce.output.reader.class",
     jobConf_.setClass("stream.reduce.output.reader.class",
       idResolver.getOutputReaderClass(), OutputReader.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
       // if reducer is a command, then output key/value classes come from the
       // idResolver
       // idResolver
       jobConf_.setOutputKeyClass(idResolver.getOutputKeyClass());
       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.OutputCollector;
 import org.apache.hadoop.mapred.Reducer;
 import org.apache.hadoop.mapred.Reducer;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
@@ -177,6 +178,17 @@ public class TestStreamingOutputKeyValueTypes extends TestStreaming {
     args.add("0");
     args.add("0");
     super.testCommandLine();
     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
   @Override
   @Test
   @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: 
     // Note the dots translated to underscore: 
     // property names have been escaped in PipeMapRed.safeEnvVarName()
     // property names have been escaped in PipeMapRed.safeEnvVarName()
     expectDefined("mapreduce_cluster_local_dir");
     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_key_class", "org.apache.hadoop.io.Text");
     expect("mapreduce_map_output_value_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
     YARN-359. Fixing commands for container signalling in Windows. (Chris Nauroth
     via vinodkv)
     via vinodkv)
+    
+    YARN-524 TestYarnVersionInfo failing if generated properties doesn't
+    include an SVN URL(stevel)
 
 
 Release 2.0.5-beta - UNRELEASED
 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-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
   NEW FEATURES
 
 
   IMPROVEMENTS
   IMPROVEMENTS
@@ -103,6 +109,20 @@ Release 2.0.5-beta - UNRELEASED
     YARN-450. Define value for * in the scheduling protocol (Zhijie Shen via
     YARN-450. Define value for * in the scheduling protocol (Zhijie Shen via
     bikas)
     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
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES
@@ -134,9 +154,6 @@ Release 2.0.5-beta - UNRELEASED
     YARN-485. TestProcfsProcessTree#testProcessTree() doesn't wait long enough 
     YARN-485. TestProcfsProcessTree#testProcessTree() doesn't wait long enough 
     for the process to die. (kkambatl via tucu)
     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.
     YARN-71. Fix the NodeManager to clean up local-dirs on restart.
     (Xuan Gong via sseth)
     (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
     YARN-515. Node Manager not getting the master key. (Robert Joseph Evans
     via jlowe)
     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
 Release 2.0.4-alpha - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -176,6 +210,9 @@ Release 2.0.4-alpha - UNRELEASED
     YARN-429. capacity-scheduler config missing from yarn-test artifact.
     YARN-429. capacity-scheduler config missing from yarn-test artifact.
     (sseth via hitesh)
     (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 
 Release 2.0.3-alpha - 2013-02-06 
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

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

@@ -256,4 +256,18 @@
     <Bug pattern="IS2_INCONSISTENT_SYNC" />
     <Bug pattern="IS2_INCONSISTENT_SYNC" />
   </Match>
   </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>
 </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
    * only
    */
    */
   public static final String AM_CONTAINER_ID_ENV = "AM_CONTAINER_ID";
   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
    * 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
   @Unstable
   void setPriority(Priority priority);
   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.
    * Get the <code>ContainerToken</code> for the container.
    * @return <code>ContainerToken</code> for the container
    * @return <code>ContainerToken</code> for the container
@@ -147,16 +135,4 @@ public interface Container extends Comparable<Container> {
   @Private
   @Private
   @Unstable
   @Unstable
   void setContainerToken(ContainerToken containerToken);
   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.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 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.ContainerToken;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 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.ContainerIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProtoOrBuilder;
 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.NodeIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
-import org.apache.hadoop.yarn.util.ProtoUtils;
     
     
 public class ContainerPBImpl extends ProtoBase<ContainerProto> implements Container {
 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 Resource resource = null;
   private Priority priority = null;
   private Priority priority = null;
   private ContainerToken containerToken = null;
   private ContainerToken containerToken = null;
-  private ContainerStatus containerStatus = null;
   
   
   public ContainerPBImpl() {
   public ContainerPBImpl() {
     builder = ContainerProto.newBuilder();
     builder = ContainerProto.newBuilder();
@@ -94,11 +88,6 @@ public class ContainerPBImpl extends ProtoBase<ContainerProto> implements Contai
             builder.getContainerToken())) {
             builder.getContainerToken())) {
       builder.setContainerToken(convertToProtoFormat(this.containerToken));
       builder.setContainerToken(convertToProtoFormat(this.containerToken));
     }
     }
-    if (this.containerStatus != null
-        && !((ContainerStatusPBImpl) this.containerStatus).getProto().equals(
-            builder.getContainerStatus())) {
-      builder.setContainerStatus(convertToProtoFormat(this.containerStatus));
-    }
   }
   }
 
 
   private void mergeLocalToProto() {
   private void mergeLocalToProto() {
@@ -115,26 +104,7 @@ public class ContainerPBImpl extends ProtoBase<ContainerProto> implements Contai
     }
     }
     viaProto = false;
     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
   @Override
   public ContainerId getId() {
   public ContainerId getId() {
     ContainerProtoOrBuilder p = viaProto ? proto : builder;
     ContainerProtoOrBuilder p = viaProto ? proto : builder;
@@ -260,35 +230,6 @@ public class ContainerPBImpl extends ProtoBase<ContainerProto> implements Contai
     this.containerToken = containerToken;
     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) {
   private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
     return new ContainerIdPBImpl(p);
     return new ContainerIdPBImpl(p);
   }
   }
@@ -329,14 +270,6 @@ public class ContainerPBImpl extends ProtoBase<ContainerProto> implements Contai
     return ((ContainerTokenPBImpl)t).getProto();
     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() {
   public String toString() {
     StringBuilder sb = new StringBuilder();
     StringBuilder sb = new StringBuilder();
     sb.append("Container: [");
     sb.append("Container: [");
@@ -345,9 +278,7 @@ public class ContainerPBImpl extends ProtoBase<ContainerProto> implements Contai
     sb.append("NodeHttpAddress: ").append(getNodeHttpAddress()).append(", ");
     sb.append("NodeHttpAddress: ").append(getNodeHttpAddress()).append(", ");
     sb.append("Resource: ").append(getResource()).append(", ");
     sb.append("Resource: ").append(getResource()).append(", ");
     sb.append("Priority: ").append(getPriority()).append(", ");
     sb.append("Priority: ").append(getPriority()).append(", ");
-    sb.append("State: ").append(getState()).append(", ");
     sb.append("Token: ").append(getContainerToken()).append(", ");
     sb.append("Token: ").append(getContainerToken()).append(", ");
-    sb.append("Status: ").append(getContainerStatus());
     sb.append("]");
     sb.append("]");
     return sb.toString();
     return sb.toString();
   }
   }
@@ -357,16 +288,7 @@ public class ContainerPBImpl extends ProtoBase<ContainerProto> implements Contai
   public int compareTo(Container other) {
   public int compareTo(Container other) {
     if (this.getId().compareTo(other.getId()) == 0) {
     if (this.getId().compareTo(other.getId()) == 0) {
       if (this.getNodeId().compareTo(other.getNodeId()) == 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 {
       } else {
         return this.getNodeId().compareTo(other.getNodeId());
         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 string node_http_address = 3;
   optional ResourceProto resource = 4;
   optional ResourceProto resource = 4;
   optional PriorityProto priority = 5;
   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 {
 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()
             + ", containerNode=" + allocatedContainer.getNodeId().getHost()
             + ":" + allocatedContainer.getNodeId().getPort()
             + ":" + allocatedContainer.getNodeId().getPort()
             + ", containerNodeURI=" + allocatedContainer.getNodeHttpAddress()
             + ", containerNodeURI=" + allocatedContainer.getNodeHttpAddress()
-            + ", containerState" + allocatedContainer.getState()
             + ", containerResourceMemory"
             + ", containerResourceMemory"
             + allocatedContainer.getResource().getMemory());
             + allocatedContainer.getResource().getMemory());
         // + ", containerToken"
         // + ", 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
  * 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
  * 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
  * 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
  * stderr and waits for the AM process to exit. Then it waits for the RM to
  * report app completion.
  * 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;
 package org.apache.hadoop.yarn.api.impl.pb.client;
 
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 
 
@@ -65,7 +66,7 @@ import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.Refre
 import com.google.protobuf.ServiceException;
 import com.google.protobuf.ServiceException;
 
 
 
 
-public class RMAdminProtocolPBClientImpl implements RMAdminProtocol {
+public class RMAdminProtocolPBClientImpl implements RMAdminProtocol, Closeable {
 
 
   private RMAdminProtocolPB proxy;
   private RMAdminProtocolPB proxy;
   
   
@@ -77,6 +78,13 @@ public class RMAdminProtocolPBClientImpl implements RMAdminProtocol {
         RMAdminProtocolPB.class, clientVersion, addr, conf);
         RMAdminProtocolPB.class, clientVersion, addr, conf);
   }
   }
 
 
+  @Override
+  public void close() {
+    if (this.proxy != null) {
+      RPC.stopProxy(this.proxy);
+    }
+  }
+
   @Override
   @Override
   public RefreshQueuesResponse refreshQueues(RefreshQueuesRequest request)
   public RefreshQueuesResponse refreshQueues(RefreshQueuesRequest request)
       throws YarnRemoteException {
       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 = 
   public static final String DEFAULT_RM_SCHEDULER = 
       "org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler";
       "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
   //Delegation token related keys
   public static final String  DELEGATION_KEY_UPDATE_INTERVAL_KEY = 
   public static final String  DELEGATION_KEY_UPDATE_INTERVAL_KEY = 
     RM_PREFIX + "delegation.key.update-interval";
     RM_PREFIX + "delegation.key.update-interval";
@@ -329,19 +334,21 @@ public class YarnConfiguration extends Configuration {
     NM_PREFIX +  "delete.thread-count";
     NM_PREFIX +  "delete.thread-count";
   public static final int DEFAULT_NM_DELETE_THREAD_COUNT = 4;
   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.*/
   /** Keytab for NM.*/
   public static final String NM_KEYTAB = NM_PREFIX + "keytab";
   public static final String NM_KEYTAB = NM_PREFIX + "keytab";
   
   
   /**List of directories to store localized files in.*/
   /**List of directories to store localized files in.*/
   public static final String NM_LOCAL_DIRS = NM_PREFIX + "local-dirs";
   public static final String NM_LOCAL_DIRS = NM_PREFIX + "local-dirs";
   public static final String DEFAULT_NM_LOCAL_DIRS = "/tmp/nm-local-dir";
   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.*/
   /** Address where the localizer IPC is.*/
   public static final String NM_LOCALIZER_ADDRESS =
   public static final String NM_LOCALIZER_ADDRESS =
     NM_PREFIX + "localizer.address";
     NM_PREFIX + "localizer.address";
@@ -710,21 +717,16 @@ public class YarnConfiguration extends Configuration {
   }
   }
   
   
   public static String getRMWebAppHostAndPort(Configuration conf) {
   public static String getRMWebAppHostAndPort(Configuration conf) {
-    int port = conf.getSocketAddr(
+    InetSocketAddress address = conf.getSocketAddr(
         YarnConfiguration.RM_WEBAPP_ADDRESS,
         YarnConfiguration.RM_WEBAPP_ADDRESS,
         YarnConfiguration.DEFAULT_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();
     StringBuffer sb = new StringBuffer();
     InetAddress resolved = address.getAddress();
     InetAddress resolved = address.getAddress();
     if (resolved == null || resolved.isAnyLocalAddress() || 
     if (resolved == null || resolved.isAnyLocalAddress() || 
         resolved.isLoopbackAddress()) {
         resolved.isLoopbackAddress()) {
-      String lh = host;
+      String lh = address.getHostName();
       try {
       try {
         lh = InetAddress.getLocalHost().getCanonicalHostName();
         lh = InetAddress.getLocalHost().getCanonicalHostName();
       } catch (UnknownHostException e) {
       } 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.setNodeHttpAddress(nodeHttpAddress);
     container.setResource(resource);
     container.setResource(resource);
     container.setPriority(priority);
     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);
     container.setContainerToken(containerToken);
     return container;
     return container;
   }
   }

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

@@ -58,10 +58,16 @@
   </property>
   </property>
   
   
   <!-- Resource Manager Configs -->
   <!-- Resource Manager Configs -->
+  <property>
+    <description>The hostname of the RM.</description>
+    <name>yarn.resourcemanager.hostname</name>
+    <value>0.0.0.0</value>
+  </property>    
+  
   <property>
   <property>
     <description>The address of the applications manager interface in the RM.</description>
     <description>The address of the applications manager interface in the RM.</description>
     <name>yarn.resourcemanager.address</name>
     <name>yarn.resourcemanager.address</name>
-    <value>0.0.0.0:8032</value>
+    <value>${yarn.resourcemanager.hostname}:8032</value>
   </property>
   </property>
 
 
   <property>
   <property>
@@ -84,7 +90,7 @@
   <property>
   <property>
     <description>The address of the scheduler interface.</description>
     <description>The address of the scheduler interface.</description>
     <name>yarn.resourcemanager.scheduler.address</name>
     <name>yarn.resourcemanager.scheduler.address</name>
-    <value>0.0.0.0:8030</value>
+    <value>${yarn.resourcemanager.hostname}:8030</value>
   </property>
   </property>
 
 
   <property>
   <property>
@@ -96,12 +102,12 @@
   <property>
   <property>
     <description>The address of the RM web application.</description>
     <description>The address of the RM web application.</description>
     <name>yarn.resourcemanager.webapp.address</name>
     <name>yarn.resourcemanager.webapp.address</name>
-    <value>0.0.0.0:8088</value>
+    <value>${yarn.resourcemanager.hostname}:8088</value>
   </property>
   </property>
 
 
   <property>
   <property>
     <name>yarn.resourcemanager.resource-tracker.address</name>
     <name>yarn.resourcemanager.resource-tracker.address</name>
-    <value>0.0.0.0:8031</value>
+    <value>${yarn.resourcemanager.hostname}:8031</value>
   </property>
   </property>
 
 
   <property>
   <property>
@@ -119,7 +125,7 @@
   <property>
   <property>
     <description>The address of the RM admin interface.</description>
     <description>The address of the RM admin interface.</description>
     <name>yarn.resourcemanager.admin.address</name>
     <name>yarn.resourcemanager.admin.address</name>
-    <value>0.0.0.0:8033</value>
+    <value>${yarn.resourcemanager.hostname}:8033</value>
   </property>
   </property>
 
 
   <property>
   <property>
@@ -278,11 +284,23 @@
     <value>86400</value>
     <value>86400</value>
   </property>
   </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 -->
   <!-- Node Manager Configs -->
+  <property>
+    <description>The hostname of the NM.</description>
+    <name>yarn.nodemanager.hostname</name>
+    <value>0.0.0.0</value>
+  </property>
+  
   <property>
   <property>
     <description>The address of the container manager in the NM.</description>
     <description>The address of the container manager in the NM.</description>
     <name>yarn.nodemanager.address</name>
     <name>yarn.nodemanager.address</name>
-    <value>0.0.0.0:0</value>
+    <value>${yarn.nodemanager.hostname}:0</value>
   </property>
   </property>
 
 
   <property>
   <property>
@@ -336,12 +354,6 @@
     <value>0</value>
     <value>0</value>
   </property>
   </property>
 
 
-  <property>
-    <description>Heartbeat interval to RM</description>
-    <name>yarn.nodemanager.heartbeat.interval-ms</name>
-    <value>1000</value>
-  </property>
-
   <property>
   <property>
     <description>Keytab for NM.</description>
     <description>Keytab for NM.</description>
     <name>yarn.nodemanager.keytab</name>
     <name>yarn.nodemanager.keytab</name>
@@ -359,10 +371,29 @@
     <value>${hadoop.tmp.dir}/nm-local-dir</value>
     <value>${hadoop.tmp.dir}/nm-local-dir</value>
   </property>
   </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>
   <property>
     <description>Address where the localizer IPC is.</description>
     <description>Address where the localizer IPC is.</description>
     <name>yarn.nodemanager.localizer.address</name>
     <name>yarn.nodemanager.localizer.address</name>
-    <value>0.0.0.0:8040</value>
+    <value>${yarn.nodemanager.hostname}:8040</value>
   </property>
   </property>
 
 
   <property>
   <property>
@@ -493,7 +524,7 @@
   <property>
   <property>
     <description>NM Webapp address.</description>
     <description>NM Webapp address.</description>
     <name>yarn.nodemanager.webapp.address</name>
     <name>yarn.nodemanager.webapp.address</name>
-    <value>0.0.0.0:8042</value>
+    <value>${yarn.nodemanager.hostname}:8042</value>
   </property>
   </property>
 
 
   <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
     // make sure they aren't Unknown
     assertTrue("getVersion returned Unknown", !YarnVersionInfo.getVersion().equals("Unknown"));
     assertTrue("getVersion returned Unknown", !YarnVersionInfo.getVersion().equals("Unknown"));
     assertTrue("getUser returned Unknown", !YarnVersionInfo.getUser().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"));
     assertTrue("getSrcChecksum returned Unknown", !YarnVersionInfo.getSrcChecksum().equals("Unknown"));
 
 
     // these could be Unknown if the VersionInfo generated from code not in svn or git
     // these could be Unknown if the VersionInfo generated from code not in svn or git
     // so just check that they return something
     // so just check that they return something
+    assertNotNull("getUrl returned null", YarnVersionInfo.getUrl());
     assertNotNull("getRevision returned null", YarnVersionInfo.getRevision());
     assertNotNull("getRevision returned null", YarnVersionInfo.getRevision());
     assertNotNull("getBranch returned null", YarnVersionInfo.getBranch());
     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 addAllContainersToCleanup(List<ContainerId> containers);
   
   
   void addAllApplicationsToCleanup(List<ApplicationId> applications);
   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);
     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) {
   private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
     return new ContainerIdPBImpl(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;
   optional NodeActionProto nodeAction = 3;
   repeated ContainerIdProto containers_to_cleanup = 4;
   repeated ContainerIdProto containers_to_cleanup = 4;
   repeated ApplicationIdProto applications_to_cleanup = 5;
   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);
     return new DeletionService(exec);
   }
   }
 
 
+  protected NMContext createNMContext(NMContainerTokenSecretManager containerTokenSecretManager) {
+    return new NMContext(containerTokenSecretManager);
+  }
+
   protected void doSecureLogin() throws IOException {
   protected void doSecureLogin() throws IOException {
     SecurityUtil.login(getConfig(), YarnConfiguration.NM_KEYTAB,
     SecurityUtil.login(getConfig(), YarnConfiguration.NM_KEYTAB,
         YarnConfiguration.NM_PRINCIPAL);
         YarnConfiguration.NM_PRINCIPAL);
@@ -137,7 +141,7 @@ public class NodeManager extends CompositeService
       containerTokenSecretManager = new NMContainerTokenSecretManager(conf);
       containerTokenSecretManager = new NMContainerTokenSecretManager(conf);
     }
     }
 
 
-    this.context = new NMContext(containerTokenSecretManager);
+    this.context = createNMContext(containerTokenSecretManager);
 
 
     this.aclsManager = new ApplicationACLsManager(conf);
     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 final Dispatcher dispatcher;
 
 
   private NodeId nodeId;
   private NodeId nodeId;
-  private long heartBeatInterval;
+  private long nextHeartBeatInterval;
   private ResourceTracker resourceTracker;
   private ResourceTracker resourceTracker;
   private InetSocketAddress rmAddress;
   private InetSocketAddress rmAddress;
   private Resource totalResource;
   private Resource totalResource;
@@ -88,6 +88,10 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
   private final NodeHealthCheckerService healthChecker;
   private final NodeHealthCheckerService healthChecker;
   private final NodeManagerMetrics metrics;
   private final NodeManagerMetrics metrics;
 
 
+  private boolean previousHeartBeatSucceeded;
+  private List<ContainerStatus> previousContainersStatuses =
+      new ArrayList<ContainerStatus>();
+
   public NodeStatusUpdaterImpl(Context context, Dispatcher dispatcher,
   public NodeStatusUpdaterImpl(Context context, Dispatcher dispatcher,
       NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) {
       NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) {
     super(NodeStatusUpdaterImpl.class.getName());
     super(NodeStatusUpdaterImpl.class.getName());
@@ -95,6 +99,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
     this.context = context;
     this.context = context;
     this.dispatcher = dispatcher;
     this.dispatcher = dispatcher;
     this.metrics = metrics;
     this.metrics = metrics;
+    this.previousHeartBeatSucceeded = true;
   }
   }
 
 
   @Override
   @Override
@@ -103,9 +108,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
         YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS,
         YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS,
         YarnConfiguration.DEFAULT_RM_RESOURCE_TRACKER_ADDRESS,
         YarnConfiguration.DEFAULT_RM_RESOURCE_TRACKER_ADDRESS,
         YarnConfiguration.DEFAULT_RM_RESOURCE_TRACKER_PORT);
         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 = 
     int memoryMb = 
         conf.getInt(
         conf.getInt(
             YarnConfiguration.NM_PMEM_MB, YarnConfiguration.DEFAULT_NM_PMEM_MB);
             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 nodeStatus = recordFactory.newRecordInstance(NodeStatus.class);
     nodeStatus.setNodeId(this.nodeId);
     nodeStatus.setNodeId(this.nodeId);
 
 
-    int numActiveContainers = 0;
     List<ContainerStatus> containersStatuses = new ArrayList<ContainerStatus>();
     List<ContainerStatus> containersStatuses = new ArrayList<ContainerStatus>();
+    if(previousHeartBeatSucceeded) {
+      previousContainersStatuses.clear();
+    } else {
+      containersStatuses.addAll(previousContainersStatuses);
+    }
+
+    int numActiveContainers = 0;
     for (Iterator<Entry<ContainerId, Container>> i =
     for (Iterator<Entry<ContainerId, Container>> i =
         this.context.getContainers().entrySet().iterator(); i.hasNext();) {
         this.context.getContainers().entrySet().iterator(); i.hasNext();) {
       Entry<ContainerId, Container> e = i.next();
       Entry<ContainerId, Container> e = i.next();
@@ -332,6 +341,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
       LOG.info("Sending out status for container: " + containerStatus);
       LOG.info("Sending out status for container: " + containerStatus);
 
 
       if (containerStatus.getState() == ContainerState.COMPLETE) {
       if (containerStatus.getState() == ContainerState.COMPLETE) {
+        previousContainersStatuses.add(containerStatus);
         // Remove
         // Remove
         i.remove();
         i.remove();
 
 
@@ -394,9 +404,6 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
         while (!isStopped) {
         while (!isStopped) {
           // Send heartbeat
           // Send heartbeat
           try {
           try {
-            synchronized (heartbeatMonitor) {
-              heartbeatMonitor.wait(heartBeatInterval);
-            }
             NodeStatus nodeStatus = getNodeStatus();
             NodeStatus nodeStatus = getNodeStatus();
             nodeStatus.setResponseId(lastHeartBeatID);
             nodeStatus.setResponseId(lastHeartBeatID);
             
             
@@ -409,7 +416,9 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
             }
             }
             NodeHeartbeatResponse response =
             NodeHeartbeatResponse response =
               resourceTracker.nodeHeartbeat(request);
               resourceTracker.nodeHeartbeat(request);
-
+            previousHeartBeatSucceeded = true;
+            //get next heartbeat interval from response
+            nextHeartBeatInterval = response.getNextHeartBeatInterval();
             // See if the master-key has rolled over
             // See if the master-key has rolled over
             if (isSecurityEnabled()) {
             if (isSecurityEnabled()) {
               MasterKey updatedMasterKey = response.getMasterKey();
               MasterKey updatedMasterKey = response.getMasterKey();
@@ -453,9 +462,21 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
                   new CMgrCompletedAppsEvent(appsToCleanup));
                   new CMgrCompletedAppsEvent(appsToCleanup));
             }
             }
           } catch (Throwable e) {
           } catch (Throwable e) {
+            previousHeartBeatSucceeded = false;
             // TODO Better error handling. Thread can die with the rest of the
             // TODO Better error handling. Thread can die with the rest of the
             // NM still running.
             // NM still running.
             LOG.error("Caught exception in status-updater", e);
             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;
 package org.apache.hadoop.yarn.server.nodemanager.api.impl.pb.client;
 
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 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;
 import com.google.protobuf.ServiceException;
 
 
-public class LocalizationProtocolPBClientImpl implements LocalizationProtocol {
+public class LocalizationProtocolPBClientImpl implements LocalizationProtocol,
+    Closeable {
 
 
   private LocalizationProtocolPB proxy;
   private LocalizationProtocolPB proxy;
   
   
@@ -44,7 +46,14 @@ public class LocalizationProtocolPBClientImpl implements LocalizationProtocol {
     proxy = (LocalizationProtocolPB)RPC.getProxy(
     proxy = (LocalizationProtocolPB)RPC.getProxy(
         LocalizationProtocolPB.class, clientVersion, addr, conf);
         LocalizationProtocolPB.class, clientVersion, addr, conf);
   }
   }
-  
+
+  @Override
+  public void close() {
+    if (this.proxy != null) {
+      RPC.stopProxy(this.proxy);
+    }
+  }
+
   @Override
   @Override
   public LocalizerHeartbeatResponse heartbeat(LocalizerStatus status)
   public LocalizerHeartbeatResponse heartbeat(LocalizerStatus status)
     throws YarnRemoteException {
     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;
 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.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
@@ -35,6 +36,11 @@ interface LocalResourcesTracker
 
 
   boolean remove(LocalizedResource req, DeletionService delService);
   boolean remove(LocalizedResource req, DeletionService delService);
 
 
+  Path getPathForLocalization(LocalResourceRequest req, Path localDirPath);
+
   String getUser();
   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.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
 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.ResourceEvent;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ResourceEventType;
+
 
 
 /**
 /**
  * A collection of {@link LocalizedResource}s all of same
  * A collection of {@link LocalizedResource}s all of same
@@ -49,17 +50,43 @@ class LocalResourcesTrackerImpl implements LocalResourcesTracker {
   private final String user;
   private final String user;
   private final Dispatcher dispatcher;
   private final Dispatcher dispatcher;
   private final ConcurrentMap<LocalResourceRequest,LocalizedResource> localrsrc;
   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,
     this(user, dispatcher,
-        new ConcurrentHashMap<LocalResourceRequest,LocalizedResource>());
+      new ConcurrentHashMap<LocalResourceRequest, LocalizedResource>(),
+      useLocalCacheDirectoryManager, conf);
   }
   }
 
 
   LocalResourcesTrackerImpl(String user, Dispatcher dispatcher,
   LocalResourcesTrackerImpl(String user, Dispatcher dispatcher,
-      ConcurrentMap<LocalResourceRequest,LocalizedResource> localrsrc) {
+      ConcurrentMap<LocalResourceRequest,LocalizedResource> localrsrc,
+      boolean useLocalCacheDirectoryManager, Configuration conf) {
     this.user = user;
     this.user = user;
     this.dispatcher = dispatcher;
     this.dispatcher = dispatcher;
     this.localrsrc = localrsrc;
     this.localrsrc = localrsrc;
+    this.useLocalCacheDirectoryManager = useLocalCacheDirectoryManager;
+    if ( this.useLocalCacheDirectoryManager) {
+      directoryManagers = new ConcurrentHashMap<Path, LocalCacheDirectoryManager>();
+      inProgressLocalResourcesMap =
+        new ConcurrentHashMap<LocalResourceRequest, Path>();
+    }
+    this.conf = conf;
   }
   }
 
 
   @Override
   @Override
@@ -73,6 +100,7 @@ class LocalResourcesTrackerImpl implements LocalResourcesTracker {
         LOG.info("Resource " + rsrc.getLocalPath()
         LOG.info("Resource " + rsrc.getLocalPath()
             + " is missing, localizing it again");
             + " is missing, localizing it again");
         localrsrc.remove(req);
         localrsrc.remove(req);
+        decrementFileCountForLocalCacheDirectory(req, rsrc);
         rsrc = null;
         rsrc = null;
       }
       }
       if (null == rsrc) {
       if (null == rsrc) {
@@ -90,7 +118,52 @@ class LocalResourcesTrackerImpl implements LocalResourcesTracker {
     rsrc.handle(event);
     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
    * This module checks if the resource which was localized is already present
    * or not
    * or not
    * 
    * 
@@ -100,7 +173,8 @@ class LocalResourcesTrackerImpl implements LocalResourcesTracker {
   public boolean isResourcePresent(LocalizedResource rsrc) {
   public boolean isResourcePresent(LocalizedResource rsrc) {
     boolean ret = true;
     boolean ret = true;
     if (rsrc.getState() == ResourceState.LOCALIZED) {
     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()) {
       if (!file.exists()) {
         ret = false;
         ret = false;
       }
       }
@@ -133,11 +207,11 @@ class LocalResourcesTrackerImpl implements LocalResourcesTracker {
       if (ResourceState.LOCALIZED.equals(rsrc.getState())) {
       if (ResourceState.LOCALIZED.equals(rsrc.getState())) {
         delService.delete(getUser(), getPathToDelete(rsrc.getLocalPath()));
         delService.delete(getUser(), getPathToDelete(rsrc.getLocalPath()));
       }
       }
+      decrementFileCountForLocalCacheDirectory(rem.getRequest(), rsrc);
       return true;
       return true;
     }
     }
   }
   }
 
 
-
   /**
   /**
    * Returns the path up to the random directory component.
    * Returns the path up to the random directory component.
    */
    */
@@ -163,4 +237,50 @@ class LocalResourcesTrackerImpl implements LocalResourcesTracker {
   public Iterator<LocalizedResource> iterator() {
   public Iterator<LocalizedResource> iterator() {
     return localrsrc.values().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.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.util.DiskChecker;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -130,7 +131,7 @@ public class ResourceLocalizationService extends CompositeService
   private RecordFactory recordFactory;
   private RecordFactory recordFactory;
   private final ScheduledExecutorService cacheCleanup;
   private final ScheduledExecutorService cacheCleanup;
 
 
-  private final LocalResourcesTracker publicRsrc;
+  private LocalResourcesTracker publicRsrc;
 
 
   private LocalDirsHandlerService dirsHandler;
   private LocalDirsHandlerService dirsHandler;
 
 
@@ -158,7 +159,6 @@ public class ResourceLocalizationService extends CompositeService
     this.delService = delService;
     this.delService = delService;
     this.dirsHandler = dirsHandler;
     this.dirsHandler = dirsHandler;
 
 
-    this.publicRsrc = new LocalResourcesTrackerImpl(null, dispatcher);
     this.cacheCleanup = new ScheduledThreadPoolExecutor(1,
     this.cacheCleanup = new ScheduledThreadPoolExecutor(1,
         new ThreadFactoryBuilder()
         new ThreadFactoryBuilder()
           .setNameFormat("ResourceLocalizationService Cache Cleanup")
           .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
   @Override
   public void init(Configuration conf) {
   public void init(Configuration conf) {
+    this.validateConf(conf);
+    this.publicRsrc =
+        new LocalResourcesTrackerImpl(null, dispatcher, true, conf);
     this.recordFactory = RecordFactoryProvider.getRecordFactory(conf);
     this.recordFactory = RecordFactoryProvider.getRecordFactory(conf);
 
 
     try {
     try {
@@ -212,6 +230,7 @@ public class ResourceLocalizationService extends CompositeService
         YarnConfiguration.NM_LOCALIZER_ADDRESS,
         YarnConfiguration.NM_LOCALIZER_ADDRESS,
         YarnConfiguration.DEFAULT_NM_LOCALIZER_ADDRESS,
         YarnConfiguration.DEFAULT_NM_LOCALIZER_ADDRESS,
         YarnConfiguration.DEFAULT_NM_LOCALIZER_PORT);
         YarnConfiguration.DEFAULT_NM_LOCALIZER_PORT);
+
     localizerTracker = createLocalizerTracker(conf);
     localizerTracker = createLocalizerTracker(conf);
     addService(localizerTracker);
     addService(localizerTracker);
     dispatcher.register(LocalizerEventType.class, localizerTracker);
     dispatcher.register(LocalizerEventType.class, localizerTracker);
@@ -306,15 +325,17 @@ public class ResourceLocalizationService extends CompositeService
   private void handleInitApplicationResources(Application app) {
   private void handleInitApplicationResources(Application app) {
     // 0) Create application tracking structs
     // 0) Create application tracking structs
     String userName = app.getUser();
     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");
       LOG.warn("Initializing application " + app + " already present");
       assert false; // TODO: FIXME assert doesn't help
       assert false; // TODO: FIXME assert doesn't help
                     // ^ The condition is benign. Tests should fail and it
                     // ^ 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
     // 1) Signal container init
     //
     //
@@ -620,6 +641,13 @@ public class ResourceLocalizationService extends CompositeService
             Path publicDirDestPath = dirsHandler.getLocalPathForWrite(
             Path publicDirDestPath = dirsHandler.getLocalPathForWrite(
                 "." + Path.SEPARATOR + ContainerLocalizer.FILECACHE,
                 "." + Path.SEPARATOR + ContainerLocalizer.FILECACHE,
                 ContainerLocalizer.getEstimatedSize(resource), true);
                 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(
             pending.put(queue.submit(new FSDownload(
                 lfs, null, conf, publicDirDestPath, resource, new Random())),
                 lfs, null, conf, publicDirDestPath, resource, new Random())),
                 request);
                 request);
@@ -654,19 +682,21 @@ public class ResourceLocalizationService extends CompositeService
               assoc.getResource().handle(
               assoc.getResource().handle(
                   new ResourceLocalizedEvent(key,
                   new ResourceLocalizedEvent(key,
                     local, FileUtil.getDU(new File(local.toUri()))));
                     local, FileUtil.getDU(new File(local.toUri()))));
+              publicRsrc.localizationCompleted(key, true);
               synchronized (attempts) {
               synchronized (attempts) {
                 attempts.remove(key);
                 attempts.remove(key);
               }
               }
             } catch (ExecutionException e) {
             } catch (ExecutionException e) {
               LOG.info("Failed to download rsrc " + assoc.getResource(),
               LOG.info("Failed to download rsrc " + assoc.getResource(),
                   e.getCause());
                   e.getCause());
+              LocalResourceRequest req = assoc.getResource().getRequest();
               dispatcher.getEventHandler().handle(
               dispatcher.getEventHandler().handle(
                   new ContainerResourceFailedEvent(
                   new ContainerResourceFailedEvent(
                     assoc.getContext().getContainerId(),
                     assoc.getContext().getContainerId(),
-                    assoc.getResource().getRequest(), e.getCause()));
+                    req, e.getCause()));
+              publicRsrc.localizationCompleted(req, false);
               List<LocalizerResourceRequestEvent> reqs;
               List<LocalizerResourceRequestEvent> reqs;
               synchronized (attempts) {
               synchronized (attempts) {
-                LocalResourceRequest req = assoc.getResource().getRequest();
                 reqs = attempts.get(req);
                 reqs = attempts.get(req);
                 if (null == reqs) {
                 if (null == reqs) {
                   LOG.error("Missing pending list for " + req);
                   LOG.error("Missing pending list for " + req);
@@ -1003,4 +1033,4 @@ public class ResourceLocalizationService extends CompositeService
     del.delete(null, dirPath, new Path[] {});
     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.protocolrecords.RegisterNodeManagerResponse;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
 import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
 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
  * 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);
       LOG.info("Got heartbeat number " + heartBeatID);
       nodeStatus.setResponseId(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;
       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;
 package org.apache.hadoop.yarn.server.nodemanager;
 
 
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
@@ -29,6 +30,7 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicInteger;
 
 
 import org.apache.commons.logging.Log;
 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.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 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.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 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.protocolrecords.RegisterNodeManagerResponse;
 import org.apache.hadoop.yarn.server.api.records.NodeAction;
 import org.apache.hadoop.yarn.server.api.records.NodeAction;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
 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.ContainerManagerImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
 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.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl;
 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.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.security.ApplicationACLsManager;
+import org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils;
 import org.apache.hadoop.yarn.service.Service;
 import org.apache.hadoop.yarn.service.Service;
 import org.apache.hadoop.yarn.service.Service.STATE;
 import org.apache.hadoop.yarn.service.Service.STATE;
 import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.apache.hadoop.yarn.util.BuilderUtils;
@@ -91,6 +97,8 @@ public class TestNodeStatusUpdater {
   private final Configuration conf = createNMConfig();
   private final Configuration conf = createNMConfig();
   private NodeManager nm;
   private NodeManager nm;
   protected NodeManager rebootedNodeManager;
   protected NodeManager rebootedNodeManager;
+  private boolean containerStatusBackupSuccessfully = true;
+  private List<ContainerStatus> completedContainerStatusList = new ArrayList<ContainerStatus>();
 
 
   @After
   @After
   public void tearDown() {
   public void tearDown() {
@@ -214,9 +222,8 @@ public class TestNodeStatusUpdater {
         Assert.assertEquals(2, activeContainers.size());
         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;
       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 {
   private class MyNodeStatusUpdater3 extends NodeStatusUpdaterImpl {
     public ResourceTracker resourceTracker;
     public ResourceTracker resourceTracker;
     private Context context;
     private Context context;
@@ -325,10 +348,9 @@ public class TestNodeStatusUpdater {
       NodeStatus nodeStatus = request.getNodeStatus();
       NodeStatus nodeStatus = request.getNodeStatus();
       nodeStatus.setResponseId(heartBeatID++);
       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;
       return nhResponse;
     }
     }
   }
   }
@@ -361,10 +383,9 @@ public class TestNodeStatusUpdater {
       LOG.info("Got heartBeatId: [" + heartBeatID +"]");
       LOG.info("Got heartBeatId: [" + heartBeatID +"]");
       NodeStatus nodeStatus = request.getNodeStatus();
       NodeStatus nodeStatus = request.getNodeStatus();
       nodeStatus.setResponseId(heartBeatID++);
       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
       if (nodeStatus.getKeepAliveApplications() != null
           && nodeStatus.getKeepAliveApplications().size() > 0) {
           && 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
   @Before
   public void clearError() {
   public void clearError() {
     nmStartError = null;
     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) {
   private void verifyNodeStartFailure(String errMessage) {
     YarnConfiguration conf = createNMConfig();
     YarnConfiguration conf = createNMConfig();
     nm.init(conf);
     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.ContainerId;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 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.Dispatcher;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
 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.server.nodemanager.containermanager.localizer.event.ResourceRequestEvent;
 import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.junit.Test;
 import org.junit.Test;
-import org.mortbay.log.Log;
 
 
 public class TestLocalResourcesTrackerImpl {
 public class TestLocalResourcesTrackerImpl {
 
 
-  @Test
+  @Test(timeout=10000)
   @SuppressWarnings("unchecked")
   @SuppressWarnings("unchecked")
   public void test() {
   public void test() {
     String user = "testuser";
     String user = "testuser";
     DrainDispatcher dispatcher = null;
     DrainDispatcher dispatcher = null;
     try {
     try {
-      dispatcher = createDispatcher(new Configuration());
+      Configuration conf = new Configuration();
+      dispatcher = createDispatcher(conf);
       EventHandler<LocalizerEvent> localizerEventHandler =
       EventHandler<LocalizerEvent> localizerEventHandler =
           mock(EventHandler.class);
           mock(EventHandler.class);
       EventHandler<LocalizerEvent> containerEventHandler =
       EventHandler<LocalizerEvent> containerEventHandler =
@@ -86,7 +87,8 @@ public class TestLocalResourcesTrackerImpl {
       localrsrc.put(req1, lr1);
       localrsrc.put(req1, lr1);
       localrsrc.put(req2, lr2);
       localrsrc.put(req2, lr2);
       LocalResourcesTracker tracker =
       LocalResourcesTracker tracker =
-          new LocalResourcesTrackerImpl(user, dispatcher, localrsrc);
+          new LocalResourcesTrackerImpl(user, dispatcher, localrsrc, false,
+            conf);
 
 
       ResourceEvent req11Event =
       ResourceEvent req11Event =
           new ResourceRequestEvent(req1, LocalResourceVisibility.PUBLIC, lc1);
           new ResourceRequestEvent(req1, LocalResourceVisibility.PUBLIC, lc1);
@@ -152,13 +154,14 @@ public class TestLocalResourcesTrackerImpl {
     }
     }
   }
   }
 
 
-  @Test
+  @Test(timeout=10000)
   @SuppressWarnings("unchecked")
   @SuppressWarnings("unchecked")
   public void testConsistency() {
   public void testConsistency() {
     String user = "testuser";
     String user = "testuser";
     DrainDispatcher dispatcher = null;
     DrainDispatcher dispatcher = null;
     try {
     try {
-      dispatcher = createDispatcher(new Configuration());
+      Configuration conf = new Configuration();
+      dispatcher = createDispatcher(conf);
       EventHandler<LocalizerEvent> localizerEventHandler = mock(EventHandler.class);
       EventHandler<LocalizerEvent> localizerEventHandler = mock(EventHandler.class);
       EventHandler<LocalizerEvent> containerEventHandler = mock(EventHandler.class);
       EventHandler<LocalizerEvent> containerEventHandler = mock(EventHandler.class);
       dispatcher.register(LocalizerEventType.class, localizerEventHandler);
       dispatcher.register(LocalizerEventType.class, localizerEventHandler);
@@ -172,7 +175,7 @@ public class TestLocalResourcesTrackerImpl {
       ConcurrentMap<LocalResourceRequest, LocalizedResource> localrsrc = new ConcurrentHashMap<LocalResourceRequest, LocalizedResource>();
       ConcurrentMap<LocalResourceRequest, LocalizedResource> localrsrc = new ConcurrentHashMap<LocalResourceRequest, LocalizedResource>();
       localrsrc.put(req1, lr1);
       localrsrc.put(req1, lr1);
       LocalResourcesTracker tracker = new LocalResourcesTrackerImpl(user,
       LocalResourcesTracker tracker = new LocalResourcesTrackerImpl(user,
-          dispatcher, localrsrc);
+          dispatcher, localrsrc, false, conf);
 
 
       ResourceEvent req11Event = new ResourceRequestEvent(req1,
       ResourceEvent req11Event = new ResourceRequestEvent(req1,
           LocalResourceVisibility.PUBLIC, lc1);
           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) {
   private boolean createdummylocalizefile(Path path) {
     boolean ret = false;
     boolean ret = false;
     File file = new File(path.toUri().getRawPath().toString());
     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.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ConcurrentMap;
 
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
@@ -76,10 +77,11 @@ public class TestResourceRetention {
 
 
   LocalResourcesTracker createMockTracker(String user, final long rsrcSize,
   LocalResourcesTracker createMockTracker(String user, final long rsrcSize,
       long nRsrcs, long timestamp, long tsstep) {
       long nRsrcs, long timestamp, long tsstep) {
+    Configuration conf = new Configuration();
     ConcurrentMap<LocalResourceRequest,LocalizedResource> trackerResources =
     ConcurrentMap<LocalResourceRequest,LocalizedResource> trackerResources =
       new ConcurrentHashMap<LocalResourceRequest,LocalizedResource>();
       new ConcurrentHashMap<LocalResourceRequest,LocalizedResource>();
     LocalResourcesTracker ret = spy(new LocalResourcesTrackerImpl(user, null,
     LocalResourcesTracker ret = spy(new LocalResourcesTrackerImpl(user, null,
-          trackerResources));
+          trackerResources, false, conf));
     for (int i = 0; i < nRsrcs; ++i) {
     for (int i = 0; i < nRsrcs; ++i) {
       final LocalResourceRequest req = new LocalResourceRequest(
       final LocalResourceRequest req = new LocalResourceRequest(
           new Path("file:///" + user + "/rsrc" + i), timestamp + i * tsstep,
           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.net.Node;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.PolicyProvider;
 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.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 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.rmnode.RMNodeStatusEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 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.resourcemanager.security.authorize.RMPolicyProvider;
+import org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils;
 import org.apache.hadoop.yarn.service.AbstractService;
 import org.apache.hadoop.yarn.service.AbstractService;
 import org.apache.hadoop.yarn.util.RackResolver;
 import org.apache.hadoop.yarn.util.RackResolver;
 
 
@@ -67,6 +69,7 @@ public class ResourceTrackerService extends AbstractService implements
   private final NMLivelinessMonitor nmLivelinessMonitor;
   private final NMLivelinessMonitor nmLivelinessMonitor;
   private final RMContainerTokenSecretManager containerTokenSecretManager;
   private final RMContainerTokenSecretManager containerTokenSecretManager;
 
 
+  private long nextHeartBeatInterval;
   private Server server;
   private Server server;
   private InetSocketAddress resourceTrackerAddress;
   private InetSocketAddress resourceTrackerAddress;
 
 
@@ -100,6 +103,14 @@ public class ResourceTrackerService extends AbstractService implements
         YarnConfiguration.DEFAULT_RM_RESOURCE_TRACKER_PORT);
         YarnConfiguration.DEFAULT_RM_RESOURCE_TRACKER_PORT);
 
 
     RackResolver.init(conf);
     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);
     super.init(conf);
   }
   }
 
 
@@ -223,9 +234,6 @@ public class ResourceTrackerService extends AbstractService implements
           new RMNodeEvent(nodeId, RMNodeEventType.DECOMMISSION));
           new RMNodeEvent(nodeId, RMNodeEventType.DECOMMISSION));
       return shutDown;
       return shutDown;
     }
     }
-
-    NodeHeartbeatResponse nodeHeartBeatResponse = recordFactory
-        .newRecordInstance(NodeHeartbeatResponse.class);
     
     
     // 3. Check if it's a 'fresh' heartbeat i.e. not duplicate heartbeat
     // 3. Check if it's a 'fresh' heartbeat i.e. not duplicate heartbeat
     NodeHeartbeatResponse lastNodeHeartbeatResponse = rmNode.getLastNodeHeartBeatResponse();
     NodeHeartbeatResponse lastNodeHeartbeatResponse = rmNode.getLastNodeHeartBeatResponse();
@@ -246,10 +254,11 @@ public class ResourceTrackerService extends AbstractService implements
     }
     }
 
 
     // Heartbeat response
     // Heartbeat response
-    nodeHeartBeatResponse.setResponseId(lastNodeHeartbeatResponse.getResponseId() + 1);
+    NodeHeartbeatResponse nodeHeartBeatResponse = YarnServerBuilderUtils
+        .newNodeHeartbeatResponse(lastNodeHeartbeatResponse.
+            getResponseId() + 1, NodeAction.NORMAL, null, null, null,
+            nextHeartBeatInterval);
     rmNode.updateNodeHeartbeatResponseForCleanup(nodeHeartBeatResponse);
     rmNode.updateNodeHeartbeatResponseForCleanup(nodeHeartBeatResponse);
-    nodeHeartBeatResponse.setNodeAction(NodeAction.NORMAL);
-
     // Check if node's masterKey needs to be updated and if the currentKey has
     // Check if node's masterKey needs to be updated and if the currentKey has
     // roller over, send it across
     // roller over, send it across
     if (isSecurityEnabled()) {
     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
       // Set the masterContainer
       appAttempt.setMasterContainer(amContainerAllocation.getContainers().get(
       appAttempt.setMasterContainer(amContainerAllocation.getContainers().get(
                                                                            0));
                                                                            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();
       RMStateStore store = appAttempt.rmContext.getStateStore();
       appAttempt.storeAttempt(store);
       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) {
     public void transition(RMContainerImpl container, RMContainerEvent event) {
       RMContainerFinishedEvent finishedEvent = (RMContainerFinishedEvent) 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
       // Inform AppAttempt
       container.eventHandler.handle(new RMAppAttemptContainerFinishedEvent(
       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 = 
     Resource normalized = 
         Resources.normalize(
         Resources.normalize(
             resourceCalculator, ask.getCapability(), minimumResource);
             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>() {
     new Comparator<FiCaSchedulerApp>() {
     @Override
     @Override
     public int compare(FiCaSchedulerApp a1, FiCaSchedulerApp a2) {
     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.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 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.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeHealthStatus;
 import org.apache.hadoop.yarn.api.records.NodeHealthStatus;
@@ -71,11 +70,11 @@ public class MockNM {
     this.resourceTracker = resourceTracker;
     this.resourceTracker = resourceTracker;
   }
   }
 
 
-  public void containerStatus(Container container) throws Exception {
+  public void containerStatus(ContainerStatus containerStatus) throws Exception {
     Map<ApplicationId, List<ContainerStatus>> conts = 
     Map<ApplicationId, List<ContainerStatus>> conts = 
         new HashMap<ApplicationId, List<ContainerStatus>>();
         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);
     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 = 
   final Map<ApplicationId, List<Container>> containers = 
     new HashMap<ApplicationId, List<Container>>();
     new HashMap<ApplicationId, List<Container>>();
   
   
+  final Map<Container, ContainerStatus> containerStatusMap =
+      new HashMap<Container, ContainerStatus>();
+
   public NodeManager(String hostName, int containerManagerPort, int httpPort,
   public NodeManager(String hostName, int containerManagerPort, int httpPort,
       String rackName, Resource capability,
       String rackName, Resource capability,
       ResourceTrackerService resourceTrackerService, RMContext rmContext)
       ResourceTrackerService resourceTrackerService, RMContext rmContext)
@@ -137,7 +140,7 @@ public class NodeManager implements ContainerManager {
     List<ContainerStatus> containerStatuses = new ArrayList<ContainerStatus>();
     List<ContainerStatus> containerStatuses = new ArrayList<ContainerStatus>();
     for (List<Container> appContainers : containers.values()) {
     for (List<Container> appContainers : containers.values()) {
       for (Container container : appContainers) {
       for (Container container : appContainers) {
-        containerStatuses.add(container.getContainerStatus());
+        containerStatuses.add(containerStatusMap.get(container));
       }
       }
     }
     }
     return containerStatuses;
     return containerStatuses;
@@ -189,8 +192,11 @@ public class NodeManager implements ContainerManager {
             null, null                                 // DKDC - Doesn't matter
             null, null                                 // DKDC - Doesn't matter
             );
             );
 
 
+    ContainerStatus containerStatus =
+        BuilderUtils.newContainerStatus(container.getId(), ContainerState.NEW,
+            "", -1000);
     applicationContainers.add(container);
     applicationContainers.add(container);
-    
+    containerStatusMap.put(container, containerStatus);
     Resources.subtractFrom(available, containerLaunchContext.getResource());
     Resources.subtractFrom(available, containerLaunchContext.getResource());
     Resources.addTo(used, containerLaunchContext.getResource());
     Resources.addTo(used, containerLaunchContext.getResource());
     
     
@@ -223,7 +229,9 @@ public class NodeManager implements ContainerManager {
     List<Container> applicationContainers = containers.get(applicationId);
     List<Container> applicationContainers = containers.get(applicationId);
     for (Container c : applicationContainers) {
     for (Container c : applicationContainers) {
       if (c.getId().compareTo(containerID) == 0) {
       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 = 
     GetContainerStatusResponse response = 
         recordFactory.newRecordInstance(GetContainerStatusResponse.class);
         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;
     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);
     Container c1 = allocated1.get(0);
     Assert.assertEquals(GB, c1.getResource().getMemory());
     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;
     int waitCount = 0;
     while (attempt1.getJustFinishedContainers().size() < 1
     while (attempt1.getJustFinishedContainers().size() < 1
         && waitCount++ != 20) {
         && 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 File hostFile = new File(TEMP_DIR + File.separator + "hostFile.txt");
   private MockRM rm;
   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
    * 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);
     submissionContext = mock(ApplicationSubmissionContext.class);
     when(submissionContext.getUser()).thenReturn(user);
     when(submissionContext.getUser()).thenReturn(user);
     when(submissionContext.getQueue()).thenReturn(queue);
     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);
     when(submissionContext.getAMContainerSpec()).thenReturn(amContainerSpec);
     
     
     unmanagedAM = false;
     unmanagedAM = false;
@@ -469,8 +470,10 @@ public class TestRMAppAttemptTransitions {
     
     
     // Mock the allocation of AM container 
     // Mock the allocation of AM container 
     Container container = mock(Container.class);
     Container container = mock(Container.class);
+    Resource resource = BuilderUtils.newResource(2048, 1);
     when(container.getId()).thenReturn(
     when(container.getId()).thenReturn(
         BuilderUtils.newContainerId(applicationAttempt.getAppAttemptId(), 1));
         BuilderUtils.newContainerId(applicationAttempt.getAppAttemptId(), 1));
+    when(container.getResource()).thenReturn(resource);
     Allocation allocation = mock(Allocation.class);
     Allocation allocation = mock(Allocation.class);
     when(allocation.getContainers()).
     when(allocation.getContainers()).
         thenReturn(Collections.singletonList(container));
         thenReturn(Collections.singletonList(container));
@@ -491,6 +494,9 @@ public class TestRMAppAttemptTransitions {
     applicationAttempt.handle(
     applicationAttempt.handle(
         new RMAppAttemptStoredEvent(
         new RMAppAttemptStoredEvent(
             applicationAttempt.getAppAttemptId(), null));
             applicationAttempt.getAppAttemptId(), null));
+    assertEquals(resource,
+        applicationAttempt.getSubmissionContext()
+        .getAMContainerSpec().getResource());
     
     
     testAppAttemptAllocatedState(container);
     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;
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
 
 import static org.junit.Assert.assertEquals;
 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.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
@@ -43,35 +42,23 @@ public class TestSchedulerUtils {
 
 
     // case negative memory
     // case negative memory
     ask.setCapability(Resources.createResource(-1024));
     ask.setCapability(Resources.createResource(-1024));
-    Resource before = ask.getCapability();
     SchedulerUtils.normalizeRequest(ask, resourceCalculator, null, minResource);
     SchedulerUtils.normalizeRequest(ask, resourceCalculator, null, minResource);
-    Resource after = ask.getCapability();
     assertEquals(minMemory, ask.getCapability().getMemory());
     assertEquals(minMemory, ask.getCapability().getMemory());
-    assertTrue(before == after);
 
 
     // case zero memory
     // case zero memory
     ask.setCapability(Resources.createResource(0));
     ask.setCapability(Resources.createResource(0));
-    before = ask.getCapability();
     SchedulerUtils.normalizeRequest(ask, resourceCalculator, null, minResource);
     SchedulerUtils.normalizeRequest(ask, resourceCalculator, null, minResource);
-    after = ask.getCapability();
     assertEquals(minMemory, ask.getCapability().getMemory());
     assertEquals(minMemory, ask.getCapability().getMemory());
-    assertTrue(before == after);
 
 
     // case memory is a multiple of minMemory
     // case memory is a multiple of minMemory
     ask.setCapability(Resources.createResource(2 * minMemory));
     ask.setCapability(Resources.createResource(2 * minMemory));
-    before = ask.getCapability();
     SchedulerUtils.normalizeRequest(ask, resourceCalculator, null, minResource);
     SchedulerUtils.normalizeRequest(ask, resourceCalculator, null, minResource);
-    after = ask.getCapability();
     assertEquals(2 * minMemory, ask.getCapability().getMemory());
     assertEquals(2 * minMemory, ask.getCapability().getMemory());
-    assertTrue(before == after);
 
 
     // case memory is not a multiple of minMemory
     // case memory is not a multiple of minMemory
     ask.setCapability(Resources.createResource(minMemory + 10));
     ask.setCapability(Resources.createResource(minMemory + 10));
-    before = ask.getCapability();
     SchedulerUtils.normalizeRequest(ask, resourceCalculator, null, minResource);
     SchedulerUtils.normalizeRequest(ask, resourceCalculator, null, minResource);
-    after = ask.getCapability();
     assertEquals(2 * minMemory, ask.getCapability().getMemory());
     assertEquals(2 * minMemory, ask.getCapability().getMemory());
-    assertTrue(before == after);
 
 
   }
   }
   
   
@@ -86,33 +73,24 @@ public class TestSchedulerUtils {
 
 
     // case negative memory/vcores
     // case negative memory/vcores
     ask.setCapability(Resources.createResource(-1024, -1));
     ask.setCapability(Resources.createResource(-1024, -1));
-    Resource before = ask.getCapability();
     SchedulerUtils.normalizeRequest(
     SchedulerUtils.normalizeRequest(
         ask, resourceCalculator, clusterResource, minResource);
         ask, resourceCalculator, clusterResource, minResource);
-    Resource after = ask.getCapability();
     assertEquals(minResource, ask.getCapability());
     assertEquals(minResource, ask.getCapability());
-    assertTrue(before == after);
 
 
     // case zero memory/vcores
     // case zero memory/vcores
     ask.setCapability(Resources.createResource(0, 0));
     ask.setCapability(Resources.createResource(0, 0));
-    before = ask.getCapability();
     SchedulerUtils.normalizeRequest(
     SchedulerUtils.normalizeRequest(
         ask, resourceCalculator, clusterResource, minResource);
         ask, resourceCalculator, clusterResource, minResource);
-    after = ask.getCapability();
     assertEquals(minResource, ask.getCapability());
     assertEquals(minResource, ask.getCapability());
     assertEquals(1, ask.getCapability().getVirtualCores());
     assertEquals(1, ask.getCapability().getVirtualCores());
     assertEquals(1024, ask.getCapability().getMemory());
     assertEquals(1024, ask.getCapability().getMemory());
-    assertTrue(before == after);
 
 
     // case non-zero memory & zero cores
     // case non-zero memory & zero cores
     ask.setCapability(Resources.createResource(1536, 0));
     ask.setCapability(Resources.createResource(1536, 0));
-    before = ask.getCapability();
     SchedulerUtils.normalizeRequest(
     SchedulerUtils.normalizeRequest(
         ask, resourceCalculator, clusterResource, minResource);
         ask, resourceCalculator, clusterResource, minResource);
-    after = ask.getCapability();
     assertEquals(Resources.createResource(2048, 1), ask.getCapability());
     assertEquals(Resources.createResource(2048, 1), ask.getCapability());
     assertEquals(1, ask.getCapability().getVirtualCores());
     assertEquals(1, ask.getCapability().getVirtualCores());
     assertEquals(2048, ask.getCapability().getMemory());
     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;
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.util.Comparator;
 import java.util.List;
 import java.util.List;
 
 
 import junit.framework.Assert;
 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.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 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.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.NodeAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
 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.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 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.After;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
+import org.mockito.Mockito;
+import static org.mockito.Mockito.*;
+
 
 
 public class TestCapacityScheduler {
 public class TestCapacityScheduler {
   private static final Log LOG = LogFactory.getLog(TestCapacityScheduler.class);
   private static final Log LOG = LogFactory.getLog(TestCapacityScheduler.class);
@@ -452,5 +460,33 @@ public class TestCapacityScheduler {
     }
     }
     return result;
     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.ApplicationTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
+import org.apache.hadoop.yarn.util.BuilderUtils;
 
 
 public class TestUtils {
 public class TestUtils {
   private static final Log LOG = LogFactory.getLog(TestUtils.class);
   private static final Log LOG = LogFactory.getLog(TestUtils.class);
@@ -136,9 +137,7 @@ public class TestUtils {
   
   
   public static ApplicationAttemptId 
   public static ApplicationAttemptId 
   getMockApplicationAttemptId(int appId, int attemptId) {
   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);  
     ApplicationAttemptId applicationAttemptId = mock(ApplicationAttemptId.class);  
     when(applicationAttemptId.getApplicationId()).thenReturn(applicationId);
     when(applicationAttemptId.getApplicationId()).thenReturn(applicationId);
     when(applicationAttemptId.getAttemptId()).thenReturn(attemptId);
     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.
   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, 
   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 
   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) 
   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 
   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>>>
  * <<<yarn.scheduler.fair.sizebasedweight>>>
   
   
     * Whether to assign shares to individual apps based on their size, rather than
     * 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>>>
  * <<<yarn.scheduler.fair.assignmultiple>>>
 
 
@@ -180,16 +183,29 @@ Allocation file format
  * <<Queue elements>>, which represent queues. Each may contain the following
  * <<Queue elements>>, which represent queues. Each may contain the following
      properties:
      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
    * 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
    * 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)
    * 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
      value of "*" means that any users can submit apps. A queue inherits the ACL of