Skip to content

Commit

Permalink
YARN-5308. FairScheduler: Move continuous scheduling related tests to…
Browse files Browse the repository at this point in the history
… TestContinuousScheduling (Kai Sasaki via Varun Saxena)
  • Loading branch information
varunsaxena committed Oct 27, 2016
1 parent ac35ee9 commit 79aeddc
Show file tree
Hide file tree
Showing 2 changed files with 187 additions and 159 deletions.
Expand Up @@ -22,39 +22,55 @@
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.server.resourcemanager.MockNodes; import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
import org.apache.hadoop.yarn.server.resourcemanager.MockRM; import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
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.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;


import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
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.util.ControlledClock; import org.apache.hadoop.yarn.util.ControlledClock;
import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.util.resource.Resources;
import org.junit.After; import org.junit.After;
import org.junit.Assert; import org.junit.Assert;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.isA;
import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.spy;

import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;


import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashSet; import java.util.HashSet;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.Set; import java.util.Set;


public class TestContinuousScheduling extends FairSchedulerTestBase { public class TestContinuousScheduling extends FairSchedulerTestBase {
private ControlledClock mockClock; private ControlledClock mockClock;
private static int delayThresholdTimeMs = 1000;


@Override @Override
public Configuration createConfiguration() { public Configuration createConfiguration() {
Configuration conf = super.createConfiguration(); Configuration conf = super.createConfiguration();
conf.setBoolean( conf.setBoolean(
FairSchedulerConfiguration.CONTINUOUS_SCHEDULING_ENABLED, true); FairSchedulerConfiguration.CONTINUOUS_SCHEDULING_ENABLED, true);
conf.setInt(FairSchedulerConfiguration.LOCALITY_DELAY_NODE_MS, 100); conf.setInt(FairSchedulerConfiguration.LOCALITY_DELAY_NODE_MS,
conf.setInt(FairSchedulerConfiguration.LOCALITY_DELAY_RACK_MS, 100); delayThresholdTimeMs);
conf.setInt(FairSchedulerConfiguration.LOCALITY_DELAY_RACK_MS,
delayThresholdTimeMs);
return conf; return conf;
} }


Expand Down Expand Up @@ -167,6 +183,175 @@ public void testSortedNodes() throws Exception {
Assert.assertEquals(2, nodes.size()); Assert.assertEquals(2, nodes.size());
} }


@Test
public void testWithNodeRemoved() throws Exception {
// Disable continuous scheduling, will invoke continuous
// scheduling once manually
scheduler = new FairScheduler();
conf = super.createConfiguration();
resourceManager = new MockRM(conf);

// TODO: This test should really be using MockRM. For now starting stuff
// that is needed at a bare minimum.
((AsyncDispatcher)resourceManager.getRMContext().getDispatcher()).start();
resourceManager.getRMContext().getStateStore().start();

// to initialize the master key
resourceManager.getRMContext().getContainerTokenSecretManager()
.rollMasterKey();

scheduler.setRMContext(resourceManager.getRMContext());
Assert.assertTrue("Continuous scheduling should be disabled.",
!scheduler.isContinuousSchedulingEnabled());
scheduler.init(conf);
scheduler.start();

// Add two nodes
RMNode node1 =
MockNodes.newNodeInfo(1, Resources.createResource(8 * 1024, 8), 1,
"127.0.0.1");
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
scheduler.handle(nodeEvent1);
RMNode node2 =
MockNodes.newNodeInfo(1, Resources.createResource(8 * 1024, 8), 2,
"127.0.0.2");
NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2);
scheduler.handle(nodeEvent2);
Assert.assertEquals("We should have two alive nodes.",
2, scheduler.getNumClusterNodes());

// Remove one node
NodeRemovedSchedulerEvent removeNode1
= new NodeRemovedSchedulerEvent(node1);
scheduler.handle(removeNode1);
Assert.assertEquals("We should only have one alive node.",
1, scheduler.getNumClusterNodes());

// Invoke the continuous scheduling once
try {
scheduler.continuousSchedulingAttempt();
} catch (Exception e) {
fail("Exception happened when doing continuous scheduling. " +
e.toString());
}
}

