|
@@ -26,6 +26,7 @@ import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.HashSet;
|
|
|
+import java.util.Iterator;
|
|
|
import java.util.List;
|
|
|
import java.util.Set;
|
|
|
|
|
@@ -36,15 +37,20 @@ import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.mapreduce.MRJobConfig;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
|
|
-import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.JobState;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
|
|
|
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
|
|
|
+import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.Job;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.job.Task;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerAssignedEvent;
|
|
|
-import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocator;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerFailedEvent;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerRequestEvent;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator;
|
|
@@ -78,6 +84,7 @@ import org.apache.hadoop.yarn.util.BuilderUtils;
|
|
|
import org.junit.After;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
+@SuppressWarnings("unchecked")
|
|
|
public class TestRMContainerAllocator {
|
|
|
|
|
|
static final Log LOG = LogFactory
|
|
@@ -338,98 +345,155 @@ public class TestRMContainerAllocator {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private static class FakeJob extends JobImpl {
|
|
|
-
|
|
|
- public FakeJob(ApplicationAttemptId appAttemptID, Configuration conf,
|
|
|
- int numMaps, int numReduces) {
|
|
|
- super(MRBuilderUtils.newJobId(appAttemptID.getApplicationId(), 0),
|
|
|
- appAttemptID, conf, null, null, null, null, null, null, null, null,
|
|
|
- true, null, System.currentTimeMillis(), null);
|
|
|
- this.jobId = getID();
|
|
|
- this.numMaps = numMaps;
|
|
|
- this.numReduces = numReduces;
|
|
|
- }
|
|
|
-
|
|
|
- private float setupProgress;
|
|
|
- private float mapProgress;
|
|
|
- private float reduceProgress;
|
|
|
- private float cleanupProgress;
|
|
|
- private final int numMaps;
|
|
|
- private final int numReduces;
|
|
|
- private JobId jobId;
|
|
|
-
|
|
|
- void setProgress(float setupProgress, float mapProgress,
|
|
|
- float reduceProgress, float cleanupProgress) {
|
|
|
- this.setupProgress = setupProgress;
|
|
|
- this.mapProgress = mapProgress;
|
|
|
- this.reduceProgress = reduceProgress;
|
|
|
- this.cleanupProgress = cleanupProgress;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public int getTotalMaps() { return this.numMaps; }
|
|
|
- @Override
|
|
|
- public int getTotalReduces() { return this.numReduces;}
|
|
|
-
|
|
|
- @Override
|
|
|
- public JobReport getReport() {
|
|
|
- return MRBuilderUtils.newJobReport(this.jobId, "job", "user",
|
|
|
- JobState.RUNNING, 0, 0, 0, this.setupProgress, this.mapProgress,
|
|
|
- this.reduceProgress, this.cleanupProgress, "jobfile", null, false);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
@Test
|
|
|
public void testReportedAppProgress() throws Exception {
|
|
|
|
|
|
LOG.info("Running testReportedAppProgress");
|
|
|
|
|
|
Configuration conf = new Configuration();
|
|
|
- MyResourceManager rm = new MyResourceManager(conf);
|
|
|
+ final MyResourceManager rm = new MyResourceManager(conf);
|
|
|
rm.start();
|
|
|
- DrainDispatcher dispatcher = (DrainDispatcher) rm.getRMContext()
|
|
|
+ DrainDispatcher rmDispatcher = (DrainDispatcher) rm.getRMContext()
|
|
|
.getDispatcher();
|
|
|
|
|
|
// Submit the application
|
|
|
- RMApp app = rm.submitApp(1024);
|
|
|
- dispatcher.await();
|
|
|
+ RMApp rmApp = rm.submitApp(1024);
|
|
|
+ rmDispatcher.await();
|
|
|
|
|
|
- MockNM amNodeManager = rm.registerNode("amNM:1234", 2048);
|
|
|
+ MockNM amNodeManager = rm.registerNode("amNM:1234", 21504);
|
|
|
amNodeManager.nodeHeartbeat(true);
|
|
|
- dispatcher.await();
|
|
|
+ rmDispatcher.await();
|
|
|
|
|
|
- ApplicationAttemptId appAttemptId = app.getCurrentAppAttempt()
|
|
|
+ final ApplicationAttemptId appAttemptId = rmApp.getCurrentAppAttempt()
|
|
|
.getAppAttemptId();
|
|
|
rm.sendAMLaunched(appAttemptId);
|
|
|
- dispatcher.await();
|
|
|
+ rmDispatcher.await();
|
|
|
|
|
|
- FakeJob job = new FakeJob(appAttemptId, conf, 2, 2);
|
|
|
- MyContainerAllocator allocator = new MyContainerAllocator(rm, conf,
|
|
|
- appAttemptId, job);
|
|
|
+ MRApp mrApp = new MRApp(appAttemptId, BuilderUtils.newContainerId(
|
|
|
+ appAttemptId, 0), 10, 10, false, this.getClass().getName(), true, 1) {
|
|
|
+ @Override
|
|
|
+ protected Dispatcher createDispatcher() {
|
|
|
+ return new DrainDispatcher();
|
|
|
+ }
|
|
|
+ protected ContainerAllocator createContainerAllocator(
|
|
|
+ ClientService clientService, AppContext context) {
|
|
|
+ return new MyContainerAllocator(rm, appAttemptId, context);
|
|
|
+ };
|
|
|
+ };
|
|
|
+
|
|
|
+ Assert.assertEquals(0.0, rmApp.getProgress(), 0.0);
|
|
|
+
|
|
|
+ mrApp.submit(conf);
|
|
|
+ Job job = mrApp.getContext().getAllJobs().entrySet().iterator().next()
|
|
|
+ .getValue();
|
|
|
+
|
|
|
+ DrainDispatcher amDispatcher = (DrainDispatcher) mrApp.getDispatcher();
|
|
|
+
|
|
|
+ MyContainerAllocator allocator = (MyContainerAllocator) mrApp
|
|
|
+ .getContainerAllocator();
|
|
|
+
|
|
|
+ mrApp.waitForState(job, JobState.RUNNING);
|
|
|
+
|
|
|
+ amDispatcher.await();
|
|
|
+ // Wait till all map-attempts request for containers
|
|
|
+ for (Task t : job.getTasks().values()) {
|
|
|
+ if (t.getType() == TaskType.MAP) {
|
|
|
+ mrApp.waitForState(t.getAttempts().values().iterator().next(),
|
|
|
+ TaskAttemptState.UNASSIGNED);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ amDispatcher.await();
|
|
|
+
|
|
|
+ allocator.schedule();
|
|
|
+ rmDispatcher.await();
|
|
|
+ amNodeManager.nodeHeartbeat(true);
|
|
|
+ rmDispatcher.await();
|
|
|
+ allocator.schedule();
|
|
|
+ rmDispatcher.await();
|
|
|
+
|
|
|
+ // Wait for all map-tasks to be running
|
|
|
+ for (Task t : job.getTasks().values()) {
|
|
|
+ if (t.getType() == TaskType.MAP) {
|
|
|
+ mrApp.waitForState(t, TaskState.RUNNING);
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
allocator.schedule(); // Send heartbeat
|
|
|
- dispatcher.await();
|
|
|
- Assert.assertEquals(0.0, app.getProgress(), 0.0);
|
|
|
+ rmDispatcher.await();
|
|
|
+ Assert.assertEquals(0.05f, job.getProgress(), 0.001f);
|
|
|
+ Assert.assertEquals(0.05f, rmApp.getProgress(), 0.001f);
|
|
|
|
|
|
- job.setProgress(100, 10, 0, 0);
|
|
|
+ // Finish off 1 map.
|
|
|
+ Iterator<Task> it = job.getTasks().values().iterator();
|
|
|
+ finishNextNTasks(mrApp, it, 1);
|
|
|
allocator.schedule();
|
|
|
- dispatcher.await();
|
|
|
- Assert.assertEquals(9.5f, app.getProgress(), 0.0);
|
|
|
+ rmDispatcher.await();
|
|
|
+ Assert.assertEquals(0.095f, job.getProgress(), 0.001f);
|
|
|
+ Assert.assertEquals(0.095f, rmApp.getProgress(), 0.001f);
|
|
|
|
|
|
- job.setProgress(100, 80, 0, 0);
|
|
|
+ // Finish off 7 more so that map-progress is 80%
|
|
|
+ finishNextNTasks(mrApp, it, 7);
|
|
|
allocator.schedule();
|
|
|
- dispatcher.await();
|
|
|
- Assert.assertEquals(41.0f, app.getProgress(), 0.0);
|
|
|
+ rmDispatcher.await();
|
|
|
+ Assert.assertEquals(0.41f, job.getProgress(), 0.001f);
|
|
|
+ Assert.assertEquals(0.41f, rmApp.getProgress(), 0.001f);
|
|
|
+
|
|
|
+ // Finish off the 2 remaining maps
|
|
|
+ finishNextNTasks(mrApp, it, 2);
|
|
|
+
|
|
|
+ // Wait till all reduce-attempts request for containers
|
|
|
+ for (Task t : job.getTasks().values()) {
|
|
|
+ if (t.getType() == TaskType.REDUCE) {
|
|
|
+ mrApp.waitForState(t.getAttempts().values().iterator().next(),
|
|
|
+ TaskAttemptState.UNASSIGNED);
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
- job.setProgress(100, 100, 20, 0);
|
|
|
allocator.schedule();
|
|
|
- dispatcher.await();
|
|
|
- Assert.assertEquals(59.0f, app.getProgress(), 0.0);
|
|
|
+ rmDispatcher.await();
|
|
|
+ amNodeManager.nodeHeartbeat(true);
|
|
|
+ rmDispatcher.await();
|
|
|
+ allocator.schedule();
|
|
|
+ rmDispatcher.await();
|
|
|
+
|
|
|
+ // Wait for all reduce-tasks to be running
|
|
|
+ for (Task t : job.getTasks().values()) {
|
|
|
+ if (t.getType() == TaskType.REDUCE) {
|
|
|
+ mrApp.waitForState(t, TaskState.RUNNING);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // Finish off 2 reduces
|
|
|
+ finishNextNTasks(mrApp, it, 2);
|
|
|
|
|
|
- job.setProgress(100, 100, 100, 100);
|
|
|
allocator.schedule();
|
|
|
- dispatcher.await();
|
|
|
- Assert.assertEquals(100.0f, app.getProgress(), 0.0);
|
|
|
+ rmDispatcher.await();
|
|
|
+ Assert.assertEquals(0.59f, job.getProgress(), 0.001f);
|
|
|
+ Assert.assertEquals(0.59f, rmApp.getProgress(), 0.001f);
|
|
|
+
|
|
|
+ // Finish off the remaining 8 reduces.
|
|
|
+ finishNextNTasks(mrApp, it, 8);
|
|
|
+ allocator.schedule();
|
|
|
+ rmDispatcher.await();
|
|
|
+ // Remaining is JobCleanup
|
|
|
+ Assert.assertEquals(0.95f, job.getProgress(), 0.001f);
|
|
|
+ Assert.assertEquals(0.95f, rmApp.getProgress(), 0.001f);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void finishNextNTasks(MRApp mrApp, Iterator<Task> it, int nextN)
|
|
|
+ throws Exception {
|
|
|
+ Task task;
|
|
|
+ for (int i=0; i<nextN; i++) {
|
|
|
+ task = it.next();
|
|
|
+ finishTask(mrApp, task);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void finishTask(MRApp mrApp, Task task) throws Exception {
|
|
|
+ TaskAttempt attempt = task.getAttempts().values().iterator().next();
|
|
|
+ mrApp.getContext().getEventHandler().handle(
|
|
|
+ new TaskAttemptEvent(attempt.getID(), TaskAttemptEventType.TA_DONE));
|
|
|
+ mrApp.waitForState(task, TaskState.SUCCEEDED);
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -438,46 +502,96 @@ public class TestRMContainerAllocator {
|
|
|
LOG.info("Running testReportedAppProgressWithOnlyMaps");
|
|
|
|
|
|
Configuration conf = new Configuration();
|
|
|
- MyResourceManager rm = new MyResourceManager(conf);
|
|
|
+ final MyResourceManager rm = new MyResourceManager(conf);
|
|
|
rm.start();
|
|
|
- DrainDispatcher dispatcher = (DrainDispatcher) rm.getRMContext()
|
|
|
+ DrainDispatcher rmDispatcher = (DrainDispatcher) rm.getRMContext()
|
|
|
.getDispatcher();
|
|
|
|
|
|
// Submit the application
|
|
|
- RMApp app = rm.submitApp(1024);
|
|
|
- dispatcher.await();
|
|
|
+ RMApp rmApp = rm.submitApp(1024);
|
|
|
+ rmDispatcher.await();
|
|
|
|
|
|
- MockNM amNodeManager = rm.registerNode("amNM:1234", 2048);
|
|
|
+ MockNM amNodeManager = rm.registerNode("amNM:1234", 11264);
|
|
|
amNodeManager.nodeHeartbeat(true);
|
|
|
- dispatcher.await();
|
|
|
+ rmDispatcher.await();
|
|
|
|
|
|
- ApplicationAttemptId appAttemptId = app.getCurrentAppAttempt()
|
|
|
+ final ApplicationAttemptId appAttemptId = rmApp.getCurrentAppAttempt()
|
|
|
.getAppAttemptId();
|
|
|
rm.sendAMLaunched(appAttemptId);
|
|
|
- dispatcher.await();
|
|
|
+ rmDispatcher.await();
|
|
|
|
|
|
- FakeJob job = new FakeJob(appAttemptId, conf, 2, 0);
|
|
|
- MyContainerAllocator allocator = new MyContainerAllocator(rm, conf,
|
|
|
- appAttemptId, job);
|
|
|
+ MRApp mrApp = new MRApp(appAttemptId, BuilderUtils.newContainerId(
|
|
|
+ appAttemptId, 0), 10, 0, false, this.getClass().getName(), true, 1) {
|
|
|
+ @Override
|
|
|
+ protected Dispatcher createDispatcher() {
|
|
|
+ return new DrainDispatcher();
|
|
|
+ }
|
|
|
+ protected ContainerAllocator createContainerAllocator(
|
|
|
+ ClientService clientService, AppContext context) {
|
|
|
+ return new MyContainerAllocator(rm, appAttemptId, context);
|
|
|
+ };
|
|
|
+ };
|
|
|
+
|
|
|
+ Assert.assertEquals(0.0, rmApp.getProgress(), 0.0);
|
|
|
+
|
|
|
+ mrApp.submit(conf);
|
|
|
+ Job job = mrApp.getContext().getAllJobs().entrySet().iterator().next()
|
|
|
+ .getValue();
|
|
|
+
|
|
|
+ DrainDispatcher amDispatcher = (DrainDispatcher) mrApp.getDispatcher();
|
|
|
+
|
|
|
+ MyContainerAllocator allocator = (MyContainerAllocator) mrApp
|
|
|
+ .getContainerAllocator();
|
|
|
+
|
|
|
+ mrApp.waitForState(job, JobState.RUNNING);
|
|
|
+
|
|
|
+ amDispatcher.await();
|
|
|
+ // Wait till all map-attempts request for containers
|
|
|
+ for (Task t : job.getTasks().values()) {
|
|
|
+ mrApp.waitForState(t.getAttempts().values().iterator().next(),
|
|
|
+ TaskAttemptState.UNASSIGNED);
|
|
|
+ }
|
|
|
+ amDispatcher.await();
|
|
|
+
|
|
|
+ allocator.schedule();
|
|
|
+ rmDispatcher.await();
|
|
|
+ amNodeManager.nodeHeartbeat(true);
|
|
|
+ rmDispatcher.await();
|
|
|
+ allocator.schedule();
|
|
|
+ rmDispatcher.await();
|
|
|
+
|
|
|
+ // Wait for all map-tasks to be running
|
|
|
+ for (Task t : job.getTasks().values()) {
|
|
|
+ mrApp.waitForState(t, TaskState.RUNNING);
|
|
|
+ }
|
|
|
|
|
|
allocator.schedule(); // Send heartbeat
|
|
|
- dispatcher.await();
|
|
|
- Assert.assertEquals(0.0, app.getProgress(), 0.0);
|
|
|
+ rmDispatcher.await();
|
|
|
+ Assert.assertEquals(0.05f, job.getProgress(), 0.001f);
|
|
|
+ Assert.assertEquals(0.05f, rmApp.getProgress(), 0.001f);
|
|
|
|
|
|
- job.setProgress(100, 10, 0, 0);
|
|
|
+ Iterator<Task> it = job.getTasks().values().iterator();
|
|
|
+
|
|
|
+ // Finish off 1 map so that map-progress is 10%
|
|
|
+ finishNextNTasks(mrApp, it, 1);
|
|
|
allocator.schedule();
|
|
|
- dispatcher.await();
|
|
|
- Assert.assertEquals(14f, app.getProgress(), 0.0);
|
|
|
+ rmDispatcher.await();
|
|
|
+ Assert.assertEquals(0.14f, job.getProgress(), 0.001f);
|
|
|
+ Assert.assertEquals(0.14f, rmApp.getProgress(), 0.001f);
|
|
|
|
|
|
- job.setProgress(100, 60, 0, 0);
|
|
|
+ // Finish off 5 more map so that map-progress is 60%
|
|
|
+ finishNextNTasks(mrApp, it, 5);
|
|
|
allocator.schedule();
|
|
|
- dispatcher.await();
|
|
|
- Assert.assertEquals(59.0f, app.getProgress(), 0.0);
|
|
|
+ rmDispatcher.await();
|
|
|
+ Assert.assertEquals(0.59f, job.getProgress(), 0.001f);
|
|
|
+ Assert.assertEquals(0.59f, rmApp.getProgress(), 0.001f);
|
|
|
|
|
|
- job.setProgress(100, 100, 0, 100);
|
|
|
+ // Finish off remaining map so that map-progress is 100%
|
|
|
+ finishNextNTasks(mrApp, it, 4);
|
|
|
allocator.schedule();
|
|
|
- dispatcher.await();
|
|
|
- Assert.assertEquals(100.0f, app.getProgress(), 0.0);
|
|
|
+ rmDispatcher.await();
|
|
|
+ Assert.assertEquals(0.95f, job.getProgress(), 0.001f);
|
|
|
+ Assert.assertEquals(0.95f, rmApp.getProgress(), 0.001f);
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -1000,7 +1114,6 @@ public class TestRMContainerAllocator {
|
|
|
|
|
|
private MyResourceManager rm;
|
|
|
|
|
|
- @SuppressWarnings("rawtypes")
|
|
|
private static AppContext createAppContext(
|
|
|
ApplicationAttemptId appAttemptId, Job job) {
|
|
|
AppContext context = mock(AppContext.class);
|
|
@@ -1028,7 +1141,15 @@ public class TestRMContainerAllocator {
|
|
|
return service;
|
|
|
}
|
|
|
|
|
|
- MyContainerAllocator(MyResourceManager rm, Configuration conf,
|
|
|
+ // Use this constructor when using a real job.
|
|
|
+ MyContainerAllocator(MyResourceManager rm,
|
|
|
+ ApplicationAttemptId appAttemptId, AppContext context) {
|
|
|
+ super(createMockClientService(), context);
|
|
|
+ this.rm = rm;
|
|
|
+ }
|
|
|
+
|
|
|
+ // Use this constructor when you are using a mocked job.
|
|
|
+ public MyContainerAllocator(MyResourceManager rm, Configuration conf,
|
|
|
ApplicationAttemptId appAttemptId, Job job) {
|
|
|
super(createMockClientService(), createAppContext(appAttemptId, job));
|
|
|
this.rm = rm;
|
|
@@ -1090,6 +1211,7 @@ public class TestRMContainerAllocator {
|
|
|
return result;
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
protected void startAllocatorThread() {
|
|
|
// override to NOT start thread
|
|
|
}
|