|
@@ -119,6 +119,9 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
// logger
|
|
|
public final static Logger LOG = Logger.getLogger(SLSRunner.class);
|
|
|
|
|
|
+ private final static int DEFAULT_MAPPER_PRIORITY = 20;
|
|
|
+ private final static int DEFAULT_REDUCER_PRIORITY = 10;
|
|
|
+
|
|
|
/**
|
|
|
* The type of trace in input.
|
|
|
*/
|
|
@@ -247,8 +250,8 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
break;
|
|
|
case SYNTH:
|
|
|
stjp = new SynthTraceJobProducer(getConf(), new Path(inputTraces[0]));
|
|
|
- nodeSet.addAll(SLSUtils.generateNodesFromSynth(stjp.getNumNodes(),
|
|
|
- stjp.getNodesPerRack()));
|
|
|
+ nodeSet.addAll(SLSUtils.generateNodes(stjp.getNumNodes(),
|
|
|
+ stjp.getNumNodes()/stjp.getNodesPerRack()));
|
|
|
break;
|
|
|
default:
|
|
|
throw new YarnException("Input configuration not recognized, "
|
|
@@ -259,6 +262,10 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
nodeSet.addAll(SLSUtils.parseNodesFromNodeFile(nodeFile));
|
|
|
}
|
|
|
|
|
|
+ if (nodeSet.size() == 0) {
|
|
|
+ throw new YarnException("No node! Please configure nodes.");
|
|
|
+ }
|
|
|
+
|
|
|
// create NM simulators
|
|
|
Random random = new Random();
|
|
|
Set<String> rackSet = new HashSet<String>();
|
|
@@ -348,7 +355,11 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
|
|
|
private void createAMForJob(Map jsonJob) throws YarnException {
|
|
|
long jobStartTime = Long.parseLong(jsonJob.get("job.start.ms").toString());
|
|
|
- long jobFinishTime = Long.parseLong(jsonJob.get("job.end.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) {
|
|
@@ -358,25 +369,49 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
String queue = jsonJob.get("job.queue.name").toString();
|
|
|
increaseQueueAppNum(queue);
|
|
|
|
|
|
- String oldAppId = jsonJob.get("job.id").toString();
|
|
|
+ String oldAppId = (String)jsonJob.get("job.id");
|
|
|
+ if (oldAppId == null) {
|
|
|
+ oldAppId = Integer.toString(AM_ID);
|
|
|
+ }
|
|
|
|
|
|
- // tasks
|
|
|
+ String amType = (String)jsonJob.get("am.type");
|
|
|
+ if (amType == null) {
|
|
|
+ amType = SLSUtils.DEFAULT_JOB_TYPE;
|
|
|
+ }
|
|
|
+
|
|
|
+ runNewAM(amType, user, queue, oldAppId, jobStartTime, jobFinishTime,
|
|
|
+ getTaskContainers(jsonJob), null);
|
|
|
+ }
|
|
|
+
|
|
|
+ private List<ContainerSimulator> getTaskContainers(Map jsonJob)
|
|
|
+ throws YarnException {
|
|
|
+ List<ContainerSimulator> containers = new ArrayList<>();
|
|
|
List tasks = (List) jsonJob.get("job.tasks");
|
|
|
if (tasks == null || tasks.size() == 0) {
|
|
|
throw new YarnException("No task for the job!");
|
|
|
}
|
|
|
|
|
|
- List<ContainerSimulator> 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
|
|
|
+
|
|
|
+ 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 =
|
|
@@ -390,17 +425,30 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
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));
|
|
|
+ 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));
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
- // create a new AM
|
|
|
- String amType = jsonJob.get("am.type").toString();
|
|
|
- runNewAM(amType, user, queue, oldAppId, jobStartTime, jobFinishTime,
|
|
|
- containerList, null);
|
|
|
+ return containers;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -463,7 +511,7 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
taskAttempt.getStartTime();
|
|
|
containerList.add(
|
|
|
new ContainerSimulator(getDefaultContainerResource(),
|
|
|
- containerLifeTime, hostname, 10, "map"));
|
|
|
+ containerLifeTime, hostname, DEFAULT_MAPPER_PRIORITY, "map"));
|
|
|
}
|
|
|
|
|
|
// reducer
|
|
@@ -479,7 +527,7 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
taskAttempt.getStartTime();
|
|
|
containerList.add(
|
|
|
new ContainerSimulator(getDefaultContainerResource(),
|
|
|
- containerLifeTime, hostname, 20, "reduce"));
|
|
|
+ containerLifeTime, hostname, DEFAULT_REDUCER_PRIORITY, "reduce"));
|
|
|
}
|
|
|
|
|
|
// Only supports the default job type currently
|
|
@@ -559,7 +607,7 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
Resource.newInstance((int) tai.getTaskInfo().getTaskMemory(),
|
|
|
(int) tai.getTaskInfo().getTaskVCores());
|
|
|
containerList.add(new ContainerSimulator(containerResource,
|
|
|
- containerLifeTime, hostname, 10, "map"));
|
|
|
+ containerLifeTime, hostname, DEFAULT_MAPPER_PRIORITY, "map"));
|
|
|
maxMapRes = Resources.componentwiseMax(maxMapRes, containerResource);
|
|
|
maxMapDur =
|
|
|
containerLifeTime > maxMapDur ? containerLifeTime : maxMapDur;
|
|
@@ -579,7 +627,7 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
Resource.newInstance((int) tai.getTaskInfo().getTaskMemory(),
|
|
|
(int) tai.getTaskInfo().getTaskVCores());
|
|
|
containerList.add(new ContainerSimulator(containerResource,
|
|
|
- containerLifeTime, hostname, 20, "reduce"));
|
|
|
+ containerLifeTime, hostname, DEFAULT_REDUCER_PRIORITY, "reduce"));
|
|
|
maxRedRes = Resources.componentwiseMax(maxRedRes, containerResource);
|
|
|
maxRedDur =
|
|
|
containerLifeTime > maxRedDur ? containerLifeTime : maxRedDur;
|