|
@@ -33,6 +33,7 @@ import java.util.Set;
|
|
|
import java.util.SortedMap;
|
|
|
import java.util.TreeMap;
|
|
|
import java.util.TreeSet;
|
|
|
+import java.util.AbstractMap.SimpleEntry;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
@@ -49,7 +50,9 @@ import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
|
|
|
+import org.apache.hadoop.yarn.api.records.Container;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
+import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.NMToken;
|
|
@@ -72,6 +75,7 @@ import org.apache.hadoop.yarn.util.RackResolver;
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
import com.google.common.base.Joiner;
|
|
|
import com.google.common.base.Preconditions;
|
|
|
+import org.apache.hadoop.yarn.util.resource.Resources;
|
|
|
|
|
|
@Private
|
|
|
@Unstable
|
|
@@ -110,8 +114,8 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
|
|
|
containerRequests = new LinkedHashSet<T>();
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
-
|
|
|
+
|
|
|
+
|
|
|
/**
|
|
|
* Class compares Resource by memory then cpu in reverse order
|
|
|
*/
|
|
@@ -144,10 +148,7 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
|
|
|
int cpu0 = arg0.getVirtualCores();
|
|
|
int cpu1 = arg1.getVirtualCores();
|
|
|
|
|
|
- if(mem0 <= mem1 && cpu0 <= cpu1) {
|
|
|
- return true;
|
|
|
- }
|
|
|
- return false;
|
|
|
+ return (mem0 <= mem1 && cpu0 <= cpu1);
|
|
|
}
|
|
|
|
|
|
//Key -> Priority
|
|
@@ -164,11 +165,22 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
|
|
|
protected final Set<ResourceRequest> ask = new TreeSet<ResourceRequest>(
|
|
|
new org.apache.hadoop.yarn.api.records.ResourceRequest.ResourceRequestComparator());
|
|
|
protected final Set<ContainerId> release = new TreeSet<ContainerId>();
|
|
|
- // pendingRelease holds history or release requests.request is removed only if
|
|
|
- // RM sends completedContainer.
|
|
|
+ // pendingRelease holds history of release requests.
|
|
|
+ // request is removed only if RM sends completedContainer.
|
|
|
// How it different from release? --> release is for per allocate() request.
|
|
|
protected Set<ContainerId> pendingRelease = new TreeSet<ContainerId>();
|
|
|
-
|
|
|
+ // change map holds container resource change requests between two allocate()
|
|
|
+ // calls, and are cleared after each successful allocate() call.
|
|
|
+ protected final Map<ContainerId, SimpleEntry<Container, Resource>> change =
|
|
|
+ new HashMap<>();
|
|
|
+ // pendingChange map holds history of container resource change requests in
|
|
|
+ // case AM needs to reregister with the ResourceManager.
|
|
|
+ // Change requests are removed from this map if RM confirms the change
|
|
|
+ // through allocate response, or if RM confirms that the container has been
|
|
|
+ // completed.
|
|
|
+ protected final Map<ContainerId, SimpleEntry<Container, Resource>>
|
|
|
+ pendingChange = new HashMap<>();
|
|
|
+
|
|
|
public AMRMClientImpl() {
|
|
|
super(AMRMClientImpl.class.getName());
|
|
|
}
|
|
@@ -241,7 +253,8 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
|
|
|
AllocateRequest allocateRequest = null;
|
|
|
List<String> blacklistToAdd = new ArrayList<String>();
|
|
|
List<String> blacklistToRemove = new ArrayList<String>();
|
|
|
-
|
|
|
+ Map<ContainerId, SimpleEntry<Container, Resource>> oldChange =
|
|
|
+ new HashMap<>();
|
|
|
try {
|
|
|
synchronized (this) {
|
|
|
askList = new ArrayList<ResourceRequest>(ask.size());
|
|
@@ -252,10 +265,30 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
|
|
|
r.getResourceName(), r.getCapability(), r.getNumContainers(),
|
|
|
r.getRelaxLocality(), r.getNodeLabelExpression()));
|
|
|
}
|
|
|
+ List<ContainerResourceChangeRequest> increaseList = new ArrayList<>();
|
|
|
+ List<ContainerResourceChangeRequest> decreaseList = new ArrayList<>();
|
|
|
+ // Save the current change for recovery
|
|
|
+ oldChange.putAll(change);
|
|
|
+ for (Map.Entry<ContainerId, SimpleEntry<Container, Resource>> entry :
|
|
|
+ change.entrySet()) {
|
|
|
+ Container container = entry.getValue().getKey();
|
|
|
+ Resource original = container.getResource();
|
|
|
+ Resource target = entry.getValue().getValue();
|
|
|
+ if (Resources.fitsIn(target, original)) {
|
|
|
+ // This is a decrease request
|
|
|
+ decreaseList.add(ContainerResourceChangeRequest.newInstance(
|
|
|
+ container.getId(), target));
|
|
|
+ } else {
|
|
|
+ // This is an increase request
|
|
|
+ increaseList.add(ContainerResourceChangeRequest.newInstance(
|
|
|
+ container.getId(), target));
|
|
|
+ }
|
|
|
+ }
|
|
|
releaseList = new ArrayList<ContainerId>(release);
|
|
|
// optimistically clear this collection assuming no RPC failure
|
|
|
ask.clear();
|
|
|
release.clear();
|
|
|
+ change.clear();
|
|
|
|
|
|
blacklistToAdd.addAll(blacklistAdditions);
|
|
|
blacklistToRemove.addAll(blacklistRemovals);
|
|
@@ -266,8 +299,9 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
|
|
|
|
|
|
allocateRequest =
|
|
|
AllocateRequest.newInstance(lastResponseId, progressIndicator,
|
|
|
- askList, releaseList, blacklistRequest);
|
|
|
- // clear blacklistAdditions and blacklistRemovals before
|
|
|
+ askList, releaseList, blacklistRequest,
|
|
|
+ increaseList, decreaseList);
|
|
|
+ // clear blacklistAdditions and blacklistRemovals before
|
|
|
// unsynchronized part
|
|
|
blacklistAdditions.clear();
|
|
|
blacklistRemovals.clear();
|
|
@@ -289,6 +323,7 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
+ change.putAll(this.pendingChange);
|
|
|
}
|
|
|
// re register with RM
|
|
|
registerApplicationMaster();
|
|
@@ -312,6 +347,23 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
|
|
|
removePendingReleaseRequests(allocateResponse
|
|
|
.getCompletedContainersStatuses());
|
|
|
}
|
|
|
+ if (!pendingChange.isEmpty()) {
|
|
|
+ List<ContainerStatus> completed =
|
|
|
+ allocateResponse.getCompletedContainersStatuses();
|
|
|
+ List<Container> changed = new ArrayList<>();
|
|
|
+ changed.addAll(allocateResponse.getIncreasedContainers());
|
|
|
+ changed.addAll(allocateResponse.getDecreasedContainers());
|
|
|
+ // remove all pending change requests that belong to the completed
|
|
|
+ // containers
|
|
|
+ for (ContainerStatus status : completed) {
|
|
|
+ ContainerId containerId = status.getContainerId();
|
|
|
+ pendingChange.remove(containerId);
|
|
|
+ }
|
|
|
+ // remove all pending change requests that have been satisfied
|
|
|
+ if (!changed.isEmpty()) {
|
|
|
+ removePendingChangeRequests(changed);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
} finally {
|
|
|
// TODO how to differentiate remote yarn exception vs error in rpc
|
|
@@ -333,7 +385,22 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
|
|
|
ask.add(oldAsk);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+ // change requests could have been added during the allocate call.
|
|
|
+ // Those are the newest requests which take precedence
|
|
|
+ // over requests cached in the oldChange map.
|
|
|
+ //
|
|
|
+ // Only insert entries from the cached oldChange map
|
|
|
+ // that do not exist in the current change map:
|
|
|
+ for (Map.Entry<ContainerId, SimpleEntry<Container, Resource>> entry :
|
|
|
+ oldChange.entrySet()) {
|
|
|
+ ContainerId oldContainerId = entry.getKey();
|
|
|
+ Container oldContainer = entry.getValue().getKey();
|
|
|
+ Resource oldResource = entry.getValue().getValue();
|
|
|
+ if (change.get(oldContainerId) == null) {
|
|
|
+ change.put(
|
|
|
+ oldContainerId, new SimpleEntry<>(oldContainer, oldResource));
|
|
|
+ }
|
|
|
+ }
|
|
|
blacklistAdditions.addAll(blacklistToAdd);
|
|
|
blacklistRemovals.addAll(blacklistToRemove);
|
|
|
}
|
|
@@ -349,6 +416,24 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ protected void removePendingChangeRequests(
|
|
|
+ List<Container> changedContainers) {
|
|
|
+ for (Container changedContainer : changedContainers) {
|
|
|
+ ContainerId containerId = changedContainer.getId();
|
|
|
+ if (pendingChange.get(containerId) == null) {
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("RM has confirmed changed resource allocation for "
|
|
|
+ + "container " + containerId + ". Current resource allocation:"
|
|
|
+ + changedContainer.getResource()
|
|
|
+ + ". Remove pending change request:"
|
|
|
+ + pendingChange.get(containerId).getValue());
|
|
|
+ }
|
|
|
+ pendingChange.remove(containerId);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@Private
|
|
|
@VisibleForTesting
|
|
|
protected void populateNMTokens(List<NMToken> nmTokens) {
|
|
@@ -479,12 +564,32 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
|
|
|
req.getCapability(), req);
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
+ public synchronized void requestContainerResourceChange(
|
|
|
+ Container container, Resource capability) {
|
|
|
+ validateContainerResourceChangeRequest(
|
|
|
+ container.getId(), container.getResource(), capability);
|
|
|
+ if (change.get(container.getId()) == null) {
|
|
|
+ change.put(container.getId(),
|
|
|
+ new SimpleEntry<>(container, capability));
|
|
|
+ } else {
|
|
|
+ change.get(container.getId()).setValue(capability);
|
|
|
+ }
|
|
|
+ if (pendingChange.get(container.getId()) == null) {
|
|
|
+ pendingChange.put(container.getId(),
|
|
|
+ new SimpleEntry<>(container, capability));
|
|
|
+ } else {
|
|
|
+ pendingChange.get(container.getId()).setValue(capability);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
public synchronized void releaseAssignedContainer(ContainerId containerId) {
|
|
|
Preconditions.checkArgument(containerId != null,
|
|
|
"ContainerId can not be null.");
|
|
|
pendingRelease.add(containerId);
|
|
|
release.add(containerId);
|
|
|
+ pendingChange.remove(containerId);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -618,7 +723,23 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
|
|
|
"Cannot specify node label with rack and node");
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
+ private void validateContainerResourceChangeRequest(
|
|
|
+ ContainerId containerId, Resource original, Resource target) {
|
|
|
+ Preconditions.checkArgument(containerId != null,
|
|
|
+ "ContainerId cannot be null");
|
|
|
+ Preconditions.checkArgument(original != null,
|
|
|
+ "Original resource capability cannot be null");
|
|
|
+ Preconditions.checkArgument(!Resources.equals(Resources.none(), original)
|
|
|
+ && Resources.fitsIn(Resources.none(), original),
|
|
|
+ "Original resource capability must be greater than 0");
|
|
|
+ Preconditions.checkArgument(target != null,
|
|
|
+ "Target resource capability cannot be null");
|
|
|
+ Preconditions.checkArgument(!Resources.equals(Resources.none(), target)
|
|
|
+ && Resources.fitsIn(Resources.none(), target),
|
|
|
+ "Target resource capability must be greater than 0");
|
|
|
+ }
|
|
|
+
|
|
|
private void addResourceRequestToAsk(ResourceRequest remoteRequest) {
|
|
|
// This code looks weird but is needed because of the following scenario.
|
|
|
// A ResourceRequest is removed from the remoteRequestTable. A 0 container
|