|
@@ -24,17 +24,14 @@ import com.google.common.annotations.VisibleForTesting;
|
|
|
import org.apache.commons.lang.StringUtils;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
|
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
|
|
import org.apache.hadoop.yarn.api.records.SchedulingRequest;
|
|
|
-import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
|
import org.apache.log4j.Logger;
|
|
|
|
|
|
import java.util.HashMap;
|
|
|
-import java.util.HashSet;
|
|
|
import java.util.Map;
|
|
|
import java.util.Set;
|
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
@@ -61,9 +58,6 @@ public class AllocationTagsManager {
|
|
|
// Application's tags to Rack
|
|
|
private Map<ApplicationId, TypeToCountedTags> perAppRackMappings =
|
|
|
new HashMap<>();
|
|
|
- // Application's Temporary containers mapping
|
|
|
- private Map<ApplicationId, Map<NodeId, Map<ContainerId, Set<String>>>>
|
|
|
- appTempMappings = new HashMap<>();
|
|
|
|
|
|
// Global tags to node mapping (used to fast return aggregated tags
|
|
|
// cardinality across apps)
|
|
@@ -76,7 +70,7 @@ public class AllocationTagsManager {
|
|
|
* Currently used both for NodeId to Tag, Count and Rack to Tag, Count
|
|
|
*/
|
|
|
@VisibleForTesting
|
|
|
- static class TypeToCountedTags<T> {
|
|
|
+ public static class TypeToCountedTags<T> {
|
|
|
// Map<Type, Map<Tag, Count>>
|
|
|
private Map<T, Map<String, Long>> typeToTagsWithCount = new HashMap<>();
|
|
|
|
|
@@ -214,7 +208,7 @@ public class AllocationTagsManager {
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
|
- Map<ApplicationId, TypeToCountedTags> getPerAppNodeMappings() {
|
|
|
+ public Map<ApplicationId, TypeToCountedTags> getPerAppNodeMappings() {
|
|
|
return perAppNodeMappings;
|
|
|
}
|
|
|
|
|
@@ -233,12 +227,6 @@ public class AllocationTagsManager {
|
|
|
return globalRackMapping;
|
|
|
}
|
|
|
|
|
|
- @VisibleForTesting
|
|
|
- public Map<NodeId, Map<ContainerId, Set<String>>> getAppTempMappings(
|
|
|
- ApplicationId applicationId) {
|
|
|
- return appTempMappings.get(applicationId);
|
|
|
- }
|
|
|
-
|
|
|
public AllocationTagsManager(RMContext context) {
|
|
|
ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
|
|
|
readLock = lock.readLock();
|
|
@@ -246,39 +234,6 @@ public class AllocationTagsManager {
|
|
|
rmContext = context;
|
|
|
}
|
|
|
|
|
|
- //
|
|
|
-
|
|
|
- /**
|
|
|
- * Method adds a temporary fake-container tag to Node mapping.
|
|
|
- * Used by the constrained placement algorithm to keep track of containers
|
|
|
- * that are currently placed on nodes but are not yet allocated.
|
|
|
- * @param nodeId
|
|
|
- * @param applicationId
|
|
|
- * @param allocationTags
|
|
|
- */
|
|
|
- public void addTempContainer(NodeId nodeId, ApplicationId applicationId,
|
|
|
- Set<String> allocationTags) {
|
|
|
- ContainerId tmpContainer = ContainerId.newContainerId(
|
|
|
- ApplicationAttemptId.newInstance(applicationId, 1), System.nanoTime());
|
|
|
-
|
|
|
- writeLock.lock();
|
|
|
- try {
|
|
|
- Map<NodeId, Map<ContainerId, Set<String>>> appTempMapping =
|
|
|
- appTempMappings.computeIfAbsent(applicationId, k -> new HashMap<>());
|
|
|
- Map<ContainerId, Set<String>> containerTempMapping =
|
|
|
- appTempMapping.computeIfAbsent(nodeId, k -> new HashMap<>());
|
|
|
- containerTempMapping.put(tmpContainer, allocationTags);
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Added TEMP container=" + tmpContainer + " with tags=["
|
|
|
- + StringUtils.join(allocationTags, ",") + "]");
|
|
|
- }
|
|
|
- } finally {
|
|
|
- writeLock.unlock();
|
|
|
- }
|
|
|
-
|
|
|
- addContainer(nodeId, tmpContainer, allocationTags);
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Notify container allocated on a node.
|
|
|
*
|
|
@@ -297,6 +252,15 @@ public class AllocationTagsManager {
|
|
|
}
|
|
|
ApplicationId applicationId =
|
|
|
containerId.getApplicationAttemptId().getApplicationId();
|
|
|
+ addTags(nodeId, applicationId, allocationTags);
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Added container=" + containerId + " with tags=["
|
|
|
+ + StringUtils.join(allocationTags, ",") + "]");
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ public void addTags(NodeId nodeId, ApplicationId applicationId,
|
|
|
+ Set<String> allocationTags) {
|
|
|
writeLock.lock();
|
|
|
try {
|
|
|
TypeToCountedTags perAppTagsMapping = perAppNodeMappings
|
|
@@ -312,11 +276,6 @@ public class AllocationTagsManager {
|
|
|
perAppRackTagsMapping.addTags(nodeRack, allocationTags);
|
|
|
globalNodeMapping.addTags(nodeId, allocationTags);
|
|
|
globalRackMapping.addTags(nodeRack, allocationTags);
|
|
|
-
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Added container=" + containerId + " with tags=["
|
|
|
- + StringUtils.join(allocationTags, ",") + "]");
|
|
|
- }
|
|
|
} finally {
|
|
|
writeLock.unlock();
|
|
|
}
|
|
@@ -339,6 +298,21 @@ public class AllocationTagsManager {
|
|
|
ApplicationId applicationId =
|
|
|
containerId.getApplicationAttemptId().getApplicationId();
|
|
|
|
|
|
+ removeTags(nodeId, applicationId, allocationTags);
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Removed container=" + containerId + " with tags=["
|
|
|
+ + StringUtils.join(allocationTags, ",") + "]");
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Helper method to just remove the tags associated with a container.
|
|
|
+ * @param nodeId
|
|
|
+ * @param applicationId
|
|
|
+ * @param allocationTags
|
|
|
+ */
|
|
|
+ public void removeTags(NodeId nodeId, ApplicationId applicationId,
|
|
|
+ Set<String> allocationTags) {
|
|
|
writeLock.lock();
|
|
|
try {
|
|
|
TypeToCountedTags perAppTagsMapping =
|
|
@@ -364,43 +338,11 @@ public class AllocationTagsManager {
|
|
|
if (perAppRackTagsMapping.isEmpty()) {
|
|
|
perAppRackMappings.remove(applicationId);
|
|
|
}
|
|
|
-
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Removed container=" + containerId + " with tags=["
|
|
|
- + StringUtils.join(allocationTags, ",") + "]");
|
|
|
- }
|
|
|
} finally {
|
|
|
writeLock.unlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Method removes temporary containers associated with an application
|
|
|
- * Used by the placement algorithm to clean temporary tags at the end of
|
|
|
- * a placement cycle.
|
|
|
- * @param applicationId Application Id.
|
|
|
- */
|
|
|
- public void cleanTempContainers(ApplicationId applicationId) {
|
|
|
-
|
|
|
- if (!appTempMappings.get(applicationId).isEmpty()) {
|
|
|
- appTempMappings.get(applicationId).entrySet().stream().forEach(nodeE -> {
|
|
|
- nodeE.getValue().entrySet().stream().forEach(containerE -> {
|
|
|
- removeContainer(nodeE.getKey(), containerE.getKey(),
|
|
|
- containerE.getValue());
|
|
|
- });
|
|
|
- });
|
|
|
- writeLock.lock();
|
|
|
- try {
|
|
|
- appTempMappings.remove(applicationId);
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Removed TEMP containers of app=" + applicationId);
|
|
|
- }
|
|
|
- } finally {
|
|
|
- writeLock.unlock();
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
|
|
|
/**
|
|
|
* Get Node cardinality for a specific tag.
|