|
@@ -37,6 +37,8 @@ import java.util.Iterator;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.Set;
|
|
|
+import java.util.concurrent.ConcurrentHashMap;
|
|
|
+import java.util.concurrent.ConcurrentMap;
|
|
|
import java.util.stream.Collectors;
|
|
|
import java.util.stream.Stream;
|
|
|
import java.util.concurrent.atomic.AtomicLong;
|
|
@@ -52,6 +54,9 @@ import org.apache.hadoop.yarn.api.records.SchedulingRequest;
|
|
|
import org.apache.hadoop.yarn.api.records.Priority;
|
|
|
import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
|
|
|
import org.apache.hadoop.yarn.api.records.ResourceSizing;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.MockRMApp;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
|
|
@@ -63,6 +68,7 @@ import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
import com.google.common.collect.ImmutableSet;
|
|
|
+import org.mockito.Mockito;
|
|
|
|
|
|
/**
|
|
|
* Test the PlacementConstraint Utility class functionality.
|
|
@@ -562,8 +568,8 @@ public class TestPlacementConstraintsUtil {
|
|
|
SchedulerNode schedulerNode3 = newSchedulerNode(n3r2.getHostName(),
|
|
|
n3r2.getRackName(), n3r2.getNodeID());
|
|
|
|
|
|
- AllocationTagNamespace namespaceAll =
|
|
|
- new AllocationTagNamespace.All();
|
|
|
+ TargetApplicationsNamespace namespaceAll =
|
|
|
+ new TargetApplicationsNamespace.All();
|
|
|
|
|
|
//***************************
|
|
|
// 1) all, anti-affinity
|
|
@@ -648,17 +654,28 @@ public class TestPlacementConstraintsUtil {
|
|
|
@Test
|
|
|
public void testNotSelfAppConstraints()
|
|
|
throws InvalidAllocationTagsQueryException {
|
|
|
- AllocationTagsManager tm = new AllocationTagsManager(rmContext);
|
|
|
- PlacementConstraintManagerService pcm =
|
|
|
- new MemoryPlacementConstraintManager();
|
|
|
- rmContext.setAllocationTagsManager(tm);
|
|
|
- rmContext.setPlacementConstraintManager(pcm);
|
|
|
-
|
|
|
long ts = System.currentTimeMillis();
|
|
|
ApplicationId application1 = BuilderUtils.newApplicationId(ts, 100);
|
|
|
ApplicationId application2 = BuilderUtils.newApplicationId(ts, 101);
|
|
|
ApplicationId application3 = BuilderUtils.newApplicationId(ts, 102);
|
|
|
|
|
|
+ ConcurrentMap<ApplicationId, RMApp> allApps = new ConcurrentHashMap<>();
|
|
|
+ allApps.put(application1, new MockRMApp(123, 1000,
|
|
|
+ RMAppState.NEW, "userA", ImmutableSet.of("")));
|
|
|
+ allApps.put(application2, new MockRMApp(124, 1001,
|
|
|
+ RMAppState.NEW, "userA", ImmutableSet.of("")));
|
|
|
+ allApps.put(application3, new MockRMApp(125, 1002,
|
|
|
+ RMAppState.NEW, "userA", ImmutableSet.of("")));
|
|
|
+
|
|
|
+ RMContext mockedContext = Mockito.spy(rmContext);
|
|
|
+ when(mockedContext.getRMApps()).thenReturn(allApps);
|
|
|
+
|
|
|
+ AllocationTagsManager tm = new AllocationTagsManager(mockedContext);
|
|
|
+ PlacementConstraintManagerService pcm =
|
|
|
+ new MemoryPlacementConstraintManager();
|
|
|
+ mockedContext.setAllocationTagsManager(tm);
|
|
|
+ mockedContext.setPlacementConstraintManager(pcm);
|
|
|
+
|
|
|
// Register App1 with anti-affinity constraint map.
|
|
|
RMNode n0r1 = rmNodes.get(0);
|
|
|
RMNode n1r1 = rmNodes.get(1);
|
|
@@ -696,8 +713,8 @@ public class TestPlacementConstraintsUtil {
|
|
|
SchedulerNode schedulerNode3 = newSchedulerNode(n3r2.getHostName(),
|
|
|
n3r2.getRackName(), n3r2.getNodeID());
|
|
|
|
|
|
- AllocationTagNamespace notSelf =
|
|
|
- new AllocationTagNamespace.NotSelf();
|
|
|
+ TargetApplicationsNamespace notSelf =
|
|
|
+ new TargetApplicationsNamespace.NotSelf();
|
|
|
|
|
|
//***************************
|
|
|
// 1) not-self, app1
|
|
@@ -800,8 +817,8 @@ public class TestPlacementConstraintsUtil {
|
|
|
SchedulerNode schedulerNode3 =newSchedulerNode(n3r2.getHostName(),
|
|
|
n3r2.getRackName(), n3r2.getNodeID());
|
|
|
|
|
|
- AllocationTagNamespace namespace =
|
|
|
- new AllocationTagNamespace.AppID(application1);
|
|
|
+ TargetApplicationsNamespace namespace =
|
|
|
+ new TargetApplicationsNamespace.AppID(application1);
|
|
|
Map<Set<String>, PlacementConstraint> constraintMap = new HashMap<>();
|
|
|
PlacementConstraint constraint2 = PlacementConstraints
|
|
|
.targetNotIn(NODE, allocationTagWithNamespace(namespace.toString(),
|
|
@@ -832,7 +849,7 @@ public class TestPlacementConstraintsUtil {
|
|
|
|
|
|
// Intra-app constraint
|
|
|
// Test with default and empty namespace
|
|
|
- AllocationTagNamespace self = new AllocationTagNamespace.Self();
|
|
|
+ TargetApplicationsNamespace self = new TargetApplicationsNamespace.Self();
|
|
|
PlacementConstraint constraint3 = PlacementConstraints
|
|
|
.targetNotIn(NODE, allocationTagWithNamespace(self.toString(),
|
|
|
"hbase-m"))
|
|
@@ -872,6 +889,88 @@ public class TestPlacementConstraintsUtil {
|
|
|
pcm.unregisterApplication(application3);
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testInterAppConstriantsByAppTag()
|
|
|
+ throws InvalidAllocationTagsQueryException {
|
|
|
+
|
|
|
+ ApplicationId application1 = BuilderUtils.newApplicationId(1000, 123);
|
|
|
+ ApplicationId application2 = BuilderUtils.newApplicationId(1001, 124);
|
|
|
+
|
|
|
+ // app1: test-tag
|
|
|
+ // app2: N/A
|
|
|
+ RMContext mockedContext = Mockito.spy(rmContext);
|
|
|
+ ConcurrentMap<ApplicationId, RMApp> allApps = new ConcurrentHashMap<>();
|
|
|
+ allApps.put(application1, new MockRMApp(123, 1000,
|
|
|
+ RMAppState.NEW, "userA", ImmutableSet.of("test-tag")));
|
|
|
+ allApps.put(application2, new MockRMApp(124, 1001,
|
|
|
+ RMAppState.NEW, "userA", ImmutableSet.of("")));
|
|
|
+ when(mockedContext.getRMApps()).thenReturn(allApps);
|
|
|
+
|
|
|
+ AllocationTagsManager tm = new AllocationTagsManager(mockedContext);
|
|
|
+ PlacementConstraintManagerService pcm =
|
|
|
+ new MemoryPlacementConstraintManager();
|
|
|
+ mockedContext.setAllocationTagsManager(tm);
|
|
|
+ mockedContext.setPlacementConstraintManager(pcm);
|
|
|
+
|
|
|
+ // Register App1 with anti-affinity constraint map.
|
|
|
+ RMNode n0r1 = rmNodes.get(0);
|
|
|
+ RMNode n1r1 = rmNodes.get(1);
|
|
|
+ RMNode n2r2 = rmNodes.get(2);
|
|
|
+ RMNode n3r2 = rmNodes.get(3);
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Place container:
|
|
|
+ * n0: app1/hbase-m(1)
|
|
|
+ * n1: ""
|
|
|
+ * n2: app1/hbase-m(1)
|
|
|
+ * n3: ""
|
|
|
+ */
|
|
|
+ tm.addContainer(n0r1.getNodeID(),
|
|
|
+ newContainerId(application1), ImmutableSet.of("hbase-m"));
|
|
|
+ tm.addContainer(n2r2.getNodeID(),
|
|
|
+ newContainerId(application1), ImmutableSet.of("hbase-m"));
|
|
|
+
|
|
|
+ SchedulerNode schedulerNode0 = newSchedulerNode(n0r1.getHostName(),
|
|
|
+ n0r1.getRackName(), n0r1.getNodeID());
|
|
|
+ SchedulerNode schedulerNode1 = newSchedulerNode(n1r1.getHostName(),
|
|
|
+ n1r1.getRackName(), n1r1.getNodeID());
|
|
|
+ SchedulerNode schedulerNode2 = newSchedulerNode(n2r2.getHostName(),
|
|
|
+ n2r2.getRackName(), n2r2.getNodeID());
|
|
|
+ SchedulerNode schedulerNode3 = newSchedulerNode(n3r2.getHostName(),
|
|
|
+ n3r2.getRackName(), n3r2.getNodeID());
|
|
|
+
|
|
|
+ TargetApplicationsNamespace namespace =
|
|
|
+ new TargetApplicationsNamespace.AppTag("test-tag");
|
|
|
+ Map<Set<String>, PlacementConstraint> constraintMap = new HashMap<>();
|
|
|
+ PlacementConstraint constraint2 = PlacementConstraints
|
|
|
+ .targetNotIn(NODE, allocationTagWithNamespace(namespace.toString(),
|
|
|
+ "hbase-m"))
|
|
|
+ .build();
|
|
|
+ Set<String> srcTags2 = ImmutableSet.of("app2");
|
|
|
+ constraintMap.put(srcTags2, constraint2);
|
|
|
+
|
|
|
+ pcm.registerApplication(application2, constraintMap);
|
|
|
+
|
|
|
+ // Anti-affinity with app-tag/test-tag/hbase-m,
|
|
|
+ // app1 has tag "test-tag" so the constraint is equally to work on app1
|
|
|
+ // onto n1 and n3 as they don't have "hbase-m" from app1.
|
|
|
+ Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
|
|
|
+ application2, createSchedulingRequest(srcTags2),
|
|
|
+ schedulerNode0, pcm, tm));
|
|
|
+ Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(
|
|
|
+ application2, createSchedulingRequest(srcTags2),
|
|
|
+ schedulerNode1, pcm, tm));
|
|
|
+ Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(
|
|
|
+ application2, createSchedulingRequest(srcTags2),
|
|
|
+ schedulerNode2, pcm, tm));
|
|
|
+ Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(
|
|
|
+ application2, createSchedulingRequest(srcTags2),
|
|
|
+ schedulerNode3, pcm, tm));
|
|
|
+
|
|
|
+ pcm.unregisterApplication(application1);
|
|
|
+ pcm.unregisterApplication(application2);
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testInvalidAllocationTagNamespace() {
|
|
|
AllocationTagsManager tm = new AllocationTagsManager(rmContext);
|