YARN-10550. Decouple NM runner logic from SLSRunner. Contributed by Szilard Nemeth
This commit is contained in:
parent
6e00a799e7
commit
ab8c360620
|
@ -63,7 +63,6 @@ public class AMRunner {
|
||||||
private Map<String, Class> amClassMap;
|
private Map<String, Class> amClassMap;
|
||||||
private TraceType inputType;
|
private TraceType inputType;
|
||||||
private String[] inputTraces;
|
private String[] inputTraces;
|
||||||
private SynthTraceJobProducer stjp;
|
|
||||||
private TaskRunner runner;
|
private TaskRunner runner;
|
||||||
private SLSRunner slsRunner;
|
private SLSRunner slsRunner;
|
||||||
private int numAMs, numTasks;
|
private int numAMs, numTasks;
|
||||||
|
@ -148,16 +147,15 @@ public class AMRunner {
|
||||||
private void startAMFromSynthGenerator() throws YarnException, IOException {
|
private void startAMFromSynthGenerator() throws YarnException, IOException {
|
||||||
Configuration localConf = new Configuration();
|
Configuration localConf = new Configuration();
|
||||||
localConf.set("fs.defaultFS", "file:///");
|
localConf.set("fs.defaultFS", "file:///");
|
||||||
// if we use the nodeFile this could have been not initialized yet.
|
//if we use the nodeFile this could have been not initialized yet.
|
||||||
if (stjp == null) {
|
if (slsRunner.getStjp() == null) {
|
||||||
stjp = new SynthTraceJobProducer(conf, new Path(inputTraces[0]));
|
slsRunner.setStjp(new SynthTraceJobProducer(conf, new Path(inputTraces[0])));
|
||||||
slsRunner.setStjp(stjp);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
SynthJob job;
|
SynthJob job;
|
||||||
// we use stjp, a reference to the job producer instantiated during node
|
// we use stjp, a reference to the job producer instantiated during node
|
||||||
// creation
|
// creation
|
||||||
while ((job = (SynthJob) stjp.getNextJob()) != null) {
|
while ((job = (SynthJob) slsRunner.getStjp().getNextJob()) != null) {
|
||||||
ReservationId reservationId = null;
|
ReservationId reservationId = null;
|
||||||
if (job.hasDeadline()) {
|
if (job.hasDeadline()) {
|
||||||
reservationId = ReservationId
|
reservationId = ReservationId
|
||||||
|
|
|
@ -0,0 +1,238 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.yarn.sls;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
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.ResourceManager;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
||||||
|
import org.apache.hadoop.yarn.sls.SLSRunner.TraceType;
|
||||||
|
import org.apache.hadoop.yarn.sls.conf.SLSConfiguration;
|
||||||
|
import org.apache.hadoop.yarn.sls.nodemanager.NMSimulator;
|
||||||
|
import org.apache.hadoop.yarn.sls.scheduler.TaskRunner;
|
||||||
|
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;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
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;
|
||||||
|
|
||||||
|
public class NMRunner {
|
||||||
|
private static final Logger LOG = LoggerFactory.getLogger(NMRunner.class);
|
||||||
|
|
||||||
|
// other simulation information
|
||||||
|
private int numNMs, numRacks;
|
||||||
|
|
||||||
|
// NM simulator
|
||||||
|
private Map<NodeId, NMSimulator> nmMap;
|
||||||
|
private Resource nodeManagerResource;
|
||||||
|
private String nodeFile;
|
||||||
|
private TaskRunner taskRunner;
|
||||||
|
private Configuration conf;
|
||||||
|
private ResourceManager rm;
|
||||||
|
private String tableMapping;
|
||||||
|
private int threadPoolSize;
|
||||||
|
private TraceType inputType;
|
||||||
|
private String[] inputTraces;
|
||||||
|
private SynthTraceJobProducer stjp;
|
||||||
|
|
||||||
|
public NMRunner(TaskRunner taskRunner, Configuration conf, ResourceManager rm, String tableMapping, int threadPoolSize) {
|
||||||
|
this.taskRunner = taskRunner;
|
||||||
|
this.conf = conf;
|
||||||
|
this.rm = rm;
|
||||||
|
this.tableMapping = tableMapping;
|
||||||
|
this.threadPoolSize = threadPoolSize;
|
||||||
|
this.nmMap = new ConcurrentHashMap<>();
|
||||||
|
this.nodeManagerResource = getNodeManagerResourceFromConf();
|
||||||
|
}
|
||||||
|
|
||||||
|
public void startNM() throws YarnException, IOException,
|
||||||
|
InterruptedException {
|
||||||
|
// nm configuration
|
||||||
|
int heartbeatInterval = conf.getInt(
|
||||||
|
SLSConfiguration.NM_HEARTBEAT_INTERVAL_MS,
|
||||||
|
SLSConfiguration.NM_HEARTBEAT_INTERVAL_MS_DEFAULT);
|
||||||
|
float resourceUtilizationRatio = conf.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<SLSRunner.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(conf, 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, tableMapping);
|
||||||
|
|
||||||
|
// create NM simulators
|
||||||
|
Random random = new Random();
|
||||||
|
Set<String> rackSet = ConcurrentHashMap.newKeySet();
|
||||||
|
int threadPoolSize = Math.max(this.threadPoolSize,
|
||||||
|
SLSConfiguration.RUNNER_POOL_SIZE_DEFAULT);
|
||||||
|
ExecutorService executorService = Executors.
|
||||||
|
newFixedThreadPool(threadPoolSize);
|
||||||
|
for (SLSRunner.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, rm, resourceUtilizationRatio, nodeLabels);
|
||||||
|
nmMap.put(nm.getNode().getNodeID(), nm);
|
||||||
|
taskRunner.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();
|
||||||
|
}
|
||||||
|
|
||||||
|
void waitForNodesRunning() throws InterruptedException {
|
||||||
|
long startTimeMS = System.currentTimeMillis();
|
||||||
|
while (true) {
|
||||||
|
int numRunningNodes = 0;
|
||||||
|
for (RMNode node : rm.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);
|
||||||
|
}
|
||||||
|
|
||||||
|
private Resource getNodeManagerResourceFromConf() {
|
||||||
|
Resource resource = Resources.createResource(0);
|
||||||
|
ResourceInformation[] infors = ResourceUtils.getResourceTypesArray();
|
||||||
|
for (ResourceInformation info : infors) {
|
||||||
|
long value;
|
||||||
|
if (info.getName().equals(ResourceInformation.MEMORY_URI)) {
|
||||||
|
value = conf.getInt(SLSConfiguration.NM_MEMORY_MB,
|
||||||
|
SLSConfiguration.NM_MEMORY_MB_DEFAULT);
|
||||||
|
} else if (info.getName().equals(ResourceInformation.VCORES_URI)) {
|
||||||
|
value = conf.getInt(SLSConfiguration.NM_VCORES,
|
||||||
|
SLSConfiguration.NM_VCORES_DEFAULT);
|
||||||
|
} else {
|
||||||
|
value = conf.getLong(SLSConfiguration.NM_PREFIX +
|
||||||
|
info.getName(), SLSConfiguration.NM_RESOURCE_DEFAULT);
|
||||||
|
}
|
||||||
|
|
||||||
|
resource.setResourceValue(info.getName(), value);
|
||||||
|
}
|
||||||
|
|
||||||
|
return resource;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setNodeFile(String nodeFile) {
|
||||||
|
this.nodeFile = nodeFile;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
public void setInputType(TraceType inputType) {
|
||||||
|
this.inputType = inputType;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setInputTraces(String[] inputTraces) {
|
||||||
|
this.inputTraces = inputTraces;
|
||||||
|
}
|
||||||
|
|
||||||
|
public int getNumNMs() {
|
||||||
|
return numNMs;
|
||||||
|
}
|
||||||
|
|
||||||
|
public int getNumRacks() {
|
||||||
|
return numRacks;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Resource getNodeManagerResource() {
|
||||||
|
return nodeManagerResource;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Map<NodeId, NMSimulator> getNmMap() {
|
||||||
|
return nmMap;
|
||||||
|
}
|
||||||
|
|
||||||
|
public SynthTraceJobProducer getStjp() {
|
||||||
|
return stjp;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setTableMapping(String tableMapping) {
|
||||||
|
this.tableMapping = tableMapping;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setRm(ResourceManager rm) {
|
||||||
|
this.rm = rm;
|
||||||
|
}
|
||||||
|
}
|
|
@ -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.ApplicationId;
|
||||||
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.NodeLabel;
|
||||||
import org.apache.hadoop.yarn.api.records.NodeState;
|
|
||||||
import org.apache.hadoop.yarn.api.records.Resource;
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
import org.apache.hadoop.yarn.api.records.ResourceInformation;
|
import org.apache.hadoop.yarn.api.records.ResourceInformation;
|
||||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
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.appmaster.AMSimulator;
|
||||||
import org.apache.hadoop.yarn.sls.conf.SLSConfiguration;
|
import org.apache.hadoop.yarn.sls.conf.SLSConfiguration;
|
||||||
import org.apache.hadoop.yarn.sls.nodemanager.NMSimulator;
|
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.TaskRunner;
|
||||||
import org.apache.hadoop.yarn.sls.scheduler.Tracker;
|
import org.apache.hadoop.yarn.sls.scheduler.Tracker;
|
||||||
import org.apache.hadoop.yarn.sls.synthetic.SynthTraceJobProducer;
|
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.apache.hadoop.yarn.util.resource.Resources;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
@ -85,32 +81,19 @@ import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Random;
|
|
||||||
import java.util.Set;
|
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
|
@Private
|
||||||
@Unstable
|
@Unstable
|
||||||
public class SLSRunner extends Configured implements Tool {
|
public class SLSRunner extends Configured implements Tool {
|
||||||
private static TaskRunner runner = new TaskRunner();
|
private static TaskRunner runner = new TaskRunner();
|
||||||
private String[] inputTraces;
|
private String[] inputTraces;
|
||||||
private int poolSize;
|
|
||||||
|
|
||||||
// NM simulator
|
|
||||||
private Map<NodeId, NMSimulator> nmMap;
|
|
||||||
private Resource nodeManagerResource;
|
|
||||||
private String nodeFile;
|
|
||||||
|
|
||||||
// metrics
|
// metrics
|
||||||
private boolean printSimulation;
|
private boolean printSimulation;
|
||||||
|
|
||||||
// other simulation information
|
private final static Map<String, Object> simulateInfoMap =
|
||||||
private int numNMs, numRacks;
|
new HashMap<>();
|
||||||
|
|
||||||
private final static Map<String, Object> simulateInfoMap = new HashMap<>();
|
|
||||||
|
|
||||||
// logger
|
// logger
|
||||||
public final static Logger LOG = LoggerFactory.getLogger(SLSRunner.class);
|
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 static boolean exitAtTheFinish = false;
|
||||||
private AMRunner amRunner;
|
private AMRunner amRunner;
|
||||||
private RMRunner rmRunner;
|
private RMRunner rmRunner;
|
||||||
|
private NMRunner nmRunner;
|
||||||
|
|
||||||
|
private SynthTraceJobProducer stjp;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The type of trace in input.
|
* 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 =
|
public static final String NETWORK_NEGATIVE_CACHE_TTL =
|
||||||
"networkaddress.cache.negative.ttl";
|
"networkaddress.cache.negative.ttl";
|
||||||
|
|
||||||
private TraceType inputType;
|
|
||||||
private SynthTraceJobProducer stjp;
|
|
||||||
|
|
||||||
public static int getRemainingApps() {
|
public static int getRemainingApps() {
|
||||||
return AMRunner.REMAINING_APPS;
|
return AMRunner.REMAINING_APPS;
|
||||||
}
|
}
|
||||||
|
|
||||||
public SLSRunner() throws ClassNotFoundException {
|
public SLSRunner() throws ClassNotFoundException, YarnException {
|
||||||
Configuration tempConf = new Configuration(false);
|
Configuration tempConf = new Configuration(false);
|
||||||
init(tempConf);
|
init(tempConf);
|
||||||
}
|
}
|
||||||
|
|
||||||
public SLSRunner(Configuration tempConf) throws ClassNotFoundException {
|
public SLSRunner(Configuration tempConf) throws ClassNotFoundException, YarnException {
|
||||||
init(tempConf);
|
init(tempConf);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -156,43 +139,31 @@ public class SLSRunner extends Configured implements Tool {
|
||||||
super.setConf(conf);
|
super.setConf(conf);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void init(Configuration tempConf) throws ClassNotFoundException {
|
private void init(Configuration tempConf) throws ClassNotFoundException, YarnException {
|
||||||
// runner configuration
|
// runner configuration
|
||||||
setConf(tempConf);
|
setConf(tempConf);
|
||||||
|
|
||||||
nmMap = new ConcurrentHashMap<>();
|
int poolSize = tempConf.getInt(SLSConfiguration.RUNNER_POOL_SIZE,
|
||||||
amRunner = new AMRunner(runner, this);
|
|
||||||
rmRunner = new RMRunner(tempConf, this);
|
|
||||||
|
|
||||||
// runner
|
|
||||||
poolSize = tempConf.getInt(SLSConfiguration.RUNNER_POOL_SIZE,
|
|
||||||
SLSConfiguration.RUNNER_POOL_SIZE_DEFAULT);
|
SLSConfiguration.RUNNER_POOL_SIZE_DEFAULT);
|
||||||
SLSRunner.runner.setQueueSize(poolSize);
|
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);
|
amRunner.init(tempConf);
|
||||||
nodeManagerResource = getNodeManagerResource();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private Resource getNodeManagerResource() {
|
private SynthTraceJobProducer getSynthJobTraceProducer() throws YarnException {
|
||||||
Resource resource = Resources.createResource(0);
|
// if we use the nodeFile this could have been not initialized yet.
|
||||||
ResourceInformation[] infors = ResourceUtils.getResourceTypesArray();
|
if (nmRunner.getStjp() != null) {
|
||||||
for (ResourceInformation info : infors) {
|
return nmRunner.getStjp();
|
||||||
long value;
|
} else {
|
||||||
if (info.getName().equals(ResourceInformation.MEMORY_URI)) {
|
try {
|
||||||
value = getConf().getInt(SLSConfiguration.NM_MEMORY_MB,
|
return new SynthTraceJobProducer(getConf(), new Path(inputTraces[0]));
|
||||||
SLSConfiguration.NM_MEMORY_MB_DEFAULT);
|
} catch (IOException e) {
|
||||||
} else if (info.getName().equals(ResourceInformation.VCORES_URI)) {
|
throw new YarnException("Failed to initialize SynthTraceJobProducer", e);
|
||||||
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;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -213,29 +184,37 @@ public class SLSRunner extends Configured implements Tool {
|
||||||
*/
|
*/
|
||||||
public void setSimulationParams(TraceType inType, String[] inTraces,
|
public void setSimulationParams(TraceType inType, String[] inTraces,
|
||||||
String nodes, String metricsOutputDir, Set<String> trackApps,
|
String nodes, String metricsOutputDir, Set<String> trackApps,
|
||||||
boolean printsimulation) {
|
boolean printsimulation) throws YarnException {
|
||||||
|
|
||||||
this.inputType = inType;
|
|
||||||
this.inputTraces = inTraces.clone();
|
this.inputTraces = inTraces.clone();
|
||||||
this.amRunner.setInputType(this.inputType);
|
this.amRunner.setInputType(inType);
|
||||||
this.amRunner.setInputTraces(this.inputTraces);
|
this.amRunner.setInputTraces(this.inputTraces);
|
||||||
this.amRunner.setTrackedApps(trackApps);
|
this.amRunner.setTrackedApps(trackApps);
|
||||||
this.nodeFile = nodes;
|
this.nmRunner.setNodeFile(nodes);
|
||||||
|
this.nmRunner.setInputType(inType);
|
||||||
|
this.nmRunner.setInputTraces(this.inputTraces);
|
||||||
this.printSimulation = printsimulation;
|
this.printSimulation = printsimulation;
|
||||||
this.rmRunner.setMetricsOutputDir(metricsOutputDir);
|
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,
|
public void start() throws IOException, ClassNotFoundException, YarnException,
|
||||||
InterruptedException {
|
InterruptedException {
|
||||||
|
|
||||||
enableDNSCaching(getConf());
|
enableDNSCaching(getConf());
|
||||||
|
|
||||||
// start resource manager
|
// start resource manager
|
||||||
rmRunner.startRM();
|
rmRunner.startRM();
|
||||||
|
nmRunner.setRm(rmRunner.getRm());
|
||||||
amRunner.setResourceManager(rmRunner.getRm());
|
amRunner.setResourceManager(rmRunner.getRm());
|
||||||
|
|
||||||
// start node managers
|
// start node managers
|
||||||
startNM();
|
nmRunner.startNM();
|
||||||
// start application masters
|
// start application masters
|
||||||
amRunner.startAM();
|
amRunner.startAM();
|
||||||
|
|
||||||
|
@ -248,7 +227,7 @@ public class SLSRunner extends Configured implements Tool {
|
||||||
// print out simulation info
|
// print out simulation info
|
||||||
printSimulationInfo();
|
printSimulationInfo();
|
||||||
// blocked until all nodes RUNNING
|
// blocked until all nodes RUNNING
|
||||||
waitForNodesRunning();
|
nmRunner.waitForNodesRunning();
|
||||||
// starting the runner once everything is ready to go,
|
// starting the runner once everything is ready to go,
|
||||||
runner.start();
|
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() {
|
Resource getDefaultContainerResource() {
|
||||||
int containerMemory = getConf().getInt(SLSConfiguration.CONTAINER_MEMORY_MB,
|
int containerMemory = getConf().getInt(SLSConfiguration.CONTAINER_MEMORY_MB,
|
||||||
SLSConfiguration.CONTAINER_MEMORY_MB_DEFAULT);
|
SLSConfiguration.CONTAINER_MEMORY_MB_DEFAULT);
|
||||||
|
@ -392,7 +273,7 @@ public class SLSRunner extends Configured implements Tool {
|
||||||
LOG.info("------------------------------------");
|
LOG.info("------------------------------------");
|
||||||
LOG.info("# nodes = {}, # racks = {}, capacity " +
|
LOG.info("# nodes = {}, # racks = {}, capacity " +
|
||||||
"of each node {}.",
|
"of each node {}.",
|
||||||
numNMs, numRacks, nodeManagerResource);
|
nmRunner.getNumNMs(), nmRunner.getNumRacks(), nmRunner.getNodeManagerResource());
|
||||||
LOG.info("------------------------------------");
|
LOG.info("------------------------------------");
|
||||||
// job
|
// job
|
||||||
LOG.info("# applications = {}, # total " +
|
LOG.info("# applications = {}, # total " +
|
||||||
|
@ -416,12 +297,12 @@ public class SLSRunner extends Configured implements Tool {
|
||||||
LOG.info("------------------------------------");
|
LOG.info("------------------------------------");
|
||||||
}
|
}
|
||||||
// package these information in the simulateInfoMap used by other places
|
// package these information in the simulateInfoMap used by other places
|
||||||
simulateInfoMap.put("Number of racks", numRacks);
|
simulateInfoMap.put("Number of racks", nmRunner.getNumRacks());
|
||||||
simulateInfoMap.put("Number of nodes", numNMs);
|
simulateInfoMap.put("Number of nodes", nmRunner.getNumNMs());
|
||||||
simulateInfoMap.put("Node memory (MB)",
|
simulateInfoMap.put("Node memory (MB)",
|
||||||
nodeManagerResource.getResourceValue(ResourceInformation.MEMORY_URI));
|
nmRunner.getNodeManagerResource().getResourceValue(ResourceInformation.MEMORY_URI));
|
||||||
simulateInfoMap.put("Node VCores",
|
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 applications", numAMs);
|
||||||
simulateInfoMap.put("Number of tasks", numTasks);
|
simulateInfoMap.put("Number of tasks", numTasks);
|
||||||
simulateInfoMap.put("Average tasks per applicaion",
|
simulateInfoMap.put("Average tasks per applicaion",
|
||||||
|
@ -434,7 +315,7 @@ public class SLSRunner extends Configured implements Tool {
|
||||||
}
|
}
|
||||||
|
|
||||||
public Map<NodeId, NMSimulator> getNmMap() {
|
public Map<NodeId, NMSimulator> getNmMap() {
|
||||||
return nmMap;
|
return nmRunner.getNmMap();
|
||||||
}
|
}
|
||||||
|
|
||||||
public static void decreaseRemainingApps() {
|
public static void decreaseRemainingApps() {
|
||||||
|
@ -458,7 +339,6 @@ public class SLSRunner extends Configured implements Tool {
|
||||||
|
|
||||||
public int run(final String[] argv) throws IOException, InterruptedException,
|
public int run(final String[] argv) throws IOException, InterruptedException,
|
||||||
ParseException, ClassNotFoundException, YarnException {
|
ParseException, ClassNotFoundException, YarnException {
|
||||||
|
|
||||||
Options options = new Options();
|
Options options = new Options();
|
||||||
|
|
||||||
// Left for compatibility
|
// Left for compatibility
|
||||||
|
@ -524,7 +404,6 @@ public class SLSRunner extends Configured implements Tool {
|
||||||
case "RUMEN":
|
case "RUMEN":
|
||||||
tempTraceType = TraceType.RUMEN;
|
tempTraceType = TraceType.RUMEN;
|
||||||
break;
|
break;
|
||||||
|
|
||||||
case "SYNTH":
|
case "SYNTH":
|
||||||
tempTraceType = TraceType.SYNTH;
|
tempTraceType = TraceType.SYNTH;
|
||||||
break;
|
break;
|
||||||
|
@ -537,7 +416,7 @@ public class SLSRunner extends Configured implements Tool {
|
||||||
|
|
||||||
setSimulationParams(tempTraceType, inputFiles, tempNodeFile, output,
|
setSimulationParams(tempTraceType, inputFiles, tempNodeFile, output,
|
||||||
trackedJobSet, cmd.hasOption("printsimulation"));
|
trackedJobSet, cmd.hasOption("printsimulation"));
|
||||||
|
|
||||||
start();
|
start();
|
||||||
|
|
||||||
return 0;
|
return 0;
|
||||||
|
|
|
@ -186,7 +186,6 @@ public class NodeInfo {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<Container> pullNewlyIncreasedContainers() {
|
public List<Container> pullNewlyIncreasedContainers() {
|
||||||
// TODO Auto-generated method stub
|
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -176,7 +176,6 @@ public class RMNodeWrapper implements RMNode {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<Container> pullNewlyIncreasedContainers() {
|
public List<Container> pullNewlyIncreasedContainers() {
|
||||||
// TODO Auto-generated method stub
|
|
||||||
return Collections.emptyList();
|
return Collections.emptyList();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -316,7 +316,7 @@ public abstract class SchedulerMetrics {
|
||||||
new Gauge<Long>() {
|
new Gauge<Long>() {
|
||||||
@Override
|
@Override
|
||||||
public Long getValue() {
|
public Long getValue() {
|
||||||
if (scheduler.getRootQueueMetrics() == null) {
|
if (isMetricsAvailable()) {
|
||||||
return 0L;
|
return 0L;
|
||||||
} else {
|
} else {
|
||||||
return scheduler.getRootQueueMetrics().getAllocatedMB();
|
return scheduler.getRootQueueMetrics().getAllocatedMB();
|
||||||
|
@ -328,7 +328,7 @@ public abstract class SchedulerMetrics {
|
||||||
new Gauge<Integer>() {
|
new Gauge<Integer>() {
|
||||||
@Override
|
@Override
|
||||||
public Integer getValue() {
|
public Integer getValue() {
|
||||||
if (scheduler.getRootQueueMetrics() == null) {
|
if (isMetricsAvailable()) {
|
||||||
return 0;
|
return 0;
|
||||||
} else {
|
} else {
|
||||||
return scheduler.getRootQueueMetrics().getAllocatedVirtualCores();
|
return scheduler.getRootQueueMetrics().getAllocatedVirtualCores();
|
||||||
|
@ -340,7 +340,7 @@ public abstract class SchedulerMetrics {
|
||||||
new Gauge<Long>() {
|
new Gauge<Long>() {
|
||||||
@Override
|
@Override
|
||||||
public Long getValue() {
|
public Long getValue() {
|
||||||
if (scheduler.getRootQueueMetrics() == null) {
|
if (isMetricsAvailable()) {
|
||||||
return 0L;
|
return 0L;
|
||||||
} else {
|
} else {
|
||||||
return scheduler.getRootQueueMetrics().getAvailableMB();
|
return scheduler.getRootQueueMetrics().getAvailableMB();
|
||||||
|
@ -352,7 +352,7 @@ public abstract class SchedulerMetrics {
|
||||||
new Gauge<Integer>() {
|
new Gauge<Integer>() {
|
||||||
@Override
|
@Override
|
||||||
public Integer getValue() {
|
public Integer getValue() {
|
||||||
if (scheduler.getRootQueueMetrics() == null) {
|
if (isMetricsAvailable()) {
|
||||||
return 0;
|
return 0;
|
||||||
} else {
|
} else {
|
||||||
return scheduler.getRootQueueMetrics().getAvailableVirtualCores();
|
return scheduler.getRootQueueMetrics().getAvailableVirtualCores();
|
||||||
|
@ -362,6 +362,10 @@ public abstract class SchedulerMetrics {
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private boolean isMetricsAvailable() {
|
||||||
|
return scheduler.getRootQueueMetrics() == null;
|
||||||
|
}
|
||||||
|
|
||||||
private void registerContainerAppNumMetrics() {
|
private void registerContainerAppNumMetrics() {
|
||||||
metrics.register("variable.running.application",
|
metrics.register("variable.running.application",
|
||||||
new Gauge<Integer>() {
|
new Gauge<Integer>() {
|
||||||
|
|
Loading…
Reference in New Issue