|
@@ -55,7 +55,6 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
|
|
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
|
|
-import org.apache.hadoop.yarn.api.records.Priority;
|
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
|
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPBImpl;
|
|
@@ -63,8 +62,6 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
|
-import org.apache.hadoop.yarn.factories.RecordFactory;
|
|
|
-import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
@@ -94,7 +91,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateS
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FifoPolicy;
|
|
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
|
-import org.apache.hadoop.yarn.util.Clock;
|
|
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
|
|
import org.junit.After;
|
|
|
import org.junit.Assert;
|
|
@@ -105,46 +101,14 @@ import org.xml.sax.SAXException;
|
|
|
import com.google.common.collect.Sets;
|
|
|
|
|
|
@SuppressWarnings("unchecked")
|
|
|
-public class TestFairScheduler {
|
|
|
+public class TestFairScheduler extends FairSchedulerTestBase {
|
|
|
+ private final static String ALLOC_FILE =
|
|
|
+ new File(TEST_DIR, "test-queues").getAbsolutePath();
|
|
|
|
|
|
- static class MockClock implements Clock {
|
|
|
- private long time = 0;
|
|
|
- @Override
|
|
|
- public long getTime() {
|
|
|
- return time;
|
|
|
- }
|
|
|
-
|
|
|
- public void tick(int seconds) {
|
|
|
- time = time + seconds * 1000;
|
|
|
- }
|
|
|
-
|
|
|
- }
|
|
|
-
|
|
|
- final static String TEST_DIR = new File(System.getProperty("test.build.data",
|
|
|
- "/tmp")).getAbsolutePath();
|
|
|
-
|
|
|
- final static String ALLOC_FILE = new File(TEST_DIR,
|
|
|
- "test-queues").getAbsolutePath();
|
|
|
-
|
|
|
- private FairScheduler scheduler;
|
|
|
- private ResourceManager resourceManager;
|
|
|
- private Configuration conf;
|
|
|
- private static RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
|
|
|
-
|
|
|
- private int APP_ID = 1; // Incrementing counter for schedling apps
|
|
|
- private int ATTEMPT_ID = 1; // Incrementing counter for scheduling attempts
|
|
|
-
|
|
|
- // HELPER METHODS
|
|
|
@Before
|
|
|
public void setUp() throws IOException {
|
|
|
scheduler = new FairScheduler();
|
|
|
conf = createConfiguration();
|
|
|
- conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 0);
|
|
|
- conf.setInt(FairSchedulerConfiguration.RM_SCHEDULER_INCREMENT_ALLOCATION_MB,
|
|
|
- 1024);
|
|
|
- conf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, 10240);
|
|
|
- // All tests assume only one assignment per node update
|
|
|
- conf.set(FairSchedulerConfiguration.ASSIGN_MULTIPLE, "false");
|
|
|
resourceManager = new ResourceManager();
|
|
|
resourceManager.init(conf);
|
|
|
|
|
@@ -198,107 +162,6 @@ public class TestFairScheduler {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private Configuration createConfiguration() {
|
|
|
- Configuration conf = new YarnConfiguration();
|
|
|
- conf.setClass(YarnConfiguration.RM_SCHEDULER, FairScheduler.class,
|
|
|
- ResourceScheduler.class);
|
|
|
- return conf;
|
|
|
- }
|
|
|
-
|
|
|
- private ApplicationAttemptId createAppAttemptId(int appId, int attemptId) {
|
|
|
- ApplicationId appIdImpl = ApplicationId.newInstance(0, appId);
|
|
|
- ApplicationAttemptId attId =
|
|
|
- ApplicationAttemptId.newInstance(appIdImpl, attemptId);
|
|
|
- return attId;
|
|
|
- }
|
|
|
-
|
|
|
- private ResourceRequest createResourceRequest(int memory, String host,
|
|
|
- int priority, int numContainers, boolean relaxLocality) {
|
|
|
- return createResourceRequest(memory, 1, host, priority, numContainers,
|
|
|
- relaxLocality);
|
|
|
- }
|
|
|
-
|
|
|
- private ResourceRequest createResourceRequest(int memory, int vcores, String host,
|
|
|
- int priority, int numContainers, boolean relaxLocality) {
|
|
|
- ResourceRequest request = recordFactory.newRecordInstance(ResourceRequest.class);
|
|
|
- request.setCapability(BuilderUtils.newResource(memory, vcores));
|
|
|
- request.setResourceName(host);
|
|
|
- request.setNumContainers(numContainers);
|
|
|
- Priority prio = recordFactory.newRecordInstance(Priority.class);
|
|
|
- prio.setPriority(priority);
|
|
|
- request.setPriority(prio);
|
|
|
- request.setRelaxLocality(relaxLocality);
|
|
|
- return request;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Creates a single container priority-1 request and submits to
|
|
|
- * scheduler.
|
|
|
- */
|
|
|
- private ApplicationAttemptId createSchedulingRequest(int memory, String queueId,
|
|
|
- String userId) {
|
|
|
- return createSchedulingRequest(memory, queueId, userId, 1);
|
|
|
- }
|
|
|
-
|
|
|
- private ApplicationAttemptId createSchedulingRequest(int memory, int vcores,
|
|
|
- String queueId, String userId) {
|
|
|
- return createSchedulingRequest(memory, vcores, queueId, userId, 1);
|
|
|
- }
|
|
|
-
|
|
|
- private ApplicationAttemptId createSchedulingRequest(int memory, String queueId,
|
|
|
- String userId, int numContainers) {
|
|
|
- return createSchedulingRequest(memory, queueId, userId, numContainers, 1);
|
|
|
- }
|
|
|
-
|
|
|
- private ApplicationAttemptId createSchedulingRequest(int memory, int vcores,
|
|
|
- String queueId, String userId, int numContainers) {
|
|
|
- return createSchedulingRequest(memory, vcores, queueId, userId, numContainers, 1);
|
|
|
- }
|
|
|
-
|
|
|
- private ApplicationAttemptId createSchedulingRequest(int memory, String queueId,
|
|
|
- String userId, int numContainers, int priority) {
|
|
|
- return createSchedulingRequest(memory, 1, queueId, userId, numContainers,
|
|
|
- priority);
|
|
|
- }
|
|
|
-
|
|
|
- private ApplicationAttemptId createSchedulingRequest(int memory, int vcores,
|
|
|
- String queueId, String userId, int numContainers, int priority) {
|
|
|
- ApplicationAttemptId id = createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++);
|
|
|
- scheduler.addApplication(id.getApplicationId(), queueId, userId);
|
|
|
- // This conditional is for testAclSubmitApplication where app is rejected
|
|
|
- // and no app is added.
|
|
|
- if (scheduler.getSchedulerApplications().containsKey(id.getApplicationId())) {
|
|
|
- scheduler.addApplicationAttempt(id, false);
|
|
|
- }
|
|
|
- List<ResourceRequest> ask = new ArrayList<ResourceRequest>();
|
|
|
- ResourceRequest request = createResourceRequest(memory, vcores, ResourceRequest.ANY,
|
|
|
- priority, numContainers, true);
|
|
|
- ask.add(request);
|
|
|
- scheduler.allocate(id, ask, new ArrayList<ContainerId>(), null, null);
|
|
|
- return id;
|
|
|
- }
|
|
|
-
|
|
|
- private void createSchedulingRequestExistingApplication(int memory, int priority,
|
|
|
- ApplicationAttemptId attId) {
|
|
|
- ResourceRequest request = createResourceRequest(memory, ResourceRequest.ANY,
|
|
|
- priority, 1, true);
|
|
|
- createSchedulingRequestExistingApplication(request, attId);
|
|
|
- }
|
|
|
-
|
|
|
- private void createSchedulingRequestExistingApplication(int memory, int vcores,
|
|
|
- int priority, ApplicationAttemptId attId) {
|
|
|
- ResourceRequest request = createResourceRequest(memory, vcores, ResourceRequest.ANY,
|
|
|
- priority, 1, true);
|
|
|
- createSchedulingRequestExistingApplication(request, attId);
|
|
|
- }
|
|
|
-
|
|
|
- private void createSchedulingRequestExistingApplication(ResourceRequest request,
|
|
|
- ApplicationAttemptId attId) {
|
|
|
- List<ResourceRequest> ask = new ArrayList<ResourceRequest>();
|
|
|
- ask.add(request);
|
|
|
- scheduler.allocate(attId, ask, new ArrayList<ContainerId>(), null, null);
|
|
|
- }
|
|
|
-
|
|
|
// TESTS
|
|
|
|
|
|
@Test(timeout=2000)
|
|
@@ -1455,7 +1318,7 @@ public class TestFairScheduler {
|
|
|
assertEquals(
|
|
|
1536, scheduler.resToPreempt(schedD, clock.getTime()).getMemory());
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Test (timeout = 5000)
|
|
|
public void testMultipleContainersWaitingForReservation() throws IOException {
|
|
|
scheduler.reinitialize(conf, resourceManager.getRMContext());
|