From 6a3929f2b2c994eec3eae17509485c619cafd4ef Mon Sep 17 00:00:00 2001 From: Wangda Tan Date: Tue, 17 Oct 2017 19:45:09 -0700 Subject: [PATCH] YARN-6608. Backport all SLS improvements from trunk to branch-2. (Carlo Curino via wangda) --- .../hadoop/metrics2/source/JvmMetrics.java | 10 + .../apache/hadoop/tools/rumen/TaskInfo.java | 29 +- hadoop-tools/hadoop-sls/pom.xml | 48 +- .../hadoop-sls/src/main/assemblies/sls.xml | 4 +- .../hadoop-sls/src/main/bin/slsrun.sh | 31 +- .../src/main/data/2jobs2min-rumen-jh.json | 705 +------------ .../src/main/html/js/thirdparty/jquery.js | 2 +- .../yarn/sls/ReservationClientUtil.java | 78 ++ .../hadoop/yarn/sls/RumenToSLSConverter.java | 8 +- .../org/apache/hadoop/yarn/sls/SLSRunner.java | 928 ++++++++++++----- .../yarn/sls/appmaster/AMSimulator.java | 259 +++-- .../yarn/sls/appmaster/MRAMSimulator.java | 230 ++--- .../yarn/sls/conf/SLSConfiguration.java | 15 + .../yarn/sls/nodemanager/NMSimulator.java | 1 - .../sls/resourcemanager/MockAMLauncher.java | 120 +++ .../sls/scheduler/FairSchedulerMetrics.java | 334 +++--- .../scheduler/ResourceSchedulerWrapper.java | 973 ------------------ .../sls/scheduler/SLSCapacityScheduler.java | 686 +++--------- .../yarn/sls/scheduler/SLSFairScheduler.java | 346 +++++++ .../yarn/sls/scheduler/SchedulerMetrics.java | 605 ++++++++++- .../yarn/sls/scheduler/SchedulerWrapper.java | 27 +- .../hadoop/yarn/sls/scheduler/TaskRunner.java | 9 +- .../hadoop/yarn/sls/scheduler/Tracker.java | 46 + .../hadoop/yarn/sls/synthetic/SynthJob.java | 306 ++++++ .../yarn/sls/synthetic/SynthJobClass.java | 180 ++++ .../sls/synthetic/SynthTraceJobProducer.java | 319 ++++++ .../hadoop/yarn/sls/synthetic/SynthUtils.java | 101 ++ .../yarn/sls/synthetic/SynthWorkload.java | 121 +++ .../yarn/sls/synthetic/package-info.java | 22 + .../hadoop/yarn/sls/utils/SLSUtils.java | 61 +- .../apache/hadoop/yarn/sls/web/SLSWebApp.java | 45 +- .../site/markdown/SchedulerLoadSimulator.md | 151 ++- .../hadoop/yarn/sls/BaseSLSRunnerTest.java | 151 +++ .../sls/TestReservationSystemInvariants.java | 77 ++ .../apache/hadoop/yarn/sls/TestSLSRunner.java | 101 +- .../yarn/sls/TestSynthJobGeneration.java | 96 ++ .../yarn/sls/appmaster/TestAMSimulator.java | 89 +- .../yarn/sls/nodemanager/TestNMSimulator.java | 32 +- .../yarn/sls/scheduler/TestTaskRunner.java | 2 +- .../hadoop/yarn/sls/utils/TestSLSUtils.java | 30 + .../hadoop/yarn/sls/web/TestSLSWebApp.java | 28 +- .../src/test/resources/capacity-scheduler.xml | 10 + .../src/test/resources/exit-invariants.txt | 8 + .../src/test/resources/fair-scheduler.xml | 8 +- .../src/test/resources/inputsls.json | 55 + .../src/test/resources/log4j.properties | 19 + .../hadoop-sls/src/test/resources/nodes.json | 84 ++ .../src/test/resources/ongoing-invariants.txt | 54 + .../src/test/resources/sls-runner.xml | 6 +- .../hadoop-sls/src/test/resources/syn.json | 53 + .../src/test/resources/yarn-site.xml | 19 +- .../pom.xml | 1 + .../InvariantViolationException.java | 35 + .../monitor/invariants/InvariantsChecker.java | 96 ++ .../invariants/MetricsInvariantChecker.java | 195 ++++ .../ReservationInvariantsChecker.java | 63 ++ .../monitor/invariants/package-info.java | 22 + .../TestMetricsInvariantChecker.java | 99 ++ .../src/test/resources/invariants.txt | 54 + 59 files changed, 5179 insertions(+), 3108 deletions(-) create mode 100644 hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/ReservationClientUtil.java create mode 100644 hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/resourcemanager/MockAMLauncher.java delete mode 100644 hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java create mode 100644 hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSFairScheduler.java create mode 100644 hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/Tracker.java create mode 100644 hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthJob.java create mode 100644 hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthJobClass.java create mode 100644 hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthTraceJobProducer.java create mode 100644 hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthUtils.java create mode 100644 hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthWorkload.java create mode 100644 hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/package-info.java create mode 100644 hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/BaseSLSRunnerTest.java create mode 100644 hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestReservationSystemInvariants.java create mode 100644 hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSynthJobGeneration.java create mode 100644 hadoop-tools/hadoop-sls/src/test/resources/exit-invariants.txt create mode 100644 hadoop-tools/hadoop-sls/src/test/resources/inputsls.json create mode 100644 hadoop-tools/hadoop-sls/src/test/resources/log4j.properties create mode 100644 hadoop-tools/hadoop-sls/src/test/resources/nodes.json create mode 100644 hadoop-tools/hadoop-sls/src/test/resources/ongoing-invariants.txt create mode 100644 hadoop-tools/hadoop-sls/src/test/resources/syn.json create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/invariants.txt diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/source/JvmMetrics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/source/JvmMetrics.java index caba170507a..c6369cdbdd3 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/source/JvmMetrics.java +++ b/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; diff --git a/hadoop-tools/hadoop-rumen/src/main/java/org/apache/hadoop/tools/rumen/TaskInfo.java b/hadoop-tools/hadoop-rumen/src/main/java/org/apache/hadoop/tools/rumen/TaskInfo.java index 9aa6373bbae..6159f85fef1 100644 --- a/hadoop-tools/hadoop-rumen/src/main/java/org/apache/hadoop/tools/rumen/TaskInfo.java +++ b/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, + long maxMemory) { + this(bytesIn, recsIn, bytesOut, recsOut, maxMemory, 1, + new ResourceUsageMetrics()); + } + + public TaskInfo(long bytesIn, int recsIn, long bytesOut, int recsOut, + 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 */ diff --git a/hadoop-tools/hadoop-sls/pom.xml b/hadoop-tools/hadoop-sls/pom.xml index 3accc087423..4b7a3e2a3e2 100644 --- a/hadoop-tools/hadoop-sls/pom.xml +++ b/hadoop-tools/hadoop-sls/pom.xml @@ -22,7 +22,6 @@ 2.9.0-SNAPSHOT ../../hadoop-project - org.apache.hadoop hadoop-sls 2.9.0-SNAPSHOT Apache Hadoop Scheduler Load Simulator @@ -70,9 +69,34 @@ jetty-util provided + + com.fasterxml.jackson.core + jackson-databind + + + org.apache.commons + commons-lang3 + 3.4 + + + + src/main/ + + html/simulate.html.template + html/simulate.info.html.template + html/track.html.template + html/css/bootstrap-responsive.min.css + html/css/bootstrap.min.css + html/js/thirdparty/bootstrap.min.js + html/js/thirdparty/d3.v3.js + html/js/thirdparty/d3-LICENSE + html/js/thirdparty/jquery.js + + + org.apache.maven.plugins @@ -113,6 +137,11 @@ src/test/resources/simulate.html.template src/test/resources/simulate.info.html.template src/test/resources/track.html.template + src/test/resources/syn.json + src/test/resources/inputsls.json + src/test/resources/nodes.json + src/test/resources/exit-invariants.txt + src/test/resources/ongoing-invariants.txt @@ -177,6 +206,23 @@ + + org.apache.maven.plugins + maven-dependency-plugin + + + deplist + compile + + list + + + + ${project.basedir}/target/hadoop-tools-deps/${project.artifactId}.tools-builtin.txt + + + + diff --git a/hadoop-tools/hadoop-sls/src/main/assemblies/sls.xml b/hadoop-tools/hadoop-sls/src/main/assemblies/sls.xml index 5b09c8160c0..085a5d86d6d 100644 --- a/hadoop-tools/hadoop-sls/src/main/assemblies/sls.xml +++ b/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. --> - + sls dir diff --git a/hadoop-tools/hadoop-sls/src/main/bin/slsrun.sh b/hadoop-tools/hadoop-sls/src/main/bin/slsrun.sh index b5807168e8f..4a109380ba2 100644 --- a/hadoop-tools/hadoop-sls/src/main/bin/slsrun.sh +++ b/hadoop-tools/hadoop-sls/src/main/bin/slsrun.sh @@ -16,7 +16,9 @@ ############################################################################### printUsage() { echo "Usage: slsrun.sh " - echo " --input-rumen|--input-sls=" + echo " --tracetype=" + echo " --tracelocation=" + echo " (deprecated --input-rumen= | --input-sls=)" echo " --output-dir=" echo " [--nodes=]" echo " [--track-jobs=]" @@ -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 diff --git a/hadoop-tools/hadoop-sls/src/main/data/2jobs2min-rumen-jh.json b/hadoop-tools/hadoop-sls/src/main/data/2jobs2min-rumen-jh.json index be868aff147..8298d60d913 100644 --- a/hadoop-tools/hadoop-sls/src/main/data/2jobs2min-rumen-jh.json +++ b/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 } diff --git a/hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/jquery.js b/hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/jquery.js index c5c648255c1..38f046c739e 100644 --- a/hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/jquery.js +++ b/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; diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/ReservationClientUtil.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/ReservationClientUtil.java new file mode 100644 index 00000000000..7c10a57b1df --- /dev/null +++ b/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 listResReq = new ArrayList(); + 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; + } +} diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/RumenToSLSConverter.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/RumenToSLSConverter.java index 0d0745c4137..76bcb157188 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/RumenToSLSConverter.java +++ b/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 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); diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java index f40f47d11d3..f999dce0ac7 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java +++ b/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 queueAppNumMap; - + // NM simulator private HashMap nmMap; private int nmMemoryMB, nmVCores; private String nodeFile; - + // AM simulator private int AM_ID; private Map amMap; @@ -99,49 +116,92 @@ public class SLSRunner { // other simulation information private int numNMs, numRacks, numAMs, numTasks; private long maxRuntime; - public final static Map simulateInfoMap = + + private final static Map simulateInfoMap = new HashMap(); // 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 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(); - queueAppNumMap = new HashMap(); - amMap = new HashMap(); - amClassMap = new HashMap(); - + 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); // 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 getSimulateInfoMap() { + return Collections.unmodifiableMap(simulateInfoMap); + } + + public void setSimulationParams(TraceType inType, String[] inTraces, + String nodes, String outDir, Set 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 nodeSet = new HashSet(); 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 rackSet = new HashSet(); 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 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()); + try (Reader input = new InputStreamReader( + new FileInputStream(inputTrace), "UTF-8")) { + Iterator jobIter = mapper.readValues( + jsonF.createParser(input), Map.class); - 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 containerList = - new ArrayList(); - 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); - } + 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 getTaskContainers(Map jsonJob) + throws YarnException { + List 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:///"); - 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; - } + File fin = new File(inputTrace); - boolean isTracked = trackedApps.contains(oldJobId); - int queueSize = queueAppNumMap.containsKey(jobQueue) ? - queueAppNumMap.get(jobQueue) : 0; - queueSize ++; - queueAppNumMap.put(jobQueue, queueSize); + try (JobTraceReader reader = new JobTraceReader( + new Path(fin.getAbsolutePath()), conf)) { + LoggedJob job = reader.getNext(); - List containerList = - new ArrayList(); - // 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); - } + while (job != null) { + try { + createAMForJob(job, baselineTimeMS); + } catch (Exception e) { + LOG.error("Failed to create an AM: {}", e.getMessage()); } - } finally { - reader.close(); + + 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 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; + + // 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 containerList = + new ArrayList(); + ArrayList keyAsArray = new ArrayList(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()); + + } + + 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 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 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"); - - CommandLineParser parser = new GnuParser(); - CommandLine cmd = parser.parse(options, args); + "print out simulation information"); + + CommandLineParser parser = new GnuParser(); + 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 trackedJobSet = new HashSet(); 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(); + } + } diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/AMSimulator.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/AMSimulator.java index e536cb64bed..61711545a14 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/AMSimulator.java +++ b/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 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(); + this.responseQueue = new LinkedBlockingQueue<>(); } - public void init(int id, int heartbeatInterval, - List 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 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; + + ReservationId reservationId = null; + + // 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(); - - // register application master - registerAM(); + 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() { + @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 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()); + conLauContext + .setApplicationACLs(new HashMap()); conLauContext.setCommands(new ArrayList()); conLauContext.setEnvironment(new HashMap()); conLauContext.setLocalResources(new HashMap()); conLauContext.setServiceData(new HashMap()); 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() { @@ -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 nodeLocalRequestMap = new HashMap(); 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; + } } diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/MRAMSimulator.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/MRAMSimulator.java index 5d005df1ee0..21bf05402b0 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/MRAMSimulator.java +++ b/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 pendingMaps = - new LinkedList(); + new LinkedList<>(); // pending failed maps private LinkedList 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 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 ask = new ArrayList(); - 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 token = rm.getRMContext().getRMApps() - .get(appAttemptId.getApplicationId()) - .getRMAppAttempt(appAttemptId).getAMRMToken(); - ugi.addTokenIdentifier(token.decodeIdentifier()); - AllocateResponse response = ugi.doAs( - new PrivilegedExceptionAction() { - @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 mergeLists(List left, List right) { + List 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 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(); + ask = new ArrayList<>(); } final AllocateRequest request = createAllocateRequest(ask); diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/conf/SLSConfiguration.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/conf/SLSConfiguration.java index 8fd5b3f770a..038f2021eed 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/conf/SLSConfiguration.java +++ b/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)); + } } diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NMSimulator.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NMSimulator.java index fb1c1f4f87b..9197b1ecef6 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NMSimulator.java +++ b/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; diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/resourcemanager/MockAMLauncher.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/resourcemanager/MockAMLauncher.java new file mode 100644 index 00000000000..b4ffb617c65 --- /dev/null +++ b/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 + *

+ * 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.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 { + private static final Log LOG = LogFactory.getLog( + MockAMLauncher.class); + + Map amMap; + SLSRunner se; + + public MockAMLauncher(SLSRunner se, RMContext rmContext, + Map 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 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); + } + } +} diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/FairSchedulerMetrics.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/FairSchedulerMetrics.java index 3b539fa6be4..420a1c95fea 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/FairSchedulerMetrics.java +++ b/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"); + } } - + + 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() { + @Override + public Long getValue() { + return getMemorySize((FSAppAttempt)getSchedulerAppAttempt(appId), + metric); + } + } + ); + + metrics.register( + "variable.app." + oldAppId + "." + metric.value + ".vcores", + new Gauge() { + @Override + public Integer getValue() { + return getVirtualCores((FSAppAttempt)getSchedulerAppAttempt(appId), + metric); + } + } + ); + } + @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() { - @Override - public Long getValue() { - return app.getDemand().getMemorySize(); + 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() { + @Override + public Long getValue() { + return getMemorySize(queue, metric); + } } - } ); - metrics.register("variable.app." + oldAppId + ".demand.vcores", - new Gauge() { - @Override - public Integer getValue() { - return app.getDemand().getVirtualCores(); + metrics.register( + "variable.queue." + queue.getName() + "." + metric.value + ".vcores", + new Gauge() { + @Override + public Integer getValue() { + return getVirtualCores(queue, metric); + } } - } - ); - metrics.register("variable.app." + oldAppId + ".usage.memory", - new Gauge() { - @Override - public Long getValue() { - return app.getResourceUsage().getMemorySize(); - } - } - ); - metrics.register("variable.app." + oldAppId + ".usage.vcores", - new Gauge() { - @Override - public Integer getValue() { - return app.getResourceUsage().getVirtualCores(); - } - } - ); - metrics.register("variable.app." + oldAppId + ".minshare.memory", - new Gauge() { - @Override - public Long getValue() { - return app.getMinShare().getMemorySize(); - } - } - ); - metrics.register("variable.app." + oldAppId + ".minshare.vcores", - new Gauge() { - @Override - public Long getValue() { - return app.getMinShare().getMemorySize(); - } - } - ); - metrics.register("variable.app." + oldAppId + ".maxshare.memory", - new Gauge() { - @Override - public Long getValue() { - return Math.min(app.getMaxShare().getMemorySize(), totalMemoryMB); - } - } - ); - metrics.register("variable.app." + oldAppId + ".maxshare.vcores", - new Gauge() { - @Override - public Integer getValue() { - return Math.min(app.getMaxShare().getVirtualCores(), totalVCores); - } - } - ); - metrics.register("variable.app." + oldAppId + ".fairshare.memory", - new Gauge() { - @Override - public Integer getValue() { - return app.getFairShare().getVirtualCores(); - } - } - ); - metrics.register("variable.app." + oldAppId + ".fairshare.vcores", - new Gauge() { - @Override - public Integer getValue() { - return app.getFairShare().getVirtualCores(); - } - } ); } @@ -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() { - @Override - public Long getValue() { - return queue.getDemand().getMemorySize(); - } - } - ); - metrics.register("variable.queue." + queueName + ".demand.vcores", - new Gauge() { - @Override - public Integer getValue() { - return queue.getDemand().getVirtualCores(); - } - } - ); - metrics.register("variable.queue." + queueName + ".usage.memory", - new Gauge() { - @Override - public Long getValue() { - return queue.getResourceUsage().getMemorySize(); - } - } - ); - metrics.register("variable.queue." + queueName + ".usage.vcores", - new Gauge() { - @Override - public Integer getValue() { - return queue.getResourceUsage().getVirtualCores(); - } - } - ); - metrics.register("variable.queue." + queueName + ".minshare.memory", - new Gauge() { - @Override - public Long getValue() { - return queue.getMinShare().getMemorySize(); - } - } - ); - metrics.register("variable.queue." + queueName + ".minshare.vcores", - new Gauge() { - @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() { @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() { - @Override - public Long getValue() { - return queue.getFairShare().getMemorySize(); - } - } - ); - metrics.register("variable.queue." + queueName + ".fairshare.vcores", - new Gauge() { - @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"); + } } } diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java deleted file mode 100644 index 1ba6acc5ebe..00000000000 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java +++ /dev/null @@ -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 - 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 schedulerHandleCounterMap; - // Timers for scheduler allocate/handle operations - private Timer schedulerAllocateTimer; - private Timer schedulerHandleTimer; - private Map schedulerHandleTimerMap; - private List schedulerHistogramList; - private Map histogramTimerMap; - private Lock samplerLock; - private Lock queueLock; - - private Configuration conf; - private ResourceScheduler scheduler; - private Map appQueueMap = - new ConcurrentHashMap(); - private BufferedWriter jobRuntimeLogBW; - - // Priority of the ResourceSchedulerWrapper shutdown hook. - public static final int SHUTDOWN_HOOK_PRIORITY = 30; - - // web app - private SLSWebApp web; - - private Map preemptionContainerMap = - new ConcurrentHashMap(); - - // metrics - private MetricRegistry metrics; - private SchedulerMetrics schedulerMetrics; - private boolean metricsON; - private String metricsOutputDir; - private BufferedWriter metricsLogBW; - private boolean running = false; - private static Map defaultSchedulerMetricsMap = - new HashMap(); - 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 queueSet; - private Set 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 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 resourceRequests, List containerIds, - List strings, List 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 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 resourceRequests, - List 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 preemptionContainers = new HashSet(); - 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 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() { - @Override - public Long getValue() { - return Runtime.getRuntime().freeMemory(); - } - } - ); - metrics.register("variable.jvm.max.memory", - new Gauge() { - @Override - public Long getValue() { - return Runtime.getRuntime().maxMemory(); - } - } - ); - metrics.register("variable.jvm.total.memory", - new Gauge() { - @Override - public Long getValue() { - return Runtime.getRuntime().totalMemory(); - } - } - ); - } - - private void registerClusterResourceMetrics() { - metrics.register("variable.cluster.allocated.memory", - new Gauge() { - @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() { - @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() { - @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() { - @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() { - @Override - public Integer getValue() { - if (scheduler == null || scheduler.getRootQueueMetrics() == null) { - return 0; - } else { - return scheduler.getRootQueueMetrics().getAppsRunning(); - } - } - } - ); - metrics.register("variable.running.container", - new Gauge() { - @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(); - 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); - } - - 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 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 queues) { - this.queueSet = queues; - } - - public Set getQueueSet() { - return this.queueSet; - } - - public void setTrackedAppSet(Set apps) { - this.trackedAppSet = apps; - } - - public Set 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) - scheduler).init(conf); - super.serviceInit(conf); - initScheduler(conf); - } - - private synchronized void initScheduler(Configuration configuration) throws - IOException { - this.applications = - new ConcurrentHashMap>(); - } - - @SuppressWarnings("unchecked") - @Override - public void serviceStart() throws Exception { - ((AbstractYarnScheduler) - scheduler).start(); - super.serviceStart(); - } - - @SuppressWarnings("unchecked") - @Override - public void serviceStop() throws Exception { - ((AbstractYarnScheduler) - 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 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 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 getTransferredContainers( - ApplicationAttemptId currentAttempt) { - return new ArrayList(); - } - - @Override - public Map> - getSchedulerApplications() { - return new HashMap>(); - } - - @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); - } - -} diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java index 2625cb76f6f..108c2bc8b1e 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java +++ b/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 schedulerHandleCounterMap; - // Timers for scheduler allocate/handle operations - private Timer schedulerAllocateTimer; - private Timer schedulerHandleTimer; - private Map schedulerHandleTimerMap; - private List schedulerHistogramList; - private Map histogramTimerMap; - private Lock samplerLock; - private Lock queueLock; private Configuration conf; - + private Map appQueueMap = new ConcurrentHashMap(); - private BufferedWriter jobRuntimeLogBW; - - // Priority of the ResourceSchedulerWrapper shutdown hook. - public static final int SHUTDOWN_HOOK_PRIORITY = 30; - - // web app - private SLSWebApp web; private Map preemptionContainerMap = new ConcurrentHashMap(); // metrics - private MetricRegistry metrics; private SchedulerMetrics schedulerMetrics; private boolean metricsON; - private String metricsOutputDir; - private BufferedWriter metricsLogBW; - private boolean running = false; - private static Map defaultSchedulerMetricsMap = - new HashMap(); - 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 queueSet; - private Set 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 resourceRequests, - List containerIds, - List strings, List strings2, - ContainerUpdates updateRequests) { + List resourceRequests, List containerIds, + List strings, List 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; + if (!metricsON) { + super.handle(schedulerEvent); + return; + } - // metrics on - Timer.Context handlerTimer = null; - Timer.Context operationTimer = null; + if (!schedulerMetrics.isRunning()) { + schedulerMetrics.setRunning(true); + } - 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()); - } - } + Timer.Context handlerTimer = null; + Timer.Context operationTimer = null; - handlerTimer = schedulerHandleTimer.time(); - operationTimer = schedulerHandleTimerMap - .get(schedulerEvent.getType()).time(); + 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); + } + } - super.handle(schedulerEvent); - } finally { - if (handlerTimer != null) handlerTimer.stop(); - if (operationTimer != null) operationTimer.stop(); - schedulerHandleCounter.inc(); - schedulerHandleCounterMap.get(schedulerEvent.getType()).inc(); + handlerTimer = schedulerMetrics.getSchedulerHandleTimer().time(); + operationTimer = schedulerMetrics.getSchedulerHandleTimer( + schedulerEvent.getType()).time(); - 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()); - } - } + 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 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]); + schedulerMetrics.updateQueueMetrics(pendingResource, allocatedResource, + queueName); + } + + private void initQueueMetrics(CSQueue queue) { + if (queue instanceof LeafQueue) { + schedulerMetrics.initQueueMetric(queue.getQueueName()); + return; } - queueLock.lock(); + for (CSQueue child : queue.getChildQueues()) { + initQueueMetrics(child); + } + } + @Override + public void serviceInit(Configuration configuration) throws Exception { + super.serviceInit(configuration); + + if (metricsON) { + initQueueMetrics(getRootQueue()); + } + } + + @Override + public void serviceStop() throws Exception { 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() { - @Override - public Long getValue() { - return Runtime.getRuntime().freeMemory(); - } - } - ); - metrics.register("variable.jvm.max.memory", - new Gauge() { - @Override - public Long getValue() { - return Runtime.getRuntime().maxMemory(); - } - } - ); - metrics.register("variable.jvm.total.memory", - new Gauge() { - @Override - public Long getValue() { - return Runtime.getRuntime().totalMemory(); - } - } - ); - } - - private void registerClusterResourceMetrics() { - metrics.register("variable.cluster.allocated.memory", - new Gauge() { - @Override - public Long getValue() { - if( getRootQueueMetrics() == null) { - return 0L; - } else { - return getRootQueueMetrics().getAllocatedMB(); - } - } - } - ); - metrics.register("variable.cluster.allocated.vcores", - new Gauge() { - @Override - public Integer getValue() { - if(getRootQueueMetrics() == null) { - return 0; - } else { - return getRootQueueMetrics().getAllocatedVirtualCores(); - } - } - } - ); - metrics.register("variable.cluster.available.memory", - new Gauge() { - @Override - public Long getValue() { - if(getRootQueueMetrics() == null) { - return 0L; - } else { - return getRootQueueMetrics().getAvailableMB(); - } - } - } - ); - metrics.register("variable.cluster.available.vcores", - new Gauge() { - @Override - public Integer getValue() { - if(getRootQueueMetrics() == null) { - return 0; - } else { - return getRootQueueMetrics().getAvailableVirtualCores(); - } - } - } - ); - } - - private void registerContainerAppNumMetrics() { - metrics.register("variable.running.application", - new Gauge() { - @Override - public Integer getValue() { - if(getRootQueueMetrics() == null) { - return 0; - } else { - return getRootQueueMetrics().getAppsRunning(); - } - } - } - ); - metrics.register("variable.running.container", - new Gauge() { - @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(); - 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); - } - - 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(); - } - } - } - } - - // the following functions are used by AMSimulator - 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) { + schedulerMetrics.tearDown(); + } catch (Exception e) { e.printStackTrace(); } + super.serviceStop(); } - private void updateQueueMetrics(String queue, - int releasedMemory, int releasedVCores) { - // update queue counters - SortedMap 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 queues) { - this.queueSet = queues; - } - - public Set getQueueSet() { - return this.queueSet; - } - - public void setTrackedAppSet(Set apps) { - this.trackedAppSet = apps; - } - - public Set 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(); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSFairScheduler.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSFairScheduler.java new file mode 100644 index 00000000000..81f66488cdc --- /dev/null +++ b/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 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 resourceRequests, List containerIds, + List blacklistAdditions, List 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 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 resourceRequests, + List 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 preemptionContainers = new HashSet(); + 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(); + } +} + diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerMetrics.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerMetrics.java index ecf516d7c98..3c7aa7759a3 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerMetrics.java +++ b/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 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.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.Gauge; -import com.codahale.metrics.MetricRegistry; +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 trackedQueues; protected MetricRegistry metrics; protected Set appTrackedMetrics; protected Set 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 schedulerHandleCounterMap; + + // Timers for scheduler allocate/handle operations + private Timer schedulerAllocateTimer; + private Timer schedulerHandleTimer; + private Map schedulerHandleTimerMap; + private List schedulerHistogramList; + private Map 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(); + metrics = new MetricRegistry(); + + appTrackedMetrics = new HashSet<>(); appTrackedMetrics.add("live.containers"); appTrackedMetrics.add("reserved.containers"); - queueTrackedMetrics = new HashSet(); + + 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(); - 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() { - @Override - public Integer getValue() { - SchedulerAppReport app = scheduler.getSchedulerAppInfo(appAttemptId); - return app.getLiveContainers().size(); + new Gauge() { + @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() { - @Override - public Integer getValue() { - SchedulerAppReport app = scheduler.getSchedulerAppInfo(appAttemptId); - return app.getReservedContainers().size(); + new Gauge() { + @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 getAppTrackedMetrics() { return appTrackedMetrics; } + public Set getQueueTrackedMetrics() { return queueTrackedMetrics; } + + private void registerJvmMetrics() { + // add JVM gauges + metrics.register("variable.jvm.free.memory", + new Gauge() { + @Override + public Long getValue() { + return Runtime.getRuntime().freeMemory(); + } + } + ); + metrics.register("variable.jvm.max.memory", + new Gauge() { + @Override + public Long getValue() { + return Runtime.getRuntime().maxMemory(); + } + } + ); + metrics.register("variable.jvm.total.memory", + new Gauge() { + @Override + public Long getValue() { + return Runtime.getRuntime().totalMemory(); + } + } + ); + } + + private void registerClusterResourceMetrics() { + metrics.register("variable.cluster.allocated.memory", + new Gauge() { + @Override + public Long getValue() { + if (scheduler.getRootQueueMetrics() == null) { + return 0L; + } else { + return scheduler.getRootQueueMetrics().getAllocatedMB(); + } + } + } + ); + metrics.register("variable.cluster.allocated.vcores", + new Gauge() { + @Override + public Integer getValue() { + if (scheduler.getRootQueueMetrics() == null) { + return 0; + } else { + return scheduler.getRootQueueMetrics().getAllocatedVirtualCores(); + } + } + } + ); + metrics.register("variable.cluster.available.memory", + new Gauge() { + @Override + public Long getValue() { + if (scheduler.getRootQueueMetrics() == null) { + return 0L; + } else { + return scheduler.getRootQueueMetrics().getAvailableMB(); + } + } + } + ); + metrics.register("variable.cluster.available.vcores", + new Gauge() { + @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() { + @Override + public Integer getValue() { + if (scheduler.getRootQueueMetrics() == null) { + return 0; + } else { + return scheduler.getRootQueueMetrics().getAppsRunning(); + } + } + } + ); + metrics.register("variable.running.container", + new Gauge() { + @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 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 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 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()); + } + } } diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerWrapper.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerWrapper.java index 44629f5347f..7112b1a6fd5 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerWrapper.java +++ b/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 getQueueSet(); - public void setQueueSet(Set queues); - public Set getTrackedAppSet(); - public void setTrackedAppSet(Set 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; } diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/TaskRunner.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/TaskRunner.java index d35290428c7..19cfe88d1ab 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/TaskRunner.java +++ b/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") diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/Tracker.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/Tracker.java new file mode 100644 index 00000000000..42a5c3c894d --- /dev/null +++ b/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 queueSet; + private Set trackedAppSet; + + public void setQueueSet(Set queues) { + queueSet = queues; + } + + public Set getQueueSet() { + return queueSet; + } + + public void setTrackedAppSet(Set apps) { + trackedAppSet = apps; + } + + public Set getTrackedAppSet() { + return trackedAppSet; + } +} diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthJob.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthJob.java new file mode 100644 index 00000000000..3ed81e1f2cd --- /dev/null +++ b/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; + } +} diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthJobClass.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthJobClass.java new file mode 100644 index 00000000000..439698f8a45 --- /dev/null +++ b/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; + } +} diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthTraceJobProducer.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthTraceJobProducer.java new file mode 100644 index 00000000000..14b0371111d --- /dev/null +++ b/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 + *

