|
@@ -17,38 +17,6 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity;
|
|
|
|
|
|
-import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.MAX_IGNORED_OVER_CAPACITY;
|
|
|
-import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.MONITORING_INTERVAL;
|
|
|
-import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.NATURAL_TERMINATION_FACTOR;
|
|
|
-import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.OBSERVE_ONLY;
|
|
|
-import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.TOTAL_PREEMPTION_PER_ROUND;
|
|
|
-import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.WAIT_TIME_BEFORE_KILL;
|
|
|
-import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE;
|
|
|
-import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType.MARK_CONTAINER_FOR_PREEMPTION;
|
|
|
-import static org.junit.Assert.assertEquals;
|
|
|
-import static org.junit.Assert.assertNotNull;
|
|
|
-import static org.junit.Assert.assertTrue;
|
|
|
-import static org.junit.Assert.fail;
|
|
|
-import static org.mockito.Matchers.any;
|
|
|
-import static org.mockito.Matchers.anyString;
|
|
|
-import static org.mockito.Matchers.argThat;
|
|
|
-import static org.mockito.Matchers.isA;
|
|
|
-import static org.mockito.Mockito.mock;
|
|
|
-import static org.mockito.Mockito.never;
|
|
|
-import static org.mockito.Mockito.times;
|
|
|
-import static org.mockito.Mockito.verify;
|
|
|
-import static org.mockito.Mockito.when;
|
|
|
-
|
|
|
-import java.util.ArrayList;
|
|
|
-import java.util.Comparator;
|
|
|
-import java.util.Deque;
|
|
|
-import java.util.LinkedList;
|
|
|
-import java.util.List;
|
|
|
-import java.util.NavigableSet;
|
|
|
-import java.util.Random;
|
|
|
-import java.util.StringTokenizer;
|
|
|
-import java.util.TreeSet;
|
|
|
-
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.service.Service;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
@@ -63,7 +31,6 @@ import org.apache.hadoop.yarn.event.EventHandler;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitor;
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.TempQueuePerPartition;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.resource.Priority;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
|
@@ -95,6 +62,32 @@ import org.mockito.ArgumentMatcher;
|
|
|
import org.mockito.invocation.InvocationOnMock;
|
|
|
import org.mockito.stubbing.Answer;
|
|
|
|
|
|
+import java.util.ArrayList;
|
|
|
+import java.util.Comparator;
|
|
|
+import java.util.Deque;
|
|
|
+import java.util.LinkedList;
|
|
|
+import java.util.List;
|
|
|
+import java.util.NavigableSet;
|
|
|
+import java.util.Random;
|
|
|
+import java.util.StringTokenizer;
|
|
|
+import java.util.TreeSet;
|
|
|
+
|
|
|
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType.MARK_CONTAINER_FOR_KILLABLE;
|
|
|
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType.MARK_CONTAINER_FOR_PREEMPTION;
|
|
|
+import static org.junit.Assert.assertEquals;
|
|
|
+import static org.junit.Assert.assertNotNull;
|
|
|
+import static org.junit.Assert.assertTrue;
|
|
|
+import static org.junit.Assert.fail;
|
|
|
+import static org.mockito.Matchers.any;
|
|
|
+import static org.mockito.Matchers.anyString;
|
|
|
+import static org.mockito.Matchers.argThat;
|
|
|
+import static org.mockito.Matchers.isA;
|
|
|
+import static org.mockito.Mockito.mock;
|
|
|
+import static org.mockito.Mockito.never;
|
|
|
+import static org.mockito.Mockito.times;
|
|
|
+import static org.mockito.Mockito.verify;
|
|
|
+import static org.mockito.Mockito.when;
|
|
|
+
|
|
|
public class TestProportionalCapacityPreemptionPolicy {
|
|
|
|
|
|
static final long TS = 3141592653L;
|
|
@@ -105,11 +98,10 @@ public class TestProportionalCapacityPreemptionPolicy {
|
|
|
float setAMResourcePercent = 0.0f;
|
|
|
Random rand = null;
|
|
|
Clock mClock = null;
|
|
|
- Configuration conf = null;
|
|
|
+ CapacitySchedulerConfiguration conf = null;
|
|
|
CapacityScheduler mCS = null;
|
|
|
RMContext rmContext = null;
|
|
|
RMNodeLabelsManager lm = null;
|
|
|
- CapacitySchedulerConfiguration schedConf = null;
|
|
|
EventHandler<SchedulerEvent> mDisp = null;
|
|
|
ResourceCalculator rc = new DefaultResourceCalculator();
|
|
|
Resource clusterResources = null;
|
|
@@ -132,7 +124,7 @@ public class TestProportionalCapacityPreemptionPolicy {
|
|
|
AMCONTAINER(0), CONTAINER(1), LABELEDCONTAINER(2);
|
|
|
int value;
|
|
|
|
|
|
- private priority(int value) {
|
|
|
+ priority(int value) {
|
|
|
this.value = value;
|
|
|
}
|
|
|
|
|
@@ -146,12 +138,17 @@ public class TestProportionalCapacityPreemptionPolicy {
|
|
|
@Before
|
|
|
@SuppressWarnings("unchecked")
|
|
|
public void setup() {
|
|
|
- conf = new Configuration(false);
|
|
|
- conf.setLong(WAIT_TIME_BEFORE_KILL, 10000);
|
|
|
- conf.setLong(MONITORING_INTERVAL, 3000);
|
|
|
+ conf = new CapacitySchedulerConfiguration(new Configuration(false));
|
|
|
+ conf.setLong(
|
|
|
+ CapacitySchedulerConfiguration.PREEMPTION_WAIT_TIME_BEFORE_KILL, 10000);
|
|
|
+ conf.setLong(CapacitySchedulerConfiguration.PREEMPTION_MONITORING_INTERVAL,
|
|
|
+ 3000);
|
|
|
// report "ideal" preempt
|
|
|
- conf.setFloat(TOTAL_PREEMPTION_PER_ROUND, (float) 1.0);
|
|
|
- conf.setFloat(NATURAL_TERMINATION_FACTOR, (float) 1.0);
|
|
|
+ conf.setFloat(CapacitySchedulerConfiguration.TOTAL_PREEMPTION_PER_ROUND,
|
|
|
+ 1.0f);
|
|
|
+ conf.setFloat(
|
|
|
+ CapacitySchedulerConfiguration.PREEMPTION_NATURAL_TERMINATION_FACTOR,
|
|
|
+ 1.0f);
|
|
|
conf.set(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES,
|
|
|
ProportionalCapacityPreemptionPolicy.class.getCanonicalName());
|
|
|
conf.setBoolean(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS, true);
|
|
@@ -164,8 +161,7 @@ public class TestProportionalCapacityPreemptionPolicy {
|
|
|
mCS = mock(CapacityScheduler.class);
|
|
|
when(mCS.getResourceCalculator()).thenReturn(rc);
|
|
|
lm = mock(RMNodeLabelsManager.class);
|
|
|
- schedConf = new CapacitySchedulerConfiguration();
|
|
|
- when(mCS.getConfiguration()).thenReturn(schedConf);
|
|
|
+ when(mCS.getConfiguration()).thenReturn(conf);
|
|
|
rmContext = mock(RMContext.class);
|
|
|
when(mCS.getRMContext()).thenReturn(rmContext);
|
|
|
when(mCS.getPreemptionManager()).thenReturn(new PreemptionManager());
|
|
@@ -271,7 +267,9 @@ public class TestProportionalCapacityPreemptionPolicy {
|
|
|
{ -1, 1, 1, 1 }, // req granularity
|
|
|
{ 3, 0, 0, 0 }, // subqueues
|
|
|
};
|
|
|
- conf.setLong(WAIT_TIME_BEFORE_KILL, killTime);
|
|
|
+ conf.setLong(
|
|
|
+ CapacitySchedulerConfiguration.PREEMPTION_WAIT_TIME_BEFORE_KILL,
|
|
|
+ killTime);
|
|
|
ProportionalCapacityPreemptionPolicy policy = buildPolicy(qData);
|
|
|
|
|
|
// ensure all pending rsrc from A get preempted from other queues
|
|
@@ -308,7 +306,9 @@ public class TestProportionalCapacityPreemptionPolicy {
|
|
|
{ -1, 1, 1, 1 }, // req granularity
|
|
|
{ 3, 0, 0, 0 }, // subqueues
|
|
|
};
|
|
|
- conf.setFloat(MAX_IGNORED_OVER_CAPACITY, (float) 0.1);
|
|
|
+ conf.setFloat(
|
|
|
+ CapacitySchedulerConfiguration.PREEMPTION_MAX_IGNORED_OVER_CAPACITY,
|
|
|
+ (float) 0.1);
|
|
|
ProportionalCapacityPreemptionPolicy policy = buildPolicy(qData);
|
|
|
policy.editSchedule();
|
|
|
// ignore 10% overcapacity to avoid jitter
|
|
@@ -330,7 +330,7 @@ public class TestProportionalCapacityPreemptionPolicy {
|
|
|
{ 3, 0, 0, 0 }, // subqueues
|
|
|
};
|
|
|
|
|
|
- schedConf.setPreemptionDisabled("root.queueB", true);
|
|
|
+ conf.setPreemptionDisabled("root.queueB", true);
|
|
|
|
|
|
ProportionalCapacityPreemptionPolicy policy = buildPolicy(qData);
|
|
|
policy.editSchedule();
|
|
@@ -343,7 +343,7 @@ public class TestProportionalCapacityPreemptionPolicy {
|
|
|
// event handler will count only events from the following test and not the
|
|
|
// previous one.
|
|
|
setup();
|
|
|
- schedConf.setPreemptionDisabled("root.queueB", false);
|
|
|
+ conf.setPreemptionDisabled("root.queueB", false);
|
|
|
ProportionalCapacityPreemptionPolicy policy2 = buildPolicy(qData);
|
|
|
|
|
|
policy2.editSchedule();
|
|
@@ -382,7 +382,7 @@ public class TestProportionalCapacityPreemptionPolicy {
|
|
|
// Need to call setup() again to reset mDisp
|
|
|
setup();
|
|
|
// Turn off preemption for queueB and it's children
|
|
|
- schedConf.setPreemptionDisabled("root.queueA.queueB", true);
|
|
|
+ conf.setPreemptionDisabled("root.queueA.queueB", true);
|
|
|
ProportionalCapacityPreemptionPolicy policy2 = buildPolicy(qData);
|
|
|
policy2.editSchedule();
|
|
|
ApplicationAttemptId expectedAttemptOnQueueC =
|
|
@@ -429,7 +429,7 @@ public class TestProportionalCapacityPreemptionPolicy {
|
|
|
// Need to call setup() again to reset mDisp
|
|
|
setup();
|
|
|
// Turn off preemption for queueB(appA)
|
|
|
- schedConf.setPreemptionDisabled("root.queueA.queueB", true);
|
|
|
+ conf.setPreemptionDisabled("root.queueA.queueB", true);
|
|
|
ProportionalCapacityPreemptionPolicy policy2 = buildPolicy(qData);
|
|
|
policy2.editSchedule();
|
|
|
// Now that queueB(appA) is not preemptable, verify that resources come
|
|
@@ -439,8 +439,8 @@ public class TestProportionalCapacityPreemptionPolicy {
|
|
|
|
|
|
setup();
|
|
|
// Turn off preemption for two of the 3 queues with over-capacity.
|
|
|
- schedConf.setPreemptionDisabled("root.queueD.queueE", true);
|
|
|
- schedConf.setPreemptionDisabled("root.queueA.queueB", true);
|
|
|
+ conf.setPreemptionDisabled("root.queueD.queueE", true);
|
|
|
+ conf.setPreemptionDisabled("root.queueA.queueB", true);
|
|
|
ProportionalCapacityPreemptionPolicy policy3 = buildPolicy(qData);
|
|
|
policy3.editSchedule();
|
|
|
|
|
@@ -481,7 +481,7 @@ public class TestProportionalCapacityPreemptionPolicy {
|
|
|
// Turn off preemption for queueA and it's children. queueF(appC)'s request
|
|
|
// should starve.
|
|
|
setup(); // Call setup() to reset mDisp
|
|
|
- schedConf.setPreemptionDisabled("root.queueA", true);
|
|
|
+ conf.setPreemptionDisabled("root.queueA", true);
|
|
|
ProportionalCapacityPreemptionPolicy policy2 = buildPolicy(qData);
|
|
|
policy2.editSchedule();
|
|
|
verify(mDisp, never()).handle(argThat(new IsPreemptionRequestFor(appA))); // queueC
|
|
@@ -505,7 +505,7 @@ public class TestProportionalCapacityPreemptionPolicy {
|
|
|
{ -1, -1, 1, 1, 1, -1, 1, 1, 1 }, // req granularity
|
|
|
{ 2, 3, 0, 0, 0, 3, 0, 0, 0 }, // subqueues
|
|
|
};
|
|
|
- schedConf.setPreemptionDisabled("root.queueA.queueC", true);
|
|
|
+ conf.setPreemptionDisabled("root.queueA.queueC", true);
|
|
|
ProportionalCapacityPreemptionPolicy policy = buildPolicy(qData);
|
|
|
policy.editSchedule();
|
|
|
// Although queueC(appB) is way over capacity and is untouchable,
|
|
@@ -529,7 +529,7 @@ public class TestProportionalCapacityPreemptionPolicy {
|
|
|
{ 3, 2, 0, 0, 2, 0, 0, 2, 0, 0 }, // subqueues
|
|
|
};
|
|
|
|
|
|
- schedConf.setPreemptionDisabled("root", true);
|
|
|
+ conf.setPreemptionDisabled("root", true);
|
|
|
ProportionalCapacityPreemptionPolicy policy = buildPolicy(qData);
|
|
|
policy.editSchedule();
|
|
|
// All queues should be non-preemptable, so request should starve.
|
|
@@ -556,7 +556,7 @@ public class TestProportionalCapacityPreemptionPolicy {
|
|
|
{ 2, 2, 0, 0, 2, 0, 0 }, // subqueues
|
|
|
};
|
|
|
// QueueE inherits non-preemption from QueueD
|
|
|
- schedConf.setPreemptionDisabled("root.queueD", true);
|
|
|
+ conf.setPreemptionDisabled("root.queueD", true);
|
|
|
ProportionalCapacityPreemptionPolicy policy = buildPolicy(qData);
|
|
|
policy.editSchedule();
|
|
|
// appC is running on QueueE. QueueE is over absMaxCap, but is not
|
|
@@ -596,7 +596,10 @@ public class TestProportionalCapacityPreemptionPolicy {
|
|
|
{ -1, 1, 1, 0 }, // req granularity
|
|
|
{ 3, 0, 0, 0 }, // subqueues
|
|
|
};
|
|
|
- conf.setFloat(NATURAL_TERMINATION_FACTOR, (float) 0.1);
|
|
|
+ conf.setFloat(
|
|
|
+ CapacitySchedulerConfiguration.PREEMPTION_NATURAL_TERMINATION_FACTOR,
|
|
|
+ (float) 0.1);
|
|
|
+
|
|
|
ProportionalCapacityPreemptionPolicy policy = buildPolicy(qData);
|
|
|
policy.editSchedule();
|
|
|
// ignore 10% imbalance between over-capacity queues
|
|
@@ -616,7 +619,10 @@ public class TestProportionalCapacityPreemptionPolicy {
|
|
|
{ -1, 1, 1, 0 }, // req granularity
|
|
|
{ 3, 0, 0, 0 }, // subqueues
|
|
|
};
|
|
|
- conf.setBoolean(OBSERVE_ONLY, true);
|
|
|
+ conf.setBoolean(CapacitySchedulerConfiguration.PREEMPTION_OBSERVE_ONLY,
|
|
|
+ true);
|
|
|
+ when(mCS.getConfiguration()).thenReturn(
|
|
|
+ new CapacitySchedulerConfiguration(conf));
|
|
|
ProportionalCapacityPreemptionPolicy policy = buildPolicy(qData);
|
|
|
policy.editSchedule();
|
|
|
// verify even severe imbalance not affected
|
|
@@ -735,7 +741,7 @@ public class TestProportionalCapacityPreemptionPolicy {
|
|
|
containers.add(rm4);
|
|
|
|
|
|
// sort them
|
|
|
- ProportionalCapacityPreemptionPolicy.sortContainers(containers);
|
|
|
+ FifoCandidatesSelector.sortContainers(containers);
|
|
|
|
|
|
// verify the "priority"-first, "reverse container-id"-second
|
|
|
// ordering is enforced correctly
|
|
@@ -957,7 +963,7 @@ public class TestProportionalCapacityPreemptionPolicy {
|
|
|
|
|
|
ProportionalCapacityPreemptionPolicy buildPolicy(int[][] qData) {
|
|
|
ProportionalCapacityPreemptionPolicy policy = new ProportionalCapacityPreemptionPolicy(
|
|
|
- conf, rmContext, mCS, mClock);
|
|
|
+ rmContext, mCS, mClock);
|
|
|
clusterResources = Resource.newInstance(
|
|
|
leafAbsCapacities(qData[0], qData[7]), 0);
|
|
|
ParentQueue mRoot = buildMockRootQueue(rand, qData);
|
|
@@ -967,11 +973,6 @@ public class TestProportionalCapacityPreemptionPolicy {
|
|
|
return policy;
|
|
|
}
|
|
|
|
|
|
- ProportionalCapacityPreemptionPolicy buildPolicy(int[][] qData,
|
|
|
- String[][] resData) {
|
|
|
- return buildPolicy(qData, resData, false);
|
|
|
- }
|
|
|
-
|
|
|
ProportionalCapacityPreemptionPolicy buildPolicy(int[][] qData,
|
|
|
String[][] resData, boolean useDominantResourceCalculator) {
|
|
|
if (useDominantResourceCalculator) {
|
|
@@ -979,7 +980,7 @@ public class TestProportionalCapacityPreemptionPolicy {
|
|
|
new DominantResourceCalculator());
|
|
|
}
|
|
|
ProportionalCapacityPreemptionPolicy policy =
|
|
|
- new ProportionalCapacityPreemptionPolicy(conf, rmContext, mCS, mClock);
|
|
|
+ new ProportionalCapacityPreemptionPolicy(rmContext, mCS, mClock);
|
|
|
clusterResources = leafAbsCapacities(parseResourceDetails(resData[0]),
|
|
|
qData[2]);
|
|
|
ParentQueue mRoot = buildMockRootQueue(rand, resData, qData);
|
|
@@ -1124,7 +1125,7 @@ public class TestProportionalCapacityPreemptionPolicy {
|
|
|
String qName = "";
|
|
|
while(tokenizer.hasMoreTokens()) {
|
|
|
qName += tokenizer.nextToken();
|
|
|
- preemptionDisabled = schedConf.getPreemptionDisabled(qName, preemptionDisabled);
|
|
|
+ preemptionDisabled = conf.getPreemptionDisabled(qName, preemptionDisabled);
|
|
|
qName += ".";
|
|
|
}
|
|
|
return preemptionDisabled;
|