@Test
public void testInterruptedException()
throws Exception {
// Disable continuous scheduling, will invoke continuous
// scheduling once manually
scheduler = new FairScheduler();
conf = super.createConfiguration();
resourceManager = new MockRM(conf);

// TODO: This test should really be using MockRM. For now starting stuff
// that is needed at a bare minimum.
((AsyncDispatcher)resourceManager.getRMContext().getDispatcher()).start();
resourceManager.getRMContext().getStateStore().start();

// to initialize the master key
resourceManager.getRMContext().getContainerTokenSecretManager()
.rollMasterKey();

scheduler.setRMContext(resourceManager.getRMContext());
scheduler.init(conf);
scheduler.start();
FairScheduler spyScheduler = spy(scheduler);
Assert.assertTrue("Continuous scheduling should be disabled.",
!spyScheduler.isContinuousSchedulingEnabled());
// Add one nodes
RMNode node1 =
MockNodes.newNodeInfo(1, Resources.createResource(8 * 1024, 8), 1,
"127.0.0.1");
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
spyScheduler.handle(nodeEvent1);
Assert.assertEquals("We should have one alive node.",
1, spyScheduler.getNumClusterNodes());
InterruptedException ie = new InterruptedException();
doThrow(new YarnRuntimeException(ie)).when(spyScheduler).
attemptScheduling(isA(FSSchedulerNode.class));
// Invoke the continuous scheduling once
try {
spyScheduler.continuousSchedulingAttempt();
fail("Expected InterruptedException to stop schedulingThread");
} catch (InterruptedException e) {
Assert.assertEquals(ie, e);
}
}

@Test
public void testThreadLifeCycle() throws InterruptedException {
scheduler.start();

Thread updateThread = scheduler.updateThread;
Thread schedulingThread = scheduler.schedulingThread;

assertTrue(updateThread.isAlive());
assertTrue(schedulingThread.isAlive());

scheduler.stop();

int numRetries = 100;
while (numRetries-- > 0 &&
(updateThread.isAlive() || schedulingThread.isAlive())) {
Thread.sleep(50);
}

assertNotEquals("One of the threads is still alive", 0, numRetries);
}

@Test
public void testFairSchedulerContinuousSchedulingInitTime() throws Exception {
scheduler.start();

int priorityValue;
Priority priority;
FSAppAttempt fsAppAttempt;
ResourceRequest request1;
ResourceRequest request2;
ApplicationAttemptId id11;

priorityValue = 1;
id11 = createAppAttemptId(1, 1);
createMockRMApp(id11);
priority = Priority.newInstance(priorityValue);
scheduler.addApplication(id11.getApplicationId(), "root.queue1", "user1",
false);
scheduler.addApplicationAttempt(id11, false, false);
fsAppAttempt = scheduler.getApplicationAttempt(id11);

String hostName = "127.0.0.1";
RMNode node1 =
MockNodes.newNodeInfo(1, Resources.createResource(16 * 1024, 16), 1,
hostName);
List<ResourceRequest> ask1 = new ArrayList<>();
request1 =
createResourceRequest(1024, 8, node1.getRackName(), priorityValue, 1,
true);
request2 =
createResourceRequest(1024, 8, ResourceRequest.ANY, priorityValue, 1,
true);
ask1.add(request1);
ask1.add(request2);
scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(), null, null,
null, null);

NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
scheduler.handle(nodeEvent1);
FSSchedulerNode node =
(FSSchedulerNode) scheduler.getSchedulerNode(node1.getNodeID());
// Tick the time and let the fsApp startTime different from initScheduler
// time
mockClock.tickSec(delayThresholdTimeMs / 1000);
scheduler.attemptScheduling(node);
Map<SchedulerRequestKey, Long> lastScheduledContainer =
fsAppAttempt.getLastScheduledContainer();
long initSchedulerTime =
lastScheduledContainer.get(TestUtils.toSchedulerKey(priority));
assertEquals(delayThresholdTimeMs, initSchedulerTime);
}

private void triggerSchedulingAttempt() throws InterruptedException { private void triggerSchedulingAttempt() throws InterruptedException {
Thread.sleep( Thread.sleep(
2 * scheduler.getConf().getContinuousSchedulingSleepMs()); 2 * scheduler.getConf().getContinuousSchedulingSleepMs());
Expand Down

0 comments on commit 79aeddc

Please sign in to comment.