+ * 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.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 weightList; + private final Map workloads; + + private final Queue 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(); + weightList = new ArrayList(); + + 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() { + @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 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 job_classes; + @JsonProperty("time_distribution") + List 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; + } + +} diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthUtils.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthUtils.java new file mode 100644 index 00000000000..a7f8c7f1933 --- /dev/null +++ b/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 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; + } +} diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthWorkload.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthWorkload.java new file mode 100644 index 00000000000..9e5fd4ef742 --- /dev/null +++ b/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 classList; + private final Trace trace; + private final SortedMap timeWeights; + + public SynthWorkload(int identifier, Trace inTrace) { + classList = new ArrayList(); + this.id = identifier; + this.trace = inTrace; + timeWeights = new TreeMap(); + 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 getWeightList() { + ArrayList ret = new ArrayList(); + 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 getClassList() { + return classList; + } + +} diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/package-info.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/package-info.java new file mode 100644 index 00000000000..e06961069df --- /dev/null +++ b/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; \ No newline at end of file diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/utils/SLSUtils.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/utils/SLSUtils.java index f1b4f078028..dbc2dab4b5e 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/utils/SLSUtils.java +++ b/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 parseNodesFromSLSTrace(String jobTrace) throws IOException { - Set nodeSet = new HashSet(); + Set nodeSet = new HashSet<>(); JsonFactory jsonF = new JsonFactory(); ObjectMapper mapper = new ObjectMapper(); Reader input = new InputStreamReader(new FileInputStream(jobTrace), "UTF-8"); try { - Iterator i = mapper.readValues( - jsonF.createJsonParser(input), Map.class); + Iterator 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 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 i = mapper.readValues( - jsonF.createJsonParser(input), Map.class); + Iterator 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 generateNodes(int numNodes, + int numRacks){ + Set 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; + } } diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/web/SLSWebApp.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/web/SLSWebApp.java index e1526965db6..154bcc9a994 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/web/SLSWebApp.java +++ b/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(); - queueAllocatedMemoryCounterMap = new HashMap(); - queueAllocatedVCoresCounterMap = new HashMap(); + 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 = "" + "No information available"; simulateInfo = MessageFormat.format(simulateInfoTemplate, empty); } else { StringBuilder info = new StringBuilder(); for (Map.Entry entry : - SLSRunner.simulateInfoMap.entrySet()) { + SLSRunner.getSimulateInfoMap().entrySet()) { info.append(""); info.append("").append(entry.getKey()).append(""); info.append("").append(entry.getValue()) @@ -221,7 +220,7 @@ public class SLSWebApp extends HttpServlet { response.setStatus(HttpServletResponse.SC_OK); // queues {0} - Set queues = wrapper.getQueueSet(); + Set 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 trackedQueues = wrapper.getQueueSet(); + Set trackedQueues = wrapper.getTracker().getQueueSet(); for(String queue : trackedQueues) { trackedQueueInfo.append(""); @@ -268,7 +267,7 @@ public class SLSWebApp extends HttpServlet { // tracked apps {1} StringBuilder trackedAppInfo = new StringBuilder(); - Set trackedApps = wrapper.getTrackedAppSet(); + Set trackedApps = wrapper.getTracker().getTrackedAppSet(); for(String job : trackedApps) { trackedAppInfo.append(""); @@ -417,7 +416,7 @@ public class SLSWebApp extends HttpServlet { // allocated resource for each queue Map queueAllocatedMemoryMap = new HashMap(); Map queueAllocatedVCoresMap = new HashMap(); - 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\":") diff --git a/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md b/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md index 2cffc863026..d1848e89759 100644 --- a/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md +++ b/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= - --output-dir= [--nodes=] - [--track-jobs=] [--print-simulation] + Usage: slsrun.sh + --tracetype= + --tracelocation= + (deprecated --input-rumen= | --input-sls=) + --output-dir= + [--nodes=] + [--track-jobs=] + [--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=`. * `--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=`. + +* `--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. diff --git a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/BaseSLSRunnerTest.java b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/BaseSLSRunnerTest.java new file mode 100644 index 00000000000..6b369f2a6f0 --- /dev/null +++ b/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 exceptionList = + Collections.synchronizedList(new ArrayList()); + + 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(); + } + } + +} diff --git a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestReservationSystemInvariants.java b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestReservationSystemInvariants.java new file mode 100644 index 00000000000..22e1e2e729e --- /dev/null +++ b/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 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() { + + } +} diff --git a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSLSRunner.java b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSLSRunner.java index 9da8ef34a20..5ab893d025c 100644 --- a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSLSRunner.java +++ b/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 + @Parameters(name = "Testing with: {1}, {0}, (nodeFile {3})") + public static Collection data() { + + 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[][] { + + // covering old commandline in tests + {capScheduler, "OLD_RUMEN", rumenTraceFile, nodeFile }, + {capScheduler, "OLD_SLS", slsTraceFile, nodeFile }, + + // covering the no nodeFile case + {capScheduler, "SYNTH", synthTraceFile, null }, + {capScheduler, "RUMEN", rumenTraceFile, null }, + {capScheduler, "SLS", slsTraceFile, null }, + + // covering new commandline and CapacityScheduler + {capScheduler, "SYNTH", synthTraceFile, nodeFile }, + {capScheduler, "RUMEN", rumenTraceFile, nodeFile }, + {capScheduler, "SLS", slsTraceFile, nodeFile }, + + // 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 { - File tempDir = new File("target", UUID.randomUUID().toString()); - final List exceptionList = - Collections.synchronizedList(new ArrayList()); - - Thread.setDefaultUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() { - @Override - public void uncaughtException(Thread t, Throwable e) { - exceptionList.add(e); - } - }); - - // 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); - - // wait for 20 seconds before stop - int count = 20; - while (count >= 0) { - Thread.sleep(1000); - - if (! exceptionList.isEmpty()) { - SLSRunner.getRunner().stop(); - Assert.fail("TestSLSRunner catched exception from child thread " + - "(TaskRunner.Task): " + exceptionList.get(0).getMessage()); - break; - } - count--; - } - - SLSRunner.getRunner().stop(); + Configuration conf = new Configuration(false); + long timeTillShutdownInsec = 20L; + runSLS(conf, timeTillShutdownInsec); } } diff --git a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSynthJobGeneration.java b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSynthJobGeneration.java new file mode 100644 index 00000000000..2b1971a8ec5 --- /dev/null +++ b/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()); + } + + } +} diff --git a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/appmaster/TestAMSimulator.java b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/appmaster/TestAMSimulator.java index fd1c8615f05..02dc26eeaf9 100644 --- a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/appmaster/TestAMSimulator.java +++ b/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 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 containers = new ArrayList(); - app.init(1, 1000, containers, rm, null, 0, 1000000l, "user1", "default", - false, "app1"); + String appId = "app1"; + String queue = "default"; + List 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(); } -} +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/nodemanager/TestNMSimulator.java b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/nodemanager/TestNMSimulator.java index f9a393298eb..2f10f7dbc2e 100644 --- a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/nodemanager/TestNMSimulator.java +++ b/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 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); diff --git a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/scheduler/TestTaskRunner.java b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/scheduler/TestTaskRunner.java index 23f2bb61677..ce6c1b30b65 100644 --- a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/scheduler/TestTaskRunner.java +++ b/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(); } diff --git a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/utils/TestSLSUtils.java b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/utils/TestSLSUtils.java index f4eda679586..30964a1bceb 100644 --- a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/utils/TestSLSUtils.java +++ b/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 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 nodes) { + Set racks = new HashSet<>(); + for (String node : nodes) { + String[] rackHostname = SLSUtils.getRackHostName(node); + racks.add(rackHostname[0]); + } + return racks.size(); + } } diff --git a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/web/TestSLSWebApp.java b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/web/TestSLSWebApp.java index 1c1e63cfd2c..c9be450fee2 100644 --- a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/web/TestSLSWebApp.java +++ b/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 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 entry : - SLSRunner.simulateInfoMap.entrySet()) { + simulateInfoMap.entrySet()) { info.append(""); info.append("" + entry.getKey() + ""); info.append("" + entry.getValue() + ""); @@ -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 entry : - SLSRunner.simulateInfoMap.entrySet()) { + for (Map.Entry entry : simulateInfoMap.entrySet()) { Assert.assertTrue("The simulate info html page should have information " + "of " + entry.getKey(), simulateInfo.contains("" + entry.getKey() + "" diff --git a/hadoop-tools/hadoop-sls/src/test/resources/capacity-scheduler.xml b/hadoop-tools/hadoop-sls/src/test/resources/capacity-scheduler.xml index 61be96ae6d4..1762265f6d8 100644 --- a/hadoop-tools/hadoop-sls/src/test/resources/capacity-scheduler.xml +++ b/hadoop-tools/hadoop-sls/src/test/resources/capacity-scheduler.xml @@ -38,6 +38,16 @@ 100 + + yarn.scheduler.capacity.root.sls_queue_1.reservable + true + + + + yarn.scheduler.capacity.root.sls_queue_1.show-reservations-as-queues + true + + yarn.scheduler.capacity.root.sls_queue_2.capacity 25 diff --git a/hadoop-tools/hadoop-sls/src/test/resources/exit-invariants.txt b/hadoop-tools/hadoop-sls/src/test/resources/exit-invariants.txt new file mode 100644 index 00000000000..b4a3228be51 --- /dev/null +++ b/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 diff --git a/hadoop-tools/hadoop-sls/src/test/resources/fair-scheduler.xml b/hadoop-tools/hadoop-sls/src/test/resources/fair-scheduler.xml index fa10359c501..7c46767737f 100644 --- a/hadoop-tools/hadoop-sls/src/test/resources/fair-scheduler.xml +++ b/hadoop-tools/hadoop-sls/src/test/resources/fair-scheduler.xml @@ -21,6 +21,7 @@ --> + drf yarn.sls.nm.memory.mb - 10240 + 100240 yarn.sls.nm.vcores - 10 + 100 yarn.sls.nm.heartbeat.interval.ms @@ -77,5 +77,5 @@ org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler org.apache.hadoop.yarn.sls.scheduler.CapacitySchedulerMetrics - + diff --git a/hadoop-tools/hadoop-sls/src/test/resources/syn.json b/hadoop-tools/hadoop-sls/src/test/resources/syn.json new file mode 100644 index 00000000000..8479d23c318 --- /dev/null +++ b/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 + } + ] + } + ] +} diff --git a/hadoop-tools/hadoop-sls/src/test/resources/yarn-site.xml b/hadoop-tools/hadoop-sls/src/test/resources/yarn-site.xml index c9f714c26ab..282aef39360 100644 --- a/hadoop-tools/hadoop-sls/src/test/resources/yarn-site.xml +++ b/hadoop-tools/hadoop-sls/src/test/resources/yarn-site.xml @@ -17,7 +17,7 @@ yarn.resourcemanager.scheduler.class - org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler + org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler @@ -69,4 +69,21 @@ yarn.scheduler.fair.assignmultiple true + + + + Enable reservation system. + yarn.resourcemanager.reservation-system.enable + true + + + + yarn.nodemanager.resource.memory-mb + 1000000 + + + yarn.nodemanager.resource.cpu-vcores + 320 + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml index 85df2c0d784..7061887c498 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml @@ -340,6 +340,7 @@ src/test/resources/submit-reservation.json src/test/resources/delete-reservation.json src/test/resources/update-reservation.json + src/test/resources/invariants.txt diff --git a/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 b/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 new file mode 100644 index 00000000000..0491756b964 --- /dev/null +++ b/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); + } +} diff --git a/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 b/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 new file mode 100644 index 00000000000..2c9031fe827 --- /dev/null +++ b/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; + } + +} diff --git a/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 b/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 new file mode 100644 index 00000000000..849cbf92569 --- /dev/null +++ b/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 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 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 e : invariants.entrySet()) { + boolean invariantsHold = (boolean) e.getValue().eval(bindings); + if (!invariantsHold) { + // filter bindings to produce minimal set + Map matchingBindings = + extractMatchingBindings(e.getKey(), bindings); + logOrThrow("Invariant \"" + e.getKey() + + "\" is NOT holding, with bindings: " + matchingBindings); + } + } + } + } catch (ScriptException e) { + logOrThrow(e.getMessage()); + } + } + + private static Map extractMatchingBindings(String inv, + SimpleBindings allBindings) { + Map matchingBindings = new HashMap<>(); + for (Map.Entry s : allBindings.entrySet()) { + if (inv.contains(s.getKey())) { + matchingBindings.put(s.getKey(), s.getValue()); + } + } + return matchingBindings; + } +} diff --git a/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 b/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 new file mode 100644 index 00000000000..2f9f03e056b --- /dev/null +++ b/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 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); + } + + } + +} diff --git a/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 b/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 new file mode 100644 index 00000000000..d9931d67775 --- /dev/null +++ b/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; \ No newline at end of file diff --git a/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 b/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 new file mode 100644 index 00000000000..35cf1e4e53d --- /dev/null +++ b/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(); + + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/invariants.txt b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/invariants.txt new file mode 100644 index 00000000000..363ed0d996e --- /dev/null +++ b/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