|
@@ -60,6 +60,7 @@ import org.apache.hadoop.tools.rumen.LoggedTaskAttempt;
|
|
import org.apache.hadoop.util.ReflectionUtils;
|
|
import org.apache.hadoop.util.ReflectionUtils;
|
|
import org.apache.hadoop.util.Tool;
|
|
import org.apache.hadoop.util.Tool;
|
|
import org.apache.hadoop.util.ToolRunner;
|
|
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.ExecutionType;
|
|
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;
|
|
@@ -113,6 +114,7 @@ public class SLSRunner extends Configured implements Tool {
|
|
// AM simulator
|
|
// AM simulator
|
|
private int AM_ID;
|
|
private int AM_ID;
|
|
private Map<String, AMSimulator> amMap;
|
|
private Map<String, AMSimulator> amMap;
|
|
|
|
+ private Map<ApplicationId, AMSimulator> appIdAMSim;
|
|
private Set<String> trackedApps;
|
|
private Set<String> trackedApps;
|
|
private Map<String, Class> amClassMap;
|
|
private Map<String, Class> amClassMap;
|
|
private static int remainingApps = 0;
|
|
private static int remainingApps = 0;
|
|
@@ -170,7 +172,7 @@ public class SLSRunner extends Configured implements Tool {
|
|
queueAppNumMap = new HashMap<>();
|
|
queueAppNumMap = new HashMap<>();
|
|
amMap = new ConcurrentHashMap<>();
|
|
amMap = new ConcurrentHashMap<>();
|
|
amClassMap = new HashMap<>();
|
|
amClassMap = new HashMap<>();
|
|
-
|
|
|
|
|
|
+ appIdAMSim = new ConcurrentHashMap<>();
|
|
// runner configuration
|
|
// runner configuration
|
|
setConf(tempConf);
|
|
setConf(tempConf);
|
|
|
|
|
|
@@ -277,7 +279,7 @@ public class SLSRunner extends Configured implements Tool {
|
|
rm = new ResourceManager() {
|
|
rm = new ResourceManager() {
|
|
@Override
|
|
@Override
|
|
protected ApplicationMasterLauncher createAMLauncher() {
|
|
protected ApplicationMasterLauncher createAMLauncher() {
|
|
- return new MockAMLauncher(se, this.rmContext, amMap);
|
|
|
|
|
|
+ return new MockAMLauncher(se, this.rmContext, appIdAMSim);
|
|
}
|
|
}
|
|
};
|
|
};
|
|
|
|
|
|
@@ -587,7 +589,7 @@ public class SLSRunner extends Configured implements Tool {
|
|
try {
|
|
try {
|
|
createAMForJob(job, baselineTimeMS);
|
|
createAMForJob(job, baselineTimeMS);
|
|
} catch (Exception e) {
|
|
} catch (Exception e) {
|
|
- LOG.error("Failed to create an AM: {}", e.getMessage());
|
|
|
|
|
|
+ LOG.error("Failed to create an AM", e);
|
|
}
|
|
}
|
|
|
|
|
|
job = reader.getNext();
|
|
job = reader.getNext();
|
|
@@ -808,7 +810,8 @@ public class SLSRunner extends Configured implements Tool {
|
|
AM_ID++;
|
|
AM_ID++;
|
|
amSim.init(heartbeatInterval, containerList, rm, this, jobStartTimeMS,
|
|
amSim.init(heartbeatInterval, containerList, rm, this, jobStartTimeMS,
|
|
jobFinishTimeMS, user, jobQueue, isTracked, oldJobId,
|
|
jobFinishTimeMS, user, jobQueue, isTracked, oldJobId,
|
|
- runner.getStartTimeMS(), amContainerResource, labelExpr, params);
|
|
|
|
|
|
+ runner.getStartTimeMS(), amContainerResource, labelExpr, params,
|
|
|
|
+ appIdAMSim);
|
|
if(reservationId != null) {
|
|
if(reservationId != null) {
|
|
// if we have a ReservationId, delegate reservation creation to
|
|
// if we have a ReservationId, delegate reservation creation to
|
|
// AMSim (reservation shape is impl specific)
|
|
// AMSim (reservation shape is impl specific)
|