浏览代码

YARN-6608. Backport all SLS improvements from trunk to branch-2. (Carlo Curino via wangda)

Wangda Tan 7 年之前
父节点
当前提交
d894f910d0
共有 59 个文件被更改,包括 5172 次插入3101 次删除
  1. 10 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/source/JvmMetrics.java
  2. 26 3
      hadoop-tools/hadoop-rumen/src/main/java/org/apache/hadoop/tools/rumen/TaskInfo.java
  3. 47 1
      hadoop-tools/hadoop-sls/pom.xml
  4. 3 1
      hadoop-tools/hadoop-sls/src/main/assemblies/sls.xml
  5. 25 6
      hadoop-tools/hadoop-sls/src/main/bin/slsrun.sh
  6. 24 681
      hadoop-tools/hadoop-sls/src/main/data/2jobs2min-rumen-jh.json
  7. 1 1
      hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/jquery.js
  8. 78 0
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/ReservationClientUtil.java
  9. 4 4
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/RumenToSLSConverter.java
  10. 649 283
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java
  11. 164 95
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/AMSimulator.java
  12. 99 131
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/MRAMSimulator.java
  13. 15 0
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/conf/SLSConfiguration.java
  14. 0 1
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NMSimulator.java
  15. 120 0
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/resourcemanager/MockAMLauncher.java
  16. 142 186
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/FairSchedulerMetrics.java
  17. 0 973
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java
  18. 130 550
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java
  19. 346 0
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSFairScheduler.java
  20. 571 34
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerMetrics.java
  21. 8 19
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerWrapper.java
  22. 4 5
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/TaskRunner.java
  23. 46 0
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/Tracker.java
  24. 306 0
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthJob.java
  25. 180 0
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthJobClass.java
  26. 319 0
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthTraceJobProducer.java
  27. 101 0
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthUtils.java
  28. 121 0
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthWorkload.java
  29. 22 0
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/package-info.java
  30. 47 14
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/utils/SLSUtils.java
  31. 22 23
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/web/SLSWebApp.java
  32. 135 16
      hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
  33. 151 0
      hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/BaseSLSRunnerTest.java
  34. 77 0
      hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestReservationSystemInvariants.java
  35. 57 38
      hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSLSRunner.java
  36. 96 0
      hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSynthJobGeneration.java
  37. 79 10
      hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/appmaster/TestAMSimulator.java
  38. 28 4
      hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/nodemanager/TestNMSimulator.java
  39. 1 1
      hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/scheduler/TestTaskRunner.java
  40. 30 0
      hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/utils/TestSLSUtils.java
  41. 14 14
      hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/web/TestSLSWebApp.java
  42. 10 0
      hadoop-tools/hadoop-sls/src/test/resources/capacity-scheduler.xml
  43. 8 0
      hadoop-tools/hadoop-sls/src/test/resources/exit-invariants.txt
  44. 5 3
      hadoop-tools/hadoop-sls/src/test/resources/fair-scheduler.xml
  45. 55 0
      hadoop-tools/hadoop-sls/src/test/resources/inputsls.json
  46. 19 0
      hadoop-tools/hadoop-sls/src/test/resources/log4j.properties
  47. 84 0
      hadoop-tools/hadoop-sls/src/test/resources/nodes.json
  48. 54 0
      hadoop-tools/hadoop-sls/src/test/resources/ongoing-invariants.txt
  49. 3 3
      hadoop-tools/hadoop-sls/src/test/resources/sls-runner.xml
  50. 53 0
      hadoop-tools/hadoop-sls/src/test/resources/syn.json
  51. 18 1
      hadoop-tools/hadoop-sls/src/test/resources/yarn-site.xml
  52. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
  53. 35 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/InvariantViolationException.java
  54. 96 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/InvariantsChecker.java
  55. 195 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/MetricsInvariantChecker.java
  56. 63 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/ReservationInvariantsChecker.java
  57. 22 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/package-info.java
  58. 99 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/TestMetricsInvariantChecker.java
  59. 54 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/invariants.txt

+ 10 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/source/JvmMetrics.java

@@ -27,6 +27,7 @@ import java.lang.management.GarbageCollectorMXBean;
 import java.util.List;
 import java.util.concurrent.ConcurrentHashMap;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.log.metrics.EventCounter;
 import org.apache.hadoop.metrics2.MetricsCollector;
@@ -59,6 +60,15 @@ public class JvmMetrics implements MetricsSource {
     }
   }
 
+  @VisibleForTesting
+  public synchronized void registerIfNeeded(){
+    // during tests impl might exist, but is not registered
+    MetricsSystem ms = DefaultMetricsSystem.instance();
+    if (ms.getSource("JvmMetrics") == null) {
+      ms.register(JvmMetrics.name(), JvmMetrics.description(), this);
+    }
+  }
+
   static final float M = 1024*1024;
   static public final float MEMORY_MAX_UNLIMITED_MB = -1;
 

+ 26 - 3
hadoop-tools/hadoop-rumen/src/main/java/org/apache/hadoop/tools/rumen/TaskInfo.java

@@ -23,21 +23,37 @@ public class TaskInfo {
   private final long bytesOut;
   private final int recsOut;
   private final long maxMemory;
+  private final long maxVcores;
   private final ResourceUsageMetrics metrics;
 
+
+  public TaskInfo(long bytesIn, int recsIn, long bytesOut, int recsOut,
+                  long maxMemory) {
+    this(bytesIn, recsIn, bytesOut, recsOut, maxMemory, 1,
+        new ResourceUsageMetrics());
+  }
+
   public TaskInfo(long bytesIn, int recsIn, long bytesOut, int recsOut,
-      long maxMemory) {
-    this(bytesIn, recsIn, bytesOut, recsOut, maxMemory, 
+                  long maxMemory, ResourceUsageMetrics
+                      metrics) {
+    this(bytesIn, recsIn, bytesOut, recsOut, maxMemory, 1, metrics);
+  }
+
+  public TaskInfo(long bytesIn, int recsIn, long bytesOut, int recsOut,
+      long maxMemory, long maxVcores) {
+    this(bytesIn, recsIn, bytesOut, recsOut, maxMemory, maxVcores,
          new ResourceUsageMetrics());
   }
   
   public TaskInfo(long bytesIn, int recsIn, long bytesOut, int recsOut,
-                  long maxMemory, ResourceUsageMetrics metrics) {
+                  long maxMemory, long maxVcores, ResourceUsageMetrics
+                      metrics) {
     this.bytesIn = bytesIn;
     this.recsIn = recsIn;
     this.bytesOut = bytesOut;
     this.recsOut = recsOut;
     this.maxMemory = maxMemory;
+    this.maxVcores = maxVcores;
     this.metrics = metrics;
   }
 
@@ -78,6 +94,13 @@ public class TaskInfo {
     return maxMemory;
   }
 
+  /**
+   * @return Vcores used by the task.
+   */
+  public long getTaskVCores() {
+    return maxVcores;
+  }
+
   /**
    * @return Resource usage metrics
    */

+ 47 - 1
hadoop-tools/hadoop-sls/pom.xml

@@ -22,7 +22,6 @@
     <version>2.9.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
-  <groupId>org.apache.hadoop</groupId>
   <artifactId>hadoop-sls</artifactId>
   <version>2.9.0-SNAPSHOT</version>
   <description>Apache Hadoop Scheduler Load Simulator</description>
@@ -70,9 +69,34 @@
       <artifactId>jetty-util</artifactId>
       <scope>provided</scope>
     </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-lang3</artifactId>
+      <version>3.4</version>
+    </dependency>
   </dependencies>
 
   <build>
+    <resources>
+      <resource>
+      <directory>src/main/</directory>
+      <includes>
+        <include>html/simulate.html.template</include>
+        <include>html/simulate.info.html.template</include>
+        <include>html/track.html.template</include>
+        <include>html/css/bootstrap-responsive.min.css</include>
+        <include>html/css/bootstrap.min.css</include>
+        <include>html/js/thirdparty/bootstrap.min.js</include>
+        <include>html/js/thirdparty/d3.v3.js</include>
+        <include>html/js/thirdparty/d3-LICENSE</include>
+        <include>html/js/thirdparty/jquery.js</include>
+      </includes>
+      </resource>
+    </resources>
     <plugins>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
@@ -113,6 +137,11 @@
             <exclude>src/test/resources/simulate.html.template</exclude>
             <exclude>src/test/resources/simulate.info.html.template</exclude>
             <exclude>src/test/resources/track.html.template</exclude>
+            <exclude>src/test/resources/syn.json</exclude>
+            <exclude>src/test/resources/inputsls.json</exclude>
+            <exclude>src/test/resources/nodes.json</exclude>
+            <exclude>src/test/resources/exit-invariants.txt</exclude>
+            <exclude>src/test/resources/ongoing-invariants.txt</exclude>
           </excludes>
         </configuration>
       </plugin>
@@ -177,6 +206,23 @@
               </execution>
             </executions>
           </plugin>
+       <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>deplist</id>
+            <phase>compile</phase>
+            <goals>
+              <goal>list</goal>
+            </goals>
+            <configuration>
+              <!-- referenced by a built-in command -->
+              <outputFile>${project.basedir}/target/hadoop-tools-deps/${project.artifactId}.tools-builtin.txt</outputFile>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
         </plugins>
       </build>
     </profile>

+ 3 - 1
hadoop-tools/hadoop-sls/src/main/assemblies/sls.xml

@@ -15,7 +15,9 @@
   See the License for the specific language governing permissions and
   limitations under the License.
 -->
-<assembly>
+<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.3"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.3 http://maven.apache.org/xsd/assembly-1.1.3.xsd">
   <id>sls</id>
   <formats>
     <format>dir</format>

+ 25 - 6
hadoop-tools/hadoop-sls/src/main/bin/slsrun.sh

@@ -16,7 +16,9 @@
 ###############################################################################
 printUsage() {
   echo "Usage: slsrun.sh <OPTIONS>"
-  echo "                 --input-rumen|--input-sls=<FILE1,FILE2,...>"
+  echo "                 --tracetype=<SYNTH | SLS | RUMEN>"
+  echo "                 --tracelocation=<FILE1,FILE2,...>"
+  echo "                 (deprecated --input-rumen=<FILE1,FILE2,...>  | --input-sls=<FILE1,FILE2,...>)"
   echo "                 --output-dir=<SLS_SIMULATION_OUTPUT_DIRECTORY>"
   echo "                 [--nodes=<SLS_NODES_FILE>]"
   echo "                 [--track-jobs=<JOBID1,JOBID2,...>]"
@@ -28,6 +30,12 @@ parseArgs() {
   for i in $*
   do
     case $i in
+    --tracetype=*)
+      tracetype=${i#*=}
+      ;;
+    --tracelocation=*)
+      tracelocation=${i#*=}
+      ;;
     --input-rumen=*)
       inputrumen=${i#*=}
       ;;
@@ -55,8 +63,8 @@ parseArgs() {
     esac
   done
 
-  if [[ "${inputrumen}" == "" && "${inputsls}" == "" ]] ; then
-    echo "Either --input-rumen or --input-sls must be specified"
+  if [[ "${inputrumen}" == "" && "${inputsls}" == "" && "${tracetype}" == "" ]] ; then
+    echo "Either --input-rumen or --input-sls or --tracetype must be specified"
     echo
     printUsage
     exit 1
@@ -82,12 +90,23 @@ calculateClasspath() {
 }
 ###############################################################################
 runSimulation() {
-  if [[ "${inputsls}" == "" ]] ; then
-    args="-inputrumen ${inputrumen}"
-  else
+  if [[ "${tracetype}" != "" ]] ; then
+    args="${args} -tracetype ${tracetype}"
+    args="${args} -tracelocation ${tracelocation}"
+  fi
+
+  if [[ "${nodes}" != "" ]] ; then
+    args="${args} -nodes ${nodes}"
+  fi
+
+  if [[ "${inputsls}" != "" ]] ; then
     args="-inputsls ${inputsls}"
   fi
 
+  if [[ "${inputrumen}" != "" ]] ; then
+    args="-inputrumen ${inputrumen}"
+  fi
+
   args="${args} -output ${outputdir}"
 
   if [[ "${nodes}" != "" ]] ; then

+ 24 - 681
hadoop-tools/hadoop-sls/src/main/data/2jobs2min-rumen-jh.json

@@ -4545,22 +4545,19 @@
     "hadoop.ssl.keystores.factory.class" : "org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory",
     "hadoop.http.authentication.kerberos.keytab" : "${user.home}/hadoop.keytab",
     "yarn.nodemanager.keytab" : "/etc/krb5.keytab",
-    "s3.blocksize" : "67108864",
     "mapreduce.task.io.sort.factor" : "10",
     "yarn.nodemanager.disk-health-checker.interval-ms" : "120000",
-    "mapreduce.job.working.dir" : "hdfs://a2115.smile.com:8020/user/jenkins",
+    "mapreduce.job.working.dir" : "hdfs://a2115.smile.com:9820/user/jenkins",
     "yarn.admin.acl" : "*",
     "mapreduce.job.speculative.speculativecap" : "0.1",
     "dfs.namenode.num.checkpoints.retained" : "2",
     "dfs.namenode.delegation.token.renew-interval" : "86400000",
     "yarn.nodemanager.resource.memory-mb" : "8192",
     "io.map.index.interval" : "128",
-    "s3.client-write-packet-size" : "65536",
     "mapreduce.task.files.preserve.failedtasks" : "false",
     "dfs.namenode.http-address" : "a2115.smile.com:20101",
     "ha.zookeeper.session-timeout.ms" : "5000",
     "hadoop.hdfs.configuration.version" : "1",
-    "s3.replication" : "3",
     "dfs.datanode.balance.bandwidthPerSec" : "1048576",
     "mapreduce.reduce.shuffle.connect.timeout" : "180000",
     "hadoop.ssl.enabled" : "false",
@@ -4584,10 +4581,9 @@
     "dfs.namenode.edits.dir" : "${dfs.namenode.name.dir}",
     "ha.health-monitor.sleep-after-disconnect.ms" : "1000",
     "dfs.encrypt.data.transfer" : "false",
-    "dfs.datanode.http.address" : "0.0.0.0:50075",
+    "dfs.datanode.http.address" : "0.0.0.0:9864",
     "mapreduce.terasort.num-rows" : "400000000",
     "mapreduce.job.map.class" : "org.apache.hadoop.examples.terasort.TeraGen$SortGenMapper",
-    "mapreduce.jobtracker.jobhistory.task.numberprogresssplits" : "12",
     "dfs.namenode.write.stale.datanode.ratio" : "0.5f",
     "dfs.client.use.datanode.hostname" : "false",
     "yarn.acl.enable" : "true",
@@ -4604,7 +4600,6 @@
     "mapreduce.reduce.log.level" : "INFO",
     "yarn.log-aggregation-enable" : "false",
     "dfs.datanode.sync.behind.writes" : "false",
-    "mapreduce.jobtracker.instrumentation" : "org.apache.hadoop.mapred.JobTrackerMetricsInst",
     "dfs.https.server.keystore.resource" : "ssl-server.xml",
     "hadoop.security.group.mapping.ldap.search.attr.group.name" : "cn",
     "dfs.namenode.replication.min" : "1",
@@ -4627,7 +4622,6 @@
     "hadoop.ssl.client.conf" : "ssl-client.xml",
     "dfs.namenode.safemode.threshold-pct" : "0.999f",
     "mapreduce.tasktracker.local.dir.minspacekill" : "0",
-    "mapreduce.jobtracker.retiredjobs.cache.size" : "1000",
     "dfs.blocksize" : "134217728",
     "yarn.resourcemanager.scheduler.class" : "org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler",
     "mapreduce.job.reduce.slowstart.completedmaps" : "0.8",
@@ -4637,14 +4631,12 @@
     "mapreduce.job.user.name" : "jenkins",
     "mapreduce.tasktracker.outofband.heartbeat" : "false",
     "io.native.lib.available" : "true",
-    "mapreduce.jobtracker.persist.jobstatus.hours" : "0",
     "dfs.client-write-packet-size" : "65536",
     "mapreduce.client.progressmonitor.pollinterval" : "1000",
     "dfs.namenode.name.dir" : "file://${hadoop.tmp.dir}/dfs/name",
     "dfs.ha.log-roll.period" : "120",
     "mapreduce.reduce.input.buffer.percent" : "0.0",
     "mapreduce.map.output.compress.codec" : "org.apache.hadoop.io.compress.SnappyCodec",
-    "mapreduce.map.skip.proc-count.auto-incr" : "true",
     "dfs.client.failover.sleep.base.millis" : "500",
     "dfs.datanode.directoryscan.threads" : "1",
     "mapreduce.jobtracker.address" : "neededForHive:999999",
@@ -4656,13 +4648,12 @@
     "yarn.scheduler.fair.preemption" : "true",
     "mapreduce.reduce.shuffle.parallelcopies" : "5",
     "yarn.nodemanager.env-whitelist" : "JAVA_HOME,HADOOP_COMMON_HOME,HADOOP_HDFS_HOME,HADOOP_CONF_DIR,YARN_HOME",
-    "mapreduce.jobtracker.heartbeats.in.second" : "100",
     "mapreduce.job.maxtaskfailures.per.tracker" : "3",
     "ipc.client.connection.maxidletime" : "10000",
     "mapreduce.shuffle.ssl.enabled" : "false",
     "dfs.namenode.invalidate.work.pct.per.iteration" : "0.32f",
     "dfs.blockreport.intervalMsec" : "21600000",
-    "fs.s3.sleepTimeSeconds" : "10",
+    "fs.s3n.sleepTimeSeconds" : "10",
     "dfs.namenode.replication.considerLoad" : "true",
     "dfs.client.block.write.retries" : "3",
     "hadoop.ssl.server.conf" : "ssl-server.xml",
@@ -4675,8 +4666,6 @@
     "dfs.replication" : "3",
     "ipc.client.tcpnodelay" : "false",
     "dfs.namenode.accesstime.precision" : "3600000",
-    "s3.stream-buffer-size" : "4096",
-    "mapreduce.jobtracker.tasktracker.maxblacklists" : "4",
     "dfs.client.read.shortcircuit.skip.checksum" : "false",
     "mapreduce.job.jvm.numtasks" : "1",
     "mapreduce.task.io.sort.mb" : "100",
@@ -4693,14 +4682,12 @@
     "kfs.stream-buffer-size" : "4096",
     "dfs.ha.tail-edits.period" : "60",
     "hadoop.security.authentication" : "simple",
-    "fs.s3.buffer.dir" : "${hadoop.tmp.dir}/s3",
+    "fs.s3n.buffer.dir" : "${hadoop.tmp.dir}/s3n",
     "rpc.engine.org.apache.hadoop.yarn.api.AMRMProtocolPB" : "org.apache.hadoop.ipc.ProtobufRpcEngine",
-    "mapreduce.jobtracker.taskscheduler" : "org.apache.hadoop.mapred.JobQueueTaskScheduler",
     "yarn.app.mapreduce.am.job.task.listener.thread-count" : "30",
     "dfs.namenode.avoid.read.stale.datanode" : "false",
     "mapreduce.job.reduces" : "0",
     "mapreduce.map.sort.spill.percent" : "0.8",
-    "dfs.client.file-block-storage-locations.timeout" : "60",
     "dfs.datanode.drop.cache.behind.writes" : "false",
     "mapreduce.job.end-notification.retry.interval" : "1",
     "mapreduce.job.maps" : "96",
@@ -4722,7 +4709,6 @@
     "dfs.namenode.replication.interval" : "3",
     "mapreduce.task.skip.start.attempts" : "2",
     "dfs.namenode.https-address" : "a2115.smile.com:20102",
-    "mapreduce.jobtracker.persist.jobstatus.dir" : "/jobtracker/jobsInfo",
     "ipc.client.kill.max" : "10",
     "dfs.ha.automatic-failover.enabled" : "false",
     "mapreduce.jobhistory.keytab" : "/etc/security/keytab/jhs.service.keytab",
@@ -4745,11 +4731,9 @@
     "io.mapfile.bloom.size" : "1048576",
     "yarn.nodemanager.container-executor.class" : "org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor",
     "mapreduce.map.maxattempts" : "4",
-    "mapreduce.jobtracker.jobhistory.block.size" : "3145728",
     "yarn.log-aggregation.retain-seconds" : "-1",
     "yarn.app.mapreduce.am.job.committer.cancel-timeout" : "60000",
     "ftp.replication" : "3",
-    "mapreduce.jobtracker.http.address" : "0.0.0.0:50030",
     "yarn.nodemanager.health-checker.script.timeout-ms" : "1200000",
     "mapreduce.jobhistory.address" : "a2115.smile.com:10020",
     "mapreduce.jobtracker.taskcache.levels" : "2",
@@ -4763,7 +4747,6 @@
     "dfs.namenode.backup.address" : "0.0.0.0:50100",
     "hadoop.util.hash.type" : "murmur",
     "dfs.block.access.key.update.interval" : "600",
-    "mapreduce.reduce.skip.proc-count.auto-incr" : "true",
     "dfs.datanode.dns.interface" : "default",
     "dfs.datanode.use.datanode.hostname" : "false",
     "mapreduce.job.output.key.class" : "org.apache.hadoop.io.Text",
@@ -4777,7 +4760,7 @@
     "mapreduce.job.dir" : "/user/jenkins/.staging/job_1369942127770_1205",
     "io.map.index.skip" : "0",
     "net.topology.node.switch.mapping.impl" : "org.apache.hadoop.net.ScriptBasedMapping",
-    "fs.s3.maxRetries" : "4",
+    "fs.s3n.maxRetries" : "4",
     "ha.failover-controller.new-active.rpc-timeout.ms" : "60000",
     "s3native.client-write-packet-size" : "65536",
     "yarn.resourcemanager.amliveliness-monitor.interval-ms" : "1000",
@@ -4800,7 +4783,6 @@
     "dfs.datanode.directoryscan.interval" : "21600",
     "yarn.resourcemanager.address" : "a2115.smile.com:8032",
     "yarn.nodemanager.health-checker.interval-ms" : "600000",
-    "dfs.client.file-block-storage-locations.num-threads" : "10",
     "yarn.resourcemanager.container-tokens.master-key-rolling-interval-secs" : "86400",
     "mapreduce.reduce.markreset.buffer.percent" : "0.0",
     "hadoop.security.group.mapping.ldap.directory.search.timeout" : "10000",
@@ -4813,7 +4795,7 @@
     "ftp.stream-buffer-size" : "4096",
     "dfs.namenode.avoid.write.stale.datanode" : "false",
     "hadoop.security.group.mapping.ldap.search.attr.member" : "member",
-    "mapreduce.output.fileoutputformat.outputdir" : "hdfs://a2115.smile.com:8020/user/jenkins/tera-gen-1",
+    "mapreduce.output.fileoutputformat.outputdir" : "hdfs://a2115.smile.com:9820/user/jenkins/tera-gen-1",
     "dfs.blockreport.initialDelay" : "0",
     "yarn.nm.liveness-monitor.expiry-interval-ms" : "600000",
     "hadoop.http.authentication.token.validity" : "36000",
@@ -4839,18 +4821,16 @@
     "yarn.scheduler.maximum-allocation-mb" : "8192",
     "yarn.nodemanager.heartbeat.interval-ms" : "1000",
     "mapreduce.job.userlog.retain.hours" : "24",
-    "dfs.namenode.secondary.http-address" : "0.0.0.0:50090",
+    "dfs.namenode.secondary.http-address" : "0.0.0.0:9868",
     "mapreduce.task.timeout" : "600000",
     "mapreduce.framework.name" : "yarn",
     "ipc.client.idlethreshold" : "4000",
     "ftp.bytes-per-checksum" : "512",
     "ipc.server.tcpnodelay" : "false",
     "dfs.namenode.stale.datanode.interval" : "30000",
-    "s3.bytes-per-checksum" : "512",
     "mapreduce.job.speculative.slowtaskthreshold" : "1.0",
     "yarn.nodemanager.localizer.cache.target-size-mb" : "10240",
     "yarn.nodemanager.remote-app-log-dir" : "/tmp/logs",
-    "fs.s3.block.size" : "67108864",
     "mapreduce.job.queuename" : "sls_queue_1",
     "dfs.client.failover.connection.retries" : "0",
     "hadoop.rpc.protection" : "authentication",
@@ -4859,7 +4839,7 @@
     "hadoop.security.auth_to_local" : "DEFAULT",
     "dfs.secondary.namenode.kerberos.internal.spnego.principal" : "${dfs.web.authentication.kerberos.principal}",
     "ftp.client-write-packet-size" : "65536",
-    "fs.defaultFS" : "hdfs://a2115.smile.com:8020",
+    "fs.defaultFS" : "hdfs://a2115.smile.com:9820",
     "yarn.nodemanager.address" : "0.0.0.0:0",
     "yarn.scheduler.fair.assignmultiple" : "true",
     "yarn.resourcemanager.scheduler.client.thread-count" : "50",
@@ -4873,7 +4853,6 @@
     "mapreduce.reduce.shuffle.read.timeout" : "180000",
     "mapreduce.admin.user.env" : "LD_LIBRARY_PATH=$HADOOP_COMMON_HOME/lib/native",
     "yarn.app.mapreduce.am.command-opts" : " -Xmx1238932873",
-    "mapreduce.local.clientfactory.class.name" : "org.apache.hadoop.mapred.LocalClientFactory",
     "dfs.namenode.checkpoint.edits.dir" : "${dfs.namenode.checkpoint.dir}",
     "fs.permissions.umask-mode" : "022",
     "dfs.client.domain.socket.data.traffic" : "false",
@@ -4904,15 +4883,14 @@
     "fs.s3n.block.size" : "67108864",
     "fs.ftp.host" : "0.0.0.0",
     "hadoop.security.group.mapping" : "org.apache.hadoop.security.JniBasedUnixGroupsMappingWithFallback",
-    "dfs.datanode.address" : "0.0.0.0:50010",
+    "dfs.datanode.address" : "0.0.0.0:9866",
     "mapreduce.map.skip.maxrecords" : "0",
-    "dfs.datanode.https.address" : "0.0.0.0:50475",
+    "dfs.datanode.https.address" : "0.0.0.0:9865",
     "file.replication" : "1",
     "yarn.resourcemanager.resource-tracker.address" : "a2115.smile.com:8031",
     "dfs.datanode.drop.cache.behind.reads" : "false",
     "hadoop.fuse.connection.timeout" : "300",
     "hadoop.work.around.non.threadsafe.getpwuid" : "false",
-    "mapreduce.jobtracker.restart.recover" : "false",
     "hadoop.tmp.dir" : "/tmp/hadoop-${user.name}",
     "mapreduce.output.fileoutputformat.compress" : "false",
     "mapreduce.tasktracker.indexcache.mb" : "10",
@@ -4922,10 +4900,8 @@
     "hadoop.kerberos.kinit.command" : "kinit",
     "dfs.datanode.du.reserved" : "0",
     "dfs.namenode.fs-limits.max-blocks-per-file" : "1048576",
-    "dfs.webhdfs.enabled" : "false",
     "file.bytes-per-checksum" : "512",
     "mapreduce.task.profile.reduces" : "0-2",
-    "mapreduce.jobtracker.handler.count" : "10",
     "dfs.client.block.write.replace-datanode-on-failure.enable" : "true",
     "mapreduce.job.output.value.class" : "org.apache.hadoop.io.Text",
     "yarn.dispatcher.exit-on-error" : "true",
@@ -4943,7 +4919,6 @@
     "mapreduce.shuffle.ssl.file.buffer.size" : "65536",
     "dfs.permissions.superusergroup" : "supergroup",
     "dfs.datanode.available-space-volume-choosing-policy.balanced-space-threshold" : "10737418240",
-    "mapreduce.jobtracker.expire.trackers.interval" : "600000",
     "mapreduce.cluster.acls.enabled" : "false",
     "yarn.nodemanager.remote-app-log-dir-suffix" : "logs",
     "ha.failover-controller.graceful-fence.connection.retries" : "1",
@@ -4953,8 +4928,6 @@
     "io.seqfile.local.dir" : "${hadoop.tmp.dir}/io/local",
     "dfs.namenode.checkpoint.check.period" : "60",
     "yarn.resourcemanager.nm.liveness-monitor.interval-ms" : "1000",
-    "mapreduce.jobtracker.maxtasks.perjob" : "-1",
-    "mapreduce.jobtracker.jobhistory.lru.cache.size" : "5",
     "file.blocksize" : "67108864",
     "tfile.io.chunk.size" : "1048576",
     "mapreduce.job.acl-modify-job" : " ",
@@ -4974,7 +4947,7 @@
     "yarn.resourcemanager.resource-tracker.client.thread-count" : "50",
     "mapreduce.tasktracker.dns.nameserver" : "default",
     "mapreduce.map.output.compress" : "true",
-    "dfs.datanode.ipc.address" : "0.0.0.0:50020",
+    "dfs.datanode.ipc.address" : "0.0.0.0:9867",
     "hadoop.ssl.require.client.cert" : "false",
     "yarn.nodemanager.delete.debug-delay-sec" : "0",
     "dfs.datanode.max.transfer.threads" : "4096"
@@ -5105,7 +5078,8 @@
   "clusterReduceMB" : -1,
   "jobMapMB" : 200,
   "jobReduceMB" : 200
-} {
+}
+{
   "priority" : "NORMAL",
   "jobID" : "job_1369942127770_1206",
   "user" : "jenkins",
@@ -9652,22 +9626,19 @@
     "hadoop.ssl.keystores.factory.class" : "org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory",
     "hadoop.http.authentication.kerberos.keytab" : "${user.home}/hadoop.keytab",
     "yarn.nodemanager.keytab" : "/etc/krb5.keytab",
-    "s3.blocksize" : "67108864",
     "mapreduce.task.io.sort.factor" : "10",
     "yarn.nodemanager.disk-health-checker.interval-ms" : "120000",
-    "mapreduce.job.working.dir" : "hdfs://a2115.smile.com:8020/user/jenkins",
+    "mapreduce.job.working.dir" : "hdfs://a2115.smile.com:9820/user/jenkins",
     "yarn.admin.acl" : "*",
     "mapreduce.job.speculative.speculativecap" : "0.1",
     "dfs.namenode.num.checkpoints.retained" : "2",
     "dfs.namenode.delegation.token.renew-interval" : "86400000",
     "yarn.nodemanager.resource.memory-mb" : "8192",
     "io.map.index.interval" : "128",
-    "s3.client-write-packet-size" : "65536",
     "mapreduce.task.files.preserve.failedtasks" : "false",
     "dfs.namenode.http-address" : "a2115.smile.com:20101",
     "ha.zookeeper.session-timeout.ms" : "5000",
     "hadoop.hdfs.configuration.version" : "1",
-    "s3.replication" : "3",
     "dfs.datanode.balance.bandwidthPerSec" : "1048576",
     "mapreduce.reduce.shuffle.connect.timeout" : "180000",
     "hadoop.ssl.enabled" : "false",
@@ -9691,10 +9662,9 @@
     "dfs.namenode.edits.dir" : "${dfs.namenode.name.dir}",
     "ha.health-monitor.sleep-after-disconnect.ms" : "1000",
     "dfs.encrypt.data.transfer" : "false",
-    "dfs.datanode.http.address" : "0.0.0.0:50075",
+    "dfs.datanode.http.address" : "0.0.0.0:9864",
     "mapreduce.terasort.num-rows" : "400000000",
     "mapreduce.job.map.class" : "org.apache.hadoop.examples.terasort.TeraGen$SortGenMapper",
-    "mapreduce.jobtracker.jobhistory.task.numberprogresssplits" : "12",
     "dfs.namenode.write.stale.datanode.ratio" : "0.5f",
     "dfs.client.use.datanode.hostname" : "false",
     "yarn.acl.enable" : "true",
@@ -9711,7 +9681,6 @@
     "mapreduce.reduce.log.level" : "INFO",
     "yarn.log-aggregation-enable" : "false",
     "dfs.datanode.sync.behind.writes" : "false",
-    "mapreduce.jobtracker.instrumentation" : "org.apache.hadoop.mapred.JobTrackerMetricsInst",
     "dfs.https.server.keystore.resource" : "ssl-server.xml",
     "hadoop.security.group.mapping.ldap.search.attr.group.name" : "cn",
     "dfs.namenode.replication.min" : "1",
@@ -9734,7 +9703,6 @@
     "hadoop.ssl.client.conf" : "ssl-client.xml",
     "dfs.namenode.safemode.threshold-pct" : "0.999f",
     "mapreduce.tasktracker.local.dir.minspacekill" : "0",
-    "mapreduce.jobtracker.retiredjobs.cache.size" : "1000",
     "dfs.blocksize" : "134217728",
     "yarn.resourcemanager.scheduler.class" : "org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler",
     "mapreduce.job.reduce.slowstart.completedmaps" : "0.8",
@@ -9744,14 +9712,12 @@
     "mapreduce.job.user.name" : "jenkins",
     "mapreduce.tasktracker.outofband.heartbeat" : "false",
     "io.native.lib.available" : "true",
-    "mapreduce.jobtracker.persist.jobstatus.hours" : "0",
     "dfs.client-write-packet-size" : "65536",
     "mapreduce.client.progressmonitor.pollinterval" : "1000",
     "dfs.namenode.name.dir" : "file://${hadoop.tmp.dir}/dfs/name",
     "dfs.ha.log-roll.period" : "120",
     "mapreduce.reduce.input.buffer.percent" : "0.0",
     "mapreduce.map.output.compress.codec" : "org.apache.hadoop.io.compress.SnappyCodec",
-    "mapreduce.map.skip.proc-count.auto-incr" : "true",
     "dfs.client.failover.sleep.base.millis" : "500",
     "dfs.datanode.directoryscan.threads" : "1",
     "mapreduce.jobtracker.address" : "neededForHive:999999",
@@ -9763,13 +9729,12 @@
     "yarn.scheduler.fair.preemption" : "true",
     "mapreduce.reduce.shuffle.parallelcopies" : "5",
     "yarn.nodemanager.env-whitelist" : "JAVA_HOME,HADOOP_COMMON_HOME,HADOOP_HDFS_HOME,HADOOP_CONF_DIR,YARN_HOME",
-    "mapreduce.jobtracker.heartbeats.in.second" : "100",
     "mapreduce.job.maxtaskfailures.per.tracker" : "3",
     "ipc.client.connection.maxidletime" : "10000",
     "mapreduce.shuffle.ssl.enabled" : "false",
     "dfs.namenode.invalidate.work.pct.per.iteration" : "0.32f",
     "dfs.blockreport.intervalMsec" : "21600000",
-    "fs.s3.sleepTimeSeconds" : "10",
+    "fs.s3n.sleepTimeSeconds" : "10",
     "dfs.namenode.replication.considerLoad" : "true",
     "dfs.client.block.write.retries" : "3",
     "hadoop.ssl.server.conf" : "ssl-server.xml",
@@ -9782,8 +9747,6 @@
     "dfs.replication" : "3",
     "ipc.client.tcpnodelay" : "false",
     "dfs.namenode.accesstime.precision" : "3600000",
-    "s3.stream-buffer-size" : "4096",
-    "mapreduce.jobtracker.tasktracker.maxblacklists" : "4",
     "dfs.client.read.shortcircuit.skip.checksum" : "false",
     "mapreduce.job.jvm.numtasks" : "1",
     "mapreduce.task.io.sort.mb" : "100",
@@ -9800,14 +9763,12 @@
     "kfs.stream-buffer-size" : "4096",
     "dfs.ha.tail-edits.period" : "60",
     "hadoop.security.authentication" : "simple",
-    "fs.s3.buffer.dir" : "${hadoop.tmp.dir}/s3",
+    "fs.s3n.buffer.dir" : "${hadoop.tmp.dir}/s3",
     "rpc.engine.org.apache.hadoop.yarn.api.AMRMProtocolPB" : "org.apache.hadoop.ipc.ProtobufRpcEngine",
-    "mapreduce.jobtracker.taskscheduler" : "org.apache.hadoop.mapred.JobQueueTaskScheduler",
     "yarn.app.mapreduce.am.job.task.listener.thread-count" : "30",
     "dfs.namenode.avoid.read.stale.datanode" : "false",
     "mapreduce.job.reduces" : "0",
     "mapreduce.map.sort.spill.percent" : "0.8",
-    "dfs.client.file-block-storage-locations.timeout" : "60",
     "dfs.datanode.drop.cache.behind.writes" : "false",
     "mapreduce.job.end-notification.retry.interval" : "1",
     "mapreduce.job.maps" : "96",
@@ -9829,7 +9790,6 @@
     "dfs.namenode.replication.interval" : "3",
     "mapreduce.task.skip.start.attempts" : "2",
     "dfs.namenode.https-address" : "a2115.smile.com:20102",
-    "mapreduce.jobtracker.persist.jobstatus.dir" : "/jobtracker/jobsInfo",
     "ipc.client.kill.max" : "10",
     "dfs.ha.automatic-failover.enabled" : "false",
     "mapreduce.jobhistory.keytab" : "/etc/security/keytab/jhs.service.keytab",
@@ -9852,11 +9812,9 @@
     "io.mapfile.bloom.size" : "1048576",
     "yarn.nodemanager.container-executor.class" : "org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor",
     "mapreduce.map.maxattempts" : "4",
-    "mapreduce.jobtracker.jobhistory.block.size" : "3145728",
     "yarn.log-aggregation.retain-seconds" : "-1",
     "yarn.app.mapreduce.am.job.committer.cancel-timeout" : "60000",
     "ftp.replication" : "3",
-    "mapreduce.jobtracker.http.address" : "0.0.0.0:50030",
     "yarn.nodemanager.health-checker.script.timeout-ms" : "1200000",
     "mapreduce.jobhistory.address" : "a2115.smile.com:10020",
     "mapreduce.jobtracker.taskcache.levels" : "2",
@@ -9870,7 +9828,6 @@
     "dfs.namenode.backup.address" : "0.0.0.0:50100",
     "hadoop.util.hash.type" : "murmur",
     "dfs.block.access.key.update.interval" : "600",
-    "mapreduce.reduce.skip.proc-count.auto-incr" : "true",
     "dfs.datanode.dns.interface" : "default",
     "dfs.datanode.use.datanode.hostname" : "false",
     "mapreduce.job.output.key.class" : "org.apache.hadoop.io.Text",
@@ -9884,7 +9841,7 @@
     "mapreduce.job.dir" : "/user/jenkins/.staging/job_1369942127770_1206",
     "io.map.index.skip" : "0",
     "net.topology.node.switch.mapping.impl" : "org.apache.hadoop.net.ScriptBasedMapping",
-    "fs.s3.maxRetries" : "4",
+    "fs.s3n.maxRetries" : "4",
     "ha.failover-controller.new-active.rpc-timeout.ms" : "60000",
     "s3native.client-write-packet-size" : "65536",
     "yarn.resourcemanager.amliveliness-monitor.interval-ms" : "1000",
@@ -9907,7 +9864,6 @@
     "dfs.datanode.directoryscan.interval" : "21600",
     "yarn.resourcemanager.address" : "a2115.smile.com:8032",
     "yarn.nodemanager.health-checker.interval-ms" : "600000",
-    "dfs.client.file-block-storage-locations.num-threads" : "10",
     "yarn.resourcemanager.container-tokens.master-key-rolling-interval-secs" : "86400",
     "mapreduce.reduce.markreset.buffer.percent" : "0.0",
     "hadoop.security.group.mapping.ldap.directory.search.timeout" : "10000",
@@ -9920,7 +9876,7 @@
     "ftp.stream-buffer-size" : "4096",
     "dfs.namenode.avoid.write.stale.datanode" : "false",
     "hadoop.security.group.mapping.ldap.search.attr.member" : "member",
-    "mapreduce.output.fileoutputformat.outputdir" : "hdfs://a2115.smile.com:8020/user/jenkins/tera-gen-2",
+    "mapreduce.output.fileoutputformat.outputdir" : "hdfs://a2115.smile.com:9820/user/jenkins/tera-gen-2",
     "dfs.blockreport.initialDelay" : "0",
     "yarn.nm.liveness-monitor.expiry-interval-ms" : "600000",
     "hadoop.http.authentication.token.validity" : "36000",
@@ -9946,18 +9902,16 @@
     "yarn.scheduler.maximum-allocation-mb" : "8192",
     "yarn.nodemanager.heartbeat.interval-ms" : "1000",
     "mapreduce.job.userlog.retain.hours" : "24",
-    "dfs.namenode.secondary.http-address" : "0.0.0.0:50090",
+    "dfs.namenode.secondary.http-address" : "0.0.0.0:9868",
     "mapreduce.task.timeout" : "600000",
     "mapreduce.framework.name" : "yarn",
     "ipc.client.idlethreshold" : "4000",
     "ftp.bytes-per-checksum" : "512",
     "ipc.server.tcpnodelay" : "false",
     "dfs.namenode.stale.datanode.interval" : "30000",
-    "s3.bytes-per-checksum" : "512",
     "mapreduce.job.speculative.slowtaskthreshold" : "1.0",
     "yarn.nodemanager.localizer.cache.target-size-mb" : "10240",
     "yarn.nodemanager.remote-app-log-dir" : "/tmp/logs",
-    "fs.s3.block.size" : "67108864",
     "mapreduce.job.queuename" : "sls_queue_1",
     "dfs.client.failover.connection.retries" : "0",
     "hadoop.rpc.protection" : "authentication",
@@ -9966,7 +9920,7 @@
     "hadoop.security.auth_to_local" : "DEFAULT",
     "dfs.secondary.namenode.kerberos.internal.spnego.principal" : "${dfs.web.authentication.kerberos.principal}",
     "ftp.client-write-packet-size" : "65536",
-    "fs.defaultFS" : "hdfs://a2115.smile.com:8020",
+    "fs.defaultFS" : "hdfs://a2115.smile.com:9820",
     "yarn.nodemanager.address" : "0.0.0.0:0",
     "yarn.scheduler.fair.assignmultiple" : "true",
     "yarn.resourcemanager.scheduler.client.thread-count" : "50",
@@ -9980,7 +9934,6 @@
     "mapreduce.reduce.shuffle.read.timeout" : "180000",
     "mapreduce.admin.user.env" : "LD_LIBRARY_PATH=$HADOOP_COMMON_HOME/lib/native",
     "yarn.app.mapreduce.am.command-opts" : " -Xmx1238932873",
-    "mapreduce.local.clientfactory.class.name" : "org.apache.hadoop.mapred.LocalClientFactory",
     "dfs.namenode.checkpoint.edits.dir" : "${dfs.namenode.checkpoint.dir}",
     "fs.permissions.umask-mode" : "022",
     "dfs.client.domain.socket.data.traffic" : "false",
@@ -10011,15 +9964,14 @@
     "fs.s3n.block.size" : "67108864",
     "fs.ftp.host" : "0.0.0.0",
     "hadoop.security.group.mapping" : "org.apache.hadoop.security.JniBasedUnixGroupsMappingWithFallback",
-    "dfs.datanode.address" : "0.0.0.0:50010",
+    "dfs.datanode.address" : "0.0.0.0:9866",
     "mapreduce.map.skip.maxrecords" : "0",
-    "dfs.datanode.https.address" : "0.0.0.0:50475",
+    "dfs.datanode.https.address" : "0.0.0.0:9865",
     "file.replication" : "1",
     "yarn.resourcemanager.resource-tracker.address" : "a2115.smile.com:8031",
     "dfs.datanode.drop.cache.behind.reads" : "false",
     "hadoop.fuse.connection.timeout" : "300",
     "hadoop.work.around.non.threadsafe.getpwuid" : "false",
-    "mapreduce.jobtracker.restart.recover" : "false",
     "hadoop.tmp.dir" : "/tmp/hadoop-${user.name}",
     "mapreduce.output.fileoutputformat.compress" : "false",
     "mapreduce.tasktracker.indexcache.mb" : "10",
@@ -10029,10 +9981,8 @@
     "hadoop.kerberos.kinit.command" : "kinit",
     "dfs.datanode.du.reserved" : "0",
     "dfs.namenode.fs-limits.max-blocks-per-file" : "1048576",
-    "dfs.webhdfs.enabled" : "false",
     "file.bytes-per-checksum" : "512",
     "mapreduce.task.profile.reduces" : "0-2",
-    "mapreduce.jobtracker.handler.count" : "10",
     "dfs.client.block.write.replace-datanode-on-failure.enable" : "true",
     "mapreduce.job.output.value.class" : "org.apache.hadoop.io.Text",
     "yarn.dispatcher.exit-on-error" : "true",
@@ -10050,7 +10000,6 @@
     "mapreduce.shuffle.ssl.file.buffer.size" : "65536",
     "dfs.permissions.superusergroup" : "supergroup",
     "dfs.datanode.available-space-volume-choosing-policy.balanced-space-threshold" : "10737418240",
-    "mapreduce.jobtracker.expire.trackers.interval" : "600000",
     "mapreduce.cluster.acls.enabled" : "false",
     "yarn.nodemanager.remote-app-log-dir-suffix" : "logs",
     "ha.failover-controller.graceful-fence.connection.retries" : "1",
@@ -10060,8 +10009,6 @@
     "io.seqfile.local.dir" : "${hadoop.tmp.dir}/io/local",
     "dfs.namenode.checkpoint.check.period" : "60",
     "yarn.resourcemanager.nm.liveness-monitor.interval-ms" : "1000",
-    "mapreduce.jobtracker.maxtasks.perjob" : "-1",
-    "mapreduce.jobtracker.jobhistory.lru.cache.size" : "5",
     "file.blocksize" : "67108864",
     "tfile.io.chunk.size" : "1048576",
     "mapreduce.job.acl-modify-job" : " ",
@@ -10081,7 +10028,7 @@
     "yarn.resourcemanager.resource-tracker.client.thread-count" : "50",
     "mapreduce.tasktracker.dns.nameserver" : "default",
     "mapreduce.map.output.compress" : "true",
-    "dfs.datanode.ipc.address" : "0.0.0.0:50020",
+    "dfs.datanode.ipc.address" : "0.0.0.0:9867",
     "hadoop.ssl.require.client.cert" : "false",
     "yarn.nodemanager.delete.debug-delay-sec" : "0",
     "dfs.datanode.max.transfer.threads" : "4096"
@@ -10212,608 +10159,4 @@
   "clusterReduceMB" : -1,
   "jobMapMB" : 200,
   "jobReduceMB" : 200
-} {
-"priority" : "NORMAL",
-"jobID" : "job_1369942127770_1207",
-"user" : "jenkins",
-"jobName" : "TeraGen",
-"submitTime" : 1371223054499,
-"finishTime" : 1371223153874,
-"queue" : "sls_queue_1",
-"mapTasks" : [ {
-"startTime" : 1371223059053,
-"taskID" : "task_1369942127770_1207_m_000000",
-"taskType" : "MAP",
-"finishTime" : 1371223078206,
-"attempts" : [ ],
-"preferredLocations" : [ ],
-"taskStatus" : "KILLED",
-"inputBytes" : -1,
-"inputRecords" : -1,
-"outputBytes" : -1,
-"outputRecords" : -1
-} ],
-"reduceTasks" : [ ],
-"launchTime" : 1371223058937,
-"totalMaps" : 1,
-"totalReduces" : 0,
-"otherTasks" : [ ],
-"jobProperties" : {
-"mapreduce.job.ubertask.enable" : "false",
-"yarn.resourcemanager.max-completed-applications" : "10000",
-"yarn.resourcemanager.delayed.delegation-token.removal-interval-ms" : "30000",
-"mapreduce.client.submit.file.replication" : "2",
-"yarn.nodemanager.container-manager.thread-count" : "20",
-"mapred.queue.default.acl-administer-jobs" : "*",
-"dfs.image.transfer.bandwidthPerSec" : "0",
-"mapreduce.tasktracker.healthchecker.interval" : "60000",
-"mapreduce.jobtracker.staging.root.dir" : "/user",
-"yarn.resourcemanager.recovery.enabled" : "false",
-"yarn.resourcemanager.am.max-retries" : "1",
-"dfs.block.access.token.lifetime" : "600",
-"fs.AbstractFileSystem.file.impl" : "org.apache.hadoop.fs.local.LocalFs",
-"mapreduce.client.completion.pollinterval" : "5000",
-"mapreduce.job.ubertask.maxreduces" : "1",
-"mapreduce.reduce.shuffle.memory.limit.percent" : "0.25",
-"dfs.domain.socket.path" : "/var/run/hdfs-sockets/dn",
-"hadoop.ssl.keystores.factory.class" : "org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory",
-"hadoop.http.authentication.kerberos.keytab" : "${user.home}/hadoop.keytab",
-"yarn.nodemanager.keytab" : "/etc/krb5.keytab",
-"s3.blocksize" : "67108864",
-"mapreduce.task.io.sort.factor" : "10",
-"yarn.nodemanager.disk-health-checker.interval-ms" : "120000",
-"mapreduce.job.working.dir" : "hdfs://a2115.smile.com:8020/user/jenkins",
-"yarn.admin.acl" : "*",
-"mapreduce.job.speculative.speculativecap" : "0.1",
-"dfs.namenode.num.checkpoints.retained" : "2",
-"dfs.namenode.delegation.token.renew-interval" : "86400000",
-"yarn.nodemanager.resource.memory-mb" : "8192",
-"io.map.index.interval" : "128",
-"s3.client-write-packet-size" : "65536",
-"mapreduce.task.files.preserve.failedtasks" : "false",
-"dfs.namenode.http-address" : "a2115.smile.com:20101",
-"ha.zookeeper.session-timeout.ms" : "5000",
-"hadoop.hdfs.configuration.version" : "1",
-"s3.replication" : "3",
-"dfs.datanode.balance.bandwidthPerSec" : "1048576",
-"mapreduce.reduce.shuffle.connect.timeout" : "180000",
-"hadoop.ssl.enabled" : "false",
-"dfs.journalnode.rpc-address" : "0.0.0.0:8485",
-"yarn.nodemanager.aux-services" : "mapreduce.shuffle",
-"mapreduce.job.counters.max" : "120",
-"dfs.datanode.readahead.bytes" : "4193404",
-"ipc.client.connect.max.retries.on.timeouts" : "45",
-"mapreduce.job.complete.cancel.delegation.tokens" : "true",
-"dfs.client.failover.max.attempts" : "15",
-"dfs.namenode.checkpoint.dir" : "file://${hadoop.tmp.dir}/dfs/namesecondary",
-"dfs.namenode.replication.work.multiplier.per.iteration" : "2",
-"fs.trash.interval" : "1",
-"yarn.resourcemanager.admin.address" : "a2115.smile.com:8033",
-"ha.health-monitor.check-interval.ms" : "1000",
-"mapreduce.job.outputformat.class" : "org.apache.hadoop.examples.terasort.TeraOutputFormat",
-"hadoop.jetty.logs.serve.aliases" : "true",
-"hadoop.http.authentication.kerberos.principal" : "HTTP/_HOST@LOCALHOST",
-"mapreduce.job.reduce.shuffle.consumer.plugin.class" : "org.apache.hadoop.mapreduce.task.reduce.Shuffle",
-"s3native.blocksize" : "67108864",
-"dfs.namenode.edits.dir" : "${dfs.namenode.name.dir}",
-"ha.health-monitor.sleep-after-disconnect.ms" : "1000",
-"dfs.encrypt.data.transfer" : "false",
-"dfs.datanode.http.address" : "0.0.0.0:50075",
-"mapreduce.terasort.num-rows" : "400000000",
-"mapreduce.job.map.class" : "org.apache.hadoop.examples.terasort.TeraGen$SortGenMapper",
-"mapreduce.jobtracker.jobhistory.task.numberprogresssplits" : "12",
-"dfs.namenode.write.stale.datanode.ratio" : "0.5f",
-"dfs.client.use.datanode.hostname" : "false",
-"yarn.acl.enable" : "true",
-"hadoop.security.instrumentation.requires.admin" : "false",
-"yarn.nodemanager.localizer.fetch.thread-count" : "4",
-"hadoop.security.authorization" : "false",
-"user.name" : "jenkins",
-"dfs.namenode.fs-limits.min-block-size" : "1048576",
-"dfs.client.failover.connection.retries.on.timeouts" : "0",
-"hadoop.security.group.mapping.ldap.search.filter.group" : "(objectClass=group)",
-"mapreduce.output.fileoutputformat.compress.codec" : "org.apache.hadoop.io.compress.DefaultCodec",
-"dfs.namenode.safemode.extension" : "30000",
-"mapreduce.shuffle.port" : "8080",
-"mapreduce.reduce.log.level" : "INFO",
-"yarn.log-aggregation-enable" : "false",
-"dfs.datanode.sync.behind.writes" : "false",
-"mapreduce.jobtracker.instrumentation" : "org.apache.hadoop.mapred.JobTrackerMetricsInst",
-"dfs.https.server.keystore.resource" : "ssl-server.xml",
-"hadoop.security.group.mapping.ldap.search.attr.group.name" : "cn",
-"dfs.namenode.replication.min" : "1",
-"mapreduce.map.java.opts" : " -Xmx825955249",
-"yarn.scheduler.fair.allocation.file" : "/etc/yarn/fair-scheduler.xml",
-"s3native.bytes-per-checksum" : "512",
-"mapreduce.tasktracker.tasks.sleeptimebeforesigkill" : "5000",
-"tfile.fs.output.buffer.size" : "262144",
-"yarn.nodemanager.local-dirs" : "${hadoop.tmp.dir}/nm-local-dir",
-"mapreduce.jobtracker.persist.jobstatus.active" : "false",
-"fs.AbstractFileSystem.hdfs.impl" : "org.apache.hadoop.fs.Hdfs",
-"mapreduce.job.map.output.collector.class" : "org.apache.hadoop.mapred.MapTask$MapOutputBuffer",
-"mapreduce.tasktracker.local.dir.minspacestart" : "0",
-"dfs.namenode.safemode.min.datanodes" : "0",
-"hadoop.security.uid.cache.secs" : "14400",
-"dfs.client.https.need-auth" : "false",
-"dfs.client.write.exclude.nodes.cache.expiry.interval.millis" : "600000",
-"dfs.client.https.keystore.resource" : "ssl-client.xml",
-"dfs.namenode.max.objects" : "0",
-"hadoop.ssl.client.conf" : "ssl-client.xml",
-"dfs.namenode.safemode.threshold-pct" : "0.999f",
-"mapreduce.tasktracker.local.dir.minspacekill" : "0",
-"mapreduce.jobtracker.retiredjobs.cache.size" : "1000",
-"dfs.blocksize" : "134217728",
-"yarn.resourcemanager.scheduler.class" : "org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler",
-"mapreduce.job.reduce.slowstart.completedmaps" : "0.8",
-"mapreduce.job.end-notification.retry.attempts" : "5",
-"mapreduce.job.inputformat.class" : "org.apache.hadoop.examples.terasort.TeraGen$RangeInputFormat",
-"mapreduce.map.memory.mb" : "1024",
-"mapreduce.job.user.name" : "jenkins",
-"mapreduce.tasktracker.outofband.heartbeat" : "false",
-"io.native.lib.available" : "true",
-"mapreduce.jobtracker.persist.jobstatus.hours" : "0",
-"dfs.client-write-packet-size" : "65536",
-"mapreduce.client.progressmonitor.pollinterval" : "1000",
-"dfs.namenode.name.dir" : "file://${hadoop.tmp.dir}/dfs/name",
-"dfs.ha.log-roll.period" : "120",
-"mapreduce.reduce.input.buffer.percent" : "0.0",
-"mapreduce.map.output.compress.codec" : "org.apache.hadoop.io.compress.SnappyCodec",
-"dfs.client.failover.sleep.base.millis" : "500",
-"dfs.datanode.directoryscan.threads" : "1",
-"mapreduce.jobtracker.address" : "neededForHive:999999",
-"mapreduce.cluster.local.dir" : "${hadoop.tmp.dir}/mapred/local",
-"yarn.scheduler.fair.user-as-default-queue" : "true",
-"mapreduce.job.application.attempt.id" : "1",
-"dfs.permissions.enabled" : "true",
-"mapreduce.tasktracker.taskcontroller" : "org.apache.hadoop.mapred.DefaultTaskController",
-"yarn.scheduler.fair.preemption" : "true",
-"mapreduce.reduce.shuffle.parallelcopies" : "5",
-"dfs.support.append" : "true",
-"yarn.nodemanager.env-whitelist" : "JAVA_HOME,HADOOP_COMMON_HOME,HADOOP_HDFS_HOME,HADOOP_CONF_DIR,YARN_HOME",
-"mapreduce.jobtracker.heartbeats.in.second" : "100",
-"mapreduce.job.maxtaskfailures.per.tracker" : "3",
-"ipc.client.connection.maxidletime" : "10000",
-"mapreduce.shuffle.ssl.enabled" : "false",
-"dfs.namenode.invalidate.work.pct.per.iteration" : "0.32f",
-"dfs.blockreport.intervalMsec" : "21600000",
-"fs.s3.sleepTimeSeconds" : "10",
-"dfs.namenode.replication.considerLoad" : "true",
-"dfs.client.block.write.retries" : "3",
-"hadoop.ssl.server.conf" : "ssl-server.xml",
-"dfs.namenode.name.dir.restore" : "false",
-"rpc.engine.org.apache.hadoop.mapreduce.v2.api.MRClientProtocolPB" : "org.apache.hadoop.ipc.ProtobufRpcEngine",
-"dfs.datanode.hdfs-blocks-metadata.enabled" : "true",
-"ha.zookeeper.parent-znode" : "/hadoop-ha",
-"mapreduce.reduce.merge.inmem.threshold" : "1000",
-"mapreduce.input.fileinputformat.split.minsize" : "0",
-"dfs.replication" : "3",
-"ipc.client.tcpnodelay" : "false",
-"dfs.namenode.accesstime.precision" : "3600000",
-"s3.stream-buffer-size" : "4096",
-"mapreduce.jobtracker.tasktracker.maxblacklists" : "4",
-"dfs.client.read.shortcircuit.skip.checksum" : "false",
-"mapreduce.job.jvm.numtasks" : "1",
-"mapreduce.task.io.sort.mb" : "100",
-"io.file.buffer.size" : "65536",
-"dfs.namenode.audit.loggers" : "default",
-"dfs.namenode.checkpoint.txns" : "1000000",
-"yarn.nodemanager.admin-env" : "MALLOC_ARENA_MAX=$MALLOC_ARENA_MAX",
-"mapreduce.job.jar" : "/user/jenkins/.staging/job_1369942127770_1207/job.jar",
-"mapreduce.job.split.metainfo.maxsize" : "10000000",
-"kfs.replication" : "3",
-"rpc.engine.org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB" : "org.apache.hadoop.ipc.ProtobufRpcEngine",
-"yarn.app.mapreduce.am.scheduler.heartbeat.interval-ms" : "1000",
-"mapreduce.reduce.maxattempts" : "4",
-"kfs.stream-buffer-size" : "4096",
-"dfs.ha.tail-edits.period" : "60",
-"hadoop.security.authentication" : "simple",
-"fs.s3.buffer.dir" : "${hadoop.tmp.dir}/s3",
-"rpc.engine.org.apache.hadoop.yarn.api.AMRMProtocolPB" : "org.apache.hadoop.ipc.ProtobufRpcEngine",
-"mapreduce.jobtracker.taskscheduler" : "org.apache.hadoop.mapred.JobQueueTaskScheduler",
-"yarn.app.mapreduce.am.job.task.listener.thread-count" : "30",
-"dfs.namenode.avoid.read.stale.datanode" : "false",
-"mapreduce.job.reduces" : "0",
-"mapreduce.map.sort.spill.percent" : "0.8",
-"dfs.client.file-block-storage-locations.timeout" : "60",
-"dfs.datanode.drop.cache.behind.writes" : "false",
-"mapreduce.job.end-notification.retry.interval" : "1",
-"mapreduce.job.maps" : "96",
-"mapreduce.job.speculative.slownodethreshold" : "1.0",
-"tfile.fs.input.buffer.size" : "262144",
-"mapreduce.map.speculative" : "false",
-"dfs.block.access.token.enable" : "false",
-"dfs.journalnode.http-address" : "0.0.0.0:8480",
-"mapreduce.job.acl-view-job" : " ",
-"mapreduce.reduce.shuffle.retry-delay.max.ms" : "60000",
-"mapreduce.job.end-notification.max.retry.interval" : "5",
-"ftp.blocksize" : "67108864",
-"mapreduce.tasktracker.http.threads" : "80",
-"mapreduce.reduce.java.opts" : " -Xmx825955249",
-"dfs.datanode.data.dir" : "file://${hadoop.tmp.dir}/dfs/data",
-"ha.failover-controller.cli-check.rpc-timeout.ms" : "20000",
-"dfs.namenode.max.extra.edits.segments.retained" : "10000",
-"dfs.https.port" : "20102",
-"dfs.namenode.replication.interval" : "3",
-"mapreduce.task.skip.start.attempts" : "2",
-"dfs.namenode.https-address" : "a2115.smile.com:20102",
-"mapreduce.jobtracker.persist.jobstatus.dir" : "/jobtracker/jobsInfo",
-"ipc.client.kill.max" : "10",
-"dfs.ha.automatic-failover.enabled" : "false",
-"mapreduce.jobhistory.keytab" : "/etc/security/keytab/jhs.service.keytab",
-"dfs.image.transfer.timeout" : "600000",
-"dfs.client.failover.sleep.max.millis" : "15000",
-"mapreduce.job.end-notification.max.attempts" : "5",
-"mapreduce.task.tmp.dir" : "./tmp",
-"dfs.default.chunk.view.size" : "32768",
-"kfs.bytes-per-checksum" : "512",
-"mapreduce.reduce.memory.mb" : "1024",
-"hadoop.http.filter.initializers" : "org.apache.hadoop.yarn.server.webproxy.amfilter.AmFilterInitializer",
-"dfs.datanode.failed.volumes.tolerated" : "0",
-"hadoop.http.authentication.type" : "simple",
-"dfs.datanode.data.dir.perm" : "700",
-"yarn.resourcemanager.client.thread-count" : "50",
-"ipc.server.listen.queue.size" : "128",
-"mapreduce.reduce.skip.maxgroups" : "0",
-"file.stream-buffer-size" : "4096",
-"dfs.namenode.fs-limits.max-directory-items" : "0",
-"io.mapfile.bloom.size" : "1048576",
-"yarn.nodemanager.container-executor.class" : "org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor",
-"mapreduce.map.maxattempts" : "4",
-"mapreduce.jobtracker.jobhistory.block.size" : "3145728",
-"yarn.log-aggregation.retain-seconds" : "-1",
-"yarn.app.mapreduce.am.job.committer.cancel-timeout" : "60000",
-"ftp.replication" : "3",
-"mapreduce.jobtracker.http.address" : "0.0.0.0:50030",
-"yarn.nodemanager.health-checker.script.timeout-ms" : "1200000",
-"mapreduce.jobhistory.address" : "a2115.smile.com:10020",
-"mapreduce.jobtracker.taskcache.levels" : "2",
-"dfs.datanode.dns.nameserver" : "default",
-"mapreduce.application.classpath" : "$HADOOP_MAPRED_HOME/share/hadoop/mapreduce/*,$HADOOP_MAPRED_HOME/share/hadoop/mapreduce/lib/*",
-"yarn.nodemanager.log.retain-seconds" : "10800",
-"mapred.child.java.opts" : "-Xmx200m",
-"dfs.replication.max" : "512",
-"map.sort.class" : "org.apache.hadoop.util.QuickSort",
-"dfs.stream-buffer-size" : "4096",
-"dfs.namenode.backup.address" : "0.0.0.0:50100",
-"hadoop.util.hash.type" : "murmur",
-"dfs.block.access.key.update.interval" : "600",
-"dfs.datanode.dns.interface" : "default",
-"dfs.datanode.use.datanode.hostname" : "false",
-"mapreduce.job.output.key.class" : "org.apache.hadoop.io.Text",
-"dfs.client.read.shortcircuit" : "false",
-"dfs.namenode.backup.http-address" : "0.0.0.0:50105",
-"yarn.nodemanager.container-monitor.interval-ms" : "3000",
-"yarn.nodemanager.disk-health-checker.min-healthy-disks" : "0.25",
-"kfs.client-write-packet-size" : "65536",
-"ha.zookeeper.acl" : "world:anyone:rwcda",
-"yarn.nodemanager.sleep-delay-before-sigkill.ms" : "250",
-"mapreduce.job.dir" : "/user/jenkins/.staging/job_1369942127770_1207",
-"io.map.index.skip" : "0",
-"net.topology.node.switch.mapping.impl" : "org.apache.hadoop.net.ScriptBasedMapping",
-"fs.s3.maxRetries" : "4",
-"ha.failover-controller.new-active.rpc-timeout.ms" : "60000",
-"s3native.client-write-packet-size" : "65536",
-"yarn.resourcemanager.amliveliness-monitor.interval-ms" : "1000",
-"hadoop.http.staticuser.user" : "dr.who",
-"mapreduce.reduce.speculative" : "false",
-"mapreduce.client.output.filter" : "FAILED",
-"mapreduce.ifile.readahead.bytes" : "4194304",
-"mapreduce.tasktracker.report.address" : "127.0.0.1:0",
-"mapreduce.task.userlog.limit.kb" : "0",
-"mapreduce.tasktracker.map.tasks.maximum" : "2",
-"hadoop.http.authentication.simple.anonymous.allowed" : "true",
-"hadoop.fuse.timer.period" : "5",
-"dfs.namenode.num.extra.edits.retained" : "1000000",
-"hadoop.rpc.socket.factory.class.default" : "org.apache.hadoop.net.StandardSocketFactory",
-"mapreduce.job.submithostname" : "a2115.smile.com",
-"dfs.namenode.handler.count" : "10",
-"fs.automatic.close" : "false",
-"mapreduce.job.submithostaddress" : "10.20.206.115",
-"mapreduce.tasktracker.healthchecker.script.timeout" : "600000",
-"dfs.datanode.directoryscan.interval" : "21600",
-"yarn.resourcemanager.address" : "a2115.smile.com:8032",
-"yarn.nodemanager.health-checker.interval-ms" : "600000",
-"dfs.client.file-block-storage-locations.num-threads" : "10",
-"yarn.resourcemanager.container-tokens.master-key-rolling-interval-secs" : "86400",
-"mapreduce.reduce.markreset.buffer.percent" : "0.0",
-"hadoop.security.group.mapping.ldap.directory.search.timeout" : "10000",
-"mapreduce.map.log.level" : "INFO",
-"dfs.bytes-per-checksum" : "512",
-"yarn.nodemanager.localizer.address" : "0.0.0.0:8040",
-"dfs.namenode.checkpoint.max-retries" : "3",
-"ha.health-monitor.rpc-timeout.ms" : "45000",
-"yarn.resourcemanager.keytab" : "/etc/krb5.keytab",
-"ftp.stream-buffer-size" : "4096",
-"dfs.namenode.avoid.write.stale.datanode" : "false",
-"hadoop.security.group.mapping.ldap.search.attr.member" : "member",
-"mapreduce.output.fileoutputformat.outputdir" : "hdfs://a2115.smile.com:8020/user/jenkins/tera-gen-1",
-"dfs.blockreport.initialDelay" : "0",
-"yarn.nm.liveness-monitor.expiry-interval-ms" : "600000",
-"hadoop.http.authentication.token.validity" : "36000",
-"dfs.namenode.delegation.token.max-lifetime" : "604800000",
-"mapreduce.job.hdfs-servers" : "${fs.defaultFS}",
-"s3native.replication" : "3",
-"yarn.nodemanager.localizer.client.thread-count" : "5",
-"dfs.heartbeat.interval" : "3",
-"rpc.engine.org.apache.hadoop.ipc.ProtocolMetaInfoPB" : "org.apache.hadoop.ipc.ProtobufRpcEngine",
-"dfs.ha.fencing.ssh.connect-timeout" : "30000",
-"yarn.resourcemanager.container.liveness-monitor.interval-ms" : "600000",
-"yarn.am.liveness-monitor.expiry-interval-ms" : "600000",
-"mapreduce.task.profile" : "false",
-"mapreduce.tasktracker.http.address" : "0.0.0.0:50060",
-"mapreduce.tasktracker.instrumentation" : "org.apache.hadoop.mapred.TaskTrackerMetricsInst",
-"mapreduce.jobhistory.webapp.address" : "a2115.smile.com:19888",
-"ha.failover-controller.graceful-fence.rpc-timeout.ms" : "5000",
-"yarn.ipc.rpc.class" : "org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC",
-"mapreduce.job.name" : "TeraGen",
-"kfs.blocksize" : "67108864",
-"yarn.resourcemanager.am-rm-tokens.master-key-rolling-interval-secs" : "86400",
-"mapreduce.job.ubertask.maxmaps" : "9",
-"yarn.scheduler.maximum-allocation-mb" : "8192",
-"yarn.nodemanager.heartbeat.interval-ms" : "1000",
-"mapreduce.job.userlog.retain.hours" : "24",
-"dfs.namenode.secondary.http-address" : "0.0.0.0:50090",
-"mapreduce.task.timeout" : "600000",
-"mapreduce.framework.name" : "yarn",
-"ipc.client.idlethreshold" : "4000",
-"ftp.bytes-per-checksum" : "512",
-"ipc.server.tcpnodelay" : "false",
-"dfs.namenode.stale.datanode.interval" : "30000",
-"s3.bytes-per-checksum" : "512",
-"mapreduce.job.speculative.slowtaskthreshold" : "1.0",
-"yarn.nodemanager.localizer.cache.target-size-mb" : "10240",
-"yarn.nodemanager.remote-app-log-dir" : "/tmp/logs",
-"fs.s3.block.size" : "67108864",
-"mapreduce.job.queuename" : "sls_queue_1",
-"dfs.client.failover.connection.retries" : "0",
-"hadoop.rpc.protection" : "authentication",
-"yarn.scheduler.minimum-allocation-mb" : "1024",
-"yarn.app.mapreduce.client-am.ipc.max-retries" : "1",
-"hadoop.security.auth_to_local" : "DEFAULT",
-"dfs.secondary.namenode.kerberos.internal.spnego.principal" : "${dfs.web.authentication.kerberos.principal}",
-"ftp.client-write-packet-size" : "65536",
-"fs.defaultFS" : "hdfs://a2115.smile.com:8020",
-"yarn.nodemanager.address" : "0.0.0.0:0",
-"yarn.scheduler.fair.assignmultiple" : "true",
-"yarn.resourcemanager.scheduler.client.thread-count" : "50",
-"mapreduce.task.merge.progress.records" : "10000",
-"file.client-write-packet-size" : "65536",
-"yarn.nodemanager.delete.thread-count" : "4",
-"yarn.resourcemanager.scheduler.address" : "a2115.smile.com:8030",
-"fs.trash.checkpoint.interval" : "0",
-"hadoop.http.authentication.signature.secret.file" : "${user.home}/hadoop-http-auth-signature-secret",
-"s3native.stream-buffer-size" : "4096",
-"mapreduce.reduce.shuffle.read.timeout" : "180000",
-"mapreduce.admin.user.env" : "LD_LIBRARY_PATH=$HADOOP_COMMON_HOME/lib/native",
-"yarn.app.mapreduce.am.command-opts" : " -Xmx1238932873",
-"dfs.namenode.checkpoint.edits.dir" : "${dfs.namenode.checkpoint.dir}",
-"fs.permissions.umask-mode" : "022",
-"dfs.client.domain.socket.data.traffic" : "false",
-"hadoop.common.configuration.version" : "0.23.0",
-"mapreduce.tasktracker.dns.interface" : "default",
-"mapreduce.output.fileoutputformat.compress.type" : "BLOCK",
-"mapreduce.ifile.readahead" : "true",
-"hadoop.security.group.mapping.ldap.ssl" : "false",
-"io.serializations" : "org.apache.hadoop.io.serializer.WritableSerialization,org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization,org.apache.hadoop.io.serializer.avro.AvroReflectSerialization",
-"yarn.nodemanager.aux-services.mapreduce.shuffle.class" : "org.apache.hadoop.mapred.ShuffleHandler",
-"fs.df.interval" : "60000",
-"mapreduce.reduce.shuffle.input.buffer.percent" : "0.70",
-"io.seqfile.compress.blocksize" : "1000000",
-"hadoop.security.groups.cache.secs" : "300",
-"ipc.client.connect.max.retries" : "10",
-"dfs.namenode.delegation.key.update-interval" : "86400000",
-"yarn.nodemanager.process-kill-wait.ms" : "2000",
-"yarn.application.classpath" : "$HADOOP_CONF_DIR,$HADOOP_COMMON_HOME/*,$HADOOP_COMMON_HOME/lib/*,$HADOOP_HDFS_HOME/*,$HADOOP_HDFS_HOME/lib/*,$HADOOP_MAPRED_HOME/*,$HADOOP_MAPRED_HOME/lib/*,$YARN_HOME/*,$YARN_HOME/lib/*",
-"yarn.app.mapreduce.client.max-retries" : "3",
-"dfs.datanode.available-space-volume-choosing-policy.balanced-space-preference-fraction" : "0.75f",
-"yarn.nodemanager.log-aggregation.compression-type" : "none",
-"hadoop.security.group.mapping.ldap.search.filter.user" : "(&(objectClass=user)(sAMAccountName={0}))",
-"yarn.nodemanager.localizer.cache.cleanup.interval-ms" : "600000",
-"dfs.image.compress" : "false",
-"mapred.mapper.new-api" : "true",
-"yarn.nodemanager.log-dirs" : "${yarn.log.dir}/userlogs",
-"dfs.namenode.kerberos.internal.spnego.principal" : "${dfs.web.authentication.kerberos.principal}",
-"fs.s3n.block.size" : "67108864",
-"fs.ftp.host" : "0.0.0.0",
-"hadoop.security.group.mapping" : "org.apache.hadoop.security.JniBasedUnixGroupsMappingWithFallback",
-"dfs.datanode.address" : "0.0.0.0:50010",
-"mapreduce.map.skip.maxrecords" : "0",
-"dfs.datanode.https.address" : "0.0.0.0:50475",
-"file.replication" : "1",
-"yarn.resourcemanager.resource-tracker.address" : "a2115.smile.com:8031",
-"dfs.datanode.drop.cache.behind.reads" : "false",
-"hadoop.fuse.connection.timeout" : "300",
-"hadoop.work.around.non.threadsafe.getpwuid" : "false",
-"mapreduce.jobtracker.restart.recover" : "false",
-"hadoop.tmp.dir" : "/tmp/hadoop-${user.name}",
-"mapreduce.output.fileoutputformat.compress" : "false",
-"mapreduce.tasktracker.indexcache.mb" : "10",
-"mapreduce.client.genericoptionsparser.used" : "true",
-"dfs.client.block.write.replace-datanode-on-failure.policy" : "DEFAULT",
-"mapreduce.job.committer.setup.cleanup.needed" : "true",
-"hadoop.kerberos.kinit.command" : "kinit",
-"dfs.datanode.du.reserved" : "0",
-"dfs.namenode.fs-limits.max-blocks-per-file" : "1048576",
-"file.bytes-per-checksum" : "512",
-"mapreduce.task.profile.reduces" : "0-2",
-"mapreduce.jobtracker.handler.count" : "10",
-"dfs.client.block.write.replace-datanode-on-failure.enable" : "true",
-"mapreduce.job.output.value.class" : "org.apache.hadoop.io.Text",
-"yarn.dispatcher.exit-on-error" : "true",
-"net.topology.script.number.args" : "100",
-"mapreduce.task.profile.maps" : "0-2",
-"dfs.namenode.decommission.interval" : "30",
-"dfs.image.compression.codec" : "org.apache.hadoop.io.compress.DefaultCodec",
-"yarn.resourcemanager.webapp.address" : "a2115.smile.com:8088",
-"mapreduce.jobtracker.system.dir" : "${hadoop.tmp.dir}/mapred/system",
-"hadoop.ssl.hostname.verifier" : "DEFAULT",
-"yarn.nodemanager.vmem-pmem-ratio" : "2.1",
-"dfs.namenode.support.allow.format" : "true",
-"mapreduce.jobhistory.principal" : "jhs/_HOST@REALM.TLD",
-"io.mapfile.bloom.error.rate" : "0.005",
-"mapreduce.shuffle.ssl.file.buffer.size" : "65536",
-"dfs.permissions.superusergroup" : "supergroup",
-"dfs.datanode.available-space-volume-choosing-policy.balanced-space-threshold" : "10737418240",
-"mapreduce.jobtracker.expire.trackers.interval" : "600000",
-"mapreduce.cluster.acls.enabled" : "false",
-"yarn.nodemanager.remote-app-log-dir-suffix" : "logs",
-"ha.failover-controller.graceful-fence.connection.retries" : "1",
-"ha.health-monitor.connect-retry-interval.ms" : "1000",
-"mapreduce.reduce.shuffle.merge.percent" : "0.66",
-"yarn.app.mapreduce.am.resource.mb" : "1536",
-"io.seqfile.local.dir" : "${hadoop.tmp.dir}/io/local",
-"dfs.namenode.checkpoint.check.period" : "60",
-"yarn.resourcemanager.nm.liveness-monitor.interval-ms" : "1000",
-"mapreduce.jobtracker.maxtasks.perjob" : "-1",
-"mapreduce.jobtracker.jobhistory.lru.cache.size" : "5",
-"file.blocksize" : "67108864",
-"tfile.io.chunk.size" : "1048576",
-"mapreduce.job.acl-modify-job" : " ",
-"yarn.nodemanager.webapp.address" : "0.0.0.0:8042",
-"mapreduce.tasktracker.reduce.tasks.maximum" : "2",
-"io.skip.checksum.errors" : "false",
-"mapreduce.cluster.temp.dir" : "${hadoop.tmp.dir}/mapred/temp",
-"yarn.app.mapreduce.am.staging-dir" : "/user",
-"dfs.namenode.edits.journal-plugin.qjournal" : "org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager",
-"dfs.datanode.handler.count" : "10",
-"fs.ftp.host.port" : "21",
-"dfs.namenode.decommission.nodes.per.interval" : "5",
-"yarn.resourcemanager.admin.client.thread-count" : "1",
-"dfs.namenode.fs-limits.max-component-length" : "0",
-"dfs.namenode.checkpoint.period" : "3600",
-"fs.AbstractFileSystem.viewfs.impl" : "org.apache.hadoop.fs.viewfs.ViewFs",
-"yarn.resourcemanager.resource-tracker.client.thread-count" : "50",
-"mapreduce.tasktracker.dns.nameserver" : "default",
-"mapreduce.map.output.compress" : "true",
-"dfs.datanode.ipc.address" : "0.0.0.0:50020",
-"hadoop.ssl.require.client.cert" : "false",
-"yarn.nodemanager.delete.debug-delay-sec" : "0",
-"dfs.datanode.max.transfer.threads" : "4096"
-},
-"computonsPerMapInputByte" : -1,
-"computonsPerMapOutputByte" : -1,
-"computonsPerReduceInputByte" : -1,
-"computonsPerReduceOutputByte" : -1,
-"heapMegabytes" : 200,
-"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" : 47021,
-"minimum" : 11143,
-"rankings" : [ {
-"datum" : 13354,
-"relativeRanking" : 0.05
-}, {
-"datum" : 14101,
-"relativeRanking" : 0.1
-}, {
-"datum" : 15609,
-"relativeRanking" : 0.15
-}, {
-"datum" : 15919,
-"relativeRanking" : 0.2
-}, {
-"datum" : 17003,
-"relativeRanking" : 0.25
-}, {
-"datum" : 17109,
-"relativeRanking" : 0.3
-}, {
-"datum" : 18342,
-"relativeRanking" : 0.35
-}, {
-"datum" : 18870,
-"relativeRanking" : 0.4
-}, {
-"datum" : 19127,
-"relativeRanking" : 0.45
-}, {
-"datum" : 19221,
-"relativeRanking" : 0.5
-}, {
-"datum" : 19481,
-"relativeRanking" : 0.55
-}, {
-"datum" : 19896,
-"relativeRanking" : 0.6
-}, {
-"datum" : 20585,
-"relativeRanking" : 0.65
-}, {
-"datum" : 20784,
-"relativeRanking" : 0.7
-}, {
-"datum" : 21452,
-"relativeRanking" : 0.75
-}, {
-"datum" : 21853,
-"relativeRanking" : 0.8
-}, {
-"datum" : 22436,
-"relativeRanking" : 0.85
-}, {
-"datum" : 32646,
-"relativeRanking" : 0.9
-}, {
-"datum" : 41553,
-"relativeRanking" : 0.95
-} ],
-"numberValues" : 96
-} ],
-"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" : 9223372036854775807,
-"minimum" : -9223372036854775808,
-"rankings" : [ ],
-"numberValues" : 0
-},
-"failedReduceAttemptCDF" : {
-"maximum" : 9223372036854775807,
-"minimum" : -9223372036854775808,
-"rankings" : [ ],
-"numberValues" : 0
-},
-"mapperTriesToSucceed" : [ 1.0 ],
-"failedMapperFraction" : 0.0,
-"relativeTime" : 0,
-"clusterMapMB" : -1,
-"clusterReduceMB" : -1,
-"jobMapMB" : 200,
-"jobReduceMB" : 200
 }

+ 1 - 1
hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/jquery.js

@@ -1504,7 +1504,7 @@ setDocument = Sizzle.setDocument = function( node ) {
 
 	// Support: IE<10
 	// Check if getElementById returns elements by name
-	// The broken getElementById methods don't pick up programatically-set names,
+	// The broken getElementById methods don't pick up programmatically-set names,
 	// so use a roundabout getElementsByName test
 	support.getById = assert(function( div ) {
 		docElem.appendChild( div ).id = expando;

+ 78 - 0
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/ReservationClientUtil.java

@@ -0,0 +1,78 @@
+/**
+ * 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.sls;
+
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
+import org.apache.hadoop.yarn.api.records.*;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Simple support class, used to create reservation requests.
+ */
+public final class ReservationClientUtil {
+
+  private ReservationClientUtil(){
+    //avoid instantiation
+  }
+
+  /**
+   * Creates a request that envelopes a MR jobs, picking max number of maps and
+   * reducers, max durations, and max resources per container.
+   *
+   * @param reservationId the id of the reservation
+   * @param name the name of a reservation
+   * @param maxMapRes maximum resources used by any mapper
+   * @param numberMaps number of mappers
+   * @param maxMapDur maximum duration of any mapper
+   * @param maxRedRes maximum resources used by any reducer
+   * @param numberReduces number of reducers
+   * @param maxRedDur maximum duration of any reducer
+   * @param arrival start time of valid range for reservation
+   * @param deadline deadline for this reservation
+   * @param queueName queue to submit to
+   * @return a submission request
+   */
+  @SuppressWarnings("checkstyle:parameternumber")
+  public static ReservationSubmissionRequest createMRReservation(
+      ReservationId reservationId, String name, Resource maxMapRes,
+      int numberMaps, long maxMapDur, Resource maxRedRes, int numberReduces,
+      long maxRedDur, long arrival, long deadline, String queueName) {
+
+    ReservationRequest mapRR = ReservationRequest.newInstance(maxMapRes,
+        numberMaps, numberMaps, maxMapDur);
+    ReservationRequest redRR = ReservationRequest.newInstance(maxRedRes,
+        numberReduces, numberReduces, maxRedDur);
+
+    List<ReservationRequest> listResReq = new ArrayList<ReservationRequest>();
+    listResReq.add(mapRR);
+    listResReq.add(redRR);
+
+    ReservationRequests reservationRequests = ReservationRequests
+        .newInstance(listResReq, ReservationRequestInterpreter.R_ORDER_NO_GAP);
+    ReservationDefinition resDef = ReservationDefinition.newInstance(arrival,
+        deadline, reservationRequests, name);
+
+    // outermost request
+    ReservationSubmissionRequest request = ReservationSubmissionRequest
+        .newInstance(resDef, queueName, reservationId);
+
+    return request;
+  }
+}

+ 4 - 4
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/RumenToSLSConverter.java

@@ -34,6 +34,9 @@ import java.util.Set;
 import java.util.TreeMap;
 import java.util.TreeSet;
 
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.GnuParser;
@@ -41,9 +44,6 @@ import org.apache.commons.cli.Options;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.sls.utils.SLSUtils;
-import org.codehaus.jackson.JsonFactory;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.map.ObjectWriter;
 
 @Private
 @Unstable
@@ -127,7 +127,7 @@ public class RumenToSLSConverter {
         ObjectMapper mapper = new ObjectMapper();
         ObjectWriter writer = mapper.writerWithDefaultPrettyPrinter();
         Iterator<Map> i = mapper.readValues(
-                new JsonFactory().createJsonParser(input), Map.class);
+            new JsonFactory().createParser(input), Map.class);
         while (i.hasNext()) {
           Map m = i.next();
           output.write(writer.writeValueAsString(createSLSJob(m)) + EOL);

+ 649 - 283
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java

@@ -31,60 +31,77 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 import java.util.Set;
+import java.util.Collections;
+import java.util.concurrent.ConcurrentHashMap;
+
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.databind.ObjectMapper;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.GnuParser;
 import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.metrics2.source.JvmMetrics;
 import org.apache.hadoop.tools.rumen.JobTraceReader;
 import org.apache.hadoop.tools.rumen.LoggedJob;
 import org.apache.hadoop.tools.rumen.LoggedTask;
 import org.apache.hadoop.tools.rumen.LoggedTaskAttempt;
+import org.apache.hadoop.tools.rumen.TaskAttemptInfo;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
-import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
 import org.apache.hadoop.yarn.sls.appmaster.AMSimulator;
 import org.apache.hadoop.yarn.sls.conf.SLSConfiguration;
 import org.apache.hadoop.yarn.sls.nodemanager.NMSimulator;
-import org.apache.hadoop.yarn.sls.scheduler.ContainerSimulator;
-import org.apache.hadoop.yarn.sls.scheduler.ResourceSchedulerWrapper;
+import org.apache.hadoop.yarn.sls.resourcemanager.MockAMLauncher;
 import org.apache.hadoop.yarn.sls.scheduler.SLSCapacityScheduler;
 import org.apache.hadoop.yarn.sls.scheduler.TaskRunner;
+import org.apache.hadoop.yarn.sls.scheduler.SLSFairScheduler;
+import org.apache.hadoop.yarn.sls.scheduler.ContainerSimulator;
 import org.apache.hadoop.yarn.sls.scheduler.SchedulerWrapper;
+import org.apache.hadoop.yarn.sls.synthetic.SynthJob;
+import org.apache.hadoop.yarn.sls.synthetic.SynthTraceJobProducer;
 import org.apache.hadoop.yarn.sls.utils.SLSUtils;
+import org.apache.hadoop.yarn.util.UTCClock;
 import org.apache.hadoop.yarn.util.resource.Resources;
-import org.codehaus.jackson.JsonFactory;
-import org.codehaus.jackson.map.ObjectMapper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @Private
 @Unstable
-public class SLSRunner {
+public class SLSRunner extends Configured implements Tool {
   // RM, Runner
   private ResourceManager rm;
   private static TaskRunner runner = new TaskRunner();
   private String[] inputTraces;
-  private Configuration conf;
   private Map<String, Integer> queueAppNumMap;
-  
+
   // NM simulator
   private HashMap<NodeId, NMSimulator> nmMap;
   private int nmMemoryMB, nmVCores;
   private String nodeFile;
-  
+
   // AM simulator
   private int AM_ID;
   private Map<String, AMSimulator> amMap;
@@ -99,49 +116,92 @@ public class SLSRunner {
   // other simulation information
   private int numNMs, numRacks, numAMs, numTasks;
   private long maxRuntime;
-  public final static Map<String, Object> simulateInfoMap =
+
+  private final static Map<String, Object> simulateInfoMap =
           new HashMap<String, Object>();
 
   // logger
   public final static Logger LOG = LoggerFactory.getLogger(SLSRunner.class);
 
-  // input traces, input-rumen or input-sls
-  private boolean isSLS;
-  
-  public SLSRunner(boolean isSLS, String inputTraces[], String nodeFile,
-                   String outputDir, Set<String> trackedApps,
-                   boolean printsimulation)
-          throws IOException, ClassNotFoundException {
-    this.isSLS = isSLS;
-    this.inputTraces = inputTraces.clone();
-    this.nodeFile = nodeFile;
-    this.trackedApps = trackedApps;
-    this.printSimulation = printsimulation;
-    metricsOutputDir = outputDir;
-    
-    nmMap = new HashMap<NodeId, NMSimulator>();
-    queueAppNumMap = new HashMap<String, Integer>();
-    amMap = new HashMap<String, AMSimulator>();
-    amClassMap = new HashMap<String, Class>();
-    
+  private final static int DEFAULT_MAPPER_PRIORITY = 20;
+  private final static int DEFAULT_REDUCER_PRIORITY = 10;
+
+  private static boolean exitAtTheFinish = false;
+
+  /**
+   * The type of trace in input.
+   */
+  public enum TraceType {
+    SLS, RUMEN, SYNTH
+  }
+
+  private TraceType inputType;
+  private SynthTraceJobProducer stjp;
+
+  public SLSRunner() throws ClassNotFoundException {
+    Configuration tempConf = new Configuration(false);
+    init(tempConf);
+  }
+
+  public SLSRunner(Configuration tempConf) throws ClassNotFoundException {
+    init(tempConf);
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    if (null != conf) {
+      // Override setConf to make sure all conf added load sls-runner.xml, see
+      // YARN-6560
+      conf.addResource("sls-runner.xml");
+    }
+    super.setConf(conf);
+  }
+
+  private void init(Configuration tempConf) throws ClassNotFoundException {
+    nmMap = new HashMap<>();
+    queueAppNumMap = new HashMap<>();
+    amMap = new ConcurrentHashMap<>();
+    amClassMap = new HashMap<>();
+
     // runner configuration
-    conf = new Configuration(false);
-    conf.addResource("sls-runner.xml");
+    setConf(tempConf);
+
     // runner
-    int poolSize = conf.getInt(SLSConfiguration.RUNNER_POOL_SIZE,
-                                SLSConfiguration.RUNNER_POOL_SIZE_DEFAULT);
+    int poolSize = tempConf.getInt(SLSConfiguration.RUNNER_POOL_SIZE,
+        SLSConfiguration.RUNNER_POOL_SIZE_DEFAULT);
     SLSRunner.runner.setQueueSize(poolSize);
     // <AMType, Class> map
-    for (Map.Entry e : conf) {
+    for (Map.Entry e : tempConf) {
       String key = e.getKey().toString();
       if (key.startsWith(SLSConfiguration.AM_TYPE)) {
         String amType = key.substring(SLSConfiguration.AM_TYPE.length());
-        amClassMap.put(amType, Class.forName(conf.get(key)));
+        amClassMap.put(amType, Class.forName(tempConf.get(key)));
       }
     }
   }
-  
-  public void start() throws Exception {
+
+  /**
+   * @return an unmodifiable view of the simulated info map.
+   */
+  public static Map<String, Object> getSimulateInfoMap() {
+    return Collections.unmodifiableMap(simulateInfoMap);
+  }
+
+  public void setSimulationParams(TraceType inType, String[] inTraces,
+      String nodes, String outDir, Set<String> trackApps,
+      boolean printsimulation) throws IOException, ClassNotFoundException {
+
+    this.inputType = inType;
+    this.inputTraces = inTraces.clone();
+    this.nodeFile = nodes;
+    this.trackedApps = trackApps;
+    this.printSimulation = printsimulation;
+    metricsOutputDir = outDir;
+
+  }
+
+  public void start() throws IOException, ClassNotFoundException, YarnException,
+      InterruptedException {
     // start resource manager
     startRM();
     // start node managers
@@ -149,10 +209,10 @@ public class SLSRunner {
     // start application masters
     startAM();
     // set queue & tracked apps information
-    ((SchedulerWrapper) rm.getResourceScheduler())
-                            .setQueueSet(this.queueAppNumMap.keySet());
-    ((SchedulerWrapper) rm.getResourceScheduler())
-                            .setTrackedAppSet(this.trackedApps);
+    ((SchedulerWrapper) rm.getResourceScheduler()).getTracker()
+        .setQueueSet(this.queueAppNumMap.keySet());
+    ((SchedulerWrapper) rm.getResourceScheduler()).getTracker()
+        .setTrackedAppSet(this.trackedApps);
     // print out simulation info
     printSimulationInfo();
     // blocked until all nodes RUNNING
@@ -160,66 +220,92 @@ public class SLSRunner {
     // starting the runner once everything is ready to go,
     runner.start();
   }
-  
-  private void startRM() throws IOException, ClassNotFoundException {
-    Configuration rmConf = new YarnConfiguration();
+
+  private void startRM() throws ClassNotFoundException, YarnException {
+    Configuration rmConf = new YarnConfiguration(getConf());
     String schedulerClass = rmConf.get(YarnConfiguration.RM_SCHEDULER);
 
-    // For CapacityScheduler we use a sub-classing instead of wrapping
-    // to allow scheduler-specific invocations from monitors to work
-    // this can be used for other schedulers as well if we care to
-    // exercise/track behaviors that are not common to the scheduler api
-    if(Class.forName(schedulerClass) == CapacityScheduler.class) {
+    if (Class.forName(schedulerClass) == CapacityScheduler.class) {
       rmConf.set(YarnConfiguration.RM_SCHEDULER,
           SLSCapacityScheduler.class.getName());
       rmConf.setBoolean(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS, true);
       rmConf.set(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES,
           ProportionalCapacityPreemptionPolicy.class.getName());
-    } else {
+    } else if (Class.forName(schedulerClass) == FairScheduler.class) {
       rmConf.set(YarnConfiguration.RM_SCHEDULER,
-              ResourceSchedulerWrapper.class.getName());
-      rmConf.set(SLSConfiguration.RM_SCHEDULER, schedulerClass);
+          SLSFairScheduler.class.getName());
+    } else if (Class.forName(schedulerClass) == FifoScheduler.class) {
+      // TODO add support for FifoScheduler
+      throw new YarnException("Fifo Scheduler is not supported yet.");
     }
 
     rmConf.set(SLSConfiguration.METRICS_OUTPUT_DIR, metricsOutputDir);
-    rm = new ResourceManager();
+
+    final SLSRunner se = this;
+    rm = new ResourceManager() {
+      @Override
+      protected ApplicationMasterLauncher createAMLauncher() {
+        return new MockAMLauncher(se, this.rmContext, amMap);
+      }
+    };
+
+    // Across runs of parametrized tests, the JvmMetrics objects is retained,
+    // but is not registered correctly
+    JvmMetrics jvmMetrics = JvmMetrics.initSingleton("ResourceManager", null);
+    jvmMetrics.registerIfNeeded();
+
+    // Init and start the actual ResourceManager
     rm.init(rmConf);
     rm.start();
   }
 
   private void startNM() throws YarnException, IOException {
     // nm configuration
-    nmMemoryMB = conf.getInt(SLSConfiguration.NM_MEMORY_MB,
-            SLSConfiguration.NM_MEMORY_MB_DEFAULT);
-    nmVCores = conf.getInt(SLSConfiguration.NM_VCORES,
-            SLSConfiguration.NM_VCORES_DEFAULT);
-    int heartbeatInterval = conf.getInt(
-            SLSConfiguration.NM_HEARTBEAT_INTERVAL_MS,
+    nmMemoryMB = getConf().getInt(SLSConfiguration.NM_MEMORY_MB,
+        SLSConfiguration.NM_MEMORY_MB_DEFAULT);
+    nmVCores = getConf().getInt(SLSConfiguration.NM_VCORES,
+        SLSConfiguration.NM_VCORES_DEFAULT);
+    int heartbeatInterval =
+        getConf().getInt(SLSConfiguration.NM_HEARTBEAT_INTERVAL_MS,
             SLSConfiguration.NM_HEARTBEAT_INTERVAL_MS_DEFAULT);
     // nm information (fetch from topology file, or from sls/rumen json file)
     Set<String> nodeSet = new HashSet<String>();
     if (nodeFile.isEmpty()) {
-      if (isSLS) {
-        for (String inputTrace : inputTraces) {
+      for (String inputTrace : inputTraces) {
+
+        switch (inputType) {
+        case SLS:
           nodeSet.addAll(SLSUtils.parseNodesFromSLSTrace(inputTrace));
-        }
-      } else {
-        for (String inputTrace : inputTraces) {
+          break;
+        case RUMEN:
           nodeSet.addAll(SLSUtils.parseNodesFromRumenTrace(inputTrace));
+          break;
+        case SYNTH:
+          stjp = new SynthTraceJobProducer(getConf(), new Path(inputTraces[0]));
+          nodeSet.addAll(SLSUtils.generateNodes(stjp.getNumNodes(),
+              stjp.getNumNodes()/stjp.getNodesPerRack()));
+          break;
+        default:
+          throw new YarnException("Input configuration not recognized, "
+              + "trace type should be SLS, RUMEN, or SYNTH");
         }
       }
-
     } else {
       nodeSet.addAll(SLSUtils.parseNodesFromNodeFile(nodeFile));
     }
+
+    if (nodeSet.size() == 0) {
+      throw new YarnException("No node! Please configure nodes.");
+    }
+
     // create NM simulators
     Random random = new Random();
     Set<String> rackSet = new HashSet<String>();
     for (String hostName : nodeSet) {
       // we randomize the heartbeat start time from zero to 1 interval
       NMSimulator nm = new NMSimulator();
-      nm.init(hostName, nmMemoryMB, nmVCores, 
-          random.nextInt(heartbeatInterval), heartbeatInterval, rm);
+      nm.init(hostName, nmMemoryMB, nmVCores, random.nextInt(heartbeatInterval),
+          heartbeatInterval, rm);
       nmMap.put(nm.getNode().getNodeID(), nm);
       runner.schedule(nm);
       rackSet.add(nm.getNode().getRackName());
@@ -234,7 +320,7 @@ public class SLSRunner {
       int numRunningNodes = 0;
       for (RMNode node : rm.getRMContext().getRMNodes().values()) {
         if (node.getState() == NodeState.RUNNING) {
-          numRunningNodes ++;
+          numRunningNodes++;
         }
       }
       if (numRunningNodes == numNMs) {
@@ -250,209 +336,433 @@ public class SLSRunner {
 
   @SuppressWarnings("unchecked")
   private void startAM() throws YarnException, IOException {
-    // application/container configuration
-    int heartbeatInterval = conf.getInt(
-            SLSConfiguration.AM_HEARTBEAT_INTERVAL_MS,
-            SLSConfiguration.AM_HEARTBEAT_INTERVAL_MS_DEFAULT);
-    int containerMemoryMB = conf.getInt(SLSConfiguration.CONTAINER_MEMORY_MB,
-            SLSConfiguration.CONTAINER_MEMORY_MB_DEFAULT);
-    int containerVCores = conf.getInt(SLSConfiguration.CONTAINER_VCORES,
-            SLSConfiguration.CONTAINER_VCORES_DEFAULT);
-    Resource containerResource =
-            BuilderUtils.newResource(containerMemoryMB, containerVCores);
-
-    // application workload
-    if (isSLS) {
-      startAMFromSLSTraces(containerResource, heartbeatInterval);
-    } else {
-      startAMFromRumenTraces(containerResource, heartbeatInterval);
+    switch (inputType) {
+    case SLS:
+      for (String inputTrace : inputTraces) {
+        startAMFromSLSTrace(inputTrace);
+      }
+      break;
+    case RUMEN:
+      long baselineTimeMS = 0;
+      for (String inputTrace : inputTraces) {
+        startAMFromRumenTrace(inputTrace, baselineTimeMS);
+      }
+      break;
+    case SYNTH:
+      startAMFromSynthGenerator();
+      break;
+    default:
+      throw new YarnException("Input configuration not recognized, "
+          + "trace type should be SLS, RUMEN, or SYNTH");
     }
+
     numAMs = amMap.size();
     remainingApps = numAMs;
   }
 
   /**
-   * parse workload information from sls trace files
+   * Parse workload from a SLS trace file.
    */
   @SuppressWarnings("unchecked")
-  private void startAMFromSLSTraces(Resource containerResource,
-                                    int heartbeatInterval) throws IOException {
-    // parse from sls traces
+  private void startAMFromSLSTrace(String inputTrace) throws IOException {
     JsonFactory jsonF = new JsonFactory();
     ObjectMapper mapper = new ObjectMapper();
-    for (String inputTrace : inputTraces) {
-      Reader input =
-          new InputStreamReader(new FileInputStream(inputTrace), "UTF-8");
-      try {
-        Iterator<Map> i = mapper.readValues(jsonF.createJsonParser(input),
-                Map.class);
-        while (i.hasNext()) {
-          Map jsonJob = i.next();
-
-          // load job information
-          long jobStartTime = Long.parseLong(
-                  jsonJob.get("job.start.ms").toString());
-          long jobFinishTime = Long.parseLong(
-                  jsonJob.get("job.end.ms").toString());
-
-          String user = (String) jsonJob.get("job.user");
-          if (user == null)  user = "default";
-          String queue = jsonJob.get("job.queue.name").toString();
-
-          String oldAppId = jsonJob.get("job.id").toString();
-          boolean isTracked = trackedApps.contains(oldAppId);
-          int queueSize = queueAppNumMap.containsKey(queue) ?
-                  queueAppNumMap.get(queue) : 0;
-          queueSize ++;
-          queueAppNumMap.put(queue, queueSize);
-          // tasks
-          List tasks = (List) jsonJob.get("job.tasks");
-          if (tasks == null || tasks.size() == 0) {
-            continue;
-          }
-          List<ContainerSimulator> containerList =
-                  new ArrayList<ContainerSimulator>();
-          for (Object o : tasks) {
-            Map jsonTask = (Map) o;
-            String hostname = jsonTask.get("container.host").toString();
-            long taskStart = Long.parseLong(
-                    jsonTask.get("container.start.ms").toString());
-            long taskFinish = Long.parseLong(
-                    jsonTask.get("container.end.ms").toString());
-            long lifeTime = taskFinish - taskStart;
-
-            // Set memory and vcores from job trace file
-            Resource res = Resources.clone(containerResource);
-            if (jsonTask.containsKey("container.memory")) {
-              int containerMemory = Integer.parseInt(
-                  jsonTask.get("container.memory").toString());
-              res.setMemorySize(containerMemory);
-            }
-
-            if (jsonTask.containsKey("container.vcores")) {
-              int containerVCores = Integer.parseInt(
-                  jsonTask.get("container.vcores").toString());
-              res.setVirtualCores(containerVCores);
-            }
-
-            int priority = Integer.parseInt(
-                    jsonTask.get("container.priority").toString());
-            String type = jsonTask.get("container.type").toString();
-            containerList.add(new ContainerSimulator(res,
-                    lifeTime, hostname, priority, type));
-          }
-
-          // create a new AM
-          String amType = jsonJob.get("am.type").toString();
-          AMSimulator amSim = (AMSimulator) ReflectionUtils.newInstance(
-                  amClassMap.get(amType), new Configuration());
-          if (amSim != null) {
-            amSim.init(AM_ID++, heartbeatInterval, containerList, rm,
-                    this, jobStartTime, jobFinishTime, user, queue,
-                    isTracked, oldAppId);
-            runner.schedule(amSim);
-            maxRuntime = Math.max(maxRuntime, jobFinishTime);
-            numTasks += containerList.size();
-            amMap.put(oldAppId, amSim);
-          }
+
+    try (Reader input = new InputStreamReader(
+        new FileInputStream(inputTrace), "UTF-8")) {
+      Iterator<Map> jobIter = mapper.readValues(
+          jsonF.createParser(input), Map.class);
+
+      while (jobIter.hasNext()) {
+        try {
+          createAMForJob(jobIter.next());
+        } catch (Exception e) {
+          LOG.error("Failed to create an AM: {}", e.getMessage());
         }
-      } finally {
-        input.close();
       }
     }
   }
 
+  private void createAMForJob(Map jsonJob) throws YarnException {
+    long jobStartTime = Long.parseLong(jsonJob.get("job.start.ms").toString());
+
+    long jobFinishTime = 0;
+    if (jsonJob.containsKey("job.end.ms")) {
+      jobFinishTime = Long.parseLong(jsonJob.get("job.end.ms").toString());
+    }
+
+    String user = (String) jsonJob.get("job.user");
+    if (user == null) {
+      user = "default";
+    }
+
+    String queue = jsonJob.get("job.queue.name").toString();
+    increaseQueueAppNum(queue);
+
+    String amType = (String)jsonJob.get("am.type");
+    if (amType == null) {
+      amType = SLSUtils.DEFAULT_JOB_TYPE;
+    }
+
+    int jobCount = 1;
+    if (jsonJob.containsKey("job.count")) {
+      jobCount = Integer.parseInt(jsonJob.get("job.count").toString());
+    }
+    jobCount = Math.max(jobCount, 1);
+
+    String oldAppId = (String)jsonJob.get("job.id");
+    // Job id is generated automatically if this job configuration allows
+    // multiple job instances
+    if(jobCount > 1) {
+      oldAppId = null;
+    }
+
+    for (int i = 0; i < jobCount; i++) {
+      runNewAM(amType, user, queue, oldAppId, jobStartTime, jobFinishTime,
+          getTaskContainers(jsonJob), null, getAMContainerResource(jsonJob));
+    }
+  }
+
+  private List<ContainerSimulator> getTaskContainers(Map jsonJob)
+      throws YarnException {
+    List<ContainerSimulator> containers = new ArrayList<>();
+    List tasks = (List) jsonJob.get("job.tasks");
+    if (tasks == null || tasks.size() == 0) {
+      throw new YarnException("No task for the job!");
+    }
+
+    for (Object o : tasks) {
+      Map jsonTask = (Map) o;
+
+      String hostname = (String) jsonTask.get("container.host");
+
+      long duration = 0;
+      if (jsonTask.containsKey("duration.ms")) {
+        duration = Integer.parseInt(jsonTask.get("duration.ms").toString());
+      } else if (jsonTask.containsKey("container.start.ms") &&
+          jsonTask.containsKey("container.end.ms")) {
+        long taskStart = Long.parseLong(jsonTask.get("container.start.ms")
+            .toString());
+        long taskFinish = Long.parseLong(jsonTask.get("container.end.ms")
+            .toString());
+        duration = taskFinish - taskStart;
+      }
+      if (duration <= 0) {
+        throw new YarnException("Duration of a task shouldn't be less or equal"
+            + " to 0!");
+      }
+
+      Resource res = getDefaultContainerResource();
+      if (jsonTask.containsKey("container.memory")) {
+        int containerMemory =
+            Integer.parseInt(jsonTask.get("container.memory").toString());
+        res.setMemorySize(containerMemory);
+      }
+
+      if (jsonTask.containsKey("container.vcores")) {
+        int containerVCores =
+            Integer.parseInt(jsonTask.get("container.vcores").toString());
+        res.setVirtualCores(containerVCores);
+      }
+
+      int priority = DEFAULT_MAPPER_PRIORITY;
+      if (jsonTask.containsKey("container.priority")) {
+        priority = Integer.parseInt(jsonTask.get("container.priority")
+            .toString());
+      }
+
+      String type = "map";
+      if (jsonTask.containsKey("container.type")) {
+        type = jsonTask.get("container.type").toString();
+      }
+
+      int count = 1;
+      if (jsonTask.containsKey("count")) {
+        count = Integer.parseInt(jsonTask.get("count").toString());
+      }
+      count = Math.max(count, 1);
+
+      for (int i = 0; i < count; i++) {
+        containers.add(
+            new ContainerSimulator(res, duration, hostname, priority, type));
+      }
+    }
+
+    return containers;
+  }
+
   /**
-   * parse workload information from rumen trace files
+   * Parse workload from a rumen trace file.
    */
   @SuppressWarnings("unchecked")
-  private void startAMFromRumenTraces(Resource containerResource,
-                                      int heartbeatInterval)
-          throws IOException {
+  private void startAMFromRumenTrace(String inputTrace, long baselineTimeMS)
+      throws IOException {
     Configuration conf = new Configuration();
     conf.set("fs.defaultFS", "file:///");
+    File fin = new File(inputTrace);
+
+    try (JobTraceReader reader = new JobTraceReader(
+        new Path(fin.getAbsolutePath()), conf)) {
+      LoggedJob job = reader.getNext();
+
+      while (job != null) {
+        try {
+          createAMForJob(job, baselineTimeMS);
+        } catch (Exception e) {
+          LOG.error("Failed to create an AM: {}", e.getMessage());
+        }
+
+        job = reader.getNext();
+      }
+    }
+  }
+
+  private void createAMForJob(LoggedJob job, long baselineTimeMs)
+      throws YarnException {
+    String user = job.getUser() == null ? "default" :
+        job.getUser().getValue();
+    String jobQueue = job.getQueue().getValue();
+    String oldJobId = job.getJobID().toString();
+    long jobStartTimeMS = job.getSubmitTime();
+    long jobFinishTimeMS = job.getFinishTime();
+    if (baselineTimeMs == 0) {
+      baselineTimeMs = job.getSubmitTime();
+    }
+    jobStartTimeMS -= baselineTimeMs;
+    jobFinishTimeMS -= baselineTimeMs;
+    if (jobStartTimeMS < 0) {
+      LOG.warn("Warning: reset job {} start time to 0.", oldJobId);
+      jobFinishTimeMS = jobFinishTimeMS - jobStartTimeMS;
+      jobStartTimeMS = 0;
+    }
+
+    increaseQueueAppNum(jobQueue);
+
+    List<ContainerSimulator> containerList = new ArrayList<>();
+    // mapper
+    for (LoggedTask mapTask : job.getMapTasks()) {
+      if (mapTask.getAttempts().size() == 0) {
+        throw new YarnException("Invalid map task, no attempt for a mapper!");
+      }
+      LoggedTaskAttempt taskAttempt =
+          mapTask.getAttempts().get(mapTask.getAttempts().size() - 1);
+      String hostname = taskAttempt.getHostName().getValue();
+      long containerLifeTime = taskAttempt.getFinishTime() -
+          taskAttempt.getStartTime();
+      containerList.add(
+          new ContainerSimulator(getDefaultContainerResource(),
+              containerLifeTime, hostname, DEFAULT_MAPPER_PRIORITY, "map"));
+    }
+
+    // reducer
+    for (LoggedTask reduceTask : job.getReduceTasks()) {
+      if (reduceTask.getAttempts().size() == 0) {
+        throw new YarnException(
+            "Invalid reduce task, no attempt for a reducer!");
+      }
+      LoggedTaskAttempt taskAttempt =
+          reduceTask.getAttempts().get(reduceTask.getAttempts().size() - 1);
+      String hostname = taskAttempt.getHostName().getValue();
+      long containerLifeTime = taskAttempt.getFinishTime() -
+          taskAttempt.getStartTime();
+      containerList.add(
+          new ContainerSimulator(getDefaultContainerResource(),
+              containerLifeTime, hostname, DEFAULT_REDUCER_PRIORITY, "reduce"));
+    }
+
+    // Only supports the default job type currently
+    runNewAM(SLSUtils.DEFAULT_JOB_TYPE, user, jobQueue, oldJobId,
+        jobStartTimeMS, jobFinishTimeMS, containerList, null,
+        getAMContainerResource(null));
+  }
+
+  private Resource getDefaultContainerResource() {
+    int containerMemory = getConf().getInt(SLSConfiguration.CONTAINER_MEMORY_MB,
+        SLSConfiguration.CONTAINER_MEMORY_MB_DEFAULT);
+    int containerVCores = getConf().getInt(SLSConfiguration.CONTAINER_VCORES,
+        SLSConfiguration.CONTAINER_VCORES_DEFAULT);
+    return Resources.createResource(containerMemory, containerVCores);
+  }
+
+  /**
+   * parse workload information from synth-generator trace files.
+   */
+  @SuppressWarnings("unchecked")
+  private void startAMFromSynthGenerator() throws YarnException, IOException {
+    Configuration localConf = new Configuration();
+    localConf.set("fs.defaultFS", "file:///");
     long baselineTimeMS = 0;
-    for (String inputTrace : inputTraces) {
-      File fin = new File(inputTrace);
-      JobTraceReader reader = new JobTraceReader(
-              new Path(fin.getAbsolutePath()), conf);
-      try {
-        LoggedJob job = null;
-        while ((job = reader.getNext()) != null) {
-          // only support MapReduce currently
-          String jobType = "mapreduce";
-          String user = job.getUser() == null ?
-                  "default" : job.getUser().getValue();
-          String jobQueue = job.getQueue().getValue();
-          String oldJobId = job.getJobID().toString();
-          long jobStartTimeMS = job.getSubmitTime();
-          long jobFinishTimeMS = job.getFinishTime();
-          if (baselineTimeMS == 0) {
-            baselineTimeMS = jobStartTimeMS;
-          }
-          jobStartTimeMS -= baselineTimeMS;
-          jobFinishTimeMS -= baselineTimeMS;
-          if (jobStartTimeMS < 0) {
-            LOG.warn("Warning: reset job {} start time to 0.", oldJobId);
-            jobFinishTimeMS = jobFinishTimeMS - jobStartTimeMS;
-            jobStartTimeMS = 0;
-          }
-
-          boolean isTracked = trackedApps.contains(oldJobId);
-          int queueSize = queueAppNumMap.containsKey(jobQueue) ?
-                  queueAppNumMap.get(jobQueue) : 0;
-          queueSize ++;
-          queueAppNumMap.put(jobQueue, queueSize);
-
-          List<ContainerSimulator> containerList =
-                  new ArrayList<ContainerSimulator>();
-          // map tasks
-          for(LoggedTask mapTask : job.getMapTasks()) {
-            if (mapTask.getAttempts().size() == 0) {
-              continue;
-            }
-            LoggedTaskAttempt taskAttempt = mapTask.getAttempts()
-                    .get(mapTask.getAttempts().size() - 1);
-            String hostname = taskAttempt.getHostName().getValue();
-            long containerLifeTime = taskAttempt.getFinishTime()
-                    - taskAttempt.getStartTime();
-            containerList.add(new ContainerSimulator(containerResource,
-                    containerLifeTime, hostname, 10, "map"));
-          }
-
-          // reduce tasks
-          for(LoggedTask reduceTask : job.getReduceTasks()) {
-            if (reduceTask.getAttempts().size() == 0) {
-              continue;
-            }
-            LoggedTaskAttempt taskAttempt = reduceTask.getAttempts()
-                    .get(reduceTask.getAttempts().size() - 1);
-            String hostname = taskAttempt.getHostName().getValue();
-            long containerLifeTime = taskAttempt.getFinishTime()
-                    - taskAttempt.getStartTime();
-            containerList.add(new ContainerSimulator(containerResource,
-                    containerLifeTime, hostname, 20, "reduce"));
-          }
-
-          // create a new AM
-          AMSimulator amSim = (AMSimulator) ReflectionUtils.newInstance(
-                  amClassMap.get(jobType), conf);
-          if (amSim != null) {
-            amSim.init(AM_ID ++, heartbeatInterval, containerList,
-                    rm, this, jobStartTimeMS, jobFinishTimeMS, user, jobQueue,
-                    isTracked, oldJobId);
-            runner.schedule(amSim);
-            maxRuntime = Math.max(maxRuntime, jobFinishTimeMS);
-            numTasks += containerList.size();
-            amMap.put(oldJobId, amSim);
-          }
+
+    // reservations use wall clock time, so need to have a reference for that
+    UTCClock clock = new UTCClock();
+    long now = clock.getTime();
+
+    try {
+
+      // if we use the nodeFile this could have been not initialized yet.
+      if (stjp == null) {
+        stjp = new SynthTraceJobProducer(getConf(), new Path(inputTraces[0]));
+      }
+
+      SynthJob job = null;
+      // we use stjp, a reference to the job producer instantiated during node
+      // creation
+      while ((job = (SynthJob) stjp.getNextJob()) != null) {
+        // only support MapReduce currently
+        String user = job.getUser();
+        String jobQueue = job.getQueueName();
+        String oldJobId = job.getJobID().toString();
+        long jobStartTimeMS = job.getSubmissionTime();
+
+        // CARLO: Finish time is only used for logging, omit for now
+        long jobFinishTimeMS = -1L;
+
+        if (baselineTimeMS == 0) {
+          baselineTimeMS = jobStartTimeMS;
+        }
+        jobStartTimeMS -= baselineTimeMS;
+        jobFinishTimeMS -= baselineTimeMS;
+        if (jobStartTimeMS < 0) {
+          LOG.warn("Warning: reset job {} start time to 0.", oldJobId);
+          jobFinishTimeMS = jobFinishTimeMS - jobStartTimeMS;
+          jobStartTimeMS = 0;
+        }
+
+        increaseQueueAppNum(jobQueue);
+
+        List<ContainerSimulator> containerList =
+            new ArrayList<ContainerSimulator>();
+        ArrayList<NodeId> keyAsArray = new ArrayList<NodeId>(nmMap.keySet());
+        Random rand = new Random(stjp.getSeed());
+
+        Resource maxMapRes = Resource.newInstance(0, 0);
+        long maxMapDur = 0;
+        // map tasks
+        for (int i = 0; i < job.getNumberMaps(); i++) {
+          TaskAttemptInfo tai = job.getTaskAttemptInfo(TaskType.MAP, i, 0);
+          RMNode node = nmMap
+              .get(keyAsArray.get(rand.nextInt(keyAsArray.size()))).getNode();
+          String hostname = "/" + node.getRackName() + "/" + node.getHostName();
+          long containerLifeTime = tai.getRuntime();
+          Resource containerResource =
+              Resource.newInstance((int) tai.getTaskInfo().getTaskMemory(),
+                  (int) tai.getTaskInfo().getTaskVCores());
+          containerList.add(new ContainerSimulator(containerResource,
+              containerLifeTime, hostname, DEFAULT_MAPPER_PRIORITY, "map"));
+          maxMapRes = Resources.componentwiseMax(maxMapRes, containerResource);
+          maxMapDur =
+              containerLifeTime > maxMapDur ? containerLifeTime : maxMapDur;
+
+        }
+
+        Resource maxRedRes = Resource.newInstance(0, 0);
+        long maxRedDur = 0;
+        // reduce tasks
+        for (int i = 0; i < job.getNumberReduces(); i++) {
+          TaskAttemptInfo tai = job.getTaskAttemptInfo(TaskType.REDUCE, i, 0);
+          RMNode node = nmMap
+              .get(keyAsArray.get(rand.nextInt(keyAsArray.size()))).getNode();
+          String hostname = "/" + node.getRackName() + "/" + node.getHostName();
+          long containerLifeTime = tai.getRuntime();
+          Resource containerResource =
+              Resource.newInstance((int) tai.getTaskInfo().getTaskMemory(),
+                  (int) tai.getTaskInfo().getTaskVCores());
+          containerList.add(new ContainerSimulator(containerResource,
+              containerLifeTime, hostname, DEFAULT_REDUCER_PRIORITY, "reduce"));
+          maxRedRes = Resources.componentwiseMax(maxRedRes, containerResource);
+          maxRedDur =
+              containerLifeTime > maxRedDur ? containerLifeTime : maxRedDur;
+
+        }
+
+        // generating reservations for the jobs that require them
+
+        ReservationSubmissionRequest rr = null;
+        if (job.hasDeadline()) {
+          ReservationId reservationId =
+              ReservationId.newInstance(this.rm.getStartTime(), AM_ID);
+
+          rr = ReservationClientUtil.createMRReservation(reservationId,
+              "reservation_" + AM_ID, maxMapRes, job.getNumberMaps(), maxMapDur,
+              maxRedRes, job.getNumberReduces(), maxRedDur,
+              now + jobStartTimeMS, now + job.getDeadline(),
+              job.getQueueName());
+
         }
-      } finally {
-        reader.close();
+
+        runNewAM(SLSUtils.DEFAULT_JOB_TYPE, user, jobQueue, oldJobId,
+            jobStartTimeMS, jobFinishTimeMS, containerList, rr,
+            getAMContainerResource(null));
+      }
+    } finally {
+      stjp.close();
+    }
+
+  }
+
+  private Resource getAMContainerResource(Map jsonJob) {
+    Resource amContainerResource =
+        SLSConfiguration.getAMContainerResource(getConf());
+
+    if (jsonJob == null) {
+      return amContainerResource;
+    }
+
+    if (jsonJob.containsKey("am.memory")) {
+      amContainerResource.setMemorySize(
+          Long.parseLong(jsonJob.get("am.memory").toString()));
+    }
+
+    if (jsonJob.containsKey("am.vcores")) {
+      amContainerResource.setVirtualCores(
+          Integer.parseInt(jsonJob.get("am.vcores").toString()));
+    }
+    return amContainerResource;
+  }
+
+  private void increaseQueueAppNum(String queue) throws YarnException {
+    SchedulerWrapper wrapper = (SchedulerWrapper)rm.getResourceScheduler();
+    String queueName = wrapper.getRealQueueName(queue);
+    Integer appNum = queueAppNumMap.get(queueName);
+    if (appNum == null) {
+      appNum = 1;
+    } else {
+      appNum++;
+    }
+
+    queueAppNumMap.put(queueName, appNum);
+  }
+
+  @SuppressWarnings("unchecked")
+  private void runNewAM(String jobType, String user,
+      String jobQueue, String oldJobId, long jobStartTimeMS,
+      long jobFinishTimeMS, List<ContainerSimulator> containerList,
+      ReservationSubmissionRequest rr, Resource amContainerResource) {
+
+    AMSimulator amSim = (AMSimulator) ReflectionUtils.newInstance(
+        amClassMap.get(jobType), new Configuration());
+
+    if (amSim != null) {
+      int heartbeatInterval = getConf().getInt(
+          SLSConfiguration.AM_HEARTBEAT_INTERVAL_MS,
+          SLSConfiguration.AM_HEARTBEAT_INTERVAL_MS_DEFAULT);
+      boolean isTracked = trackedApps.contains(oldJobId);
+
+      if (oldJobId == null) {
+        oldJobId = Integer.toString(AM_ID);
       }
+      AM_ID++;
+
+      amSim.init(heartbeatInterval, containerList, rm, this, jobStartTimeMS,
+          jobFinishTimeMS, user, jobQueue, isTracked, oldJobId, rr,
+          runner.getStartTimeMS(), amContainerResource);
+      runner.schedule(amSim);
+      maxRuntime = Math.max(maxRuntime, jobFinishTimeMS);
+      numTasks += containerList.size();
+      amMap.put(oldJobId, amSim);
     }
   }
-  
+
   private void printSimulationInfo() {
     if (printSimulation) {
       // node
@@ -468,7 +778,7 @@ public class SLSRunner {
       LOG.info("JobId\tQueue\tAMType\tDuration\t#Tasks");
       for (Map.Entry<String, AMSimulator> entry : amMap.entrySet()) {
         AMSimulator am = entry.getValue();
-        LOG.info(entry.getKey() + "\t" + am.getQueue() + "\t" + am.getAMType() 
+        LOG.info(entry.getKey() + "\t" + am.getQueue() + "\t" + am.getAMType()
             + "\t" + am.getDuration() + "\t" + am.getNumTasks());
       }
       LOG.info("------------------------------------");
@@ -502,69 +812,125 @@ public class SLSRunner {
     return nmMap;
   }
 
-  public static TaskRunner getRunner() {
-    return runner;
-  }
-
   public static void decreaseRemainingApps() {
-    remainingApps --;
+    remainingApps--;
 
     if (remainingApps == 0) {
       LOG.info("SLSRunner tears down.");
-      System.exit(0);
+      if (exitAtTheFinish) {
+        System.exit(0);
+      }
     }
   }
 
-  public static void main(String args[]) throws Exception {
+  public void stop() throws InterruptedException {
+    rm.stop();
+    runner.stop();
+  }
+
+  public int run(final String[] argv) throws IOException, InterruptedException,
+      ParseException, ClassNotFoundException, YarnException {
+
     Options options = new Options();
+
+    // Left for compatibility
     options.addOption("inputrumen", true, "input rumen files");
     options.addOption("inputsls", true, "input sls files");
+
+    // New more general format
+    options.addOption("tracetype", true, "the type of trace");
+    options.addOption("tracelocation", true, "input trace files");
+
     options.addOption("nodes", true, "input topology");
     options.addOption("output", true, "output directory");
     options.addOption("trackjobs", true,
-            "jobs to be tracked during simulating");
+        "jobs to be tracked during simulating");
     options.addOption("printsimulation", false,
-            "print out simulation information");
-    
+        "print out simulation information");
+
     CommandLineParser parser = new GnuParser();
-    CommandLine cmd = parser.parse(options, args);
+    CommandLine cmd = parser.parse(options, argv);
+
+    String traceType = null;
+    String traceLocation = null;
+
+    // compatibility with old commandline
+    if (cmd.hasOption("inputrumen")) {
+      traceType = "RUMEN";
+      traceLocation = cmd.getOptionValue("inputrumen");
+    }
+    if (cmd.hasOption("inputsls")) {
+      traceType = "SLS";
+      traceLocation = cmd.getOptionValue("inputsls");
+    }
+
+    if (cmd.hasOption("tracetype")) {
+      traceType = cmd.getOptionValue("tracetype");
+      traceLocation = cmd.getOptionValue("tracelocation");
+    }
 
-    String inputRumen = cmd.getOptionValue("inputrumen");
-    String inputSLS = cmd.getOptionValue("inputsls");
     String output = cmd.getOptionValue("output");
-    
-    if ((inputRumen == null && inputSLS == null) || output == null) {
-      System.err.println();
-      System.err.println("ERROR: Missing input or output file");
-      System.err.println();
-      System.err.println("Options: -inputrumen|-inputsls FILE,FILE... " +
-              "-output FILE [-nodes FILE] [-trackjobs JobId,JobId...] " +
-              "[-printsimulation]");
-      System.err.println();
-      System.exit(1);
-    }
-    
+
     File outputFile = new File(output);
-    if (! outputFile.exists()
-            && ! outputFile.mkdirs()) {
+    if (!outputFile.exists() && !outputFile.mkdirs()) {
       System.err.println("ERROR: Cannot create output directory "
-              + outputFile.getAbsolutePath());
-      System.exit(1);
+          + outputFile.getAbsolutePath());
+      throw new YarnException("Cannot create output directory");
     }
-    
+
     Set<String> trackedJobSet = new HashSet<String>();
     if (cmd.hasOption("trackjobs")) {
       String trackjobs = cmd.getOptionValue("trackjobs");
       String jobIds[] = trackjobs.split(",");
       trackedJobSet.addAll(Arrays.asList(jobIds));
     }
-    
-    String nodeFile = cmd.hasOption("nodes") ? cmd.getOptionValue("nodes") : "";
 
-    boolean isSLS = inputSLS != null;
-    String inputFiles[] = isSLS ? inputSLS.split(",") : inputRumen.split(",");
-    SLSRunner sls = new SLSRunner(isSLS, inputFiles, nodeFile, output,
+    String tempNodeFile =
+        cmd.hasOption("nodes") ? cmd.getOptionValue("nodes") : "";
+
+    TraceType tempTraceType = TraceType.SLS;
+    switch (traceType) {
+    case "SLS":
+      tempTraceType = TraceType.SLS;
+      break;
+    case "RUMEN":
+      tempTraceType = TraceType.RUMEN;
+      break;
+
+    case "SYNTH":
+      tempTraceType = TraceType.SYNTH;
+      break;
+    default:
+      printUsage();
+      throw new YarnException("Misconfigured input");
+    }
+
+    String[] inputFiles = traceLocation.split(",");
+
+    setSimulationParams(tempTraceType, inputFiles, tempNodeFile, output,
         trackedJobSet, cmd.hasOption("printsimulation"));
-    sls.start();
+
+    start();
+
+    return 0;
   }
+
+  public static void main(String[] argv) throws Exception {
+    exitAtTheFinish = true;
+    ToolRunner.run(new Configuration(), new SLSRunner(), argv);
+  }
+
+  static void printUsage() {
+    System.err.println();
+    System.err.println("ERROR: Wrong tracetype");
+    System.err.println();
+    System.err.println(
+        "Options: -tracetype " + "SLS|RUMEN|SYNTH -tracelocation FILE,FILE... "
+            + "(deprecated alternative options --inputsls FILE, FILE,... "
+            + " | --inputrumen FILE,FILE,...)"
+            + "-output FILE [-nodes FILE] [-trackjobs JobId,JobId...] "
+            + "[-printsimulation]");
+    System.err.println();
+  }
+
 }

+ 164 - 95
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/AMSimulator.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.sls.appmaster;
 
 import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
 import java.nio.ByteBuffer;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
@@ -34,25 +35,24 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
-import org.apache.hadoop.yarn.api.protocolrecords
-        .FinishApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
-
-import org.apache.hadoop.yarn.api.protocolrecords
-        .RegisterApplicationMasterRequest;
-import org.apache.hadoop.yarn.api.protocolrecords
-        .RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -60,12 +60,8 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
+import org.apache.hadoop.yarn.sls.scheduler.SchedulerMetrics;
 import org.apache.hadoop.yarn.util.Records;
-
 import org.apache.hadoop.yarn.sls.scheduler.ContainerSimulator;
 import org.apache.hadoop.yarn.sls.scheduler.SchedulerWrapper;
 import org.apache.hadoop.yarn.sls.SLSRunner;
@@ -90,7 +86,7 @@ public abstract class AMSimulator extends TaskRunner.Task {
           RecordFactoryProvider.getRecordFactory(null);
   // response queue
   protected final BlockingQueue<AllocateResponse> responseQueue;
-  protected int RESPONSE_ID = 1;
+  private int responseId = 0;
   // user name
   protected String user;  
   // queue name
@@ -98,6 +94,7 @@ public abstract class AMSimulator extends TaskRunner.Task {
   // am type
   protected String amtype;
   // job start/end time
+  private long baselineTimeMS;
   protected long traceStartTimeMS;
   protected long traceFinishTimeMS;
   protected long simulateStartTimeMS;
@@ -107,28 +104,41 @@ public abstract class AMSimulator extends TaskRunner.Task {
   // progress
   protected int totalContainers;
   protected int finishedContainers;
+
+  // waiting for AM container
+  volatile boolean isAMContainerRunning = false;
+  volatile Container amContainer;
   
-  protected final Logger LOG = LoggerFactory.getLogger(AMSimulator.class);
+  private static final Logger LOG = LoggerFactory.getLogger(AMSimulator.class);
+
+  private Resource amContainerResource;
+
+  private ReservationSubmissionRequest reservationRequest;
 
   public AMSimulator() {
-    this.responseQueue = new LinkedBlockingQueue<AllocateResponse>();
+    this.responseQueue = new LinkedBlockingQueue<>();
   }
 
-  public void init(int id, int heartbeatInterval, 
-      List<ContainerSimulator> containerList, ResourceManager rm, SLSRunner se,
-      long traceStartTime, long traceFinishTime, String user, String queue, 
-      boolean isTracked, String oldAppId) {
-    super.init(traceStartTime, traceStartTime + 1000000L * heartbeatInterval,
-            heartbeatInterval);
-    this.user = user;
-    this.rm = rm;
-    this.se = se;
-    this.user = user;
-    this.queue = queue;
-    this.oldAppId = oldAppId;
-    this.isTracked = isTracked;
-    this.traceStartTimeMS = traceStartTime;
-    this.traceFinishTimeMS = traceFinishTime;
+  @SuppressWarnings("checkstyle:parameternumber")
+  public void init(int heartbeatInterval,
+      List<ContainerSimulator> containerList, ResourceManager resourceManager,
+      SLSRunner slsRunnner, long startTime, long finishTime, String simUser,
+      String simQueue, boolean tracked, String oldApp,
+      ReservationSubmissionRequest rr, long baseTimeMS,
+      Resource amContainerResource) {
+    super.init(startTime, startTime + 1000000L * heartbeatInterval,
+        heartbeatInterval);
+    this.user = simUser;
+    this.rm = resourceManager;
+    this.se = slsRunnner;
+    this.queue = simQueue;
+    this.oldAppId = oldApp;
+    this.isTracked = tracked;
+    this.baselineTimeMS = baseTimeMS;
+    this.traceStartTimeMS = startTime;
+    this.traceFinishTimeMS = finishTime;
+    this.reservationRequest = rr;
+    this.amContainerResource = amContainerResource;
   }
 
   /**
@@ -136,29 +146,66 @@ public abstract class AMSimulator extends TaskRunner.Task {
    */
   @Override
   public void firstStep() throws Exception {
-    simulateStartTimeMS = System.currentTimeMillis() - 
-                          SLSRunner.getRunner().getStartTimeMS();
+    simulateStartTimeMS = System.currentTimeMillis() - baselineTimeMS;
 
-    // submit application, waiting until ACCEPTED
-    submitApp();
+    ReservationId reservationId = null;
 
-    // register application master
-    registerAM();
+    // submit a reservation if one is required, exceptions naturally happen
+    // when the reservation does not fit, catch, log, and move on running job
+    // without reservation.
+    try {
+      reservationId = submitReservationWhenSpecified();
+    } catch (UndeclaredThrowableException y) {
+      LOG.warn("Unable to place reservation: " + y.getMessage());
+    }
+
+    // submit application, waiting until ACCEPTED
+    submitApp(reservationId);
 
     // track app metrics
     trackApp();
   }
 
+  public synchronized void notifyAMContainerLaunched(Container masterContainer)
+      throws Exception {
+    this.amContainer = masterContainer;
+    this.appAttemptId = masterContainer.getId().getApplicationAttemptId();
+    registerAM();
+    isAMContainerRunning = true;
+  }
+
+  private ReservationId submitReservationWhenSpecified()
+      throws IOException, InterruptedException {
+    if (reservationRequest != null) {
+      UserGroupInformation ugi = UserGroupInformation.createRemoteUser(user);
+      ugi.doAs(new PrivilegedExceptionAction<Object>() {
+        @Override
+        public Object run() throws YarnException, IOException {
+          rm.getClientRMService().submitReservation(reservationRequest);
+          LOG.info("RESERVATION SUCCESSFULLY SUBMITTED "
+              + reservationRequest.getReservationId());
+          return null;
+
+        }
+      });
+      return reservationRequest.getReservationId();
+    } else {
+      return null;
+    }
+  }
+
   @Override
   public void middleStep() throws Exception {
-    // process responses in the queue
-    processResponseQueue();
-    
-    // send out request
-    sendContainerRequest();
-    
-    // check whether finish
-    checkStop();
+    if (isAMContainerRunning) {
+      // process responses in the queue
+      processResponseQueue();
+
+      // send out request
+      sendContainerRequest();
+
+      // check whether finish
+      checkStop();
+    }
   }
 
   @Override
@@ -168,6 +215,22 @@ public abstract class AMSimulator extends TaskRunner.Task {
     if (isTracked) {
       untrackApp();
     }
+
+    // Finish AM container
+    if (amContainer != null) {
+      LOG.info("AM container = {} reported to finish", amContainer.getId());
+      se.getNmMap().get(amContainer.getNodeId()).cleanupContainer(
+          amContainer.getId());
+    } else {
+      LOG.info("AM container is null");
+    }
+
+    if (null == appAttemptId) {
+      // If appAttemptId == null, AM is not launched from RM's perspective, so
+      // it's unnecessary to finish am as well
+      return;
+    }
+
     // unregister application master
     final FinishApplicationMasterRequest finishAMRequest = recordFactory
                   .newRecordInstance(FinishApplicationMasterRequest.class);
@@ -187,13 +250,14 @@ public abstract class AMSimulator extends TaskRunner.Task {
       }
     });
 
-    simulateFinishTimeMS = System.currentTimeMillis() -
-        SLSRunner.getRunner().getStartTimeMS();
+    simulateFinishTimeMS = System.currentTimeMillis() - baselineTimeMS;
     // record job running information
-    ((SchedulerWrapper)rm.getResourceScheduler())
-         .addAMRuntime(appId, 
-                      traceStartTimeMS, traceFinishTimeMS, 
-                      simulateStartTimeMS, simulateFinishTimeMS);
+    SchedulerMetrics schedulerMetrics =
+            ((SchedulerWrapper)rm.getResourceScheduler()).getSchedulerMetrics();
+    if (schedulerMetrics != null) {
+      schedulerMetrics.addAMRuntime(appId, traceStartTimeMS, traceFinishTimeMS,
+              simulateStartTimeMS, simulateFinishTimeMS);
+    }
   }
   
   protected ResourceRequest createResourceRequest(
@@ -213,7 +277,7 @@ public abstract class AMSimulator extends TaskRunner.Task {
       List<ContainerId> toRelease) {
     AllocateRequest allocateRequest =
             recordFactory.newRecordInstance(AllocateRequest.class);
-    allocateRequest.setResponseId(RESPONSE_ID ++);
+    allocateRequest.setResponseId(responseId++);
     allocateRequest.setAskList(ask);
     allocateRequest.setReleaseList(toRelease);
     return allocateRequest;
@@ -229,7 +293,7 @@ public abstract class AMSimulator extends TaskRunner.Task {
   
   protected abstract void checkStop();
   
-  private void submitApp()
+  private void submitApp(ReservationId reservationId)
           throws YarnException, InterruptedException, IOException {
     // ask for new application
     GetNewApplicationRequest newAppRequest =
@@ -249,14 +313,19 @@ public abstract class AMSimulator extends TaskRunner.Task {
     appSubContext.setPriority(Priority.newInstance(0));
     ContainerLaunchContext conLauContext = 
         Records.newRecord(ContainerLaunchContext.class);
-    conLauContext.setApplicationACLs(
-        new HashMap<ApplicationAccessType, String>());
+    conLauContext
+        .setApplicationACLs(new HashMap<ApplicationAccessType, String>());
     conLauContext.setCommands(new ArrayList<String>());
     conLauContext.setEnvironment(new HashMap<String, String>());
     conLauContext.setLocalResources(new HashMap<String, LocalResource>());
     conLauContext.setServiceData(new HashMap<String, ByteBuffer>());
     appSubContext.setAMContainerSpec(conLauContext);
-    appSubContext.setUnmanagedAM(true);
+    appSubContext.setResource(amContainerResource);
+
+    if(reservationId != null) {
+      appSubContext.setReservationID(reservationId);
+    }
+
     subAppRequest.setApplicationSubmissionContext(appSubContext);
     UserGroupInformation ugi = UserGroupInformation.createRemoteUser(user);
     ugi.doAs(new PrivilegedExceptionAction<Object>() {
@@ -267,22 +336,6 @@ public abstract class AMSimulator extends TaskRunner.Task {
       }
     });
     LOG.info("Submit a new application {}", appId);
-    
-    // waiting until application ACCEPTED
-    RMApp app = rm.getRMContext().getRMApps().get(appId);
-    while(app.getState() != RMAppState.ACCEPTED) {
-      Thread.sleep(10);
-    }
-
-    // Waiting until application attempt reach LAUNCHED
-    // "Unmanaged AM must register after AM attempt reaches LAUNCHED state"
-    this.appAttemptId = rm.getRMContext().getRMApps().get(appId)
-        .getCurrentAppAttempt().getAppAttemptId();
-    RMAppAttempt rmAppAttempt = rm.getRMContext().getRMApps().get(appId)
-        .getCurrentAppAttempt();
-    while (rmAppAttempt.getAppAttemptState() != RMAppAttemptState.LAUNCHED) {
-      Thread.sleep(10);
-    }
   }
 
   private void registerAM()
@@ -314,14 +367,20 @@ public abstract class AMSimulator extends TaskRunner.Task {
 
   private void trackApp() {
     if (isTracked) {
-      ((SchedulerWrapper) rm.getResourceScheduler())
-              .addTrackedApp(appAttemptId, oldAppId);
+      SchedulerMetrics schedulerMetrics =
+          ((SchedulerWrapper)rm.getResourceScheduler()).getSchedulerMetrics();
+      if (schedulerMetrics != null) {
+        schedulerMetrics.addTrackedApp(appId, oldAppId);
+      }
     }
   }
   public void untrackApp() {
     if (isTracked) {
-      ((SchedulerWrapper) rm.getResourceScheduler())
-              .removeTrackedApp(appAttemptId, oldAppId);
+      SchedulerMetrics schedulerMetrics =
+          ((SchedulerWrapper)rm.getResourceScheduler()).getSchedulerMetrics();
+      if (schedulerMetrics != null) {
+        schedulerMetrics.removeTrackedApp(oldAppId);
+      }
     }
   }
   
@@ -332,26 +391,28 @@ public abstract class AMSimulator extends TaskRunner.Task {
     Map<String, ResourceRequest> nodeLocalRequestMap = new HashMap<String, ResourceRequest>();
     ResourceRequest anyRequest = null;
     for (ContainerSimulator cs : csList) {
-      String rackHostNames[] = SLSUtils.getRackHostName(cs.getHostname());
-      // check rack local
-      String rackname = rackHostNames[0];
-      if (rackLocalRequestMap.containsKey(rackname)) {
-        rackLocalRequestMap.get(rackname).setNumContainers(
-            rackLocalRequestMap.get(rackname).getNumContainers() + 1);
-      } else {
-        ResourceRequest request = createResourceRequest(
-                cs.getResource(), rackname, priority, 1);
-        rackLocalRequestMap.put(rackname, request);
-      }
-      // check node local
-      String hostname = rackHostNames[1];
-      if (nodeLocalRequestMap.containsKey(hostname)) {
-        nodeLocalRequestMap.get(hostname).setNumContainers(
-            nodeLocalRequestMap.get(hostname).getNumContainers() + 1);
-      } else {
-        ResourceRequest request = createResourceRequest(
-                cs.getResource(), hostname, priority, 1);
-        nodeLocalRequestMap.put(hostname, request);
+      if (cs.getHostname() != null) {
+        String[] rackHostNames = SLSUtils.getRackHostName(cs.getHostname());
+        // check rack local
+        String rackname = "/" + rackHostNames[0];
+        if (rackLocalRequestMap.containsKey(rackname)) {
+          rackLocalRequestMap.get(rackname).setNumContainers(
+              rackLocalRequestMap.get(rackname).getNumContainers() + 1);
+        } else {
+          ResourceRequest request =
+              createResourceRequest(cs.getResource(), rackname, priority, 1);
+          rackLocalRequestMap.put(rackname, request);
+        }
+        // check node local
+        String hostname = rackHostNames[1];
+        if (nodeLocalRequestMap.containsKey(hostname)) {
+          nodeLocalRequestMap.get(hostname).setNumContainers(
+              nodeLocalRequestMap.get(hostname).getNumContainers() + 1);
+        } else {
+          ResourceRequest request =
+              createResourceRequest(cs.getResource(), hostname, priority, 1);
+          nodeLocalRequestMap.put(hostname, request);
+        }
       }
       // any
       if (anyRequest == null) {
@@ -382,4 +443,12 @@ public abstract class AMSimulator extends TaskRunner.Task {
   public int getNumTasks() {
     return totalContainers;
   }
+
+  public ApplicationId getApplicationId() {
+    return appId;
+  }
+
+  public ApplicationAttemptId getApplicationAttemptId() {
+    return appAttemptId;
+  }
 }

+ 99 - 131
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/MRAMSimulator.java

@@ -32,7 +32,9 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
@@ -40,8 +42,6 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
-import org.apache.hadoop.yarn.server.utils.BuilderUtils;
-
 import org.apache.hadoop.yarn.sls.scheduler.ContainerSimulator;
 import org.apache.hadoop.yarn.sls.SLSRunner;
 import org.slf4j.Logger;
@@ -63,10 +63,10 @@ public class MRAMSimulator extends AMSimulator {
   
   private static final int PRIORITY_REDUCE = 10;
   private static final int PRIORITY_MAP = 20;
-  
+
   // pending maps
   private LinkedList<ContainerSimulator> pendingMaps =
-          new LinkedList<ContainerSimulator>();
+          new LinkedList<>();
   
   // pending failed maps
   private LinkedList<ContainerSimulator> pendingFailedMaps =
@@ -107,106 +107,55 @@ public class MRAMSimulator extends AMSimulator {
   private int mapTotal = 0;
   private int reduceFinished = 0;
   private int reduceTotal = 0;
-  // waiting for AM container 
-  private boolean isAMContainerRunning = false;
-  private Container amContainer;
+
   // finished
   private boolean isFinished = false;
-  // resource for AM container
-  private final static int MR_AM_CONTAINER_RESOURCE_MEMORY_MB = 1024;
-  private final static int MR_AM_CONTAINER_RESOURCE_VCORES = 1;
 
-  public final Logger LOG = LoggerFactory.getLogger(MRAMSimulator.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(MRAMSimulator.class);
 
-  public void init(int id, int heartbeatInterval,
+  @SuppressWarnings("checkstyle:parameternumber")
+  public void init(int heartbeatInterval,
       List<ContainerSimulator> containerList, ResourceManager rm, SLSRunner se,
       long traceStartTime, long traceFinishTime, String user, String queue, 
-      boolean isTracked, String oldAppId) {
-    super.init(id, heartbeatInterval, containerList, rm, se, 
-              traceStartTime, traceFinishTime, user, queue,
-              isTracked, oldAppId);
+      boolean isTracked, String oldAppId, ReservationSubmissionRequest rr,
+      long baselineStartTimeMS, Resource amContainerResource) {
+    super.init(heartbeatInterval, containerList, rm, se,
+        traceStartTime, traceFinishTime, user, queue, isTracked, oldAppId,
+        rr, baselineStartTimeMS, amContainerResource);
     amtype = "mapreduce";
     
     // get map/reduce tasks
     for (ContainerSimulator cs : containerList) {
       if (cs.getType().equals("map")) {
         cs.setPriority(PRIORITY_MAP);
-        pendingMaps.add(cs);
+        allMaps.add(cs);
       } else if (cs.getType().equals("reduce")) {
         cs.setPriority(PRIORITY_REDUCE);
-        pendingReduces.add(cs);
+        allReduces.add(cs);
       }
     }
-    allMaps.addAll(pendingMaps);
-    allReduces.addAll(pendingReduces);
-    mapTotal = pendingMaps.size();
-    reduceTotal = pendingReduces.size();
+
+    LOG.info("Added new job with {} mapper and {} reducers",
+        allMaps.size(), allReduces.size());
+
+    mapTotal = allMaps.size();
+    reduceTotal = allReduces.size();
     totalContainers = mapTotal + reduceTotal;
   }
 
   @Override
-  public void firstStep() throws Exception {
-    super.firstStep();
-    
-    requestAMContainer();
-  }
-
-  /**
-   * send out request for AM container
-   */
-  protected void requestAMContainer()
-          throws YarnException, IOException, InterruptedException {
-    List<ResourceRequest> ask = new ArrayList<ResourceRequest>();
-    ResourceRequest amRequest = createResourceRequest(
-            BuilderUtils.newResource(MR_AM_CONTAINER_RESOURCE_MEMORY_MB,
-                    MR_AM_CONTAINER_RESOURCE_VCORES),
-            ResourceRequest.ANY, 1, 1);
-    ask.add(amRequest);
-    LOG.debug("Application {} sends out allocate request for its AM", appId);
-    final AllocateRequest request = this.createAllocateRequest(ask);
-
-    UserGroupInformation ugi =
-            UserGroupInformation.createRemoteUser(appAttemptId.toString());
-    Token<AMRMTokenIdentifier> token = rm.getRMContext().getRMApps()
-            .get(appAttemptId.getApplicationId())
-            .getRMAppAttempt(appAttemptId).getAMRMToken();
-    ugi.addTokenIdentifier(token.decodeIdentifier());
-    AllocateResponse response = ugi.doAs(
-            new PrivilegedExceptionAction<AllocateResponse>() {
-      @Override
-      public AllocateResponse run() throws Exception {
-        return rm.getApplicationMasterService().allocate(request);
-      }
-    });
-    if (response != null) {
-      responseQueue.put(response);
+  public synchronized void notifyAMContainerLaunched(Container masterContainer)
+      throws Exception {
+    if (null != masterContainer) {
+      restart();
+      super.notifyAMContainerLaunched(masterContainer);
     }
   }
 
   @Override
   @SuppressWarnings("unchecked")
-  protected void processResponseQueue()
-          throws InterruptedException, YarnException, IOException {
-    // Check whether receive the am container
-    if (!isAMContainerRunning) {
-      if (!responseQueue.isEmpty()) {
-        AllocateResponse response = responseQueue.take();
-        if (response != null
-            && !response.getAllocatedContainers().isEmpty()) {
-          // Get AM container
-          Container container = response.getAllocatedContainers().get(0);
-          se.getNmMap().get(container.getNodeId())
-              .addNewContainer(container, -1L);
-          // Start AM container
-          amContainer = container;
-          LOG.debug("Application {} starts its AM container ({}).", appId,
-              amContainer.getId());
-          isAMContainerRunning = true;
-        }
-      }
-      return;
-    }
-
+  protected void processResponseQueue() throws Exception {
     while (! responseQueue.isEmpty()) {
       AllocateResponse response = responseQueue.take();
 
@@ -227,11 +176,15 @@ public class MRAMSimulator extends AMSimulator {
               assignedReduces.remove(containerId);
               reduceFinished ++;
               finishedContainers ++;
-            } else {
+            } else if (amContainer.getId().equals(containerId)){
               // am container released event
               isFinished = true;
               LOG.info("Application {} goes to finish.", appId);
             }
+
+            if (mapFinished >= mapTotal && reduceFinished >= reduceTotal) {
+              lastStep();
+            }
           } else {
             // container to be killed
             if (assignedMaps.containsKey(containerId)) {
@@ -242,10 +195,9 @@ public class MRAMSimulator extends AMSimulator {
               LOG.debug("Application {} has one reducer killed ({}).",
                   appId, containerId);
               pendingFailedReduces.add(assignedReduces.remove(containerId));
-            } else {
-              LOG.info("Application {}'s AM is going to be killed." +
-                  " Restarting...", appId);
-              restart();
+            } else if (amContainer.getId().equals(containerId)){
+              LOG.info("Application {}'s AM is " +
+                  "going to be killed. Waiting for rescheduling...", appId);
             }
           }
         }
@@ -253,11 +205,8 @@ public class MRAMSimulator extends AMSimulator {
       
       // check finished
       if (isAMContainerRunning &&
-              (mapFinished == mapTotal) &&
-              (reduceFinished == reduceTotal)) {
-        // to release the AM container
-        se.getNmMap().get(amContainer.getNodeId())
-                .cleanupContainer(amContainer.getId());
+              (mapFinished >= mapTotal) &&
+              (reduceFinished >= reduceTotal)) {
         isAMContainerRunning = false;
         LOG.debug("Application {} sends out event to clean up"
             + " its AM container.", appId);
@@ -291,21 +240,38 @@ public class MRAMSimulator extends AMSimulator {
    */
   private void restart()
           throws YarnException, IOException, InterruptedException {
-    // clear 
-    finishedContainers = 0;
+    // clear
     isFinished = false;
-    mapFinished = 0;
-    reduceFinished = 0;
     pendingFailedMaps.clear();
     pendingMaps.clear();
     pendingReduces.clear();
     pendingFailedReduces.clear();
-    pendingMaps.addAll(allMaps);
-    pendingReduces.addAll(pendingReduces);
-    isAMContainerRunning = false;
+
+    // Only add totalMaps - finishedMaps
+    int added = 0;
+    for (ContainerSimulator cs : allMaps) {
+      if (added >= mapTotal - mapFinished) {
+        break;
+      }
+      pendingMaps.add(cs);
+    }
+
+    // And same, only add totalReduces - finishedReduces
+    added = 0;
+    for (ContainerSimulator cs : allReduces) {
+      if (added >= reduceTotal - reduceFinished) {
+        break;
+      }
+      pendingReduces.add(cs);
+    }
     amContainer = null;
-    // resent am container request
-    requestAMContainer();
+  }
+
+  private List<ContainerSimulator> mergeLists(List<ContainerSimulator> left, List<ContainerSimulator> right) {
+    List<ContainerSimulator> list = new ArrayList<>();
+    list.addAll(left);
+    list.addAll(right);
+    return list;
   }
 
   @Override
@@ -317,42 +283,44 @@ public class MRAMSimulator extends AMSimulator {
 
     // send out request
     List<ResourceRequest> ask = null;
-    if (isAMContainerRunning) {
-      if (mapFinished != mapTotal) {
-        // map phase
-        if (! pendingMaps.isEmpty()) {
-          ask = packageRequests(pendingMaps, PRIORITY_MAP);
-          LOG.debug("Application {} sends out request for {} mappers.",
-              appId, pendingMaps.size());
-          scheduledMaps.addAll(pendingMaps);
-          pendingMaps.clear();
-        } else if (! pendingFailedMaps.isEmpty() && scheduledMaps.isEmpty()) {
-          ask = packageRequests(pendingFailedMaps, PRIORITY_MAP);
-          LOG.debug("Application {} sends out requests for {} failed mappers.",
-              appId, pendingFailedMaps.size());
-          scheduledMaps.addAll(pendingFailedMaps);
-          pendingFailedMaps.clear();
-        }
-      } else if (reduceFinished != reduceTotal) {
-        // reduce phase
-        if (! pendingReduces.isEmpty()) {
-          ask = packageRequests(pendingReduces, PRIORITY_REDUCE);
-          LOG.debug("Application {} sends out requests for {} reducers.",
-              appId, pendingReduces.size());
-          scheduledReduces.addAll(pendingReduces);
-          pendingReduces.clear();
-        } else if (! pendingFailedReduces.isEmpty()
-                && scheduledReduces.isEmpty()) {
-          ask = packageRequests(pendingFailedReduces, PRIORITY_REDUCE);
-          LOG.debug("Application {} sends out request for {} failed reducers.",
-              appId, pendingFailedReduces.size());
-          scheduledReduces.addAll(pendingFailedReduces);
-          pendingFailedReduces.clear();
-        }
+    if (mapFinished != mapTotal) {
+      // map phase
+      if (!pendingMaps.isEmpty()) {
+        ask = packageRequests(mergeLists(pendingMaps, scheduledMaps),
+            PRIORITY_MAP);
+        LOG.debug("Application {} sends out request for {} mappers.",
+            appId, pendingMaps.size());
+        scheduledMaps.addAll(pendingMaps);
+        pendingMaps.clear();
+      } else if (!pendingFailedMaps.isEmpty()) {
+        ask = packageRequests(mergeLists(pendingFailedMaps, scheduledMaps),
+            PRIORITY_MAP);
+        LOG.debug("Application {} sends out requests for {} failed mappers.",
+            appId, pendingFailedMaps.size());
+        scheduledMaps.addAll(pendingFailedMaps);
+        pendingFailedMaps.clear();
+      }
+    } else if (reduceFinished != reduceTotal) {
+      // reduce phase
+      if (!pendingReduces.isEmpty()) {
+        ask = packageRequests(mergeLists(pendingReduces, scheduledReduces),
+            PRIORITY_REDUCE);
+        LOG.debug("Application {} sends out requests for {} reducers.",
+                appId, pendingReduces.size());
+        scheduledReduces.addAll(pendingReduces);
+        pendingReduces.clear();
+      } else if (!pendingFailedReduces.isEmpty()) {
+        ask = packageRequests(mergeLists(pendingFailedReduces, scheduledReduces),
+            PRIORITY_REDUCE);
+        LOG.debug("Application {} sends out request for {} failed reducers.",
+            appId, pendingFailedReduces.size());
+        scheduledReduces.addAll(pendingFailedReduces);
+        pendingFailedReduces.clear();
       }
     }
+
     if (ask == null) {
-      ask = new ArrayList<ResourceRequest>();
+      ask = new ArrayList<>();
     }
     
     final AllocateRequest request = createAllocateRequest(ask);

+ 15 - 0
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/conf/SLSConfiguration.java

@@ -20,6 +20,8 @@ package org.apache.hadoop.yarn.sls.conf;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.Resource;
 
 @Private
 @Unstable
@@ -62,6 +64,14 @@ public class SLSConfiguration {
   public static final int AM_HEARTBEAT_INTERVAL_MS_DEFAULT = 1000;
   public static final String AM_TYPE = AM_PREFIX + "type.";
 
+  public static final String AM_CONTAINER_MEMORY = AM_PREFIX +
+      "container.memory";
+  public static final int AM_CONTAINER_MEMORY_DEFAULT = 1024;
+
+  public static final String AM_CONTAINER_VCORES = AM_PREFIX +
+      "container.vcores";
+  public static final int AM_CONTAINER_VCORES_DEFAULT = 1;
+
   // container
   public static final String CONTAINER_PREFIX = PREFIX + "container.";
   public static final String CONTAINER_MEMORY_MB = CONTAINER_PREFIX
@@ -70,4 +80,9 @@ public class SLSConfiguration {
   public static final String CONTAINER_VCORES = CONTAINER_PREFIX + "vcores";
   public static final int CONTAINER_VCORES_DEFAULT = 1;
 
+  public static Resource getAMContainerResource(Configuration conf) {
+    return Resource.newInstance(
+        conf.getLong(AM_CONTAINER_MEMORY, AM_CONTAINER_MEMORY_DEFAULT),
+        conf.getInt(AM_CONTAINER_VCORES, AM_CONTAINER_VCORES_DEFAULT));
+  }
 }

+ 0 - 1
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NMSimulator.java

@@ -19,7 +19,6 @@
 package org.apache.hadoop.yarn.sls.nodemanager;
 
 import java.io.IOException;
-import java.text.MessageFormat;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;

+ 120 - 0
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/resourcemanager/MockAMLauncher.java

@@ -0,0 +1,120 @@
+/**
+ * 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.yarn.sls.resourcemanager;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
+import org.apache.hadoop.yarn.sls.SLSRunner;
+import org.apache.hadoop.yarn.sls.appmaster.AMSimulator;
+
+import java.util.Map;
+
+public class MockAMLauncher extends ApplicationMasterLauncher
+    implements EventHandler<AMLauncherEvent> {
+  private static final Log LOG = LogFactory.getLog(
+      MockAMLauncher.class);
+
+  Map<String, AMSimulator> amMap;
+  SLSRunner se;
+
+  public MockAMLauncher(SLSRunner se, RMContext rmContext,
+      Map<String, AMSimulator> amMap) {
+    super(rmContext);
+    this.amMap = amMap;
+    this.se = se;
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    // Do nothing
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    // Do nothing
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    // Do nothing
+  }
+
+  private void setupAMRMToken(RMAppAttempt appAttempt) {
+    // Setup AMRMToken
+    Token<AMRMTokenIdentifier> amrmToken =
+        super.context.getAMRMTokenSecretManager().createAndGetAMRMToken(
+            appAttempt.getAppAttemptId());
+    ((RMAppAttemptImpl) appAttempt).setAMRMToken(amrmToken);
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public void handle(AMLauncherEvent event) {
+    if (AMLauncherEventType.LAUNCH == event.getType()) {
+      ApplicationId appId =
+          event.getAppAttempt().getAppAttemptId().getApplicationId();
+
+      // find AMSimulator
+      for (AMSimulator ams : amMap.values()) {
+        if (ams.getApplicationId() != null && ams.getApplicationId().equals(
+            appId)) {
+          try {
+            Container amContainer = event.getAppAttempt().getMasterContainer();
+
+            setupAMRMToken(event.getAppAttempt());
+
+            // Notify RMAppAttempt to change state
+            super.context.getDispatcher().getEventHandler().handle(
+                new RMAppAttemptEvent(event.getAppAttempt().getAppAttemptId(),
+                    RMAppAttemptEventType.LAUNCHED));
+
+            ams.notifyAMContainerLaunched(
+                event.getAppAttempt().getMasterContainer());
+            LOG.info("Notify AM launcher launched:" + amContainer.getId());
+
+            se.getNmMap().get(amContainer.getNodeId())
+                .addNewContainer(amContainer, 100000000L);
+
+            return;
+          } catch (Exception e) {
+            throw new YarnRuntimeException(e);
+          }
+        }
+      }
+
+      throw new YarnRuntimeException(
+          "Didn't find any AMSimulator for applicationId=" + appId);
+    }
+  }
+}

+ 142 - 186
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/FairSchedulerMetrics.java

@@ -18,16 +18,17 @@
 
 package org.apache.hadoop.yarn.sls.scheduler;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair
-    .FSAppAttempt;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.Schedulable;
+import org.apache.hadoop.yarn.sls.SLSRunner;
 
 import com.codahale.metrics.Gauge;
-import org.apache.hadoop.yarn.sls.SLSRunner;
 
 @Private
 @Unstable
@@ -37,114 +38,131 @@ public class FairSchedulerMetrics extends SchedulerMetrics {
   private int totalVCores = Integer.MAX_VALUE;
   private boolean maxReset = false;
 
+  @VisibleForTesting
+  public enum Metric {
+    DEMAND("demand"),
+    USAGE("usage"),
+    MINSHARE("minshare"),
+    MAXSHARE("maxshare"),
+    FAIRSHARE("fairshare");
+
+    private String value;
+
+    Metric(String value) {
+      this.value = value;
+    }
+
+    @VisibleForTesting
+    public String getValue() {
+      return value;
+    }
+  }
+
   public FairSchedulerMetrics() {
     super();
-    appTrackedMetrics.add("demand.memory");
-    appTrackedMetrics.add("demand.vcores");
-    appTrackedMetrics.add("usage.memory");
-    appTrackedMetrics.add("usage.vcores");
-    appTrackedMetrics.add("minshare.memory");
-    appTrackedMetrics.add("minshare.vcores");
-    appTrackedMetrics.add("maxshare.memory");
-    appTrackedMetrics.add("maxshare.vcores");
-    appTrackedMetrics.add("fairshare.memory");
-    appTrackedMetrics.add("fairshare.vcores");
-    queueTrackedMetrics.add("demand.memory");
-    queueTrackedMetrics.add("demand.vcores");
-    queueTrackedMetrics.add("usage.memory");
-    queueTrackedMetrics.add("usage.vcores");
-    queueTrackedMetrics.add("minshare.memory");
-    queueTrackedMetrics.add("minshare.vcores");
-    queueTrackedMetrics.add("maxshare.memory");
-    queueTrackedMetrics.add("maxshare.vcores");
-    queueTrackedMetrics.add("fairshare.memory");
-    queueTrackedMetrics.add("fairshare.vcores");
+
+    for (Metric metric: Metric.values()) {
+      appTrackedMetrics.add(metric.value + ".memory");
+      appTrackedMetrics.add(metric.value + ".vcores");
+      queueTrackedMetrics.add(metric.value + ".memory");
+      queueTrackedMetrics.add(metric.value + ".vcores");
+    }
   }
-  
-  @Override
-  public void trackApp(ApplicationAttemptId appAttemptId, String oldAppId) {
-    super.trackApp(appAttemptId, oldAppId);
-    FairScheduler fair = (FairScheduler) scheduler;
-    final FSAppAttempt app = fair.getSchedulerApp(appAttemptId);
-    metrics.register("variable.app." + oldAppId + ".demand.memory",
-      new Gauge<Long>() {
-        @Override
-        public Long getValue() {
-          return app.getDemand().getMemorySize();
-        }
-      }
-    );
-    metrics.register("variable.app." + oldAppId + ".demand.vcores",
-      new Gauge<Integer>() {
-        @Override
-        public Integer getValue() {
-          return app.getDemand().getVirtualCores();
-        }
-      }
-    );
-    metrics.register("variable.app." + oldAppId + ".usage.memory",
-      new Gauge<Long>() {
-        @Override
-        public Long getValue() {
-          return app.getResourceUsage().getMemorySize();
-        }
-      }
-    );
-    metrics.register("variable.app." + oldAppId + ".usage.vcores",
-      new Gauge<Integer>() {
-        @Override
-        public Integer getValue() {
-          return app.getResourceUsage().getVirtualCores();
-        }
-      }
-    );
-    metrics.register("variable.app." + oldAppId + ".minshare.memory",
-      new Gauge<Long>() {
-        @Override
-        public Long getValue() {
-          return app.getMinShare().getMemorySize();
-        }
-      }
-    );
-    metrics.register("variable.app." + oldAppId + ".minshare.vcores",
-      new Gauge<Long>() {
-        @Override
-        public Long getValue() {
-          return app.getMinShare().getMemorySize();
-        }
-      }
-    );
-    metrics.register("variable.app." + oldAppId + ".maxshare.memory",
-      new Gauge<Long>() {
-        @Override
-        public Long getValue() {
-          return Math.min(app.getMaxShare().getMemorySize(), totalMemoryMB);
+
+  private long getMemorySize(Schedulable schedulable, Metric metric) {
+    if (schedulable != null) {
+      switch (metric) {
+      case DEMAND:
+        return schedulable.getDemand().getMemorySize();
+      case USAGE:
+        return schedulable.getResourceUsage().getMemorySize();
+      case MINSHARE:
+        return schedulable.getMinShare().getMemorySize();
+      case MAXSHARE:
+        return schedulable.getMaxShare().getMemorySize();
+      case FAIRSHARE:
+        return schedulable.getFairShare().getMemorySize();
+      default:
+        return 0L;
+      }
+    }
+
+    return 0L;
+  }
+
+  private int getVirtualCores(Schedulable schedulable, Metric metric) {
+    if (schedulable != null) {
+      switch (metric) {
+      case DEMAND:
+        return schedulable.getDemand().getVirtualCores();
+      case USAGE:
+        return schedulable.getResourceUsage().getVirtualCores();
+      case MINSHARE:
+        return schedulable.getMinShare().getVirtualCores();
+      case MAXSHARE:
+        return schedulable.getMaxShare().getVirtualCores();
+      case FAIRSHARE:
+        return schedulable.getFairShare().getVirtualCores();
+      default:
+        return 0;
+      }
+    }
+
+    return 0;
+  }
+
+  private void registerAppMetrics(final ApplicationId appId, String oldAppId,
+      final Metric metric) {
+    metrics.register(
+        "variable.app." + oldAppId + "." + metric.value + ".memory",
+        new Gauge<Long>() {
+          @Override
+          public Long getValue() {
+            return getMemorySize((FSAppAttempt)getSchedulerAppAttempt(appId),
+                metric);
+          }
         }
-      }
     );
-    metrics.register("variable.app." + oldAppId + ".maxshare.vcores",
-      new Gauge<Integer>() {
-        @Override
-        public Integer getValue() {
-          return Math.min(app.getMaxShare().getVirtualCores(), totalVCores);
+
+    metrics.register(
+        "variable.app." + oldAppId + "." + metric.value + ".vcores",
+        new Gauge<Integer>() {
+          @Override
+          public Integer getValue() {
+            return getVirtualCores((FSAppAttempt)getSchedulerAppAttempt(appId),
+                metric);
+          }
         }
-      }
     );
-    metrics.register("variable.app." + oldAppId + ".fairshare.memory",
-      new Gauge<Integer>() {
-        @Override
-        public Integer getValue() {
-          return app.getFairShare().getVirtualCores();
+  }
+
+  @Override
+  public void trackApp(ApplicationId appId, String oldAppId) {
+    super.trackApp(appId, oldAppId);
+
+    for (Metric metric: Metric.values()) {
+      registerAppMetrics(appId, oldAppId, metric);
+    }
+  }
+
+  private void registerQueueMetrics(final FSQueue queue, final Metric metric) {
+    metrics.register(
+        "variable.queue." + queue.getName() + "." + metric.value + ".memory",
+        new Gauge<Long>() {
+          @Override
+          public Long getValue() {
+            return getMemorySize(queue, metric);
+          }
         }
-      }
     );
-    metrics.register("variable.app." + oldAppId + ".fairshare.vcores",
-      new Gauge<Integer>() {
-        @Override
-        public Integer getValue() {
-          return app.getFairShare().getVirtualCores();
+    metrics.register(
+        "variable.queue." + queue.getName() + "." + metric.value + ".vcores",
+        new Gauge<Integer>() {
+          @Override
+          public Integer getValue() {
+            return getVirtualCores(queue, metric);
+          }
         }
-      }
     );
   }
 
@@ -153,68 +171,25 @@ public class FairSchedulerMetrics extends SchedulerMetrics {
     trackedQueues.add(queueName);
     FairScheduler fair = (FairScheduler) scheduler;
     final FSQueue queue = fair.getQueueManager().getQueue(queueName);
-    metrics.register("variable.queue." + queueName + ".demand.memory",
-      new Gauge<Long>() {
-        @Override
-        public Long getValue() {
-          return queue.getDemand().getMemorySize();
-        }
-      }
-    );
-    metrics.register("variable.queue." + queueName + ".demand.vcores",
-      new Gauge<Integer>() {
-        @Override
-        public Integer getValue() {
-          return queue.getDemand().getVirtualCores();
-        }
-      }
-    );
-    metrics.register("variable.queue." + queueName + ".usage.memory",
-      new Gauge<Long>() {
-        @Override
-        public Long getValue() {
-          return queue.getResourceUsage().getMemorySize();
-        }
-      }
-    );
-    metrics.register("variable.queue." + queueName + ".usage.vcores",
-      new Gauge<Integer>() {
-        @Override
-        public Integer getValue() {
-          return queue.getResourceUsage().getVirtualCores();
-        }
-      }
-    );
-    metrics.register("variable.queue." + queueName + ".minshare.memory",
-      new Gauge<Long>() {
-        @Override
-        public Long getValue() {
-          return queue.getMinShare().getMemorySize();
-        }
-      }
-    );
-    metrics.register("variable.queue." + queueName + ".minshare.vcores",
-      new Gauge<Integer>() {
-        @Override
-        public Integer getValue() {
-          return queue.getMinShare().getVirtualCores();
-        }
-      }
-    );
+    registerQueueMetrics(queue, Metric.DEMAND);
+    registerQueueMetrics(queue, Metric.USAGE);
+    registerQueueMetrics(queue, Metric.MINSHARE);
+    registerQueueMetrics(queue, Metric.FAIRSHARE);
+
     metrics.register("variable.queue." + queueName + ".maxshare.memory",
       new Gauge<Long>() {
         @Override
         public Long getValue() {
-          if (! maxReset &&
-                  SLSRunner.simulateInfoMap.containsKey("Number of nodes") &&
-                  SLSRunner.simulateInfoMap.containsKey("Node memory (MB)") &&
-                  SLSRunner.simulateInfoMap.containsKey("Node VCores")) {
-            int numNMs = Integer.parseInt(
-                  SLSRunner.simulateInfoMap.get("Number of nodes").toString());
-            int numMemoryMB = Integer.parseInt(
-                  SLSRunner.simulateInfoMap.get("Node memory (MB)").toString());
-            int numVCores = Integer.parseInt(
-                  SLSRunner.simulateInfoMap.get("Node VCores").toString());
+          if (! maxReset
+              && SLSRunner.getSimulateInfoMap().containsKey("Number of nodes")
+              && SLSRunner.getSimulateInfoMap().containsKey("Node memory (MB)")
+              && SLSRunner.getSimulateInfoMap().containsKey("Node VCores")) {
+            int numNMs = Integer.parseInt(SLSRunner.getSimulateInfoMap()
+                .get("Number of nodes").toString());
+            int numMemoryMB = Integer.parseInt(SLSRunner.getSimulateInfoMap()
+                .get("Node memory (MB)").toString());
+            int numVCores = Integer.parseInt(SLSRunner.getSimulateInfoMap()
+                .get("Node VCores").toString());
 
             totalMemoryMB = numNMs * numMemoryMB;
             totalVCores = numNMs * numVCores;
@@ -233,36 +208,17 @@ public class FairSchedulerMetrics extends SchedulerMetrics {
         }
       }
     );
-    metrics.register("variable.queue." + queueName + ".fairshare.memory",
-      new Gauge<Long>() {
-        @Override
-        public Long getValue() {
-          return queue.getFairShare().getMemorySize();
-        }
-      }
-    );
-    metrics.register("variable.queue." + queueName + ".fairshare.vcores",
-      new Gauge<Integer>() {
-        @Override
-        public Integer getValue() {
-          return queue.getFairShare().getVirtualCores();
-        }
-      }
-    );
   }
 
   @Override
   public void untrackQueue(String queueName) {
     trackedQueues.remove(queueName);
-    metrics.remove("variable.queue." + queueName + ".demand.memory");
-    metrics.remove("variable.queue." + queueName + ".demand.vcores");
-    metrics.remove("variable.queue." + queueName + ".usage.memory");
-    metrics.remove("variable.queue." + queueName + ".usage.vcores");
-    metrics.remove("variable.queue." + queueName + ".minshare.memory");
-    metrics.remove("variable.queue." + queueName + ".minshare.vcores");
-    metrics.remove("variable.queue." + queueName + ".maxshare.memory");
-    metrics.remove("variable.queue." + queueName + ".maxshare.vcores");
-    metrics.remove("variable.queue." + queueName + ".fairshare.memory");
-    metrics.remove("variable.queue." + queueName + ".fairshare.vcores");
+
+    for (Metric metric: Metric.values()) {
+      metrics.remove("variable.queue." + queueName + "." +
+          metric.value + ".memory");
+      metrics.remove("variable.queue." + queueName + "." +
+          metric.value + ".vcores");
+    }
   }
 }

+ 0 - 973
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java

@@ -1,973 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.yarn.sls.scheduler;
-
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-
-import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.util.ShutdownHookManager;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.QueueACL;
-import org.apache.hadoop.yarn.api.records.QueueInfo;
-import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
-import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
-import org.apache.hadoop.yarn.sls.SLSRunner;
-import org.apache.hadoop.yarn.sls.conf.SLSConfiguration;
-import org.apache.hadoop.yarn.sls.web.SLSWebApp;
-import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
-import org.apache.hadoop.yarn.util.resource.Resources;
-
-import com.codahale.metrics.Counter;
-import com.codahale.metrics.CsvReporter;
-import com.codahale.metrics.Gauge;
-import com.codahale.metrics.Histogram;
-import com.codahale.metrics.MetricRegistry;
-import com.codahale.metrics.SlidingWindowReservoir;
-import com.codahale.metrics.Timer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@Private
-@Unstable
-public class ResourceSchedulerWrapper
-    extends AbstractYarnScheduler<SchedulerApplicationAttempt, SchedulerNode>
-    implements SchedulerWrapper,ResourceScheduler,Configurable {
-  private static final String EOL = System.getProperty("line.separator");
-  private static final int SAMPLING_SIZE = 60;
-  private ScheduledExecutorService pool;
-  // counters for scheduler allocate/handle operations
-  private Counter schedulerAllocateCounter;
-  private Counter schedulerHandleCounter;
-  private Map<SchedulerEventType, Counter> schedulerHandleCounterMap;
-  // Timers for scheduler allocate/handle operations
-  private Timer schedulerAllocateTimer;
-  private Timer schedulerHandleTimer;
-  private Map<SchedulerEventType, Timer> schedulerHandleTimerMap;
-  private List<Histogram> schedulerHistogramList;
-  private Map<Histogram, Timer> histogramTimerMap;
-  private Lock samplerLock;
-  private Lock queueLock;
-
-  private Configuration conf;
-  private ResourceScheduler scheduler;
-  private Map<ApplicationId, String> appQueueMap =
-          new ConcurrentHashMap<ApplicationId, String>();
-  private BufferedWriter jobRuntimeLogBW;
-
-  // Priority of the ResourceSchedulerWrapper shutdown hook.
-  public static final int SHUTDOWN_HOOK_PRIORITY = 30;
-
-  // web app
-  private SLSWebApp web;
-
-  private Map<ContainerId, Resource> preemptionContainerMap =
-          new ConcurrentHashMap<ContainerId, Resource>();
-
-  // metrics
-  private MetricRegistry metrics;
-  private SchedulerMetrics schedulerMetrics;
-  private boolean metricsON;
-  private String metricsOutputDir;
-  private BufferedWriter metricsLogBW;
-  private boolean running = false;
-  private static Map<Class, Class> defaultSchedulerMetricsMap =
-          new HashMap<Class, Class>();
-  static {
-    defaultSchedulerMetricsMap.put(FairScheduler.class,
-            FairSchedulerMetrics.class);
-    defaultSchedulerMetricsMap.put(FifoScheduler.class,
-            FifoSchedulerMetrics.class);
-    defaultSchedulerMetricsMap.put(CapacityScheduler.class,
-            CapacitySchedulerMetrics.class);
-  }
-  // must set by outside
-  private Set<String> queueSet;
-  private Set<String> trackedAppSet;
-
-  public final Logger LOG =
-      LoggerFactory.getLogger(ResourceSchedulerWrapper.class);
-
-  public ResourceSchedulerWrapper() {
-    super(ResourceSchedulerWrapper.class.getName());
-    samplerLock = new ReentrantLock();
-    queueLock = new ReentrantLock();
-  }
-
-  @Override
-  public void setConf(Configuration conf) {
-    this.conf = conf;
-    // set scheduler
-    Class<? extends ResourceScheduler> klass = conf.getClass(
-        SLSConfiguration.RM_SCHEDULER, null, ResourceScheduler.class);
-
-    scheduler = ReflectionUtils.newInstance(klass, conf);
-    // start metrics
-    metricsON = conf.getBoolean(SLSConfiguration.METRICS_SWITCH, true);
-    if (metricsON) {
-      try {
-        initMetrics();
-      } catch (Exception e) {
-        e.printStackTrace();
-      }
-    }
-
-    ShutdownHookManager.get().addShutdownHook(new Runnable() {
-      @Override
-      public void run() {
-        try {
-          if (metricsLogBW != null)  {
-            metricsLogBW.write("]");
-            metricsLogBW.close();
-          }
-          if (web != null) {
-            web.stop();
-          }
-          tearDown();
-        } catch (Exception e) {
-          e.printStackTrace();
-        }
-      }
-    }, SHUTDOWN_HOOK_PRIORITY);
-  }
-
-  @Override
-  public Allocation allocate(ApplicationAttemptId attemptId,
-      List<ResourceRequest> resourceRequests, List<ContainerId> containerIds,
-      List<String> strings, List<String> strings2,
-      ContainerUpdates updateRequests) {
-    if (metricsON) {
-      final Timer.Context context = schedulerAllocateTimer.time();
-      Allocation allocation = null;
-      try {
-        allocation = scheduler.allocate(attemptId, resourceRequests,
-                containerIds, strings, strings2, updateRequests);
-        return allocation;
-      } finally {
-        context.stop();
-        schedulerAllocateCounter.inc();
-        try {
-          updateQueueWithAllocateRequest(allocation, attemptId,
-                  resourceRequests, containerIds);
-        } catch (IOException e) {
-          e.printStackTrace();
-        }
-      }
-    } else {
-      return scheduler.allocate(attemptId,
-              resourceRequests, containerIds, strings, strings2, updateRequests);
-    }
-  }
-
-  @Override
-  public void handle(SchedulerEvent schedulerEvent) {
-    // metrics off
-    if (! metricsON) {
-      scheduler.handle(schedulerEvent);
-      return;
-    }
-    if(!running)    running = true;
-
-    // metrics on
-    Timer.Context handlerTimer = null;
-    Timer.Context operationTimer = null;
-
-    NodeUpdateSchedulerEventWrapper eventWrapper;
-    try {
-      //if (schedulerEvent instanceof NodeUpdateSchedulerEvent) {
-      if (schedulerEvent.getType() == SchedulerEventType.NODE_UPDATE
-              && schedulerEvent instanceof NodeUpdateSchedulerEvent) {
-        eventWrapper = new NodeUpdateSchedulerEventWrapper(
-                (NodeUpdateSchedulerEvent)schedulerEvent);
-        schedulerEvent = eventWrapper;
-        updateQueueWithNodeUpdate(eventWrapper);
-      } else if (schedulerEvent.getType() == SchedulerEventType.APP_ATTEMPT_REMOVED
-          && schedulerEvent instanceof AppAttemptRemovedSchedulerEvent) {
-        // check if having AM Container, update resource usage information
-        AppAttemptRemovedSchedulerEvent appRemoveEvent =
-            (AppAttemptRemovedSchedulerEvent) schedulerEvent;
-        ApplicationAttemptId appAttemptId =
-                appRemoveEvent.getApplicationAttemptID();
-        String queue = appQueueMap.get(appAttemptId.getApplicationId());
-        SchedulerAppReport app = scheduler.getSchedulerAppInfo(appAttemptId);
-        if (! app.getLiveContainers().isEmpty()) {  // have 0 or 1
-          // should have one container which is AM container
-          RMContainer rmc = app.getLiveContainers().iterator().next();
-          updateQueueMetrics(queue,
-                  rmc.getContainer().getResource().getMemorySize(),
-                  rmc.getContainer().getResource().getVirtualCores());
-        }
-      }
-
-      handlerTimer = schedulerHandleTimer.time();
-      operationTimer = schedulerHandleTimerMap
-              .get(schedulerEvent.getType()).time();
-
-      scheduler.handle(schedulerEvent);
-    } finally {
-      if (handlerTimer != null)     handlerTimer.stop();
-      if (operationTimer != null)   operationTimer.stop();
-      schedulerHandleCounter.inc();
-      schedulerHandleCounterMap.get(schedulerEvent.getType()).inc();
-
-      if (schedulerEvent.getType() == SchedulerEventType.APP_REMOVED
-          && schedulerEvent instanceof AppRemovedSchedulerEvent) {
-        SLSRunner.decreaseRemainingApps();
-        AppRemovedSchedulerEvent appRemoveEvent =
-                (AppRemovedSchedulerEvent) schedulerEvent;
-        appQueueMap.remove(appRemoveEvent.getApplicationID());
-      } else if (schedulerEvent.getType() == SchedulerEventType.APP_ADDED
-          && schedulerEvent instanceof AppAddedSchedulerEvent) {
-        AppAddedSchedulerEvent appAddEvent =
-                (AppAddedSchedulerEvent) schedulerEvent;
-        String queueName = appAddEvent.getQueue();
-        appQueueMap.put(appAddEvent.getApplicationId(), queueName);
-      }
-    }
-  }
-
-  private void updateQueueWithNodeUpdate(
-          NodeUpdateSchedulerEventWrapper eventWrapper) {
-    RMNodeWrapper node = (RMNodeWrapper) eventWrapper.getRMNode();
-    List<UpdatedContainerInfo> containerList = node.getContainerUpdates();
-    for (UpdatedContainerInfo info : containerList) {
-      for (ContainerStatus status : info.getCompletedContainers()) {
-        ContainerId containerId = status.getContainerId();
-        SchedulerAppReport app = scheduler.getSchedulerAppInfo(
-                containerId.getApplicationAttemptId());
-
-        if (app == null) {
-          // this happens for the AM container
-          // The app have already removed when the NM sends the release
-          // information.
-          continue;
-        }
-
-        String queue =
-            appQueueMap.get(containerId.getApplicationAttemptId()
-              .getApplicationId());
-        int releasedMemory = 0, releasedVCores = 0;
-        if (status.getExitStatus() == ContainerExitStatus.SUCCESS) {
-          for (RMContainer rmc : app.getLiveContainers()) {
-            if (rmc.getContainerId() == containerId) {
-              releasedMemory += rmc.getContainer().getResource().getMemorySize();
-              releasedVCores += rmc.getContainer()
-                      .getResource().getVirtualCores();
-              break;
-            }
-          }
-        } else if (status.getExitStatus() == ContainerExitStatus.ABORTED) {
-          if (preemptionContainerMap.containsKey(containerId)) {
-            Resource preResource = preemptionContainerMap.get(containerId);
-            releasedMemory += preResource.getMemorySize();
-            releasedVCores += preResource.getVirtualCores();
-            preemptionContainerMap.remove(containerId);
-          }
-        }
-        // update queue counters
-        updateQueueMetrics(queue, releasedMemory, releasedVCores);
-      }
-    }
-  }
-
-  private void updateQueueWithAllocateRequest(Allocation allocation,
-                        ApplicationAttemptId attemptId,
-                        List<ResourceRequest> resourceRequests,
-                        List<ContainerId> containerIds) throws IOException {
-    // update queue information
-    Resource pendingResource = Resources.createResource(0, 0);
-    Resource allocatedResource = Resources.createResource(0, 0);
-    String queueName = appQueueMap.get(attemptId.getApplicationId());
-    // container requested
-    for (ResourceRequest request : resourceRequests) {
-      if (request.getResourceName().equals(ResourceRequest.ANY)) {
-        Resources.addTo(pendingResource,
-                Resources.multiply(request.getCapability(),
-                        request.getNumContainers()));
-      }
-    }
-    // container allocated
-    for (Container container : allocation.getContainers()) {
-      Resources.addTo(allocatedResource, container.getResource());
-      Resources.subtractFrom(pendingResource, container.getResource());
-    }
-    // container released from AM
-    SchedulerAppReport report = scheduler.getSchedulerAppInfo(attemptId);
-    for (ContainerId containerId : containerIds) {
-      Container container = null;
-      for (RMContainer c : report.getLiveContainers()) {
-        if (c.getContainerId().equals(containerId)) {
-          container = c.getContainer();
-          break;
-        }
-      }
-      if (container != null) {
-        // released allocated containers
-        Resources.subtractFrom(allocatedResource, container.getResource());
-      } else {
-        for (RMContainer c : report.getReservedContainers()) {
-          if (c.getContainerId().equals(containerId)) {
-            container = c.getContainer();
-            break;
-          }
-        }
-        if (container != null) {
-          // released reserved containers
-          Resources.subtractFrom(pendingResource, container.getResource());
-        }
-      }
-    }
-    // containers released/preemption from scheduler
-    Set<ContainerId> preemptionContainers = new HashSet<ContainerId>();
-    if (allocation.getContainerPreemptions() != null) {
-      preemptionContainers.addAll(allocation.getContainerPreemptions());
-    }
-    if (allocation.getStrictContainerPreemptions() != null) {
-      preemptionContainers.addAll(allocation.getStrictContainerPreemptions());
-    }
-    if (! preemptionContainers.isEmpty()) {
-      for (ContainerId containerId : preemptionContainers) {
-        if (! preemptionContainerMap.containsKey(containerId)) {
-          Container container = null;
-          for (RMContainer c : report.getLiveContainers()) {
-            if (c.getContainerId().equals(containerId)) {
-              container = c.getContainer();
-              break;
-            }
-          }
-          if (container != null) {
-            preemptionContainerMap.put(containerId, container.getResource());
-          }
-        }
-
-      }
-    }
-
-    // update metrics
-    SortedMap<String, Counter> counterMap = metrics.getCounters();
-    String names[] = new String[]{
-            "counter.queue." + queueName + ".pending.memory",
-            "counter.queue." + queueName + ".pending.cores",
-            "counter.queue." + queueName + ".allocated.memory",
-            "counter.queue." + queueName + ".allocated.cores"};
-    long values[] = new long[]{pendingResource.getMemorySize(),
-            pendingResource.getVirtualCores(),
-            allocatedResource.getMemorySize(), allocatedResource.getVirtualCores()};
-    for (int i = names.length - 1; i >= 0; i --) {
-      if (! counterMap.containsKey(names[i])) {
-        metrics.counter(names[i]);
-        counterMap = metrics.getCounters();
-      }
-      counterMap.get(names[i]).inc(values[i]);
-    }
-
-    queueLock.lock();
-    try {
-      if (! schedulerMetrics.isTracked(queueName)) {
-        schedulerMetrics.trackQueue(queueName);
-      }
-    } finally {
-      queueLock.unlock();
-    }
-  }
-
-  private void tearDown() throws IOException {
-    // close job runtime writer
-    if (jobRuntimeLogBW != null) {
-      jobRuntimeLogBW.close();
-    }
-    // shut pool
-    if (pool != null)  pool.shutdown();
-  }
-
-  @SuppressWarnings("unchecked")
-  private void initMetrics() throws Exception {
-    metrics = new MetricRegistry();
-    // configuration
-    metricsOutputDir = conf.get(SLSConfiguration.METRICS_OUTPUT_DIR);
-    int metricsWebAddressPort = conf.getInt(
-            SLSConfiguration.METRICS_WEB_ADDRESS_PORT,
-            SLSConfiguration.METRICS_WEB_ADDRESS_PORT_DEFAULT);
-    // create SchedulerMetrics for current scheduler
-    String schedulerMetricsType = conf.get(scheduler.getClass().getName());
-    Class schedulerMetricsClass = schedulerMetricsType == null?
-            defaultSchedulerMetricsMap.get(scheduler.getClass()) :
-            Class.forName(schedulerMetricsType);
-    schedulerMetrics = (SchedulerMetrics)ReflectionUtils
-            .newInstance(schedulerMetricsClass, new Configuration());
-    schedulerMetrics.init(scheduler, metrics);
-
-    // register various metrics
-    registerJvmMetrics();
-    registerClusterResourceMetrics();
-    registerContainerAppNumMetrics();
-    registerSchedulerMetrics();
-
-    // .csv output
-    initMetricsCSVOutput();
-
-    // start web app to provide real-time tracking
-    web = new SLSWebApp(this, metricsWebAddressPort);
-    web.start();
-
-    // a thread to update histogram timer
-    pool = new ScheduledThreadPoolExecutor(2);
-    pool.scheduleAtFixedRate(new HistogramsRunnable(), 0, 1000,
-            TimeUnit.MILLISECONDS);
-
-    // a thread to output metrics for real-tiem tracking
-    pool.scheduleAtFixedRate(new MetricsLogRunnable(), 0, 1000,
-            TimeUnit.MILLISECONDS);
-
-    // application running information
-    jobRuntimeLogBW =
-        new BufferedWriter(new OutputStreamWriter(new FileOutputStream(
-            metricsOutputDir + "/jobruntime.csv"), "UTF-8"));
-    jobRuntimeLogBW.write("JobID,real_start_time,real_end_time," +
-            "simulate_start_time,simulate_end_time" + EOL);
-    jobRuntimeLogBW.flush();
-  }
-
-  private void registerJvmMetrics() {
-    // add JVM gauges
-    metrics.register("variable.jvm.free.memory",
-      new Gauge<Long>() {
-        @Override
-        public Long getValue() {
-          return Runtime.getRuntime().freeMemory();
-        }
-      }
-    );
-    metrics.register("variable.jvm.max.memory",
-      new Gauge<Long>() {
-        @Override
-        public Long getValue() {
-          return Runtime.getRuntime().maxMemory();
-        }
-      }
-    );
-    metrics.register("variable.jvm.total.memory",
-      new Gauge<Long>() {
-        @Override
-        public Long getValue() {
-          return Runtime.getRuntime().totalMemory();
-        }
-      }
-    );
-  }
-
-  private void registerClusterResourceMetrics() {
-    metrics.register("variable.cluster.allocated.memory",
-      new Gauge<Long>() {
-        @Override
-        public Long getValue() {
-          if(scheduler == null || scheduler.getRootQueueMetrics() == null) {
-            return 0L;
-          } else {
-            return scheduler.getRootQueueMetrics().getAllocatedMB();
-          }
-        }
-      }
-    );
-    metrics.register("variable.cluster.allocated.vcores",
-      new Gauge<Integer>() {
-        @Override
-        public Integer getValue() {
-          if(scheduler == null || scheduler.getRootQueueMetrics() == null) {
-            return 0;
-          } else {
-            return scheduler.getRootQueueMetrics().getAllocatedVirtualCores();
-          }
-        }
-      }
-    );
-    metrics.register("variable.cluster.available.memory",
-      new Gauge<Long>() {
-        @Override
-        public Long getValue() {
-          if(scheduler == null || scheduler.getRootQueueMetrics() == null) {
-            return 0L;
-          } else {
-            return scheduler.getRootQueueMetrics().getAvailableMB();
-          }
-        }
-      }
-    );
-    metrics.register("variable.cluster.available.vcores",
-      new Gauge<Integer>() {
-        @Override
-        public Integer getValue() {
-          if(scheduler == null || scheduler.getRootQueueMetrics() == null) {
-            return 0;
-          } else {
-            return scheduler.getRootQueueMetrics().getAvailableVirtualCores();
-          }
-        }
-      }
-    );
-  }
-
-  private void registerContainerAppNumMetrics() {
-    metrics.register("variable.running.application",
-      new Gauge<Integer>() {
-        @Override
-        public Integer getValue() {
-          if (scheduler == null || scheduler.getRootQueueMetrics() == null) {
-            return 0;
-          } else {
-            return scheduler.getRootQueueMetrics().getAppsRunning();
-          }
-        }
-      }
-    );
-    metrics.register("variable.running.container",
-      new Gauge<Integer>() {
-        @Override
-        public Integer getValue() {
-          if(scheduler == null || scheduler.getRootQueueMetrics() == null) {
-            return 0;
-          } else {
-            return scheduler.getRootQueueMetrics().getAllocatedContainers();
-          }
-        }
-      }
-    );
-  }
-
-  private void registerSchedulerMetrics() {
-    samplerLock.lock();
-    try {
-      // counters for scheduler operations
-      schedulerAllocateCounter = metrics.counter(
-              "counter.scheduler.operation.allocate");
-      schedulerHandleCounter = metrics.counter(
-              "counter.scheduler.operation.handle");
-      schedulerHandleCounterMap = new HashMap<SchedulerEventType, Counter>();
-      for (SchedulerEventType e : SchedulerEventType.values()) {
-        Counter counter = metrics.counter(
-                "counter.scheduler.operation.handle." + e);
-        schedulerHandleCounterMap.put(e, counter);
-      }
-      // timers for scheduler operations
-      int timeWindowSize = conf.getInt(
-              SLSConfiguration.METRICS_TIMER_WINDOW_SIZE,
-              SLSConfiguration.METRICS_TIMER_WINDOW_SIZE_DEFAULT);
-      schedulerAllocateTimer = new Timer(
-              new SlidingWindowReservoir(timeWindowSize));
-      schedulerHandleTimer = new Timer(
-              new SlidingWindowReservoir(timeWindowSize));
-      schedulerHandleTimerMap = new HashMap<SchedulerEventType, Timer>();
-      for (SchedulerEventType e : SchedulerEventType.values()) {
-        Timer timer = new Timer(new SlidingWindowReservoir(timeWindowSize));
-        schedulerHandleTimerMap.put(e, timer);
-      }
-      // histogram for scheduler operations (Samplers)
-      schedulerHistogramList = new ArrayList<Histogram>();
-      histogramTimerMap = new HashMap<Histogram, Timer>();
-      Histogram schedulerAllocateHistogram = new Histogram(
-              new SlidingWindowReservoir(SAMPLING_SIZE));
-      metrics.register("sampler.scheduler.operation.allocate.timecost",
-              schedulerAllocateHistogram);
-      schedulerHistogramList.add(schedulerAllocateHistogram);
-      histogramTimerMap.put(schedulerAllocateHistogram, schedulerAllocateTimer);
-      Histogram schedulerHandleHistogram = new Histogram(
-              new SlidingWindowReservoir(SAMPLING_SIZE));
-      metrics.register("sampler.scheduler.operation.handle.timecost",
-              schedulerHandleHistogram);
-      schedulerHistogramList.add(schedulerHandleHistogram);
-      histogramTimerMap.put(schedulerHandleHistogram, schedulerHandleTimer);
-      for (SchedulerEventType e : SchedulerEventType.values()) {
-        Histogram histogram = new Histogram(
-                new SlidingWindowReservoir(SAMPLING_SIZE));
-        metrics.register(
-                "sampler.scheduler.operation.handle." + e + ".timecost",
-                histogram);
-        schedulerHistogramList.add(histogram);
-        histogramTimerMap.put(histogram, schedulerHandleTimerMap.get(e));
-      }
-    } finally {
-      samplerLock.unlock();
-    }
-  }
-
-  private void initMetricsCSVOutput() {
-    int timeIntervalMS = conf.getInt(
-            SLSConfiguration.METRICS_RECORD_INTERVAL_MS,
-            SLSConfiguration.METRICS_RECORD_INTERVAL_MS_DEFAULT);
-    File dir = new File(metricsOutputDir + "/metrics");
-    if(! dir.exists()
-            && ! dir.mkdirs()) {
-      LOG.error("Cannot create directory {}", dir.getAbsoluteFile());
-    }
-    final CsvReporter reporter = CsvReporter.forRegistry(metrics)
-            .formatFor(Locale.US)
-            .convertRatesTo(TimeUnit.SECONDS)
-            .convertDurationsTo(TimeUnit.MILLISECONDS)
-            .build(new File(metricsOutputDir + "/metrics"));
-    reporter.start(timeIntervalMS, TimeUnit.MILLISECONDS);
-  }
-
-  class HistogramsRunnable implements Runnable {
-    @Override
-    public void run() {
-      samplerLock.lock();
-      try {
-        for (Histogram histogram : schedulerHistogramList) {
-          Timer timer = histogramTimerMap.get(histogram);
-          histogram.update((int) timer.getSnapshot().getMean());
-        }
-      } finally {
-        samplerLock.unlock();
-      }
-    }
-  }
-
-  class MetricsLogRunnable implements Runnable {
-    private boolean firstLine = true;
-    public MetricsLogRunnable() {
-      try {
-        metricsLogBW =
-            new BufferedWriter(new OutputStreamWriter(new FileOutputStream(
-                metricsOutputDir + "/realtimetrack.json"), "UTF-8"));
-        metricsLogBW.write("[");
-      } catch (IOException e) {
-        e.printStackTrace();
-      }
-    }
-
-    @Override
-    public void run() {
-      if(running) {
-        // all WebApp to get real tracking json
-        String metrics = web.generateRealTimeTrackingMetrics();
-        // output
-        try {
-          if(firstLine) {
-            metricsLogBW.write(metrics + EOL);
-            firstLine = false;
-          } else {
-            metricsLogBW.write("," + metrics + EOL);
-          }
-          metricsLogBW.flush();
-        } catch (IOException e) {
-          e.printStackTrace();
-        }
-      }
-    }
-  }
-
-  // the following functions are used by AMSimulator
-  public void addAMRuntime(ApplicationId appId,
-                           long traceStartTimeMS, long traceEndTimeMS,
-                           long simulateStartTimeMS, long simulateEndTimeMS) {
-    if (metricsON) {
-      try {
-        // write job runtime information
-        StringBuilder sb = new StringBuilder();
-        sb.append(appId).append(",").append(traceStartTimeMS).append(",")
-            .append(traceEndTimeMS).append(",").append(simulateStartTimeMS)
-            .append(",").append(simulateEndTimeMS);
-        jobRuntimeLogBW.write(sb.toString() + EOL);
-        jobRuntimeLogBW.flush();
-      } catch (IOException e) {
-        e.printStackTrace();
-      }
-    }
-  }
-
-  private void updateQueueMetrics(String queue,
-                                  long releasedMemory, int releasedVCores) {
-    // update queue counters
-    SortedMap<String, Counter> counterMap = metrics.getCounters();
-    if (releasedMemory != 0) {
-      String name = "counter.queue." + queue + ".allocated.memory";
-      if (! counterMap.containsKey(name)) {
-        metrics.counter(name);
-        counterMap = metrics.getCounters();
-      }
-      counterMap.get(name).inc(-releasedMemory);
-    }
-    if (releasedVCores != 0) {
-      String name = "counter.queue." + queue + ".allocated.cores";
-      if (! counterMap.containsKey(name)) {
-        metrics.counter(name);
-        counterMap = metrics.getCounters();
-      }
-      counterMap.get(name).inc(-releasedVCores);
-    }
-  }
-
-  public void setQueueSet(Set<String> queues) {
-    this.queueSet = queues;
-  }
-
-  public Set<String> getQueueSet() {
-    return this.queueSet;
-  }
-
-  public void setTrackedAppSet(Set<String> apps) {
-    this.trackedAppSet = apps;
-  }
-
-  public Set<String> getTrackedAppSet() {
-    return this.trackedAppSet;
-  }
-
-  public MetricRegistry getMetrics() {
-    return metrics;
-  }
-
-  public SchedulerMetrics getSchedulerMetrics() {
-    return schedulerMetrics;
-  }
-
-  // API open to out classes
-  public void addTrackedApp(ApplicationAttemptId appAttemptId,
-                            String oldAppId) {
-    if (metricsON) {
-      schedulerMetrics.trackApp(appAttemptId, oldAppId);
-    }
-  }
-
-  public void removeTrackedApp(ApplicationAttemptId appAttemptId,
-                               String oldAppId) {
-    if (metricsON) {
-      schedulerMetrics.untrackApp(appAttemptId, oldAppId);
-    }
-  }
-
-  @Override
-  public Configuration getConf() {
-    return conf;
-  }
-
-  @SuppressWarnings("unchecked")
-  @Override
-  public void serviceInit(Configuration conf) throws Exception {
-    ((AbstractYarnScheduler<SchedulerApplicationAttempt, SchedulerNode>)
-        scheduler).init(conf);
-    super.serviceInit(conf);
-    initScheduler(conf);
-  }
-
-  private synchronized void initScheduler(Configuration configuration) throws
-  IOException {
-    this.applications =
-        new ConcurrentHashMap<ApplicationId,
-        SchedulerApplication<SchedulerApplicationAttempt>>();
-  }
-
-  @SuppressWarnings("unchecked")
-  @Override
-  public void serviceStart() throws Exception {
-    ((AbstractYarnScheduler<SchedulerApplicationAttempt, SchedulerNode>)
-        scheduler).start();
-    super.serviceStart();
-  }
-
-  @SuppressWarnings("unchecked")
-  @Override
-  public void serviceStop() throws Exception {
-    ((AbstractYarnScheduler<SchedulerApplicationAttempt, SchedulerNode>)
-        scheduler).stop();
-    super.serviceStop();
-  }
-
-  @Override
-  public void setRMContext(RMContext rmContext) {
-    scheduler.setRMContext(rmContext);
-  }
-
-  @Override
-  public void reinitialize(Configuration conf, RMContext rmContext)
-      throws IOException {
-    scheduler.reinitialize(conf, rmContext);
-  }
-
-  @Override
-  public void recover(RMStateStore.RMState rmState) throws Exception {
-    scheduler.recover(rmState);
-  }
-
-  @Override
-  public QueueInfo getQueueInfo(String s, boolean b, boolean b2)
-          throws IOException {
-    return scheduler.getQueueInfo(s, b, b2);
-  }
-
-  @Override
-  public List<QueueUserACLInfo> getQueueUserAclInfo() {
-    return scheduler.getQueueUserAclInfo();
-  }
-
-  @Override
-  public Resource getMinimumResourceCapability() {
-    return scheduler.getMinimumResourceCapability();
-  }
-
-  @Override
-  public Resource getMaximumResourceCapability() {
-    return scheduler.getMaximumResourceCapability();
-  }
-
-  @Override
-  public ResourceCalculator getResourceCalculator() {
-    return scheduler.getResourceCalculator();
-  }
-
-  @Override
-  public int getNumClusterNodes() {
-    return scheduler.getNumClusterNodes();
-  }
-
-  @Override
-  public SchedulerNodeReport getNodeReport(NodeId nodeId) {
-    return scheduler.getNodeReport(nodeId);
-  }
-
-  @Override
-  public SchedulerAppReport getSchedulerAppInfo(
-          ApplicationAttemptId attemptId) {
-    return scheduler.getSchedulerAppInfo(attemptId);
-  }
-
-  @Override
-  public QueueMetrics getRootQueueMetrics() {
-    return scheduler.getRootQueueMetrics();
-  }
-
-  @Override
-  public synchronized boolean checkAccess(UserGroupInformation callerUGI,
-      QueueACL acl, String queueName) {
-    return scheduler.checkAccess(callerUGI, acl, queueName);
-  }
-
-  @Override
-  public ApplicationResourceUsageReport getAppResourceUsageReport(
-      ApplicationAttemptId appAttemptId) {
-    return scheduler.getAppResourceUsageReport(appAttemptId);
-  }
-
-  @Override
-  public List<ApplicationAttemptId> getAppsInQueue(String queue) {
-    return scheduler.getAppsInQueue(queue);
-  }
-
-  @Override
-  public RMContainer getRMContainer(ContainerId containerId) {
-    return null;
-  }
-
-  @Override
-  public String moveApplication(ApplicationId appId, String newQueue)
-      throws YarnException {
-    return scheduler.moveApplication(appId, newQueue);
-  }
-
-  @Override
-  @LimitedPrivate("yarn")
-  @Unstable
-  public Resource getClusterResource() {
-    return super.getClusterResource();
-  }
-
-  @Override
-  public synchronized List<Container> getTransferredContainers(
-      ApplicationAttemptId currentAttempt) {
-    return new ArrayList<Container>();
-  }
-
-  @Override
-  public Map<ApplicationId, SchedulerApplication<SchedulerApplicationAttempt>>
-      getSchedulerApplications() {
-    return new HashMap<ApplicationId,
-        SchedulerApplication<SchedulerApplicationAttempt>>();
-  }
-
-  @Override
-  protected void completedContainerInternal(RMContainer rmContainer,
-      ContainerStatus containerStatus, RMContainerEventType event) {
-    // do nothing
-  }
-
-  @Override
-  public Priority checkAndGetApplicationPriority(Priority priority,
-      UserGroupInformation user, String queueName, ApplicationId applicationId)
-      throws YarnException {
-    // TODO Dummy implementation.
-    return Priority.newInstance(0);
-  }
-
-}

+ 130 - 550
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java

@@ -17,176 +17,104 @@
  */
 package org.apache.hadoop.yarn.sls.scheduler;
 
-import com.codahale.metrics.Counter;
-import com.codahale.metrics.CsvReporter;
-import com.codahale.metrics.Gauge;
-import com.codahale.metrics.Histogram;
-import com.codahale.metrics.MetricRegistry;
-import com.codahale.metrics.SlidingWindowReservoir;
-import com.codahale.metrics.Timer;
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileOutputStream;
 import java.io.IOException;
-import java.io.OutputStreamWriter;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
-import java.util.SortedMap;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
 import org.apache.hadoop.yarn.sls.SLSRunner;
 import org.apache.hadoop.yarn.sls.conf.SLSConfiguration;
-import org.apache.hadoop.yarn.sls.web.SLSWebApp;
 import org.apache.hadoop.yarn.util.resource.Resources;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
+import com.codahale.metrics.Timer;
+
+@Private
+@Unstable
 public class SLSCapacityScheduler extends CapacityScheduler implements
         SchedulerWrapper,Configurable {
-  private static final String EOL = System.getProperty("line.separator");
-  private static final int SAMPLING_SIZE = 60;
-  private ScheduledExecutorService pool;
-  // counters for scheduler allocate/handle operations
-  private Counter schedulerAllocateCounter;
-  private Counter schedulerHandleCounter;
-  private Map<SchedulerEventType, Counter> schedulerHandleCounterMap;
-  // Timers for scheduler allocate/handle operations
-  private Timer schedulerAllocateTimer;
-  private Timer schedulerHandleTimer;
-  private Map<SchedulerEventType, Timer> schedulerHandleTimerMap;
-  private List<Histogram> schedulerHistogramList;
-  private Map<Histogram, Timer> histogramTimerMap;
-  private Lock samplerLock;
-  private Lock queueLock;
 
   private Configuration conf;
-
+ 
   private Map<ApplicationAttemptId, String> appQueueMap =
           new ConcurrentHashMap<ApplicationAttemptId, String>();
-  private BufferedWriter jobRuntimeLogBW;
-
-  // Priority of the ResourceSchedulerWrapper shutdown hook.
-  public static final int SHUTDOWN_HOOK_PRIORITY = 30;
-
-  // web app
-  private SLSWebApp web;
 
   private Map<ContainerId, Resource> preemptionContainerMap =
           new ConcurrentHashMap<ContainerId, Resource>();
 
   // metrics
-  private MetricRegistry metrics;
   private SchedulerMetrics schedulerMetrics;
   private boolean metricsON;
-  private String metricsOutputDir;
-  private BufferedWriter metricsLogBW;
-  private boolean running = false;
-  private static Map<Class, Class> defaultSchedulerMetricsMap =
-          new HashMap<Class, Class>();
-  static {
-    defaultSchedulerMetricsMap.put(FairScheduler.class,
-            FairSchedulerMetrics.class);
-    defaultSchedulerMetricsMap.put(FifoScheduler.class,
-            FifoSchedulerMetrics.class);
-    defaultSchedulerMetricsMap.put(CapacityScheduler.class,
-            CapacitySchedulerMetrics.class);
-  }
-  // must set by outside
-  private Set<String> queueSet;
-  private Set<String> trackedAppSet;
+  private Tracker tracker;
 
-  public final Logger LOG = LoggerFactory.getLogger(SLSCapacityScheduler.class);
+  public Tracker getTracker() {
+    return tracker;
+  }
 
   public SLSCapacityScheduler() {
-    samplerLock = new ReentrantLock();
-    queueLock = new ReentrantLock();
+    tracker = new Tracker();
   }
 
   @Override
   public void setConf(Configuration conf) {
     this.conf = conf;
     super.setConf(conf);
-    // start metrics
     metricsON = conf.getBoolean(SLSConfiguration.METRICS_SWITCH, true);
     if (metricsON) {
       try {
-        initMetrics();
+        schedulerMetrics = SchedulerMetrics.getInstance(conf,
+            CapacityScheduler.class);
+        schedulerMetrics.init(this, conf);
       } catch (Exception e) {
         e.printStackTrace();
       }
     }
-
-    ShutdownHookManager.get().addShutdownHook(new Runnable() {
-      @Override
-      public void run() {
-        try {
-          if (metricsLogBW != null)  {
-            metricsLogBW.write("]");
-            metricsLogBW.close();
-          }
-          if (web != null) {
-            web.stop();
-          }
-          tearDown();
-        } catch (Exception e) {
-          e.printStackTrace();
-        }
-      }
-    }, SHUTDOWN_HOOK_PRIORITY);
   }
 
   @Override
   public Allocation allocate(ApplicationAttemptId attemptId,
-                             List<ResourceRequest> resourceRequests,
-                             List<ContainerId> containerIds,
-                             List<String> strings, List<String> strings2,
-                             ContainerUpdates updateRequests) {
+      List<ResourceRequest> resourceRequests, List<ContainerId> containerIds,
+      List<String> strings, List<String> strings2,
+      ContainerUpdates updateRequests) {
     if (metricsON) {
-      final Timer.Context context = schedulerAllocateTimer.time();
+      final Timer.Context context = schedulerMetrics.getSchedulerAllocateTimer()
+          .time();
       Allocation allocation = null;
       try {
-        allocation = super.allocate(attemptId, resourceRequests,
-                containerIds, strings, strings2, updateRequests);
+        allocation = super
+            .allocate(attemptId, resourceRequests, containerIds, strings,
+                strings2, updateRequests);
         return allocation;
       } finally {
         context.stop();
-        schedulerAllocateCounter.inc();
+        schedulerMetrics.increaseSchedulerAllocationCounter();
         try {
           updateQueueWithAllocateRequest(allocation, attemptId,
                   resourceRequests, containerIds);
@@ -195,81 +123,83 @@ public class SLSCapacityScheduler extends CapacityScheduler implements
         }
       }
     } else {
-      return super.allocate(attemptId,
-              resourceRequests, containerIds, strings, strings2, updateRequests);
+      return super.allocate(attemptId, resourceRequests, containerIds, strings,
+          strings2, updateRequests);
     }
   }
 
   @Override
   public void handle(SchedulerEvent schedulerEvent) {
-	    // metrics off
-	    if (! metricsON) {
-	      super.handle(schedulerEvent);
-	      return;
-	    }
-	    if(!running)    running = true;
-
-	    // metrics on
-	    Timer.Context handlerTimer = null;
-	    Timer.Context operationTimer = null;
-
-	    NodeUpdateSchedulerEventWrapper eventWrapper;
-	    try {
-	      //if (schedulerEvent instanceof NodeUpdateSchedulerEvent) {
-	      if (schedulerEvent.getType() == SchedulerEventType.NODE_UPDATE
-	              && schedulerEvent instanceof NodeUpdateSchedulerEvent) {
-	        eventWrapper = new NodeUpdateSchedulerEventWrapper(
-	                (NodeUpdateSchedulerEvent)schedulerEvent);
-	        schedulerEvent = eventWrapper;
-	        updateQueueWithNodeUpdate(eventWrapper);
-	      } else if (schedulerEvent.getType() == SchedulerEventType.APP_ATTEMPT_REMOVED
-	          && schedulerEvent instanceof AppAttemptRemovedSchedulerEvent) {
-	        // check if having AM Container, update resource usage information
-	        AppAttemptRemovedSchedulerEvent appRemoveEvent =
-	            (AppAttemptRemovedSchedulerEvent) schedulerEvent;
-	        ApplicationAttemptId appAttemptId =
-	                appRemoveEvent.getApplicationAttemptID();
-	        String queue = appQueueMap.get(appAttemptId);
-	        SchedulerAppReport app = super.getSchedulerAppInfo(appAttemptId);
-	        if (! app.getLiveContainers().isEmpty()) {  // have 0 or 1
-	          // should have one container which is AM container
-	          RMContainer rmc = app.getLiveContainers().iterator().next();
-	          updateQueueMetrics(queue,
-	                  rmc.getContainer().getResource().getMemory(),
-	                  rmc.getContainer().getResource().getVirtualCores());
-	        }
-	      }
-
-	      handlerTimer = schedulerHandleTimer.time();
-	      operationTimer = schedulerHandleTimerMap
-	              .get(schedulerEvent.getType()).time();
-
-	      super.handle(schedulerEvent);
-	    } finally {
-	      if (handlerTimer != null)     handlerTimer.stop();
-	      if (operationTimer != null)   operationTimer.stop();
-	      schedulerHandleCounter.inc();
-	      schedulerHandleCounterMap.get(schedulerEvent.getType()).inc();
-
-	      if (schedulerEvent.getType() == SchedulerEventType.APP_ATTEMPT_REMOVED
-	          && schedulerEvent instanceof AppAttemptRemovedSchedulerEvent) {
-	        SLSRunner.decreaseRemainingApps();
-	        AppAttemptRemovedSchedulerEvent appRemoveEvent =
-	                (AppAttemptRemovedSchedulerEvent) schedulerEvent;
-	        ApplicationAttemptId appAttemptId =
-	                appRemoveEvent.getApplicationAttemptID();
-	        appQueueMap.remove(appRemoveEvent.getApplicationAttemptID());
-	      } else if (schedulerEvent.getType() == SchedulerEventType.APP_ATTEMPT_ADDED
-	          && schedulerEvent instanceof AppAttemptAddedSchedulerEvent) {
-	        AppAttemptAddedSchedulerEvent appAddEvent =
-	                (AppAttemptAddedSchedulerEvent) schedulerEvent;
-          SchedulerApplication app =
-              applications.get(appAddEvent.getApplicationAttemptId()
-                  .getApplicationId());
-          appQueueMap.put(appAddEvent.getApplicationAttemptId(), app.getQueue()
-              .getQueueName());
-	      }
-	    }
+    if (!metricsON) {
+      super.handle(schedulerEvent);
+      return;
+    }
+
+    if (!schedulerMetrics.isRunning()) {
+      schedulerMetrics.setRunning(true);
+    }
+
+    Timer.Context handlerTimer = null;
+    Timer.Context operationTimer = null;
+
+    NodeUpdateSchedulerEventWrapper eventWrapper;
+    try {
+      if (schedulerEvent.getType() == SchedulerEventType.NODE_UPDATE
+          && schedulerEvent instanceof NodeUpdateSchedulerEvent) {
+        eventWrapper = new NodeUpdateSchedulerEventWrapper(
+            (NodeUpdateSchedulerEvent)schedulerEvent);
+        schedulerEvent = eventWrapper;
+        updateQueueWithNodeUpdate(eventWrapper);
+      } else if (schedulerEvent.getType() ==
+          SchedulerEventType.APP_ATTEMPT_REMOVED
+          && schedulerEvent instanceof AppAttemptRemovedSchedulerEvent) {
+        // check if having AM Container, update resource usage information
+        AppAttemptRemovedSchedulerEvent appRemoveEvent =
+            (AppAttemptRemovedSchedulerEvent) schedulerEvent;
+        ApplicationAttemptId appAttemptId =
+            appRemoveEvent.getApplicationAttemptID();
+        String queue = appQueueMap.get(appAttemptId);
+        SchedulerAppReport app = super.getSchedulerAppInfo(appAttemptId);
+        if (!app.getLiveContainers().isEmpty()) {  // have 0 or 1
+          // should have one container which is AM container
+          RMContainer rmc = app.getLiveContainers().iterator().next();
+          schedulerMetrics.updateQueueMetricsByRelease(
+              rmc.getContainer().getResource(), queue);
+        }
+      }
+
+      handlerTimer = schedulerMetrics.getSchedulerHandleTimer().time();
+      operationTimer = schedulerMetrics.getSchedulerHandleTimer(
+          schedulerEvent.getType()).time();
+
+      super.handle(schedulerEvent);
+    } finally {
+      if (handlerTimer != null) {
+        handlerTimer.stop();
+      }
+      if (operationTimer != null) {
+        operationTimer.stop();
+      }
+      schedulerMetrics.increaseSchedulerHandleCounter(schedulerEvent.getType());
+
+      if (schedulerEvent.getType() == SchedulerEventType.APP_ATTEMPT_REMOVED
+          && schedulerEvent instanceof AppAttemptRemovedSchedulerEvent) {
+        SLSRunner.decreaseRemainingApps();
+        AppAttemptRemovedSchedulerEvent appRemoveEvent =
+            (AppAttemptRemovedSchedulerEvent) schedulerEvent;
+        appQueueMap.remove(appRemoveEvent.getApplicationAttemptID());
+      } else if (schedulerEvent.getType() ==
+          SchedulerEventType.APP_ATTEMPT_ADDED
+          && schedulerEvent instanceof AppAttemptAddedSchedulerEvent) {
+        AppAttemptAddedSchedulerEvent appAddEvent =
+            (AppAttemptAddedSchedulerEvent) schedulerEvent;
+        SchedulerApplication app =
+            applications.get(appAddEvent.getApplicationAttemptId()
+                .getApplicationId());
+        appQueueMap.put(appAddEvent.getApplicationAttemptId(), app.getQueue()
+            .getQueueName());
+      }
+    }
   }
 
   private void updateQueueWithNodeUpdate(
@@ -294,7 +224,7 @@ public class SLSCapacityScheduler extends CapacityScheduler implements
         if (status.getExitStatus() == ContainerExitStatus.SUCCESS) {
           for (RMContainer rmc : app.getLiveContainers()) {
             if (rmc.getContainerId() == containerId) {
-              releasedMemory += rmc.getContainer().getResource().getMemory();
+              releasedMemory += rmc.getContainer().getResource().getMemorySize();
               releasedVCores += rmc.getContainer()
                       .getResource().getVirtualCores();
               break;
@@ -303,13 +233,14 @@ public class SLSCapacityScheduler extends CapacityScheduler implements
         } else if (status.getExitStatus() == ContainerExitStatus.ABORTED) {
           if (preemptionContainerMap.containsKey(containerId)) {
             Resource preResource = preemptionContainerMap.get(containerId);
-            releasedMemory += preResource.getMemory();
+            releasedMemory += preResource.getMemorySize();
             releasedVCores += preResource.getVirtualCores();
             preemptionContainerMap.remove(containerId);
           }
         }
         // update queue counters
-        updateQueueMetrics(queue, releasedMemory, releasedVCores);
+        schedulerMetrics.updateQueueMetricsByRelease(
+            Resource.newInstance(releasedMemory, releasedVCores), queue);
       }
     }
   }
@@ -388,405 +319,54 @@ public class SLSCapacityScheduler extends CapacityScheduler implements
     }
 
     // update metrics
-    SortedMap<String, Counter> counterMap = metrics.getCounters();
-    String names[] = new String[]{
-            "counter.queue." + queueName + ".pending.memory",
-            "counter.queue." + queueName + ".pending.cores",
-            "counter.queue." + queueName + ".allocated.memory",
-            "counter.queue." + queueName + ".allocated.cores"};
-    int values[] = new int[]{pendingResource.getMemory(),
-            pendingResource.getVirtualCores(),
-            allocatedResource.getMemory(), allocatedResource.getVirtualCores()};
-    for (int i = names.length - 1; i >= 0; i --) {
-      if (! counterMap.containsKey(names[i])) {
-        metrics.counter(names[i]);
-        counterMap = metrics.getCounters();
-      }
-      counterMap.get(names[i]).inc(values[i]);
-    }
-
-    queueLock.lock();
-    try {
-      if (! schedulerMetrics.isTracked(queueName)) {
-        schedulerMetrics.trackQueue(queueName);
-      }
-    } finally {
-      queueLock.unlock();
-    }
-  }
-
-  private void tearDown() throws IOException {
-    // close job runtime writer
-    if (jobRuntimeLogBW != null) {
-      jobRuntimeLogBW.close();
-    }
-    // shut pool
-    if (pool != null)  pool.shutdown();
-  }
-
-  @SuppressWarnings("unchecked")
-  private void initMetrics() throws Exception {
-    metrics = new MetricRegistry();
-    // configuration
-    metricsOutputDir = conf.get(SLSConfiguration.METRICS_OUTPUT_DIR);
-    int metricsWebAddressPort = conf.getInt(
-            SLSConfiguration.METRICS_WEB_ADDRESS_PORT,
-            SLSConfiguration.METRICS_WEB_ADDRESS_PORT_DEFAULT);
-    // create SchedulerMetrics for current scheduler
-    String schedulerMetricsType = conf.get(CapacityScheduler.class.getName());
-    Class schedulerMetricsClass = schedulerMetricsType == null?
-            defaultSchedulerMetricsMap.get(CapacityScheduler.class) :
-            Class.forName(schedulerMetricsType);
-    schedulerMetrics = (SchedulerMetrics)ReflectionUtils
-            .newInstance(schedulerMetricsClass, new Configuration());
-    schedulerMetrics.init(this, metrics);
-
-    // register various metrics
-    registerJvmMetrics();
-    registerClusterResourceMetrics();
-    registerContainerAppNumMetrics();
-    registerSchedulerMetrics();
-
-    // .csv output
-    initMetricsCSVOutput();
-
-    // start web app to provide real-time tracking
-    web = new SLSWebApp(this, metricsWebAddressPort);
-    web.start();
-
-    // a thread to update histogram timer
-    pool = new ScheduledThreadPoolExecutor(2);
-    pool.scheduleAtFixedRate(new HistogramsRunnable(), 0, 1000,
-            TimeUnit.MILLISECONDS);
-
-    // a thread to output metrics for real-tiem tracking
-    pool.scheduleAtFixedRate(new MetricsLogRunnable(), 0, 1000,
-            TimeUnit.MILLISECONDS);
-
-    // application running information
-    jobRuntimeLogBW = new BufferedWriter(
-        new OutputStreamWriter(new FileOutputStream(
-            metricsOutputDir + "/jobruntime.csv"), StandardCharsets.UTF_8));
-    jobRuntimeLogBW.write("JobID,real_start_time,real_end_time," +
-            "simulate_start_time,simulate_end_time" + EOL);
-    jobRuntimeLogBW.flush();
-  }
-
-  private void registerJvmMetrics() {
-    // add JVM gauges
-    metrics.register("variable.jvm.free.memory",
-      new Gauge<Long>() {
-        @Override
-        public Long getValue() {
-          return Runtime.getRuntime().freeMemory();
-        }
-      }
-    );
-    metrics.register("variable.jvm.max.memory",
-      new Gauge<Long>() {
-        @Override
-        public Long getValue() {
-          return Runtime.getRuntime().maxMemory();
-        }
-      }
-    );
-    metrics.register("variable.jvm.total.memory",
-      new Gauge<Long>() {
-        @Override
-        public Long getValue() {
-          return Runtime.getRuntime().totalMemory();
-        }
-      }
-    );
-  }
-
-  private void registerClusterResourceMetrics() {
-    metrics.register("variable.cluster.allocated.memory",
-      new Gauge<Long>() {
-        @Override
-        public Long getValue() {
-          if( getRootQueueMetrics() == null) {
-            return 0L;
-          } else {
-            return getRootQueueMetrics().getAllocatedMB();
-          }
-        }
-      }
-    );
-    metrics.register("variable.cluster.allocated.vcores",
-      new Gauge<Integer>() {
-        @Override
-        public Integer getValue() {
-          if(getRootQueueMetrics() == null) {
-            return 0;
-          } else {
-            return getRootQueueMetrics().getAllocatedVirtualCores();
-          }
-        }
-      }
-    );
-    metrics.register("variable.cluster.available.memory",
-      new Gauge<Long>() {
-        @Override
-        public Long getValue() {
-          if(getRootQueueMetrics() == null) {
-            return 0L;
-          } else {
-            return getRootQueueMetrics().getAvailableMB();
-          }
-        }
-      }
-    );
-    metrics.register("variable.cluster.available.vcores",
-      new Gauge<Integer>() {
-        @Override
-        public Integer getValue() {
-          if(getRootQueueMetrics() == null) {
-            return 0;
-          } else {
-            return getRootQueueMetrics().getAvailableVirtualCores();
-          }
-        }
-      }
-    );
+    schedulerMetrics.updateQueueMetrics(pendingResource, allocatedResource,
+        queueName);
   }
 
-  private void registerContainerAppNumMetrics() {
-    metrics.register("variable.running.application",
-      new Gauge<Integer>() {
-        @Override
-        public Integer getValue() {
-          if(getRootQueueMetrics() == null) {
-            return 0;
-          } else {
-            return getRootQueueMetrics().getAppsRunning();
-          }
-        }
-      }
-    );
-    metrics.register("variable.running.container",
-      new Gauge<Integer>() {
-        @Override
-        public Integer getValue() {
-          if(getRootQueueMetrics() == null) {
-            return 0;
-          } else {
-            return getRootQueueMetrics().getAllocatedContainers();
-          }
-        }
-      }
-    );
-  }
-
-  private void registerSchedulerMetrics() {
-    samplerLock.lock();
-    try {
-      // counters for scheduler operations
-      schedulerAllocateCounter = metrics.counter(
-              "counter.scheduler.operation.allocate");
-      schedulerHandleCounter = metrics.counter(
-              "counter.scheduler.operation.handle");
-      schedulerHandleCounterMap = new HashMap<SchedulerEventType, Counter>();
-      for (SchedulerEventType e : SchedulerEventType.values()) {
-        Counter counter = metrics.counter(
-                "counter.scheduler.operation.handle." + e);
-        schedulerHandleCounterMap.put(e, counter);
-      }
-      // timers for scheduler operations
-      int timeWindowSize = conf.getInt(
-              SLSConfiguration.METRICS_TIMER_WINDOW_SIZE,
-              SLSConfiguration.METRICS_TIMER_WINDOW_SIZE_DEFAULT);
-      schedulerAllocateTimer = new Timer(
-              new SlidingWindowReservoir(timeWindowSize));
-      schedulerHandleTimer = new Timer(
-              new SlidingWindowReservoir(timeWindowSize));
-      schedulerHandleTimerMap = new HashMap<SchedulerEventType, Timer>();
-      for (SchedulerEventType e : SchedulerEventType.values()) {
-        Timer timer = new Timer(new SlidingWindowReservoir(timeWindowSize));
-        schedulerHandleTimerMap.put(e, timer);
-      }
-      // histogram for scheduler operations (Samplers)
-      schedulerHistogramList = new ArrayList<Histogram>();
-      histogramTimerMap = new HashMap<Histogram, Timer>();
-      Histogram schedulerAllocateHistogram = new Histogram(
-              new SlidingWindowReservoir(SAMPLING_SIZE));
-      metrics.register("sampler.scheduler.operation.allocate.timecost",
-              schedulerAllocateHistogram);
-      schedulerHistogramList.add(schedulerAllocateHistogram);
-      histogramTimerMap.put(schedulerAllocateHistogram, schedulerAllocateTimer);
-      Histogram schedulerHandleHistogram = new Histogram(
-              new SlidingWindowReservoir(SAMPLING_SIZE));
-      metrics.register("sampler.scheduler.operation.handle.timecost",
-              schedulerHandleHistogram);
-      schedulerHistogramList.add(schedulerHandleHistogram);
-      histogramTimerMap.put(schedulerHandleHistogram, schedulerHandleTimer);
-      for (SchedulerEventType e : SchedulerEventType.values()) {
-        Histogram histogram = new Histogram(
-                new SlidingWindowReservoir(SAMPLING_SIZE));
-        metrics.register(
-                "sampler.scheduler.operation.handle." + e + ".timecost",
-                histogram);
-        schedulerHistogramList.add(histogram);
-        histogramTimerMap.put(histogram, schedulerHandleTimerMap.get(e));
-      }
-    } finally {
-      samplerLock.unlock();
+  private void initQueueMetrics(CSQueue queue) {
+    if (queue instanceof LeafQueue) {
+      schedulerMetrics.initQueueMetric(queue.getQueueName());
+      return;
     }
-  }
 
-  private void initMetricsCSVOutput() {
-    int timeIntervalMS = conf.getInt(
-            SLSConfiguration.METRICS_RECORD_INTERVAL_MS,
-            SLSConfiguration.METRICS_RECORD_INTERVAL_MS_DEFAULT);
-    File dir = new File(metricsOutputDir + "/metrics");
-    if(! dir.exists()
-            && ! dir.mkdirs()) {
-      LOG.error("Cannot create directory {}", dir.getAbsoluteFile());
+    for (CSQueue child : queue.getChildQueues()) {
+      initQueueMetrics(child);
     }
-    final CsvReporter reporter = CsvReporter.forRegistry(metrics)
-            .formatFor(Locale.US)
-            .convertRatesTo(TimeUnit.SECONDS)
-            .convertDurationsTo(TimeUnit.MILLISECONDS)
-            .build(new File(metricsOutputDir + "/metrics"));
-    reporter.start(timeIntervalMS, TimeUnit.MILLISECONDS);
   }
+  @Override
+  public void serviceInit(Configuration configuration) throws Exception {
+    super.serviceInit(configuration);
 
-  class HistogramsRunnable implements Runnable {
-    @Override
-    public void run() {
-      samplerLock.lock();
-      try {
-        for (Histogram histogram : schedulerHistogramList) {
-          Timer timer = histogramTimerMap.get(histogram);
-          histogram.update((int) timer.getSnapshot().getMean());
-        }
-      } finally {
-        samplerLock.unlock();
-      }
-    }
-  }
-
-  class MetricsLogRunnable implements Runnable {
-    private boolean firstLine = true;
-    public MetricsLogRunnable() {
-      try {
-        metricsLogBW = new BufferedWriter(
-            new OutputStreamWriter(new FileOutputStream(
-                metricsOutputDir + "/realtimetrack.json"),
-                StandardCharsets.UTF_8));
-        metricsLogBW.write("[");
-      } catch (IOException e) {
-        e.printStackTrace();
-      }
-    }
-
-    @Override
-    public void run() {
-      if(running) {
-        // all WebApp to get real tracking json
-        String metrics = web.generateRealTimeTrackingMetrics();
-        // output
-        try {
-          if(firstLine) {
-            metricsLogBW.write(metrics + EOL);
-            firstLine = false;
-          } else {
-            metricsLogBW.write("," + metrics + EOL);
-          }
-          metricsLogBW.flush();
-        } catch (IOException e) {
-          e.printStackTrace();
-        }
-      }
+    if (metricsON) {
+      initQueueMetrics(getRootQueue());
     }
   }
 
-  // the following functions are used by AMSimulator
-  public void addAMRuntime(ApplicationId appId,
-                           long traceStartTimeMS, long traceEndTimeMS,
-                           long simulateStartTimeMS, long simulateEndTimeMS) {
-
+  @Override
+  public void serviceStop() throws Exception {
     try {
-      // write job runtime information
-      StringBuilder sb = new StringBuilder();
-      sb.append(appId).append(",").append(traceStartTimeMS).append(",")
-              .append(traceEndTimeMS).append(",").append(simulateStartTimeMS)
-              .append(",").append(simulateEndTimeMS);
-      jobRuntimeLogBW.write(sb.toString() + EOL);
-      jobRuntimeLogBW.flush();
-    } catch (IOException e) {
+      schedulerMetrics.tearDown();
+    } catch (Exception e) {
       e.printStackTrace();
     }
+    super.serviceStop();
   }
 
-  private void updateQueueMetrics(String queue,
-                                  int releasedMemory, int releasedVCores) {
-    // update queue counters
-    SortedMap<String, Counter> counterMap = metrics.getCounters();
-    if (releasedMemory != 0) {
-      String name = "counter.queue." + queue + ".allocated.memory";
-      if (! counterMap.containsKey(name)) {
-        metrics.counter(name);
-        counterMap = metrics.getCounters();
-      }
-      counterMap.get(name).inc(-releasedMemory);
-    }
-    if (releasedVCores != 0) {
-      String name = "counter.queue." + queue + ".allocated.cores";
-      if (! counterMap.containsKey(name)) {
-        metrics.counter(name);
-        counterMap = metrics.getCounters();
-      }
-      counterMap.get(name).inc(-releasedVCores);
-    }
-  }
-
-  public void setQueueSet(Set<String> queues) {
-    this.queueSet = queues;
-  }
-
-  public Set<String> getQueueSet() {
-    return this.queueSet;
-  }
-
-  public void setTrackedAppSet(Set<String> apps) {
-    this.trackedAppSet = apps;
-  }
-
-  public Set<String> getTrackedAppSet() {
-    return this.trackedAppSet;
-  }
-
-  public MetricRegistry getMetrics() {
-    return metrics;
-  }
 
   public SchedulerMetrics getSchedulerMetrics() {
     return schedulerMetrics;
   }
 
-  // API open to out classes
-  public void addTrackedApp(ApplicationAttemptId appAttemptId,
-                            String oldAppId) {
-    if (metricsON) {
-      schedulerMetrics.trackApp(appAttemptId, oldAppId);
-    }
-  }
-
-  public void removeTrackedApp(ApplicationAttemptId appAttemptId,
-                               String oldAppId) {
-    if (metricsON) {
-      schedulerMetrics.untrackApp(appAttemptId, oldAppId);
-    }
-  }
-
   @Override
   public Configuration getConf() {
     return conf;
   }
 
-
-
-
-}
-
+  public String getRealQueueName(String queue) throws YarnException {
+    if (getQueue(queue) == null) {
+      throw new YarnException("Can't find the queue by the given name: " + queue
+          + "! Please check if queue " + queue + " is in the allocation file.");
+    }
+    return getQueue(queue).getQueueName();
+  }
+}

+ 346 - 0
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSFairScheduler.java

@@ -0,0 +1,346 @@
+/**
+ * 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.sls.scheduler;
+
+import com.codahale.metrics.Timer;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSLeafQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
+import org.apache.hadoop.yarn.sls.SLSRunner;
+import org.apache.hadoop.yarn.sls.conf.SLSConfiguration;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+@Private
+@Unstable
+public class SLSFairScheduler extends FairScheduler
+    implements SchedulerWrapper, Configurable {
+  private SchedulerMetrics schedulerMetrics;
+  private boolean metricsON;
+  private Tracker tracker;
+
+  private Map<ContainerId, Resource> preemptionContainerMap =
+      new ConcurrentHashMap<>();
+
+  public SchedulerMetrics getSchedulerMetrics() {
+    return schedulerMetrics;
+  }
+
+  public Tracker getTracker() {
+    return tracker;
+  }
+
+  public SLSFairScheduler() {
+    tracker = new Tracker();
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    super.setConfig(conf);
+
+    metricsON = conf.getBoolean(SLSConfiguration.METRICS_SWITCH, true);
+    if (metricsON) {
+      try {
+        schedulerMetrics = SchedulerMetrics.getInstance(conf,
+            FairScheduler.class);
+        schedulerMetrics.init(this, conf);
+      } catch (Exception e) {
+        e.printStackTrace();
+      }
+    }
+  }
+
+  @Override
+  public Allocation allocate(ApplicationAttemptId attemptId,
+      List<ResourceRequest> resourceRequests, List<ContainerId> containerIds,
+      List<String> blacklistAdditions, List<String> blacklistRemovals,
+      ContainerUpdates updateRequests) {
+    if (metricsON) {
+      final Timer.Context context = schedulerMetrics.getSchedulerAllocateTimer()
+          .time();
+      Allocation allocation = null;
+      try {
+        allocation = super.allocate(attemptId, resourceRequests, containerIds,
+            blacklistAdditions, blacklistRemovals, updateRequests);
+        return allocation;
+      } finally {
+        context.stop();
+        schedulerMetrics.increaseSchedulerAllocationCounter();
+        try {
+          updateQueueWithAllocateRequest(allocation, attemptId,
+              resourceRequests, containerIds);
+        } catch (IOException e) {
+          e.printStackTrace();
+        }
+      }
+    } else {
+      return super.allocate(attemptId, resourceRequests, containerIds,
+          blacklistAdditions, blacklistRemovals, updateRequests);
+    }
+  }
+
+  @Override
+  public void handle(SchedulerEvent schedulerEvent) {
+    // metrics off
+    if (!metricsON) {
+      super.handle(schedulerEvent);
+      return;
+    }
+
+    // metrics on
+    if(!schedulerMetrics.isRunning()) {
+      schedulerMetrics.setRunning(true);
+    }
+
+    Timer.Context handlerTimer = null;
+    Timer.Context operationTimer = null;
+
+    NodeUpdateSchedulerEventWrapper eventWrapper;
+    try {
+      if (schedulerEvent.getType() == SchedulerEventType.NODE_UPDATE
+          && schedulerEvent instanceof NodeUpdateSchedulerEvent) {
+        eventWrapper = new NodeUpdateSchedulerEventWrapper(
+            (NodeUpdateSchedulerEvent)schedulerEvent);
+        schedulerEvent = eventWrapper;
+        updateQueueWithNodeUpdate(eventWrapper);
+      } else if (
+          schedulerEvent.getType() == SchedulerEventType.APP_ATTEMPT_REMOVED
+          && schedulerEvent instanceof AppAttemptRemovedSchedulerEvent) {
+        // check if having AM Container, update resource usage information
+        AppAttemptRemovedSchedulerEvent appRemoveEvent =
+            (AppAttemptRemovedSchedulerEvent) schedulerEvent;
+        ApplicationAttemptId appAttemptId =
+            appRemoveEvent.getApplicationAttemptID();
+        String queueName = getSchedulerApp(appAttemptId).getQueue().getName();
+        SchedulerAppReport app = getSchedulerAppInfo(appAttemptId);
+        if (!app.getLiveContainers().isEmpty()) {  // have 0 or 1
+          // should have one container which is AM container
+          RMContainer rmc = app.getLiveContainers().iterator().next();
+          schedulerMetrics.updateQueueMetricsByRelease(
+              rmc.getContainer().getResource(), queueName);
+        }
+      }
+
+      handlerTimer = schedulerMetrics.getSchedulerHandleTimer().time();
+      operationTimer = schedulerMetrics.getSchedulerHandleTimer(
+          schedulerEvent.getType()).time();
+
+      super.handle(schedulerEvent);
+    } finally {
+      if (handlerTimer != null) {
+        handlerTimer.stop();
+      }
+      if (operationTimer != null) {
+        operationTimer.stop();
+      }
+      schedulerMetrics.increaseSchedulerHandleCounter(schedulerEvent.getType());
+
+      if (schedulerEvent.getType() == SchedulerEventType.APP_ATTEMPT_REMOVED
+          && schedulerEvent instanceof AppAttemptRemovedSchedulerEvent) {
+        SLSRunner.decreaseRemainingApps();
+      }
+    }
+  }
+
+  private void updateQueueWithNodeUpdate(
+      NodeUpdateSchedulerEventWrapper eventWrapper) {
+    RMNodeWrapper node = (RMNodeWrapper) eventWrapper.getRMNode();
+    List<UpdatedContainerInfo> containerList = node.getContainerUpdates();
+    for (UpdatedContainerInfo info : containerList) {
+      for (ContainerStatus status : info.getCompletedContainers()) {
+        ContainerId containerId = status.getContainerId();
+        SchedulerAppReport app = super.getSchedulerAppInfo(
+            containerId.getApplicationAttemptId());
+
+        if (app == null) {
+          // this happens for the AM container
+          // The app have already removed when the NM sends the release
+          // information.
+          continue;
+        }
+
+        int releasedMemory = 0, releasedVCores = 0;
+        if (status.getExitStatus() == ContainerExitStatus.SUCCESS) {
+          for (RMContainer rmc : app.getLiveContainers()) {
+            if (rmc.getContainerId() == containerId) {
+              Resource resource = rmc.getContainer().getResource();
+              releasedMemory += resource.getMemorySize();
+              releasedVCores += resource.getVirtualCores();
+              break;
+            }
+          }
+        } else if (status.getExitStatus() == ContainerExitStatus.ABORTED) {
+          if (preemptionContainerMap.containsKey(containerId)) {
+            Resource preResource = preemptionContainerMap.get(containerId);
+            releasedMemory += preResource.getMemorySize();
+            releasedVCores += preResource.getVirtualCores();
+            preemptionContainerMap.remove(containerId);
+          }
+        }
+        // update queue counters
+        String queue = getSchedulerApp(containerId.getApplicationAttemptId()).
+            getQueueName();
+        schedulerMetrics.updateQueueMetricsByRelease(
+            Resource.newInstance(releasedMemory, releasedVCores), queue);
+      }
+    }
+  }
+
+  private void updateQueueWithAllocateRequest(Allocation allocation,
+      ApplicationAttemptId attemptId,
+      List<ResourceRequest> resourceRequests,
+      List<ContainerId> containerIds) throws IOException {
+    // update queue information
+    Resource pendingResource = Resources.createResource(0, 0);
+    Resource allocatedResource = Resources.createResource(0, 0);
+    // container requested
+    for (ResourceRequest request : resourceRequests) {
+      if (request.getResourceName().equals(ResourceRequest.ANY)) {
+        Resources.addTo(pendingResource,
+            Resources.multiply(request.getCapability(),
+                request.getNumContainers()));
+      }
+    }
+    // container allocated
+    for (Container container : allocation.getContainers()) {
+      Resources.addTo(allocatedResource, container.getResource());
+      Resources.subtractFrom(pendingResource, container.getResource());
+    }
+    // container released from AM
+    SchedulerAppReport report = super.getSchedulerAppInfo(attemptId);
+    for (ContainerId containerId : containerIds) {
+      Container container = null;
+      for (RMContainer c : report.getLiveContainers()) {
+        if (c.getContainerId().equals(containerId)) {
+          container = c.getContainer();
+          break;
+        }
+      }
+      if (container != null) {
+        // released allocated containers
+        Resources.subtractFrom(allocatedResource, container.getResource());
+      } else {
+        for (RMContainer c : report.getReservedContainers()) {
+          if (c.getContainerId().equals(containerId)) {
+            container = c.getContainer();
+            break;
+          }
+        }
+        if (container != null) {
+          // released reserved containers
+          Resources.subtractFrom(pendingResource, container.getResource());
+        }
+      }
+    }
+    // containers released/preemption from scheduler
+    Set<ContainerId> preemptionContainers = new HashSet<ContainerId>();
+    if (allocation.getContainerPreemptions() != null) {
+      preemptionContainers.addAll(allocation.getContainerPreemptions());
+    }
+    if (allocation.getStrictContainerPreemptions() != null) {
+      preemptionContainers.addAll(allocation.getStrictContainerPreemptions());
+    }
+    if (!preemptionContainers.isEmpty()) {
+      for (ContainerId containerId : preemptionContainers) {
+        if (!preemptionContainerMap.containsKey(containerId)) {
+          Container container = null;
+          for (RMContainer c : report.getLiveContainers()) {
+            if (c.getContainerId().equals(containerId)) {
+              container = c.getContainer();
+              break;
+            }
+          }
+          if (container != null) {
+            preemptionContainerMap.put(containerId, container.getResource());
+          }
+        }
+
+      }
+    }
+
+    // update metrics
+    String queueName = getSchedulerApp(attemptId).getQueueName();
+    schedulerMetrics.updateQueueMetrics(pendingResource, allocatedResource,
+        queueName);
+  }
+
+  private void initQueueMetrics(FSQueue queue) {
+    if (queue instanceof FSLeafQueue) {
+      schedulerMetrics.initQueueMetric(queue.getQueueName());
+      return;
+    }
+
+    for (FSQueue child : queue.getChildQueues()) {
+      initQueueMetrics(child);
+    }
+  }
+
+  @Override
+  public void serviceInit(Configuration conf) throws Exception {
+    super.serviceInit(conf);
+    if (metricsON) {
+      initQueueMetrics(getQueueManager().getRootQueue());
+    }
+  }
+
+  @Override
+  public void serviceStop() throws Exception {
+    try {
+      schedulerMetrics.tearDown();
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+    super.serviceStop();
+  }
+
+  public String getRealQueueName(String queue) throws YarnException {
+    if (!getQueueManager().exists(queue)) {
+      throw new YarnException("Can't find the queue by the given name: " + queue
+          + "! Please check if queue " + queue + " is in the allocation file.");
+    }
+    return getQueueManager().getQueue(queue).getQueueName();
+  }
+}
+

+ 571 - 34
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerMetrics.java

@@ -18,66 +18,218 @@
 
 package org.apache.hadoop.yarn.sls.scheduler;
 
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Map;
 import java.util.Set;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.OutputStreamWriter;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.SortedMap;
+import java.util.Locale;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.concurrent.locks.Lock;
 
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler
-        .ResourceScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler
-        .SchedulerAppReport;
-
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.CsvReporter;
 import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Histogram;
 import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.SlidingWindowReservoir;
+import com.codahale.metrics.Timer;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
+import org.apache.hadoop.yarn.sls.conf.SLSConfiguration;
+import org.apache.hadoop.yarn.sls.web.SLSWebApp;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @Private
 @Unstable
 public abstract class SchedulerMetrics {
+  private static final String EOL = System.getProperty("line.separator");
+  private static final int SAMPLING_SIZE = 60;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SchedulerMetrics.class);
+
   protected ResourceScheduler scheduler;
   protected Set<String> trackedQueues;
   protected MetricRegistry metrics;
   protected Set<String> appTrackedMetrics;
   protected Set<String> queueTrackedMetrics;
-  
+
+  private Configuration conf;
+  private ScheduledExecutorService pool;
+  private SLSWebApp web;
+
+  // metrics
+  private String metricsOutputDir;
+  private BufferedWriter metricsLogBW;
+  private BufferedWriter jobRuntimeLogBW;
+  private boolean running = false;
+
+  // counters for scheduler allocate/handle operations
+  private Counter schedulerAllocateCounter;
+  private Counter schedulerHandleCounter;
+  private Map<SchedulerEventType, Counter> schedulerHandleCounterMap;
+
+  // Timers for scheduler allocate/handle operations
+  private Timer schedulerAllocateTimer;
+  private Timer schedulerHandleTimer;
+  private Map<SchedulerEventType, Timer> schedulerHandleTimerMap;
+  private List<Histogram> schedulerHistogramList;
+  private Map<Histogram, Timer> histogramTimerMap;
+  private Lock samplerLock;
+  private Lock queueLock;
+
+  static Class getSchedulerMetricsClass(Configuration conf,
+      Class schedulerClass) throws ClassNotFoundException {
+    Class metricClass = null;
+    String schedulerMetricsType = conf.get(schedulerClass.getName());
+    if (schedulerMetricsType != null) {
+      metricClass = Class.forName(schedulerMetricsType);
+    }
+
+    if (schedulerClass.equals(FairScheduler.class)) {
+      metricClass = FairSchedulerMetrics.class;
+    } else if (schedulerClass.equals(CapacityScheduler.class)) {
+      metricClass = CapacitySchedulerMetrics.class;
+    } else if (schedulerClass.equals(FifoScheduler.class)) {
+      metricClass = FifoSchedulerMetrics.class;
+    }
+
+    return metricClass;
+  }
+
+  @SuppressWarnings("unchecked")
+  static SchedulerMetrics getInstance(Configuration conf, Class schedulerClass)
+      throws ClassNotFoundException {
+    Class schedulerMetricClass = getSchedulerMetricsClass(conf, schedulerClass);
+    return (SchedulerMetrics) ReflectionUtils
+        .newInstance(schedulerMetricClass, new Configuration());
+  }
+
   public SchedulerMetrics() {
-    appTrackedMetrics = new HashSet<String>();
+    metrics = new MetricRegistry();
+
+    appTrackedMetrics = new HashSet<>();
     appTrackedMetrics.add("live.containers");
     appTrackedMetrics.add("reserved.containers");
-    queueTrackedMetrics = new HashSet<String>();
+
+    queueTrackedMetrics = new HashSet<>();
+    trackedQueues = new HashSet<>();
+
+    samplerLock = new ReentrantLock();
+    queueLock = new ReentrantLock();
   }
-  
-  public void init(ResourceScheduler scheduler, MetricRegistry metrics) {
-    this.scheduler = scheduler;
-    this.trackedQueues = new HashSet<String>();
-    this.metrics = metrics;
+
+  void init(ResourceScheduler resourceScheduler, Configuration config)
+      throws Exception {
+    this.scheduler = resourceScheduler;
+    this.conf = config;
+
+    metricsOutputDir = conf.get(SLSConfiguration.METRICS_OUTPUT_DIR);
+
+    // register various metrics
+    registerJvmMetrics();
+    registerClusterResourceMetrics();
+    registerContainerAppNumMetrics();
+    registerSchedulerMetrics();
+
+    // .csv output
+    initMetricsCSVOutput();
+
+    // start web app to provide real-time tracking
+    int metricsWebAddressPort = conf.getInt(
+        SLSConfiguration.METRICS_WEB_ADDRESS_PORT,
+        SLSConfiguration.METRICS_WEB_ADDRESS_PORT_DEFAULT);
+    web = new SLSWebApp((SchedulerWrapper)scheduler, metricsWebAddressPort);
+    web.start();
+
+    // a thread to update histogram timer
+    pool = new ScheduledThreadPoolExecutor(2);
+    pool.scheduleAtFixedRate(new HistogramsRunnable(), 0, 1000,
+        TimeUnit.MILLISECONDS);
+
+    // a thread to output metrics for real-tiem tracking
+    pool.scheduleAtFixedRate(new MetricsLogRunnable(), 0, 1000,
+        TimeUnit.MILLISECONDS);
+
+    // application running information
+    jobRuntimeLogBW =
+        new BufferedWriter(new OutputStreamWriter(new FileOutputStream(
+            metricsOutputDir + "/jobruntime.csv"), "UTF-8"));
+    jobRuntimeLogBW.write("JobID,real_start_time,real_end_time," +
+        "simulate_start_time,simulate_end_time" + EOL);
+    jobRuntimeLogBW.flush();
   }
-  
-  public void trackApp(final ApplicationAttemptId appAttemptId,
-                       String oldAppId) {
+
+  public MetricRegistry getMetrics() {
+    return metrics;
+  }
+
+  protected SchedulerApplicationAttempt getSchedulerAppAttempt(
+      ApplicationId appId) {
+    AbstractYarnScheduler yarnScheduler = (AbstractYarnScheduler)scheduler;
+    SchedulerApplication app = (SchedulerApplication)yarnScheduler
+        .getSchedulerApplications().get(appId);
+    if (app == null) {
+      return null;
+    }
+    return app.getCurrentAppAttempt();
+  }
+
+  public void trackApp(final ApplicationId appId, String oldAppId) {
     metrics.register("variable.app." + oldAppId + ".live.containers",
-      new Gauge<Integer>() {
-        @Override
-        public Integer getValue() {
-          SchedulerAppReport app = scheduler.getSchedulerAppInfo(appAttemptId);
-          return app.getLiveContainers().size();
+        new Gauge<Integer>() {
+          @Override
+          public Integer getValue() {
+            SchedulerApplicationAttempt appAttempt =
+                getSchedulerAppAttempt(appId);
+            if (appAttempt != null) {
+              return appAttempt.getLiveContainers().size();
+            } else {
+              return 0;
+            }
+          }
         }
-      }
     );
     metrics.register("variable.app." + oldAppId + ".reserved.containers",
-      new Gauge<Integer>() {
-        @Override
-        public Integer getValue() {
-          SchedulerAppReport app = scheduler.getSchedulerAppInfo(appAttemptId);
-          return app.getReservedContainers().size();
+        new Gauge<Integer>() {
+          @Override
+          public Integer getValue() {
+            SchedulerApplicationAttempt appAttempt =
+                getSchedulerAppAttempt(appId);
+            if (appAttempt != null) {
+              return appAttempt.getReservedContainers().size();
+            } else {
+              return 0;
+            }
+          }
         }
-      }
     );
   }
-  
-  public void untrackApp(ApplicationAttemptId appAttemptId,
-      String oldAppId) {
+
+  public void untrackApp(String oldAppId) {
     for (String m : appTrackedMetrics) {
       metrics.remove("variable.app." + oldAppId + "." + m);
     }
@@ -98,7 +250,392 @@ public abstract class SchedulerMetrics {
   public Set<String> getAppTrackedMetrics() {
     return appTrackedMetrics;
   }
+
   public Set<String> getQueueTrackedMetrics() {
     return queueTrackedMetrics;
   }
+
+  private void registerJvmMetrics() {
+    // add JVM gauges
+    metrics.register("variable.jvm.free.memory",
+        new Gauge<Long>() {
+          @Override
+          public Long getValue() {
+            return Runtime.getRuntime().freeMemory();
+          }
+        }
+    );
+    metrics.register("variable.jvm.max.memory",
+        new Gauge<Long>() {
+          @Override
+          public Long getValue() {
+            return Runtime.getRuntime().maxMemory();
+          }
+        }
+    );
+    metrics.register("variable.jvm.total.memory",
+        new Gauge<Long>() {
+          @Override
+          public Long getValue() {
+            return Runtime.getRuntime().totalMemory();
+          }
+        }
+    );
+  }
+
+  private void registerClusterResourceMetrics() {
+    metrics.register("variable.cluster.allocated.memory",
+        new Gauge<Long>() {
+          @Override
+          public Long getValue() {
+            if (scheduler.getRootQueueMetrics() == null) {
+              return 0L;
+            } else {
+              return scheduler.getRootQueueMetrics().getAllocatedMB();
+            }
+          }
+        }
+    );
+    metrics.register("variable.cluster.allocated.vcores",
+        new Gauge<Integer>() {
+          @Override
+          public Integer getValue() {
+            if (scheduler.getRootQueueMetrics() == null) {
+              return 0;
+            } else {
+              return scheduler.getRootQueueMetrics().getAllocatedVirtualCores();
+            }
+          }
+        }
+    );
+    metrics.register("variable.cluster.available.memory",
+        new Gauge<Long>() {
+          @Override
+          public Long getValue() {
+            if (scheduler.getRootQueueMetrics() == null) {
+              return 0L;
+            } else {
+              return scheduler.getRootQueueMetrics().getAvailableMB();
+            }
+          }
+        }
+    );
+    metrics.register("variable.cluster.available.vcores",
+        new Gauge<Integer>() {
+          @Override
+          public Integer getValue() {
+            if (scheduler.getRootQueueMetrics() == null) {
+              return 0;
+            } else {
+              return scheduler.getRootQueueMetrics().getAvailableVirtualCores();
+            }
+          }
+        }
+    );
+  }
+
+  private void registerContainerAppNumMetrics() {
+    metrics.register("variable.running.application",
+        new Gauge<Integer>() {
+          @Override
+          public Integer getValue() {
+            if (scheduler.getRootQueueMetrics() == null) {
+              return 0;
+            } else {
+              return scheduler.getRootQueueMetrics().getAppsRunning();
+            }
+          }
+        }
+    );
+    metrics.register("variable.running.container",
+        new Gauge<Integer>() {
+          @Override
+          public Integer getValue() {
+            if (scheduler.getRootQueueMetrics() == null) {
+              return 0;
+            } else {
+              return scheduler.getRootQueueMetrics().getAllocatedContainers();
+            }
+          }
+        }
+    );
+  }
+
+  private void registerSchedulerMetrics() {
+    samplerLock.lock();
+    try {
+      // counters for scheduler operations
+      schedulerAllocateCounter = metrics.counter(
+          "counter.scheduler.operation.allocate");
+      schedulerHandleCounter = metrics.counter(
+          "counter.scheduler.operation.handle");
+      schedulerHandleCounterMap = new HashMap<>();
+      for (SchedulerEventType e : SchedulerEventType.values()) {
+        Counter counter = metrics.counter(
+            "counter.scheduler.operation.handle." + e);
+        schedulerHandleCounterMap.put(e, counter);
+      }
+      // timers for scheduler operations
+      int timeWindowSize = conf.getInt(
+          SLSConfiguration.METRICS_TIMER_WINDOW_SIZE,
+          SLSConfiguration.METRICS_TIMER_WINDOW_SIZE_DEFAULT);
+      schedulerAllocateTimer = new Timer(
+          new SlidingWindowReservoir(timeWindowSize));
+      schedulerHandleTimer = new Timer(
+          new SlidingWindowReservoir(timeWindowSize));
+      schedulerHandleTimerMap = new HashMap<>();
+      for (SchedulerEventType e : SchedulerEventType.values()) {
+        Timer timer = new Timer(new SlidingWindowReservoir(timeWindowSize));
+        schedulerHandleTimerMap.put(e, timer);
+      }
+      // histogram for scheduler operations (Samplers)
+      schedulerHistogramList = new ArrayList<>();
+      histogramTimerMap = new HashMap<>();
+      Histogram schedulerAllocateHistogram = new Histogram(
+          new SlidingWindowReservoir(SAMPLING_SIZE));
+      metrics.register("sampler.scheduler.operation.allocate.timecost",
+          schedulerAllocateHistogram);
+      schedulerHistogramList.add(schedulerAllocateHistogram);
+      histogramTimerMap.put(schedulerAllocateHistogram, schedulerAllocateTimer);
+      Histogram schedulerHandleHistogram = new Histogram(
+          new SlidingWindowReservoir(SAMPLING_SIZE));
+      metrics.register("sampler.scheduler.operation.handle.timecost",
+          schedulerHandleHistogram);
+      schedulerHistogramList.add(schedulerHandleHistogram);
+      histogramTimerMap.put(schedulerHandleHistogram, schedulerHandleTimer);
+      for (SchedulerEventType e : SchedulerEventType.values()) {
+        Histogram histogram = new Histogram(
+            new SlidingWindowReservoir(SAMPLING_SIZE));
+        metrics.register(
+            "sampler.scheduler.operation.handle." + e + ".timecost",
+            histogram);
+        schedulerHistogramList.add(histogram);
+        histogramTimerMap.put(histogram, schedulerHandleTimerMap.get(e));
+      }
+    } finally {
+      samplerLock.unlock();
+    }
+  }
+
+  private void initMetricsCSVOutput() {
+    int timeIntervalMS = conf.getInt(
+        SLSConfiguration.METRICS_RECORD_INTERVAL_MS,
+        SLSConfiguration.METRICS_RECORD_INTERVAL_MS_DEFAULT);
+    File dir = new File(metricsOutputDir + "/metrics");
+    if(!dir.exists() && !dir.mkdirs()) {
+      LOG.error("Cannot create directory {}", dir.getAbsoluteFile());
+    }
+    final CsvReporter reporter = CsvReporter.forRegistry(metrics)
+        .formatFor(Locale.US)
+        .convertRatesTo(TimeUnit.SECONDS)
+        .convertDurationsTo(TimeUnit.MILLISECONDS)
+        .build(new File(metricsOutputDir + "/metrics"));
+    reporter.start(timeIntervalMS, TimeUnit.MILLISECONDS);
+  }
+
+  boolean isRunning() {
+    return running;
+  }
+
+  void setRunning(boolean running) {
+    this.running = running;
+  }
+
+  class HistogramsRunnable implements Runnable {
+    @Override
+    public void run() {
+      samplerLock.lock();
+      try {
+        for (Histogram histogram : schedulerHistogramList) {
+          Timer timer = histogramTimerMap.get(histogram);
+          histogram.update((int) timer.getSnapshot().getMean());
+        }
+      } finally {
+        samplerLock.unlock();
+      }
+    }
+  }
+
+  class MetricsLogRunnable implements Runnable {
+    private boolean firstLine = true;
+
+    MetricsLogRunnable() {
+      try {
+        metricsLogBW =
+            new BufferedWriter(new OutputStreamWriter(new FileOutputStream(
+                metricsOutputDir + "/realtimetrack.json"), "UTF-8"));
+        metricsLogBW.write("[");
+      } catch (IOException e) {
+        LOG.info(e.getMessage());
+      }
+    }
+
+    @Override
+    public void run() {
+      if(running) {
+        // all WebApp to get real tracking json
+        String trackingMetrics = web.generateRealTimeTrackingMetrics();
+        // output
+        try {
+          if(firstLine) {
+            metricsLogBW.write(trackingMetrics + EOL);
+            firstLine = false;
+          } else {
+            metricsLogBW.write("," + trackingMetrics + EOL);
+          }
+          metricsLogBW.flush();
+        } catch (IOException e) {
+          LOG.info(e.getMessage());
+        }
+      }
+    }
+  }
+
+  void tearDown() throws Exception {
+    if (metricsLogBW != null)  {
+      metricsLogBW.write("]");
+      metricsLogBW.close();
+    }
+
+    if (web != null) {
+      web.stop();
+    }
+
+    if (jobRuntimeLogBW != null) {
+      jobRuntimeLogBW.close();
+    }
+
+    if (pool != null) {
+      pool.shutdown();
+    }
+  }
+
+  void increaseSchedulerAllocationCounter() {
+    schedulerAllocateCounter.inc();
+  }
+
+  void increaseSchedulerHandleCounter(SchedulerEventType schedulerEventType) {
+    schedulerHandleCounter.inc();
+    schedulerHandleCounterMap.get(schedulerEventType).inc();
+  }
+
+  Timer getSchedulerAllocateTimer() {
+    return schedulerAllocateTimer;
+  }
+
+  Timer getSchedulerHandleTimer() {
+    return schedulerHandleTimer;
+  }
+
+  Timer getSchedulerHandleTimer(SchedulerEventType schedulerEventType) {
+    return schedulerHandleTimerMap.get(schedulerEventType);
+  }
+
+  private enum QueueMetric {
+    PENDING_MEMORY("pending.memory"),
+    PENDING_VCORES("pending.cores"),
+    ALLOCATED_MEMORY("allocated.memory"),
+    ALLOCATED_VCORES("allocated.cores");
+
+    private String value;
+
+    QueueMetric(String value) {
+      this.value = value;
+    }
+  }
+
+  private String getQueueMetricName(String queue, QueueMetric metric) {
+    return "counter.queue." + queue + "." + metric.value;
+  }
+
+  private void traceQueueIfNotTraced(String queue) {
+    queueLock.lock();
+    try {
+      if (!isTracked(queue)) {
+        trackQueue(queue);
+      }
+    } finally {
+      queueLock.unlock();
+    }
+  }
+
+  void initQueueMetric(String queueName){
+    SortedMap<String, Counter> counterMap = metrics.getCounters();
+
+    for (QueueMetric queueMetric : QueueMetric.values()) {
+      String metricName = getQueueMetricName(queueName, queueMetric);
+      if (!counterMap.containsKey(metricName)) {
+        metrics.counter(metricName);
+        counterMap = metrics.getCounters();
+      }
+    }
+
+    traceQueueIfNotTraced(queueName);
+  }
+
+  void updateQueueMetrics(Resource pendingResource, Resource allocatedResource,
+      String queueName) {
+    SortedMap<String, Counter> counterMap = metrics.getCounters();
+    for(QueueMetric metric : QueueMetric.values()) {
+      String metricName = getQueueMetricName(queueName, metric);
+      if (!counterMap.containsKey(metricName)) {
+        metrics.counter(metricName);
+        counterMap = metrics.getCounters();
+      }
+
+      if (metric == QueueMetric.PENDING_MEMORY) {
+        counterMap.get(metricName).inc(pendingResource.getMemorySize());
+      } else if (metric == QueueMetric.PENDING_VCORES) {
+        counterMap.get(metricName).inc(pendingResource.getVirtualCores());
+      } else if (metric == QueueMetric.ALLOCATED_MEMORY) {
+        counterMap.get(metricName).inc(allocatedResource.getMemorySize());
+      } else if (metric == QueueMetric.ALLOCATED_VCORES){
+        counterMap.get(metricName).inc(allocatedResource.getVirtualCores());
+      }
+    }
+
+    traceQueueIfNotTraced(queueName);
+  }
+
+  void updateQueueMetricsByRelease(Resource releaseResource, String queue) {
+    SortedMap<String, Counter> counterMap = metrics.getCounters();
+    String name = getQueueMetricName(queue, QueueMetric.ALLOCATED_MEMORY);
+    if (!counterMap.containsKey(name)) {
+      metrics.counter(name);
+      counterMap = metrics.getCounters();
+    }
+    counterMap.get(name).inc(-releaseResource.getMemorySize());
+
+    String vcoreMetric =
+        getQueueMetricName(queue, QueueMetric.ALLOCATED_VCORES);
+    if (!counterMap.containsKey(vcoreMetric)) {
+      metrics.counter(vcoreMetric);
+      counterMap = metrics.getCounters();
+    }
+    counterMap.get(vcoreMetric).inc(-releaseResource.getVirtualCores());
+  }
+
+  public void addTrackedApp(ApplicationId appId,
+      String oldAppId) {
+    trackApp(appId, oldAppId);
+  }
+
+  public void removeTrackedApp(String oldAppId) {
+    untrackApp(oldAppId);
+  }
+
+  public void addAMRuntime(ApplicationId appId, long traceStartTimeMS,
+      long traceEndTimeMS, long simulateStartTimeMS, long simulateEndTimeMS) {
+    try {
+      // write job runtime information
+      StringBuilder sb = new StringBuilder();
+      sb.append(appId).append(",").append(traceStartTimeMS).append(",")
+          .append(traceEndTimeMS).append(",").append(simulateStartTimeMS)
+          .append(",").append(simulateEndTimeMS);
+      jobRuntimeLogBW.write(sb.toString() + EOL);
+      jobRuntimeLogBW.flush();
+    } catch (IOException e) {
+      LOG.info(e.getMessage());
+    }
+  }
 }

+ 8 - 19
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerWrapper.java

@@ -17,27 +17,16 @@
  */
 package org.apache.hadoop.yarn.sls.scheduler;
 
-import java.util.Set;
-
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-
-import com.codahale.metrics.MetricRegistry;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 
+@Private
+@Unstable
 public interface SchedulerWrapper {
+  SchedulerMetrics getSchedulerMetrics();
 
-	public MetricRegistry getMetrics();
-	public SchedulerMetrics getSchedulerMetrics();
-	public Set<String> getQueueSet();
-	public void setQueueSet(Set<String> queues);
-	public Set<String> getTrackedAppSet();
-	public void setTrackedAppSet(Set<String> apps);
-	public void addTrackedApp(ApplicationAttemptId appAttemptId,
-              String oldAppId);
-	public void removeTrackedApp(ApplicationAttemptId appAttemptId,
-                 String oldAppId);
-	public void addAMRuntime(ApplicationId appId,
-              long traceStartTimeMS, long traceEndTimeMS,
-              long simulateStartTimeMS, long simulateEndTimeMS);
+  Tracker getTracker();
 
+  String getRealQueueName(String queue) throws YarnException;
 }

+ 4 - 5
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/TaskRunner.java

@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.yarn.sls.scheduler;
 
-import java.io.IOException;
 import java.text.MessageFormat;
 import java.util.Queue;
 import java.util.concurrent.DelayQueue;
@@ -27,7 +26,6 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.yarn.exceptions.YarnException;
 
 @Private
 @Unstable
@@ -148,8 +146,8 @@ public class TaskRunner {
 
   @SuppressWarnings("unchecked")
   public void start() {
-    if (executor != null) {
-      throw new IllegalStateException("Already started");
+    if (executor != null && !executor.isTerminated()) {
+      throw new IllegalStateException("Executor already running");
     }
     DelayQueue preStartQueue = queue;
 
@@ -164,8 +162,9 @@ public class TaskRunner {
     }
   }
   
-  public void stop() {
+  public void stop() throws InterruptedException {
     executor.shutdownNow();
+    executor.awaitTermination(20, TimeUnit.SECONDS);
   }
 
   @SuppressWarnings("unchecked")

+ 46 - 0
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/Tracker.java

@@ -0,0 +1,46 @@
+/**
+ * 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.sls.scheduler;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+import java.util.Set;
+
+@Private
+@Unstable
+public class Tracker {
+  private Set<String> queueSet;
+  private Set<String> trackedAppSet;
+
+  public void setQueueSet(Set<String> queues) {
+    queueSet = queues;
+  }
+
+  public Set<String> getQueueSet() {
+    return queueSet;
+  }
+
+  public void setTrackedAppSet(Set<String> apps) {
+    trackedAppSet = apps;
+  }
+
+  public Set<String> getTrackedAppSet() {
+    return trackedAppSet;
+  }
+}

+ 306 - 0
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthJob.java

@@ -0,0 +1,306 @@
+/**
+ * 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.sls.synthetic;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.math3.distribution.LogNormalDistribution;
+import org.apache.commons.math3.random.JDKRandomGenerator;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.TaskStatus.State;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.tools.rumen.*;
+import org.apache.hadoop.tools.rumen.Pre21JobHistoryConstants.Values;
+
+import java.util.Arrays;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.hadoop.mapreduce.MRJobConfig.QUEUE_NAME;
+
+/**
+ * Generates random task data for a synthetic job.
+ */
+public class SynthJob implements JobStory {
+
+  @SuppressWarnings("StaticVariableName")
+  private static Log LOG = LogFactory.getLog(SynthJob.class);
+
+  private final Configuration conf;
+  private final int id;
+
+  @SuppressWarnings("ConstantName")
+  private static final AtomicInteger sequence = new AtomicInteger(0);
+  private final String name;
+  private final String queueName;
+  private final SynthJobClass jobClass;
+
+  // job timing
+  private final long submitTime;
+  private final long duration;
+  private final long deadline;
+
+  private final int numMapTasks;
+  private final int numRedTasks;
+  private final long mapMaxMemory;
+  private final long reduceMaxMemory;
+  private final long mapMaxVcores;
+  private final long reduceMaxVcores;
+  private final long[] mapRuntime;
+  private final float[] reduceRuntime;
+  private long totMapRuntime;
+  private long totRedRuntime;
+
+  public SynthJob(JDKRandomGenerator rand, Configuration conf,
+      SynthJobClass jobClass, long actualSubmissionTime) {
+
+    this.conf = conf;
+    this.jobClass = jobClass;
+
+    this.duration = MILLISECONDS.convert(jobClass.getDur(), SECONDS);
+    this.numMapTasks = jobClass.getMtasks();
+    this.numRedTasks = jobClass.getRtasks();
+
+    // sample memory distributions, correct for sub-minAlloc sizes
+    long tempMapMaxMemory = jobClass.getMapMaxMemory();
+    this.mapMaxMemory = tempMapMaxMemory < MRJobConfig.DEFAULT_MAP_MEMORY_MB
+        ? MRJobConfig.DEFAULT_MAP_MEMORY_MB : tempMapMaxMemory;
+    long tempReduceMaxMemory = jobClass.getReduceMaxMemory();
+    this.reduceMaxMemory =
+            tempReduceMaxMemory < MRJobConfig.DEFAULT_REDUCE_MEMORY_MB
+            ? MRJobConfig.DEFAULT_REDUCE_MEMORY_MB : tempReduceMaxMemory;
+
+    // sample vcores distributions, correct for sub-minAlloc sizes
+    long tempMapMaxVCores = jobClass.getMapMaxVcores();
+    this.mapMaxVcores = tempMapMaxVCores < MRJobConfig.DEFAULT_MAP_CPU_VCORES
+        ? MRJobConfig.DEFAULT_MAP_CPU_VCORES : tempMapMaxVCores;
+    long tempReduceMaxVcores = jobClass.getReduceMaxVcores();
+    this.reduceMaxVcores =
+        tempReduceMaxVcores < MRJobConfig.DEFAULT_REDUCE_CPU_VCORES
+            ? MRJobConfig.DEFAULT_REDUCE_CPU_VCORES : tempReduceMaxVcores;
+
+    if (numMapTasks > 0) {
+      conf.setLong(MRJobConfig.MAP_MEMORY_MB, this.mapMaxMemory);
+      conf.set(MRJobConfig.MAP_JAVA_OPTS,
+          "-Xmx" + (this.mapMaxMemory - 100) + "m");
+    }
+
+    if (numRedTasks > 0) {
+      conf.setLong(MRJobConfig.REDUCE_MEMORY_MB, this.reduceMaxMemory);
+      conf.set(MRJobConfig.REDUCE_JAVA_OPTS,
+          "-Xmx" + (this.reduceMaxMemory - 100) + "m");
+    }
+
+    boolean hasDeadline =
+        (rand.nextDouble() <= jobClass.jobClass.chance_of_reservation);
+
+    LogNormalDistribution deadlineFactor =
+        SynthUtils.getLogNormalDist(rand, jobClass.jobClass.deadline_factor_avg,
+            jobClass.jobClass.deadline_factor_stddev);
+
+    double deadlineFactorSample =
+        (deadlineFactor != null) ? deadlineFactor.sample() : -1;
+
+    this.queueName = jobClass.workload.getQueueName();
+
+    this.submitTime = MILLISECONDS.convert(actualSubmissionTime, SECONDS);
+
+    this.deadline =
+        hasDeadline ? MILLISECONDS.convert(actualSubmissionTime, SECONDS)
+            + (long) Math.ceil(deadlineFactorSample * duration) : -1;
+
+    conf.set(QUEUE_NAME, queueName);
+
+    // name and initialize job randomness
+    final long seed = rand.nextLong();
+    rand.setSeed(seed);
+    id = sequence.getAndIncrement();
+
+    name = String.format(jobClass.getClassName() + "_%06d", id);
+    LOG.debug(name + " (" + seed + ")");
+
+    LOG.info("JOB TIMING`: job: " + name + " submission:" + submitTime
+        + " deadline:" + deadline + " duration:" + duration
+        + " deadline-submission: " + (deadline - submitTime));
+
+    // generate map and reduce runtimes
+    mapRuntime = new long[numMapTasks];
+    for (int i = 0; i < numMapTasks; i++) {
+      mapRuntime[i] = jobClass.getMapTimeSample();
+      totMapRuntime += mapRuntime[i];
+    }
+    reduceRuntime = new float[numRedTasks];
+    for (int i = 0; i < numRedTasks; i++) {
+      reduceRuntime[i] = jobClass.getReduceTimeSample();
+      totRedRuntime += (long) Math.ceil(reduceRuntime[i]);
+    }
+  }
+
+  public boolean hasDeadline() {
+    return deadline > 0;
+  }
+
+  @Override
+  public String getName() {
+    return name;
+  }
+
+  @Override
+  public String getUser() {
+    return jobClass.getUserName();
+  }
+
+  @Override
+  public JobID getJobID() {
+    return new JobID("job_mock_" + name, id);
+  }
+
+  @Override
+  public Values getOutcome() {
+    return Values.SUCCESS;
+  }
+
+  @Override
+  public long getSubmissionTime() {
+    return submitTime;
+  }
+
+  @Override
+  public int getNumberMaps() {
+    return numMapTasks;
+  }
+
+  @Override
+  public int getNumberReduces() {
+    return numRedTasks;
+  }
+
+  @Override
+  public TaskInfo getTaskInfo(TaskType taskType, int taskNumber) {
+    switch (taskType) {
+    case MAP:
+      return new TaskInfo(-1, -1, -1, -1, mapMaxMemory, mapMaxVcores);
+    case REDUCE:
+      return new TaskInfo(-1, -1, -1, -1, reduceMaxMemory, reduceMaxVcores);
+    default:
+      throw new IllegalArgumentException("Not interested");
+    }
+  }
+
+  @Override
+  public InputSplit[] getInputSplits() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public TaskAttemptInfo getTaskAttemptInfo(TaskType taskType, int taskNumber,
+      int taskAttemptNumber) {
+    switch (taskType) {
+    case MAP:
+      return new MapTaskAttemptInfo(State.SUCCEEDED,
+          getTaskInfo(taskType, taskNumber), mapRuntime[taskNumber], null);
+
+    case REDUCE:
+      // We assume uniform split between pull/sort/reduce
+      // aligned with naive progress reporting assumptions
+      return new ReduceTaskAttemptInfo(State.SUCCEEDED,
+          getTaskInfo(taskType, taskNumber),
+          (long) Math.round((reduceRuntime[taskNumber] / 3)),
+          (long) Math.round((reduceRuntime[taskNumber] / 3)),
+          (long) Math.round((reduceRuntime[taskNumber] / 3)), null);
+
+    default:
+      break;
+    }
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public TaskAttemptInfo getMapTaskAttemptInfoAdjusted(int taskNumber,
+      int taskAttemptNumber, int locality) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public org.apache.hadoop.mapred.JobConf getJobConf() {
+    return new JobConf(conf);
+  }
+
+  @Override
+  public String getQueueName() {
+    return queueName;
+  }
+
+  @Override
+  public String toString() {
+    return "SynthJob [\n" + "  workload=" + jobClass.getWorkload().getId()
+        + "\n" + "  jobClass="
+        + jobClass.getWorkload().getClassList().indexOf(jobClass) + "\n"
+        + "  conf=" + conf + ",\n" + "  id=" + id + ",\n" + "  name=" + name
+        + ",\n" + "  mapRuntime=" + Arrays.toString(mapRuntime) + ",\n"
+        + "  reduceRuntime=" + Arrays.toString(reduceRuntime) + ",\n"
+        + "  submitTime=" + submitTime + ",\n" + "  numMapTasks=" + numMapTasks
+        + ",\n" + "  numRedTasks=" + numRedTasks + ",\n" + "  mapMaxMemory="
+        + mapMaxMemory + ",\n" + "  reduceMaxMemory=" + reduceMaxMemory + ",\n"
+        + "  queueName=" + queueName + "\n" + "]";
+  }
+
+  public SynthJobClass getJobClass() {
+    return jobClass;
+  }
+
+  public long getTotalSlotTime() {
+    return totMapRuntime + totRedRuntime;
+  }
+
+  public long getDuration() {
+    return duration;
+  }
+
+  public long getDeadline() {
+    return deadline;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (!(other instanceof SynthJob)) {
+      return false;
+    }
+    SynthJob o = (SynthJob) other;
+    return Arrays.equals(mapRuntime, o.mapRuntime)
+        && Arrays.equals(reduceRuntime, o.reduceRuntime)
+        && submitTime == o.submitTime && numMapTasks == o.numMapTasks
+        && numRedTasks == o.numRedTasks && mapMaxMemory == o.mapMaxMemory
+        && reduceMaxMemory == o.reduceMaxMemory
+        && mapMaxVcores == o.mapMaxVcores
+        && reduceMaxVcores == o.reduceMaxVcores && queueName.equals(o.queueName)
+        && jobClass.equals(o.jobClass) && totMapRuntime == o.totMapRuntime
+        && totRedRuntime == o.totRedRuntime;
+  }
+
+  @Override
+  public int hashCode() {
+    // could have a bad distr; investigate if a relevant use case exists
+    return jobClass.hashCode() * (int) submitTime;
+  }
+}

+ 180 - 0
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthJobClass.java

@@ -0,0 +1,180 @@
+/**
+ * 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.sls.synthetic;
+
+import org.apache.commons.math3.distribution.AbstractRealDistribution;
+import org.apache.commons.math3.distribution.LogNormalDistribution;
+import org.apache.commons.math3.random.JDKRandomGenerator;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.tools.rumen.JobStory;
+import org.apache.hadoop.yarn.sls.synthetic.SynthTraceJobProducer.JobClass;
+import org.apache.hadoop.yarn.sls.synthetic.SynthTraceJobProducer.Trace;
+
+/**
+ * This is a class that represent a class of Jobs. It is used to generate an
+ * individual job, by picking random durations, task counts, container size,
+ * etc.
+ */
+public class SynthJobClass {
+
+  private final JDKRandomGenerator rand;
+  private final LogNormalDistribution dur;
+  private final LogNormalDistribution mapRuntime;
+  private final LogNormalDistribution redRuntime;
+  private final LogNormalDistribution mtasks;
+  private final LogNormalDistribution rtasks;
+  private final LogNormalDistribution mapMem;
+  private final LogNormalDistribution redMem;
+  private final LogNormalDistribution mapVcores;
+  private final LogNormalDistribution redVcores;
+
+  private final Trace trace;
+  @SuppressWarnings("VisibilityModifier")
+  protected final SynthWorkload workload;
+  @SuppressWarnings("VisibilityModifier")
+  protected final JobClass jobClass;
+
+  public SynthJobClass(JDKRandomGenerator rand, Trace trace,
+      SynthWorkload workload, int classId) {
+
+    this.trace = trace;
+    this.workload = workload;
+    this.rand = new JDKRandomGenerator();
+    this.rand.setSeed(rand.nextLong());
+    jobClass = trace.workloads.get(workload.getId()).job_classes.get(classId);
+
+    this.dur = SynthUtils.getLogNormalDist(rand, jobClass.dur_avg,
+        jobClass.dur_stddev);
+    this.mapRuntime = SynthUtils.getLogNormalDist(rand, jobClass.mtime_avg,
+        jobClass.mtime_stddev);
+    this.redRuntime = SynthUtils.getLogNormalDist(rand, jobClass.rtime_avg,
+        jobClass.rtime_stddev);
+    this.mtasks = SynthUtils.getLogNormalDist(rand, jobClass.mtasks_avg,
+        jobClass.mtasks_stddev);
+    this.rtasks = SynthUtils.getLogNormalDist(rand, jobClass.rtasks_avg,
+        jobClass.rtasks_stddev);
+
+    this.mapMem = SynthUtils.getLogNormalDist(rand, jobClass.map_max_memory_avg,
+        jobClass.map_max_memory_stddev);
+    this.redMem = SynthUtils.getLogNormalDist(rand,
+        jobClass.reduce_max_memory_avg, jobClass.reduce_max_memory_stddev);
+    this.mapVcores = SynthUtils.getLogNormalDist(rand,
+        jobClass.map_max_vcores_avg, jobClass.map_max_vcores_stddev);
+    this.redVcores = SynthUtils.getLogNormalDist(rand,
+        jobClass.reduce_max_vcores_avg, jobClass.reduce_max_vcores_stddev);
+  }
+
+  public JobStory getJobStory(Configuration conf, long actualSubmissionTime) {
+    return new SynthJob(rand, conf, this, actualSubmissionTime);
+  }
+
+  @Override
+  public String toString() {
+    return "SynthJobClass [workload=" + workload.getName() + ", class="
+        + jobClass.class_name + " job_count=" + jobClass.class_weight + ", dur="
+        + ((dur != null) ? dur.getNumericalMean() : 0) + ", mapRuntime="
+        + ((mapRuntime != null) ? mapRuntime.getNumericalMean() : 0)
+        + ", redRuntime="
+        + ((redRuntime != null) ? redRuntime.getNumericalMean() : 0)
+        + ", mtasks=" + ((mtasks != null) ? mtasks.getNumericalMean() : 0)
+        + ", rtasks=" + ((rtasks != null) ? rtasks.getNumericalMean() : 0)
+        + ", chance_of_reservation=" + jobClass.chance_of_reservation + "]\n";
+
+  }
+
+  public double getClassWeight() {
+    return jobClass.class_weight;
+  }
+
+  public long getDur() {
+    return genLongSample(dur);
+  }
+
+  public int getMtasks() {
+    return genIntSample(mtasks);
+  }
+
+  public int getRtasks() {
+    return genIntSample(rtasks);
+  }
+
+  public long getMapMaxMemory() {
+    return genLongSample(mapMem);
+  }
+
+  public long getReduceMaxMemory() {
+    return genLongSample(redMem);
+  }
+
+  public long getMapMaxVcores() {
+    return genLongSample(mapVcores);
+  }
+
+  public long getReduceMaxVcores() {
+    return genLongSample(redVcores);
+  }
+
+  public SynthWorkload getWorkload() {
+    return workload;
+  }
+
+  public int genIntSample(AbstractRealDistribution dist) {
+    if (dist == null) {
+      return 0;
+    }
+    double baseSample = dist.sample();
+    if (baseSample < 0) {
+      baseSample = 0;
+    }
+    return (int) (Integer.MAX_VALUE & (long) Math.ceil(baseSample));
+  }
+
+  public long genLongSample(AbstractRealDistribution dist) {
+    return dist != null ? (long) Math.ceil(dist.sample()) : 0;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (!(other instanceof SynthJobClass)) {
+      return false;
+    }
+    SynthJobClass o = (SynthJobClass) other;
+    return workload.equals(o.workload);
+  }
+
+  @Override
+  public int hashCode() {
+    return workload.hashCode() * workload.getId();
+  }
+
+  public String getClassName() {
+    return jobClass.class_name;
+  }
+
+  public long getMapTimeSample() {
+    return genLongSample(mapRuntime);
+  }
+
+  public long getReduceTimeSample() {
+    return genLongSample(redRuntime);
+  }
+
+  public String getUserName() {
+    return jobClass.user_name;
+  }
+}

+ 319 - 0
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthTraceJobProducer.java

@@ -0,0 +1,319 @@
+/**
+ * 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.yarn.sls.synthetic;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.math3.random.JDKRandomGenerator;
+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.tools.rumen.JobStory;
+import org.apache.hadoop.tools.rumen.JobStoryProducer;
+import org.codehaus.jackson.annotate.JsonProperty;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import javax.xml.bind.annotation.XmlRootElement;
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.codehaus.jackson.JsonParser.Feature.INTERN_FIELD_NAMES;
+import static org.codehaus.jackson.map.DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES;
+
+/**
+ * This is a JobStoryProducer that operates from distribution of different
+ * workloads. The .json input file is used to determine how many jobs, which
+ * size, number of maps/reducers and their duration, as well as the temporal
+ * distributed of submissions. For each parameter we control avg and stdev, and
+ * generate values via normal or log-normal distributions.
+ */
+public class SynthTraceJobProducer implements JobStoryProducer {
+
+  @SuppressWarnings("StaticVariableName")
+  private static final Log LOG = LogFactory.getLog(SynthTraceJobProducer.class);
+
+  private final Configuration conf;
+  private final AtomicInteger numJobs;
+  private final Trace trace;
+  private final long seed;
+
+  private int totalWeight;
+  private final List<Double> weightList;
+  private final Map<Integer, SynthWorkload> workloads;
+
+  private final Queue<StoryParams> listStoryParams;
+
+  private final JDKRandomGenerator rand;
+
+  public static final String SLS_SYNTHETIC_TRACE_FILE =
+      "sls.synthetic" + ".trace_file";
+
+  public SynthTraceJobProducer(Configuration conf) throws IOException {
+    this(conf, new Path(conf.get(SLS_SYNTHETIC_TRACE_FILE)));
+  }
+
+  public SynthTraceJobProducer(Configuration conf, Path path)
+      throws IOException {
+
+    LOG.info("SynthTraceJobProducer");
+
+    this.conf = conf;
+    this.rand = new JDKRandomGenerator();
+    workloads = new HashMap<Integer, SynthWorkload>();
+    weightList = new ArrayList<Double>();
+
+    ObjectMapper mapper = new ObjectMapper();
+    mapper.configure(INTERN_FIELD_NAMES, true);
+    mapper.configure(FAIL_ON_UNKNOWN_PROPERTIES, false);
+
+    FileSystem ifs = path.getFileSystem(conf);
+    FSDataInputStream fileIn = ifs.open(path);
+
+    this.trace = mapper.readValue(fileIn, Trace.class);
+    seed = trace.rand_seed;
+    rand.setSeed(seed);
+
+    this.numJobs = new AtomicInteger(trace.num_jobs);
+
+    for (int workloadId = 0; workloadId < trace.workloads
+        .size(); workloadId++) {
+      SynthWorkload workload = new SynthWorkload(workloadId, trace);
+      for (int classId =
+          0; classId < trace.workloads.get(workloadId).job_classes
+              .size(); classId++) {
+        SynthJobClass cls = new SynthJobClass(rand, trace, workload, classId);
+        workload.add(cls);
+      }
+      workloads.put(workloadId, workload);
+    }
+
+    for (int i = 0; i < workloads.size(); i++) {
+      double w = workloads.get(i).getWorkloadWeight();
+      totalWeight += w;
+      weightList.add(w);
+    }
+
+    // create priority queue to keep start-time sorted
+    listStoryParams =
+        new PriorityQueue<>(10, new Comparator<StoryParams>() {
+          @Override
+          public int compare(StoryParams o1, StoryParams o2) {
+            long value = o2.actualSubmissionTime - o1.actualSubmissionTime;
+            if ((int)value != value) {
+              throw new ArithmeticException("integer overflow");
+            }
+            return (int)value;
+          }
+        });
+
+    // initialize it
+    createStoryParams();
+    LOG.info("Generated " + listStoryParams.size() + " deadlines for "
+        + this.numJobs.get() + " jobs ");
+  }
+
+  public long getSeed() {
+    return seed;
+  }
+
+  public int getNodesPerRack() {
+    return trace.nodes_per_rack < 1 ? 1: trace.nodes_per_rack;
+  }
+
+  public int getNumNodes() {
+    return trace.num_nodes;
+  }
+
+  /**
+   * Class used to parse a trace configuration file.
+   */
+  @SuppressWarnings({ "membername", "checkstyle:visibilitymodifier" })
+  @XmlRootElement
+  public static class Trace {
+    @JsonProperty("description")
+    String description;
+    @JsonProperty("num_nodes")
+    int num_nodes;
+    @JsonProperty("nodes_per_rack")
+    int nodes_per_rack;
+    @JsonProperty("num_jobs")
+    int num_jobs;
+
+    // in sec (selects a portion of time_distribution
+    @JsonProperty("rand_seed")
+    long rand_seed;
+    @JsonProperty("workloads")
+    List<Workload> workloads;
+
+  }
+
+  /**
+   * Class used to parse a workload from file.
+   */
+  @SuppressWarnings({ "membername", "checkstyle:visibilitymodifier" })
+  public static class Workload {
+    @JsonProperty("workload_name")
+    String workload_name;
+    // used to change probability this workload is picked for each job
+    @JsonProperty("workload_weight")
+    double workload_weight;
+    @JsonProperty("queue_name")
+    String queue_name;
+    @JsonProperty("job_classes")
+    List<JobClass> job_classes;
+    @JsonProperty("time_distribution")
+    List<TimeSample> time_distribution;
+  }
+
+  /**
+   * Class used to parse a job class from file.
+   */
+  @SuppressWarnings({ "membername", "checkstyle:visibilitymodifier" })
+  public static class JobClass {
+
+    @JsonProperty("class_name")
+    String class_name;
+    @JsonProperty("user_name")
+    String user_name;
+
+    // used to change probability this class is chosen
+    @JsonProperty("class_weight")
+    double class_weight;
+
+    // reservation related params
+    @JsonProperty("chance_of_reservation")
+    double chance_of_reservation;
+    @JsonProperty("deadline_factor_avg")
+    double deadline_factor_avg;
+    @JsonProperty("deadline_factor_stddev")
+    double deadline_factor_stddev;
+
+    // durations in sec
+    @JsonProperty("dur_avg")
+    double dur_avg;
+    @JsonProperty("dur_stddev")
+    double dur_stddev;
+    @JsonProperty("mtime_avg")
+    double mtime_avg;
+    @JsonProperty("mtime_stddev")
+    double mtime_stddev;
+    @JsonProperty("rtime_avg")
+    double rtime_avg;
+    @JsonProperty("rtime_stddev")
+    double rtime_stddev;
+
+    // number of tasks
+    @JsonProperty("mtasks_avg")
+    double mtasks_avg;
+    @JsonProperty("mtasks_stddev")
+    double mtasks_stddev;
+    @JsonProperty("rtasks_avg")
+    double rtasks_avg;
+    @JsonProperty("rtasks_stddev")
+    double rtasks_stddev;
+
+    // memory in MB
+    @JsonProperty("map_max_memory_avg")
+    long map_max_memory_avg;
+    @JsonProperty("map_max_memory_stddev")
+    double map_max_memory_stddev;
+    @JsonProperty("reduce_max_memory_avg")
+    long reduce_max_memory_avg;
+    @JsonProperty("reduce_max_memory_stddev")
+    double reduce_max_memory_stddev;
+
+    // vcores
+    @JsonProperty("map_max_vcores_avg")
+    long map_max_vcores_avg;
+    @JsonProperty("map_max_vcores_stddev")
+    double map_max_vcores_stddev;
+    @JsonProperty("reduce_max_vcores_avg")
+    long reduce_max_vcores_avg;
+    @JsonProperty("reduce_max_vcores_stddev")
+    double reduce_max_vcores_stddev;
+
+  }
+
+  /**
+   * This is used to define time-varying probability of a job start-time (e.g.,
+   * to simulate daily patterns).
+   */
+  @SuppressWarnings({ "membername", "checkstyle:visibilitymodifier" })
+  public static class TimeSample {
+    // in sec
+    @JsonProperty("time")
+    int time;
+    @JsonProperty("weight")
+    double jobs;
+  }
+
+  static class StoryParams {
+    private SynthJobClass pickedJobClass;
+    private long actualSubmissionTime;
+
+    StoryParams(SynthJobClass pickedJobClass, long actualSubmissionTime) {
+      this.pickedJobClass = pickedJobClass;
+      this.actualSubmissionTime = actualSubmissionTime;
+    }
+  }
+
+
+  void createStoryParams() {
+
+    for (int i = 0; i < numJobs.get(); i++) {
+      int workload = SynthUtils.getWeighted(weightList, rand);
+      SynthWorkload pickedWorkload = workloads.get(workload);
+      long jobClass =
+          SynthUtils.getWeighted(pickedWorkload.getWeightList(), rand);
+      SynthJobClass pickedJobClass =
+          pickedWorkload.getClassList().get((int) jobClass);
+      long actualSubmissionTime = pickedWorkload.getBaseSubmissionTime(rand);
+      // long actualSubmissionTime = (i + 1) * 10;
+      listStoryParams
+          .add(new StoryParams(pickedJobClass, actualSubmissionTime));
+    }
+  }
+
+  @Override
+  public JobStory getNextJob() throws IOException {
+    if (numJobs.decrementAndGet() < 0) {
+      return null;
+    }
+    StoryParams storyParams = listStoryParams.poll();
+    return storyParams.pickedJobClass.getJobStory(conf,
+        storyParams.actualSubmissionTime);
+  }
+
+  @Override
+  public void close() {
+  }
+
+  @Override
+  public String toString() {
+    return "SynthTraceJobProducer [ conf=" + conf + ", numJobs=" + numJobs
+        + ", weightList=" + weightList + ", r=" + rand + ", totalWeight="
+        + totalWeight + ", workloads=" + workloads + "]";
+  }
+
+  public int getNumJobs() {
+    return trace.num_jobs;
+  }
+
+}

+ 101 - 0
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthUtils.java

@@ -0,0 +1,101 @@
+/**
+ * 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.sls.synthetic;
+
+import org.apache.commons.math3.distribution.LogNormalDistribution;
+import org.apache.commons.math3.distribution.NormalDistribution;
+import org.apache.commons.math3.random.JDKRandomGenerator;
+
+import java.util.Collection;
+import java.util.Random;
+
+/**
+ * Utils for the Synthetic generator.
+ */
+public final class SynthUtils {
+
+  private SynthUtils(){
+    //class is not meant to be instantiated
+  }
+
+  public static int getWeighted(Collection<Double> weights, Random rr) {
+
+    double totalWeight = 0;
+    for (Double i : weights) {
+      totalWeight += i;
+    }
+
+    double rand = rr.nextDouble() * totalWeight;
+
+    double cur = 0;
+    int ind = 0;
+    for (Double i : weights) {
+      cur += i;
+      if (cur > rand) {
+        break;
+      }
+      ind++;
+    }
+
+    return ind;
+  }
+
+  public static NormalDistribution getNormalDist(JDKRandomGenerator rand,
+      double average, double stdDev) {
+
+    if (average <= 0) {
+      return null;
+    }
+
+    // set default for missing param
+    if (stdDev == 0) {
+      stdDev = average / 6;
+    }
+
+    NormalDistribution ret = new NormalDistribution(average, stdDev,
+        NormalDistribution.DEFAULT_INVERSE_ABSOLUTE_ACCURACY);
+    ret.reseedRandomGenerator(rand.nextLong());
+    return ret;
+  }
+
+  public static LogNormalDistribution getLogNormalDist(JDKRandomGenerator rand,
+      double mean, double stdDev) {
+
+    if (mean <= 0) {
+      return null;
+    }
+
+    // set default for missing param
+    if (stdDev == 0) {
+      stdDev = mean / 6;
+    }
+
+    // derive lognormal parameters for X = LogNormal(mu, sigma)
+    // sigma^2 = ln (1+Var[X]/(E[X])^2)
+    // mu = ln(E[X]) - 1/2 * sigma^2
+    double var = stdDev * stdDev;
+    double sigmasq = Math.log1p(var / (mean * mean));
+    double sigma = Math.sqrt(sigmasq);
+    double mu = Math.log(mean) - 0.5 * sigmasq;
+
+    LogNormalDistribution ret = new LogNormalDistribution(mu, sigma,
+        LogNormalDistribution.DEFAULT_INVERSE_ABSOLUTE_ACCURACY);
+    ret.reseedRandomGenerator(rand.nextLong());
+    return ret;
+  }
+}

+ 121 - 0
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthWorkload.java

@@ -0,0 +1,121 @@
+/**
+ * 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.sls.synthetic;
+
+import org.apache.hadoop.yarn.sls.synthetic.SynthTraceJobProducer.Trace;
+
+import java.util.*;
+
+/**
+ * This class represent a workload (made up of multiple SynthJobClass(es)). It
+ * also stores the temporal distributions of jobs in this workload.
+ */
+public class SynthWorkload {
+
+  private final int id;
+  private final List<SynthJobClass> classList;
+  private final Trace trace;
+  private final SortedMap<Integer, Double> timeWeights;
+
+  public SynthWorkload(int identifier, Trace inTrace) {
+    classList = new ArrayList<SynthJobClass>();
+    this.id = identifier;
+    this.trace = inTrace;
+    timeWeights = new TreeMap<Integer, Double>();
+    for (SynthTraceJobProducer.TimeSample ts : trace.workloads
+        .get(id).time_distribution) {
+      timeWeights.put(ts.time, ts.jobs);
+    }
+  }
+
+  public boolean add(SynthJobClass s) {
+    return classList.add(s);
+  }
+
+  public List<Double> getWeightList() {
+    ArrayList<Double> ret = new ArrayList<Double>();
+    for (SynthJobClass s : classList) {
+      ret.add(s.getClassWeight());
+    }
+    return ret;
+  }
+
+  public int getId() {
+    return id;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (!(other instanceof SynthWorkload)) {
+      return false;
+    }
+    // assume ID determines job classes by construction
+    return getId() == ((SynthWorkload) other).getId();
+  }
+
+  @Override
+  public int hashCode() {
+    return getId();
+  }
+
+  @Override
+  public String toString() {
+    return "SynthWorkload " + trace.workloads.get(id).workload_name + "[\n"
+        + classList + "]\n";
+  }
+
+  public String getName() {
+    return trace.workloads.get(id).workload_name;
+  }
+
+  public double getWorkloadWeight() {
+    return trace.workloads.get(id).workload_weight;
+  }
+
+  public String getQueueName() {
+    return trace.workloads.get(id).queue_name;
+  }
+
+  public long getBaseSubmissionTime(Random rand) {
+
+    // pick based on weights the "bucket" for this start time
+    int position = SynthUtils.getWeighted(timeWeights.values(), rand);
+
+    int[] time = new int[timeWeights.keySet().size()];
+    int index = 0;
+    for (Integer i : timeWeights.keySet()) {
+      time[index++] = i;
+    }
+
+    // uniformly pick a time between start and end time of this bucket
+    int startRange = time[position];
+    int endRange = startRange;
+    // if there is no subsequent bucket pick startRange
+    if (position < timeWeights.keySet().size() - 1) {
+      endRange = time[position + 1];
+      return startRange + rand.nextInt((endRange - startRange));
+    } else {
+      return startRange;
+    }
+  }
+
+  public List<SynthJobClass> getClassList() {
+    return classList;
+  }
+
+}

+ 22 - 0
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/package-info.java

@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Classes comprising the synthetic load generator for SLS.
+ */
+package org.apache.hadoop.yarn.sls.synthetic;

+ 47 - 14
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/utils/SLSUtils.java

@@ -28,6 +28,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -37,12 +39,11 @@ import org.apache.hadoop.tools.rumen.JobTraceReader;
 import org.apache.hadoop.tools.rumen.LoggedJob;
 import org.apache.hadoop.tools.rumen.LoggedTask;
 import org.apache.hadoop.tools.rumen.LoggedTaskAttempt;
-import org.codehaus.jackson.JsonFactory;
-import org.codehaus.jackson.map.ObjectMapper;
 
 @Private
 @Unstable
 public class SLSUtils {
+  public final static String DEFAULT_JOB_TYPE = "mapreduce";
 
   // hostname includes the network path and the host name. for example
   // "/default-rack/hostFoo" or "/coreSwitchA/TORSwitchB/hostBar".
@@ -100,22 +101,15 @@ public class SLSUtils {
    */
   public static Set<String> parseNodesFromSLSTrace(String jobTrace)
           throws IOException {
-    Set<String> nodeSet = new HashSet<String>();
+    Set<String> nodeSet = new HashSet<>();
     JsonFactory jsonF = new JsonFactory();
     ObjectMapper mapper = new ObjectMapper();
     Reader input =
         new InputStreamReader(new FileInputStream(jobTrace), "UTF-8");
     try {
-      Iterator<Map> i = mapper.readValues(
-              jsonF.createJsonParser(input), Map.class);
+      Iterator<Map> i = mapper.readValues(jsonF.createParser(input), Map.class);
       while (i.hasNext()) {
-        Map jsonE = i.next();
-        List tasks = (List) jsonE.get("job.tasks");
-        for (Object o : tasks) {
-          Map jsonTask = (Map) o;
-          String hostname = jsonTask.get("container.host").toString();
-          nodeSet.add(hostname);
-        }
+        addNodes(nodeSet, i.next());
       }
     } finally {
       input.close();
@@ -123,6 +117,29 @@ public class SLSUtils {
     return nodeSet;
   }
 
+  private static void addNodes(Set<String> nodeSet, Map jsonEntry) {
+    if (jsonEntry.containsKey("num.nodes")) {
+      int numNodes = Integer.parseInt(jsonEntry.get("num.nodes").toString());
+      int numRacks = 1;
+      if (jsonEntry.containsKey("num.racks")) {
+        numRacks = Integer.parseInt(
+            jsonEntry.get("num.racks").toString());
+      }
+      nodeSet.addAll(generateNodes(numNodes, numRacks));
+    }
+
+    if (jsonEntry.containsKey("job.tasks")) {
+      List tasks = (List) jsonEntry.get("job.tasks");
+      for (Object o : tasks) {
+        Map jsonTask = (Map) o;
+        String hostname = (String) jsonTask.get("container.host");
+        if (hostname != null) {
+          nodeSet.add(hostname);
+        }
+      }
+    }
+  }
+
   /**
    * parse the input node file, return each host name
    */
@@ -134,8 +151,7 @@ public class SLSUtils {
     Reader input =
         new InputStreamReader(new FileInputStream(nodeFile), "UTF-8");
     try {
-      Iterator<Map> i = mapper.readValues(
-              jsonF.createJsonParser(input), Map.class);
+      Iterator<Map> i = mapper.readValues(jsonF.createParser(input), Map.class);
       while (i.hasNext()) {
         Map jsonE = i.next();
         String rack = "/" + jsonE.get("rack");
@@ -150,4 +166,21 @@ public class SLSUtils {
     }
     return nodeSet;
   }
+
+  public static Set<? extends String> generateNodes(int numNodes,
+      int numRacks){
+    Set<String> nodeSet = new HashSet<>();
+    if (numRacks < 1) {
+      numRacks = 1;
+    }
+
+    if (numRacks > numNodes) {
+      numRacks = numNodes;
+    }
+
+    for (int i = 0; i < numNodes; i++) {
+      nodeSet.add("/rack" + i % numRacks + "/node" + i);
+    }
+    return nodeSet;
+  }
 }

+ 22 - 23
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/web/SLSWebApp.java

@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.yarn.sls.web;
 
-import java.io.File;
 import java.io.IOException;
 import java.io.ObjectInputStream;
 import java.text.MessageFormat;
@@ -26,11 +25,12 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 
+import javax.servlet.ServletException;
 import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
-import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
@@ -38,12 +38,12 @@ import org.apache.hadoop.yarn.sls.SLSRunner;
 import org.apache.hadoop.yarn.sls.scheduler.FairSchedulerMetrics;
 import org.apache.hadoop.yarn.sls.scheduler.SchedulerMetrics;
 import org.apache.hadoop.yarn.sls.scheduler.SchedulerWrapper;
+
 import org.mortbay.jetty.Handler;
 import org.mortbay.jetty.Request;
 import org.mortbay.jetty.Server;
 import org.mortbay.jetty.handler.AbstractHandler;
 import org.mortbay.jetty.handler.ResourceHandler;
-
 import com.codahale.metrics.Counter;
 import com.codahale.metrics.Gauge;
 import com.codahale.metrics.Histogram;
@@ -84,12 +84,12 @@ public class SLSWebApp extends HttpServlet {
     // load templates
     ClassLoader cl = Thread.currentThread().getContextClassLoader();
     try {
-      simulateInfoTemplate = FileUtils.readFileToString(new File(
-              cl.getResource("simulate.info.html.template").getFile()));
-      simulateTemplate = FileUtils.readFileToString(new File(
-              cl.getResource("simulate.html.template").getFile()));
-      trackTemplate = FileUtils.readFileToString(new File(
-              cl.getResource("track.html.template").getFile()));
+      simulateInfoTemplate = IOUtils.toString(
+          cl.getResourceAsStream("html/simulate.info.html.template"));
+      simulateTemplate = IOUtils.toString(
+          cl.getResourceAsStream("html/simulate.html.template"));
+      trackTemplate = IOUtils.toString(
+          cl.getResourceAsStream("html/track.html.template"));
     } catch (IOException e) {
       e.printStackTrace();
     }
@@ -105,24 +105,23 @@ public class SLSWebApp extends HttpServlet {
 
   public SLSWebApp(SchedulerWrapper wrapper, int metricsAddressPort) {
     this.wrapper = wrapper;
-    metrics = wrapper.getMetrics();
-    handleOperTimecostHistogramMap =
-            new HashMap<SchedulerEventType, Histogram>();
-    queueAllocatedMemoryCounterMap = new HashMap<String, Counter>();
-    queueAllocatedVCoresCounterMap = new HashMap<String, Counter>();
+    handleOperTimecostHistogramMap = new HashMap<>();
+    queueAllocatedMemoryCounterMap = new HashMap<>();
+    queueAllocatedVCoresCounterMap = new HashMap<>();
     schedulerMetrics = wrapper.getSchedulerMetrics();
+    metrics = schedulerMetrics.getMetrics();
     port = metricsAddressPort;
   }
 
   public void start() throws Exception {
-    // static files
     final ResourceHandler staticHandler = new ResourceHandler();
     staticHandler.setResourceBase("html");
 
     Handler handler = new AbstractHandler() {
       @Override
       public void handle(String target, HttpServletRequest request,
-                         HttpServletResponse response, int dispatch) {
+          HttpServletResponse response, int dispatch)
+          throws IOException, ServletException {
         try{
           // timeunit
           int timeunit = 1000;   // second, divide millionsecond / 1000
@@ -183,14 +182,14 @@ public class SLSWebApp extends HttpServlet {
     response.setStatus(HttpServletResponse.SC_OK);
 
     String simulateInfo;
-    if (SLSRunner.simulateInfoMap.isEmpty()) {
+    if (SLSRunner.getSimulateInfoMap().isEmpty()) {
       String empty = "<tr><td colspan='2' align='center'>" +
               "No information available</td></tr>";
       simulateInfo = MessageFormat.format(simulateInfoTemplate, empty);
     } else {
       StringBuilder info = new StringBuilder();
       for (Map.Entry<String, Object> entry :
-              SLSRunner.simulateInfoMap.entrySet()) {
+              SLSRunner.getSimulateInfoMap().entrySet()) {
         info.append("<tr>");
         info.append("<td class='td1'>").append(entry.getKey()).append("</td>");
         info.append("<td class='td2'>").append(entry.getValue())
@@ -221,7 +220,7 @@ public class SLSWebApp extends HttpServlet {
     response.setStatus(HttpServletResponse.SC_OK);
 
     // queues {0}
-    Set<String> queues = wrapper.getQueueSet();
+    Set<String> queues = wrapper.getTracker().getQueueSet();
     StringBuilder queueInfo = new StringBuilder();
 
     int i = 0;
@@ -260,7 +259,7 @@ public class SLSWebApp extends HttpServlet {
 
     // tracked queues {0}
     StringBuilder trackedQueueInfo = new StringBuilder();
-    Set<String> trackedQueues = wrapper.getQueueSet();
+    Set<String> trackedQueues = wrapper.getTracker().getQueueSet();
     for(String queue : trackedQueues) {
       trackedQueueInfo.append("<option value='Queue ").append(queue)
               .append("'>").append(queue).append("</option>");
@@ -268,7 +267,7 @@ public class SLSWebApp extends HttpServlet {
 
     // tracked apps {1}
     StringBuilder trackedAppInfo = new StringBuilder();
-    Set<String> trackedApps = wrapper.getTrackedAppSet();
+    Set<String> trackedApps = wrapper.getTracker().getTrackedAppSet();
     for(String job : trackedApps) {
       trackedAppInfo.append("<option value='Job ").append(job)
               .append("'>").append(job).append("</option>");
@@ -417,7 +416,7 @@ public class SLSWebApp extends HttpServlet {
     // allocated resource for each queue
     Map<String, Double> queueAllocatedMemoryMap = new HashMap<String, Double>();
     Map<String, Long> queueAllocatedVCoresMap = new HashMap<String, Long>();
-    for (String queue : wrapper.getQueueSet()) {
+    for (String queue : wrapper.getTracker().getQueueSet()) {
       // memory
       String key = "counter.queue." + queue + ".allocated.memory";
       if (! queueAllocatedMemoryCounterMap.containsKey(queue) &&
@@ -457,7 +456,7 @@ public class SLSWebApp extends HttpServlet {
             .append(",\"cluster.available.memory\":").append(availableMemoryGB)
             .append(",\"cluster.available.vcores\":").append(availableVCoresGB);
 
-    for (String queue : wrapper.getQueueSet()) {
+    for (String queue : wrapper.getTracker().getQueueSet()) {
       sb.append(",\"queue.").append(queue).append(".allocated.memory\":")
               .append(queueAllocatedMemoryMap.get(queue));
       sb.append(",\"queue.").append(queue).append(".allocated.vcores\":")

+ 135 - 16
hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md

@@ -27,9 +27,11 @@ Yarn Scheduler Load Simulator (SLS)
     * [Metrics](#Metrics)
         * [Real-time Tracking](#Real-time_Tracking)
         * [Offline Analysis](#Offline_Analysis)
+    * [Synthetic Load Generator](#SynthGen)
     * [Appendix](#Appendix)
         * [Resources](#Resources)
         * [SLS JSON input file format](#SLS_JSON_input_file_format)
+        * [SYNTH JSON input file format](#SYNTH_JSON_input_file_format)
         * [Simulator input topology file format](#Simulator_input_topology_file_format)
 
 Overview
@@ -72,7 +74,7 @@ The following figure illustrates the implementation architecture of the simulato
 
 ![The architecture of the simulator](images/sls_arch.png)
 
-The simulator takes input of workload traces, and fetches the cluster and applications information. For each NM and AM, the simulator builds a simulator to simulate their running. All NM/AM simulators run in a thread pool. The simulator reuses Yarn Resource Manager, and builds a wrapper out of the scheduler. The Scheduler Wrapper can track the scheduler behaviors and generates several logs, which are the outputs of the simulator and can be further analyzed.
+The simulator takes input of workload traces, or synthetic load distributions and generaters the cluster and applications information. For each NM and AM, the simulator builds a simulator to simulate their running. All NM/AM simulators run in a thread pool. The simulator reuses Yarn Resource Manager, and builds a wrapper out of the scheduler. The Scheduler Wrapper can track the scheduler behaviors and generates several logs, which are the outputs of the simulator and can be further analyzed.
 
 ### Usecases
 
@@ -97,7 +99,7 @@ This section will show how to use the simulator. Here let `$HADOOP_ROOT` represe
 
 *   `bin`: contains running scripts for the simulator.
 
-*   `html`: contains several html/css/js files we needed for real-time tracking.
+*   `html`: Users can also reproduce those real-time tracking charts in offline mode. Just upload the `realtimetrack.json` to `$HADOOP_ROOT/share/hadoop/tools/sls/html/showSimulationTrace.html`. For browser security problem, need to put files `realtimetrack.json` and `showSimulationTrace.html` in the same directory.
 
 *   `sample-conf`: specifies the simulator configurations.
 
@@ -179,17 +181,30 @@ The simulator supports two types of input files: the rumen traces and its own in
 
     $ cd $HADOOP_ROOT/share/hadoop/tools/sls
     $ bin/slsrun.sh
-      --input-rumen |--input-sls=<TRACE_FILE1,TRACE_FILE2,...>
-      --output-dir=<SLS_SIMULATION_OUTPUT_DIRECTORY> [--nodes=<SLS_NODES_FILE>]
-        [--track-jobs=<JOBID1,JOBID2,...>] [--print-simulation]
+      Usage: slsrun.sh <OPTIONS>
+                 --tracetype=<SYNTH | SLS | RUMEN>
+                 --tracelocation=<FILE1,FILE2,...>
+                 (deprecated --input-rumen=<FILE1,FILE2,...>  | --input-sls=<FILE1,FILE2,...>)
+                 --output-dir=<SLS_SIMULATION_OUTPUT_DIRECTORY>
+                 [--nodes=<SLS_NODES_FILE>]
+                 [--track-jobs=<JOBID1,JOBID2,...>]
+                 [--print-simulation]
+
 
 *   `--input-rumen`: The input rumen trace files. Users can input multiple
     files, separated by comma. One example trace is provided in
     `$HADOOP_ROOT/share/hadoop/tools/sls/sample-data/2jobs2min-rumen-jh.json`.
+    This is equivalent to `--tracetype=RUMEN --tracelocation=<path_to_trace>`.
 
 *   `--input-sls`: Simulator its own file format. The simulator also
     provides a tool to convert rumen traces to sls traces (`rumen2sls.sh`).
     Refer to appendix for an example of sls input json file.
+    This is equivalent to `--tracetype=SLS --tracelocation=<path_to_trace>`.
+
+*   `--tracetype`: This is the new way to configure the trace generation and
+    takes values RUMEN, SLS, or SYNTH, to trigger the three type of load generation
+
+*   `--tracelocation`: Path to the input file, matching the tracetype above.
 
 *   `--output-dir`: The output directory for generated running logs and
     metrics.
@@ -281,30 +296,57 @@ After the simulator finishes, all logs are saved in the output directory specifi
 
 Users can also reproduce those real-time tracking charts in offline mode. Just upload the `realtimetrack.json` to `$HADOOP_ROOT/share/hadoop/tools/sls/html/showSimulationTrace.html`. For browser security problem, need to put files `realtimetrack.json` and `showSimulationTrace.html` in the same directory.
 
+
+Synthetic Load Generator
+------------------------
+The Synthetic Load Generator complements the extensive nature of SLS-native and RUMEN traces, by providing a
+distribution-driven generation of load. The load generator is organized as a JobStoryProducer
+(compatible with rumen, and thus gridmix for later integration). We seed the Random number generator so
+that results randomized but deterministic---hence reproducible.
+We organize the jobs being generated around */workloads/job_class* hierarchy, which allow to easily
+group jobs with similar behaviors and categorize them (e.g., jobs with long running containers, or maponly
+computations, etc..). The user can control average and standard deviations for many of the
+important parameters, such as number of mappers/reducers, duration of mapper/reducers, size
+(mem/cpu) of containers, chance of reservation, etc. We use weighted-random sampling (whenever we
+pick among a small number of options) or LogNormal distributions (to avoid negative values) when we
+pick from wide ranges of values---see appendix on LogNormal distributions.
+
+The SYNTH mode of SLS is very convenient to generate very large loads without the need for extensive input
+files. This allows to easily explore wide range of use cases (e.g., imagine simulating 100k jobs, and in different
+runs simply tune the average number of mappers, or average task duration), in an efficient and compact way.
+
 Appendix
 --------
 
 ### Resources
 
 [YARN-1021](https://issues.apache.org/jira/browse/YARN-1021) is the main JIRA that introduces Yarn Scheduler Load Simulator to Hadoop Yarn project.
+[YARN-6363](https://issues.apache.org/jira/browse/YARN-6363) is the main JIRA that introduces the Synthetic Load Generator to SLS.
 
 ### SLS JSON input file format
 
 Here we provide an example format of the sls json file, which contains 2 jobs. The first job has 3 map tasks and the second one has 2 map tasks.
 
     {
-      "am.type" : "mapreduce",
-      "job.start.ms" : 0,
-      "job.end.ms" : 95375,
-      "job.queue.name" : "sls_queue_1",
-      "job.id" : "job_1",
-      "job.user" : "default",
+      "num.nodes": 3,  // total number of nodes in the cluster
+      "num.racks": 1   // total number of racks in the cluster, it divides num.nodes into the racks evenly, optional, the default value is 1
+    }
+    {
+      "am.type" : "mapreduce", // type of AM, optional, the default value is "mapreduce"
+      "job.start.ms" : 0,      // job start time
+      "job.end.ms" : 95375,    // job finish time, optional, the default value is 0
+      "job.queue.name" : "sls_queue_1", // the queue job will be submitted to
+      "job.id" : "job_1",      // the job id used to track the job, optional. The default value, an zero-based integer increasing with number of jobs, is used if this is not specified or job.count > 1
+      "job.user" : "default",  // user, optional, the default value is "default"
+      "job.count" : 1,         // number of jobs, optional, the default value is 1
       "job.tasks" : [ {
-        "container.host" : "/default-rack/node1",
-        "container.start.ms" : 6664,
-        "container.end.ms" : 23707,
-        "container.priority" : 20,
-        "container.type" : "map"
+        "count": 1,    // number of tasks, optional, the default value is 1
+        "container.host" : "/default-rack/node1",  // host the container asks for
+        "container.start.ms" : 6664,  // container start time, optional
+        "container.end.ms" : 23707,   // container finish time, optional
+        "duration.ms":  50000,        // duration of the container, optional if start and end time is specified
+        "container.priority" : 20,    // priority of the container, optional, the default value is 20
+        "container.type" : "map"      // type of the container, could be "map" or "reduce", optional, the default value is "map"
       }, {
         "container.host" : "/default-rack/node3",
         "container.start.ms" : 6665,
@@ -341,6 +383,77 @@ Here we provide an example format of the sls json file, which contains 2 jobs. T
       } ]
     }
 
+
+### SYNTH JSON input file format
+Here we provide an example format of the synthetic generator json file. We use *(json-non-conforming)* inline comments to explain the use of each parameter.
+
+    {
+      "description" : "tiny jobs workload",    //description of the meaning of this collection of workloads
+      "num_nodes" : 10,  //total nodes in the simulated cluster
+      "nodes_per_rack" : 4, //number of nodes in each simulated rack
+      "num_jobs" : 10, // total number of jobs being simulated
+      "rand_seed" : 2, //the random seed used for deterministic randomized runs
+
+      // a list of “workloads”, each of which has job classes, and temporal properties
+      "workloads" : [
+        {
+          "workload_name" : "tiny-test", // name of the workload
+          "workload_weight": 0.5,  // used for weighted random selection of which workload to sample from
+          "queue_name" : "sls_queue_1", //queue the job will be submitted to
+
+        //different classes of jobs for this workload
+           "job_classes" : [
+            {
+              "class_name" : "class_1", //name of the class
+              "class_weight" : 1.0, //used for weighted random selection of class within workload
+
+              //nextr group controls average and standard deviation of a LogNormal distribution that
+              //determines the number of mappers and reducers for thejob.
+              "mtasks_avg" : 5,
+              "mtasks_stddev" : 1,
+              "rtasks_avg" : 5,
+              "rtasks_stddev" : 1,
+
+              //averge and stdev input param of LogNormal distribution controlling job duration
+              "dur_avg" : 60,
+              "dur_stddev" : 5,
+
+              //averge and stdev input param of LogNormal distribution controlling mappers and reducers durations
+              "mtime_avg" : 10,
+              "mtime_stddev" : 2,
+              "rtime_avg" : 20,
+              "rtime_stddev" : 4,
+
+              //averge and stdev input param of LogNormal distribution controlling memory and cores for map and reduce
+              "map_max_memory_avg" : 1024,
+              "map_max_memory_stddev" : 0.001,
+              "reduce_max_memory_avg" : 2048,
+              "reduce_max_memory_stddev" : 0.001,
+              "map_max_vcores_avg" : 1,
+              "map_max_vcores_stddev" : 0.001,
+              "reduce_max_vcores_avg" : 2,
+              "reduce_max_vcores_stddev" : 0.001,
+
+              //probability of running this job with a reservation
+              "chance_of_reservation" : 0.5,
+              //input parameters of LogNormal distribution that determines the deadline slack (as a multiplier of job duration)
+              "deadline_factor_avg" : 10.0,
+              "deadline_factor_stddev" : 0.001,
+            }
+           ],
+        // for each workload determines with what probability each time bucket is picked to choose the job starttime.
+        // In the example below the jobs have twice as much chance to start in the first minute than in the second minute
+        // of simulation, and then zero chance thereafter.
+          "time_distribution" : [
+            { "time" : 1, "weight" : 66 },
+            { "time" : 60, "weight" : 33 },
+            { "time" : 120, "jobs" : 0 }
+         ]
+        }
+     ]
+    }
+
+
 ### Simulator input topology file format
 
 Here is an example input topology file which has 3 nodes organized in 1 rack.
@@ -355,3 +468,9 @@ Here is an example input topology file which has 3 nodes organized in 1 rack.
         "node" : "node3"
       }]
     }
+
+### Notes on LogNormal distribution:
+LogNormal distributions represent well many of the parameters we see in practice (e.g., most jobs have
+a small number of mappers, but few might be very large, and few very small, but greater than zero. It is
+however worth noticing that it might be tricky to use, as the average is typically on the right side of the
+peak (most common value) of the distribution, because the distribution has a one-side tail.

+ 151 - 0
hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/BaseSLSRunnerTest.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.sls;
+
+import net.jcip.annotations.NotThreadSafe;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.monitor.invariants.MetricsInvariantChecker;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameter;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+
+/**
+ * This is a base class to ease the implementation of SLS-based tests.
+ */
+@RunWith(value = Parameterized.class)
+@NotThreadSafe
+@SuppressWarnings("VisibilityModifier")
+public abstract class BaseSLSRunnerTest {
+
+  @Parameter(value = 0)
+  public String schedulerType;
+
+  @Parameter(value = 1)
+  public String traceType;
+
+  @Parameter(value = 2)
+  public String traceLocation;
+
+  @Parameter(value = 3)
+  public String nodeFile;
+
+  protected SLSRunner sls;
+  protected String ongoingInvariantFile;
+  protected String exitInvariantFile;
+
+  @Before
+  public abstract void setup();
+
+  @After
+  public void tearDown() throws InterruptedException {
+    sls.stop();
+  }
+
+  public void runSLS(Configuration conf, long timeout) throws Exception {
+    File tempDir = new File("target", UUID.randomUUID().toString());
+    final List<Throwable> exceptionList =
+        Collections.synchronizedList(new ArrayList<Throwable>());
+
+    Thread.setDefaultUncaughtExceptionHandler(
+        new Thread.UncaughtExceptionHandler() {
+          @Override
+          public void uncaughtException(Thread t, Throwable e) {
+            e.printStackTrace();
+            exceptionList.add(e);
+          }
+        });
+
+    // start the simulator
+    File slsOutputDir = new File(tempDir.getAbsolutePath() + "/slsoutput/");
+
+    String[] args;
+
+    switch (traceType) {
+    case "OLD_SLS":
+      args = new String[] {"-inputsls", traceLocation, "-output",
+          slsOutputDir.getAbsolutePath() };
+      break;
+    case "OLD_RUMEN":
+      args = new String[] {"-inputrumen", traceLocation, "-output",
+          slsOutputDir.getAbsolutePath() };
+      break;
+    default:
+      args = new String[] {"-tracetype", traceType, "-tracelocation",
+          traceLocation, "-output", slsOutputDir.getAbsolutePath() };
+    }
+
+    if (nodeFile != null) {
+      args = ArrayUtils.addAll(args, new String[] {"-nodes", nodeFile });
+    }
+
+    // enable continuous invariant checks
+    conf.set(YarnConfiguration.RM_SCHEDULER, schedulerType);
+    if (ongoingInvariantFile != null) {
+      conf.set(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES,
+          MetricsInvariantChecker.class.getCanonicalName());
+      conf.set(MetricsInvariantChecker.INVARIANTS_FILE, ongoingInvariantFile);
+      conf.setBoolean(MetricsInvariantChecker.THROW_ON_VIOLATION, true);
+    }
+
+    sls = new SLSRunner(conf);
+    sls.run(args);
+
+    // wait for timeout seconds before stop, unless there is an uncaught
+    // exception in which
+    // case fail fast.
+    while (timeout >= 0) {
+      Thread.sleep(1000);
+
+      if (!exceptionList.isEmpty()) {
+        sls.stop();
+        Assert.fail("TestSLSRunner catched exception from child thread "
+            + "(TaskRunner.Task): " + exceptionList);
+        break;
+      }
+      timeout--;
+    }
+    shutdownHookInvariantCheck();
+  }
+
+  /**
+   * Checks exit invariants (e.g., number of apps submitted, completed, etc.).
+   */
+  private void shutdownHookInvariantCheck() {
+
+    if(exitInvariantFile!=null) {
+      MetricsInvariantChecker ic = new MetricsInvariantChecker();
+      Configuration conf = new Configuration();
+      conf.set(MetricsInvariantChecker.INVARIANTS_FILE, exitInvariantFile);
+      conf.setBoolean(MetricsInvariantChecker.THROW_ON_VIOLATION, true);
+      ic.init(conf, null, null);
+      ic.editSchedule();
+    }
+  }
+
+}

+ 77 - 0
hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestReservationSystemInvariants.java

@@ -0,0 +1,77 @@
+/**
+ * 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.sls;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.monitor.invariants.InvariantsChecker;
+import org.apache.hadoop.yarn.server.resourcemanager.monitor.invariants.ReservationInvariantsChecker;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import net.jcip.annotations.NotThreadSafe;
+
+/**
+ * This test performs an SLS run enabling a
+ * {@code ReservationInvariantsChecker}.
+ */
+@RunWith(value = Parameterized.class)
+@NotThreadSafe
+public class TestReservationSystemInvariants extends BaseSLSRunnerTest {
+
+  @Parameters(name = "Testing with: {1}, {0}, (nodeFile {3})")
+  public static Collection<Object[]> data() {
+    // Test with both schedulers, and all three trace types
+    return Arrays.asList(new Object[][] {
+        {CapacityScheduler.class.getCanonicalName(), "SYNTH",
+            "src/test/resources/syn.json", null},
+        {FairScheduler.class.getCanonicalName(), "SYNTH",
+            "src/test/resources/syn.json", null}
+    });
+  }
+
+  @Test(timeout = 120000)
+  @SuppressWarnings("all")
+  public void testSimulatorRunning() throws Exception {
+
+    Configuration conf = new Configuration(false);
+    conf.set(YarnConfiguration.RM_SCHEDULER, schedulerType);
+    conf.setBoolean(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS, true);
+    conf.set(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES,
+        ReservationInvariantsChecker.class.getCanonicalName());
+    conf.setBoolean(InvariantsChecker.THROW_ON_VIOLATION, true);
+
+
+    long timeTillShutDownInSec = 90;
+    runSLS(conf, timeTillShutDownInSec);
+
+  }
+
+  @Override
+  public void setup() {
+
+  }
+}

+ 57 - 38
hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSLSRunner.java

@@ -18,53 +18,72 @@
 
 package org.apache.hadoop.yarn.sls;
 
-import org.junit.Assert;
+import net.jcip.annotations.NotThreadSafe;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
+import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.*;
 
-import java.io.File;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.UUID;
+import java.util.*;
 
-public class TestSLSRunner {
+/**
+ * This test performs simple runs of the SLS with different trace types and
+ * schedulers.
+ */
+@RunWith(value = Parameterized.class)
+@NotThreadSafe
+public class TestSLSRunner extends BaseSLSRunnerTest {
 
-  @Test
-  @SuppressWarnings("all")
-  public void testSimulatorRunning() throws Exception {
-    File tempDir = new File("target", UUID.randomUUID().toString());
-    final List<Throwable> exceptionList =
-        Collections.synchronizedList(new ArrayList<Throwable>());
+  @Parameters(name = "Testing with: {1}, {0}, (nodeFile {3})")
+  public static Collection<Object[]> data() {
 
-    Thread.setDefaultUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
-      @Override
-      public void uncaughtException(Thread t, Throwable e) {
-        exceptionList.add(e);
-      }
-    });
+    String capScheduler = CapacityScheduler.class.getCanonicalName();
+    String fairScheduler = FairScheduler.class.getCanonicalName();
+    String slsTraceFile = "src/test/resources/inputsls.json";
+    String rumenTraceFile = "src/main/data/2jobs2min-rumen-jh.json";
+    String synthTraceFile = "src/test/resources/syn.json";
+    String nodeFile = "src/test/resources/nodes.json";
+
+    // Test with both schedulers, and all three load producers.
+    return Arrays.asList(new Object[][] {
 
-    // start the simulator
-    File slsOutputDir = new File(tempDir.getAbsolutePath() + "/slsoutput/");
-    String args[] = new String[]{
-            "-inputrumen", "src/main/data/2jobs2min-rumen-jh.json",
-            "-output", slsOutputDir.getAbsolutePath()};
-    SLSRunner.main(args);
+        // covering old commandline in tests
+        {capScheduler, "OLD_RUMEN", rumenTraceFile, nodeFile },
+        {capScheduler, "OLD_SLS", slsTraceFile, nodeFile },
 
-    // wait for 20 seconds before stop
-    int count = 20;
-    while (count >= 0) {
-      Thread.sleep(1000);
+        // covering the no nodeFile case
+        {capScheduler, "SYNTH", synthTraceFile, null },
+        {capScheduler, "RUMEN", rumenTraceFile, null },
+        {capScheduler, "SLS", slsTraceFile, null },
 
-      if (! exceptionList.isEmpty()) {
-        SLSRunner.getRunner().stop();
-        Assert.fail("TestSLSRunner catched exception from child thread " +
-            "(TaskRunner.Task): " + exceptionList.get(0).getMessage());
-        break;
-      }
-      count--;
-    }
+        // covering new commandline and CapacityScheduler
+        {capScheduler, "SYNTH", synthTraceFile, nodeFile },
+        {capScheduler, "RUMEN", rumenTraceFile, nodeFile },
+        {capScheduler, "SLS", slsTraceFile, nodeFile },
 
-    SLSRunner.getRunner().stop();
+        // covering FairScheduler
+        {fairScheduler, "SYNTH", synthTraceFile, nodeFile },
+        {fairScheduler, "RUMEN", rumenTraceFile, nodeFile },
+        {fairScheduler, "SLS", slsTraceFile, nodeFile }
+    });
+  }
+
+  @Before
+  public void setup() {
+    ongoingInvariantFile = "src/test/resources/ongoing-invariants.txt";
+    exitInvariantFile = "src/test/resources/exit-invariants.txt";
+  }
+
+  @Test(timeout = 120000)
+  @SuppressWarnings("all")
+  public void testSimulatorRunning() throws Exception {
+    Configuration conf = new Configuration(false);
+    long timeTillShutdownInsec = 20L;
+    runSLS(conf, timeTillShutdownInsec);
   }
 
 }

+ 96 - 0
hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSynthJobGeneration.java

@@ -0,0 +1,96 @@
+/**
+ * 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.sls;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.tools.rumen.TaskAttemptInfo;
+import org.apache.hadoop.yarn.sls.synthetic.SynthJob;
+import org.apache.hadoop.yarn.sls.synthetic.SynthTraceJobProducer;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.log4j.Logger;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Simple test class driving the {@code SynthTraceJobProducer}, and validating
+ * jobs produce are within expected range.
+ */
+public class TestSynthJobGeneration {
+
+  public final static Logger LOG =
+      Logger.getLogger(TestSynthJobGeneration.class);
+
+  @Test
+  public void test() throws IllegalArgumentException, IOException {
+
+    Configuration conf = new Configuration();
+
+    conf.set(SynthTraceJobProducer.SLS_SYNTHETIC_TRACE_FILE,
+        "src/test/resources/syn.json");
+
+    SynthTraceJobProducer stjp = new SynthTraceJobProducer(conf);
+
+    SynthJob js = (SynthJob) stjp.getNextJob();
+
+    int jobCount = 0;
+
+    while (js != null) {
+      LOG.info((jobCount++) + " " + js.getQueueName() + " -- "
+          + js.getJobClass().getClassName() + " (conf: "
+          + js.getJobConf().get(MRJobConfig.QUEUE_NAME) + ") " + " submission: "
+          + js.getSubmissionTime() + ", " + " duration: " + js.getDuration()
+          + " numMaps: " + js.getNumberMaps() + " numReduces: "
+          + js.getNumberReduces());
+
+      validateJob(js);
+      js = (SynthJob) stjp.getNextJob();
+    }
+
+    Assert.assertEquals(stjp.getNumJobs(), jobCount);
+  }
+
+  private void validateJob(SynthJob js) {
+
+    assertTrue(js.getSubmissionTime() > 0);
+    assertTrue(js.getDuration() > 0);
+    assertTrue(js.getNumberMaps() >= 0);
+    assertTrue(js.getNumberReduces() >= 0);
+    assertTrue(js.getNumberMaps() + js.getNumberReduces() > 0);
+    assertTrue(js.getTotalSlotTime() >= 0);
+
+    for (int i = 0; i < js.getNumberMaps(); i++) {
+      TaskAttemptInfo tai = js.getTaskAttemptInfo(TaskType.MAP, i, 0);
+      assertTrue(tai.getRuntime() > 0);
+    }
+
+    for (int i = 0; i < js.getNumberReduces(); i++) {
+      TaskAttemptInfo tai = js.getTaskAttemptInfo(TaskType.REDUCE, i, 0);
+      assertTrue(tai.getRuntime() > 0);
+    }
+
+    if (js.hasDeadline()) {
+      assertTrue(js.getDeadline() > js.getSubmissionTime() + js.getDuration());
+    }
+
+  }
+}

+ 79 - 10
hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/appmaster/TestAMSimulator.java

@@ -17,32 +17,62 @@
  */
 package org.apache.hadoop.yarn.sls.appmaster;
 
+import com.codahale.metrics.MetricRegistry;
+import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
 import org.apache.hadoop.yarn.sls.conf.SLSConfiguration;
-import org.apache.hadoop.yarn.sls.scheduler.ContainerSimulator;
+import org.apache.hadoop.yarn.sls.scheduler.*;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.List;
 
+@RunWith(Parameterized.class)
 public class TestAMSimulator {
   private ResourceManager rm;
   private YarnConfiguration conf;
+  private Path metricOutputDir;
+
+  private Class slsScheduler;
+  private Class scheduler;
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> params() {
+    return Arrays.asList(new Object[][] {
+        {SLSFairScheduler.class, FairScheduler.class},
+        {SLSCapacityScheduler.class, CapacityScheduler.class}
+    });
+  }
+
+  public TestAMSimulator(Class slsScheduler, Class scheduler) {
+    this.slsScheduler = slsScheduler;
+    this.scheduler = scheduler;
+  }
 
   @Before
   public void setup() {
+    createMetricOutputDir();
+
     conf = new YarnConfiguration();
-    conf.set(YarnConfiguration.RM_SCHEDULER,
-        "org.apache.hadoop.yarn.sls.scheduler.ResourceSchedulerWrapper");
-    conf.set(SLSConfiguration.RM_SCHEDULER,
-        "org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler");
-    conf.setBoolean(SLSConfiguration.METRICS_SWITCH, false);
+    conf.set(SLSConfiguration.METRICS_OUTPUT_DIR, metricOutputDir.toString());
+    conf.set(YarnConfiguration.RM_SCHEDULER, slsScheduler.getName());
+    conf.set(SLSConfiguration.RM_SCHEDULER, scheduler.getName());
+    conf.setBoolean(SLSConfiguration.METRICS_SWITCH, true);
     rm = new ResourceManager();
     rm.init(conf);
     rm.start();
@@ -64,14 +94,51 @@ public class TestAMSimulator {
     }
   }
 
+  private void verifySchedulerMetrics(String appId) {
+    if (scheduler.equals(FairScheduler.class)) {
+      SchedulerMetrics schedulerMetrics = ((SchedulerWrapper)
+          rm.getResourceScheduler()).getSchedulerMetrics();
+      MetricRegistry metricRegistry = schedulerMetrics.getMetrics();
+      for (FairSchedulerMetrics.Metric metric :
+          FairSchedulerMetrics.Metric.values()) {
+        String key = "variable.app." + appId + "." + metric.getValue() +
+            ".memory";
+        Assert.assertTrue(metricRegistry.getGauges().containsKey(key));
+        Assert.assertNotNull(metricRegistry.getGauges().get(key).getValue());
+      }
+    }
+  }
+
+  private void createMetricOutputDir() {
+    Path testDir = Paths.get(System.getProperty("test.build.data"));
+    try {
+      metricOutputDir = Files.createTempDirectory(testDir, "output");
+    } catch (IOException e) {
+      Assert.fail(e.toString());
+    }
+  }
+
+  private void deleteMetricOutputDir() {
+    try {
+      FileUtils.deleteDirectory(metricOutputDir.toFile());
+    } catch (IOException e) {
+      Assert.fail(e.toString());
+    }
+  }
+
   @Test
   public void testAMSimulator() throws Exception {
     // Register one app
     MockAMSimulator app = new MockAMSimulator();
-    List<ContainerSimulator> containers = new ArrayList<ContainerSimulator>();
-    app.init(1, 1000, containers, rm, null, 0, 1000000l, "user1", "default",
-        false, "app1");
+    String appId = "app1";
+    String queue = "default";
+    List<ContainerSimulator> containers = new ArrayList<>();
+    app.init(1000, containers, rm, null, 0, 1000000L, "user1", queue, true,
+        appId, null, 0, SLSConfiguration.getAMContainerResource(conf));
     app.firstStep();
+
+    verifySchedulerMetrics(appId);
+
     Assert.assertEquals(1, rm.getRMContext().getRMApps().size());
     Assert.assertNotNull(rm.getRMContext().getRMApps().get(app.appId));
 
@@ -82,5 +149,7 @@ public class TestAMSimulator {
   @After
   public void tearDown() {
     rm.stop();
+
+    deleteMetricOutputDir();
   }
-}
+}

+ 28 - 4
hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/nodemanager/TestNMSimulator.java

@@ -21,26 +21,50 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.sls.conf.SLSConfiguration;
+import org.apache.hadoop.yarn.sls.scheduler.SLSCapacityScheduler;
+import org.apache.hadoop.yarn.sls.scheduler.SLSFairScheduler;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
+import java.util.Arrays;
+import java.util.Collection;
+
+@RunWith(Parameterized.class)
 public class TestNMSimulator {
   private final int GB = 1024;
   private ResourceManager rm;
   private YarnConfiguration conf;
 
+  private Class slsScheduler;
+  private Class scheduler;
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> params() {
+    return Arrays.asList(new Object[][] {
+        {SLSFairScheduler.class, FairScheduler.class},
+        {SLSCapacityScheduler.class, CapacityScheduler.class}
+    });
+  }
+
+  public TestNMSimulator(Class slsScheduler, Class scheduler) {
+    this.slsScheduler = slsScheduler;
+    this.scheduler = scheduler;
+  }
+
   @Before
   public void setup() {
     conf = new YarnConfiguration();
-    conf.set(YarnConfiguration.RM_SCHEDULER,
-        "org.apache.hadoop.yarn.sls.scheduler.ResourceSchedulerWrapper");
-    conf.set(SLSConfiguration.RM_SCHEDULER,
-        "org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler");
+    conf.set(YarnConfiguration.RM_SCHEDULER, slsScheduler.getName());
+    conf.set(SLSConfiguration.RM_SCHEDULER, scheduler.getName());
     conf.setBoolean(SLSConfiguration.METRICS_SWITCH, false);
     rm = new ResourceManager();
     rm.init(conf);

+ 1 - 1
hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/scheduler/TestTaskRunner.java

@@ -35,7 +35,7 @@ public class TestTaskRunner {
   }
 
   @After
-  public void cleanUp() {
+  public void cleanUp() throws InterruptedException {
     runner.stop();
   }
 

+ 30 - 0
hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/utils/TestSLSUtils.java

@@ -21,6 +21,9 @@ package org.apache.hadoop.yarn.sls.utils;
 import org.junit.Assert;
 import org.junit.Test;
 
+import java.util.HashSet;
+import java.util.Set;
+
 public class TestSLSUtils {
 
   @Test
@@ -36,4 +39,31 @@ public class TestSLSUtils {
     Assert.assertEquals(rackHostname[1], "node1");
   }
 
+  @Test
+  public void testGenerateNodes() {
+    Set<? extends String> nodes = SLSUtils.generateNodes(3, 3);
+    Assert.assertEquals("Number of nodes is wrong.", 3, nodes.size());
+    Assert.assertEquals("Number of racks is wrong.", 3, getNumRack(nodes));
+
+    nodes = SLSUtils.generateNodes(3, 1);
+    Assert.assertEquals("Number of nodes is wrong.", 3, nodes.size());
+    Assert.assertEquals("Number of racks is wrong.", 1, getNumRack(nodes));
+
+    nodes = SLSUtils.generateNodes(3, 4);
+    Assert.assertEquals("Number of nodes is wrong.", 3, nodes.size());
+    Assert.assertEquals("Number of racks is wrong.", 3, getNumRack(nodes));
+
+    nodes = SLSUtils.generateNodes(3, 0);
+    Assert.assertEquals("Number of nodes is wrong.", 3, nodes.size());
+    Assert.assertEquals("Number of racks is wrong.", 1, getNumRack(nodes));
+  }
+
+  private int getNumRack(Set<? extends String> nodes) {
+    Set<String> racks = new HashSet<>();
+    for (String node : nodes) {
+      String[] rackHostname = SLSUtils.getRackHostName(node);
+      racks.add(rackHostname[0]);
+    }
+    return racks.size();
+  }
 }

+ 14 - 14
hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/web/TestSLSWebApp.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.yarn.sls.web;
 
 import org.junit.Assert;
 import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.yarn.sls.SLSRunner;
 import org.junit.Test;
 
 import java.io.File;
@@ -28,6 +27,7 @@ import java.text.MessageFormat;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
+import java.util.HashMap;
 
 public class TestSLSWebApp {
 
@@ -36,20 +36,21 @@ public class TestSLSWebApp {
     String simulateInfoTemplate = FileUtils.readFileToString(
             new File("src/main/html/simulate.info.html.template"));
 
-    SLSRunner.simulateInfoMap.put("Number of racks", 10);
-    SLSRunner.simulateInfoMap.put("Number of nodes", 100);
-    SLSRunner.simulateInfoMap.put("Node memory (MB)", 1024);
-    SLSRunner.simulateInfoMap.put("Node VCores", 1);
-    SLSRunner.simulateInfoMap.put("Number of applications", 100);
-    SLSRunner.simulateInfoMap.put("Number of tasks", 1000);
-    SLSRunner.simulateInfoMap.put("Average tasks per applicaion", 10);
-    SLSRunner.simulateInfoMap.put("Number of queues", 4);
-    SLSRunner.simulateInfoMap.put("Average applications per queue", 25);
-    SLSRunner.simulateInfoMap.put("Estimated simulate time (s)", 10000);
+    Map<String, Object> simulateInfoMap = new HashMap<>();
+    simulateInfoMap.put("Number of racks", 10);
+    simulateInfoMap.put("Number of nodes", 100);
+    simulateInfoMap.put("Node memory (MB)", 1024);
+    simulateInfoMap.put("Node VCores", 1);
+    simulateInfoMap.put("Number of applications", 100);
+    simulateInfoMap.put("Number of tasks", 1000);
+    simulateInfoMap.put("Average tasks per applicaion", 10);
+    simulateInfoMap.put("Number of queues", 4);
+    simulateInfoMap.put("Average applications per queue", 25);
+    simulateInfoMap.put("Estimated simulate time (s)", 10000);
 
     StringBuilder info = new StringBuilder();
     for (Map.Entry<String, Object> entry :
-            SLSRunner.simulateInfoMap.entrySet()) {
+        simulateInfoMap.entrySet()) {
       info.append("<tr>");
       info.append("<td class='td1'>" + entry.getKey() + "</td>");
       info.append("<td class='td2'>" + entry.getValue() + "</td>");
@@ -60,8 +61,7 @@ public class TestSLSWebApp {
             MessageFormat.format(simulateInfoTemplate, info.toString());
     Assert.assertTrue("The simulate info html page should not be empty",
             simulateInfo.length() > 0);
-    for (Map.Entry<String, Object> entry :
-            SLSRunner.simulateInfoMap.entrySet()) {
+    for (Map.Entry<String, Object> entry : simulateInfoMap.entrySet()) {
       Assert.assertTrue("The simulate info html page should have information "
               + "of " + entry.getKey(), simulateInfo.contains("<td class='td1'>"
               + entry.getKey() + "</td><td class='td2'>"

+ 10 - 0
hadoop-tools/hadoop-sls/src/test/resources/capacity-scheduler.xml

@@ -38,6 +38,16 @@
     <value>100</value>
   </property>
 
+  <property>
+    <name>yarn.scheduler.capacity.root.sls_queue_1.reservable</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.root.sls_queue_1.show-reservations-as-queues</name>
+    <value>true</value>
+  </property>
+
   <property>
     <name>yarn.scheduler.capacity.root.sls_queue_2.capacity</name>
     <value>25</value>

+ 8 - 0
hadoop-tools/hadoop-sls/src/test/resources/exit-invariants.txt

@@ -0,0 +1,8 @@
+ActiveApplications >= 0
+AppsCompleted >= 0
+AppsFailed >= 0
+AppsKilled >= 0
+AppsPending >= 0
+AppsRunning >= 0
+AppsSubmitted >= 0
+PendingContainers >= 0

+ 5 - 3
hadoop-tools/hadoop-sls/src/test/resources/fair-scheduler.xml

@@ -21,6 +21,7 @@
 -->
 
 <allocations>
+  <defaultQueueSchedulingPolicy>drf</defaultQueueSchedulingPolicy>
   <user name="jenkins">
     <!-- Limit on running jobs for the user across all pools. If more
       jobs than this are submitted, only the first <maxRunningJobs> will
@@ -31,20 +32,21 @@
   <userMaxAppsDefault>1000</userMaxAppsDefault>
   <queue name="sls_queue_1">
     <minResources>1024 mb, 1 vcores</minResources>
-    <schedulingMode>fair</schedulingMode>
+    <schedulingPolicy>drf</schedulingPolicy>
     <weight>0.25</weight>
     <minSharePreemptionTimeout>2</minSharePreemptionTimeout>
+    <reservation>true</reservation>
   </queue>
   <queue name="sls_queue_2">
     <minResources>1024 mb, 1 vcores</minResources>
-    <schedulingMode>fair</schedulingMode>
+    <schedulingMode>drf</schedulingMode>
     <weight>0.25</weight>
     <minSharePreemptionTimeout>2</minSharePreemptionTimeout>
   </queue>
   <queue name="sls_queue_3">
     <minResources>1024 mb, 1 vcores</minResources>
     <weight>0.5</weight>
-    <schedulingMode>fair</schedulingMode>
+    <schedulingMode>drf</schedulingMode>
     <minSharePreemptionTimeout>2</minSharePreemptionTimeout>
   </queue>
 </allocations>

+ 55 - 0
hadoop-tools/hadoop-sls/src/test/resources/inputsls.json

@@ -0,0 +1,55 @@
+{
+  "am.type": "mapreduce",
+  "job.start.ms": 0,
+  "job.end.ms": 95375,
+  "job.queue.name": "sls_queue_1",
+  "job.id": "job_1",
+  "job.user": "default",
+  "job.tasks": [
+    {
+      "container.host": "/default-rack/node1",
+      "container.start.ms": 6664,
+      "container.end.ms": 23707,
+      "container.priority": 20,
+      "container.type": "map"
+    },
+    {
+      "container.host": "/default-rack/node3",
+      "container.start.ms": 6665,
+      "container.end.ms": 21593,
+      "container.priority": 20,
+      "container.type": "map"
+    },
+    {
+      "container.host": "/default-rack/node2",
+      "container.start.ms": 68770,
+      "container.end.ms": 86613,
+      "container.priority": 20,
+      "container.type": "map"
+    }
+  ]
+}
+{
+  "am.type": "mapreduce",
+  "job.start.ms": 105204,
+  "job.end.ms": 197256,
+  "job.queue.name": "sls_queue_2",
+  "job.id": "job_2",
+  "job.user": "default",
+  "job.tasks": [
+    {
+      "container.host": "/default-rack/node1",
+      "container.start.ms": 111822,
+      "container.end.ms": 133985,
+      "container.priority": 20,
+      "container.type": "map"
+    },
+    {
+      "container.host": "/default-rack/node2",
+      "container.start.ms": 111788,
+      "container.end.ms": 131377,
+      "container.priority": 20,
+      "container.type": "map"
+    }
+  ]
+}

+ 19 - 0
hadoop-tools/hadoop-sls/src/test/resources/log4j.properties

@@ -0,0 +1,19 @@
+#   Licensed 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.
+
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=info,stdout
+log4j.threshold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2} (%F:%M(%L)) - %m%n

+ 84 - 0
hadoop-tools/hadoop-sls/src/test/resources/nodes.json

@@ -0,0 +1,84 @@
+{
+  "rack": "rack1",
+  "nodes": [
+    {
+      "node": "node1"
+    },
+    {
+      "node": "node2"
+    },
+    {
+      "node": "node3"
+    },
+    {
+      "node": "node4"
+    }
+  ]
+}
+{
+  "rack": "rack2",
+  "nodes": [
+    {
+      "node": "node5"
+    },
+    {
+      "node": "node6"
+    },
+    {
+      "node": "node7"
+    },
+    {
+      "node": "node8"
+    }
+  ]
+}
+{
+  "rack": "rack3",
+  "nodes": [
+    {
+      "node": "node9"
+    },
+    {
+      "node": "node10"
+    },
+    {
+      "node": "node11"
+    },
+    {
+      "node": "node12"
+    }
+  ]
+}
+{
+  "rack": "rack4",
+  "nodes": [
+    {
+      "node": "node13"
+    },
+    {
+      "node": "node14"
+    },
+    {
+      "node": "node15"
+    },
+    {
+      "node": "node16"
+    }
+  ]
+}
+{
+  "rack": "rack5",
+  "nodes": [
+    {
+      "node": "node17"
+    },
+    {
+      "node": "node18"
+    },
+    {
+      "node": "node19"
+    },
+    {
+    }
+  ]
+}

+ 54 - 0
hadoop-tools/hadoop-sls/src/test/resources/ongoing-invariants.txt

@@ -0,0 +1,54 @@
+running_0 >= 0
+running_60 >= 0
+running_300 >= 0
+running_1440 >= 0
+AppsSubmitted >= 0
+AppsRunning >= 0
+AppsPending >= 0
+AppsCompleted >= 0
+AppsKilled >= 0
+AppsFailed >= 0
+AllocatedMB >= 0
+AllocatedVCores >= 0
+AllocatedContainers >= 0
+AggregateContainersAllocated >= 0
+AggregateNodeLocalContainersAllocated >= 0
+AggregateRackLocalContainersAllocated >= 0
+AggregateOffSwitchContainersAllocated >= 0
+AggregateContainersReleased >= 0
+AggregateContainersPreempted >= 0
+AvailableMB >= 0
+AvailableVCores >= 0
+PendingMB >= 0
+PendingVCores >= 0
+PendingContainers >= 0
+ReservedMB >= 0
+ReservedVCores >= 0
+ReservedContainers >= 0
+ActiveUsers >= 0
+ActiveApplications >= 0
+AppAttemptFirstContainerAllocationDelayNumOps >= 0
+AppAttemptFirstContainerAllocationDelayAvgTime >= 0
+MemNonHeapUsedM >= 0
+MemNonHeapCommittedM >= 0
+MemNonHeapMaxM >= 0 || MemNonHeapMaxM == -1
+MemHeapUsedM >= 0
+MemHeapCommittedM >= 0
+MemHeapMaxM >= 0
+MemMaxM >= 0
+GcCountPS_Scavenge >= 0
+GcTimeMillisPS_Scavenge >= 0
+GcCountPS_MarkSweep >= 0
+GcTimeMillisPS_MarkSweep >= 0
+GcCount >= 0
+GcTimeMillis >= 0
+ThreadsNew >= 0
+ThreadsRunnable >= 0
+ThreadsBlocked >= 0
+ThreadsWaiting >= 0
+ThreadsTimedWaiting >= 0
+ThreadsTerminated >= 0
+LogFatal >= 0
+LogError >= 0
+LogWarn >= 0
+LogInfo >= 0

+ 3 - 3
hadoop-tools/hadoop-sls/src/test/resources/sls-runner.xml

@@ -25,11 +25,11 @@
   <!-- Nodes configuration -->
   <property>
     <name>yarn.sls.nm.memory.mb</name>
-    <value>10240</value>
+    <value>100240</value>
   </property>
   <property>
     <name>yarn.sls.nm.vcores</name>
-    <value>10</value>
+    <value>100</value>
   </property>
   <property>
     <name>yarn.sls.nm.heartbeat.interval.ms</name>
@@ -77,5 +77,5 @@
     <name>org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler</name>
     <value>org.apache.hadoop.yarn.sls.scheduler.CapacitySchedulerMetrics</value>
   </property>
-  
+
 </configuration>

+ 53 - 0
hadoop-tools/hadoop-sls/src/test/resources/syn.json

@@ -0,0 +1,53 @@
+{
+  "description": "tiny jobs workload",
+  "num_nodes": 20,
+  "nodes_per_rack": 4,
+  "num_jobs": 10,
+  "rand_seed": 2,
+  "workloads": [
+    {
+      "workload_name": "tiny-test",
+      "workload_weight": 0.5,
+      "description": "Sort jobs",
+      "queue_name": "sls_queue_1",
+      "job_classes": [
+        {
+          "class_name": "class_1",
+          "user_name": "foobar",
+          "class_weight": 1.0,
+          "mtasks_avg": 5,
+          "mtasks_stddev": 1,
+          "rtasks_avg": 5,
+          "rtasks_stddev": 1,
+          "dur_avg": 60,
+          "dur_stddev": 5,
+          "mtime_avg": 10,
+          "mtime_stddev": 2,
+          "rtime_avg": 20,
+          "rtime_stddev": 4,
+          "map_max_memory_avg": 1024,
+          "map_max_memory_stddev": 0.001,
+          "reduce_max_memory_avg": 2048,
+          "reduce_max_memory_stddev": 0.001,
+          "map_max_vcores_avg": 1,
+          "map_max_vcores_stddev": 0.001,
+          "reduce_max_vcores_avg": 2,
+          "reduce_max_vcores_stddev": 0.001,
+          "chance_of_reservation": 0.5,
+          "deadline_factor_avg": 10.0,
+          "deadline_factor_stddev": 0.001
+        }
+      ],
+      "time_distribution": [
+        {
+          "time": 1,
+          "weight": 100
+        },
+        {
+          "time": 60,
+          "jobs": 0
+        }
+      ]
+    }
+  ]
+}

+ 18 - 1
hadoop-tools/hadoop-sls/src/test/resources/yarn-site.xml

@@ -17,7 +17,7 @@
 <configuration>
   <property>
     <name>yarn.resourcemanager.scheduler.class</name>
-	  <value>org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler</value>
+    <value>org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler</value>
     <!-- <value>org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler</value> -->
   </property>
 
@@ -69,4 +69,21 @@
     <name>yarn.scheduler.fair.assignmultiple</name>
     <value>true</value>
   </property>
+
+
+  <property>
+    <description>Enable reservation system.</description>
+    <name>yarn.resourcemanager.reservation-system.enable</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>yarn.nodemanager.resource.memory-mb</name>
+    <value>1000000</value>
+  </property>
+  <property>
+    <name>yarn.nodemanager.resource.cpu-vcores</name>
+    <value>320</value>
+  </property>
+
 </configuration>

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml

@@ -340,6 +340,7 @@
             <exclude>src/test/resources/submit-reservation.json</exclude>
             <exclude>src/test/resources/delete-reservation.json</exclude>
             <exclude>src/test/resources/update-reservation.json</exclude>
+            <exclude>src/test/resources/invariants.txt</exclude>
           </excludes>
         </configuration>
       </plugin>

+ 35 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/InvariantViolationException.java

@@ -0,0 +1,35 @@
+/**
+ * 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.resourcemanager.monitor.invariants;
+
+
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+
+/**
+ * This exception represents the violation of an internal invariant.
+ */
+public class InvariantViolationException extends YarnRuntimeException {
+
+  public InvariantViolationException(String s) {
+    super(s);
+  }
+
+  public InvariantViolationException(String s, Exception e) {
+    super(s, e);
+  }
+}

+ 96 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/InvariantsChecker.java

@@ -0,0 +1,96 @@
+/**
+ * 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.resourcemanager.monitor.invariants;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Abstract invariant checker, that setup common context for invariants
+ * checkers.
+ */
+public abstract class InvariantsChecker implements SchedulingEditPolicy {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(InvariantsChecker.class);
+  public static final String THROW_ON_VIOLATION =
+      "yarn.resourcemanager.invariant-checker.throw-on-violation";
+  public static final String INVARIANT_MONITOR_INTERVAL =
+      "yarn.resourcemanager.invariant-checker.monitor-interval";
+
+  private Configuration conf;
+  private RMContext context;
+  private ResourceScheduler scheduler;
+  private boolean throwOnInvariantViolation;
+  private long monitoringInterval;
+
+  @Override
+  public void init(Configuration config, RMContext rmContext,
+      ResourceScheduler scheduler) {
+    this.conf = config;
+    this.context = rmContext;
+    this.scheduler = scheduler;
+    this.throwOnInvariantViolation =
+        conf.getBoolean(InvariantsChecker.THROW_ON_VIOLATION, false);
+    this.monitoringInterval =
+        conf.getLong(InvariantsChecker.INVARIANT_MONITOR_INTERVAL, 1000L);
+
+    LOG.info("Invariant checker " + this.getPolicyName()
+        + " enabled. Monitoring every " + monitoringInterval
+        + "ms, throwOnViolation=" + throwOnInvariantViolation);
+  }
+
+  @Override
+  public long getMonitoringInterval() {
+    return monitoringInterval;
+  }
+
+  @Override
+  public String getPolicyName() {
+    return this.getClass().getSimpleName();
+  }
+
+  public void logOrThrow(String message) throws InvariantViolationException {
+    if (getThrowOnInvariantViolation()) {
+      throw new InvariantViolationException(message);
+    } else {
+      LOG.warn(message);
+    }
+  }
+
+  public boolean getThrowOnInvariantViolation() {
+    return throwOnInvariantViolation;
+  }
+
+  public Configuration getConf() {
+    return conf;
+  }
+
+  public RMContext getContext() {
+    return context;
+  }
+
+  public ResourceScheduler getScheduler() {
+    return scheduler;
+  }
+
+}

+ 195 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/MetricsInvariantChecker.java

@@ -0,0 +1,195 @@
+/**
+ * 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.resourcemanager.monitor.invariants;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.metrics2.AbstractMetric;
+import org.apache.hadoop.metrics2.MetricsRecord;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.impl.MetricsCollectorImpl;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.source.JvmMetrics;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.script.Compilable;
+import javax.script.CompiledScript;
+import javax.script.ScriptEngineManager;
+import javax.script.ScriptException;
+import javax.script.SimpleBindings;
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * This policy checks at every invocation that a given set of invariants
+ * (specified in a file) are respected over QueueMetrics and JvmMetrics. The
+ * file may contain arbitrary (Javascrip) boolean expression over the metrics
+ * variables.
+ *
+ * The right set of invariants depends on the deployment environment, a large
+ * number of complex invariant can make this check expensive.
+ *
+ * The MetricsInvariantChecker can be configured to throw a RuntimeException or
+ * simlpy warn in the logs if an invariant is not respected.
+ */
+public class MetricsInvariantChecker extends InvariantsChecker {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(MetricsInvariantChecker.class);
+  public static final String INVARIANTS_FILE =
+      "yarn.resourcemanager.invariant-checker.file";
+
+  private MetricsSystem metricsSystem;
+  private MetricsCollectorImpl collector;
+  private SimpleBindings bindings;
+  private ScriptEngineManager manager;
+  private Compilable scriptEngine;
+  private String invariantFile;
+  private Map<String, CompiledScript> invariants;
+  private CompiledScript combinedInvariants;
+
+  // set of metrics we monitor
+  private QueueMetrics queueMetrics;
+  private JvmMetrics jvmMetrics;
+
+  @Override
+  public void init(Configuration config, RMContext rmContext,
+                   ResourceScheduler preemptableResourceScheduler) {
+
+    super.init(config, rmContext, preemptableResourceScheduler);
+
+    this.metricsSystem = DefaultMetricsSystem.instance();
+    this.queueMetrics =
+        QueueMetrics.forQueue(metricsSystem, "root", null, false, getConf());
+    this.jvmMetrics = (JvmMetrics) metricsSystem.getSource("JvmMetrics");
+
+    // at first collect all metrics
+    collector = new MetricsCollectorImpl();
+    queueMetrics.getMetrics(collector, true);
+    jvmMetrics.getMetrics(collector, true);
+
+    // prepare bindings and evaluation engine
+    this.bindings = new SimpleBindings();
+    this.manager = new ScriptEngineManager();
+    this.scriptEngine = (Compilable) manager.getEngineByName("JavaScript");
+
+    // load metrics invariant from file
+    this.invariantFile = getConf().get(MetricsInvariantChecker.INVARIANTS_FILE);
+
+    this.invariants = new HashMap<>();
+
+    // preload all bindings
+    queueMetrics.getMetrics(collector, true);
+    jvmMetrics.getMetrics(collector, true);
+    for (MetricsRecord record : collector.getRecords()) {
+      for (AbstractMetric am : record.metrics()) {
+        bindings.put(am.name().replace(' ', '_'), am.value());
+      }
+    }
+
+    StringBuilder sb = new StringBuilder();
+    try {
+      List<String> tempInv =
+          Files.readLines(new File(invariantFile), Charsets.UTF_8);
+
+
+      boolean first = true;
+      // precompile individual invariants
+      for (String inv : tempInv) {
+
+        if(first) {
+          first = false;
+        } else {
+          sb.append("&&");
+        }
+
+        invariants.put(inv, scriptEngine.compile(inv));
+        sb.append(" (");
+        sb.append(inv);
+        sb.append(") ");
+      }
+
+      // create a single large combined invariant for speed of checking
+      combinedInvariants = scriptEngine.compile(sb.toString());
+
+    } catch (IOException e) {
+      throw new RuntimeException(
+          "Error loading invariant file: " + e.getMessage());
+    } catch (ScriptException e) {
+      throw new RuntimeException("Error compiling invariant " + e.getMessage());
+    }
+
+  }
+
+  @Override
+  public void editSchedule() {
+    // grab all changed metrics and update bindings
+    collector.clear();
+    queueMetrics.getMetrics(collector, false);
+    jvmMetrics.getMetrics(collector, false);
+
+    for (MetricsRecord record : collector.getRecords()) {
+      for (AbstractMetric am : record.metrics()) {
+        bindings.put(am.name().replace(' ', '_'), am.value());
+      }
+    }
+
+    // evaluate all invariants with new bindings
+    try {
+
+      // fastpath check all invariants at once (much faster)
+      boolean allInvHold = (boolean) combinedInvariants.eval(bindings);
+
+      // if any fails, check individually to produce more insightful log
+      if (!allInvHold) {
+        for (Map.Entry<String, CompiledScript> e : invariants.entrySet()) {
+          boolean invariantsHold = (boolean) e.getValue().eval(bindings);
+          if (!invariantsHold) {
+            // filter bindings to produce minimal set
+            Map<String, Object> matchingBindings =
+                extractMatchingBindings(e.getKey(), bindings);
+            logOrThrow("Invariant \"" + e.getKey()
+                + "\" is NOT holding, with bindings: " + matchingBindings);
+          }
+        }
+      }
+    } catch (ScriptException e) {
+      logOrThrow(e.getMessage());
+    }
+  }
+
+  private static Map<String, Object> extractMatchingBindings(String inv,
+      SimpleBindings allBindings) {
+    Map<String, Object> matchingBindings = new HashMap<>();
+    for (Map.Entry<String, Object> s : allBindings.entrySet()) {
+      if (inv.contains(s.getKey())) {
+        matchingBindings.put(s.getKey(), s.getValue());
+      }
+    }
+    return matchingBindings;
+  }
+}

+ 63 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/ReservationInvariantsChecker.java

@@ -0,0 +1,63 @@
+/**
+ * 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.resourcemanager.monitor.invariants;
+
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.Plan;
+import org.apache.hadoop.yarn.util.UTCClock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/**
+ * Invariant checker that checks certain reservation invariants are respected.
+ */
+public class ReservationInvariantsChecker extends InvariantsChecker {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ReservationInvariantsChecker.class);
+
+  private UTCClock clock = new UTCClock();
+
+  @Override
+  public void editSchedule() {
+    Collection<Plan> plans =
+        getContext().getReservationSystem().getAllPlans().values();
+
+    try {
+      for (Plan plan : plans) {
+        long currReservations =
+            plan.getReservationsAtTime(clock.getTime()).size();
+        long numberReservationQueues = getContext().getScheduler()
+            .getQueueInfo(plan.getQueueName(), true, false).getChildQueues()
+            .size();
+        if (currReservations != numberReservationQueues - 1) {
+          logOrThrow("Number of reservations (" + currReservations
+              + ") does NOT match the number of reservationQueues ("
+              + (numberReservationQueues - 1) + "), while it should.");
+        }
+      }
+    } catch (IOException io) {
+      throw new InvariantViolationException("Issue during invariant check: ",
+          io);
+    }
+
+  }
+
+}

+ 22 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/package-info.java

@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Monitoring policies, used to check invariants.
+ */
+package org.apache.hadoop.yarn.server.resourcemanager.monitor.invariants;

+ 99 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/TestMetricsInvariantChecker.java

@@ -0,0 +1,99 @@
+/**
+ * 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.resourcemanager.monitor.invariants;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.source.JvmMetrics;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
+import org.apache.log4j.Logger;
+import org.junit.Before;
+import org.junit.Test;
+
+import static junit.framework.TestCase.fail;
+
+/**
+ * This class tests the {@code MetricsInvariantChecker} by running it multiple
+ * time and reporting the time it takes to execute, as well as verifying that
+ * the invariant throws in case the invariants are not respected.
+ */
+public class TestMetricsInvariantChecker {
+  public final static Logger LOG =
+      Logger.getLogger(TestMetricsInvariantChecker.class);
+
+  private MetricsSystem metricsSystem;
+  private MetricsInvariantChecker ic;
+  private Configuration conf;
+
+  @Before
+  public void setup() {
+    this.metricsSystem = DefaultMetricsSystem.instance();
+    JvmMetrics.initSingleton("ResourceManager", null);
+    this.ic = new MetricsInvariantChecker();
+    this.conf = new Configuration();
+    conf.set(MetricsInvariantChecker.INVARIANTS_FILE,
+        "src/test/resources/invariants.txt");
+    conf.setBoolean(MetricsInvariantChecker.THROW_ON_VIOLATION, true);
+    ic.init(conf, null, null);
+  }
+
+  @Test(timeout = 5000)
+  public void testManyRuns() {
+
+    QueueMetrics qm =
+        QueueMetrics.forQueue(metricsSystem, "root", null, false, conf);
+    qm.setAvailableResourcesToQueue(Resource.newInstance(1, 1));
+
+    int numIterations = 1000;
+    long start = System.currentTimeMillis();
+    for (int i = 0; i < numIterations; i++) {
+      ic.editSchedule();
+    }
+    long end = System.currentTimeMillis();
+
+    System.out.println("Runtime per iteration (avg of " + numIterations
+        + " iterations): " + (end - start) + " tot time");
+
+  }
+
+  @Test
+  public void testViolation() {
+
+    // create a "wrong" condition in which the invariants are not respected
+    QueueMetrics qm =
+        QueueMetrics.forQueue(metricsSystem, "root", null, false, conf);
+    qm.setAvailableResourcesToQueue(Resource.newInstance(-1, -1));
+
+    // test with throwing exception turned on
+    try {
+      ic.editSchedule();
+      fail();
+    } catch (InvariantViolationException i) {
+      // expected
+    }
+
+    // test log-only mode
+    conf.setBoolean(MetricsInvariantChecker.THROW_ON_VIOLATION, false);
+    ic.init(conf, null, null);
+    ic.editSchedule();
+
+  }
+
+}

+ 54 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/invariants.txt

@@ -0,0 +1,54 @@
+running_0 >= 0
+running_60 >= 0
+running_300 >= 0
+running_1440 >= 0
+AppsSubmitted >= 0
+AppsRunning >= 0
+AppsPending >= 0
+AppsCompleted >= 0
+AppsKilled >= 0
+AppsFailed >= 0
+AllocatedMB >= 0
+AllocatedVCores >= 0
+AllocatedContainers >= 0
+AggregateContainersAllocated >= 0
+AggregateNodeLocalContainersAllocated >= 0
+AggregateRackLocalContainersAllocated >= 0
+AggregateOffSwitchContainersAllocated >= 0
+AggregateContainersReleased >= 0
+AggregateContainersPreempted >= 0
+AvailableMB >= 0
+AvailableVCores >= 0
+PendingMB >= 0
+PendingVCores >= 0
+PendingContainers >= 0
+ReservedMB >= 0
+ReservedVCores >= 0
+ReservedContainers >= 0
+ActiveUsers >= 0
+ActiveApplications >= 0
+AppAttemptFirstContainerAllocationDelayNumOps >= 0
+AppAttemptFirstContainerAllocationDelayAvgTime >= 0
+MemNonHeapUsedM >= 0
+MemNonHeapCommittedM >= 0
+MemNonHeapMaxM >= 0 || MemNonHeapMaxM == -1
+MemHeapUsedM >= 0
+MemHeapCommittedM >= 0
+MemHeapMaxM >= 0
+MemMaxM >= 0
+GcCountPS_Scavenge >= 0
+GcTimeMillisPS_Scavenge >= 0
+GcCountPS_MarkSweep >= 0
+GcTimeMillisPS_MarkSweep >= 0
+GcCount >= 0
+GcTimeMillis >= 0
+ThreadsNew >= 0
+ThreadsRunnable >= 0
+ThreadsBlocked >= 0
+ThreadsWaiting >= 0
+ThreadsTimedWaiting >= 0
+ThreadsTerminated >= 0
+LogFatal >= 0
+LogError >= 0
+LogWarn >= 0
+LogInfo >= 0