|
@@ -25,13 +25,16 @@ import java.util.Queue;
|
|
|
import java.util.Random;
|
|
|
import java.util.concurrent.Callable;
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
|
+import java.util.concurrent.ConcurrentMap;
|
|
|
import java.util.concurrent.Future;
|
|
|
import java.util.concurrent.LinkedBlockingQueue;
|
|
|
import java.util.concurrent.ThreadPoolExecutor;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
+import java.util.concurrent.TimeoutException;
|
|
|
import java.util.function.Supplier;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
|
@@ -40,10 +43,12 @@ import org.apache.hadoop.yarn.api.records.Priority;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
|
|
@@ -393,6 +398,64 @@ public class TestActivitiesManager {
|
|
|
testingTimes);
|
|
|
}
|
|
|
|
|
|
+ @Test (timeout = 10000)
|
|
|
+ public void testAppActivitiesMaxQueueLengthUpdate()
|
|
|
+ throws TimeoutException, InterruptedException {
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ int configuredAppActivitiesMaxQueueLength = 1;
|
|
|
+ conf.setInt(YarnConfiguration.
|
|
|
+ RM_ACTIVITIES_MANAGER_APP_ACTIVITIES_MAX_QUEUE_LENGTH,
|
|
|
+ configuredAppActivitiesMaxQueueLength);
|
|
|
+ conf.setInt(YarnConfiguration.RM_ACTIVITIES_MANAGER_CLEANUP_INTERVAL_MS,
|
|
|
+ 500);
|
|
|
+ ConcurrentMap<NodeId, RMNode> mockNodes = new ConcurrentHashMap<>();
|
|
|
+ int numNodes = 5;
|
|
|
+ for (int i = 0; i < numNodes; i++) {
|
|
|
+ mockNodes.put(NodeId.newInstance("node" + i, 0), mock(RMNode.class));
|
|
|
+ }
|
|
|
+ CapacityScheduler cs = Mockito.mock(CapacityScheduler.class);
|
|
|
+ RMContext mockRMContext = Mockito.mock(RMContext.class);
|
|
|
+ Mockito.when(mockRMContext.getRMNodes()).thenReturn(mockNodes);
|
|
|
+ Mockito.when(mockRMContext.getYarnConfiguration()).thenReturn(conf);
|
|
|
+ Mockito.when(mockRMContext.getScheduler()).thenReturn(cs);
|
|
|
+ /*
|
|
|
+ * Test for async-scheduling with multi-node placement disabled
|
|
|
+ */
|
|
|
+ Mockito.when(cs.isMultiNodePlacementEnabled()).thenReturn(false);
|
|
|
+ int numAsyncSchedulerThreads = 3;
|
|
|
+ Mockito.when(cs.getNumAsyncSchedulerThreads())
|
|
|
+ .thenReturn(numAsyncSchedulerThreads);
|
|
|
+ ActivitiesManager newActivitiesManager =
|
|
|
+ new ActivitiesManager(mockRMContext);
|
|
|
+ Assert.assertEquals(1,
|
|
|
+ newActivitiesManager.getAppActivitiesMaxQueueLength());
|
|
|
+ newActivitiesManager.init(conf);
|
|
|
+ newActivitiesManager.start();
|
|
|
+ GenericTestUtils.waitFor(
|
|
|
+ () -> newActivitiesManager.getAppActivitiesMaxQueueLength()
|
|
|
+ == numNodes * numAsyncSchedulerThreads, 100, 3000);
|
|
|
+ Assert.assertEquals(15,
|
|
|
+ newActivitiesManager.getAppActivitiesMaxQueueLength());
|
|
|
+ /*
|
|
|
+ * Test for HB-driven scheduling with multi-node placement disabled
|
|
|
+ */
|
|
|
+ Mockito.when(cs.getNumAsyncSchedulerThreads()).thenReturn(0);
|
|
|
+ GenericTestUtils.waitFor(
|
|
|
+ () -> newActivitiesManager.getAppActivitiesMaxQueueLength()
|
|
|
+ == numNodes * 1.2, 100, 3000);
|
|
|
+ Assert.assertEquals(6,
|
|
|
+ newActivitiesManager.getAppActivitiesMaxQueueLength());
|
|
|
+ /*
|
|
|
+ * Test for scheduling with multi-node placement enabled
|
|
|
+ */
|
|
|
+ Mockito.when(cs.isMultiNodePlacementEnabled()).thenReturn(true);
|
|
|
+ GenericTestUtils.waitFor(
|
|
|
+ () -> newActivitiesManager.getAppActivitiesMaxQueueLength()
|
|
|
+ == configuredAppActivitiesMaxQueueLength, 100, 3000);
|
|
|
+ Assert.assertEquals(1,
|
|
|
+ newActivitiesManager.getAppActivitiesMaxQueueLength());
|
|
|
+ }
|
|
|
+
|
|
|
private void testManyTimes(String testingName,
|
|
|
Supplier<Void> supplier, int testingTimes) {
|
|
|
long totalTime = 0;
|