上一期指路
上一期讲到taskExecutor向resourceManager发起注册,这次接着上次分析的注册成功后回调TaskExecutor中的onRegistrationSuccess方法。
1.TaskExecutor的内部类ResourceManagerRegistrationListener#onRegistrationSuccess->TaskExecutor#establishResourceManagerConnection
private void establishResourceManagerConnection(
ResourceManagerGateway resourceManagerGateway,
ResourceID resourceManagerResourceId,
InstanceID taskExecutorRegistrationId,
ClusterInformation clusterInformation) {
final CompletableFuture<Acknowledge> slotReportResponseFuture = resourceManagerGateway.sendSlotReport(
getResourceID(),
taskExecutorRegistrationId,
taskSlotTable.createSlotReport(getResourceID()),
taskManagerConfiguration.getTimeout());
slotReportResponseFuture.whenCompleteAsync(
(acknowledge, throwable) -> {
if (throwable != null) {
reconnectToResourceManager(new TaskManagerException("Failed to send initial slot report to ResourceManager.", throwable));
}
}, getMainThreadExecutor());
// monitor the resource manager as heartbeat target
resourceManagerHeartbeatManager.monitorTarget(resourceManagerResourceId, new HeartbeatTarget<TaskExecutorHeartbeatPayload>() {
@Override
public void receiveHeartbeat(ResourceID resourceID, TaskExecutorHeartbeatPayload heartbeatPayload) {
resourceManagerGateway.heartbeatFromTaskManager(resourceID, heartbeatPayload);
}
@Override
public void requestHeartbeat(ResourceID resourceID, TaskExecutorHeartbeatPayload heartbeatPayload) {
// the TaskManager won't send heartbeat requests to the ResourceManager
}
});
// set the propagated blob server address
final InetSocketAddress blobServerAddress = new InetSocketAddress(
clusterInformation.getBlobServerHostname(),
clusterInformation.getBlobServerPort());
blobCacheService.setBlobServerAddress(blobServerAddress);
establishedResourceManagerConnection = new EstablishedResourceManagerConnection(
resourceManagerGateway,
resourceManagerResourceId,
taskExecutorRegistrationId);
stopRegistrationTimeout();
}
2.ResourceManager#sendSlotReport
public CompletableFuture<Acknowledge> sendSlotReport(ResourceID taskManagerResourceId, InstanceID taskManagerRegistrationId, SlotReport slotReport, Time timeout) {
final WorkerRegistration<WorkerType> workerTypeWorkerRegistration = taskExecutors.get(taskManagerResourceId);
if (workerTypeWorkerRegistration.getInstanceID().equals(taskManagerRegistrationId)) {
if (slotManager.registerTaskManager(workerTypeWorkerRegistration, slotReport)) {
onWorkerRegistered(workerTypeWorkerRegistration.getWorker());
}
return CompletableFuture.completedFuture(Acknowledge.get());
} else {
return FutureUtils.completedExceptionally(new ResourceManagerException(String.format("Unknown TaskManager registration id %s.", taskManagerRegistrationId)));
}
}
①taskExecutors.get(taskManagerResourceId)
之前放入缓存中的Worker注册信息取出
②slotManager.registerTaskManager
向 slotManager 注册 slot 信息
3.SlotManagerImpl#registerTaskManager
public boolean registerTaskManager(final TaskExecutorConnection taskExecutorConnection, SlotReport initialSlotReport) {
checkInit();
LOG.debug("Registering TaskManager {} under {} at the SlotManager.", taskExecutorConnection.getResourceID().getStringWithMetadata(), taskExecutorConnection.getInstanceID());
// we identify task managers by their instance id
if (taskManagerRegistrations.containsKey(taskExecutorConnection.getInstanceID())) {
reportSlotStatus(taskExecutorConnection.getInstanceID(), initialSlotReport);
return false;
} else {
if (isMaxSlotNumExceededAfterRegistration(initialSlotReport)) {
LOG.info("The total number of slots exceeds the max limitation {}, release the excess resource.", maxSlotNum);
resourceActions.releaseResource(taskExecutorConnection.getInstanceID(), new FlinkException("The total number of slots exceeds the max limitation."));
return false;
}
// first register the TaskManager
ArrayList<SlotID> reportedSlots = new ArrayList<>();
for (SlotStatus slotStatus : initialSlotReport) {
reportedSlots.add(slotStatus.getSlotID());
}
TaskManagerRegistration taskManagerRegistration = new TaskManagerRegistration(
taskExecutorConnection,
reportedSlots);
taskManagerRegistrations.put(taskExecutorConnection.getInstanceID(), taskManagerRegistration);
// next register the new slots
for (SlotStatus slotStatus : initialSlotReport) {
registerSlot(
slotStatus.getSlotID(),
slotStatus.getAllocationID(),
slotStatus.getJobID(),
slotStatus.getResourceProfile(),
taskExecutorConnection);
}
return true;
}
}
①checkInit
检查slotManager是否启动
②reportSlotStatus
我们通过tm的实例id来标识它们,已经连接过, 直接报告slot的状态
③ArrayList<SlotID> reportedSlots
没有连接过,第一次注册TaskManager
④registerSlot
开始注册slot
4.SlotManagerImpl#registerSlot
private void registerSlot(
SlotID slotId,
AllocationID allocationId,
JobID jobId,
ResourceProfile resourceProfile,
TaskExecutorConnection taskManagerConnection) {
if (slots.containsKey(slotId)) {
// remove the old slot first
removeSlot(
slotId,
new SlotManagerException(
String.format(
"Re-registration of slot %s. This indicates that the TaskExecutor has re-connected.",
slotId)));
}
final TaskManagerSlot slot = createAndRegisterTaskManagerSlot(slotId, resourceProfile, taskManagerConnection);
final PendingTaskManagerSlot pendingTaskManagerSlot;
if (allocationId == null) {
pendingTaskManagerSlot = findExactlyMatchingPendingTaskManagerSlot(resourceProfile);
} else {
pendingTaskManagerSlot = null;
}
if (pendingTaskManagerSlot == null) {
updateSlot(slotId, allocationId, jobId);
} else {
pendingSlots.remove(pendingTaskManagerSlot.getTaskManagerSlotId());
final PendingSlotRequest assignedPendingSlotRequest = pendingTaskManagerSlot.getAssignedPendingSlotRequest();
if (assignedPendingSlotRequest == null) {
handleFreeSlot(slot);
} else {
assignedPendingSlotRequest.unassignPendingTaskManagerSlot();
allocateSlot(slot, assignedPendingSlotRequest);
}
}
}
①removeSlot
移除缓存中已存在的旧slot
②createAndRegisterTaskManagerSlot
构建tm的slot
③findExactlyMatchingPendingTaskManagerSlot
获取去队列中挂起的slot请求
④updateSlot
如果队列中挂起的slot请求为空,更新slot状态
⑤pendingSlots.remove
如果不为空
先将队列之前挂起的slot清空
获取现在挂起的请求
handleFreeSlot(slot)
如果现在挂起的请求为空,说明挂起的请求已经满足了,处理空闲的slot
assignedPendingSlotRequest.unassignPendingTaskManagerSlot()
allocateSlot
如果现在挂起的请求不为空,则表示slot要被分配,清空挂起队列并执行分配
5.SlotManagerImpl#allocateSlot
private void allocateSlot(TaskManagerSlot taskManagerSlot, PendingSlotRequest pendingSlotRequest) {
Preconditions.checkState(taskManagerSlot.getState() == SlotState.FREE);
TaskExecutorConnection taskExecutorConnection = taskManagerSlot.getTaskManagerConnection();
TaskExecutorGateway gateway = taskExecutorConnection.getTaskExecutorGateway();
final CompletableFuture<Acknowledge> completableFuture = new CompletableFuture<>();
final AllocationID allocationId = pendingSlotRequest.getAllocationId();
final SlotID slotId = taskManagerSlot.getSlotId();
final InstanceID instanceID = taskManagerSlot.getInstanceId();
taskManagerSlot.assignPendingSlotRequest(pendingSlotRequest);
pendingSlotRequest.setRequestFuture(completableFuture);
returnPendingTaskManagerSlotIfAssigned(pendingSlotRequest);
TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.get(instanceID);
if (taskManagerRegistration == null) {
throw new IllegalStateException("Could not find a registered task manager for instance id " +
instanceID + '.');
}
taskManagerRegistration.markUsed();
// RPC call to the task manager
CompletableFuture<Acknowledge> requestFuture = gateway.requestSlot(
slotId,
pendingSlotRequest.getJobId(),
allocationId,
pendingSlotRequest.getResourceProfile(),
pendingSlotRequest.getTargetAddress(),
resourceManagerId,
taskManagerRequestTimeout);
requestFuture.whenComplete(
(Acknowledge acknowledge, Throwable throwable) -> {
if (acknowledge != null) {
completableFuture.complete(acknowledge);
} else {
completableFuture.completeExceptionally(throwable);
}
});
completableFuture.whenCompleteAsync(
(Acknowledge acknowledge, Throwable throwable) -> {
try {
if (acknowledge != null) {
updateSlot(slotId, allocationId, pendingSlotRequest.getJobId());
} else {
if (throwable instanceof SlotOccupiedException) {
SlotOccupiedException exception = (SlotOccupiedException) throwable;
updateSlot(slotId, exception.getAllocationId(), exception.getJobId());
} else {
removeSlotRequestFromSlot(slotId, allocationId);
}
if (!(throwable instanceof CancellationException)) {
handleFailedSlotRequest(slotId, allocationId, throwable);
} else {
LOG.debug("Slot allocation request {} has been cancelled.", allocationId, throwable);
}
}
} catch (Exception e) {
LOG.error("Error while completing the slot allocation.", e);
}
},
mainThreadExecutor);
}
6.TaskExecutor#requestSlot
public CompletableFuture<Acknowledge> requestSlot(
final SlotID slotId,
final JobID jobId,
final AllocationID allocationId,
final ResourceProfile resourceProfile,
final String targetAddress,
final ResourceManagerId resourceManagerId,
final Time timeout) {
// TODO: Filter invalid requests from the resource manager by using the instance/registration Id
log.info("Receive slot request {} for job {} from resource manager with leader id {}.",
allocationId, jobId, resourceManagerId);
if (!isConnectedToResourceManager(resourceManagerId)) {
final String message = String.format("TaskManager is not connected to the resource manager %s.", resourceManagerId);
log.debug(message);
return FutureUtils.completedExceptionally(new TaskManagerException(message));
}
try {
allocateSlot(
slotId,
jobId,
allocationId,
resourceProfile);
} catch (SlotAllocationException sae) {
return FutureUtils.completedExceptionally(sae);
}
final JobTable.Job job;
try {
job = jobTable.getOrCreateJob(jobId, () -> registerNewJobAndCreateServices(jobId, targetAddress));
} catch (Exception e) {
// free the allocated slot
try {
taskSlotTable.freeSlot(allocationId);
} catch (SlotNotFoundException slotNotFoundException) {
// slot no longer existent, this should actually never happen, because we've
// just allocated the slot. So let's fail hard in this case!
onFatalError(slotNotFoundException);
}
// release local state under the allocation id.
localStateStoresManager.releaseLocalStateForAllocationId(allocationId);
// sanity check
if (!taskSlotTable.isSlotFree(slotId.getSlotNumber())) {
onFatalError(new Exception("Could not free slot " + slotId));
}
return FutureUtils.completedExceptionally(new SlotAllocationException("Could not create new job.", e));
}
if (job.isConnected()) {
offerSlotsToJobManager(jobId);
}
return CompletableFuture.completedFuture(Acknowledge.get());
}
7.TaskExecutor#offerSlotsToJobManager
->TaskExecutor#internalOfferSlotsToJobManager
private void internalOfferSlotsToJobManager(JobTable.Connection jobManagerConnection) {
final JobID jobId = jobManagerConnection.getJobId();
if (taskSlotTable.hasAllocatedSlots(jobId)) {
log.info("Offer reserved slots to the leader of job {}.", jobId);
final JobMasterGateway jobMasterGateway = jobManagerConnection.getJobManagerGateway();
final Iterator<TaskSlot<Task>> reservedSlotsIterator = taskSlotTable.getAllocatedSlots(jobId);
final JobMasterId jobMasterId = jobManagerConnection.getJobMasterId();
final Collection<SlotOffer> reservedSlots = new HashSet<>(2);
while (reservedSlotsIterator.hasNext()) {
SlotOffer offer = reservedSlotsIterator.next().generateSlotOffer();
reservedSlots.add(offer);
}
CompletableFuture<Collection<SlotOffer>> acceptedSlotsFuture = jobMasterGateway.offerSlots(
getResourceID(),
reservedSlots,
taskManagerConfiguration.getTimeout());
acceptedSlotsFuture.whenCompleteAsync(
handleAcceptedSlotOffers(jobId, jobMasterGateway, jobMasterId, reservedSlots),
getMainThreadExecutor());
} else {
log.debug("There are no unassigned slots for the job {}.", jobId);
}
}
8.JobMaster#offerSlots->SlotPoolImpl#offerSlots->SlotPoolImpl#offerSlot
boolean offerSlot(
final TaskManagerLocation taskManagerLocation,
final TaskManagerGateway taskManagerGateway,
final SlotOffer slotOffer) {
componentMainThreadExecutor.assertRunningInMainThread();
// check if this TaskManager is valid
final ResourceID resourceID = taskManagerLocation.getResourceID();
final AllocationID allocationID = slotOffer.getAllocationId();
if (!registeredTaskManagers.contains(resourceID)) {
log.debug("Received outdated slot offering [{}] from unregistered TaskManager: {}",
slotOffer.getAllocationId(), taskManagerLocation);
return false;
}
// check whether we have already using this slot
AllocatedSlot existingSlot;
if ((existingSlot = allocatedSlots.get(allocationID)) != null ||
(existingSlot = availableSlots.get(allocationID)) != null) {
// we need to figure out if this is a repeated offer for the exact same slot,
// or another offer that comes from a different TaskManager after the ResourceManager
// re-tried the request
// we write this in terms of comparing slot IDs, because the Slot IDs are the identifiers of
// the actual slots on the TaskManagers
// Note: The slotOffer should have the SlotID
final SlotID existingSlotId = existingSlot.getSlotId();
final SlotID newSlotId = new SlotID(taskManagerLocation.getResourceID(), slotOffer.getSlotIndex());
if (existingSlotId.equals(newSlotId)) {
log.info("Received repeated offer for slot [{}]. Ignoring.", allocationID);
// return true here so that the sender will get a positive acknowledgement to the retry
// and mark the offering as a success
return true;
} else {
// the allocation has been fulfilled by another slot, reject the offer so the task executor
// will offer the slot to the resource manager
return false;
}
}
final AllocatedSlot allocatedSlot = new AllocatedSlot(
allocationID,
taskManagerLocation,
slotOffer.getSlotIndex(),
slotOffer.getResourceProfile(),
taskManagerGateway);
// use the slot to fulfill pending request, in requested order
tryFulfillSlotRequestOrMakeAvailable(allocatedSlot);
// we accepted the request in any case. slot will be released after it idled for
// too long and timed out
return true;
}
总览
这一期涉及到的源码流程图