|
@@ -41,17 +41,25 @@ import org.apache.commons.cli.CommandLine;
|
|
import org.apache.commons.cli.CommandLineParser;
|
|
import org.apache.commons.cli.CommandLineParser;
|
|
import org.apache.commons.cli.GnuParser;
|
|
import org.apache.commons.cli.GnuParser;
|
|
import org.apache.commons.cli.Options;
|
|
import org.apache.commons.cli.Options;
|
|
|
|
+import org.apache.commons.cli.ParseException;
|
|
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
|
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
|
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
|
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
|
+import org.apache.hadoop.conf.Configured;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
|
|
+import org.apache.hadoop.mapreduce.TaskType;
|
|
import org.apache.hadoop.tools.rumen.JobTraceReader;
|
|
import org.apache.hadoop.tools.rumen.JobTraceReader;
|
|
import org.apache.hadoop.tools.rumen.LoggedJob;
|
|
import org.apache.hadoop.tools.rumen.LoggedJob;
|
|
import org.apache.hadoop.tools.rumen.LoggedTask;
|
|
import org.apache.hadoop.tools.rumen.LoggedTask;
|
|
import org.apache.hadoop.tools.rumen.LoggedTaskAttempt;
|
|
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.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.NodeId;
|
|
import org.apache.hadoop.yarn.api.records.NodeState;
|
|
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.api.records.Resource;
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
@@ -67,25 +75,27 @@ import org.apache.hadoop.yarn.sls.conf.SLSConfiguration;
|
|
import org.apache.hadoop.yarn.sls.nodemanager.NMSimulator;
|
|
import org.apache.hadoop.yarn.sls.nodemanager.NMSimulator;
|
|
import org.apache.hadoop.yarn.sls.resourcemanager.MockAMLauncher;
|
|
import org.apache.hadoop.yarn.sls.resourcemanager.MockAMLauncher;
|
|
import org.apache.hadoop.yarn.sls.scheduler.*;
|
|
import org.apache.hadoop.yarn.sls.scheduler.*;
|
|
|
|
+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.sls.utils.SLSUtils;
|
|
|
|
+import org.apache.hadoop.yarn.util.UTCClock;
|
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
|
import org.apache.log4j.Logger;
|
|
import org.apache.log4j.Logger;
|
|
|
|
|
|
@Private
|
|
@Private
|
|
@Unstable
|
|
@Unstable
|
|
-public class SLSRunner {
|
|
|
|
|
|
+public class SLSRunner extends Configured implements Tool {
|
|
// RM, Runner
|
|
// RM, Runner
|
|
private ResourceManager rm;
|
|
private ResourceManager rm;
|
|
private static TaskRunner runner = new TaskRunner();
|
|
private static TaskRunner runner = new TaskRunner();
|
|
private String[] inputTraces;
|
|
private String[] inputTraces;
|
|
- private Configuration conf;
|
|
|
|
private Map<String, Integer> queueAppNumMap;
|
|
private Map<String, Integer> queueAppNumMap;
|
|
-
|
|
|
|
|
|
+
|
|
// NM simulator
|
|
// NM simulator
|
|
private HashMap<NodeId, NMSimulator> nmMap;
|
|
private HashMap<NodeId, NMSimulator> nmMap;
|
|
private int nmMemoryMB, nmVCores;
|
|
private int nmMemoryMB, nmVCores;
|
|
private String nodeFile;
|
|
private String nodeFile;
|
|
-
|
|
|
|
|
|
+
|
|
// AM simulator
|
|
// AM simulator
|
|
private int AM_ID;
|
|
private int AM_ID;
|
|
private Map<String, AMSimulator> amMap;
|
|
private Map<String, AMSimulator> amMap;
|
|
@@ -106,43 +116,64 @@ public class SLSRunner {
|
|
// logger
|
|
// logger
|
|
public final static Logger LOG = Logger.getLogger(SLSRunner.class);
|
|
public final static Logger LOG = Logger.getLogger(SLSRunner.class);
|
|
|
|
|
|
- // input traces, input-rumen or input-sls
|
|
|
|
- private boolean isSLS;
|
|
|
|
-
|
|
|
|
- public SLSRunner(boolean isSLS, String inputTraces[], String nodeFile,
|
|
|
|
- String outputDir, Set<String> trackedApps,
|
|
|
|
- boolean printsimulation)
|
|
|
|
- throws IOException, ClassNotFoundException {
|
|
|
|
- this.isSLS = isSLS;
|
|
|
|
- this.inputTraces = inputTraces.clone();
|
|
|
|
- this.nodeFile = nodeFile;
|
|
|
|
- this.trackedApps = trackedApps;
|
|
|
|
- this.printSimulation = printsimulation;
|
|
|
|
- metricsOutputDir = outputDir;
|
|
|
|
-
|
|
|
|
|
|
+ /**
|
|
|
|
+ * 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);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void init(Configuration tempConf) throws ClassNotFoundException {
|
|
nmMap = new HashMap<>();
|
|
nmMap = new HashMap<>();
|
|
queueAppNumMap = new HashMap<>();
|
|
queueAppNumMap = new HashMap<>();
|
|
amMap = new ConcurrentHashMap<>();
|
|
amMap = new ConcurrentHashMap<>();
|
|
amClassMap = new HashMap<>();
|
|
amClassMap = new HashMap<>();
|
|
-
|
|
|
|
|
|
+
|
|
// runner configuration
|
|
// runner configuration
|
|
- conf = new Configuration(false);
|
|
|
|
- conf.addResource("sls-runner.xml");
|
|
|
|
|
|
+ tempConf.addResource("sls-runner.xml");
|
|
|
|
+ super.setConf(tempConf);
|
|
|
|
+
|
|
// runner
|
|
// 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);
|
|
SLSRunner.runner.setQueueSize(poolSize);
|
|
// <AMType, Class> map
|
|
// <AMType, Class> map
|
|
- for (Map.Entry e : conf) {
|
|
|
|
|
|
+ for (Map.Entry e : tempConf) {
|
|
String key = e.getKey().toString();
|
|
String key = e.getKey().toString();
|
|
if (key.startsWith(SLSConfiguration.AM_TYPE)) {
|
|
if (key.startsWith(SLSConfiguration.AM_TYPE)) {
|
|
String amType = key.substring(SLSConfiguration.AM_TYPE.length());
|
|
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 {
|
|
|
|
|
|
+
|
|
|
|
+ public void setSimulationParams(TraceType inType, String[] inTraces,
|
|
|
|
+ String nodes, String outDir, Set<String> trackApps,
|
|
|
|
+ boolean printsimulation) throws IOException, ClassNotFoundException {
|
|
|
|
+
|
|
|
|
+ this.inputType = inType;
|
|
|
|
+ this.inputTraces = inTraces.clone();
|
|
|
|
+ this.nodeFile = nodes;
|
|
|
|
+ this.trackedApps = trackApps;
|
|
|
|
+ this.printSimulation = printsimulation;
|
|
|
|
+ metricsOutputDir = outDir;
|
|
|
|
+
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public void start() throws IOException, ClassNotFoundException, YarnException,
|
|
|
|
+ InterruptedException {
|
|
// start resource manager
|
|
// start resource manager
|
|
startRM();
|
|
startRM();
|
|
// start node managers
|
|
// start node managers
|
|
@@ -151,9 +182,9 @@ public class SLSRunner {
|
|
startAM();
|
|
startAM();
|
|
// set queue & tracked apps information
|
|
// set queue & tracked apps information
|
|
((SchedulerWrapper) rm.getResourceScheduler()).getTracker()
|
|
((SchedulerWrapper) rm.getResourceScheduler()).getTracker()
|
|
- .setQueueSet(this.queueAppNumMap.keySet());
|
|
|
|
|
|
+ .setQueueSet(this.queueAppNumMap.keySet());
|
|
((SchedulerWrapper) rm.getResourceScheduler()).getTracker()
|
|
((SchedulerWrapper) rm.getResourceScheduler()).getTracker()
|
|
- .setTrackedAppSet(this.trackedApps);
|
|
|
|
|
|
+ .setTrackedAppSet(this.trackedApps);
|
|
// print out simulation info
|
|
// print out simulation info
|
|
printSimulationInfo();
|
|
printSimulationInfo();
|
|
// blocked until all nodes RUNNING
|
|
// blocked until all nodes RUNNING
|
|
@@ -162,23 +193,23 @@ public class SLSRunner {
|
|
runner.start();
|
|
runner.start();
|
|
}
|
|
}
|
|
|
|
|
|
- private void startRM() throws Exception {
|
|
|
|
- Configuration rmConf = new YarnConfiguration();
|
|
|
|
|
|
+ private void startRM() throws ClassNotFoundException, YarnException {
|
|
|
|
+ Configuration rmConf = new YarnConfiguration(getConf());
|
|
String schedulerClass = rmConf.get(YarnConfiguration.RM_SCHEDULER);
|
|
String schedulerClass = rmConf.get(YarnConfiguration.RM_SCHEDULER);
|
|
|
|
|
|
// For CapacityScheduler we use a sub-classing instead of wrapping
|
|
// For CapacityScheduler we use a sub-classing instead of wrapping
|
|
// to allow scheduler-specific invocations from monitors to work
|
|
// to allow scheduler-specific invocations from monitors to work
|
|
// this can be used for other schedulers as well if we care to
|
|
// this can be used for other schedulers as well if we care to
|
|
// exercise/track behaviors that are not common to the scheduler api
|
|
// 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,
|
|
rmConf.set(YarnConfiguration.RM_SCHEDULER,
|
|
SLSCapacityScheduler.class.getName());
|
|
SLSCapacityScheduler.class.getName());
|
|
} else if (Class.forName(schedulerClass) == FairScheduler.class) {
|
|
} else if (Class.forName(schedulerClass) == FairScheduler.class) {
|
|
rmConf.set(YarnConfiguration.RM_SCHEDULER,
|
|
rmConf.set(YarnConfiguration.RM_SCHEDULER,
|
|
SLSFairScheduler.class.getName());
|
|
SLSFairScheduler.class.getName());
|
|
- } else if (Class.forName(schedulerClass) == FifoScheduler.class){
|
|
|
|
|
|
+ } else if (Class.forName(schedulerClass) == FifoScheduler.class) {
|
|
// TODO add support for FifoScheduler
|
|
// TODO add support for FifoScheduler
|
|
- throw new Exception("Fifo Scheduler is not supported yet.");
|
|
|
|
|
|
+ throw new YarnException("Fifo Scheduler is not supported yet.");
|
|
}
|
|
}
|
|
|
|
|
|
rmConf.set(SLSConfiguration.METRICS_OUTPUT_DIR, metricsOutputDir);
|
|
rmConf.set(SLSConfiguration.METRICS_OUTPUT_DIR, metricsOutputDir);
|
|
@@ -196,37 +227,47 @@ public class SLSRunner {
|
|
|
|
|
|
private void startNM() throws YarnException, IOException {
|
|
private void startNM() throws YarnException, IOException {
|
|
// nm configuration
|
|
// 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);
|
|
SLSConfiguration.NM_HEARTBEAT_INTERVAL_MS_DEFAULT);
|
|
// nm information (fetch from topology file, or from sls/rumen json file)
|
|
// nm information (fetch from topology file, or from sls/rumen json file)
|
|
Set<String> nodeSet = new HashSet<String>();
|
|
Set<String> nodeSet = new HashSet<String>();
|
|
if (nodeFile.isEmpty()) {
|
|
if (nodeFile.isEmpty()) {
|
|
- if (isSLS) {
|
|
|
|
- for (String inputTrace : inputTraces) {
|
|
|
|
|
|
+ for (String inputTrace : inputTraces) {
|
|
|
|
+
|
|
|
|
+ switch (inputType) {
|
|
|
|
+ case SLS:
|
|
nodeSet.addAll(SLSUtils.parseNodesFromSLSTrace(inputTrace));
|
|
nodeSet.addAll(SLSUtils.parseNodesFromSLSTrace(inputTrace));
|
|
- }
|
|
|
|
- } else {
|
|
|
|
- for (String inputTrace : inputTraces) {
|
|
|
|
|
|
+ break;
|
|
|
|
+ case RUMEN:
|
|
nodeSet.addAll(SLSUtils.parseNodesFromRumenTrace(inputTrace));
|
|
nodeSet.addAll(SLSUtils.parseNodesFromRumenTrace(inputTrace));
|
|
|
|
+ break;
|
|
|
|
+ case SYNTH:
|
|
|
|
+ stjp = new SynthTraceJobProducer(getConf(), new Path(inputTraces[0]));
|
|
|
|
+ nodeSet.addAll(SLSUtils.generateNodesFromSynth(stjp.getNumNodes(),
|
|
|
|
+ stjp.getNodesPerRack()));
|
|
|
|
+ break;
|
|
|
|
+ default:
|
|
|
|
+ throw new YarnException("Input configuration not recognized, "
|
|
|
|
+ + "trace type should be SLS, RUMEN, or SYNTH");
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
} else {
|
|
} else {
|
|
nodeSet.addAll(SLSUtils.parseNodesFromNodeFile(nodeFile));
|
|
nodeSet.addAll(SLSUtils.parseNodesFromNodeFile(nodeFile));
|
|
}
|
|
}
|
|
|
|
+
|
|
// create NM simulators
|
|
// create NM simulators
|
|
Random random = new Random();
|
|
Random random = new Random();
|
|
Set<String> rackSet = new HashSet<String>();
|
|
Set<String> rackSet = new HashSet<String>();
|
|
for (String hostName : nodeSet) {
|
|
for (String hostName : nodeSet) {
|
|
// we randomize the heartbeat start time from zero to 1 interval
|
|
// we randomize the heartbeat start time from zero to 1 interval
|
|
NMSimulator nm = new NMSimulator();
|
|
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);
|
|
nmMap.put(nm.getNode().getNodeID(), nm);
|
|
runner.schedule(nm);
|
|
runner.schedule(nm);
|
|
rackSet.add(nm.getNode().getRackName());
|
|
rackSet.add(nm.getNode().getRackName());
|
|
@@ -241,39 +282,50 @@ public class SLSRunner {
|
|
int numRunningNodes = 0;
|
|
int numRunningNodes = 0;
|
|
for (RMNode node : rm.getRMContext().getRMNodes().values()) {
|
|
for (RMNode node : rm.getRMContext().getRMNodes().values()) {
|
|
if (node.getState() == NodeState.RUNNING) {
|
|
if (node.getState() == NodeState.RUNNING) {
|
|
- numRunningNodes ++;
|
|
|
|
|
|
+ numRunningNodes++;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (numRunningNodes == numNMs) {
|
|
if (numRunningNodes == numNMs) {
|
|
break;
|
|
break;
|
|
}
|
|
}
|
|
- LOG.info(MessageFormat.format("SLSRunner is waiting for all " +
|
|
|
|
- "nodes RUNNING. {0} of {1} NMs initialized.",
|
|
|
|
- numRunningNodes, numNMs));
|
|
|
|
|
|
+ LOG.info(MessageFormat.format(
|
|
|
|
+ "SLSRunner is waiting for all "
|
|
|
|
+ + "nodes RUNNING. {0} of {1} NMs initialized.",
|
|
|
|
+ numRunningNodes, numNMs));
|
|
Thread.sleep(1000);
|
|
Thread.sleep(1000);
|
|
}
|
|
}
|
|
LOG.info(MessageFormat.format("SLSRunner takes {0} ms to launch all nodes.",
|
|
LOG.info(MessageFormat.format("SLSRunner takes {0} ms to launch all nodes.",
|
|
- (System.currentTimeMillis() - startTimeMS)));
|
|
|
|
|
|
+ (System.currentTimeMillis() - startTimeMS)));
|
|
}
|
|
}
|
|
|
|
|
|
@SuppressWarnings("unchecked")
|
|
@SuppressWarnings("unchecked")
|
|
private void startAM() throws YarnException, IOException {
|
|
private void startAM() throws YarnException, IOException {
|
|
// application/container configuration
|
|
// application/container configuration
|
|
- int heartbeatInterval = conf.getInt(
|
|
|
|
- SLSConfiguration.AM_HEARTBEAT_INTERVAL_MS,
|
|
|
|
|
|
+ int heartbeatInterval =
|
|
|
|
+ getConf().getInt(SLSConfiguration.AM_HEARTBEAT_INTERVAL_MS,
|
|
SLSConfiguration.AM_HEARTBEAT_INTERVAL_MS_DEFAULT);
|
|
SLSConfiguration.AM_HEARTBEAT_INTERVAL_MS_DEFAULT);
|
|
- int containerMemoryMB = conf.getInt(SLSConfiguration.CONTAINER_MEMORY_MB,
|
|
|
|
|
|
+ int containerMemoryMB =
|
|
|
|
+ getConf().getInt(SLSConfiguration.CONTAINER_MEMORY_MB,
|
|
SLSConfiguration.CONTAINER_MEMORY_MB_DEFAULT);
|
|
SLSConfiguration.CONTAINER_MEMORY_MB_DEFAULT);
|
|
- int containerVCores = conf.getInt(SLSConfiguration.CONTAINER_VCORES,
|
|
|
|
- SLSConfiguration.CONTAINER_VCORES_DEFAULT);
|
|
|
|
|
|
+ int containerVCores = getConf().getInt(SLSConfiguration.CONTAINER_VCORES,
|
|
|
|
+ SLSConfiguration.CONTAINER_VCORES_DEFAULT);
|
|
Resource containerResource =
|
|
Resource containerResource =
|
|
- BuilderUtils.newResource(containerMemoryMB, containerVCores);
|
|
|
|
|
|
+ BuilderUtils.newResource(containerMemoryMB, containerVCores);
|
|
|
|
|
|
// application workload
|
|
// application workload
|
|
- if (isSLS) {
|
|
|
|
|
|
+ switch (inputType) {
|
|
|
|
+ case SLS:
|
|
startAMFromSLSTraces(containerResource, heartbeatInterval);
|
|
startAMFromSLSTraces(containerResource, heartbeatInterval);
|
|
- } else {
|
|
|
|
|
|
+ break;
|
|
|
|
+ case RUMEN:
|
|
startAMFromRumenTraces(containerResource, heartbeatInterval);
|
|
startAMFromRumenTraces(containerResource, heartbeatInterval);
|
|
|
|
+ break;
|
|
|
|
+ case SYNTH:
|
|
|
|
+ startAMFromSynthGenerator(heartbeatInterval);
|
|
|
|
+ break;
|
|
|
|
+ default:
|
|
|
|
+ throw new YarnException("Input configuration not recognized, "
|
|
|
|
+ + "trace type should be SLS, RUMEN, or SYNTH");
|
|
}
|
|
}
|
|
numAMs = amMap.size();
|
|
numAMs = amMap.size();
|
|
remainingApps = numAMs;
|
|
remainingApps = numAMs;
|
|
@@ -284,7 +336,7 @@ public class SLSRunner {
|
|
*/
|
|
*/
|
|
@SuppressWarnings("unchecked")
|
|
@SuppressWarnings("unchecked")
|
|
private void startAMFromSLSTraces(Resource containerResource,
|
|
private void startAMFromSLSTraces(Resource containerResource,
|
|
- int heartbeatInterval) throws IOException {
|
|
|
|
|
|
+ int heartbeatInterval) throws IOException {
|
|
// parse from sls traces
|
|
// parse from sls traces
|
|
JsonFactory jsonF = new JsonFactory();
|
|
JsonFactory jsonF = new JsonFactory();
|
|
ObjectMapper mapper = new ObjectMapper();
|
|
ObjectMapper mapper = new ObjectMapper();
|
|
@@ -292,26 +344,28 @@ public class SLSRunner {
|
|
Reader input =
|
|
Reader input =
|
|
new InputStreamReader(new FileInputStream(inputTrace), "UTF-8");
|
|
new InputStreamReader(new FileInputStream(inputTrace), "UTF-8");
|
|
try {
|
|
try {
|
|
- Iterator<Map> i = mapper.readValues(jsonF.createParser(input),
|
|
|
|
- Map.class);
|
|
|
|
|
|
+ Iterator<Map> i =
|
|
|
|
+ mapper.readValues(jsonF.createParser(input), Map.class);
|
|
while (i.hasNext()) {
|
|
while (i.hasNext()) {
|
|
Map jsonJob = i.next();
|
|
Map jsonJob = i.next();
|
|
|
|
|
|
// load job information
|
|
// load job information
|
|
- long jobStartTime = Long.parseLong(
|
|
|
|
- jsonJob.get("job.start.ms").toString());
|
|
|
|
- long jobFinishTime = Long.parseLong(
|
|
|
|
- jsonJob.get("job.end.ms").toString());
|
|
|
|
|
|
+ long jobStartTime =
|
|
|
|
+ Long.parseLong(jsonJob.get("job.start.ms").toString());
|
|
|
|
+ long jobFinishTime =
|
|
|
|
+ Long.parseLong(jsonJob.get("job.end.ms").toString());
|
|
|
|
|
|
String user = (String) jsonJob.get("job.user");
|
|
String user = (String) jsonJob.get("job.user");
|
|
- if (user == null) user = "default";
|
|
|
|
|
|
+ if (user == null) {
|
|
|
|
+ user = "default";
|
|
|
|
+ }
|
|
String queue = jsonJob.get("job.queue.name").toString();
|
|
String queue = jsonJob.get("job.queue.name").toString();
|
|
|
|
|
|
String oldAppId = jsonJob.get("job.id").toString();
|
|
String oldAppId = jsonJob.get("job.id").toString();
|
|
boolean isTracked = trackedApps.contains(oldAppId);
|
|
boolean isTracked = trackedApps.contains(oldAppId);
|
|
- int queueSize = queueAppNumMap.containsKey(queue) ?
|
|
|
|
- queueAppNumMap.get(queue) : 0;
|
|
|
|
- queueSize ++;
|
|
|
|
|
|
+ int queueSize =
|
|
|
|
+ queueAppNumMap.containsKey(queue) ? queueAppNumMap.get(queue) : 0;
|
|
|
|
+ queueSize++;
|
|
queueAppNumMap.put(queue, queueSize);
|
|
queueAppNumMap.put(queue, queueSize);
|
|
// tasks
|
|
// tasks
|
|
List tasks = (List) jsonJob.get("job.tasks");
|
|
List tasks = (List) jsonJob.get("job.tasks");
|
|
@@ -319,45 +373,45 @@ public class SLSRunner {
|
|
continue;
|
|
continue;
|
|
}
|
|
}
|
|
List<ContainerSimulator> containerList =
|
|
List<ContainerSimulator> containerList =
|
|
- new ArrayList<ContainerSimulator>();
|
|
|
|
|
|
+ new ArrayList<ContainerSimulator>();
|
|
for (Object o : tasks) {
|
|
for (Object o : tasks) {
|
|
Map jsonTask = (Map) o;
|
|
Map jsonTask = (Map) o;
|
|
String hostname = jsonTask.get("container.host").toString();
|
|
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 taskStart =
|
|
|
|
+ Long.parseLong(jsonTask.get("container.start.ms").toString());
|
|
|
|
+ long taskFinish =
|
|
|
|
+ Long.parseLong(jsonTask.get("container.end.ms").toString());
|
|
long lifeTime = taskFinish - taskStart;
|
|
long lifeTime = taskFinish - taskStart;
|
|
|
|
|
|
// Set memory and vcores from job trace file
|
|
// Set memory and vcores from job trace file
|
|
Resource res = Resources.clone(containerResource);
|
|
Resource res = Resources.clone(containerResource);
|
|
if (jsonTask.containsKey("container.memory")) {
|
|
if (jsonTask.containsKey("container.memory")) {
|
|
- int containerMemory = Integer.parseInt(
|
|
|
|
- jsonTask.get("container.memory").toString());
|
|
|
|
|
|
+ int containerMemory =
|
|
|
|
+ Integer.parseInt(jsonTask.get("container.memory").toString());
|
|
res.setMemorySize(containerMemory);
|
|
res.setMemorySize(containerMemory);
|
|
}
|
|
}
|
|
|
|
|
|
if (jsonTask.containsKey("container.vcores")) {
|
|
if (jsonTask.containsKey("container.vcores")) {
|
|
- int containerVCores = Integer.parseInt(
|
|
|
|
- jsonTask.get("container.vcores").toString());
|
|
|
|
|
|
+ int containerVCores =
|
|
|
|
+ Integer.parseInt(jsonTask.get("container.vcores").toString());
|
|
res.setVirtualCores(containerVCores);
|
|
res.setVirtualCores(containerVCores);
|
|
}
|
|
}
|
|
|
|
|
|
- int priority = Integer.parseInt(
|
|
|
|
- jsonTask.get("container.priority").toString());
|
|
|
|
|
|
+ int priority =
|
|
|
|
+ Integer.parseInt(jsonTask.get("container.priority").toString());
|
|
String type = jsonTask.get("container.type").toString();
|
|
String type = jsonTask.get("container.type").toString();
|
|
- containerList.add(new ContainerSimulator(res,
|
|
|
|
- lifeTime, hostname, priority, type));
|
|
|
|
|
|
+ containerList.add(new ContainerSimulator(res, lifeTime, hostname,
|
|
|
|
+ priority, type));
|
|
}
|
|
}
|
|
|
|
|
|
// create a new AM
|
|
// create a new AM
|
|
String amType = jsonJob.get("am.type").toString();
|
|
String amType = jsonJob.get("am.type").toString();
|
|
- AMSimulator amSim = (AMSimulator) ReflectionUtils.newInstance(
|
|
|
|
- amClassMap.get(amType), new Configuration());
|
|
|
|
|
|
+ AMSimulator amSim = (AMSimulator) ReflectionUtils
|
|
|
|
+ .newInstance(amClassMap.get(amType), new Configuration());
|
|
if (amSim != null) {
|
|
if (amSim != null) {
|
|
- amSim.init(AM_ID++, heartbeatInterval, containerList, rm,
|
|
|
|
- this, jobStartTime, jobFinishTime, user, queue,
|
|
|
|
- isTracked, oldAppId);
|
|
|
|
|
|
+ amSim.init(AM_ID++, heartbeatInterval, containerList, rm, this,
|
|
|
|
+ jobStartTime, jobFinishTime, user, queue, isTracked, oldAppId,
|
|
|
|
+ null, runner.getStartTimeMS());
|
|
runner.schedule(amSim);
|
|
runner.schedule(amSim);
|
|
maxRuntime = Math.max(maxRuntime, jobFinishTime);
|
|
maxRuntime = Math.max(maxRuntime, jobFinishTime);
|
|
numTasks += containerList.size();
|
|
numTasks += containerList.size();
|
|
@@ -375,22 +429,21 @@ public class SLSRunner {
|
|
*/
|
|
*/
|
|
@SuppressWarnings("unchecked")
|
|
@SuppressWarnings("unchecked")
|
|
private void startAMFromRumenTraces(Resource containerResource,
|
|
private void startAMFromRumenTraces(Resource containerResource,
|
|
- int heartbeatInterval)
|
|
|
|
- throws IOException {
|
|
|
|
|
|
+ int heartbeatInterval) throws IOException {
|
|
Configuration conf = new Configuration();
|
|
Configuration conf = new Configuration();
|
|
conf.set("fs.defaultFS", "file:///");
|
|
conf.set("fs.defaultFS", "file:///");
|
|
long baselineTimeMS = 0;
|
|
long baselineTimeMS = 0;
|
|
for (String inputTrace : inputTraces) {
|
|
for (String inputTrace : inputTraces) {
|
|
File fin = new File(inputTrace);
|
|
File fin = new File(inputTrace);
|
|
- JobTraceReader reader = new JobTraceReader(
|
|
|
|
- new Path(fin.getAbsolutePath()), conf);
|
|
|
|
|
|
+ JobTraceReader reader =
|
|
|
|
+ new JobTraceReader(new Path(fin.getAbsolutePath()), conf);
|
|
try {
|
|
try {
|
|
LoggedJob job = null;
|
|
LoggedJob job = null;
|
|
while ((job = reader.getNext()) != null) {
|
|
while ((job = reader.getNext()) != null) {
|
|
// only support MapReduce currently
|
|
// only support MapReduce currently
|
|
String jobType = "mapreduce";
|
|
String jobType = "mapreduce";
|
|
- String user = job.getUser() == null ?
|
|
|
|
- "default" : job.getUser().getValue();
|
|
|
|
|
|
+ String user =
|
|
|
|
+ job.getUser() == null ? "default" : job.getUser().getValue();
|
|
String jobQueue = job.getQueue().getValue();
|
|
String jobQueue = job.getQueue().getValue();
|
|
String oldJobId = job.getJobID().toString();
|
|
String oldJobId = job.getJobID().toString();
|
|
long jobStartTimeMS = job.getSubmitTime();
|
|
long jobStartTimeMS = job.getSubmitTime();
|
|
@@ -407,48 +460,48 @@ public class SLSRunner {
|
|
}
|
|
}
|
|
|
|
|
|
boolean isTracked = trackedApps.contains(oldJobId);
|
|
boolean isTracked = trackedApps.contains(oldJobId);
|
|
- int queueSize = queueAppNumMap.containsKey(jobQueue) ?
|
|
|
|
- queueAppNumMap.get(jobQueue) : 0;
|
|
|
|
- queueSize ++;
|
|
|
|
|
|
+ int queueSize = queueAppNumMap.containsKey(jobQueue)
|
|
|
|
+ ? queueAppNumMap.get(jobQueue) : 0;
|
|
|
|
+ queueSize++;
|
|
queueAppNumMap.put(jobQueue, queueSize);
|
|
queueAppNumMap.put(jobQueue, queueSize);
|
|
|
|
|
|
List<ContainerSimulator> containerList =
|
|
List<ContainerSimulator> containerList =
|
|
- new ArrayList<ContainerSimulator>();
|
|
|
|
|
|
+ new ArrayList<ContainerSimulator>();
|
|
// map tasks
|
|
// map tasks
|
|
- for(LoggedTask mapTask : job.getMapTasks()) {
|
|
|
|
|
|
+ for (LoggedTask mapTask : job.getMapTasks()) {
|
|
if (mapTask.getAttempts().size() == 0) {
|
|
if (mapTask.getAttempts().size() == 0) {
|
|
continue;
|
|
continue;
|
|
}
|
|
}
|
|
- LoggedTaskAttempt taskAttempt = mapTask.getAttempts()
|
|
|
|
- .get(mapTask.getAttempts().size() - 1);
|
|
|
|
|
|
+ LoggedTaskAttempt taskAttempt =
|
|
|
|
+ mapTask.getAttempts().get(mapTask.getAttempts().size() - 1);
|
|
String hostname = taskAttempt.getHostName().getValue();
|
|
String hostname = taskAttempt.getHostName().getValue();
|
|
- long containerLifeTime = taskAttempt.getFinishTime()
|
|
|
|
- - taskAttempt.getStartTime();
|
|
|
|
|
|
+ long containerLifeTime =
|
|
|
|
+ taskAttempt.getFinishTime() - taskAttempt.getStartTime();
|
|
containerList.add(new ContainerSimulator(containerResource,
|
|
containerList.add(new ContainerSimulator(containerResource,
|
|
- containerLifeTime, hostname, 10, "map"));
|
|
|
|
|
|
+ containerLifeTime, hostname, 10, "map"));
|
|
}
|
|
}
|
|
|
|
|
|
// reduce tasks
|
|
// reduce tasks
|
|
- for(LoggedTask reduceTask : job.getReduceTasks()) {
|
|
|
|
|
|
+ for (LoggedTask reduceTask : job.getReduceTasks()) {
|
|
if (reduceTask.getAttempts().size() == 0) {
|
|
if (reduceTask.getAttempts().size() == 0) {
|
|
continue;
|
|
continue;
|
|
}
|
|
}
|
|
LoggedTaskAttempt taskAttempt = reduceTask.getAttempts()
|
|
LoggedTaskAttempt taskAttempt = reduceTask.getAttempts()
|
|
- .get(reduceTask.getAttempts().size() - 1);
|
|
|
|
|
|
+ .get(reduceTask.getAttempts().size() - 1);
|
|
String hostname = taskAttempt.getHostName().getValue();
|
|
String hostname = taskAttempt.getHostName().getValue();
|
|
- long containerLifeTime = taskAttempt.getFinishTime()
|
|
|
|
- - taskAttempt.getStartTime();
|
|
|
|
|
|
+ long containerLifeTime =
|
|
|
|
+ taskAttempt.getFinishTime() - taskAttempt.getStartTime();
|
|
containerList.add(new ContainerSimulator(containerResource,
|
|
containerList.add(new ContainerSimulator(containerResource,
|
|
- containerLifeTime, hostname, 20, "reduce"));
|
|
|
|
|
|
+ containerLifeTime, hostname, 20, "reduce"));
|
|
}
|
|
}
|
|
|
|
|
|
// create a new AM
|
|
// create a new AM
|
|
- AMSimulator amSim = (AMSimulator) ReflectionUtils.newInstance(
|
|
|
|
- amClassMap.get(jobType), conf);
|
|
|
|
|
|
+ AMSimulator amSim = (AMSimulator) ReflectionUtils
|
|
|
|
+ .newInstance(amClassMap.get(jobType), conf);
|
|
if (amSim != null) {
|
|
if (amSim != null) {
|
|
- amSim.init(AM_ID ++, heartbeatInterval, containerList,
|
|
|
|
- rm, this, jobStartTimeMS, jobFinishTimeMS, user, jobQueue,
|
|
|
|
- isTracked, oldJobId);
|
|
|
|
|
|
+ amSim.init(AM_ID++, heartbeatInterval, containerList, rm, this,
|
|
|
|
+ jobStartTimeMS, jobFinishTimeMS, user, jobQueue, isTracked,
|
|
|
|
+ oldJobId, null, runner.getStartTimeMS());
|
|
runner.schedule(amSim);
|
|
runner.schedule(amSim);
|
|
maxRuntime = Math.max(maxRuntime, jobFinishTimeMS);
|
|
maxRuntime = Math.max(maxRuntime, jobFinishTimeMS);
|
|
numTasks += containerList.size();
|
|
numTasks += containerList.size();
|
|
@@ -460,34 +513,168 @@ public class SLSRunner {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * parse workload information from synth-generator trace files.
|
|
|
|
+ */
|
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
|
+ private void startAMFromSynthGenerator(int heartbeatInterval)
|
|
|
|
+ throws 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 jobType = "mapreduce";
|
|
|
|
+ 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 " + oldJobId + " start time to 0.");
|
|
|
|
+ jobFinishTimeMS = jobFinishTimeMS - jobStartTimeMS;
|
|
|
|
+ jobStartTimeMS = 0;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ boolean isTracked = trackedApps.contains(oldJobId);
|
|
|
|
+ int queueSize = queueAppNumMap.containsKey(jobQueue)
|
|
|
|
+ ? queueAppNumMap.get(jobQueue) : 0;
|
|
|
|
+ queueSize++;
|
|
|
|
+ queueAppNumMap.put(jobQueue, queueSize);
|
|
|
|
+
|
|
|
|
+ List<ContainerSimulator> containerList =
|
|
|
|
+ new ArrayList<ContainerSimulator>();
|
|
|
|
+ ArrayList<NodeId> keyAsArray = new ArrayList<NodeId>(nmMap.keySet());
|
|
|
|
+ Random rand = new Random(stjp.getSeed());
|
|
|
|
+
|
|
|
|
+ Resource maxMapRes = Resource.newInstance(0, 0);
|
|
|
|
+ long maxMapDur = 0;
|
|
|
|
+ // map tasks
|
|
|
|
+ for (int i = 0; i < job.getNumberMaps(); i++) {
|
|
|
|
+ TaskAttemptInfo tai = job.getTaskAttemptInfo(TaskType.MAP, i, 0);
|
|
|
|
+ RMNode node = nmMap
|
|
|
|
+ .get(keyAsArray.get(rand.nextInt(keyAsArray.size()))).getNode();
|
|
|
|
+ String hostname = "/" + node.getRackName() + "/" + node.getHostName();
|
|
|
|
+ long containerLifeTime = tai.getRuntime();
|
|
|
|
+ Resource containerResource =
|
|
|
|
+ Resource.newInstance((int) tai.getTaskInfo().getTaskMemory(),
|
|
|
|
+ (int) tai.getTaskInfo().getTaskVCores());
|
|
|
|
+ containerList.add(new ContainerSimulator(containerResource,
|
|
|
|
+ containerLifeTime, hostname, 10, "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, 20, "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());
|
|
|
|
+
|
|
|
|
+ }
|
|
|
|
+ // create a new AM
|
|
|
|
+ AMSimulator amSim = (AMSimulator) ReflectionUtils
|
|
|
|
+ .newInstance(amClassMap.get(jobType), localConf);
|
|
|
|
+ if (amSim != null) {
|
|
|
|
+ amSim.init(AM_ID++, heartbeatInterval, containerList, rm, this,
|
|
|
|
+ jobStartTimeMS, jobFinishTimeMS, user, jobQueue, isTracked,
|
|
|
|
+ oldJobId, rr, runner.getStartTimeMS());
|
|
|
|
+ runner.schedule(amSim);
|
|
|
|
+ maxRuntime = Math.max(maxRuntime, jobFinishTimeMS);
|
|
|
|
+ numTasks += containerList.size();
|
|
|
|
+ amMap.put(oldJobId, amSim);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ } finally {
|
|
|
|
+ stjp.close();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ }
|
|
|
|
+
|
|
private void printSimulationInfo() {
|
|
private void printSimulationInfo() {
|
|
if (printSimulation) {
|
|
if (printSimulation) {
|
|
// node
|
|
// node
|
|
LOG.info("------------------------------------");
|
|
LOG.info("------------------------------------");
|
|
- LOG.info(MessageFormat.format("# nodes = {0}, # racks = {1}, capacity " +
|
|
|
|
- "of each node {2} MB memory and {3} vcores.",
|
|
|
|
- numNMs, numRacks, nmMemoryMB, nmVCores));
|
|
|
|
|
|
+ LOG.info(MessageFormat.format(
|
|
|
|
+ "# nodes = {0}, # racks = {1}, capacity "
|
|
|
|
+ + "of each node {2} MB memory and {3} vcores.",
|
|
|
|
+ numNMs, numRacks, nmMemoryMB, nmVCores));
|
|
LOG.info("------------------------------------");
|
|
LOG.info("------------------------------------");
|
|
// job
|
|
// job
|
|
- LOG.info(MessageFormat.format("# applications = {0}, # total " +
|
|
|
|
- "tasks = {1}, average # tasks per application = {2}",
|
|
|
|
- numAMs, numTasks, (int)(Math.ceil((numTasks + 0.0) / numAMs))));
|
|
|
|
|
|
+ LOG.info(MessageFormat.format(
|
|
|
|
+ "# applications = {0}, # total "
|
|
|
|
+ + "tasks = {1}, average # tasks per application = {2}",
|
|
|
|
+ numAMs, numTasks, (int) (Math.ceil((numTasks + 0.0) / numAMs))));
|
|
LOG.info("JobId\tQueue\tAMType\tDuration\t#Tasks");
|
|
LOG.info("JobId\tQueue\tAMType\tDuration\t#Tasks");
|
|
for (Map.Entry<String, AMSimulator> entry : amMap.entrySet()) {
|
|
for (Map.Entry<String, AMSimulator> entry : amMap.entrySet()) {
|
|
AMSimulator am = entry.getValue();
|
|
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());
|
|
+ "\t" + am.getDuration() + "\t" + am.getNumTasks());
|
|
}
|
|
}
|
|
LOG.info("------------------------------------");
|
|
LOG.info("------------------------------------");
|
|
// queue
|
|
// queue
|
|
- LOG.info(MessageFormat.format("number of queues = {0} average " +
|
|
|
|
- "number of apps = {1}", queueAppNumMap.size(),
|
|
|
|
- (int)(Math.ceil((numAMs + 0.0) / queueAppNumMap.size()))));
|
|
|
|
|
|
+ LOG.info(MessageFormat.format(
|
|
|
|
+ "number of queues = {0} average " + "number of apps = {1}",
|
|
|
|
+ queueAppNumMap.size(),
|
|
|
|
+ (int) (Math.ceil((numAMs + 0.0) / queueAppNumMap.size()))));
|
|
LOG.info("------------------------------------");
|
|
LOG.info("------------------------------------");
|
|
// runtime
|
|
// runtime
|
|
- LOG.info(MessageFormat.format("estimated simulation time is {0}" +
|
|
|
|
- " seconds", (long)(Math.ceil(maxRuntime / 1000.0))));
|
|
|
|
|
|
+ LOG.info(
|
|
|
|
+ MessageFormat.format("estimated simulation time is {0}" + " seconds",
|
|
|
|
+ (long) (Math.ceil(maxRuntime / 1000.0))));
|
|
LOG.info("------------------------------------");
|
|
LOG.info("------------------------------------");
|
|
}
|
|
}
|
|
// package these information in the simulateInfoMap used by other places
|
|
// package these information in the simulateInfoMap used by other places
|
|
@@ -510,69 +697,121 @@ public class SLSRunner {
|
|
return nmMap;
|
|
return nmMap;
|
|
}
|
|
}
|
|
|
|
|
|
- public static TaskRunner getRunner() {
|
|
|
|
- return runner;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
public static void decreaseRemainingApps() {
|
|
public static void decreaseRemainingApps() {
|
|
- remainingApps --;
|
|
|
|
|
|
+ remainingApps--;
|
|
|
|
|
|
if (remainingApps == 0) {
|
|
if (remainingApps == 0) {
|
|
LOG.info("SLSRunner tears down.");
|
|
LOG.info("SLSRunner tears down.");
|
|
- 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();
|
|
Options options = new Options();
|
|
|
|
+
|
|
|
|
+ // Left for compatibility
|
|
options.addOption("inputrumen", true, "input rumen files");
|
|
options.addOption("inputrumen", true, "input rumen files");
|
|
options.addOption("inputsls", true, "input sls 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("nodes", true, "input topology");
|
|
options.addOption("output", true, "output directory");
|
|
options.addOption("output", true, "output directory");
|
|
options.addOption("trackjobs", true,
|
|
options.addOption("trackjobs", true,
|
|
- "jobs to be tracked during simulating");
|
|
|
|
|
|
+ "jobs to be tracked during simulating");
|
|
options.addOption("printsimulation", false,
|
|
options.addOption("printsimulation", false,
|
|
- "print out simulation information");
|
|
|
|
-
|
|
|
|
|
|
+ "print out simulation information");
|
|
|
|
+
|
|
CommandLineParser parser = new GnuParser();
|
|
CommandLineParser parser = new GnuParser();
|
|
- CommandLine cmd = parser.parse(options, args);
|
|
|
|
|
|
+ CommandLine cmd = parser.parse(options, argv);
|
|
|
|
|
|
- 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);
|
|
|
|
|
|
+ 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 output = cmd.getOptionValue("output");
|
|
|
|
+
|
|
File outputFile = new File(output);
|
|
File outputFile = new File(output);
|
|
- if (! outputFile.exists()
|
|
|
|
- && ! outputFile.mkdirs()) {
|
|
|
|
|
|
+ if (!outputFile.exists() && !outputFile.mkdirs()) {
|
|
System.err.println("ERROR: Cannot create output directory "
|
|
System.err.println("ERROR: Cannot create output directory "
|
|
- + outputFile.getAbsolutePath());
|
|
|
|
- System.exit(1);
|
|
|
|
|
|
+ + outputFile.getAbsolutePath());
|
|
|
|
+ throw new YarnException("Cannot create output directory");
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
Set<String> trackedJobSet = new HashSet<String>();
|
|
Set<String> trackedJobSet = new HashSet<String>();
|
|
if (cmd.hasOption("trackjobs")) {
|
|
if (cmd.hasOption("trackjobs")) {
|
|
String trackjobs = cmd.getOptionValue("trackjobs");
|
|
String trackjobs = cmd.getOptionValue("trackjobs");
|
|
String jobIds[] = trackjobs.split(",");
|
|
String jobIds[] = trackjobs.split(",");
|
|
trackedJobSet.addAll(Arrays.asList(jobIds));
|
|
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"));
|
|
trackedJobSet, cmd.hasOption("printsimulation"));
|
|
- sls.start();
|
|
|
|
|
|
+
|
|
|
|
+ start();
|
|
|
|
+
|
|
|
|
+ return 0;
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ public static void main(String[] argv) throws Exception {
|
|
|
|
+ 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();
|
|
|
|
+ }
|
|
|
|
+
|
|
}
|
|
}
|