|
@@ -23,12 +23,10 @@ import java.io.IOException;
|
|
|
import java.io.InputStreamReader;
|
|
|
import java.io.Reader;
|
|
|
import java.security.Security;
|
|
|
-import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.HashSet;
|
|
|
import java.util.Iterator;
|
|
|
-import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.Random;
|
|
|
import java.util.Set;
|
|
@@ -59,13 +57,10 @@ import org.apache.hadoop.net.DNSToSwitchMapping;
|
|
|
import org.apache.hadoop.net.TableMapping;
|
|
|
import org.apache.hadoop.tools.rumen.JobTraceReader;
|
|
|
import org.apache.hadoop.tools.rumen.LoggedJob;
|
|
|
-import org.apache.hadoop.tools.rumen.LoggedTask;
|
|
|
-import org.apache.hadoop.tools.rumen.LoggedTaskAttempt;
|
|
|
import org.apache.hadoop.util.ReflectionUtils;
|
|
|
import org.apache.hadoop.util.Tool;
|
|
|
import org.apache.hadoop.util.ToolRunner;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
-import org.apache.hadoop.yarn.api.records.ExecutionType;
|
|
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
|
|
import org.apache.hadoop.yarn.api.records.NodeLabel;
|
|
|
import org.apache.hadoop.yarn.api.records.NodeState;
|
|
@@ -89,7 +84,6 @@ import org.apache.hadoop.yarn.sls.scheduler.SLSCapacityScheduler;
|
|
|
import org.apache.hadoop.yarn.sls.scheduler.SchedulerMetrics;
|
|
|
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.SynthTraceJobProducer;
|
|
@@ -138,13 +132,8 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
// logger
|
|
|
public final static Logger LOG = LoggerFactory.getLogger(SLSRunner.class);
|
|
|
|
|
|
- private final static int DEFAULT_MAPPER_PRIORITY = 20;
|
|
|
- private final static int DEFAULT_REDUCER_PRIORITY = 10;
|
|
|
-
|
|
|
private static boolean exitAtTheFinish = false;
|
|
|
|
|
|
- private static final String DEFAULT_USER = "default";
|
|
|
-
|
|
|
/**
|
|
|
* The type of trace in input.
|
|
|
*/
|
|
@@ -472,7 +461,10 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
|
|
|
while (jobIter.hasNext()) {
|
|
|
try {
|
|
|
- createAMForJob(jobIter.next());
|
|
|
+ Map jsonJob = jobIter.next();
|
|
|
+ AMDefinitionSLS amDef = AMDefinitionFactory.createFromSlsTrace(
|
|
|
+ jsonJob, this);
|
|
|
+ startAMs(amDef);
|
|
|
} catch (Exception e) {
|
|
|
LOG.error("Failed to create an AM: {}", e.getMessage());
|
|
|
}
|
|
@@ -480,150 +472,29 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private void createAMForJob(Map jsonJob) throws YarnException {
|
|
|
- long jobStartTime = Long.parseLong(
|
|
|
- jsonJob.get(SLSConfiguration.JOB_START_MS).toString());
|
|
|
-
|
|
|
- long jobFinishTime = 0;
|
|
|
- if (jsonJob.containsKey(SLSConfiguration.JOB_END_MS)) {
|
|
|
- jobFinishTime = Long.parseLong(
|
|
|
- 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);
|
|
|
- if (user == null) {
|
|
|
- user = "default";
|
|
|
- }
|
|
|
-
|
|
|
- String queue = jsonJob.get(SLSConfiguration.JOB_QUEUE_NAME).toString();
|
|
|
- increaseQueueAppNum(queue);
|
|
|
-
|
|
|
- String amType = (String)jsonJob.get(SLSConfiguration.AM_TYPE);
|
|
|
- if (amType == null) {
|
|
|
- amType = SLSUtils.DEFAULT_JOB_TYPE;
|
|
|
- }
|
|
|
-
|
|
|
- int jobCount = 1;
|
|
|
- if (jsonJob.containsKey(SLSConfiguration.JOB_COUNT)) {
|
|
|
- jobCount = Integer.parseInt(
|
|
|
- jsonJob.get(SLSConfiguration.JOB_COUNT).toString());
|
|
|
- }
|
|
|
- jobCount = Math.max(jobCount, 1);
|
|
|
-
|
|
|
- String oldAppId = (String)jsonJob.get(SLSConfiguration.JOB_ID);
|
|
|
- // Job id is generated automatically if this job configuration allows
|
|
|
- // multiple job instances
|
|
|
- if(jobCount > 1) {
|
|
|
- oldAppId = null;
|
|
|
- }
|
|
|
-
|
|
|
- for (int i = 0; i < jobCount; i++) {
|
|
|
- runNewAM(amType, user, queue, oldAppId, jobStartTime, jobFinishTime,
|
|
|
- getTaskContainers(jsonJob), getAMContainerResource(jsonJob),
|
|
|
- jobLabelExpr);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private List<ContainerSimulator> getTaskContainers(Map jsonJob)
|
|
|
- throws YarnException {
|
|
|
- List<ContainerSimulator> containers = new ArrayList<>();
|
|
|
- List tasks = (List) jsonJob.get(SLSConfiguration.JOB_TASKS);
|
|
|
- if (tasks == null || tasks.size() == 0) {
|
|
|
- throw new YarnException("No task for the job!");
|
|
|
- }
|
|
|
-
|
|
|
- for (Object o : tasks) {
|
|
|
- Map jsonTask = (Map) o;
|
|
|
-
|
|
|
- String hostname = (String) jsonTask.get(SLSConfiguration.TASK_HOST);
|
|
|
-
|
|
|
- long duration = 0;
|
|
|
- if (jsonTask.containsKey(SLSConfiguration.TASK_DURATION_MS)) {
|
|
|
- duration = Integer.parseInt(
|
|
|
- jsonTask.get(SLSConfiguration.TASK_DURATION_MS).toString());
|
|
|
- } else if (jsonTask.containsKey(SLSConfiguration.DURATION_MS)) {
|
|
|
- // Also support "duration.ms" for backward compatibility
|
|
|
- duration = Integer.parseInt(
|
|
|
- jsonTask.get(SLSConfiguration.DURATION_MS).toString());
|
|
|
- } else if (jsonTask.containsKey(SLSConfiguration.TASK_START_MS) &&
|
|
|
- jsonTask.containsKey(SLSConfiguration.TASK_END_MS)) {
|
|
|
- long taskStart = Long.parseLong(
|
|
|
- jsonTask.get(SLSConfiguration.TASK_START_MS).toString());
|
|
|
- long taskFinish = Long.parseLong(
|
|
|
- jsonTask.get(SLSConfiguration.TASK_END_MS).toString());
|
|
|
- duration = taskFinish - taskStart;
|
|
|
- }
|
|
|
- if (duration <= 0) {
|
|
|
- throw new YarnException("Duration of a task shouldn't be less or equal"
|
|
|
- + " to 0!");
|
|
|
- }
|
|
|
-
|
|
|
- Resource res = getResourceForContainer(jsonTask);
|
|
|
-
|
|
|
- int priority = DEFAULT_MAPPER_PRIORITY;
|
|
|
- if (jsonTask.containsKey(SLSConfiguration.TASK_PRIORITY)) {
|
|
|
- priority = Integer.parseInt(
|
|
|
- jsonTask.get(SLSConfiguration.TASK_PRIORITY).toString());
|
|
|
- }
|
|
|
-
|
|
|
- String type = "map";
|
|
|
- if (jsonTask.containsKey(SLSConfiguration.TASK_TYPE)) {
|
|
|
- type = jsonTask.get(SLSConfiguration.TASK_TYPE).toString();
|
|
|
- }
|
|
|
-
|
|
|
- int count = 1;
|
|
|
- if (jsonTask.containsKey(SLSConfiguration.COUNT)) {
|
|
|
- count = Integer.parseInt(
|
|
|
- jsonTask.get(SLSConfiguration.COUNT).toString());
|
|
|
- }
|
|
|
- count = Math.max(count, 1);
|
|
|
-
|
|
|
- ExecutionType executionType = ExecutionType.GUARANTEED;
|
|
|
- if (jsonTask.containsKey(SLSConfiguration.TASK_EXECUTION_TYPE)) {
|
|
|
- executionType = ExecutionType.valueOf(
|
|
|
- jsonTask.get(SLSConfiguration.TASK_EXECUTION_TYPE).toString());
|
|
|
- }
|
|
|
- long allocationId = -1;
|
|
|
- if (jsonTask.containsKey(SLSConfiguration.TASK_ALLOCATION_ID)) {
|
|
|
- allocationId = Long.parseLong(
|
|
|
- jsonTask.get(SLSConfiguration.TASK_ALLOCATION_ID).toString());
|
|
|
- }
|
|
|
-
|
|
|
- long requestDelay = 0;
|
|
|
- if (jsonTask.containsKey(SLSConfiguration.TASK_REQUEST_DELAY)) {
|
|
|
- requestDelay = Long.parseLong(
|
|
|
- jsonTask.get(SLSConfiguration.TASK_REQUEST_DELAY).toString());
|
|
|
- }
|
|
|
- requestDelay = Math.max(requestDelay, 0);
|
|
|
-
|
|
|
- for (int i = 0; i < count; i++) {
|
|
|
- containers.add(
|
|
|
- new ContainerSimulator(res, duration, hostname, priority, type,
|
|
|
- executionType, allocationId, requestDelay));
|
|
|
- }
|
|
|
+ private void startAMs(AMDefinition amDef) {
|
|
|
+ for (int i = 0; i < amDef.getJobCount(); i++) {
|
|
|
+ JobDefinition jobDef = JobDefinition.Builder.create()
|
|
|
+ .withAmDefinition(amDef)
|
|
|
+ .withDeadline(-1)
|
|
|
+ .withReservationId(null)
|
|
|
+ .withParams(null)
|
|
|
+ .build();
|
|
|
+ runNewAM(jobDef);
|
|
|
}
|
|
|
-
|
|
|
- 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);
|
|
|
- }
|
|
|
+ private void startAMs(AMDefinition amDef, ReservationId reservationId,
|
|
|
+ Map<String, String> params, long deadline) {
|
|
|
+ for (int i = 0; i < amDef.getJobCount(); i++) {
|
|
|
+ JobDefinition jobDef = JobDefinition.Builder.create()
|
|
|
+ .withAmDefinition(amDef)
|
|
|
+ .withReservationId(reservationId)
|
|
|
+ .withParams(params)
|
|
|
+ .withDeadline(deadline)
|
|
|
+ .build();
|
|
|
+ runNewAM(jobDef);
|
|
|
}
|
|
|
-
|
|
|
- return res;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -642,76 +513,19 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
|
|
|
while (job != null) {
|
|
|
try {
|
|
|
- createAMForJob(job, baselineTimeMS);
|
|
|
+ AMDefinitionRumen amDef =
|
|
|
+ AMDefinitionFactory.createFromRumenTrace(job, baselineTimeMS,
|
|
|
+ this);
|
|
|
+ startAMs(amDef);
|
|
|
} catch (Exception e) {
|
|
|
LOG.error("Failed to create an AM", e);
|
|
|
}
|
|
|
-
|
|
|
job = reader.getNext();
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private void createAMForJob(LoggedJob job, long baselineTimeMs)
|
|
|
- throws YarnException {
|
|
|
- String user = job.getUser() == null ? "default" :
|
|
|
- job.getUser().getValue();
|
|
|
- String jobQueue = job.getQueue().getValue();
|
|
|
- String oldJobId = job.getJobID().toString();
|
|
|
- long jobStartTimeMS = job.getSubmitTime();
|
|
|
- long jobFinishTimeMS = job.getFinishTime();
|
|
|
- if (baselineTimeMs == 0) {
|
|
|
- baselineTimeMs = job.getSubmitTime();
|
|
|
- }
|
|
|
- jobStartTimeMS -= baselineTimeMs;
|
|
|
- jobFinishTimeMS -= baselineTimeMs;
|
|
|
- if (jobStartTimeMS < 0) {
|
|
|
- LOG.warn("Warning: reset job {} start time to 0.", oldJobId);
|
|
|
- jobFinishTimeMS = jobFinishTimeMS - jobStartTimeMS;
|
|
|
- jobStartTimeMS = 0;
|
|
|
- }
|
|
|
-
|
|
|
- increaseQueueAppNum(jobQueue);
|
|
|
-
|
|
|
- List<ContainerSimulator> containerList = new ArrayList<>();
|
|
|
- // mapper
|
|
|
- for (LoggedTask mapTask : job.getMapTasks()) {
|
|
|
- if (mapTask.getAttempts().size() == 0) {
|
|
|
- throw new YarnException("Invalid map task, no attempt for a mapper!");
|
|
|
- }
|
|
|
- LoggedTaskAttempt taskAttempt =
|
|
|
- mapTask.getAttempts().get(mapTask.getAttempts().size() - 1);
|
|
|
- String hostname = taskAttempt.getHostName().getValue();
|
|
|
- long containerLifeTime = taskAttempt.getFinishTime() -
|
|
|
- taskAttempt.getStartTime();
|
|
|
- containerList.add(
|
|
|
- new ContainerSimulator(getDefaultContainerResource(),
|
|
|
- containerLifeTime, hostname, DEFAULT_MAPPER_PRIORITY, "map"));
|
|
|
- }
|
|
|
-
|
|
|
- // reducer
|
|
|
- for (LoggedTask reduceTask : job.getReduceTasks()) {
|
|
|
- if (reduceTask.getAttempts().size() == 0) {
|
|
|
- throw new YarnException(
|
|
|
- "Invalid reduce task, no attempt for a reducer!");
|
|
|
- }
|
|
|
- LoggedTaskAttempt taskAttempt =
|
|
|
- reduceTask.getAttempts().get(reduceTask.getAttempts().size() - 1);
|
|
|
- String hostname = taskAttempt.getHostName().getValue();
|
|
|
- long containerLifeTime = taskAttempt.getFinishTime() -
|
|
|
- taskAttempt.getStartTime();
|
|
|
- containerList.add(
|
|
|
- new ContainerSimulator(getDefaultContainerResource(),
|
|
|
- containerLifeTime, hostname, DEFAULT_REDUCER_PRIORITY, "reduce"));
|
|
|
- }
|
|
|
-
|
|
|
- // Only supports the default job type currently
|
|
|
- runNewAM(SLSUtils.DEFAULT_JOB_TYPE, user, jobQueue, oldJobId,
|
|
|
- jobStartTimeMS, jobFinishTimeMS, containerList,
|
|
|
- getAMContainerResource(null));
|
|
|
- }
|
|
|
-
|
|
|
- private Resource getDefaultContainerResource() {
|
|
|
+ Resource getDefaultContainerResource() {
|
|
|
int containerMemory = getConf().getInt(SLSConfiguration.CONTAINER_MEMORY_MB,
|
|
|
SLSConfiguration.CONTAINER_MEMORY_MB_DEFAULT);
|
|
|
int containerVCores = getConf().getInt(SLSConfiguration.CONTAINER_VCORES,
|
|
@@ -726,94 +540,26 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
private void startAMFromSynthGenerator() throws YarnException, IOException {
|
|
|
Configuration localConf = new Configuration();
|
|
|
localConf.set("fs.defaultFS", "file:///");
|
|
|
- long baselineTimeMS = 0;
|
|
|
-
|
|
|
// if we use the nodeFile this could have been not initialized yet.
|
|
|
if (stjp == null) {
|
|
|
stjp = new SynthTraceJobProducer(getConf(), new Path(inputTraces[0]));
|
|
|
}
|
|
|
|
|
|
- SynthJob job = null;
|
|
|
+ SynthJob job;
|
|
|
// we use stjp, a reference to the job producer instantiated during node
|
|
|
// creation
|
|
|
while ((job = (SynthJob) stjp.getNextJob()) != null) {
|
|
|
- // only support MapReduce currently
|
|
|
- String user = job.getUser() == null ? DEFAULT_USER :
|
|
|
- job.getUser();
|
|
|
- String jobQueue = job.getQueueName();
|
|
|
- String oldJobId = job.getJobID().toString();
|
|
|
- long jobStartTimeMS = job.getSubmissionTime();
|
|
|
-
|
|
|
- // CARLO: Finish time is only used for logging, omit for now
|
|
|
- long jobFinishTimeMS = jobStartTimeMS + job.getDuration();
|
|
|
-
|
|
|
- if (baselineTimeMS == 0) {
|
|
|
- baselineTimeMS = jobStartTimeMS;
|
|
|
- }
|
|
|
- jobStartTimeMS -= baselineTimeMS;
|
|
|
- jobFinishTimeMS -= baselineTimeMS;
|
|
|
- if (jobStartTimeMS < 0) {
|
|
|
- LOG.warn("Warning: reset job {} start time to 0.", oldJobId);
|
|
|
- jobFinishTimeMS = jobFinishTimeMS - jobStartTimeMS;
|
|
|
- jobStartTimeMS = 0;
|
|
|
- }
|
|
|
-
|
|
|
- increaseQueueAppNum(jobQueue);
|
|
|
-
|
|
|
- List<ContainerSimulator> containerList =
|
|
|
- new ArrayList<ContainerSimulator>();
|
|
|
- ArrayList<NodeId> keyAsArray = new ArrayList<NodeId>(nmMap.keySet());
|
|
|
- Random rand = new Random(stjp.getSeed());
|
|
|
-
|
|
|
- for (SynthJob.SynthTask task : job.getTasks()) {
|
|
|
- RMNode node = nmMap.get(keyAsArray.get(rand.nextInt(keyAsArray.size())))
|
|
|
- .getNode();
|
|
|
- String hostname = "/" + node.getRackName() + "/" + node.getHostName();
|
|
|
- long containerLifeTime = task.getTime();
|
|
|
- Resource containerResource = Resource
|
|
|
- .newInstance((int) task.getMemory(), (int) task.getVcores());
|
|
|
- containerList.add(
|
|
|
- new ContainerSimulator(containerResource, containerLifeTime,
|
|
|
- hostname, task.getPriority(), task.getType(),
|
|
|
- task.getExecutionType()));
|
|
|
- }
|
|
|
-
|
|
|
-
|
|
|
ReservationId reservationId = null;
|
|
|
-
|
|
|
- if(job.hasDeadline()){
|
|
|
+ if (job.hasDeadline()) {
|
|
|
reservationId = ReservationId
|
|
|
- .newInstance(this.rm.getStartTime(), AM_ID);
|
|
|
- }
|
|
|
-
|
|
|
- runNewAM(job.getType(), user, jobQueue, oldJobId,
|
|
|
- jobStartTimeMS, jobFinishTimeMS, containerList, reservationId,
|
|
|
- job.getDeadline(), getAMContainerResource(null), null,
|
|
|
- job.getParams());
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private Resource getAMContainerResource(Map jsonJob) {
|
|
|
- Resource amContainerResource =
|
|
|
- SLSConfiguration.getAMContainerResource(getConf());
|
|
|
-
|
|
|
- if (jsonJob == null) {
|
|
|
- return amContainerResource;
|
|
|
- }
|
|
|
-
|
|
|
- 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);
|
|
|
+ .newInstance(rm.getStartTime(), AM_ID);
|
|
|
}
|
|
|
+ AMDefinitionSynth amDef = AMDefinitionFactory.createFromSynth(job, this);
|
|
|
+ startAMs(amDef, reservationId, job.getParams(), job.getDeadline());
|
|
|
}
|
|
|
-
|
|
|
- return amContainerResource;
|
|
|
}
|
|
|
|
|
|
- private void increaseQueueAppNum(String queue) throws YarnException {
|
|
|
+ void increaseQueueAppNum(String queue) throws YarnException {
|
|
|
SchedulerWrapper wrapper = (SchedulerWrapper)rm.getResourceScheduler();
|
|
|
String queueName = wrapper.getRealQueueName(queue);
|
|
|
Integer appNum = queueAppNumMap.get(queueName);
|
|
@@ -830,32 +576,16 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private void runNewAM(String jobType, String user,
|
|
|
- String jobQueue, String oldJobId, long jobStartTimeMS,
|
|
|
- long jobFinishTimeMS, List<ContainerSimulator> containerList,
|
|
|
- Resource amContainerResource) {
|
|
|
- runNewAM(jobType, user, jobQueue, oldJobId, jobStartTimeMS,
|
|
|
- jobFinishTimeMS, containerList, null, -1,
|
|
|
- amContainerResource, null, null);
|
|
|
- }
|
|
|
-
|
|
|
- private void runNewAM(String jobType, String user,
|
|
|
- String jobQueue, String oldJobId, long jobStartTimeMS,
|
|
|
- long jobFinishTimeMS, List<ContainerSimulator> containerList,
|
|
|
- Resource amContainerResource, String labelExpr) {
|
|
|
- runNewAM(jobType, user, jobQueue, oldJobId, jobStartTimeMS,
|
|
|
- jobFinishTimeMS, containerList, null, -1,
|
|
|
- amContainerResource, labelExpr, null);
|
|
|
+ private AMSimulator createAmSimulator(String jobType) {
|
|
|
+ return (AMSimulator) ReflectionUtils.newInstance(
|
|
|
+ amClassMap.get(jobType), new Configuration());
|
|
|
}
|
|
|
|
|
|
- @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(
|
|
|
- amClassMap.get(jobType), new Configuration());
|
|
|
+ private void runNewAM(JobDefinition jobDef) {
|
|
|
+ AMDefinition amDef = jobDef.getAmDefinition();
|
|
|
+ String oldJobId = amDef.getOldAppId();
|
|
|
+ AMSimulator amSim =
|
|
|
+ createAmSimulator(amDef.getAmType());
|
|
|
|
|
|
if (amSim != null) {
|
|
|
int heartbeatInterval = getConf().getInt(
|
|
@@ -867,19 +597,17 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
oldJobId = Integer.toString(AM_ID);
|
|
|
}
|
|
|
AM_ID++;
|
|
|
- amSim.init(heartbeatInterval, containerList, rm, this, jobStartTimeMS,
|
|
|
- jobFinishTimeMS, user, jobQueue, isTracked, oldJobId,
|
|
|
- runner.getStartTimeMS(), amContainerResource, labelExpr, params,
|
|
|
- appIdAMSim);
|
|
|
- if(reservationId != null) {
|
|
|
+ amSim.init(amDef, rm, this, isTracked, runner.getStartTimeMS(), heartbeatInterval, appIdAMSim);
|
|
|
+ if (jobDef.getReservationId() != null) {
|
|
|
// if we have a ReservationId, delegate reservation creation to
|
|
|
// AMSim (reservation shape is impl specific)
|
|
|
UTCClock clock = new UTCClock();
|
|
|
- amSim.initReservation(reservationId, deadline, clock.getTime());
|
|
|
+ amSim.initReservation(jobDef.getReservationId(), jobDef.getDeadline(),
|
|
|
+ clock.getTime());
|
|
|
}
|
|
|
runner.schedule(amSim);
|
|
|
- maxRuntime = Math.max(maxRuntime, jobFinishTimeMS);
|
|
|
- numTasks += containerList.size();
|
|
|
+ maxRuntime = Math.max(maxRuntime, amDef.getJobFinishTime());
|
|
|
+ numTasks += amDef.getTaskContainers().size();
|
|
|
amMap.put(oldJobId, amSim);
|
|
|
}
|
|
|
}
|
|
@@ -1121,4 +849,12 @@ public class SLSRunner extends Configured implements Tool {
|
|
|
return result;
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ public ResourceManager getRm() {
|
|
|
+ return rm;
|
|
|
+ }
|
|
|
+
|
|
|
+ public SynthTraceJobProducer getStjp() {
|
|
|
+ return stjp;
|
|
|
+ }
|
|
|
}
|