|
@@ -30,9 +30,11 @@ import java.util.LinkedHashSet;
|
|
import java.util.LinkedList;
|
|
import java.util.LinkedList;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
import java.util.Map;
|
|
import java.util.Map;
|
|
|
|
+import java.util.Queue;
|
|
import java.util.Set;
|
|
import java.util.Set;
|
|
import java.util.TreeSet;
|
|
import java.util.TreeSet;
|
|
import java.util.AbstractMap.SimpleEntry;
|
|
import java.util.AbstractMap.SimpleEntry;
|
|
|
|
+import java.util.concurrent.ConcurrentHashMap;
|
|
|
|
|
|
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
|
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
|
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
|
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
|
@@ -60,9 +62,11 @@ import org.apache.hadoop.yarn.api.records.ProfileCapability;
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
|
|
import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
|
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
|
|
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
|
|
import org.apache.hadoop.yarn.api.records.Token;
|
|
import org.apache.hadoop.yarn.api.records.Token;
|
|
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
|
|
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
|
|
import org.apache.hadoop.yarn.api.records.UpdatedContainer;
|
|
import org.apache.hadoop.yarn.api.records.UpdatedContainer;
|
|
|
|
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
|
|
import org.apache.hadoop.yarn.client.ClientRMProxy;
|
|
import org.apache.hadoop.yarn.client.ClientRMProxy;
|
|
import org.apache.hadoop.yarn.client.api.AMRMClient;
|
|
import org.apache.hadoop.yarn.client.api.AMRMClient;
|
|
import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
|
|
import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
|
|
@@ -106,6 +110,12 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
|
|
protected final Set<String> blacklistedNodes = new HashSet<String>();
|
|
protected final Set<String> blacklistedNodes = new HashSet<String>();
|
|
protected final Set<String> blacklistAdditions = new HashSet<String>();
|
|
protected final Set<String> blacklistAdditions = new HashSet<String>();
|
|
protected final Set<String> blacklistRemovals = new HashSet<String>();
|
|
protected final Set<String> blacklistRemovals = new HashSet<String>();
|
|
|
|
+ private Map<Set<String>, PlacementConstraint> placementConstraints =
|
|
|
|
+ new HashMap<>();
|
|
|
|
+ private Queue<Collection<SchedulingRequest>> batchedSchedulingRequests =
|
|
|
|
+ new LinkedList<>();
|
|
|
|
+ private Map<Set<String>, List<SchedulingRequest>> outstandingSchedRequests =
|
|
|
|
+ new ConcurrentHashMap<>();
|
|
|
|
|
|
protected Map<String, Resource> resourceProfilesMap;
|
|
protected Map<String, Resource> resourceProfilesMap;
|
|
|
|
|
|
@@ -218,14 +228,26 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
|
|
}
|
|
}
|
|
super.serviceStop();
|
|
super.serviceStop();
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public RegisterApplicationMasterResponse registerApplicationMaster(
|
|
public RegisterApplicationMasterResponse registerApplicationMaster(
|
|
String appHostName, int appHostPort, String appTrackingUrl)
|
|
String appHostName, int appHostPort, String appTrackingUrl)
|
|
throws YarnException, IOException {
|
|
throws YarnException, IOException {
|
|
|
|
+ return registerApplicationMaster(appHostName, appHostPort, appTrackingUrl,
|
|
|
|
+ null);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public RegisterApplicationMasterResponse registerApplicationMaster(
|
|
|
|
+ String appHostName, int appHostPort, String appTrackingUrl,
|
|
|
|
+ Map<Set<String>, PlacementConstraint> placementConstraintsMap)
|
|
|
|
+ throws YarnException, IOException {
|
|
this.appHostName = appHostName;
|
|
this.appHostName = appHostName;
|
|
this.appHostPort = appHostPort;
|
|
this.appHostPort = appHostPort;
|
|
this.appTrackingUrl = appTrackingUrl;
|
|
this.appTrackingUrl = appTrackingUrl;
|
|
|
|
+ if (placementConstraintsMap != null && !placementConstraintsMap.isEmpty()) {
|
|
|
|
+ this.placementConstraints.putAll(placementConstraintsMap);
|
|
|
|
+ }
|
|
Preconditions.checkArgument(appHostName != null,
|
|
Preconditions.checkArgument(appHostName != null,
|
|
"The host name should not be null");
|
|
"The host name should not be null");
|
|
Preconditions.checkArgument(appHostPort >= -1, "Port number of the host"
|
|
Preconditions.checkArgument(appHostPort >= -1, "Port number of the host"
|
|
@@ -240,6 +262,9 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
|
|
RegisterApplicationMasterRequest request =
|
|
RegisterApplicationMasterRequest request =
|
|
RegisterApplicationMasterRequest.newInstance(this.appHostName,
|
|
RegisterApplicationMasterRequest.newInstance(this.appHostName,
|
|
this.appHostPort, this.appTrackingUrl);
|
|
this.appHostPort, this.appTrackingUrl);
|
|
|
|
+ if (!this.placementConstraints.isEmpty()) {
|
|
|
|
+ request.setPlacementConstraints(this.placementConstraints);
|
|
|
|
+ }
|
|
RegisterApplicationMasterResponse response =
|
|
RegisterApplicationMasterResponse response =
|
|
rmClient.registerApplicationMaster(request);
|
|
rmClient.registerApplicationMaster(request);
|
|
synchronized (this) {
|
|
synchronized (this) {
|
|
@@ -248,10 +273,22 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
|
|
populateNMTokens(response.getNMTokensFromPreviousAttempts());
|
|
populateNMTokens(response.getNMTokensFromPreviousAttempts());
|
|
}
|
|
}
|
|
this.resourceProfilesMap = response.getResourceProfiles();
|
|
this.resourceProfilesMap = response.getResourceProfiles();
|
|
|
|
+ List<Container> prevContainers =
|
|
|
|
+ response.getContainersFromPreviousAttempts();
|
|
|
|
+ removeFromOutstandingSchedulingRequests(prevContainers);
|
|
|
|
+ recreateSchedulingRequestBatch();
|
|
}
|
|
}
|
|
return response;
|
|
return response;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @Override
|
|
|
|
+ public void addSchedulingRequests(
|
|
|
|
+ Collection<SchedulingRequest> schedulingRequests) {
|
|
|
|
+ synchronized (this.batchedSchedulingRequests) {
|
|
|
|
+ this.batchedSchedulingRequests.add(schedulingRequests);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public AllocateResponse allocate(float progressIndicator)
|
|
public AllocateResponse allocate(float progressIndicator)
|
|
throws YarnException, IOException {
|
|
throws YarnException, IOException {
|
|
@@ -288,6 +325,7 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
|
|
.responseId(lastResponseId).progress(progressIndicator)
|
|
.responseId(lastResponseId).progress(progressIndicator)
|
|
.askList(askList).resourceBlacklistRequest(blacklistRequest)
|
|
.askList(askList).resourceBlacklistRequest(blacklistRequest)
|
|
.releaseList(releaseList).updateRequests(updateList).build();
|
|
.releaseList(releaseList).updateRequests(updateList).build();
|
|
|
|
+ populateSchedulingRequests(allocateRequest);
|
|
// clear blacklistAdditions and blacklistRemovals before
|
|
// clear blacklistAdditions and blacklistRemovals before
|
|
// unsynchronized part
|
|
// unsynchronized part
|
|
blacklistAdditions.clear();
|
|
blacklistAdditions.clear();
|
|
@@ -296,6 +334,10 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
|
|
|
|
|
|
try {
|
|
try {
|
|
allocateResponse = rmClient.allocate(allocateRequest);
|
|
allocateResponse = rmClient.allocate(allocateRequest);
|
|
|
|
+ removeFromOutstandingSchedulingRequests(
|
|
|
|
+ allocateResponse.getAllocatedContainers());
|
|
|
|
+ removeFromOutstandingSchedulingRequests(
|
|
|
|
+ allocateResponse.getContainersFromPreviousAttempts());
|
|
} catch (ApplicationMasterNotRegisteredException e) {
|
|
} catch (ApplicationMasterNotRegisteredException e) {
|
|
LOG.warn("ApplicationMaster is out of sync with ResourceManager,"
|
|
LOG.warn("ApplicationMaster is out of sync with ResourceManager,"
|
|
+ " hence resyncing.");
|
|
+ " hence resyncing.");
|
|
@@ -397,6 +439,104 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
|
|
return allocateResponse;
|
|
return allocateResponse;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private void populateSchedulingRequests(AllocateRequest allocateRequest) {
|
|
|
|
+ synchronized (this.batchedSchedulingRequests) {
|
|
|
|
+ if (!this.batchedSchedulingRequests.isEmpty()) {
|
|
|
|
+ List<SchedulingRequest> newReqs = new LinkedList<>();
|
|
|
|
+ Iterator<Collection<SchedulingRequest>> iter =
|
|
|
|
+ this.batchedSchedulingRequests.iterator();
|
|
|
|
+ while (iter.hasNext()) {
|
|
|
|
+ Collection<SchedulingRequest> requests = iter.next();
|
|
|
|
+ newReqs.addAll(requests);
|
|
|
|
+ addToOutstandingSchedulingRequests(requests);
|
|
|
|
+ iter.remove();
|
|
|
|
+ }
|
|
|
|
+ allocateRequest.setSchedulingRequests(newReqs);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void recreateSchedulingRequestBatch() {
|
|
|
|
+ List<SchedulingRequest> batched = new ArrayList<>();
|
|
|
|
+ synchronized (this.outstandingSchedRequests) {
|
|
|
|
+ for (List<SchedulingRequest> schedReqs :
|
|
|
|
+ this.outstandingSchedRequests.values()) {
|
|
|
|
+ batched.addAll(schedReqs);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ synchronized (this.batchedSchedulingRequests) {
|
|
|
|
+ this.batchedSchedulingRequests.add(batched);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void addToOutstandingSchedulingRequests(
|
|
|
|
+ Collection<SchedulingRequest> requests) {
|
|
|
|
+ for (SchedulingRequest req : requests) {
|
|
|
|
+ List<SchedulingRequest> schedulingRequests =
|
|
|
|
+ this.outstandingSchedRequests.computeIfAbsent(
|
|
|
|
+ req.getAllocationTags(), x -> new LinkedList<>());
|
|
|
|
+ SchedulingRequest matchingReq = null;
|
|
|
|
+ synchronized (schedulingRequests) {
|
|
|
|
+ for (SchedulingRequest schedReq : schedulingRequests) {
|
|
|
|
+ if (isMatching(req, schedReq)) {
|
|
|
|
+ matchingReq = schedReq;
|
|
|
|
+ break;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ if (matchingReq != null) {
|
|
|
|
+ matchingReq.getResourceSizing().setNumAllocations(
|
|
|
|
+ req.getResourceSizing().getNumAllocations());
|
|
|
|
+ } else {
|
|
|
|
+ schedulingRequests.add(req);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private boolean isMatching(SchedulingRequest schedReq1,
|
|
|
|
+ SchedulingRequest schedReq2) {
|
|
|
|
+ return schedReq1.getPriority().equals(schedReq2.getPriority()) &&
|
|
|
|
+ schedReq1.getExecutionType().getExecutionType().equals(
|
|
|
|
+ schedReq1.getExecutionType().getExecutionType()) &&
|
|
|
|
+ schedReq1.getAllocationRequestId() ==
|
|
|
|
+ schedReq2.getAllocationRequestId();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void removeFromOutstandingSchedulingRequests(
|
|
|
|
+ Collection<Container> containers) {
|
|
|
|
+ if (containers == null || containers.isEmpty()) {
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+ for (Container container : containers) {
|
|
|
|
+ if (container.getAllocationTags() != null &&
|
|
|
|
+ !container.getAllocationTags().isEmpty()) {
|
|
|
|
+ List<SchedulingRequest> schedReqs =
|
|
|
|
+ this.outstandingSchedRequests.get(container.getAllocationTags());
|
|
|
|
+ if (schedReqs != null && !schedReqs.isEmpty()) {
|
|
|
|
+ synchronized (schedReqs) {
|
|
|
|
+ Iterator<SchedulingRequest> iter = schedReqs.iterator();
|
|
|
|
+ while (iter.hasNext()) {
|
|
|
|
+ SchedulingRequest schedReq = iter.next();
|
|
|
|
+ if (schedReq.getPriority().equals(container.getPriority()) &&
|
|
|
|
+ schedReq.getAllocationRequestId() ==
|
|
|
|
+ container.getAllocationRequestId()) {
|
|
|
|
+ int numAllocations =
|
|
|
|
+ schedReq.getResourceSizing().getNumAllocations();
|
|
|
|
+ numAllocations--;
|
|
|
|
+ if (numAllocations == 0) {
|
|
|
|
+ iter.remove();
|
|
|
|
+ } else {
|
|
|
|
+ schedReq.getResourceSizing()
|
|
|
|
+ .setNumAllocations(numAllocations);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
private List<UpdateContainerRequest> createUpdateList() {
|
|
private List<UpdateContainerRequest> createUpdateList() {
|
|
List<UpdateContainerRequest> updateList = new ArrayList<>();
|
|
List<UpdateContainerRequest> updateList = new ArrayList<>();
|
|
for (Map.Entry<ContainerId, SimpleEntry<Container,
|
|
for (Map.Entry<ContainerId, SimpleEntry<Container,
|