YARN-6423. Queue metrics doesn't work for Fair Scheduler in SLS (yufeigu via rkanter)
This commit is contained in:
parent
e4321ec843
commit
475f933b41
@ -69,12 +69,15 @@
|
|||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
|
||||||
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
||||||
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;
|
||||||
import org.apache.hadoop.yarn.sls.resourcemanager.MockAMLauncher;
|
import org.apache.hadoop.yarn.sls.resourcemanager.MockAMLauncher;
|
||||||
import org.apache.hadoop.yarn.sls.scheduler.*;
|
import org.apache.hadoop.yarn.sls.scheduler.SLSCapacityScheduler;
|
||||||
|
import org.apache.hadoop.yarn.sls.scheduler.TaskRunner;
|
||||||
|
import org.apache.hadoop.yarn.sls.scheduler.SLSFairScheduler;
|
||||||
|
import org.apache.hadoop.yarn.sls.scheduler.ContainerSimulator;
|
||||||
|
import org.apache.hadoop.yarn.sls.scheduler.SchedulerWrapper;
|
||||||
import org.apache.hadoop.yarn.sls.synthetic.SynthJob;
|
import org.apache.hadoop.yarn.sls.synthetic.SynthJob;
|
||||||
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.sls.utils.SLSUtils;
|
||||||
@ -197,10 +200,6 @@ private void startRM() throws ClassNotFoundException, YarnException {
|
|||||||
Configuration rmConf = new YarnConfiguration(getConf());
|
Configuration rmConf = new YarnConfiguration(getConf());
|
||||||
String schedulerClass = rmConf.get(YarnConfiguration.RM_SCHEDULER);
|
String schedulerClass = rmConf.get(YarnConfiguration.RM_SCHEDULER);
|
||||||
|
|
||||||
// For CapacityScheduler we use a sub-classing instead of wrapping
|
|
||||||
// to allow scheduler-specific invocations from monitors to work
|
|
||||||
// this can be used for other schedulers as well if we care to
|
|
||||||
// exercise/track behaviors that are not common to the scheduler api
|
|
||||||
if (Class.forName(schedulerClass) == CapacityScheduler.class) {
|
if (Class.forName(schedulerClass) == CapacityScheduler.class) {
|
||||||
rmConf.set(YarnConfiguration.RM_SCHEDULER,
|
rmConf.set(YarnConfiguration.RM_SCHEDULER,
|
||||||
SLSCapacityScheduler.class.getName());
|
SLSCapacityScheduler.class.getName());
|
||||||
@ -300,91 +299,85 @@ private void waitForNodesRunning() throws InterruptedException {
|
|||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
private void startAM() throws YarnException, IOException {
|
private void startAM() throws YarnException, IOException {
|
||||||
// application/container configuration
|
|
||||||
int heartbeatInterval =
|
|
||||||
getConf().getInt(SLSConfiguration.AM_HEARTBEAT_INTERVAL_MS,
|
|
||||||
SLSConfiguration.AM_HEARTBEAT_INTERVAL_MS_DEFAULT);
|
|
||||||
int containerMemoryMB =
|
|
||||||
getConf().getInt(SLSConfiguration.CONTAINER_MEMORY_MB,
|
|
||||||
SLSConfiguration.CONTAINER_MEMORY_MB_DEFAULT);
|
|
||||||
int containerVCores = getConf().getInt(SLSConfiguration.CONTAINER_VCORES,
|
|
||||||
SLSConfiguration.CONTAINER_VCORES_DEFAULT);
|
|
||||||
Resource containerResource =
|
|
||||||
BuilderUtils.newResource(containerMemoryMB, containerVCores);
|
|
||||||
|
|
||||||
// application workload
|
|
||||||
switch (inputType) {
|
switch (inputType) {
|
||||||
case SLS:
|
case SLS:
|
||||||
startAMFromSLSTraces(containerResource, heartbeatInterval);
|
for (String inputTrace : inputTraces) {
|
||||||
|
startAMFromSLSTrace(inputTrace);
|
||||||
|
}
|
||||||
break;
|
break;
|
||||||
case RUMEN:
|
case RUMEN:
|
||||||
startAMFromRumenTraces(containerResource, heartbeatInterval);
|
long baselineTimeMS = 0;
|
||||||
|
for (String inputTrace : inputTraces) {
|
||||||
|
startAMFromRumenTrace(inputTrace, baselineTimeMS);
|
||||||
|
}
|
||||||
break;
|
break;
|
||||||
case SYNTH:
|
case SYNTH:
|
||||||
startAMFromSynthGenerator(heartbeatInterval);
|
startAMFromSynthGenerator();
|
||||||
break;
|
break;
|
||||||
default:
|
default:
|
||||||
throw new YarnException("Input configuration not recognized, "
|
throw new YarnException("Input configuration not recognized, "
|
||||||
+ "trace type should be SLS, RUMEN, or SYNTH");
|
+ "trace type should be SLS, RUMEN, or SYNTH");
|
||||||
}
|
}
|
||||||
|
|
||||||
numAMs = amMap.size();
|
numAMs = amMap.size();
|
||||||
remainingApps = numAMs;
|
remainingApps = numAMs;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* parse workload information from sls trace files
|
* Parse workload from a SLS trace file.
|
||||||
*/
|
*/
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
private void startAMFromSLSTraces(Resource containerResource,
|
private void startAMFromSLSTrace(String inputTrace) throws IOException {
|
||||||
int heartbeatInterval) throws IOException {
|
|
||||||
// parse from sls traces
|
|
||||||
JsonFactory jsonF = new JsonFactory();
|
JsonFactory jsonF = new JsonFactory();
|
||||||
ObjectMapper mapper = new ObjectMapper();
|
ObjectMapper mapper = new ObjectMapper();
|
||||||
for (String inputTrace : inputTraces) {
|
|
||||||
Reader input =
|
|
||||||
new InputStreamReader(new FileInputStream(inputTrace), "UTF-8");
|
|
||||||
try {
|
|
||||||
Iterator<Map> i =
|
|
||||||
mapper.readValues(jsonF.createParser(input), Map.class);
|
|
||||||
while (i.hasNext()) {
|
|
||||||
Map jsonJob = i.next();
|
|
||||||
|
|
||||||
// load job information
|
try (Reader input = new InputStreamReader(
|
||||||
long jobStartTime =
|
new FileInputStream(inputTrace), "UTF-8")) {
|
||||||
Long.parseLong(jsonJob.get("job.start.ms").toString());
|
Iterator<Map> jobIter = mapper.readValues(
|
||||||
long jobFinishTime =
|
jsonF.createParser(input), Map.class);
|
||||||
Long.parseLong(jsonJob.get("job.end.ms").toString());
|
|
||||||
|
while (jobIter.hasNext()) {
|
||||||
|
try {
|
||||||
|
createAMForJob(jobIter.next());
|
||||||
|
} catch (Exception e) {
|
||||||
|
LOG.error("Failed to create an AM: " + e.getMessage());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
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());
|
||||||
|
|
||||||
String user = (String) jsonJob.get("job.user");
|
String user = (String) jsonJob.get("job.user");
|
||||||
if (user == null) {
|
if (user == null) {
|
||||||
user = "default";
|
user = "default";
|
||||||
}
|
}
|
||||||
|
|
||||||
String queue = jsonJob.get("job.queue.name").toString();
|
String queue = jsonJob.get("job.queue.name").toString();
|
||||||
|
increaseQueueAppNum(queue);
|
||||||
|
|
||||||
String oldAppId = jsonJob.get("job.id").toString();
|
String oldAppId = jsonJob.get("job.id").toString();
|
||||||
boolean isTracked = trackedApps.contains(oldAppId);
|
|
||||||
int queueSize =
|
|
||||||
queueAppNumMap.containsKey(queue) ? queueAppNumMap.get(queue) : 0;
|
|
||||||
queueSize++;
|
|
||||||
queueAppNumMap.put(queue, queueSize);
|
|
||||||
// tasks
|
// tasks
|
||||||
List tasks = (List) jsonJob.get("job.tasks");
|
List tasks = (List) jsonJob.get("job.tasks");
|
||||||
if (tasks == null || tasks.size() == 0) {
|
if (tasks == null || tasks.size() == 0) {
|
||||||
continue;
|
throw new YarnException("No task for the job!");
|
||||||
}
|
}
|
||||||
List<ContainerSimulator> containerList =
|
|
||||||
new ArrayList<ContainerSimulator>();
|
List<ContainerSimulator> containerList = new ArrayList<>();
|
||||||
for (Object o : tasks) {
|
for (Object o : tasks) {
|
||||||
Map jsonTask = (Map) o;
|
Map jsonTask = (Map) o;
|
||||||
String hostname = jsonTask.get("container.host").toString();
|
String hostname = jsonTask.get("container.host").toString();
|
||||||
long taskStart =
|
long taskStart = Long.parseLong(jsonTask.get("container.start.ms")
|
||||||
Long.parseLong(jsonTask.get("container.start.ms").toString());
|
.toString());
|
||||||
long taskFinish =
|
long taskFinish = Long.parseLong(jsonTask.get("container.end.ms")
|
||||||
Long.parseLong(jsonTask.get("container.end.ms").toString());
|
.toString());
|
||||||
long lifeTime = taskFinish - taskStart;
|
long lifeTime = taskFinish - taskStart;
|
||||||
|
|
||||||
// Set memory and vcores from job trace file
|
// Set memory and vcores from job trace file
|
||||||
Resource res = Resources.clone(containerResource);
|
Resource res = getDefaultContainerResource();
|
||||||
if (jsonTask.containsKey("container.memory")) {
|
if (jsonTask.containsKey("container.memory")) {
|
||||||
int containerMemory =
|
int containerMemory =
|
||||||
Integer.parseInt(jsonTask.get("container.memory").toString());
|
Integer.parseInt(jsonTask.get("container.memory").toString());
|
||||||
@ -397,129 +390,116 @@ private void startAMFromSLSTraces(Resource containerResource,
|
|||||||
res.setVirtualCores(containerVCores);
|
res.setVirtualCores(containerVCores);
|
||||||
}
|
}
|
||||||
|
|
||||||
int priority =
|
int priority = Integer.parseInt(jsonTask.get("container.priority")
|
||||||
Integer.parseInt(jsonTask.get("container.priority").toString());
|
.toString());
|
||||||
String type = jsonTask.get("container.type").toString();
|
String type = jsonTask.get("container.type").toString();
|
||||||
containerList.add(new ContainerSimulator(res, lifeTime, hostname,
|
containerList.add(
|
||||||
priority, type));
|
new ContainerSimulator(res, lifeTime, hostname, priority, type));
|
||||||
}
|
}
|
||||||
|
|
||||||
// create a new AM
|
// create a new AM
|
||||||
String amType = jsonJob.get("am.type").toString();
|
String amType = jsonJob.get("am.type").toString();
|
||||||
AMSimulator amSim = (AMSimulator) ReflectionUtils
|
runNewAM(amType, user, queue, oldAppId, jobStartTime, jobFinishTime,
|
||||||
.newInstance(amClassMap.get(amType), new Configuration());
|
containerList, null);
|
||||||
if (amSim != null) {
|
|
||||||
amSim.init(AM_ID++, heartbeatInterval, containerList, rm, this,
|
|
||||||
jobStartTime, jobFinishTime, user, queue, isTracked, oldAppId,
|
|
||||||
null, runner.getStartTimeMS());
|
|
||||||
runner.schedule(amSim);
|
|
||||||
maxRuntime = Math.max(maxRuntime, jobFinishTime);
|
|
||||||
numTasks += containerList.size();
|
|
||||||
amMap.put(oldAppId, amSim);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Parse workload from a rumen trace file.
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
private void startAMFromRumenTrace(String inputTrace, long baselineTimeMS)
|
||||||
|
throws IOException {
|
||||||
|
Configuration conf = new Configuration();
|
||||||
|
conf.set("fs.defaultFS", "file:///");
|
||||||
|
File fin = new File(inputTrace);
|
||||||
|
|
||||||
|
try (JobTraceReader reader = new JobTraceReader(
|
||||||
|
new Path(fin.getAbsolutePath()), conf)) {
|
||||||
|
LoggedJob job = reader.getNext();
|
||||||
|
|
||||||
|
while (job != null) {
|
||||||
|
try {
|
||||||
|
createAMForJob(job, baselineTimeMS);
|
||||||
|
} catch (Exception e) {
|
||||||
|
LOG.error("Failed to create an AM: " + e.getMessage());
|
||||||
}
|
}
|
||||||
} finally {
|
|
||||||
input.close();
|
job = reader.getNext();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
private void createAMForJob(LoggedJob job, long baselineTimeMs)
|
||||||
* parse workload information from rumen trace files
|
throws YarnException {
|
||||||
*/
|
String user = job.getUser() == null ? "default" :
|
||||||
@SuppressWarnings("unchecked")
|
job.getUser().getValue();
|
||||||
private void startAMFromRumenTraces(Resource containerResource,
|
|
||||||
int heartbeatInterval) throws IOException {
|
|
||||||
Configuration conf = new Configuration();
|
|
||||||
conf.set("fs.defaultFS", "file:///");
|
|
||||||
long baselineTimeMS = 0;
|
|
||||||
for (String inputTrace : inputTraces) {
|
|
||||||
File fin = new File(inputTrace);
|
|
||||||
JobTraceReader reader =
|
|
||||||
new JobTraceReader(new Path(fin.getAbsolutePath()), conf);
|
|
||||||
try {
|
|
||||||
LoggedJob job = null;
|
|
||||||
while ((job = reader.getNext()) != null) {
|
|
||||||
// only support MapReduce currently
|
|
||||||
String jobType = "mapreduce";
|
|
||||||
String user =
|
|
||||||
job.getUser() == null ? "default" : job.getUser().getValue();
|
|
||||||
String jobQueue = job.getQueue().getValue();
|
String jobQueue = job.getQueue().getValue();
|
||||||
String oldJobId = job.getJobID().toString();
|
String oldJobId = job.getJobID().toString();
|
||||||
long jobStartTimeMS = job.getSubmitTime();
|
long jobStartTimeMS = job.getSubmitTime();
|
||||||
long jobFinishTimeMS = job.getFinishTime();
|
long jobFinishTimeMS = job.getFinishTime();
|
||||||
if (baselineTimeMS == 0) {
|
if (baselineTimeMs == 0) {
|
||||||
baselineTimeMS = jobStartTimeMS;
|
baselineTimeMs = job.getSubmitTime();
|
||||||
}
|
}
|
||||||
jobStartTimeMS -= baselineTimeMS;
|
jobStartTimeMS -= baselineTimeMs;
|
||||||
jobFinishTimeMS -= baselineTimeMS;
|
jobFinishTimeMS -= baselineTimeMs;
|
||||||
if (jobStartTimeMS < 0) {
|
if (jobStartTimeMS < 0) {
|
||||||
LOG.warn("Warning: reset job " + oldJobId + " start time to 0.");
|
LOG.warn("Warning: reset job " + oldJobId + " start time to 0.");
|
||||||
jobFinishTimeMS = jobFinishTimeMS - jobStartTimeMS;
|
jobFinishTimeMS = jobFinishTimeMS - jobStartTimeMS;
|
||||||
jobStartTimeMS = 0;
|
jobStartTimeMS = 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
boolean isTracked = trackedApps.contains(oldJobId);
|
increaseQueueAppNum(jobQueue);
|
||||||
int queueSize = queueAppNumMap.containsKey(jobQueue)
|
|
||||||
? queueAppNumMap.get(jobQueue) : 0;
|
|
||||||
queueSize++;
|
|
||||||
queueAppNumMap.put(jobQueue, queueSize);
|
|
||||||
|
|
||||||
List<ContainerSimulator> containerList =
|
List<ContainerSimulator> containerList = new ArrayList<>();
|
||||||
new ArrayList<ContainerSimulator>();
|
// mapper
|
||||||
// map tasks
|
|
||||||
for (LoggedTask mapTask : job.getMapTasks()) {
|
for (LoggedTask mapTask : job.getMapTasks()) {
|
||||||
if (mapTask.getAttempts().size() == 0) {
|
if (mapTask.getAttempts().size() == 0) {
|
||||||
continue;
|
throw new YarnException("Invalid map task, no attempt for a mapper!");
|
||||||
}
|
}
|
||||||
LoggedTaskAttempt taskAttempt =
|
LoggedTaskAttempt taskAttempt =
|
||||||
mapTask.getAttempts().get(mapTask.getAttempts().size() - 1);
|
mapTask.getAttempts().get(mapTask.getAttempts().size() - 1);
|
||||||
String hostname = taskAttempt.getHostName().getValue();
|
String hostname = taskAttempt.getHostName().getValue();
|
||||||
long containerLifeTime =
|
long containerLifeTime = taskAttempt.getFinishTime() -
|
||||||
taskAttempt.getFinishTime() - taskAttempt.getStartTime();
|
taskAttempt.getStartTime();
|
||||||
containerList.add(new ContainerSimulator(containerResource,
|
containerList.add(
|
||||||
|
new ContainerSimulator(getDefaultContainerResource(),
|
||||||
containerLifeTime, hostname, 10, "map"));
|
containerLifeTime, hostname, 10, "map"));
|
||||||
}
|
}
|
||||||
|
|
||||||
// reduce tasks
|
// reducer
|
||||||
for (LoggedTask reduceTask : job.getReduceTasks()) {
|
for (LoggedTask reduceTask : job.getReduceTasks()) {
|
||||||
if (reduceTask.getAttempts().size() == 0) {
|
if (reduceTask.getAttempts().size() == 0) {
|
||||||
continue;
|
throw new YarnException(
|
||||||
|
"Invalid reduce task, no attempt for a reducer!");
|
||||||
}
|
}
|
||||||
LoggedTaskAttempt taskAttempt = reduceTask.getAttempts()
|
LoggedTaskAttempt taskAttempt =
|
||||||
.get(reduceTask.getAttempts().size() - 1);
|
reduceTask.getAttempts().get(reduceTask.getAttempts().size() - 1);
|
||||||
String hostname = taskAttempt.getHostName().getValue();
|
String hostname = taskAttempt.getHostName().getValue();
|
||||||
long containerLifeTime =
|
long containerLifeTime = taskAttempt.getFinishTime() -
|
||||||
taskAttempt.getFinishTime() - taskAttempt.getStartTime();
|
taskAttempt.getStartTime();
|
||||||
containerList.add(new ContainerSimulator(containerResource,
|
containerList.add(
|
||||||
|
new ContainerSimulator(getDefaultContainerResource(),
|
||||||
containerLifeTime, hostname, 20, "reduce"));
|
containerLifeTime, hostname, 20, "reduce"));
|
||||||
}
|
}
|
||||||
|
|
||||||
// create a new AM
|
// Only supports the default job type currently
|
||||||
AMSimulator amSim = (AMSimulator) ReflectionUtils
|
runNewAM(SLSUtils.DEFAULT_JOB_TYPE, user, jobQueue, oldJobId,
|
||||||
.newInstance(amClassMap.get(jobType), conf);
|
jobStartTimeMS, jobFinishTimeMS, containerList, null);
|
||||||
if (amSim != null) {
|
|
||||||
amSim.init(AM_ID++, heartbeatInterval, containerList, rm, this,
|
|
||||||
jobStartTimeMS, jobFinishTimeMS, user, jobQueue, isTracked,
|
|
||||||
oldJobId, null, runner.getStartTimeMS());
|
|
||||||
runner.schedule(amSim);
|
|
||||||
maxRuntime = Math.max(maxRuntime, jobFinishTimeMS);
|
|
||||||
numTasks += containerList.size();
|
|
||||||
amMap.put(oldJobId, amSim);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
} finally {
|
|
||||||
reader.close();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private Resource getDefaultContainerResource() {
|
||||||
|
int containerMemory = getConf().getInt(SLSConfiguration.CONTAINER_MEMORY_MB,
|
||||||
|
SLSConfiguration.CONTAINER_MEMORY_MB_DEFAULT);
|
||||||
|
int containerVCores = getConf().getInt(SLSConfiguration.CONTAINER_VCORES,
|
||||||
|
SLSConfiguration.CONTAINER_VCORES_DEFAULT);
|
||||||
|
return Resources.createResource(containerMemory, containerVCores);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* parse workload information from synth-generator trace files.
|
* parse workload information from synth-generator trace files.
|
||||||
*/
|
*/
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
private void startAMFromSynthGenerator(int heartbeatInterval)
|
private void startAMFromSynthGenerator() throws YarnException, IOException {
|
||||||
throws IOException {
|
|
||||||
Configuration localConf = new Configuration();
|
Configuration localConf = new Configuration();
|
||||||
localConf.set("fs.defaultFS", "file:///");
|
localConf.set("fs.defaultFS", "file:///");
|
||||||
long baselineTimeMS = 0;
|
long baselineTimeMS = 0;
|
||||||
@ -540,7 +520,6 @@ private void startAMFromSynthGenerator(int heartbeatInterval)
|
|||||||
// creation
|
// creation
|
||||||
while ((job = (SynthJob) stjp.getNextJob()) != null) {
|
while ((job = (SynthJob) stjp.getNextJob()) != null) {
|
||||||
// only support MapReduce currently
|
// only support MapReduce currently
|
||||||
String jobType = "mapreduce";
|
|
||||||
String user = job.getUser();
|
String user = job.getUser();
|
||||||
String jobQueue = job.getQueueName();
|
String jobQueue = job.getQueueName();
|
||||||
String oldJobId = job.getJobID().toString();
|
String oldJobId = job.getJobID().toString();
|
||||||
@ -560,11 +539,7 @@ private void startAMFromSynthGenerator(int heartbeatInterval)
|
|||||||
jobStartTimeMS = 0;
|
jobStartTimeMS = 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
boolean isTracked = trackedApps.contains(oldJobId);
|
increaseQueueAppNum(jobQueue);
|
||||||
int queueSize = queueAppNumMap.containsKey(jobQueue)
|
|
||||||
? queueAppNumMap.get(jobQueue) : 0;
|
|
||||||
queueSize++;
|
|
||||||
queueAppNumMap.put(jobQueue, queueSize);
|
|
||||||
|
|
||||||
List<ContainerSimulator> containerList =
|
List<ContainerSimulator> containerList =
|
||||||
new ArrayList<ContainerSimulator>();
|
new ArrayList<ContainerSimulator>();
|
||||||
@ -625,18 +600,9 @@ private void startAMFromSynthGenerator(int heartbeatInterval)
|
|||||||
job.getQueueName());
|
job.getQueueName());
|
||||||
|
|
||||||
}
|
}
|
||||||
// create a new AM
|
|
||||||
AMSimulator amSim = (AMSimulator) ReflectionUtils
|
runNewAM(SLSUtils.DEFAULT_JOB_TYPE, user, jobQueue, oldJobId,
|
||||||
.newInstance(amClassMap.get(jobType), localConf);
|
jobStartTimeMS, jobFinishTimeMS, containerList, rr);
|
||||||
if (amSim != null) {
|
|
||||||
amSim.init(AM_ID++, heartbeatInterval, containerList, rm, this,
|
|
||||||
jobStartTimeMS, jobFinishTimeMS, user, jobQueue, isTracked,
|
|
||||||
oldJobId, rr, runner.getStartTimeMS());
|
|
||||||
runner.schedule(amSim);
|
|
||||||
maxRuntime = Math.max(maxRuntime, jobFinishTimeMS);
|
|
||||||
numTasks += containerList.size();
|
|
||||||
amMap.put(oldJobId, amSim);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
stjp.close();
|
stjp.close();
|
||||||
@ -644,6 +610,42 @@ private void startAMFromSynthGenerator(int heartbeatInterval)
|
|||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void increaseQueueAppNum(String queue) throws YarnException {
|
||||||
|
SchedulerWrapper wrapper = (SchedulerWrapper)rm.getResourceScheduler();
|
||||||
|
String queueName = wrapper.getRealQueueName(queue);
|
||||||
|
Integer appNum = queueAppNumMap.get(queueName);
|
||||||
|
if (appNum == null) {
|
||||||
|
appNum = 1;
|
||||||
|
} else {
|
||||||
|
appNum++;
|
||||||
|
}
|
||||||
|
|
||||||
|
queueAppNumMap.put(queueName, appNum);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void runNewAM(String jobType, String user,
|
||||||
|
String jobQueue, String oldJobId, long jobStartTimeMS,
|
||||||
|
long jobFinishTimeMS, List<ContainerSimulator> containerList,
|
||||||
|
ReservationSubmissionRequest rr) {
|
||||||
|
|
||||||
|
AMSimulator amSim = (AMSimulator) ReflectionUtils.newInstance(
|
||||||
|
amClassMap.get(jobType), new Configuration());
|
||||||
|
|
||||||
|
if (amSim != null) {
|
||||||
|
int heartbeatInterval = getConf().getInt(
|
||||||
|
SLSConfiguration.AM_HEARTBEAT_INTERVAL_MS,
|
||||||
|
SLSConfiguration.AM_HEARTBEAT_INTERVAL_MS_DEFAULT);
|
||||||
|
boolean isTracked = trackedApps.contains(oldJobId);
|
||||||
|
amSim.init(AM_ID++, heartbeatInterval, containerList,
|
||||||
|
rm, this, jobStartTimeMS, jobFinishTimeMS, user, jobQueue,
|
||||||
|
isTracked, oldJobId, rr, runner.getStartTimeMS());
|
||||||
|
runner.schedule(amSim);
|
||||||
|
maxRuntime = Math.max(maxRuntime, jobFinishTimeMS);
|
||||||
|
numTasks += containerList.size();
|
||||||
|
amMap.put(oldJobId, amSim);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private void printSimulationInfo() {
|
private void printSimulationInfo() {
|
||||||
if (printSimulation) {
|
if (printSimulation) {
|
||||||
// node
|
// node
|
||||||
|
@ -35,6 +35,7 @@
|
|||||||
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
||||||
import org.apache.hadoop.yarn.api.records.Resource;
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||||
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
||||||
@ -360,4 +361,12 @@ public SchedulerMetrics getSchedulerMetrics() {
|
|||||||
public Configuration getConf() {
|
public Configuration getConf() {
|
||||||
return conf;
|
return conf;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public String getRealQueueName(String queue) throws YarnException {
|
||||||
|
if (getQueue(queue) == null) {
|
||||||
|
throw new YarnException("Can't find the queue by the given name: " + queue
|
||||||
|
+ "! Please check if queue " + queue + " is in the allocation file.");
|
||||||
|
}
|
||||||
|
return getQueue(queue).getQueueName();
|
||||||
|
}
|
||||||
}
|
}
|
@ -29,6 +29,7 @@
|
|||||||
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
||||||
import org.apache.hadoop.yarn.api.records.Resource;
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||||
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
||||||
@ -333,5 +334,13 @@ public void serviceStop() throws Exception {
|
|||||||
}
|
}
|
||||||
super.serviceStop();
|
super.serviceStop();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public String getRealQueueName(String queue) throws YarnException {
|
||||||
|
if (!getQueueManager().exists(queue)) {
|
||||||
|
throw new YarnException("Can't find the queue by the given name: " + queue
|
||||||
|
+ "! Please check if queue " + queue + " is in the allocation file.");
|
||||||
|
}
|
||||||
|
return getQueueManager().getQueue(queue).getQueueName();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -19,10 +19,14 @@
|
|||||||
|
|
||||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||||
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||||
|
|
||||||
@Private
|
@Private
|
||||||
@Unstable
|
@Unstable
|
||||||
public interface SchedulerWrapper {
|
public interface SchedulerWrapper {
|
||||||
SchedulerMetrics getSchedulerMetrics();
|
SchedulerMetrics getSchedulerMetrics();
|
||||||
|
|
||||||
Tracker getTracker();
|
Tracker getTracker();
|
||||||
|
|
||||||
|
String getRealQueueName(String queue) throws YarnException;
|
||||||
}
|
}
|
||||||
|
@ -43,7 +43,7 @@
|
|||||||
@Private
|
@Private
|
||||||
@Unstable
|
@Unstable
|
||||||
public class SLSUtils {
|
public class SLSUtils {
|
||||||
public static final int SHUTDOWN_HOOK_PRIORITY = 30;
|
public final static String DEFAULT_JOB_TYPE = "mapreduce";
|
||||||
|
|
||||||
// hostname includes the network path and the host name. for example
|
// hostname includes the network path and the host name. for example
|
||||||
// "/default-rack/hostFoo" or "/coreSwitchA/TORSwitchB/hostBar".
|
// "/default-rack/hostFoo" or "/coreSwitchA/TORSwitchB/hostBar".
|
||||||
|
Loading…
Reference in New Issue
Block a user