|
@@ -23,6 +23,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
|
|
|
|
+import org.apache.hadoop.yarn.util.ControlledClock;
|
|
import org.junit.After;
|
|
import org.junit.After;
|
|
import static org.junit.Assert.assertEquals;
|
|
import static org.junit.Assert.assertEquals;
|
|
import static org.junit.Assert.assertFalse;
|
|
import static org.junit.Assert.assertFalse;
|
|
@@ -49,6 +50,9 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
|
|
private static final File ALLOC_FILE = new File(TEST_DIR, "test-queues");
|
|
private static final File ALLOC_FILE = new File(TEST_DIR, "test-queues");
|
|
private static final int GB = 1024;
|
|
private static final int GB = 1024;
|
|
|
|
|
|
|
|
+ // Scheduler clock
|
|
|
|
+ private final ControlledClock clock = new ControlledClock();
|
|
|
|
+
|
|
// Node Capacity = NODE_CAPACITY_MULTIPLE * (1 GB or 1 vcore)
|
|
// Node Capacity = NODE_CAPACITY_MULTIPLE * (1 GB or 1 vcore)
|
|
private static final int NODE_CAPACITY_MULTIPLE = 4;
|
|
private static final int NODE_CAPACITY_MULTIPLE = 4;
|
|
|
|
|
|
@@ -60,25 +64,28 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
|
|
// Starving app that is expected to instigate preemption
|
|
// Starving app that is expected to instigate preemption
|
|
private FSAppAttempt starvingApp;
|
|
private FSAppAttempt starvingApp;
|
|
|
|
|
|
- @Parameterized.Parameters
|
|
|
|
- public static Collection<Boolean[]> getParameters() {
|
|
|
|
- return Arrays.asList(new Boolean[][] {
|
|
|
|
- {true}, {false}});
|
|
|
|
|
|
+ @Parameterized.Parameters(name = "{0}")
|
|
|
|
+ public static Collection<Object[]> getParameters() {
|
|
|
|
+ return Arrays.asList(new Object[][] {
|
|
|
|
+ {"FairSharePreemption", true},
|
|
|
|
+ {"MinSharePreemption", false}});
|
|
}
|
|
}
|
|
|
|
|
|
- public TestFairSchedulerPreemption(Boolean fairshare) throws IOException {
|
|
|
|
|
|
+ public TestFairSchedulerPreemption(String name, boolean fairshare)
|
|
|
|
+ throws IOException {
|
|
fairsharePreemption = fairshare;
|
|
fairsharePreemption = fairshare;
|
|
writeAllocFile();
|
|
writeAllocFile();
|
|
}
|
|
}
|
|
|
|
|
|
@Before
|
|
@Before
|
|
- public void setup() {
|
|
|
|
|
|
+ public void setup() throws IOException {
|
|
createConfiguration();
|
|
createConfiguration();
|
|
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE,
|
|
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE,
|
|
ALLOC_FILE.getAbsolutePath());
|
|
ALLOC_FILE.getAbsolutePath());
|
|
conf.setBoolean(FairSchedulerConfiguration.PREEMPTION, true);
|
|
conf.setBoolean(FairSchedulerConfiguration.PREEMPTION, true);
|
|
conf.setFloat(FairSchedulerConfiguration.PREEMPTION_THRESHOLD, 0f);
|
|
conf.setFloat(FairSchedulerConfiguration.PREEMPTION_THRESHOLD, 0f);
|
|
conf.setInt(FairSchedulerConfiguration.WAIT_TIME_BEFORE_KILL, 0);
|
|
conf.setInt(FairSchedulerConfiguration.WAIT_TIME_BEFORE_KILL, 0);
|
|
|
|
+ setupCluster();
|
|
}
|
|
}
|
|
|
|
|
|
@After
|
|
@After
|
|
@@ -166,8 +173,9 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
|
|
|
|
|
|
private void setupCluster() throws IOException {
|
|
private void setupCluster() throws IOException {
|
|
resourceManager = new MockRM(conf);
|
|
resourceManager = new MockRM(conf);
|
|
- resourceManager.start();
|
|
|
|
scheduler = (FairScheduler) resourceManager.getResourceScheduler();
|
|
scheduler = (FairScheduler) resourceManager.getResourceScheduler();
|
|
|
|
+ scheduler.setClock(clock);
|
|
|
|
+ resourceManager.start();
|
|
|
|
|
|
// Create and add two nodes to the cluster
|
|
// Create and add two nodes to the cluster
|
|
addNode(NODE_CAPACITY_MULTIPLE * GB, NODE_CAPACITY_MULTIPLE);
|
|
addNode(NODE_CAPACITY_MULTIPLE * GB, NODE_CAPACITY_MULTIPLE);
|
|
@@ -197,7 +205,7 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
|
|
*
|
|
*
|
|
* @param queueName queue name
|
|
* @param queueName queue name
|
|
*/
|
|
*/
|
|
- private void takeAllResource(String queueName) {
|
|
|
|
|
|
+ private void takeAllResources(String queueName) {
|
|
// Create an app that takes up all the resources on the cluster
|
|
// Create an app that takes up all the resources on the cluster
|
|
ApplicationAttemptId appAttemptId
|
|
ApplicationAttemptId appAttemptId
|
|
= createSchedulingRequest(GB, 1, queueName, "default",
|
|
= createSchedulingRequest(GB, 1, queueName, "default",
|
|
@@ -227,8 +235,8 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
|
|
NODE_CAPACITY_MULTIPLE * rmNodes.size() / 2);
|
|
NODE_CAPACITY_MULTIPLE * rmNodes.size() / 2);
|
|
starvingApp = scheduler.getSchedulerApp(appAttemptId);
|
|
starvingApp = scheduler.getSchedulerApp(appAttemptId);
|
|
|
|
|
|
- // Sleep long enough to pass
|
|
|
|
- Thread.sleep(10);
|
|
|
|
|
|
+ // Move clock enough to identify starvation
|
|
|
|
+ clock.tickSec(1);
|
|
scheduler.update();
|
|
scheduler.update();
|
|
}
|
|
}
|
|
|
|
|
|
@@ -243,14 +251,13 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
|
|
*/
|
|
*/
|
|
private void submitApps(String queue1, String queue2)
|
|
private void submitApps(String queue1, String queue2)
|
|
throws InterruptedException {
|
|
throws InterruptedException {
|
|
- takeAllResource(queue1);
|
|
|
|
|
|
+ takeAllResources(queue1);
|
|
preemptHalfResources(queue2);
|
|
preemptHalfResources(queue2);
|
|
}
|
|
}
|
|
|
|
|
|
private void verifyPreemption() throws InterruptedException {
|
|
private void verifyPreemption() throws InterruptedException {
|
|
- // Sleep long enough for four containers to be preempted. Note that the
|
|
|
|
- // starved app must be queued four times for containers to be preempted.
|
|
|
|
- for (int i = 0; i < 10000; i++) {
|
|
|
|
|
|
+ // Sleep long enough for four containers to be preempted.
|
|
|
|
+ for (int i = 0; i < 100; i++) {
|
|
if (greedyApp.getLiveContainers().size() == 4) {
|
|
if (greedyApp.getLiveContainers().size() == 4) {
|
|
break;
|
|
break;
|
|
}
|
|
}
|
|
@@ -268,7 +275,7 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
|
|
|
|
|
|
private void verifyNoPreemption() throws InterruptedException {
|
|
private void verifyNoPreemption() throws InterruptedException {
|
|
// Sleep long enough to ensure not even one container is preempted.
|
|
// Sleep long enough to ensure not even one container is preempted.
|
|
- for (int i = 0; i < 600; i++) {
|
|
|
|
|
|
+ for (int i = 0; i < 100; i++) {
|
|
if (greedyApp.getLiveContainers().size() != 8) {
|
|
if (greedyApp.getLiveContainers().size() != 8) {
|
|
break;
|
|
break;
|
|
}
|
|
}
|
|
@@ -279,7 +286,6 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
|
|
|
|
|
|
@Test
|
|
@Test
|
|
public void testPreemptionWithinSameLeafQueue() throws Exception {
|
|
public void testPreemptionWithinSameLeafQueue() throws Exception {
|
|
- setupCluster();
|
|
|
|
String queue = "root.preemptable.child-1";
|
|
String queue = "root.preemptable.child-1";
|
|
submitApps(queue, queue);
|
|
submitApps(queue, queue);
|
|
if (fairsharePreemption) {
|
|
if (fairsharePreemption) {
|
|
@@ -291,21 +297,18 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
|
|
|
|
|
|
@Test
|
|
@Test
|
|
public void testPreemptionBetweenTwoSiblingLeafQueues() throws Exception {
|
|
public void testPreemptionBetweenTwoSiblingLeafQueues() throws Exception {
|
|
- setupCluster();
|
|
|
|
submitApps("root.preemptable.child-1", "root.preemptable.child-2");
|
|
submitApps("root.preemptable.child-1", "root.preemptable.child-2");
|
|
verifyPreemption();
|
|
verifyPreemption();
|
|
}
|
|
}
|
|
|
|
|
|
@Test
|
|
@Test
|
|
public void testPreemptionBetweenNonSiblingQueues() throws Exception {
|
|
public void testPreemptionBetweenNonSiblingQueues() throws Exception {
|
|
- setupCluster();
|
|
|
|
submitApps("root.preemptable.child-1", "root.nonpreemptable.child-1");
|
|
submitApps("root.preemptable.child-1", "root.nonpreemptable.child-1");
|
|
verifyPreemption();
|
|
verifyPreemption();
|
|
}
|
|
}
|
|
|
|
|
|
@Test
|
|
@Test
|
|
public void testNoPreemptionFromDisallowedQueue() throws Exception {
|
|
public void testNoPreemptionFromDisallowedQueue() throws Exception {
|
|
- setupCluster();
|
|
|
|
submitApps("root.nonpreemptable.child-1", "root.preemptable.child-1");
|
|
submitApps("root.nonpreemptable.child-1", "root.preemptable.child-1");
|
|
verifyNoPreemption();
|
|
verifyNoPreemption();
|
|
}
|
|
}
|
|
@@ -331,9 +334,7 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
|
|
|
|
|
|
@Test
|
|
@Test
|
|
public void testPreemptionSelectNonAMContainer() throws Exception {
|
|
public void testPreemptionSelectNonAMContainer() throws Exception {
|
|
- setupCluster();
|
|
|
|
-
|
|
|
|
- takeAllResource("root.preemptable.child-1");
|
|
|
|
|
|
+ takeAllResources("root.preemptable.child-1");
|
|
setNumAMContainersPerNode(2);
|
|
setNumAMContainersPerNode(2);
|
|
preemptHalfResources("root.preemptable.child-2");
|
|
preemptHalfResources("root.preemptable.child-2");
|
|
|
|
|