|
@@ -18,6 +18,7 @@
|
|
|
package org.apache.hadoop.yarn.server.resourcemanager.reservation;
|
|
|
|
|
|
import static org.mockito.Matchers.any;
|
|
|
+import static org.mockito.Matchers.anySetOf;
|
|
|
import static org.mockito.Mockito.doReturn;
|
|
|
import static org.mockito.Mockito.mock;
|
|
|
import static org.mockito.Mockito.when;
|
|
@@ -29,6 +30,7 @@ import java.util.Random;
|
|
|
import java.util.Set;
|
|
|
import java.util.TreeMap;
|
|
|
|
|
|
+import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.yarn.api.records.ReservationDefinition;
|
|
|
import org.apache.hadoop.yarn.api.records.ReservationId;
|
|
|
import org.apache.hadoop.yarn.api.records.ReservationRequest;
|
|
@@ -41,6 +43,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
|
|
@@ -61,6 +64,44 @@ public class ReservationSystemTestUtil {
|
|
|
return ReservationId.newInstance(rand.nextLong(), rand.nextLong());
|
|
|
}
|
|
|
|
|
|
+ public static ReservationSchedulerConfiguration createConf(
|
|
|
+ String reservationQ, long timeWindow, float instConstraint,
|
|
|
+ float avgConstraint) {
|
|
|
+ ReservationSchedulerConfiguration conf = mock
|
|
|
+ (ReservationSchedulerConfiguration.class);
|
|
|
+ when(conf.getReservationWindow(reservationQ)).thenReturn(timeWindow);
|
|
|
+ when(conf.getInstantaneousMaxCapacity(reservationQ)).thenReturn
|
|
|
+ (instConstraint);
|
|
|
+ when(conf.getAverageCapacity(reservationQ)).thenReturn(avgConstraint);
|
|
|
+ return conf;
|
|
|
+ }
|
|
|
+
|
|
|
+ public static void validateReservationQueue(
|
|
|
+ AbstractReservationSystem reservationSystem, String planQName) {
|
|
|
+ Plan plan = reservationSystem.getPlan(planQName);
|
|
|
+ Assert.assertNotNull(plan);
|
|
|
+ Assert.assertTrue(plan instanceof InMemoryPlan);
|
|
|
+ Assert.assertEquals(planQName, plan.getQueueName());
|
|
|
+ Assert.assertEquals(8192, plan.getTotalCapacity().getMemory());
|
|
|
+ Assert.assertTrue(
|
|
|
+ plan.getReservationAgent() instanceof GreedyReservationAgent);
|
|
|
+ Assert.assertTrue(
|
|
|
+ plan.getSharingPolicy() instanceof CapacityOverTimePolicy);
|
|
|
+ }
|
|
|
+
|
|
|
+ public static void validateNewReservationQueue(
|
|
|
+ AbstractReservationSystem reservationSystem, String newQ) {
|
|
|
+ Plan newPlan = reservationSystem.getPlan(newQ);
|
|
|
+ Assert.assertNotNull(newPlan);
|
|
|
+ Assert.assertTrue(newPlan instanceof InMemoryPlan);
|
|
|
+ Assert.assertEquals(newQ, newPlan.getQueueName());
|
|
|
+ Assert.assertEquals(1024, newPlan.getTotalCapacity().getMemory());
|
|
|
+ Assert
|
|
|
+ .assertTrue(newPlan.getReservationAgent() instanceof GreedyReservationAgent);
|
|
|
+ Assert
|
|
|
+ .assertTrue(newPlan.getSharingPolicy() instanceof CapacityOverTimePolicy);
|
|
|
+ }
|
|
|
+
|
|
|
@SuppressWarnings("unchecked")
|
|
|
public CapacityScheduler mockCapacityScheduler(int numContainers)
|
|
|
throws IOException {
|
|
@@ -70,15 +111,38 @@ public class ReservationSystemTestUtil {
|
|
|
|
|
|
CapacityScheduler cs = Mockito.spy(new CapacityScheduler());
|
|
|
cs.setConf(new YarnConfiguration());
|
|
|
+
|
|
|
+ RMContext mockRmContext = createRMContext(conf);
|
|
|
+
|
|
|
+ cs.setRMContext(mockRmContext);
|
|
|
+ try {
|
|
|
+ cs.serviceInit(conf);
|
|
|
+ } catch (Exception e) {
|
|
|
+ Assert.fail(e.getMessage());
|
|
|
+ }
|
|
|
+
|
|
|
+ initializeRMContext(numContainers, cs, mockRmContext);
|
|
|
+ return cs;
|
|
|
+ }
|
|
|
+
|
|
|
+ public static void initializeRMContext(int numContainers,
|
|
|
+ AbstractYarnScheduler scheduler, RMContext mockRMContext) {
|
|
|
+
|
|
|
+ when(mockRMContext.getScheduler()).thenReturn(scheduler);
|
|
|
+ Resource r = calculateClusterResource(numContainers);
|
|
|
+ doReturn(r).when(scheduler).getClusterResource();
|
|
|
+ }
|
|
|
+
|
|
|
+ public static RMContext createRMContext(Configuration conf) {
|
|
|
RMContext mockRmContext =
|
|
|
Mockito.spy(new RMContextImpl(null, null, null, null, null, null,
|
|
|
new RMContainerTokenSecretManager(conf),
|
|
|
new NMTokenSecretManagerInRM(conf),
|
|
|
new ClientToAMTokenSecretManagerInRM(), null));
|
|
|
-
|
|
|
+
|
|
|
RMNodeLabelsManager nlm = mock(RMNodeLabelsManager.class);
|
|
|
when(
|
|
|
- nlm.getQueueResource(any(String.class), any(Set.class),
|
|
|
+ nlm.getQueueResource(any(String.class), anySetOf(String.class),
|
|
|
any(Resource.class))).thenAnswer(new Answer<Resource>() {
|
|
|
@Override
|
|
|
public Resource answer(InvocationOnMock invocation) throws Throwable {
|
|
@@ -86,7 +150,7 @@ public class ReservationSystemTestUtil {
|
|
|
return (Resource) args[2];
|
|
|
}
|
|
|
});
|
|
|
-
|
|
|
+
|
|
|
when(nlm.getResourceByLabel(any(String.class), any(Resource.class)))
|
|
|
.thenAnswer(new Answer<Resource>() {
|
|
|
@Override
|
|
@@ -95,19 +159,9 @@ public class ReservationSystemTestUtil {
|
|
|
return (Resource) args[1];
|
|
|
}
|
|
|
});
|
|
|
-
|
|
|
+
|
|
|
mockRmContext.setNodeLabelManager(nlm);
|
|
|
-
|
|
|
- cs.setRMContext(mockRmContext);
|
|
|
- try {
|
|
|
- cs.serviceInit(conf);
|
|
|
- } catch (Exception e) {
|
|
|
- Assert.fail(e.getMessage());
|
|
|
- }
|
|
|
- when(mockRmContext.getScheduler()).thenReturn(cs);
|
|
|
- Resource r = Resource.newInstance(numContainers * 1024, numContainers);
|
|
|
- doReturn(r).when(cs).getClusterResource();
|
|
|
- return cs;
|
|
|
+ return mockRmContext;
|
|
|
}
|
|
|
|
|
|
public static void setupQueueConfiguration(CapacitySchedulerConfiguration conf) {
|
|
@@ -174,7 +228,7 @@ public class ReservationSystemTestUtil {
|
|
|
// Define 2nd-level queues
|
|
|
final String A1 = A + ".a1";
|
|
|
final String A2 = A + ".a2";
|
|
|
- conf.setQueues(A, new String[] { "a1", "a2" });
|
|
|
+ conf.setQueues(A, new String[]{"a1", "a2"});
|
|
|
conf.setCapacity(A1, 30);
|
|
|
conf.setCapacity(A2, 70);
|
|
|
}
|
|
@@ -254,4 +308,9 @@ public class ReservationSystemTestUtil {
|
|
|
return req;
|
|
|
}
|
|
|
|
|
|
+ public static Resource calculateClusterResource(int numContainers) {
|
|
|
+ Resource clusterResource = Resource.newInstance(numContainers * 1024,
|
|
|
+ numContainers);
|
|
|
+ return clusterResource;
|
|
|
+ }
|
|
|
}
|