研究调度器,我们首先要知道调度器的客户是谁,谁需要调度,有以下两种来源:
(1)作业提交的时候,申请运行AppMaster需要的容器时
(2)作业运行期间,AppMaster申请运行map以及reduce任务,或者spark任务等等所需要的容器
这一节主要分析第一种请求的源码,即申请AppMaster需要的容器时候的整个过程。申请运行AppMaster需要的容器的起源:
当AppMaster运行需要的容器是在RMAppAttempt对象处于Submttied状态,收到ATTEMPT_ADDED事件,然后调用SchedulerTransition()进行申请的:
RMAppAttempt的存在的实例是RMAppAttemptImpl,对应申请过程的状态机跳变事件的代码为:
@Override
public RMAppAttemptState transition(RMAppAttemptImpl appAttempt,
RMAppAttemptEvent event) {
ApplicationSubmissionContext subCtx = appAttempt.submissionContext;
if (!subCtx.getUnmanagedAM()) {
// Need reset #containers before create new attempt, because this request
// will be passed to scheduler, and scheduler will deduct the number after
// AM container allocated
// Currently, following fields are all hard coded,
// TODO: change these fields when we want to support
// priority or multiple containers AM container allocation.
for (ResourceRequest amReq : appAttempt.amReqs) {
amReq.setNumContainers(1);
amReq.setPriority(AM_CONTAINER_PRIORITY);
}
设置好容器个数优先级参数后,进行容器的申请,所谓的getUnmanagedAM是判断AM是否是通过RM进行申请的,如果是就是正常的AM,如果不是说明AM是用户自己通过命令行起起来的而不通过RM进行申请。
// AM resource has been checked when submission
Allocation amContainerAllocation =
appAttempt.scheduler.allocate(
appAttempt.applicationAttemptId,
appAttempt.amReqs,
EMPTY_CONTAINER_RELEASE_LIST,
amBlacklist.getAdditions(),
amBlacklist.getRemovals());
if (amContainerAllocation != null
&& amContainerAllocation.getContainers() != null) {
assert (amContainerAllocation.getContainers().size() == 0);
}
return RMAppAttemptState.SCHEDULED;
} else {
// save state and then go to LAUNCHED state
appAttempt.storeAttempt();
return RMAppAttemptState.LAUNCHED_UNMANAGED_SAVING;
}
根据所选择的调度器的不同,调用对应调度器的allocate函数,这里调用的是FairScheduler的allocate函数,如果该RMAppAttempt的allocate没有收揽到容器(amContainerAllocation.getContainers().size() == 0),那么将会停留在RMAppAttemptState.SCHEDULED。appAttempt.scheduler.allocate()这个函数中,一旦收揽到了容器,会把触发RMContainerEventType.ACQUIRED事件,从而推动RMAppAttempt的状态机事件RMAppAttemptState.CONTAINER_ALLOCATED,状态机的之间都是互相推动相辅相成。
所谓allocate函数只不过是从新分配容器的列表中,收揽过来RMContainer对象,并为这些容器办理NMToken等对象,然后打包成一个Allocation对象。
@Override
public Allocation allocate(ApplicationAttemptId appAttemptId,
List<ResourceRequest> ask, List<ContainerId> release,
List<String> blacklistAdditions, List<String> blacklistRemovals) {
// Make sure this application exists
FSAppAttempt application = getSchedulerApp(appAttemptId);
if (application == null) {
LOG.info("Calling allocate on removed " +
"or non existant application " + appAttemptId);
return EMPTY_ALLOCATION;
}
// Sanity check
SchedulerUtils.normalizeRequests(ask, DOMINANT_RESOURCE_CALCULATOR,
getClusterResource(), minimumAllocation, getMaximumResourceCapability(),
incrAllocation);
// Record container allocation start time
application.recordContainerRequestTime(getClock().getTime());
// Release containers
releaseContainers(release, application);
synchronized (application) {
if (!ask.isEmpty()) {
if (LOG.isDebugEnabled()) {
LOG.debug("allocate: pre-update" +
" applicationAttemptId=" + appAttemptId +
" application=" + application.getApplicationId());
}
application.showRequests();
// Update application requests
application.updateResourceRequests(ask);
application.showRequests();
}
Set<ContainerId> preemptionContainerIds =
application.getPreemptionContainerIds();
if (LOG.isDebugEnabled()) {
LOG.debug(
"allocate: post-update" + " applicationAttemptId=" + appAttemptId
+ " #ask=" + ask.size() + " reservation= " + application
.getCurrentReservation());
LOG.debug("Preempting " + preemptionContainerIds.size()
+ " container(s)");
}
if (application.isWaitingForAMContainer(application.getApplicationId())) {
// Allocate is for AM and update AM blacklist for this
application.updateAMBlacklist(
blacklistAdditions, blacklistRemovals);
} else {
application.updateBlacklist(blacklistAdditions, blacklistRemovals);
}
ContainersAndNMTokensAllocation allocation =
application.pullNewlyAllocatedContainersAndNMTokens();
// Record container allocation time
if (!(allocation.getContainerList().isEmpty())) {
application.recordContainerAllocationTime(getClock().getTime());
}
return new Allocation(allocation.getContainerList(),
application.getHeadroom(), preemptionContainerIds, null, null,
allocation.getNMTokenList());
}
}
根据上面我们传入的参数:
appAttempt.scheduler.allocate(
appAttempt.applicationAttemptId,
appAttempt.amReqs,
EMPTY_CONTAINER_RELEASE_LIST,
amBlacklist.getAdditions(),
amBlacklist.getRemovals());
传入需要release的container为空,而release container上面部分都是一些验证工作,我们继续往下看关键代码
application.updateResourceRequests(ask);
这个函数的作用是把这些资源请求,更新到本次app尝试的对应的appSchedulingInfo中,appSchedulingInfo是每个app尝试相关的调度信息。
public synchronized void updateResourceRequests(
List<ResourceRequest> requests) {
if (!isStopped) {
appSchedulingInfo.updateResourceRequests(requests, false);
}
}
我们看一下这个appAttempt对应的信息,有相关的id对应app的id,还有所在队列的一些信息。以及需要对资源请求进行更新的请求信息,优先级信息,是否是黑名单等等。
public AppSchedulingInfo(ApplicationAttemptId appAttemptId,
String user, Queue queue, ActiveUsersManager activeUsersManager,
long epoch) {
this.applicationAttemptId = appAttemptId;
this.applicationId = appAttemptId.getApplicationId();
this.queue = queue;
this.queueName = queue.getQueueName();
this.user = user;
this.activeUsersManager = activeUsersManager;
this.containerIdCounter = new AtomicLong(epoch << EPOCH_BIT_SHIFT);
}
final Set<Priority> priorities = new TreeSet<Priority>(
new org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.Comparator());
final Map<Priority, Map<String, ResourceRequest>> requests =
new HashMap<Priority, Map<String, ResourceRequest>>();
private AtomicBoolean userBlacklistChanged = new AtomicBoolean(false);
private Set<String> userBlacklist = new HashSet<>();
private Set<String> amBlacklist = new HashSet<>();
FairScheduler的allocate函数,继续往下看,先不考虑其中的抢占的代码,之后会专门对抢占的情况进行分析:
public synchronized ContainersAndNMTokensAllocation
pullNewlyAllocatedContainersAndNMTokens() {
List<Container> returnContainerList =
new ArrayList<Container>(newlyAllocatedContainers.size());
List<NMToken> nmTokens = new ArrayList<NMToken>();
for (Iterator<RMContainer> i = newlyAllocatedContainers.iterator(); i
.hasNext();) {
RMContainer rmContainer = i.next();
Container container = rmContainer.getContainer();
try {
// create container token and NMToken altogether.
container.setContainerToken(rmContext.getContainerTokenSecretManager()
.createContainerToken(container.getId(), container.getNodeId(),
getUser(), container.getResource(), container.getPriority(),
rmContainer.getCreationTime(), this.logAggregationContext));
NMToken nmToken =
rmContext.getNMTokenSecretManager().createAndGetNMToken(getUser(),
getApplicationAttemptId(), container);
if (nmToken != null) {
nmTokens.add(nmToken);
}
} catch (IllegalArgumentException e) {
// DNS might be down, skip returning this container.
LOG.error("Error trying to assign container token and NM token to" +
" an allocated container " + container.getId(), e);
continue;
}
returnContainerList.add(container);
i.remove();
rmContainer.handle(new RMContainerEvent(rmContainer.getContainerId(),
RMContainerEventType.ACQUIRED));
}
return new ContainersAndNMTokensAllocation(returnContainerList, nmTokens);
}
对符合token要求的container返回,并且触发这些container的RMContainerEventType.ACQUIRED状态,然后在FairScheduler中将符合要求的资源返回给 RMApp。
return new Allocation(allocation.getContainerList(),
application.getHeadroom(), preemptionContainerIds, null, null,
allocation.getNMTokenList());
}
FairScheduler核心
(1)接收容器请求,处理并响应
allocate(ApplicationAttemptId appAttemptId,List ask, List release,List blacklistAdditions, List blacklistRemovals)
这一部分的第一种情况在上述已经纤细分析其过程,接下来主要分析第二种情况,即作业运行期间,AppMaster申请运行map以及reduce任务,或者spark任务等等所需要的容器,我这里以Map Reduce为例。
Map或者Reduce运行需要的容器是在TaskAttempt对象处于New状态,收到TA_SCHEDULE事件,然后调用RequestContainerTransition()进行申请的,与申请AM所需的容器不同,前者是作业提交到RM端时,RM自己进行申请的;后者是AM通过心跳来向RM申请任务需要的容器,下面来看一下第二种情况。
以下是状态机跳变的事件函数RequestContainerTransition,这是一个单弧跳变:
static class RequestContainerTransition implements
SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
private final boolean rescheduled;
public RequestContainerTransition(boolean rescheduled) {
this.rescheduled = rescheduled;
}
@SuppressWarnings("unchecked")
@Override
public void transition(TaskAttemptImpl taskAttempt,
TaskAttemptEvent event) {
// Tell any speculator that we're requesting a container
taskAttempt.eventHandler.handle
(new SpeculatorEvent(taskAttempt.getID().getTaskId(), +1));
//request for container
if (rescheduled) {
taskAttempt.eventHandler.handle(
ContainerRequestEvent.createContainerRequestEventForFailedContainer(
taskAttempt.attemptId,
taskAttempt.resourceCapability));
} else {
taskAttempt.eventHandler.handle(new ContainerRequestEvent(
taskAttempt.attemptId, taskAttempt.resourceCapability,
taskAttempt.dataLocalHosts.toArray(
new String[taskAttempt.dataLocalHosts.size()]),
taskAttempt.dataLocalRacks.toArray(
new String[taskAttempt.dataLocalRacks.size()])));
}
}
}
触发了ContainerAllocator.EventType.CONTAINER_REQ状态
public ContainerRequestEvent(TaskAttemptId attemptID,
Resource capability,
String[] hosts, String[] racks) {
super(attemptID, ContainerAllocator.EventType.CONTAINER_REQ);
this.capability = capability;
this.hosts = hosts;
this.racks = racks;
}
此状态会由RMContainerAllocator.handleEvent()进行处理,接下来继续看处理过程:
protected synchronized void handleEvent(ContainerAllocatorEvent event) {
recalculateReduceSchedule = true;
if (event.getType() == ContainerAllocator.EventType.CONTAINER_REQ) {
ContainerRequestEvent reqEvent = (ContainerRequestEvent) event;
boolean isMap = reqEvent.getAttemptID().getTaskId().getTaskType().
equals(TaskType.MAP);
if (isMap) {
handleMapContainerRequest(reqEvent);
} else {
handleReduceContainerRequest(reqEvent);
}
} else if (
event.getType() == ContainerAllocator.EventType.CONTAINER_DEALLOCATE) {
LOG.info("Processing the event " + event.toString());
TaskAttemptId aId = event.getAttemptID();
boolean removed = scheduledRequests.remove(aId);
if (!removed) {
ContainerId containerId = assignedRequests.get(aId);
if (containerId != null) {
removed = true;
assignedRequests.remove(aId);
containersReleased++;
pendingRelease.add(containerId);
release(containerId);
}
}
if (!removed) {
LOG.error("Could not deallocate container for task attemptId " +
aId);
}
} else if (
event.getType() == ContainerAllocator.EventType.CONTAINER_FAILED) {
ContainerFailedEvent fEv = (ContainerFailedEvent) event;
String host = getHost(fEv.getContMgrAddress());
containerFailedOnHost(host);
}
}
针对不同的TASK类型做不同的处理,我们假设这里的 任务是map任务,会调用如下代码:
private void handleMapContainerRequest(ContainerRequestEvent reqEvent) {
assert(reqEvent.getAttemptID().getTaskId().getTaskType().equals(
TaskType.MAP));
Resource supportedMaxContainerCapability = getMaxContainerCapability();
JobId jobId = getJob().getID();
if (mapResourceRequest.equals(Resources.none())) {
mapResourceRequest = reqEvent.getCapability();
eventHandler.handle(new JobHistoryEvent(jobId,
new NormalizedResourceEvent(
org.apache.hadoop.mapreduce.TaskType.MAP,
mapResourceRequest.getMemory())));
LOG.info("mapResourceRequest:" + mapResourceRequest);
}
boolean mapContainerRequestAccepted = true;
if (mapResourceRequest.getMemory() >
supportedMaxContainerCapability.getMemory()
||
mapResourceRequest.getVirtualCores() >
supportedMaxContainerCapability.getVirtualCores()) {
mapContainerRequestAccepted = false;
}
if(mapContainerRequestAccepted) {
// set the resources
reqEvent.getCapability().setMemory(
mapResourceRequest.getMemory());
reqEvent.getCapability().setVirtualCores(
mapResourceRequest.getVirtualCores());
scheduledRequests.addMap(reqEvent); //maps are immediately scheduled
} else {
String diagMsg = "The required MAP capability is more than the " +
"supported max container capability in the cluster. Killing" +
" the Job. mapResourceRequest: " + mapResourceRequest +
" maxContainerCapability:" + supportedMaxContainerCapability;
LOG.info(diagMsg);
eventHandler.handle(new JobDiagnosticsUpdateEvent(jobId, diagMsg));
eventHandler.handle(new JobEvent(jobId, JobEventType.JOB_KILL));
}
}
然后关键的是调用scheduledRequests.addMap(reqEvent)。
void addMap(ContainerRequestEvent event) {
ContainerRequest request = null;
if (event.getEarlierAttemptFailed()) {
earlierFailedMaps.add(event.getAttemptID());
request = new ContainerRequest(event, PRIORITY_FAST_FAIL_MAP);
LOG.info("Added "+event.getAttemptID()+" to list of failed maps");
} else {
for (String host : event.getHosts()) {
LinkedList<TaskAttemptId> list = mapsHostMapping.get(host);
if (list == null) {
list = new LinkedList<TaskAttemptId>();
mapsHostMapping.put(host, list);
}
list.add(event.getAttemptID());
if (LOG.isDebugEnabled()) {
LOG.debug("Added attempt req to host " + host);
}
}
for (String rack: event.getRacks()) {
LinkedList<TaskAttemptId> list = mapsRackMapping.get(rack);
if (list == null) {
list = new LinkedList<TaskAttemptId>();
mapsRackMapping.put(rack, list);
}
list.add(event.getAttemptID());
if (LOG.isDebugEnabled()) {
LOG.debug("Added attempt req to rack " + rack);
}
}
request = new ContainerRequest(event, PRIORITY_MAP);
}
maps.put(event.getAttemptID(), request);
addContainerReq(request);
}
生成 请求信息,添加到对应的数据结构中。
protected void addContainerReq(ContainerRequest req) {
// Create resource requests
for (String host : req.hosts) {
// Data-local
if (!isNodeBlacklisted(host)) {
addResourceRequest(req.priority, host, req.capability);
}
}
// Nothing Rack-local for now
for (String rack : req.racks) {
addResourceRequest(req.priority, rack, req.capability);
}
// Off-switch
addResourceRequest(req.priority, ResourceRequest.ANY, req.capability);
}
private void addResourceRequest(Priority priority, String resourceName,
Resource capability) {
Map<String, Map<Resource, ResourceRequest>> remoteRequests =
this.remoteRequestsTable.get(priority);
if (remoteRequests == null) {
remoteRequests = new HashMap<String, Map<Resource, ResourceRequest>>();
this.remoteRequestsTable.put(priority, remoteRequests);
if (LOG.isDebugEnabled()) {
LOG.debug("Added priority=" + priority);
}
}
Map<Resource, ResourceRequest> reqMap = remoteRequests.get(resourceName);
if (reqMap == null) {
reqMap = new HashMap<Resource, ResourceRequest>();
remoteRequests.put(resourceName, reqMap);
}
ResourceRequest remoteRequest = reqMap.get(capability);
if (remoteRequest == null) {
remoteRequest = recordFactory.newRecordInstance(ResourceRequest.class);
remoteRequest.setPriority(priority);
remoteRequest.setResourceName(resourceName);
remoteRequest.setCapability(capability);
remoteRequest.setNumContainers(0);
reqMap.put(capability, remoteRequest);
}
remoteRequest.setNumContainers(remoteRequest.getNumContainers() + 1);
// Note this down for next interaction with ResourceManager
addResourceRequestToAsk(remoteRequest);
if (LOG.isDebugEnabled()) {
LOG.debug("addResourceRequest:" + " applicationId="
+ applicationId.getId() + " priority=" + priority.getPriority()
+ " resourceName=" + resourceName + " numContainers="
+ remoteRequest.getNumContainers() + " #asks=" + ask.size());
}
}
private void addResourceRequestToAsk(ResourceRequest remoteRequest) {
// because objects inside the resource map can be deleted ask can end up
// containing an object that matches new resource object but with different
// numContainers. So existing values must be replaced explicitly
ask.remove(remoteRequest);
ask.add(remoteRequest);
}
上述是生成请求的阶段,接下来就是发送请求了,如何从AM端走到RM端:
AppMaster是通过心跳将运行Map或者Reduce的所需的容器发送给RM端FairScheduler的在RMContainerAllocator.hearbeat()中最终会调用RMContainerRequestor的makeRemoteRequest()方法。
protected synchronized void heartbeat() throws Exception {
scheduleStats.updateAndLogIfChanged("Before Scheduling: ");
List<Container> allocatedContainers = getResources();
if (allocatedContainers != null && allocatedContainers.size() > 0) {
scheduledRequests.assign(allocatedContainers);
}
int completedMaps = getJob().getCompletedMaps();
int completedTasks = completedMaps + getJob().getCompletedReduces();
if ((lastCompletedTasks != completedTasks) ||
(scheduledRequests.maps.size() > 0)) {
lastCompletedTasks = completedTasks;
recalculateReduceSchedule = true;
}
if (recalculateReduceSchedule) {
boolean reducerPreempted = preemptReducesIfNeeded();
if (!reducerPreempted) {
// Only schedule new reducers if no reducer preemption happens for
// this heartbeat
scheduleReduces(getJob().getTotalMaps(), completedMaps,
scheduledRequests.maps.size(), scheduledRequests.reduces.size(),
assignedRequests.maps.size(), assignedRequests.reduces.size(),
mapResourceRequest, reduceResourceRequest, pendingReduces.size(),
maxReduceRampupLimit, reduceSlowStart);
}
recalculateReduceSchedule = false;
}
scheduleStats.updateAndLogIfChanged("After Scheduling: ");
}
通过函数 getResources()获取 List allocatedContainers = getResources(),而在getResources函数中会调用 makeRemoteRequest函数:
protected AllocateResponse makeRemoteRequest() throws YarnException,
IOException {
applyRequestLimits();
ResourceBlacklistRequest blacklistRequest =
ResourceBlacklistRequest.newInstance(new ArrayList<String>(blacklistAdditions),
new ArrayList<String>(blacklistRemovals));
AllocateRequest allocateRequest =
AllocateRequest.newInstance(lastResponseID,
super.getApplicationProgress(), new ArrayList<ResourceRequest>(ask),
new ArrayList<ContainerId>(release), blacklistRequest);
AllocateResponse allocateResponse = scheduler.allocate(allocateRequest);
lastResponseID = allocateResponse.getResponseId();
availableResources = allocateResponse.getAvailableResources();
lastClusterNmCount = clusterNmCount;
clusterNmCount = allocateResponse.getNumClusterNodes();
int numCompletedContainers =
allocateResponse.getCompletedContainersStatuses().size();
if (ask.size() > 0 || release.size() > 0) {
LOG.info("getResources() for " + applicationId + ":" + " ask="
+ ask.size() + " release= " + release.size() + " newContainers="
+ allocateResponse.getAllocatedContainers().size()
+ " finishedContainers=" + numCompletedContainers
+ " resourcelimit=" + availableResources + " knownNMs="
+ clusterNmCount);
}
ask.clear();
release.clear();
if (numCompletedContainers > 0) {
// re-send limited requests when a container completes to trigger asking
// for more containers
requestLimitsToUpdate.addAll(requestLimits.keySet());
}
if (blacklistAdditions.size() > 0 || blacklistRemovals.size() > 0) {
LOG.info("Update the blacklist for " + applicationId +
": blacklistAdditions=" + blacklistAdditions.size() +
" blacklistRemovals=" + blacklistRemovals.size());
}
blacklistAdditions.clear();
blacklistRemovals.clear();
return allocateResponse;
}
在AM端通过代理模式,RPC调用RM端FairScheduler的allocate函数:
public AllocateResponse allocate(AllocateRequest request)
throws YarnException, IOException {
AllocateRequestProto requestProto =
((AllocateRequestPBImpl) request).getProto();
try {
return new AllocateResponsePBImpl(proxy.allocate(null, requestProto));
} catch (ServiceException e) {
RPCUtil.unwrapAndThrowException(e);
return null;
}
}
然后allocate请求就传到了 RM端的ApplicationMasterService,即RPC调用对应的服务端的allocate()函数, public AllocateResponse allocate(AllocateRequest request),在这个函数中会调用:
// Send new requests to appAttempt.
Allocation allocation =
this.rScheduler.allocate(appAttemptId, ask, release,
blacklistAdditions, blacklistRemovals);
这里的rScheduler对应我们的调度器就是 FairScheduler的allocate函数,终于到了调度器中的allocate函数,接下来对应的分析就和上述第一种情况一模一样了。
(2)对容器请求调度,分配容器将在下一节对这个过程进行纤细分析
- 持续调度
- Node_UPDATE