YARN-7798. Refactor SLS Reservation Creation. Contributed by Young Chen.
This commit is contained in:
parent
0c139d5bcf
commit
16be42d309
@ -57,7 +57,6 @@
|
|||||||
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.protocolrecords.ReservationSubmissionRequest;
|
|
||||||
import org.apache.hadoop.yarn.api.records.NodeId;
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
||||||
import org.apache.hadoop.yarn.api.records.NodeState;
|
import org.apache.hadoop.yarn.api.records.NodeState;
|
||||||
import org.apache.hadoop.yarn.api.records.ReservationId;
|
import org.apache.hadoop.yarn.api.records.ReservationId;
|
||||||
@ -444,7 +443,7 @@ private void createAMForJob(Map jsonJob) throws YarnException {
|
|||||||
|
|
||||||
for (int i = 0; i < jobCount; i++) {
|
for (int i = 0; i < jobCount; i++) {
|
||||||
runNewAM(amType, user, queue, oldAppId, jobStartTime, jobFinishTime,
|
runNewAM(amType, user, queue, oldAppId, jobStartTime, jobFinishTime,
|
||||||
getTaskContainers(jsonJob), null, getAMContainerResource(jsonJob));
|
getTaskContainers(jsonJob), getAMContainerResource(jsonJob));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -607,7 +606,7 @@ private void createAMForJob(LoggedJob job, long baselineTimeMs)
|
|||||||
|
|
||||||
// Only supports the default job type currently
|
// Only supports the default job type currently
|
||||||
runNewAM(SLSUtils.DEFAULT_JOB_TYPE, user, jobQueue, oldJobId,
|
runNewAM(SLSUtils.DEFAULT_JOB_TYPE, user, jobQueue, oldJobId,
|
||||||
jobStartTimeMS, jobFinishTimeMS, containerList, null,
|
jobStartTimeMS, jobFinishTimeMS, containerList,
|
||||||
getAMContainerResource(null));
|
getAMContainerResource(null));
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -628,10 +627,6 @@ private void startAMFromSynthGenerator() throws YarnException, IOException {
|
|||||||
localConf.set("fs.defaultFS", "file:///");
|
localConf.set("fs.defaultFS", "file:///");
|
||||||
long baselineTimeMS = 0;
|
long baselineTimeMS = 0;
|
||||||
|
|
||||||
// reservations use wall clock time, so need to have a reference for that
|
|
||||||
UTCClock clock = new UTCClock();
|
|
||||||
long now = clock.getTime();
|
|
||||||
|
|
||||||
try {
|
try {
|
||||||
|
|
||||||
// if we use the nodeFile this could have been not initialized yet.
|
// if we use the nodeFile this could have been not initialized yet.
|
||||||
@ -670,13 +665,12 @@ private void startAMFromSynthGenerator() throws YarnException, IOException {
|
|||||||
ArrayList<NodeId> keyAsArray = new ArrayList<NodeId>(nmMap.keySet());
|
ArrayList<NodeId> keyAsArray = new ArrayList<NodeId>(nmMap.keySet());
|
||||||
Random rand = new Random(stjp.getSeed());
|
Random rand = new Random(stjp.getSeed());
|
||||||
|
|
||||||
Resource maxMapRes = Resource.newInstance(0, 0);
|
|
||||||
long maxMapDur = 0;
|
|
||||||
// map tasks
|
// map tasks
|
||||||
for (int i = 0; i < job.getNumberMaps(); i++) {
|
for (int i = 0; i < job.getNumberMaps(); i++) {
|
||||||
TaskAttemptInfo tai = job.getTaskAttemptInfo(TaskType.MAP, i, 0);
|
TaskAttemptInfo tai = job.getTaskAttemptInfo(TaskType.MAP, i, 0);
|
||||||
RMNode node = nmMap
|
RMNode node =
|
||||||
.get(keyAsArray.get(rand.nextInt(keyAsArray.size()))).getNode();
|
nmMap.get(keyAsArray.get(rand.nextInt(keyAsArray.size())))
|
||||||
|
.getNode();
|
||||||
String hostname = "/" + node.getRackName() + "/" + node.getHostName();
|
String hostname = "/" + node.getRackName() + "/" + node.getHostName();
|
||||||
long containerLifeTime = tai.getRuntime();
|
long containerLifeTime = tai.getRuntime();
|
||||||
Resource containerResource =
|
Resource containerResource =
|
||||||
@ -684,55 +678,39 @@ private void startAMFromSynthGenerator() throws YarnException, IOException {
|
|||||||
(int) tai.getTaskInfo().getTaskVCores());
|
(int) tai.getTaskInfo().getTaskVCores());
|
||||||
containerList.add(new ContainerSimulator(containerResource,
|
containerList.add(new ContainerSimulator(containerResource,
|
||||||
containerLifeTime, hostname, DEFAULT_MAPPER_PRIORITY, "map"));
|
containerLifeTime, hostname, DEFAULT_MAPPER_PRIORITY, "map"));
|
||||||
maxMapRes = Resources.componentwiseMax(maxMapRes, containerResource);
|
|
||||||
maxMapDur =
|
|
||||||
containerLifeTime > maxMapDur ? containerLifeTime : maxMapDur;
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
Resource maxRedRes = Resource.newInstance(0, 0);
|
|
||||||
long maxRedDur = 0;
|
|
||||||
// reduce tasks
|
// reduce tasks
|
||||||
for (int i = 0; i < job.getNumberReduces(); i++) {
|
for (int i = 0; i < job.getNumberReduces(); i++) {
|
||||||
TaskAttemptInfo tai = job.getTaskAttemptInfo(TaskType.REDUCE, i, 0);
|
TaskAttemptInfo tai = job.getTaskAttemptInfo(TaskType.REDUCE, i, 0);
|
||||||
RMNode node = nmMap
|
RMNode node =
|
||||||
.get(keyAsArray.get(rand.nextInt(keyAsArray.size()))).getNode();
|
nmMap.get(keyAsArray.get(rand.nextInt(keyAsArray.size())))
|
||||||
|
.getNode();
|
||||||
String hostname = "/" + node.getRackName() + "/" + node.getHostName();
|
String hostname = "/" + node.getRackName() + "/" + node.getHostName();
|
||||||
long containerLifeTime = tai.getRuntime();
|
long containerLifeTime = tai.getRuntime();
|
||||||
Resource containerResource =
|
Resource containerResource =
|
||||||
Resource.newInstance((int) tai.getTaskInfo().getTaskMemory(),
|
Resource.newInstance((int) tai.getTaskInfo().getTaskMemory(),
|
||||||
(int) tai.getTaskInfo().getTaskVCores());
|
(int) tai.getTaskInfo().getTaskVCores());
|
||||||
containerList.add(new ContainerSimulator(containerResource,
|
containerList.add(
|
||||||
containerLifeTime, hostname, DEFAULT_REDUCER_PRIORITY, "reduce"));
|
new ContainerSimulator(containerResource, containerLifeTime,
|
||||||
maxRedRes = Resources.componentwiseMax(maxRedRes, containerResource);
|
hostname, DEFAULT_REDUCER_PRIORITY, "reduce"));
|
||||||
maxRedDur =
|
|
||||||
containerLifeTime > maxRedDur ? containerLifeTime : maxRedDur;
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// generating reservations for the jobs that require them
|
ReservationId reservationId = null;
|
||||||
|
|
||||||
ReservationSubmissionRequest rr = null;
|
|
||||||
if (job.hasDeadline()) {
|
if (job.hasDeadline()) {
|
||||||
ReservationId reservationId =
|
reservationId =
|
||||||
ReservationId.newInstance(this.rm.getStartTime(), AM_ID);
|
ReservationId.newInstance(this.rm.getStartTime(), AM_ID);
|
||||||
|
|
||||||
rr = ReservationClientUtil.createMRReservation(reservationId,
|
|
||||||
"reservation_" + AM_ID, maxMapRes, job.getNumberMaps(), maxMapDur,
|
|
||||||
maxRedRes, job.getNumberReduces(), maxRedDur,
|
|
||||||
now + jobStartTimeMS, now + job.getDeadline(),
|
|
||||||
job.getQueueName());
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
runNewAM(SLSUtils.DEFAULT_JOB_TYPE, user, jobQueue, oldJobId,
|
runNewAM(SLSUtils.DEFAULT_JOB_TYPE, user, jobQueue, oldJobId,
|
||||||
jobStartTimeMS, jobFinishTimeMS, containerList, rr,
|
jobStartTimeMS, jobFinishTimeMS, containerList, reservationId,
|
||||||
getAMContainerResource(null));
|
job.getDeadline(), getAMContainerResource(null));
|
||||||
|
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
stjp.close();
|
stjp.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private Resource getAMContainerResource(Map jsonJob) {
|
private Resource getAMContainerResource(Map jsonJob) {
|
||||||
@ -772,7 +750,17 @@ private void increaseQueueAppNum(String queue) throws YarnException {
|
|||||||
private void runNewAM(String jobType, String user,
|
private void runNewAM(String jobType, String user,
|
||||||
String jobQueue, String oldJobId, long jobStartTimeMS,
|
String jobQueue, String oldJobId, long jobStartTimeMS,
|
||||||
long jobFinishTimeMS, List<ContainerSimulator> containerList,
|
long jobFinishTimeMS, List<ContainerSimulator> containerList,
|
||||||
ReservationSubmissionRequest rr, Resource amContainerResource) {
|
Resource amContainerResource) {
|
||||||
|
runNewAM(jobType, user, jobQueue, oldJobId, jobStartTimeMS,
|
||||||
|
jobFinishTimeMS, containerList, null, -1,
|
||||||
|
amContainerResource);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void runNewAM(String jobType, String user,
|
||||||
|
String jobQueue, String oldJobId, long jobStartTimeMS,
|
||||||
|
long jobFinishTimeMS, List<ContainerSimulator> containerList,
|
||||||
|
ReservationId reservationId, long deadline,
|
||||||
|
Resource amContainerResource) {
|
||||||
|
|
||||||
AMSimulator amSim = (AMSimulator) ReflectionUtils.newInstance(
|
AMSimulator amSim = (AMSimulator) ReflectionUtils.newInstance(
|
||||||
amClassMap.get(jobType), new Configuration());
|
amClassMap.get(jobType), new Configuration());
|
||||||
@ -787,10 +775,15 @@ private void runNewAM(String jobType, String user,
|
|||||||
oldJobId = Integer.toString(AM_ID);
|
oldJobId = Integer.toString(AM_ID);
|
||||||
}
|
}
|
||||||
AM_ID++;
|
AM_ID++;
|
||||||
|
|
||||||
amSim.init(heartbeatInterval, containerList, rm, this, jobStartTimeMS,
|
amSim.init(heartbeatInterval, containerList, rm, this, jobStartTimeMS,
|
||||||
jobFinishTimeMS, user, jobQueue, isTracked, oldJobId, rr,
|
jobFinishTimeMS, user, jobQueue, isTracked, oldJobId,
|
||||||
runner.getStartTimeMS(), amContainerResource);
|
runner.getStartTimeMS(), amContainerResource);
|
||||||
|
if(reservationId != 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());
|
||||||
|
}
|
||||||
runner.schedule(amSim);
|
runner.schedule(amSim);
|
||||||
maxRuntime = Math.max(maxRuntime, jobFinishTimeMS);
|
maxRuntime = Math.max(maxRuntime, jobFinishTimeMS);
|
||||||
numTasks += containerList.size();
|
numTasks += containerList.size();
|
||||||
|
@ -85,7 +85,7 @@ public abstract class AMSimulator extends TaskRunner.Task {
|
|||||||
protected final BlockingQueue<AllocateResponse> responseQueue;
|
protected final BlockingQueue<AllocateResponse> responseQueue;
|
||||||
private int responseId = 0;
|
private int responseId = 0;
|
||||||
// user name
|
// user name
|
||||||
protected String user;
|
private String user;
|
||||||
// queue name
|
// queue name
|
||||||
protected String queue;
|
protected String queue;
|
||||||
// am type
|
// am type
|
||||||
@ -105,7 +105,7 @@ public abstract class AMSimulator extends TaskRunner.Task {
|
|||||||
// waiting for AM container
|
// waiting for AM container
|
||||||
volatile boolean isAMContainerRunning = false;
|
volatile boolean isAMContainerRunning = false;
|
||||||
volatile Container amContainer;
|
volatile Container amContainer;
|
||||||
|
|
||||||
private static final Logger LOG = LoggerFactory.getLogger(AMSimulator.class);
|
private static final Logger LOG = LoggerFactory.getLogger(AMSimulator.class);
|
||||||
|
|
||||||
private Resource amContainerResource;
|
private Resource amContainerResource;
|
||||||
@ -120,9 +120,8 @@ public AMSimulator() {
|
|||||||
public void init(int heartbeatInterval,
|
public void init(int heartbeatInterval,
|
||||||
List<ContainerSimulator> containerList, ResourceManager resourceManager,
|
List<ContainerSimulator> containerList, ResourceManager resourceManager,
|
||||||
SLSRunner slsRunnner, long startTime, long finishTime, String simUser,
|
SLSRunner slsRunnner, long startTime, long finishTime, String simUser,
|
||||||
String simQueue, boolean tracked, String oldApp,
|
String simQueue, boolean tracked, String oldApp, long baseTimeMS,
|
||||||
ReservationSubmissionRequest rr, long baseTimeMS,
|
Resource amResource) {
|
||||||
Resource amContainerResource) {
|
|
||||||
super.init(startTime, startTime + 1000000L * heartbeatInterval,
|
super.init(startTime, startTime + 1000000L * heartbeatInterval,
|
||||||
heartbeatInterval);
|
heartbeatInterval);
|
||||||
this.user = simUser;
|
this.user = simUser;
|
||||||
@ -134,8 +133,7 @@ public void init(int heartbeatInterval,
|
|||||||
this.baselineTimeMS = baseTimeMS;
|
this.baselineTimeMS = baseTimeMS;
|
||||||
this.traceStartTimeMS = startTime;
|
this.traceStartTimeMS = startTime;
|
||||||
this.traceFinishTimeMS = finishTime;
|
this.traceFinishTimeMS = finishTime;
|
||||||
this.reservationRequest = rr;
|
this.amContainerResource = amResource;
|
||||||
this.amContainerResource = amContainerResource;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -171,6 +169,10 @@ public synchronized void notifyAMContainerLaunched(Container masterContainer)
|
|||||||
isAMContainerRunning = true;
|
isAMContainerRunning = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected void setReservationRequest(ReservationSubmissionRequest rr){
|
||||||
|
this.reservationRequest = rr;
|
||||||
|
}
|
||||||
|
|
||||||
private ReservationId submitReservationWhenSpecified()
|
private ReservationId submitReservationWhenSpecified()
|
||||||
throws IOException, InterruptedException {
|
throws IOException, InterruptedException {
|
||||||
if (reservationRequest != null) {
|
if (reservationRequest != null) {
|
||||||
@ -256,7 +258,7 @@ public Object run() throws Exception {
|
|||||||
simulateStartTimeMS, simulateFinishTimeMS);
|
simulateStartTimeMS, simulateFinishTimeMS);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
protected ResourceRequest createResourceRequest(
|
protected ResourceRequest createResourceRequest(
|
||||||
Resource resource, String host, int priority, int numContainers) {
|
Resource resource, String host, int priority, int numContainers) {
|
||||||
ResourceRequest request = recordFactory
|
ResourceRequest request = recordFactory
|
||||||
@ -269,7 +271,7 @@ protected ResourceRequest createResourceRequest(
|
|||||||
request.setPriority(prio);
|
request.setPriority(prio);
|
||||||
return request;
|
return request;
|
||||||
}
|
}
|
||||||
|
|
||||||
protected AllocateRequest createAllocateRequest(List<ResourceRequest> ask,
|
protected AllocateRequest createAllocateRequest(List<ResourceRequest> ask,
|
||||||
List<ContainerId> toRelease) {
|
List<ContainerId> toRelease) {
|
||||||
AllocateRequest allocateRequest =
|
AllocateRequest allocateRequest =
|
||||||
@ -279,36 +281,39 @@ protected AllocateRequest createAllocateRequest(List<ResourceRequest> ask,
|
|||||||
allocateRequest.setReleaseList(toRelease);
|
allocateRequest.setReleaseList(toRelease);
|
||||||
return allocateRequest;
|
return allocateRequest;
|
||||||
}
|
}
|
||||||
|
|
||||||
protected AllocateRequest createAllocateRequest(List<ResourceRequest> ask) {
|
protected AllocateRequest createAllocateRequest(List<ResourceRequest> ask) {
|
||||||
return createAllocateRequest(ask, new ArrayList<ContainerId>());
|
return createAllocateRequest(ask, new ArrayList<ContainerId>());
|
||||||
}
|
}
|
||||||
|
|
||||||
protected abstract void processResponseQueue() throws Exception;
|
protected abstract void processResponseQueue() throws Exception;
|
||||||
|
|
||||||
protected abstract void sendContainerRequest() throws Exception;
|
protected abstract void sendContainerRequest() throws Exception;
|
||||||
|
|
||||||
|
public abstract void initReservation(
|
||||||
|
ReservationId reservationId, long deadline, long now);
|
||||||
|
|
||||||
protected abstract void checkStop();
|
protected abstract void checkStop();
|
||||||
|
|
||||||
private void submitApp(ReservationId reservationId)
|
private void submitApp(ReservationId reservationId)
|
||||||
throws YarnException, InterruptedException, IOException {
|
throws YarnException, InterruptedException, IOException {
|
||||||
// ask for new application
|
// ask for new application
|
||||||
GetNewApplicationRequest newAppRequest =
|
GetNewApplicationRequest newAppRequest =
|
||||||
Records.newRecord(GetNewApplicationRequest.class);
|
Records.newRecord(GetNewApplicationRequest.class);
|
||||||
GetNewApplicationResponse newAppResponse =
|
GetNewApplicationResponse newAppResponse =
|
||||||
rm.getClientRMService().getNewApplication(newAppRequest);
|
rm.getClientRMService().getNewApplication(newAppRequest);
|
||||||
appId = newAppResponse.getApplicationId();
|
appId = newAppResponse.getApplicationId();
|
||||||
|
|
||||||
// submit the application
|
// submit the application
|
||||||
final SubmitApplicationRequest subAppRequest =
|
final SubmitApplicationRequest subAppRequest =
|
||||||
Records.newRecord(SubmitApplicationRequest.class);
|
Records.newRecord(SubmitApplicationRequest.class);
|
||||||
ApplicationSubmissionContext appSubContext =
|
ApplicationSubmissionContext appSubContext =
|
||||||
Records.newRecord(ApplicationSubmissionContext.class);
|
Records.newRecord(ApplicationSubmissionContext.class);
|
||||||
appSubContext.setApplicationId(appId);
|
appSubContext.setApplicationId(appId);
|
||||||
appSubContext.setMaxAppAttempts(1);
|
appSubContext.setMaxAppAttempts(1);
|
||||||
appSubContext.setQueue(queue);
|
appSubContext.setQueue(queue);
|
||||||
appSubContext.setPriority(Priority.newInstance(0));
|
appSubContext.setPriority(Priority.newInstance(0));
|
||||||
ContainerLaunchContext conLauContext =
|
ContainerLaunchContext conLauContext =
|
||||||
Records.newRecord(ContainerLaunchContext.class);
|
Records.newRecord(ContainerLaunchContext.class);
|
||||||
conLauContext.setApplicationACLs(new HashMap<>());
|
conLauContext.setApplicationACLs(new HashMap<>());
|
||||||
conLauContext.setCommands(new ArrayList<>());
|
conLauContext.setCommands(new ArrayList<>());
|
||||||
@ -379,7 +384,7 @@ public void untrackApp() {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
protected List<ResourceRequest> packageRequests(
|
protected List<ResourceRequest> packageRequests(
|
||||||
List<ContainerSimulator> csList, int priority) {
|
List<ContainerSimulator> csList, int priority) {
|
||||||
// create requests
|
// create requests
|
||||||
|
@ -21,6 +21,7 @@
|
|||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.security.PrivilegedExceptionAction;
|
import java.security.PrivilegedExceptionAction;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collection;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.LinkedList;
|
import java.util.LinkedList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
@ -34,6 +35,7 @@
|
|||||||
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
|
import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
|
||||||
import org.apache.hadoop.yarn.api.records.Container;
|
import org.apache.hadoop.yarn.api.records.Container;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ReservationId;
|
||||||
import org.apache.hadoop.yarn.api.records.Resource;
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
|
import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
@ -42,8 +44,10 @@
|
|||||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||||
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
|
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
|
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
|
||||||
|
import org.apache.hadoop.yarn.sls.ReservationClientUtil;
|
||||||
import org.apache.hadoop.yarn.sls.scheduler.ContainerSimulator;
|
import org.apache.hadoop.yarn.sls.scheduler.ContainerSimulator;
|
||||||
import org.apache.hadoop.yarn.sls.SLSRunner;
|
import org.apache.hadoop.yarn.sls.SLSRunner;
|
||||||
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
@ -51,51 +55,51 @@
|
|||||||
@Unstable
|
@Unstable
|
||||||
public class MRAMSimulator extends AMSimulator {
|
public class MRAMSimulator extends AMSimulator {
|
||||||
/*
|
/*
|
||||||
Vocabulary Used:
|
Vocabulary Used:
|
||||||
pending -> requests which are NOT yet sent to RM
|
pending -> requests which are NOT yet sent to RM
|
||||||
scheduled -> requests which are sent to RM but not yet assigned
|
scheduled -> requests which are sent to RM but not yet assigned
|
||||||
assigned -> requests which are assigned to a container
|
assigned -> requests which are assigned to a container
|
||||||
completed -> request corresponding to which container has completed
|
completed -> request corresponding to which container has completed
|
||||||
|
|
||||||
Maps are scheduled as soon as their requests are received. Reduces are
|
Maps are scheduled as soon as their requests are received. Reduces are
|
||||||
scheduled when all maps have finished (not support slow-start currently).
|
scheduled when all maps have finished (not support slow-start currently).
|
||||||
*/
|
*/
|
||||||
|
|
||||||
private static final int PRIORITY_REDUCE = 10;
|
private static final int PRIORITY_REDUCE = 10;
|
||||||
private static final int PRIORITY_MAP = 20;
|
private static final int PRIORITY_MAP = 20;
|
||||||
|
|
||||||
// pending maps
|
// pending maps
|
||||||
private LinkedList<ContainerSimulator> pendingMaps =
|
private LinkedList<ContainerSimulator> pendingMaps =
|
||||||
new LinkedList<>();
|
new LinkedList<>();
|
||||||
|
|
||||||
// pending failed maps
|
// pending failed maps
|
||||||
private LinkedList<ContainerSimulator> pendingFailedMaps =
|
private LinkedList<ContainerSimulator> pendingFailedMaps =
|
||||||
new LinkedList<ContainerSimulator>();
|
new LinkedList<ContainerSimulator>();
|
||||||
|
|
||||||
// scheduled maps
|
// scheduled maps
|
||||||
private LinkedList<ContainerSimulator> scheduledMaps =
|
private LinkedList<ContainerSimulator> scheduledMaps =
|
||||||
new LinkedList<ContainerSimulator>();
|
new LinkedList<ContainerSimulator>();
|
||||||
|
|
||||||
// assigned maps
|
// assigned maps
|
||||||
private Map<ContainerId, ContainerSimulator> assignedMaps =
|
private Map<ContainerId, ContainerSimulator> assignedMaps =
|
||||||
new HashMap<ContainerId, ContainerSimulator>();
|
new HashMap<ContainerId, ContainerSimulator>();
|
||||||
|
|
||||||
// reduces which are not yet scheduled
|
// reduces which are not yet scheduled
|
||||||
private LinkedList<ContainerSimulator> pendingReduces =
|
private LinkedList<ContainerSimulator> pendingReduces =
|
||||||
new LinkedList<ContainerSimulator>();
|
new LinkedList<ContainerSimulator>();
|
||||||
|
|
||||||
// pending failed reduces
|
// pending failed reduces
|
||||||
private LinkedList<ContainerSimulator> pendingFailedReduces =
|
private LinkedList<ContainerSimulator> pendingFailedReduces =
|
||||||
new LinkedList<ContainerSimulator>();
|
new LinkedList<ContainerSimulator>();
|
||||||
|
|
||||||
// scheduled reduces
|
// scheduled reduces
|
||||||
private LinkedList<ContainerSimulator> scheduledReduces =
|
private LinkedList<ContainerSimulator> scheduledReduces =
|
||||||
new LinkedList<ContainerSimulator>();
|
new LinkedList<ContainerSimulator>();
|
||||||
|
|
||||||
// assigned reduces
|
// assigned reduces
|
||||||
private Map<ContainerId, ContainerSimulator> assignedReduces =
|
private Map<ContainerId, ContainerSimulator> assignedReduces =
|
||||||
new HashMap<ContainerId, ContainerSimulator>();
|
new HashMap<ContainerId, ContainerSimulator>();
|
||||||
|
|
||||||
// all maps & reduces
|
// all maps & reduces
|
||||||
private LinkedList<ContainerSimulator> allMaps =
|
private LinkedList<ContainerSimulator> allMaps =
|
||||||
new LinkedList<ContainerSimulator>();
|
new LinkedList<ContainerSimulator>();
|
||||||
@ -117,14 +121,14 @@ scheduled when all maps have finished (not support slow-start currently).
|
|||||||
@SuppressWarnings("checkstyle:parameternumber")
|
@SuppressWarnings("checkstyle:parameternumber")
|
||||||
public void init(int heartbeatInterval,
|
public void init(int heartbeatInterval,
|
||||||
List<ContainerSimulator> containerList, ResourceManager rm, SLSRunner se,
|
List<ContainerSimulator> containerList, ResourceManager rm, SLSRunner se,
|
||||||
long traceStartTime, long traceFinishTime, String user, String queue,
|
long traceStartTime, long traceFinishTime, String user, String queue,
|
||||||
boolean isTracked, String oldAppId, ReservationSubmissionRequest rr,
|
boolean isTracked, String oldAppId, long baselineStartTimeMS,
|
||||||
long baselineStartTimeMS, Resource amContainerResource) {
|
Resource amContainerResource) {
|
||||||
super.init(heartbeatInterval, containerList, rm, se,
|
super.init(heartbeatInterval, containerList, rm, se,
|
||||||
traceStartTime, traceFinishTime, user, queue, isTracked, oldAppId,
|
traceStartTime, traceFinishTime, user, queue, isTracked, oldAppId,
|
||||||
rr, baselineStartTimeMS, amContainerResource);
|
baselineStartTimeMS, amContainerResource);
|
||||||
amtype = "mapreduce";
|
amtype = "mapreduce";
|
||||||
|
|
||||||
// get map/reduce tasks
|
// get map/reduce tasks
|
||||||
for (ContainerSimulator cs : containerList) {
|
for (ContainerSimulator cs : containerList) {
|
||||||
if (cs.getType().equals("map")) {
|
if (cs.getType().equals("map")) {
|
||||||
@ -202,7 +206,7 @@ protected void processResponseQueue() throws Exception {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// check finished
|
// check finished
|
||||||
if (isAMContainerRunning &&
|
if (isAMContainerRunning &&
|
||||||
(mapFinished >= mapTotal) &&
|
(mapFinished >= mapTotal) &&
|
||||||
@ -234,7 +238,7 @@ protected void processResponseQueue() throws Exception {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* restart running because of the am container killed
|
* restart running because of the am container killed
|
||||||
*/
|
*/
|
||||||
@ -322,7 +326,7 @@ protected void sendContainerRequest()
|
|||||||
if (ask == null) {
|
if (ask == null) {
|
||||||
ask = new ArrayList<>();
|
ask = new ArrayList<>();
|
||||||
}
|
}
|
||||||
|
|
||||||
final AllocateRequest request = createAllocateRequest(ask);
|
final AllocateRequest request = createAllocateRequest(ask);
|
||||||
if (totalContainers == 0) {
|
if (totalContainers == 0) {
|
||||||
request.setProgress(1.0f);
|
request.setProgress(1.0f);
|
||||||
@ -348,6 +352,38 @@ public AllocateResponse run() throws Exception {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void initReservation(ReservationId reservationId, long deadline,
|
||||||
|
long now) {
|
||||||
|
|
||||||
|
Resource mapRes = getMaxResource(allMaps);
|
||||||
|
long mapDur = getMaxDuration(allMaps);
|
||||||
|
Resource redRes = getMaxResource(allReduces);
|
||||||
|
long redDur = getMaxDuration(allReduces);
|
||||||
|
|
||||||
|
ReservationSubmissionRequest rr = ReservationClientUtil.
|
||||||
|
createMRReservation(reservationId,
|
||||||
|
"reservation_" + reservationId.getId(), mapRes, allMaps.size(),
|
||||||
|
mapDur, redRes, allReduces.size(), redDur, now + traceStartTimeMS,
|
||||||
|
now + deadline, queue);
|
||||||
|
|
||||||
|
setReservationRequest(rr);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Helper to compute the component-wise maximum resource used by any container
|
||||||
|
private Resource getMaxResource(Collection<ContainerSimulator> containers) {
|
||||||
|
return containers.parallelStream()
|
||||||
|
.map(ContainerSimulator::getResource)
|
||||||
|
.reduce(Resource.newInstance(0, 0), Resources::componentwiseMax);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Helper to compute the maximum resource used by any map container
|
||||||
|
private long getMaxDuration(Collection<ContainerSimulator> containers) {
|
||||||
|
return containers.parallelStream()
|
||||||
|
.mapToLong(ContainerSimulator::getLifeTime)
|
||||||
|
.reduce(0L, Long::max);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void checkStop() {
|
protected void checkStop() {
|
||||||
if (isFinished) {
|
if (isFinished) {
|
||||||
|
@ -78,7 +78,7 @@ public void setup() {
|
|||||||
exitInvariantFile = "src/test/resources/exit-invariants.txt";
|
exitInvariantFile = "src/test/resources/exit-invariants.txt";
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout = 60000)
|
@Test(timeout = 90000)
|
||||||
@SuppressWarnings("all")
|
@SuppressWarnings("all")
|
||||||
public void testSimulatorRunning() throws Exception {
|
public void testSimulatorRunning() throws Exception {
|
||||||
Configuration conf = new Configuration(false);
|
Configuration conf = new Configuration(false);
|
||||||
|
@ -19,6 +19,7 @@
|
|||||||
|
|
||||||
import com.codahale.metrics.MetricRegistry;
|
import com.codahale.metrics.MetricRegistry;
|
||||||
import org.apache.commons.io.FileUtils;
|
import org.apache.commons.io.FileUtils;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ReservationId;
|
||||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
|
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
|
||||||
@ -89,6 +90,10 @@ protected void sendContainerRequest()
|
|||||||
throws YarnException, IOException, InterruptedException {
|
throws YarnException, IOException, InterruptedException {
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void initReservation(ReservationId id, long deadline, long now){
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void checkStop() {
|
protected void checkStop() {
|
||||||
}
|
}
|
||||||
@ -134,7 +139,7 @@ public void testAMSimulator() throws Exception {
|
|||||||
String queue = "default";
|
String queue = "default";
|
||||||
List<ContainerSimulator> containers = new ArrayList<>();
|
List<ContainerSimulator> containers = new ArrayList<>();
|
||||||
app.init(1000, containers, rm, null, 0, 1000000L, "user1", queue, true,
|
app.init(1000, containers, rm, null, 0, 1000000L, "user1", queue, true,
|
||||||
appId, null, 0, SLSConfiguration.getAMContainerResource(conf));
|
appId, 0, SLSConfiguration.getAMContainerResource(conf));
|
||||||
app.firstStep();
|
app.firstStep();
|
||||||
|
|
||||||
verifySchedulerMetrics(appId);
|
verifySchedulerMetrics(appId);
|
||||||
|
Loading…
Reference in New Issue
Block a user