|
@@ -405,7 +405,6 @@ public class ApplicationMasterService extends AbstractService implements
|
|
|
|
|
|
ApplicationAttemptId appAttemptId =
|
|
|
amrmTokenIdentifier.getApplicationAttemptId();
|
|
|
- ApplicationId applicationId = appAttemptId.getApplicationId();
|
|
|
|
|
|
this.amLivelinessMonitor.receivedPing(appAttemptId);
|
|
|
|
|
@@ -422,8 +421,10 @@ public class ApplicationMasterService extends AbstractService implements
|
|
|
AllocateResponse lastResponse = lock.getAllocateResponse();
|
|
|
if (!hasApplicationMasterRegistered(appAttemptId)) {
|
|
|
String message =
|
|
|
- "AM is not registered for known application attempt: " + appAttemptId
|
|
|
- + " or RM had restarted after AM registered . AM should re-register.";
|
|
|
+ "AM is not registered for known application attempt: "
|
|
|
+ + appAttemptId
|
|
|
+ + " or RM had restarted after AM registered. "
|
|
|
+ + " AM should re-register.";
|
|
|
throw new ApplicationMasterNotRegisteredException(message);
|
|
|
}
|
|
|
|
|
@@ -438,185 +439,10 @@ public class ApplicationMasterService extends AbstractService implements
|
|
|
throw new InvalidApplicationMasterRequestException(message);
|
|
|
}
|
|
|
|
|
|
- //filter illegal progress values
|
|
|
- float filteredProgress = request.getProgress();
|
|
|
- if (Float.isNaN(filteredProgress) || filteredProgress == Float.NEGATIVE_INFINITY
|
|
|
- || filteredProgress < 0) {
|
|
|
- request.setProgress(0);
|
|
|
- } else if (filteredProgress > 1 || filteredProgress == Float.POSITIVE_INFINITY) {
|
|
|
- request.setProgress(1);
|
|
|
- }
|
|
|
-
|
|
|
- // Send the status update to the appAttempt.
|
|
|
- this.rmContext.getDispatcher().getEventHandler().handle(
|
|
|
- new RMAppAttemptStatusupdateEvent(appAttemptId, request
|
|
|
- .getProgress()));
|
|
|
-
|
|
|
- List<ResourceRequest> ask = request.getAskList();
|
|
|
- List<ContainerId> release = request.getReleaseList();
|
|
|
-
|
|
|
- ResourceBlacklistRequest blacklistRequest =
|
|
|
- request.getResourceBlacklistRequest();
|
|
|
- List<String> blacklistAdditions =
|
|
|
- (blacklistRequest != null) ?
|
|
|
- blacklistRequest.getBlacklistAdditions() : Collections.EMPTY_LIST;
|
|
|
- List<String> blacklistRemovals =
|
|
|
- (blacklistRequest != null) ?
|
|
|
- blacklistRequest.getBlacklistRemovals() : Collections.EMPTY_LIST;
|
|
|
- RMApp app =
|
|
|
- this.rmContext.getRMApps().get(applicationId);
|
|
|
-
|
|
|
- // set label expression for Resource Requests if resourceName=ANY
|
|
|
- ApplicationSubmissionContext asc = app.getApplicationSubmissionContext();
|
|
|
- for (ResourceRequest req : ask) {
|
|
|
- if (null == req.getNodeLabelExpression()
|
|
|
- && ResourceRequest.ANY.equals(req.getResourceName())) {
|
|
|
- req.setNodeLabelExpression(asc.getNodeLabelExpression());
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- Resource maximumCapacity = rScheduler.getMaximumResourceCapability();
|
|
|
-
|
|
|
- // sanity check
|
|
|
- try {
|
|
|
- RMServerUtils.normalizeAndValidateRequests(ask,
|
|
|
- maximumCapacity, app.getQueue(),
|
|
|
- rScheduler, rmContext);
|
|
|
- } catch (InvalidResourceRequestException e) {
|
|
|
- LOG.warn("Invalid resource ask by application " + appAttemptId, e);
|
|
|
- throw e;
|
|
|
- }
|
|
|
-
|
|
|
- try {
|
|
|
- RMServerUtils.validateBlacklistRequest(blacklistRequest);
|
|
|
- } catch (InvalidResourceBlacklistRequestException e) {
|
|
|
- LOG.warn("Invalid blacklist request by application " + appAttemptId, e);
|
|
|
- throw e;
|
|
|
- }
|
|
|
-
|
|
|
- // In the case of work-preserving AM restart, it's possible for the
|
|
|
- // AM to release containers from the earlier attempt.
|
|
|
- if (!app.getApplicationSubmissionContext()
|
|
|
- .getKeepContainersAcrossApplicationAttempts()) {
|
|
|
- try {
|
|
|
- RMServerUtils.validateContainerReleaseRequest(release, appAttemptId);
|
|
|
- } catch (InvalidContainerReleaseException e) {
|
|
|
- LOG.warn("Invalid container release by application " + appAttemptId,
|
|
|
- e);
|
|
|
- throw e;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- // Split Update Resource Requests into increase and decrease.
|
|
|
- // No Exceptions are thrown here. All update errors are aggregated
|
|
|
- // and returned to the AM.
|
|
|
- List<UpdateContainerRequest> increaseResourceReqs = new ArrayList<>();
|
|
|
- List<UpdateContainerRequest> decreaseResourceReqs = new ArrayList<>();
|
|
|
- List<UpdateContainerError> updateContainerErrors =
|
|
|
- RMServerUtils.validateAndSplitUpdateResourceRequests(rmContext,
|
|
|
- request, maximumCapacity, increaseResourceReqs,
|
|
|
- decreaseResourceReqs);
|
|
|
-
|
|
|
- // Send new requests to appAttempt.
|
|
|
- Allocation allocation;
|
|
|
- RMAppAttemptState state =
|
|
|
- app.getRMAppAttempt(appAttemptId).getAppAttemptState();
|
|
|
- if (state.equals(RMAppAttemptState.FINAL_SAVING) ||
|
|
|
- state.equals(RMAppAttemptState.FINISHING) ||
|
|
|
- app.isAppFinalStateStored()) {
|
|
|
- LOG.warn(appAttemptId + " is in " + state +
|
|
|
- " state, ignore container allocate request.");
|
|
|
- allocation = EMPTY_ALLOCATION;
|
|
|
- } else {
|
|
|
- allocation =
|
|
|
- this.rScheduler.allocate(appAttemptId, ask, release,
|
|
|
- blacklistAdditions, blacklistRemovals,
|
|
|
- increaseResourceReqs, decreaseResourceReqs);
|
|
|
- }
|
|
|
-
|
|
|
- if (!blacklistAdditions.isEmpty() || !blacklistRemovals.isEmpty()) {
|
|
|
- LOG.info("blacklist are updated in Scheduler." +
|
|
|
- "blacklistAdditions: " + blacklistAdditions + ", " +
|
|
|
- "blacklistRemovals: " + blacklistRemovals);
|
|
|
- }
|
|
|
- RMAppAttempt appAttempt = app.getRMAppAttempt(appAttemptId);
|
|
|
- AllocateResponse allocateResponse =
|
|
|
+ AllocateResponse response =
|
|
|
recordFactory.newRecordInstance(AllocateResponse.class);
|
|
|
- if (allocation.getNMTokens() != null &&
|
|
|
- !allocation.getNMTokens().isEmpty()) {
|
|
|
- allocateResponse.setNMTokens(allocation.getNMTokens());
|
|
|
- }
|
|
|
-
|
|
|
- // Notify the AM of container update errors
|
|
|
- if (!updateContainerErrors.isEmpty()) {
|
|
|
- allocateResponse.setUpdateErrors(updateContainerErrors);
|
|
|
- }
|
|
|
- // update the response with the deltas of node status changes
|
|
|
- List<RMNode> updatedNodes = new ArrayList<RMNode>();
|
|
|
- if(app.pullRMNodeUpdates(updatedNodes) > 0) {
|
|
|
- List<NodeReport> updatedNodeReports = new ArrayList<NodeReport>();
|
|
|
- for(RMNode rmNode: updatedNodes) {
|
|
|
- SchedulerNodeReport schedulerNodeReport =
|
|
|
- rScheduler.getNodeReport(rmNode.getNodeID());
|
|
|
- Resource used = BuilderUtils.newResource(0, 0);
|
|
|
- int numContainers = 0;
|
|
|
- if (schedulerNodeReport != null) {
|
|
|
- used = schedulerNodeReport.getUsedResource();
|
|
|
- numContainers = schedulerNodeReport.getNumContainers();
|
|
|
- }
|
|
|
- NodeId nodeId = rmNode.getNodeID();
|
|
|
- NodeReport report =
|
|
|
- BuilderUtils.newNodeReport(nodeId, rmNode.getState(),
|
|
|
- rmNode.getHttpAddress(), rmNode.getRackName(), used,
|
|
|
- rmNode.getTotalCapability(), numContainers,
|
|
|
- rmNode.getHealthReport(), rmNode.getLastHealthReportTime(),
|
|
|
- rmNode.getNodeLabels());
|
|
|
-
|
|
|
- updatedNodeReports.add(report);
|
|
|
- }
|
|
|
- allocateResponse.setUpdatedNodes(updatedNodeReports);
|
|
|
- }
|
|
|
-
|
|
|
- allocateResponse.setAllocatedContainers(allocation.getContainers());
|
|
|
- allocateResponse.setCompletedContainersStatuses(appAttempt
|
|
|
- .pullJustFinishedContainers());
|
|
|
- allocateResponse.setResponseId(lastResponse.getResponseId() + 1);
|
|
|
- allocateResponse.setAvailableResources(allocation.getResourceLimit());
|
|
|
-
|
|
|
- // Handling increased/decreased containers
|
|
|
- List<UpdatedContainer> updatedContainers = new ArrayList<>();
|
|
|
- if (allocation.getIncreasedContainers() != null) {
|
|
|
- for (Container c : allocation.getIncreasedContainers()) {
|
|
|
- updatedContainers.add(
|
|
|
- UpdatedContainer.newInstance(
|
|
|
- ContainerUpdateType.INCREASE_RESOURCE, c));
|
|
|
- }
|
|
|
- }
|
|
|
- if (allocation.getDecreasedContainers() != null) {
|
|
|
- for (Container c : allocation.getDecreasedContainers()) {
|
|
|
- updatedContainers.add(
|
|
|
- UpdatedContainer.newInstance(
|
|
|
- ContainerUpdateType.DECREASE_RESOURCE, c));
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- allocateResponse.setUpdatedContainers(updatedContainers);
|
|
|
-
|
|
|
- allocateResponse.setNumClusterNodes(this.rScheduler.getNumClusterNodes());
|
|
|
-
|
|
|
- // add collector address for this application
|
|
|
- if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) {
|
|
|
- allocateResponse.setCollectorAddr(
|
|
|
- this.rmContext.getRMApps().get(applicationId).getCollectorAddr());
|
|
|
- }
|
|
|
-
|
|
|
- // add preemption to the allocateResponse message (if any)
|
|
|
- allocateResponse
|
|
|
- .setPreemptionMessage(generatePreemptionMessage(allocation));
|
|
|
-
|
|
|
- // Set application priority
|
|
|
- allocateResponse.setApplicationPriority(app
|
|
|
- .getApplicationPriority());
|
|
|
+ allocateInternal(amrmTokenIdentifier.getApplicationAttemptId(),
|
|
|
+ request, response);
|
|
|
|
|
|
// update AMRMToken if the token is rolled-up
|
|
|
MasterKeyData nextMasterKey =
|
|
@@ -624,21 +450,24 @@ public class ApplicationMasterService extends AbstractService implements
|
|
|
|
|
|
if (nextMasterKey != null
|
|
|
&& nextMasterKey.getMasterKey().getKeyId() != amrmTokenIdentifier
|
|
|
- .getKeyId()) {
|
|
|
+ .getKeyId()) {
|
|
|
+ RMApp app =
|
|
|
+ this.rmContext.getRMApps().get(appAttemptId.getApplicationId());
|
|
|
+ RMAppAttempt appAttempt = app.getRMAppAttempt(appAttemptId);
|
|
|
RMAppAttemptImpl appAttemptImpl = (RMAppAttemptImpl)appAttempt;
|
|
|
Token<AMRMTokenIdentifier> amrmToken = appAttempt.getAMRMToken();
|
|
|
if (nextMasterKey.getMasterKey().getKeyId() !=
|
|
|
appAttemptImpl.getAMRMTokenKeyId()) {
|
|
|
LOG.info("The AMRMToken has been rolled-over. Send new AMRMToken back"
|
|
|
- + " to application: " + applicationId);
|
|
|
+ + " to application: " + appAttemptId.getApplicationId());
|
|
|
amrmToken = rmContext.getAMRMTokenSecretManager()
|
|
|
.createAndGetAMRMToken(appAttemptId);
|
|
|
appAttemptImpl.setAMRMToken(amrmToken);
|
|
|
}
|
|
|
- allocateResponse.setAMRMToken(org.apache.hadoop.yarn.api.records.Token
|
|
|
- .newInstance(amrmToken.getIdentifier(), amrmToken.getKind()
|
|
|
- .toString(), amrmToken.getPassword(), amrmToken.getService()
|
|
|
- .toString()));
|
|
|
+ response.setAMRMToken(org.apache.hadoop.yarn.api.records.Token
|
|
|
+ .newInstance(amrmToken.getIdentifier(), amrmToken.getKind()
|
|
|
+ .toString(), amrmToken.getPassword(), amrmToken.getService()
|
|
|
+ .toString()));
|
|
|
}
|
|
|
|
|
|
/*
|
|
@@ -646,11 +475,227 @@ public class ApplicationMasterService extends AbstractService implements
|
|
|
* need to worry about unregister call occurring in between (which
|
|
|
* removes the lock object).
|
|
|
*/
|
|
|
- lock.setAllocateResponse(allocateResponse);
|
|
|
- return allocateResponse;
|
|
|
+ response.setResponseId(lastResponse.getResponseId() + 1);
|
|
|
+ lock.setAllocateResponse(response);
|
|
|
+ return response;
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ protected void allocateInternal(ApplicationAttemptId appAttemptId,
|
|
|
+ AllocateRequest request, AllocateResponse allocateResponse)
|
|
|
+ throws YarnException {
|
|
|
+
|
|
|
+ //filter illegal progress values
|
|
|
+ float filteredProgress = request.getProgress();
|
|
|
+ if (Float.isNaN(filteredProgress) ||
|
|
|
+ filteredProgress == Float.NEGATIVE_INFINITY ||
|
|
|
+ filteredProgress < 0) {
|
|
|
+ request.setProgress(0);
|
|
|
+ } else if (filteredProgress > 1 ||
|
|
|
+ filteredProgress == Float.POSITIVE_INFINITY) {
|
|
|
+ request.setProgress(1);
|
|
|
+ }
|
|
|
+
|
|
|
+ // Send the status update to the appAttempt.
|
|
|
+ this.rmContext.getDispatcher().getEventHandler().handle(
|
|
|
+ new RMAppAttemptStatusupdateEvent(appAttemptId, request
|
|
|
+ .getProgress()));
|
|
|
+
|
|
|
+ List<ResourceRequest> ask = request.getAskList();
|
|
|
+ List<ContainerId> release = request.getReleaseList();
|
|
|
+
|
|
|
+ ResourceBlacklistRequest blacklistRequest =
|
|
|
+ request.getResourceBlacklistRequest();
|
|
|
+ List<String> blacklistAdditions =
|
|
|
+ (blacklistRequest != null) ?
|
|
|
+ blacklistRequest.getBlacklistAdditions() : Collections.EMPTY_LIST;
|
|
|
+ List<String> blacklistRemovals =
|
|
|
+ (blacklistRequest != null) ?
|
|
|
+ blacklistRequest.getBlacklistRemovals() : Collections.EMPTY_LIST;
|
|
|
+ RMApp app =
|
|
|
+ this.rmContext.getRMApps().get(appAttemptId.getApplicationId());
|
|
|
+
|
|
|
+ // set label expression for Resource Requests if resourceName=ANY
|
|
|
+ ApplicationSubmissionContext asc = app.getApplicationSubmissionContext();
|
|
|
+ for (ResourceRequest req : ask) {
|
|
|
+ if (null == req.getNodeLabelExpression()
|
|
|
+ && ResourceRequest.ANY.equals(req.getResourceName())) {
|
|
|
+ req.setNodeLabelExpression(asc.getNodeLabelExpression());
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ Resource maximumCapacity = rScheduler.getMaximumResourceCapability();
|
|
|
+
|
|
|
+ // sanity check
|
|
|
+ try {
|
|
|
+ RMServerUtils.normalizeAndValidateRequests(ask,
|
|
|
+ maximumCapacity, app.getQueue(),
|
|
|
+ rScheduler, rmContext);
|
|
|
+ } catch (InvalidResourceRequestException e) {
|
|
|
+ LOG.warn("Invalid resource ask by application " + appAttemptId, e);
|
|
|
+ throw e;
|
|
|
+ }
|
|
|
+
|
|
|
+ try {
|
|
|
+ RMServerUtils.validateBlacklistRequest(blacklistRequest);
|
|
|
+ } catch (InvalidResourceBlacklistRequestException e) {
|
|
|
+ LOG.warn("Invalid blacklist request by application " + appAttemptId, e);
|
|
|
+ throw e;
|
|
|
+ }
|
|
|
+
|
|
|
+ // In the case of work-preserving AM restart, it's possible for the
|
|
|
+ // AM to release containers from the earlier attempt.
|
|
|
+ if (!app.getApplicationSubmissionContext()
|
|
|
+ .getKeepContainersAcrossApplicationAttempts()) {
|
|
|
+ try {
|
|
|
+ RMServerUtils.validateContainerReleaseRequest(release, appAttemptId);
|
|
|
+ } catch (InvalidContainerReleaseException e) {
|
|
|
+ LOG.warn("Invalid container release by application " + appAttemptId,
|
|
|
+ e);
|
|
|
+ throw e;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // Split Update Resource Requests into increase and decrease.
|
|
|
+ // No Exceptions are thrown here. All update errors are aggregated
|
|
|
+ // and returned to the AM.
|
|
|
+ List<UpdateContainerRequest> increaseResourceReqs = new ArrayList<>();
|
|
|
+ List<UpdateContainerRequest> decreaseResourceReqs = new ArrayList<>();
|
|
|
+ List<UpdateContainerError> updateContainerErrors =
|
|
|
+ RMServerUtils.validateAndSplitUpdateResourceRequests(
|
|
|
+ rmContext, request, maximumCapacity,
|
|
|
+ increaseResourceReqs, decreaseResourceReqs);
|
|
|
+
|
|
|
+ // Send new requests to appAttempt.
|
|
|
+ Allocation allocation;
|
|
|
+ RMAppAttemptState state =
|
|
|
+ app.getRMAppAttempt(appAttemptId).getAppAttemptState();
|
|
|
+ if (state.equals(RMAppAttemptState.FINAL_SAVING) ||
|
|
|
+ state.equals(RMAppAttemptState.FINISHING) ||
|
|
|
+ app.isAppFinalStateStored()) {
|
|
|
+ LOG.warn(appAttemptId + " is in " + state +
|
|
|
+ " state, ignore container allocate request.");
|
|
|
+ allocation = EMPTY_ALLOCATION;
|
|
|
+ } else {
|
|
|
+ allocation =
|
|
|
+ this.rScheduler.allocate(appAttemptId, ask, release,
|
|
|
+ blacklistAdditions, blacklistRemovals,
|
|
|
+ increaseResourceReqs, decreaseResourceReqs);
|
|
|
+ }
|
|
|
+
|
|
|
+ if (!blacklistAdditions.isEmpty() || !blacklistRemovals.isEmpty()) {
|
|
|
+ LOG.info("blacklist are updated in Scheduler." +
|
|
|
+ "blacklistAdditions: " + blacklistAdditions + ", " +
|
|
|
+ "blacklistRemovals: " + blacklistRemovals);
|
|
|
+ }
|
|
|
+ RMAppAttempt appAttempt = app.getRMAppAttempt(appAttemptId);
|
|
|
+
|
|
|
+ if (allocation.getNMTokens() != null &&
|
|
|
+ !allocation.getNMTokens().isEmpty()) {
|
|
|
+ allocateResponse.setNMTokens(allocation.getNMTokens());
|
|
|
+ }
|
|
|
+
|
|
|
+ // Notify the AM of container update errors
|
|
|
+ addToUpdateContainerErrors(allocateResponse, updateContainerErrors);
|
|
|
+
|
|
|
+ // update the response with the deltas of node status changes
|
|
|
+ List<RMNode> updatedNodes = new ArrayList<RMNode>();
|
|
|
+ if(app.pullRMNodeUpdates(updatedNodes) > 0) {
|
|
|
+ List<NodeReport> updatedNodeReports = new ArrayList<NodeReport>();
|
|
|
+ for(RMNode rmNode: updatedNodes) {
|
|
|
+ SchedulerNodeReport schedulerNodeReport =
|
|
|
+ rScheduler.getNodeReport(rmNode.getNodeID());
|
|
|
+ Resource used = BuilderUtils.newResource(0, 0);
|
|
|
+ int numContainers = 0;
|
|
|
+ if (schedulerNodeReport != null) {
|
|
|
+ used = schedulerNodeReport.getUsedResource();
|
|
|
+ numContainers = schedulerNodeReport.getNumContainers();
|
|
|
+ }
|
|
|
+ NodeId nodeId = rmNode.getNodeID();
|
|
|
+ NodeReport report =
|
|
|
+ BuilderUtils.newNodeReport(nodeId, rmNode.getState(),
|
|
|
+ rmNode.getHttpAddress(), rmNode.getRackName(), used,
|
|
|
+ rmNode.getTotalCapability(), numContainers,
|
|
|
+ rmNode.getHealthReport(), rmNode.getLastHealthReportTime(),
|
|
|
+ rmNode.getNodeLabels());
|
|
|
+
|
|
|
+ updatedNodeReports.add(report);
|
|
|
+ }
|
|
|
+ allocateResponse.setUpdatedNodes(updatedNodeReports);
|
|
|
+ }
|
|
|
+
|
|
|
+ addToAllocatedContainers(allocateResponse, allocation.getContainers());
|
|
|
+
|
|
|
+ allocateResponse.setCompletedContainersStatuses(appAttempt
|
|
|
+ .pullJustFinishedContainers());
|
|
|
+ allocateResponse.setAvailableResources(allocation.getResourceLimit());
|
|
|
+
|
|
|
+ // Handling increased containers
|
|
|
+ addToUpdatedContainers(
|
|
|
+ allocateResponse, ContainerUpdateType.INCREASE_RESOURCE,
|
|
|
+ allocation.getIncreasedContainers());
|
|
|
+
|
|
|
+ // Handling decreased containers
|
|
|
+ addToUpdatedContainers(
|
|
|
+ allocateResponse, ContainerUpdateType.DECREASE_RESOURCE,
|
|
|
+ allocation.getDecreasedContainers());
|
|
|
+
|
|
|
+ allocateResponse.setNumClusterNodes(this.rScheduler.getNumClusterNodes());
|
|
|
+
|
|
|
+ // add collector address for this application
|
|
|
+ if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) {
|
|
|
+ allocateResponse.setCollectorAddr(
|
|
|
+ this.rmContext.getRMApps().get(appAttemptId.getApplicationId())
|
|
|
+ .getCollectorAddr());
|
|
|
+ }
|
|
|
+
|
|
|
+ // add preemption to the allocateResponse message (if any)
|
|
|
+ allocateResponse
|
|
|
+ .setPreemptionMessage(generatePreemptionMessage(allocation));
|
|
|
+
|
|
|
+ // Set application priority
|
|
|
+ allocateResponse.setApplicationPriority(app
|
|
|
+ .getApplicationPriority());
|
|
|
+ }
|
|
|
+
|
|
|
+ protected void addToUpdateContainerErrors(AllocateResponse allocateResponse,
|
|
|
+ List<UpdateContainerError> updateContainerErrors) {
|
|
|
+ if (!updateContainerErrors.isEmpty()) {
|
|
|
+ if (allocateResponse.getUpdateErrors() != null
|
|
|
+ && !allocateResponse.getUpdateErrors().isEmpty()) {
|
|
|
+ updateContainerErrors = new ArrayList<>(updateContainerErrors);
|
|
|
+ updateContainerErrors.addAll(allocateResponse.getUpdateErrors());
|
|
|
+ }
|
|
|
+ allocateResponse.setUpdateErrors(updateContainerErrors);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ protected void addToUpdatedContainers(AllocateResponse allocateResponse,
|
|
|
+ ContainerUpdateType updateType, List<Container> updatedContainers) {
|
|
|
+ if (updatedContainers != null && updatedContainers.size() > 0) {
|
|
|
+ ArrayList<UpdatedContainer> containersToSet = new ArrayList<>();
|
|
|
+ if (allocateResponse.getUpdatedContainers() != null &&
|
|
|
+ !allocateResponse.getUpdatedContainers().isEmpty()) {
|
|
|
+ containersToSet.addAll(allocateResponse.getUpdatedContainers());
|
|
|
+ }
|
|
|
+ for (Container updatedContainer : updatedContainers) {
|
|
|
+ containersToSet.add(
|
|
|
+ UpdatedContainer.newInstance(updateType, updatedContainer));
|
|
|
+ }
|
|
|
+ allocateResponse.setUpdatedContainers(containersToSet);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ protected void addToAllocatedContainers(AllocateResponse allocateResponse,
|
|
|
+ List<Container> allocatedContainers) {
|
|
|
+ if (allocateResponse.getAllocatedContainers() != null
|
|
|
+ && !allocateResponse.getAllocatedContainers().isEmpty()) {
|
|
|
+ allocatedContainers = new ArrayList<>(allocatedContainers);
|
|
|
+ allocatedContainers.addAll(allocateResponse.getAllocatedContainers());
|
|
|
+ }
|
|
|
+ allocateResponse.setAllocatedContainers(allocatedContainers);
|
|
|
+ }
|
|
|
+
|
|
|
private PreemptionMessage generatePreemptionMessage(Allocation allocation){
|
|
|
PreemptionMessage pMsg = null;
|
|
|
// assemble strict preemption request
|