|
@@ -60,6 +60,7 @@ import org.apache.hadoop.util.Tool;
|
|
import org.apache.hadoop.util.ToolRunner;
|
|
import org.apache.hadoop.util.ToolRunner;
|
|
import org.apache.hadoop.yarn.api.records.ExecutionType;
|
|
import org.apache.hadoop.yarn.api.records.ExecutionType;
|
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
|
|
|
+import org.apache.hadoop.yarn.api.records.NodeLabel;
|
|
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.ReservationId;
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
@@ -298,30 +299,20 @@ public class SLSRunner extends Configured implements Tool {
|
|
SLSConfiguration.NM_RESOURCE_UTILIZATION_RATIO,
|
|
SLSConfiguration.NM_RESOURCE_UTILIZATION_RATIO,
|
|
SLSConfiguration.NM_RESOURCE_UTILIZATION_RATIO_DEFAULT);
|
|
SLSConfiguration.NM_RESOURCE_UTILIZATION_RATIO_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)
|
|
- Map<String, Resource> nodeResourceMap = new HashMap<>();
|
|
|
|
- Set<? extends String> nodeSet;
|
|
|
|
|
|
+ Set<NodeDetails> nodeSet = null;
|
|
if (nodeFile.isEmpty()) {
|
|
if (nodeFile.isEmpty()) {
|
|
for (String inputTrace : inputTraces) {
|
|
for (String inputTrace : inputTraces) {
|
|
switch (inputType) {
|
|
switch (inputType) {
|
|
case SLS:
|
|
case SLS:
|
|
nodeSet = SLSUtils.parseNodesFromSLSTrace(inputTrace);
|
|
nodeSet = SLSUtils.parseNodesFromSLSTrace(inputTrace);
|
|
- for (String node : nodeSet) {
|
|
|
|
- nodeResourceMap.put(node, null);
|
|
|
|
- }
|
|
|
|
break;
|
|
break;
|
|
case RUMEN:
|
|
case RUMEN:
|
|
nodeSet = SLSUtils.parseNodesFromRumenTrace(inputTrace);
|
|
nodeSet = SLSUtils.parseNodesFromRumenTrace(inputTrace);
|
|
- for (String node : nodeSet) {
|
|
|
|
- nodeResourceMap.put(node, null);
|
|
|
|
- }
|
|
|
|
break;
|
|
break;
|
|
case SYNTH:
|
|
case SYNTH:
|
|
stjp = new SynthTraceJobProducer(getConf(), new Path(inputTraces[0]));
|
|
stjp = new SynthTraceJobProducer(getConf(), new Path(inputTraces[0]));
|
|
nodeSet = SLSUtils.generateNodes(stjp.getNumNodes(),
|
|
nodeSet = SLSUtils.generateNodes(stjp.getNumNodes(),
|
|
stjp.getNumNodes()/stjp.getNodesPerRack());
|
|
stjp.getNumNodes()/stjp.getNodesPerRack());
|
|
- for (String node : nodeSet) {
|
|
|
|
- nodeResourceMap.put(node, null);
|
|
|
|
- }
|
|
|
|
break;
|
|
break;
|
|
default:
|
|
default:
|
|
throw new YarnException("Input configuration not recognized, "
|
|
throw new YarnException("Input configuration not recognized, "
|
|
@@ -329,11 +320,11 @@ public class SLSRunner extends Configured implements Tool {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
} else {
|
|
} else {
|
|
- nodeResourceMap = SLSUtils.parseNodesFromNodeFile(nodeFile,
|
|
|
|
|
|
+ nodeSet = SLSUtils.parseNodesFromNodeFile(nodeFile,
|
|
nodeManagerResource);
|
|
nodeManagerResource);
|
|
}
|
|
}
|
|
|
|
|
|
- if (nodeResourceMap.size() == 0) {
|
|
|
|
|
|
+ if (nodeSet == null || nodeSet.isEmpty()) {
|
|
throw new YarnException("No node! Please configure nodes.");
|
|
throw new YarnException("No node! Please configure nodes.");
|
|
}
|
|
}
|
|
|
|
|
|
@@ -344,20 +335,21 @@ public class SLSRunner extends Configured implements Tool {
|
|
SLSConfiguration.RUNNER_POOL_SIZE_DEFAULT);
|
|
SLSConfiguration.RUNNER_POOL_SIZE_DEFAULT);
|
|
ExecutorService executorService = Executors.
|
|
ExecutorService executorService = Executors.
|
|
newFixedThreadPool(threadPoolSize);
|
|
newFixedThreadPool(threadPoolSize);
|
|
- for (Map.Entry<String, Resource> entry : nodeResourceMap.entrySet()) {
|
|
|
|
|
|
+ for (NodeDetails nodeDetails : nodeSet) {
|
|
executorService.submit(new Runnable() {
|
|
executorService.submit(new Runnable() {
|
|
@Override public void run() {
|
|
@Override public void run() {
|
|
try {
|
|
try {
|
|
// 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();
|
|
Resource nmResource = nodeManagerResource;
|
|
Resource nmResource = nodeManagerResource;
|
|
- String hostName = entry.getKey();
|
|
|
|
- if (entry.getValue() != null) {
|
|
|
|
- nmResource = entry.getValue();
|
|
|
|
|
|
+ String hostName = nodeDetails.getHostname();
|
|
|
|
+ if (nodeDetails.getNodeResource() != null) {
|
|
|
|
+ nmResource = nodeDetails.getNodeResource();
|
|
}
|
|
}
|
|
|
|
+ Set<NodeLabel> nodeLabels = nodeDetails.getLabels();
|
|
nm.init(hostName, nmResource,
|
|
nm.init(hostName, nmResource,
|
|
random.nextInt(heartbeatInterval),
|
|
random.nextInt(heartbeatInterval),
|
|
- heartbeatInterval, rm, resourceUtilizationRatio);
|
|
|
|
|
|
+ heartbeatInterval, rm, resourceUtilizationRatio, nodeLabels);
|
|
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());
|
|
@@ -452,6 +444,11 @@ public class SLSRunner extends Configured implements Tool {
|
|
jsonJob.get(SLSConfiguration.JOB_END_MS).toString());
|
|
jsonJob.get(SLSConfiguration.JOB_END_MS).toString());
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ String jobLabelExpr = null;
|
|
|
|
+ if (jsonJob.containsKey(SLSConfiguration.JOB_LABEL_EXPR)) {
|
|
|
|
+ jobLabelExpr = jsonJob.get(SLSConfiguration.JOB_LABEL_EXPR).toString();
|
|
|
|
+ }
|
|
|
|
+
|
|
String user = (String) jsonJob.get(SLSConfiguration.JOB_USER);
|
|
String user = (String) jsonJob.get(SLSConfiguration.JOB_USER);
|
|
if (user == null) {
|
|
if (user == null) {
|
|
user = "default";
|
|
user = "default";
|
|
@@ -481,7 +478,8 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
|
|
|
for (int i = 0; i < jobCount; i++) {
|
|
for (int i = 0; i < jobCount; i++) {
|
|
runNewAM(amType, user, queue, oldAppId, jobStartTime, jobFinishTime,
|
|
runNewAM(amType, user, queue, oldAppId, jobStartTime, jobFinishTime,
|
|
- getTaskContainers(jsonJob), getAMContainerResource(jsonJob));
|
|
|
|
|
|
+ getTaskContainers(jsonJob), getAMContainerResource(jsonJob),
|
|
|
|
+ jobLabelExpr);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -730,7 +728,7 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
|
|
|
runNewAM(job.getType(), user, jobQueue, oldJobId,
|
|
runNewAM(job.getType(), user, jobQueue, oldJobId,
|
|
jobStartTimeMS, jobFinishTimeMS, containerList, reservationId,
|
|
jobStartTimeMS, jobFinishTimeMS, containerList, reservationId,
|
|
- job.getDeadline(), getAMContainerResource(null),
|
|
|
|
|
|
+ job.getDeadline(), getAMContainerResource(null), null,
|
|
job.getParams());
|
|
job.getParams());
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -775,15 +773,24 @@ public class SLSRunner extends Configured implements Tool {
|
|
Resource amContainerResource) {
|
|
Resource amContainerResource) {
|
|
runNewAM(jobType, user, jobQueue, oldJobId, jobStartTimeMS,
|
|
runNewAM(jobType, user, jobQueue, oldJobId, jobStartTimeMS,
|
|
jobFinishTimeMS, containerList, null, -1,
|
|
jobFinishTimeMS, containerList, null, -1,
|
|
- amContainerResource, null);
|
|
|
|
|
|
+ amContainerResource, null, null);
|
|
}
|
|
}
|
|
|
|
|
|
private void runNewAM(String jobType, String user,
|
|
private void runNewAM(String jobType, String user,
|
|
String jobQueue, String oldJobId, long jobStartTimeMS,
|
|
String jobQueue, String oldJobId, long jobStartTimeMS,
|
|
long jobFinishTimeMS, List<ContainerSimulator> containerList,
|
|
long jobFinishTimeMS, List<ContainerSimulator> containerList,
|
|
- ReservationId reservationId, long deadline, Resource amContainerResource,
|
|
|
|
- Map<String, String> params) {
|
|
|
|
|
|
+ Resource amContainerResource, String labelExpr) {
|
|
|
|
+ runNewAM(jobType, user, jobQueue, oldJobId, jobStartTimeMS,
|
|
|
|
+ jobFinishTimeMS, containerList, null, -1,
|
|
|
|
+ amContainerResource, labelExpr, null);
|
|
|
|
+ }
|
|
|
|
|
|
|
|
+ @SuppressWarnings("checkstyle:parameternumber")
|
|
|
|
+ private void runNewAM(String jobType, String user,
|
|
|
|
+ String jobQueue, String oldJobId, long jobStartTimeMS,
|
|
|
|
+ long jobFinishTimeMS, List<ContainerSimulator> containerList,
|
|
|
|
+ ReservationId reservationId, long deadline, Resource amContainerResource,
|
|
|
|
+ String labelExpr, Map<String, String> params) {
|
|
AMSimulator amSim = (AMSimulator) ReflectionUtils.newInstance(
|
|
AMSimulator amSim = (AMSimulator) ReflectionUtils.newInstance(
|
|
amClassMap.get(jobType), new Configuration());
|
|
amClassMap.get(jobType), new Configuration());
|
|
|
|
|
|
@@ -799,7 +806,7 @@ public class SLSRunner extends Configured implements Tool {
|
|
AM_ID++;
|
|
AM_ID++;
|
|
amSim.init(heartbeatInterval, containerList, rm, this, jobStartTimeMS,
|
|
amSim.init(heartbeatInterval, containerList, rm, this, jobStartTimeMS,
|
|
jobFinishTimeMS, user, jobQueue, isTracked, oldJobId,
|
|
jobFinishTimeMS, user, jobQueue, isTracked, oldJobId,
|
|
- runner.getStartTimeMS(), amContainerResource, params);
|
|
|
|
|
|
+ runner.getStartTimeMS(), amContainerResource, labelExpr, params);
|
|
if(reservationId != null) {
|
|
if(reservationId != null) {
|
|
// if we have a ReservationId, delegate reservation creation to
|
|
// if we have a ReservationId, delegate reservation creation to
|
|
// AMSim (reservation shape is impl specific)
|
|
// AMSim (reservation shape is impl specific)
|
|
@@ -985,4 +992,42 @@ public class SLSRunner extends Configured implements Tool {
|
|
System.err.println();
|
|
System.err.println();
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Class to encapsulate all details about the node.
|
|
|
|
+ */
|
|
|
|
+ @Private
|
|
|
|
+ @Unstable
|
|
|
|
+ public static class NodeDetails {
|
|
|
|
+ private String hostname;
|
|
|
|
+ private Resource nodeResource;
|
|
|
|
+ private Set<NodeLabel> labels;
|
|
|
|
+
|
|
|
|
+ public NodeDetails(String nodeHostname) {
|
|
|
|
+ this.hostname = nodeHostname;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public String getHostname() {
|
|
|
|
+ return hostname;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public void setHostname(String hostname) {
|
|
|
|
+ this.hostname = hostname;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public Resource getNodeResource() {
|
|
|
|
+ return nodeResource;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public void setNodeResource(Resource nodeResource) {
|
|
|
|
+ this.nodeResource = nodeResource;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public Set<NodeLabel> getLabels() {
|
|
|
|
+ return labels;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public void setLabels(Set<NodeLabel> labels) {
|
|
|
|
+ this.labels = labels;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|