|
@@ -59,11 +59,9 @@ import org.apache.hadoop.util.ToolRunner;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
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.Resource;
|
|
|
import org.apache.hadoop.yarn.api.records.ResourceInformation;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
|
|
import org.apache.hadoop.yarn.sls.appmaster.AMSimulator;
|
|
|
import org.apache.hadoop.yarn.sls.conf.SLSConfiguration;
|
|
|
import org.apache.hadoop.yarn.sls.nodemanager.NMSimulator;
|
|
@@ -71,8 +69,6 @@ import org.apache.hadoop.yarn.sls.scheduler.SchedulerWrapper;
|
|
|
import org.apache.hadoop.yarn.sls.scheduler.TaskRunner;
|
|
|
import org.apache.hadoop.yarn.sls.scheduler.Tracker;
|
|
|
import org.apache.hadoop.yarn.sls.synthetic.SynthTraceJobProducer;
|
|
|
-import org.apache.hadoop.yarn.sls.utils.SLSUtils;
|
|
|
-import org.apache.hadoop.yarn.util.resource.ResourceUtils;
|
|
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
@@ -85,32 +81,19 @@ import java.util.Collections;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.HashSet;
|
|
|
import java.util.Map;
|
|
|
-import java.util.Random;
|
|
|
import java.util.Set;
|
|
|
-import java.util.concurrent.ConcurrentHashMap;
|
|
|
-import java.util.concurrent.ExecutorService;
|
|
|
-import java.util.concurrent.Executors;
|
|
|
-import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
@Private
|
|
|
@Unstable
|
|
|
public class SLSRunner extends Configured implements Tool {
|
|
|
private static TaskRunner runner = new TaskRunner();
|
|
|
private String[] inputTraces;
|
|
|
- private int poolSize;
|
|
|
-
|
|
|
- // NM simulator
|
|
|
- private Map<NodeId, NMSimulator> nmMap;
|
|
|
- private Resource nodeManagerResource;
|
|
|
- private String nodeFile;
|
|
|
|
|
|
// metrics
|
|
|
private boolean printSimulation;
|
|
|
|
|
|
- // other simulation information
|
|
|
- private int numNMs, numRacks;
|
|
|
-
|
|
|
- private final static Map<String, Object> simulateInfoMap = new HashMap<>();
|
|
|
+ private final static Map<String, Object> simulateInfoMap =
|
|
|
+ new HashMap<>();
|
|
|
|
|
|
// logger
|
|
|
public final static Logger LOG = LoggerFactory.getLogger(SLSRunner.class);
|
|
@@ -118,6 +101,9 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
private static boolean exitAtTheFinish = false;
|
|
|
private AMRunner amRunner;
|
|
|
private RMRunner rmRunner;
|
|
|
+ private NMRunner nmRunner;
|
|
|
+
|
|
|
+ private SynthTraceJobProducer stjp;
|
|
|
|
|
|
/**
|
|
|
* The type of trace in input.
|
|
@@ -130,19 +116,16 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
public static final String NETWORK_NEGATIVE_CACHE_TTL =
|
|
|
"networkaddress.cache.negative.ttl";
|
|
|
|
|
|
- private TraceType inputType;
|
|
|
- private SynthTraceJobProducer stjp;
|
|
|
-
|
|
|
public static int getRemainingApps() {
|
|
|
return AMRunner.REMAINING_APPS;
|
|
|
}
|
|
|
|
|
|
- public SLSRunner() throws ClassNotFoundException {
|
|
|
+ public SLSRunner() throws ClassNotFoundException, YarnException {
|
|
|
Configuration tempConf = new Configuration(false);
|
|
|
init(tempConf);
|
|
|
}
|
|
|
|
|
|
- public SLSRunner(Configuration tempConf) throws ClassNotFoundException {
|
|
|
+ public SLSRunner(Configuration tempConf) throws ClassNotFoundException, YarnException {
|
|
|
init(tempConf);
|
|
|
}
|
|
|
|
|
@@ -156,43 +139,31 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
super.setConf(conf);
|
|
|
}
|
|
|
|
|
|
- private void init(Configuration tempConf) throws ClassNotFoundException {
|
|
|
+ private void init(Configuration tempConf) throws ClassNotFoundException, YarnException {
|
|
|
// runner configuration
|
|
|
setConf(tempConf);
|
|
|
-
|
|
|
- nmMap = new ConcurrentHashMap<>();
|
|
|
- amRunner = new AMRunner(runner, this);
|
|
|
- rmRunner = new RMRunner(tempConf, this);
|
|
|
-
|
|
|
- // runner
|
|
|
- poolSize = tempConf.getInt(SLSConfiguration.RUNNER_POOL_SIZE,
|
|
|
+
|
|
|
+ int poolSize = tempConf.getInt(SLSConfiguration.RUNNER_POOL_SIZE,
|
|
|
SLSConfiguration.RUNNER_POOL_SIZE_DEFAULT);
|
|
|
SLSRunner.runner.setQueueSize(poolSize);
|
|
|
|
|
|
+ rmRunner = new RMRunner(getConf(), this);
|
|
|
+ nmRunner = new NMRunner(runner, getConf(), rmRunner.getRm(), rmRunner.getTableMapping(), poolSize);
|
|
|
+ amRunner = new AMRunner(runner, this);
|
|
|
amRunner.init(tempConf);
|
|
|
- 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);
|
|
|
+ private SynthTraceJobProducer getSynthJobTraceProducer() throws YarnException {
|
|
|
+ // if we use the nodeFile this could have been not initialized yet.
|
|
|
+ if (nmRunner.getStjp() != null) {
|
|
|
+ return nmRunner.getStjp();
|
|
|
+ } else {
|
|
|
+ try {
|
|
|
+ return new SynthTraceJobProducer(getConf(), new Path(inputTraces[0]));
|
|
|
+ } catch (IOException e) {
|
|
|
+ throw new YarnException("Failed to initialize SynthTraceJobProducer", e);
|
|
|
}
|
|
|
-
|
|
|
- resource.setResourceValue(info.getName(), value);
|
|
|
}
|
|
|
-
|
|
|
- return resource;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -213,29 +184,37 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
*/
|
|
|
public void setSimulationParams(TraceType inType, String[] inTraces,
|
|
|
String nodes, String metricsOutputDir, Set<String> trackApps,
|
|
|
- boolean printsimulation) {
|
|
|
-
|
|
|
- this.inputType = inType;
|
|
|
+ boolean printsimulation) throws YarnException {
|
|
|
this.inputTraces = inTraces.clone();
|
|
|
- this.amRunner.setInputType(this.inputType);
|
|
|
+ this.amRunner.setInputType(inType);
|
|
|
this.amRunner.setInputTraces(this.inputTraces);
|
|
|
this.amRunner.setTrackedApps(trackApps);
|
|
|
- this.nodeFile = nodes;
|
|
|
+ this.nmRunner.setNodeFile(nodes);
|
|
|
+ this.nmRunner.setInputType(inType);
|
|
|
+ this.nmRunner.setInputTraces(this.inputTraces);
|
|
|
this.printSimulation = printsimulation;
|
|
|
this.rmRunner.setMetricsOutputDir(metricsOutputDir);
|
|
|
- this.rmRunner.setTableMapping(metricsOutputDir + "/tableMapping.csv");
|
|
|
+ String tableMapping = metricsOutputDir + "/tableMapping.csv";
|
|
|
+ this.rmRunner.setTableMapping(tableMapping);
|
|
|
+ this.nmRunner.setTableMapping(tableMapping);
|
|
|
+
|
|
|
+ //We need this.inputTraces to set before creating SynthTraceJobProducer
|
|
|
+ if (inType == TraceType.SYNTH) {
|
|
|
+ this.stjp = getSynthJobTraceProducer();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
public void start() throws IOException, ClassNotFoundException, YarnException,
|
|
|
InterruptedException {
|
|
|
-
|
|
|
enableDNSCaching(getConf());
|
|
|
|
|
|
// start resource manager
|
|
|
rmRunner.startRM();
|
|
|
+ nmRunner.setRm(rmRunner.getRm());
|
|
|
amRunner.setResourceManager(rmRunner.getRm());
|
|
|
+
|
|
|
// start node managers
|
|
|
- startNM();
|
|
|
+ nmRunner.startNM();
|
|
|
// start application masters
|
|
|
amRunner.startAM();
|
|
|
|
|
@@ -248,7 +227,7 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
// print out simulation info
|
|
|
printSimulationInfo();
|
|
|
// blocked until all nodes RUNNING
|
|
|
- waitForNodesRunning();
|
|
|
+ nmRunner.waitForNodesRunning();
|
|
|
// starting the runner once everything is ready to go,
|
|
|
runner.start();
|
|
|
}
|
|
@@ -270,104 +249,6 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private void startNM() throws YarnException, IOException,
|
|
|
- InterruptedException {
|
|
|
- // nm configuration
|
|
|
- int heartbeatInterval = getConf().getInt(
|
|
|
- SLSConfiguration.NM_HEARTBEAT_INTERVAL_MS,
|
|
|
- SLSConfiguration.NM_HEARTBEAT_INTERVAL_MS_DEFAULT);
|
|
|
- float resourceUtilizationRatio = getConf().getFloat(
|
|
|
- SLSConfiguration.NM_RESOURCE_UTILIZATION_RATIO,
|
|
|
- SLSConfiguration.NM_RESOURCE_UTILIZATION_RATIO_DEFAULT);
|
|
|
- // nm information (fetch from topology file, or from sls/rumen json file)
|
|
|
- Set<NodeDetails> nodeSet = null;
|
|
|
- if (nodeFile.isEmpty()) {
|
|
|
- for (String inputTrace : inputTraces) {
|
|
|
- switch (inputType) {
|
|
|
- case SLS:
|
|
|
- nodeSet = SLSUtils.parseNodesFromSLSTrace(inputTrace);
|
|
|
- break;
|
|
|
- case RUMEN:
|
|
|
- nodeSet = SLSUtils.parseNodesFromRumenTrace(inputTrace);
|
|
|
- break;
|
|
|
- case SYNTH:
|
|
|
- stjp = new SynthTraceJobProducer(getConf(), new Path(inputTraces[0]));
|
|
|
- nodeSet = 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 = SLSUtils.parseNodesFromNodeFile(nodeFile,
|
|
|
- nodeManagerResource);
|
|
|
- }
|
|
|
-
|
|
|
- if (nodeSet == null || nodeSet.isEmpty()) {
|
|
|
- throw new YarnException("No node! Please configure nodes.");
|
|
|
- }
|
|
|
-
|
|
|
- SLSUtils.generateNodeTableMapping(nodeSet, rmRunner.getTableMapping());
|
|
|
-
|
|
|
- // create NM simulators
|
|
|
- Random random = new Random();
|
|
|
- Set<String> rackSet = ConcurrentHashMap.newKeySet();
|
|
|
- int threadPoolSize = Math.max(poolSize,
|
|
|
- SLSConfiguration.RUNNER_POOL_SIZE_DEFAULT);
|
|
|
- ExecutorService executorService = Executors.
|
|
|
- newFixedThreadPool(threadPoolSize);
|
|
|
- for (NodeDetails nodeDetails : nodeSet) {
|
|
|
- executorService.submit(new Runnable() {
|
|
|
- @Override public void run() {
|
|
|
- try {
|
|
|
- // we randomize the heartbeat start time from zero to 1 interval
|
|
|
- NMSimulator nm = new NMSimulator();
|
|
|
- Resource nmResource = nodeManagerResource;
|
|
|
- String hostName = nodeDetails.getHostname();
|
|
|
- if (nodeDetails.getNodeResource() != null) {
|
|
|
- nmResource = nodeDetails.getNodeResource();
|
|
|
- }
|
|
|
- Set<NodeLabel> nodeLabels = nodeDetails.getLabels();
|
|
|
- nm.init(hostName, nmResource,
|
|
|
- random.nextInt(heartbeatInterval),
|
|
|
- heartbeatInterval, rmRunner.getRm(), resourceUtilizationRatio, nodeLabels);
|
|
|
- nmMap.put(nm.getNode().getNodeID(), nm);
|
|
|
- runner.schedule(nm);
|
|
|
- rackSet.add(nm.getNode().getRackName());
|
|
|
- } catch (IOException | YarnException e) {
|
|
|
- LOG.error("Got an error while adding node", e);
|
|
|
- }
|
|
|
- }
|
|
|
- });
|
|
|
- }
|
|
|
- executorService.shutdown();
|
|
|
- executorService.awaitTermination(10, TimeUnit.MINUTES);
|
|
|
- numRacks = rackSet.size();
|
|
|
- numNMs = nmMap.size();
|
|
|
- }
|
|
|
-
|
|
|
- private void waitForNodesRunning() throws InterruptedException {
|
|
|
- long startTimeMS = System.currentTimeMillis();
|
|
|
- while (true) {
|
|
|
- int numRunningNodes = 0;
|
|
|
- for (RMNode node : rmRunner.getRm().getRMContext().getRMNodes().values()) {
|
|
|
- if (node.getState() == NodeState.RUNNING) {
|
|
|
- numRunningNodes++;
|
|
|
- }
|
|
|
- }
|
|
|
- if (numRunningNodes == numNMs) {
|
|
|
- break;
|
|
|
- }
|
|
|
- LOG.info("SLSRunner is waiting for all nodes RUNNING."
|
|
|
- + " {} of {} NMs initialized.", numRunningNodes, numNMs);
|
|
|
- Thread.sleep(1000);
|
|
|
- }
|
|
|
- LOG.info("SLSRunner takes {} ms to launch all nodes.",
|
|
|
- System.currentTimeMillis() - startTimeMS);
|
|
|
- }
|
|
|
-
|
|
|
Resource getDefaultContainerResource() {
|
|
|
int containerMemory = getConf().getInt(SLSConfiguration.CONTAINER_MEMORY_MB,
|
|
|
SLSConfiguration.CONTAINER_MEMORY_MB_DEFAULT);
|
|
@@ -392,7 +273,7 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
LOG.info("------------------------------------");
|
|
|
LOG.info("# nodes = {}, # racks = {}, capacity " +
|
|
|
"of each node {}.",
|
|
|
- numNMs, numRacks, nodeManagerResource);
|
|
|
+ nmRunner.getNumNMs(), nmRunner.getNumRacks(), nmRunner.getNodeManagerResource());
|
|
|
LOG.info("------------------------------------");
|
|
|
// job
|
|
|
LOG.info("# applications = {}, # total " +
|
|
@@ -416,12 +297,12 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
LOG.info("------------------------------------");
|
|
|
}
|
|
|
// package these information in the simulateInfoMap used by other places
|
|
|
- simulateInfoMap.put("Number of racks", numRacks);
|
|
|
- simulateInfoMap.put("Number of nodes", numNMs);
|
|
|
+ simulateInfoMap.put("Number of racks", nmRunner.getNumRacks());
|
|
|
+ simulateInfoMap.put("Number of nodes", nmRunner.getNumNMs());
|
|
|
simulateInfoMap.put("Node memory (MB)",
|
|
|
- nodeManagerResource.getResourceValue(ResourceInformation.MEMORY_URI));
|
|
|
+ nmRunner.getNodeManagerResource().getResourceValue(ResourceInformation.MEMORY_URI));
|
|
|
simulateInfoMap.put("Node VCores",
|
|
|
- nodeManagerResource.getResourceValue(ResourceInformation.VCORES_URI));
|
|
|
+ nmRunner.getNodeManagerResource().getResourceValue(ResourceInformation.VCORES_URI));
|
|
|
simulateInfoMap.put("Number of applications", numAMs);
|
|
|
simulateInfoMap.put("Number of tasks", numTasks);
|
|
|
simulateInfoMap.put("Average tasks per applicaion",
|
|
@@ -434,7 +315,7 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
}
|
|
|
|
|
|
public Map<NodeId, NMSimulator> getNmMap() {
|
|
|
- return nmMap;
|
|
|
+ return nmRunner.getNmMap();
|
|
|
}
|
|
|
|
|
|
public static void decreaseRemainingApps() {
|
|
@@ -458,7 +339,6 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
|
|
|
public int run(final String[] argv) throws IOException, InterruptedException,
|
|
|
ParseException, ClassNotFoundException, YarnException {
|
|
|
-
|
|
|
Options options = new Options();
|
|
|
|
|
|
// Left for compatibility
|
|
@@ -524,7 +404,6 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
case "RUMEN":
|
|
|
tempTraceType = TraceType.RUMEN;
|
|
|
break;
|
|
|
-
|
|
|
case "SYNTH":
|
|
|
tempTraceType = TraceType.SYNTH;
|
|
|
break;
|
|
@@ -537,7 +416,7 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
|
|
|
setSimulationParams(tempTraceType, inputFiles, tempNodeFile, output,
|
|
|
trackedJobSet, cmd.hasOption("printsimulation"));
|
|
|
-
|
|
|
+
|
|
|
start();
|
|
|
|
|
|
return 0;
|