|
@@ -62,6 +62,7 @@ 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.api.records.ResourceInformation;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
|
|
@@ -84,6 +85,7 @@ 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.ResourceUtils;
|
|
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
@@ -99,7 +101,7 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
|
|
|
// NM simulator
|
|
|
private HashMap<NodeId, NMSimulator> nmMap;
|
|
|
- private int nmMemoryMB, nmVCores;
|
|
|
+ private Resource nodeManagerResource;
|
|
|
private String nodeFile;
|
|
|
|
|
|
// AM simulator
|
|
@@ -178,6 +180,30 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
amClassMap.put(amType, Class.forName(tempConf.get(key)));
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ nodeManagerResource = getNodeManagerResource();
|
|
|
+ }
|
|
|
+
|
|
|
+ private Resource getNodeManagerResource() {
|
|
|
+ Resource resource = Resources.createResource(0);
|
|
|
+ ResourceInformation[] infors = ResourceUtils.getResourceTypesArray();
|
|
|
+ for (ResourceInformation info : infors) {
|
|
|
+ long value;
|
|
|
+ if (info.getName().equals(ResourceInformation.MEMORY_URI)) {
|
|
|
+ value = getConf().getInt(SLSConfiguration.NM_MEMORY_MB,
|
|
|
+ SLSConfiguration.NM_MEMORY_MB_DEFAULT);
|
|
|
+ } else if (info.getName().equals(ResourceInformation.VCORES_URI)) {
|
|
|
+ value = getConf().getInt(SLSConfiguration.NM_VCORES,
|
|
|
+ SLSConfiguration.NM_VCORES_DEFAULT);
|
|
|
+ } else {
|
|
|
+ value = getConf().getLong(SLSConfiguration.NM_PREFIX +
|
|
|
+ info.getName(), SLSConfiguration.NM_RESOURCE_DEFAULT);
|
|
|
+ }
|
|
|
+
|
|
|
+ resource.setResourceValue(info.getName(), value);
|
|
|
+ }
|
|
|
+
|
|
|
+ return resource;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -261,10 +287,6 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
|
|
|
private void startNM() throws YarnException, IOException {
|
|
|
// nm configuration
|
|
|
- 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);
|
|
@@ -304,7 +326,7 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
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),
|
|
|
+ nm.init(hostName, nodeManagerResource, random.nextInt(heartbeatInterval),
|
|
|
heartbeatInterval, rm);
|
|
|
nmMap.put(nm.getNode().getNodeID(), nm);
|
|
|
runner.schedule(nm);
|
|
@@ -460,18 +482,7 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
+ " to 0!");
|
|
|
}
|
|
|
|
|
|
- Resource res = getDefaultContainerResource();
|
|
|
- if (jsonTask.containsKey(SLSConfiguration.TASK_MEMORY)) {
|
|
|
- int containerMemory = Integer.parseInt(
|
|
|
- jsonTask.get(SLSConfiguration.TASK_MEMORY).toString());
|
|
|
- res.setMemorySize(containerMemory);
|
|
|
- }
|
|
|
-
|
|
|
- if (jsonTask.containsKey(SLSConfiguration.CONTAINER_VCORES)) {
|
|
|
- int containerVCores = Integer.parseInt(
|
|
|
- jsonTask.get(SLSConfiguration.CONTAINER_VCORES).toString());
|
|
|
- res.setVirtualCores(containerVCores);
|
|
|
- }
|
|
|
+ Resource res = getResourceForContainer(jsonTask);
|
|
|
|
|
|
int priority = DEFAULT_MAPPER_PRIORITY;
|
|
|
if (jsonTask.containsKey(SLSConfiguration.TASK_PRIORITY)) {
|
|
@@ -500,6 +511,21 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
return containers;
|
|
|
}
|
|
|
|
|
|
+ private Resource getResourceForContainer(Map jsonTask) {
|
|
|
+ Resource res = getDefaultContainerResource();
|
|
|
+ ResourceInformation[] infors = ResourceUtils.getResourceTypesArray();
|
|
|
+ for (ResourceInformation info : infors) {
|
|
|
+ if (jsonTask.containsKey(SLSConfiguration.TASK_PREFIX + info.getName())) {
|
|
|
+ long value = Long.parseLong(
|
|
|
+ jsonTask.get(SLSConfiguration.TASK_PREFIX + info.getName())
|
|
|
+ .toString());
|
|
|
+ res.setResourceValue(info.getName(), value);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ return res;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Parse workload from a rumen trace file.
|
|
|
*/
|
|
@@ -717,15 +743,15 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
return amContainerResource;
|
|
|
}
|
|
|
|
|
|
- if (jsonJob.containsKey(SLSConfiguration.AM_MEMORY)) {
|
|
|
- amContainerResource.setMemorySize(
|
|
|
- Long.parseLong(jsonJob.get(SLSConfiguration.AM_MEMORY).toString()));
|
|
|
+ ResourceInformation[] infors = ResourceUtils.getResourceTypesArray();
|
|
|
+ for (ResourceInformation info : infors) {
|
|
|
+ String key = SLSConfiguration.JOB_AM_PREFIX + info.getName();
|
|
|
+ if (jsonJob.containsKey(key)) {
|
|
|
+ long value = Long.parseLong(jsonJob.get(key).toString());
|
|
|
+ amContainerResource.setResourceValue(info.getName(), value);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
- if (jsonJob.containsKey(SLSConfiguration.AM_VCORES)) {
|
|
|
- amContainerResource.setVirtualCores(
|
|
|
- Integer.parseInt(jsonJob.get(SLSConfiguration.AM_VCORES).toString()));
|
|
|
- }
|
|
|
return amContainerResource;
|
|
|
}
|
|
|
|
|
@@ -777,8 +803,8 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
// node
|
|
|
LOG.info("------------------------------------");
|
|
|
LOG.info("# nodes = {}, # racks = {}, capacity " +
|
|
|
- "of each node {} MB memory and {} vcores.",
|
|
|
- numNMs, numRacks, nmMemoryMB, nmVCores);
|
|
|
+ "of each node {}.",
|
|
|
+ numNMs, numRacks, nodeManagerResource);
|
|
|
LOG.info("------------------------------------");
|
|
|
// job
|
|
|
LOG.info("# applications = {}, # total " +
|
|
@@ -804,8 +830,10 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
// package these information in the simulateInfoMap used by other places
|
|
|
simulateInfoMap.put("Number of racks", numRacks);
|
|
|
simulateInfoMap.put("Number of nodes", numNMs);
|
|
|
- simulateInfoMap.put("Node memory (MB)", nmMemoryMB);
|
|
|
- simulateInfoMap.put("Node VCores", nmVCores);
|
|
|
+ simulateInfoMap.put("Node memory (MB)",
|
|
|
+ nodeManagerResource.getResourceValue(ResourceInformation.MEMORY_URI));
|
|
|
+ simulateInfoMap.put("Node VCores",
|
|
|
+ nodeManagerResource.getResourceValue(ResourceInformation.VCORES_URI));
|
|
|
simulateInfoMap.put("Number of applications", numAMs);
|
|
|
simulateInfoMap.put("Number of tasks", numTasks);
|
|
|
simulateInfoMap.put("Average tasks per applicaion",
|