|
@@ -182,8 +182,6 @@ public class TestCapacityScheduler extends TestCase {
|
|
|
}
|
|
|
mapTaskCtr = 0;
|
|
|
redTaskCtr = 0;
|
|
|
- super.setMaxVirtualMemoryForTask(jobConf.getMaxVirtualMemoryForTask());
|
|
|
- super.setMaxPhysicalMemoryForTask(jobConf.getMaxPhysicalMemoryForTask());
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -232,7 +230,7 @@ public class TestCapacityScheduler extends TestCase {
|
|
|
}
|
|
|
return task;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public Task obtainNewReduceTask(final TaskTrackerStatus tts,
|
|
|
int clusterSize, int ignored) throws IOException {
|
|
@@ -727,7 +725,7 @@ public class TestCapacityScheduler extends TestCase {
|
|
|
private FakeJobInProgress submitJob(int state, JobConf jobConf) throws IOException {
|
|
|
FakeJobInProgress job =
|
|
|
new FakeJobInProgress(new JobID("test", ++jobCounter),
|
|
|
- (jobConf == null ? new JobConf() : jobConf), taskTrackerManager,
|
|
|
+ (jobConf == null ? new JobConf(conf) : jobConf), taskTrackerManager,
|
|
|
jobConf.getUser());
|
|
|
job.getStatus().setRunState(state);
|
|
|
taskTrackerManager.submitJob(job);
|
|
@@ -1498,12 +1496,6 @@ public class TestCapacityScheduler extends TestCase {
|
|
|
LOG.debug("Starting the scheduler.");
|
|
|
taskTrackerManager = new FakeTaskTrackerManager(1, 1, 1);
|
|
|
|
|
|
- // Limited TT - 1GB vmem and 512MB pmem
|
|
|
- taskTrackerManager.getTaskTracker("tt1").getResourceStatus()
|
|
|
- .setTotalVirtualMemory(1 * 1024 * 1024 * 1024L);
|
|
|
- taskTrackerManager.getTaskTracker("tt1").getResourceStatus()
|
|
|
- .setTotalPhysicalMemory(512 * 1024 * 1024L);
|
|
|
-
|
|
|
taskTrackerManager.addQueues(new String[] { "default" });
|
|
|
ArrayList<FakeQueueInfo> queues = new ArrayList<FakeQueueInfo>();
|
|
|
queues.add(new FakeQueueInfo("default", 100.0f, true, 25));
|
|
@@ -1513,11 +1505,11 @@ public class TestCapacityScheduler extends TestCase {
|
|
|
// memory-based scheduling disabled by default.
|
|
|
scheduler.start();
|
|
|
|
|
|
- LOG.debug("Submit one high memory(3GB vmem, 1GBpmem) job of 1 map task "
|
|
|
- + "and 1 reduce task.");
|
|
|
+ LOG.debug("Submit one high memory job of 1 3GB map task "
|
|
|
+ + "and 1 1GB reduce task.");
|
|
|
JobConf jConf = new JobConf();
|
|
|
- jConf.setMaxVirtualMemoryForTask(3 * 1024 * 1024 * 1024L); // 3GB vmem
|
|
|
- jConf.setMaxPhysicalMemoryForTask(1 * 1024 * 1024 * 1024L); // 1 GB pmem
|
|
|
+ jConf.setMemoryForMapTask(3 * 1024L); // 3GB
|
|
|
+ jConf.setMemoryForReduceTask(1 * 1024L); // 1 GB
|
|
|
jConf.setNumMapTasks(1);
|
|
|
jConf.setNumReduceTasks(1);
|
|
|
jConf.setQueueName("default");
|
|
@@ -1532,197 +1524,57 @@ public class TestCapacityScheduler extends TestCase {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Test to verify that highPmemJobs are scheduled like all other jobs when
|
|
|
- * physical-memory based scheduling is not enabled.
|
|
|
- * @throws IOException
|
|
|
- */
|
|
|
- public void testDisabledPmemBasedScheduling()
|
|
|
- throws IOException {
|
|
|
-
|
|
|
- LOG.debug("Starting the scheduler.");
|
|
|
- taskTrackerManager = new FakeTaskTrackerManager(1, 1, 1);
|
|
|
-
|
|
|
- // Limited TT - 100GB vmem and 500MB pmem
|
|
|
- TaskTrackerStatus.ResourceStatus ttStatus =
|
|
|
- taskTrackerManager.getTaskTracker("tt1").getResourceStatus();
|
|
|
- ttStatus.setTotalVirtualMemory(100 * 1024 * 1024 * 1024L);
|
|
|
- ttStatus.setReservedVirtualMemory(0);
|
|
|
- ttStatus.setTotalPhysicalMemory(500 * 1024 * 1024L);
|
|
|
- ttStatus.setReservedPhysicalMemory(0);
|
|
|
-
|
|
|
- taskTrackerManager.addQueues(new String[] { "default" });
|
|
|
- ArrayList<FakeQueueInfo> queues = new ArrayList<FakeQueueInfo>();
|
|
|
- queues.add(new FakeQueueInfo("default", 100.0f, true, 25));
|
|
|
- resConf.setFakeQueues(queues);
|
|
|
- scheduler.setResourceManagerConf(resConf);
|
|
|
- scheduler.setTaskTrackerManager(taskTrackerManager);
|
|
|
- // enable vmem-based scheduling. pmem based scheduling disabled by default.
|
|
|
- scheduler.getConf().setLong(JobConf.MAPRED_TASK_DEFAULT_MAXVMEM_PROPERTY,
|
|
|
- 1536 * 1024 * 1024L);
|
|
|
- scheduler.getConf().setLong(JobConf.UPPER_LIMIT_ON_TASK_VMEM_PROPERTY,
|
|
|
- 3 * 1024 * 1024 * 1024L);
|
|
|
- scheduler.start();
|
|
|
-
|
|
|
- LOG.debug("Submit one high pmem(3GB vmem, 1GBpmem) job of 1 map task "
|
|
|
- + "and 1 reduce task.");
|
|
|
- JobConf jConf = new JobConf();
|
|
|
- jConf.setMaxVirtualMemoryForTask(3 * 1024 * 1024 * 1024L); // 3GB vmem
|
|
|
- jConf.setMaxPhysicalMemoryForTask(1 * 1024 * 1024 * 1024L); // 1 GB pmem
|
|
|
- jConf.setNumMapTasks(1);
|
|
|
- jConf.setNumReduceTasks(1);
|
|
|
- jConf.setQueueName("default");
|
|
|
- jConf.setUser("u1");
|
|
|
- submitJobAndInit(JobStatus.RUNNING, jConf);
|
|
|
-
|
|
|
- // assert that all tasks are launched even though they transgress the
|
|
|
- // scheduling limits.
|
|
|
-
|
|
|
- checkAssignment("tt1", "attempt_test_0001_m_000001_0 on tt1");
|
|
|
- checkAssignment("tt1", "attempt_test_0001_r_000001_0 on tt1");
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Test HighMemoryJobs.
|
|
|
- * @throws IOException
|
|
|
- */
|
|
|
- public void testHighMemoryJobs()
|
|
|
- throws IOException {
|
|
|
-
|
|
|
- LOG.debug("Starting the scheduler.");
|
|
|
- taskTrackerManager = new FakeTaskTrackerManager(1, 1, 1);
|
|
|
-
|
|
|
- TaskTrackerStatus.ResourceStatus ttStatus =
|
|
|
- taskTrackerManager.getTaskTracker("tt1").getResourceStatus();
|
|
|
- ttStatus.setTotalVirtualMemory(3 * 1024 * 1024 * 1024L);
|
|
|
- ttStatus.setReservedVirtualMemory(0);
|
|
|
- ttStatus.setTotalPhysicalMemory(1 * 1024 * 1024 * 1024L);
|
|
|
- ttStatus.setReservedPhysicalMemory(0);
|
|
|
- // Normal job on this TT would be 1.5GB vmem, 0.5GB pmem
|
|
|
-
|
|
|
- taskTrackerManager.addQueues(new String[] { "default" });
|
|
|
- ArrayList<FakeQueueInfo> queues = new ArrayList<FakeQueueInfo>();
|
|
|
- queues.add(new FakeQueueInfo("default", 100.0f, true, 25));
|
|
|
- resConf.setFakeQueues(queues);
|
|
|
- scheduler.setTaskTrackerManager(taskTrackerManager);
|
|
|
- // enabled memory-based scheduling
|
|
|
- scheduler.getConf().setLong(JobConf.MAPRED_TASK_DEFAULT_MAXVMEM_PROPERTY,
|
|
|
- 1536 * 1024 * 1024L);
|
|
|
- scheduler.getConf().setLong(JobConf.UPPER_LIMIT_ON_TASK_VMEM_PROPERTY,
|
|
|
- 3 * 1024 * 1024 * 1024L);
|
|
|
- resConf.setDefaultPercentOfPmemInVmem(33.3f);
|
|
|
- resConf.setLimitMaxPmemForTasks(1 * 1024 * 1024 * 1024L);
|
|
|
- scheduler.setResourceManagerConf(resConf);
|
|
|
- scheduler.start();
|
|
|
-
|
|
|
- LOG.debug("Submit one high memory(1600MB vmem, 400MB pmem) job of "
|
|
|
- + "1 map task and 1 reduce task.");
|
|
|
- JobConf jConf = new JobConf();
|
|
|
- jConf.setMaxVirtualMemoryForTask(1600 * 1024 * 1024L); // 1.6GB vmem
|
|
|
- jConf.setMaxPhysicalMemoryForTask(400 * 1024 * 1024L); // 400MB pmem
|
|
|
- jConf.setNumMapTasks(1);
|
|
|
- jConf.setNumReduceTasks(1);
|
|
|
- jConf.setQueueName("default");
|
|
|
- jConf.setUser("u1");
|
|
|
- jConf.setMapSpeculativeExecution(false);
|
|
|
- jConf.setReduceSpeculativeExecution(false);
|
|
|
- FakeJobInProgress job1 = submitJobAndInit(JobStatus.PREP, jConf);
|
|
|
- checkAssignment("tt1", "attempt_test_0001_m_000001_0 on tt1");
|
|
|
-
|
|
|
- // No more tasks of this job can run on the TT because of lack of vmem
|
|
|
- assertNull(scheduler.assignTasks(tracker("tt1")));
|
|
|
-
|
|
|
- // Let attempt_test_0001_m_000001_0 finish, task assignment should succeed.
|
|
|
- taskTrackerManager.finishTask("tt1", "attempt_test_0001_m_000001_0", job1);
|
|
|
- checkAssignment("tt1", "attempt_test_0001_r_000001_0 on tt1");
|
|
|
-
|
|
|
- LOG.debug("Submit another high memory(1200MB vmem, 800MB pmem) job of "
|
|
|
- + "1 map task and 0 reduces.");
|
|
|
- jConf.setMaxVirtualMemoryForTask(1200 * 1024 * 1024L);
|
|
|
- jConf.setMaxPhysicalMemoryForTask(800 * 1024 * 1024L);
|
|
|
- jConf.setNumMapTasks(1);
|
|
|
- jConf.setNumReduceTasks(0);
|
|
|
- jConf.setQueueName("default");
|
|
|
- jConf.setUser("u1");
|
|
|
- jConf.setMapSpeculativeExecution(false);
|
|
|
- jConf.setReduceSpeculativeExecution(false);
|
|
|
- submitJobAndInit(JobStatus.PREP, jConf); // job2
|
|
|
-
|
|
|
- // This job shouldn't run the TT now because of lack of pmem
|
|
|
- assertNull(scheduler.assignTasks(tracker("tt1")));
|
|
|
-
|
|
|
- // Let attempt_test_0001_m_000002_0 finish, task assignment should succeed.
|
|
|
- taskTrackerManager.finishTask("tt1", "attempt_test_0001_r_000001_0", job1);
|
|
|
- checkAssignment("tt1", "attempt_test_0002_m_000001_0 on tt1");
|
|
|
-
|
|
|
- LOG.debug("Submit a normal memory(200MB vmem, 100MB pmem) job of "
|
|
|
- + "0 maps and 1 reduce task.");
|
|
|
- jConf.setMaxVirtualMemoryForTask(200 * 1024 * 1024L);
|
|
|
- jConf.setMaxPhysicalMemoryForTask(100 * 1024 * 1024L);
|
|
|
- jConf.setNumMapTasks(0);
|
|
|
- jConf.setNumReduceTasks(1);
|
|
|
- jConf.setQueueName("default");
|
|
|
- jConf.setUser("u1");
|
|
|
- submitJobAndInit(JobStatus.PREP, jConf); // job3
|
|
|
-
|
|
|
- checkAssignment("tt1", "attempt_test_0003_r_000001_0 on tt1");
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Test HADOOP-4979.
|
|
|
- * Bug fix for making sure we always return null to TT if there is a
|
|
|
- * high-mem job, and not look at reduce jobs (if map tasks are high-mem)
|
|
|
- * or vice-versa.
|
|
|
+ * Test reverting HADOOP-4979. If there is a high-mem job, we should now look
|
|
|
+ * at reduce jobs (if map tasks are high-mem) or vice-versa.
|
|
|
+ *
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- public void testHighMemoryBlocking()
|
|
|
+ public void testHighMemoryBlockingAcrossTaskTypes()
|
|
|
throws IOException {
|
|
|
|
|
|
// 2 map and 1 reduce slots
|
|
|
taskTrackerManager = new FakeTaskTrackerManager(1, 2, 1);
|
|
|
|
|
|
- TaskTrackerStatus.ResourceStatus ttStatus =
|
|
|
- taskTrackerManager.getTaskTracker("tt1").getResourceStatus();
|
|
|
- ttStatus.setTotalVirtualMemory(3 * 1024 * 1024 * 1024L);
|
|
|
- ttStatus.setReservedVirtualMemory(0);
|
|
|
- ttStatus.setTotalPhysicalMemory(1536 * 1024 * 1024L);
|
|
|
- ttStatus.setReservedPhysicalMemory(0);
|
|
|
- // Normal job on this TT would be 1GB vmem, 0.5GB pmem
|
|
|
-
|
|
|
taskTrackerManager.addQueues(new String[] { "default" });
|
|
|
ArrayList<FakeQueueInfo> queues = new ArrayList<FakeQueueInfo>();
|
|
|
queues.add(new FakeQueueInfo("default", 100.0f, true, 25));
|
|
|
resConf.setFakeQueues(queues);
|
|
|
scheduler.setTaskTrackerManager(taskTrackerManager);
|
|
|
// enabled memory-based scheduling
|
|
|
- scheduler.getConf().setLong(JobConf.MAPRED_TASK_DEFAULT_MAXVMEM_PROPERTY,
|
|
|
- 1 * 1024 * 1024 * 1024L);
|
|
|
- scheduler.getConf().setLong(JobConf.UPPER_LIMIT_ON_TASK_VMEM_PROPERTY,
|
|
|
- 3 * 1024 * 1024 * 1024L);
|
|
|
- resConf.setDefaultPercentOfPmemInVmem(33.3f);
|
|
|
- resConf.setLimitMaxPmemForTasks(1536 * 1024 * 1024L);
|
|
|
+ // Normal job in the cluster would be 1GB maps/reduces
|
|
|
+ scheduler.getConf().setLong(
|
|
|
+ JobTracker.MAPRED_CLUSTER_MAX_MAP_MEMORY_MB_PROPERTY,
|
|
|
+ 2 * 1024);
|
|
|
+ scheduler.getConf().setLong(
|
|
|
+ JobTracker.MAPRED_CLUSTER_MAP_MEMORY_MB_PROPERTY, 1 * 1024);
|
|
|
+ scheduler.getConf().setLong(
|
|
|
+ JobTracker.MAPRED_CLUSTER_MAX_REDUCE_MEMORY_MB_PROPERTY,
|
|
|
+ 1 * 1024);
|
|
|
+ scheduler.getConf().setLong(
|
|
|
+ JobTracker.MAPRED_CLUSTER_REDUCE_MEMORY_MB_PROPERTY, 1 * 1024);
|
|
|
scheduler.setResourceManagerConf(resConf);
|
|
|
scheduler.start();
|
|
|
|
|
|
- // We need a situation where the scheduler needs to run a map task,
|
|
|
- // but the available one has a high-mem requirement. There should
|
|
|
- // be another job whose maps or reduces can run, but they shouldn't
|
|
|
- // be scheduled.
|
|
|
+ // The situation : Two jobs in the queue. First job with only maps and no
|
|
|
+ // reduces and is a high memory job. Second job is a normal job with both maps and reduces.
|
|
|
+ // First job cannot run for want of memory for maps. In this case, second job's reduces should run.
|
|
|
|
|
|
- LOG.debug("Submit one high memory(2GB vmem, 400MB pmem) job of "
|
|
|
+ LOG.debug("Submit one high memory(2GB maps, 0MB reduces) job of "
|
|
|
+ "2 map tasks");
|
|
|
- JobConf jConf = new JobConf();
|
|
|
- jConf.setMaxVirtualMemoryForTask(2 * 1024 * 1024 * 1024L); // 2GB vmem
|
|
|
- jConf.setMaxPhysicalMemoryForTask(400 * 1024 * 1024L); // 400MB pmem
|
|
|
+ JobConf jConf = new JobConf(conf);
|
|
|
+ jConf.setMemoryForMapTask(2 * 1024);
|
|
|
+ jConf.setMemoryForReduceTask(0);
|
|
|
jConf.setNumMapTasks(2);
|
|
|
jConf.setNumReduceTasks(0);
|
|
|
jConf.setQueueName("default");
|
|
|
jConf.setUser("u1");
|
|
|
FakeJobInProgress job1 = submitJobAndInit(JobStatus.PREP, jConf);
|
|
|
- LOG.debug("Submit another regular memory(900MB vmem, 200MB pmem) job of "
|
|
|
+
|
|
|
+ LOG.debug("Submit another regular memory(1GB vmem maps/reduces) job of "
|
|
|
+ "2 map/red tasks");
|
|
|
- jConf = new JobConf();
|
|
|
- jConf.setMaxVirtualMemoryForTask(900 * 1024 * 1024L); // 900MB vmem
|
|
|
- jConf.setMaxPhysicalMemoryForTask(200 * 1024 * 1024L); // 200MB pmem
|
|
|
+ jConf = new JobConf(conf);
|
|
|
+ jConf.setMemoryForMapTask(1 * 1024);
|
|
|
+ jConf.setMemoryForReduceTask(1 * 1024);
|
|
|
jConf.setNumMapTasks(2);
|
|
|
jConf.setNumReduceTasks(2);
|
|
|
jConf.setQueueName("default");
|
|
@@ -1732,75 +1584,8 @@ public class TestCapacityScheduler extends TestCase {
|
|
|
// first, a map from j1 will run
|
|
|
checkAssignment("tt1", "attempt_test_0001_m_000001_0 on tt1");
|
|
|
// at this point, the scheduler tries to schedule another map from j1.
|
|
|
- // there isn't enough space. There is space to run the second job's
|
|
|
- // map or reduce task, but they shouldn't be scheduled
|
|
|
- assertNull(scheduler.assignTasks(tracker("tt1")));
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * test invalid highMemoryJobs
|
|
|
- * @throws IOException
|
|
|
- */
|
|
|
- public void testHighMemoryJobWithInvalidRequirements()
|
|
|
- throws IOException {
|
|
|
- LOG.debug("Starting the scheduler.");
|
|
|
- taskTrackerManager = new FakeTaskTrackerManager(1, 1, 1);
|
|
|
- TaskTrackerStatus.ResourceStatus ttStatus =
|
|
|
- taskTrackerManager.getTaskTracker("tt1").getResourceStatus();
|
|
|
- ttStatus.setTotalVirtualMemory(3 * 1024 * 1024 * 1024);
|
|
|
- ttStatus.setReservedVirtualMemory(0);
|
|
|
- ttStatus.setTotalPhysicalMemory(1 * 1024 * 1024 * 1024);
|
|
|
- ttStatus.setReservedPhysicalMemory(0);
|
|
|
-
|
|
|
- ArrayList<FakeQueueInfo> queues = new ArrayList<FakeQueueInfo>();
|
|
|
- queues.add(new FakeQueueInfo("default", 100.0f, true, 25));
|
|
|
- taskTrackerManager.addQueues(new String[] { "default" });
|
|
|
- resConf.setFakeQueues(queues);
|
|
|
- scheduler.setTaskTrackerManager(taskTrackerManager);
|
|
|
- // enabled memory-based scheduling
|
|
|
- long vmemUpperLimit = 1 * 1024 * 1024 * 1024L;
|
|
|
- long vmemDefault = 1536 * 1024 * 1024L;
|
|
|
- long pmemUpperLimit = vmemUpperLimit;
|
|
|
- scheduler.getConf().setLong(JobConf.MAPRED_TASK_DEFAULT_MAXVMEM_PROPERTY,
|
|
|
- vmemDefault);
|
|
|
- scheduler.getConf().setLong(JobConf.UPPER_LIMIT_ON_TASK_VMEM_PROPERTY,
|
|
|
- vmemUpperLimit);
|
|
|
- resConf.setDefaultPercentOfPmemInVmem(33.3f);
|
|
|
- resConf.setLimitMaxPmemForTasks(pmemUpperLimit);
|
|
|
- scheduler.setResourceManagerConf(resConf);
|
|
|
- scheduler.start();
|
|
|
-
|
|
|
- LOG.debug("Submit one invalid high ram(5GB vmem, 3GB pmem) job of "
|
|
|
- + "1 map, 0 reduce tasks.");
|
|
|
- long jobMaxVmem = 5 * 1024 * 1024 * 1024L;
|
|
|
- long jobMaxPmem = 3 * 1024 * 1024 * 1024L;
|
|
|
- JobConf jConf = new JobConf();
|
|
|
- jConf.setMaxVirtualMemoryForTask(jobMaxVmem);
|
|
|
- jConf.setMaxPhysicalMemoryForTask(jobMaxPmem);
|
|
|
- jConf.setNumMapTasks(1);
|
|
|
- jConf.setNumReduceTasks(0);
|
|
|
- jConf.setQueueName("default");
|
|
|
- jConf.setUser("u1");
|
|
|
-
|
|
|
- boolean throwsException = false;
|
|
|
- String msg = null;
|
|
|
- FakeJobInProgress job;
|
|
|
- try {
|
|
|
- job = submitJob(JobStatus.PREP, jConf);
|
|
|
- } catch (IOException ioe) {
|
|
|
- // job has to fail
|
|
|
- throwsException = true;
|
|
|
- msg = ioe.getMessage();
|
|
|
- }
|
|
|
-
|
|
|
- assertTrue(throwsException);
|
|
|
- job = (FakeJobInProgress) taskTrackerManager.getJobs().toArray()[0];
|
|
|
- assertTrue(msg.matches(job.getJobID() + " \\(" + jobMaxVmem + "vmem, "
|
|
|
- + jobMaxPmem + "pmem\\) exceeds the cluster's max-memory-limits \\("
|
|
|
- + vmemUpperLimit + "vmem, " + pmemUpperLimit
|
|
|
- + "pmem\\). Cannot run in this cluster, so killing it."));
|
|
|
- // For job, no cleanup task needed so gets killed immediately.
|
|
|
- assertTrue(job.getStatus().getRunState() == JobStatus.KILLED);
|
|
|
+ // there isn't enough space. The second job's reduce should be scheduled.
|
|
|
+ checkAssignment("tt1", "attempt_test_0002_r_000001_0 on tt1");
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -1811,13 +1596,7 @@ public class TestCapacityScheduler extends TestCase {
|
|
|
throws IOException {
|
|
|
|
|
|
LOG.debug("Starting the scheduler.");
|
|
|
- taskTrackerManager = new FakeTaskTrackerManager(1, 1, 1);
|
|
|
- TaskTrackerStatus.ResourceStatus ttStatus =
|
|
|
- taskTrackerManager.getTaskTracker("tt1").getResourceStatus();
|
|
|
- ttStatus.setTotalVirtualMemory(3 * 1024 * 1024 * 1024L);
|
|
|
- ttStatus.setReservedVirtualMemory(0);
|
|
|
- ttStatus.setTotalPhysicalMemory(1 * 1024 * 1024 * 1024L);
|
|
|
- ttStatus.setReservedPhysicalMemory(0);
|
|
|
+ taskTrackerManager = new FakeTaskTrackerManager(2, 2, 2);
|
|
|
|
|
|
ArrayList<FakeQueueInfo> queues = new ArrayList<FakeQueueInfo>();
|
|
|
queues.add(new FakeQueueInfo("default", 100.0f, true, 25));
|
|
@@ -1825,68 +1604,65 @@ public class TestCapacityScheduler extends TestCase {
|
|
|
resConf.setFakeQueues(queues);
|
|
|
scheduler.setTaskTrackerManager(taskTrackerManager);
|
|
|
// enabled memory-based scheduling
|
|
|
- scheduler.getConf().setLong(JobConf.MAPRED_TASK_DEFAULT_MAXVMEM_PROPERTY,
|
|
|
- 1536 * 1024 * 1024L);
|
|
|
- scheduler.getConf().setLong(JobConf.UPPER_LIMIT_ON_TASK_VMEM_PROPERTY,
|
|
|
- 4 * 1024 * 1024 * 1024L);
|
|
|
- resConf.setDefaultPercentOfPmemInVmem(33.3f);
|
|
|
- resConf.setLimitMaxPmemForTasks(2 * 1024 * 1024 * 1024L);
|
|
|
+ // Normal jobs 1GB maps/reduces. 2GB limit on maps/reduces
|
|
|
+ scheduler.getConf().setLong(
|
|
|
+ JobTracker.MAPRED_CLUSTER_MAX_MAP_MEMORY_MB_PROPERTY,
|
|
|
+ 2 * 1024);
|
|
|
+ scheduler.getConf().setLong(
|
|
|
+ JobTracker.MAPRED_CLUSTER_MAP_MEMORY_MB_PROPERTY, 1 * 1024);
|
|
|
+ scheduler.getConf().setLong(
|
|
|
+ JobTracker.MAPRED_CLUSTER_MAX_REDUCE_MEMORY_MB_PROPERTY,
|
|
|
+ 2 * 1024);
|
|
|
+ scheduler.getConf().setLong(
|
|
|
+ JobTracker.MAPRED_CLUSTER_REDUCE_MEMORY_MB_PROPERTY, 1 * 1024);
|
|
|
scheduler.setResourceManagerConf(resConf);
|
|
|
scheduler.start();
|
|
|
|
|
|
- LOG.debug("Submit one high memory(4GB vmem, 512MB pmem) job of "
|
|
|
+ LOG.debug("Submit one normal memory(1GB maps/reduces) job of "
|
|
|
+ "1 map, 0 reduce tasks.");
|
|
|
- JobConf jConf = new JobConf();
|
|
|
- jConf.setMaxVirtualMemoryForTask(4 * 1024 * 1024 * 1024L);
|
|
|
- jConf.setMaxPhysicalMemoryForTask(512 * 1024 * 1024L);
|
|
|
+ JobConf jConf = new JobConf(conf);
|
|
|
+ jConf.setMemoryForMapTask(1 * 1024);
|
|
|
+ jConf.setMemoryForReduceTask(1 * 1024);
|
|
|
jConf.setNumMapTasks(1);
|
|
|
- jConf.setNumReduceTasks(0);
|
|
|
+ jConf.setNumReduceTasks(1);
|
|
|
jConf.setQueueName("default");
|
|
|
jConf.setUser("u1");
|
|
|
FakeJobInProgress job1 = submitJobAndInit(JobStatus.PREP, jConf);
|
|
|
- // TTs should not run these jobs i.e. cluster blocked because of lack of
|
|
|
- // vmem
|
|
|
- assertNull(scheduler.assignTasks(tracker("tt1")));
|
|
|
- assertNull(scheduler.assignTasks(tracker("tt1")));
|
|
|
|
|
|
- // Job should still be alive
|
|
|
- assertTrue(job1.getStatus().getRunState() == JobStatus.RUNNING);
|
|
|
+ // Fill the second tt with this job.
|
|
|
+ checkAssignment("tt2", "attempt_test_0001_m_000001_0 on tt2");
|
|
|
+ checkAssignment("tt2", "attempt_test_0001_r_000001_0 on tt2");
|
|
|
|
|
|
- LOG.debug("Submit a normal job of 1 map, 0 reduce tasks.");
|
|
|
- // Use cluster-wide defaults
|
|
|
- jConf.setMaxVirtualMemoryForTask(JobConf.DISABLED_MEMORY_LIMIT);
|
|
|
- jConf.setMaxPhysicalMemoryForTask(JobConf.DISABLED_MEMORY_LIMIT);
|
|
|
+ LOG.debug("Submit one high memory(2GB maps/reduces) job of "
|
|
|
+ + "2 map, 2 reduce tasks.");
|
|
|
+ jConf = new JobConf(conf);
|
|
|
+ jConf.setMemoryForMapTask(2 * 1024);
|
|
|
+ jConf.setMemoryForReduceTask(2 * 1024);
|
|
|
+ jConf.setNumMapTasks(2);
|
|
|
+ jConf.setNumReduceTasks(2);
|
|
|
+ jConf.setQueueName("default");
|
|
|
+ jConf.setUser("u1");
|
|
|
FakeJobInProgress job2 = submitJobAndInit(JobStatus.PREP, jConf);
|
|
|
|
|
|
- // cluster should still be blocked for job1 and so even job2 should not run
|
|
|
- // even though it is a normal job
|
|
|
- assertNull(scheduler.assignTasks(tracker("tt1")));
|
|
|
-
|
|
|
- scheduler.taskTrackerManager.killJob(job2.getJobID());
|
|
|
- scheduler.taskTrackerManager.killJob(job1.getJobID());
|
|
|
+ checkAssignment("tt1", "attempt_test_0002_m_000001_0 on tt1");
|
|
|
+ checkAssignment("tt1", "attempt_test_0002_r_000001_0 on tt1");
|
|
|
|
|
|
- LOG.debug("Submit one high memory(2GB vmem, 2GB pmem) job of "
|
|
|
+ LOG.debug("Submit one normal memory(1GB maps/reduces) job of "
|
|
|
+ "1 map, 0 reduce tasks.");
|
|
|
- jConf.setMaxVirtualMemoryForTask(2 * 1024 * 1024 * 1024L);
|
|
|
- jConf.setMaxPhysicalMemoryForTask(2 * 1024 * 1024 * 1024L);
|
|
|
+ jConf = new JobConf(conf);
|
|
|
+ jConf.setMemoryForMapTask(1 * 1024);
|
|
|
+ jConf.setMemoryForReduceTask(1 * 1024);
|
|
|
+ jConf.setNumMapTasks(1);
|
|
|
+ jConf.setNumReduceTasks(1);
|
|
|
+ jConf.setQueueName("default");
|
|
|
+ jConf.setUser("u1");
|
|
|
FakeJobInProgress job3 = submitJobAndInit(JobStatus.PREP, jConf);
|
|
|
- // TTs should not run these jobs i.e. cluster blocked because of lack of
|
|
|
- // pmem now.
|
|
|
- assertNull(scheduler.assignTasks(tracker("tt1")));
|
|
|
- assertNull(scheduler.assignTasks(tracker("tt1")));
|
|
|
-
|
|
|
- // Job should still be alive
|
|
|
- assertTrue(job3.getStatus().getRunState() == JobStatus.RUNNING);
|
|
|
|
|
|
- LOG.debug("Submit a normal job of 1 map, 0 reduce tasks.");
|
|
|
- // Use cluster-wide defaults
|
|
|
- jConf.setMaxVirtualMemoryForTask(JobConf.DISABLED_MEMORY_LIMIT);
|
|
|
- jConf.setMaxPhysicalMemoryForTask(JobConf.DISABLED_MEMORY_LIMIT);
|
|
|
- submitJobAndInit(JobStatus.PREP, jConf); // job4
|
|
|
-
|
|
|
- // cluster should still be blocked for job3 and so even job4 should not run
|
|
|
- // even though it is a normal job
|
|
|
+ // Job2 cannot fit on tt2 or tt1. Blocking. Job3 also will not run.
|
|
|
assertNull(scheduler.assignTasks(tracker("tt1")));
|
|
|
+ assertNull(scheduler.assignTasks(tracker("tt2")));
|
|
|
+ assertNull(scheduler.assignTasks(tracker("tt1")));
|
|
|
+ assertNull(scheduler.assignTasks(tracker("tt2")));
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -1900,13 +1676,6 @@ public class TestCapacityScheduler extends TestCase {
|
|
|
// create a cluster with a single node.
|
|
|
LOG.debug("Starting cluster with 1 tasktracker, 2 map and 2 reduce slots");
|
|
|
taskTrackerManager = new FakeTaskTrackerManager(1, 2, 2);
|
|
|
- TaskTrackerStatus.ResourceStatus ttStatus =
|
|
|
- taskTrackerManager.getTaskTracker("tt1").getResourceStatus();
|
|
|
- LOG.debug("Assume TT has 4 GB virtual mem and 2 GB RAM");
|
|
|
- ttStatus.setTotalVirtualMemory(4 * 1024 * 1024 * 1024L);
|
|
|
- ttStatus.setReservedVirtualMemory(0);
|
|
|
- ttStatus.setTotalPhysicalMemory(2 * 1024 * 1024 * 1024L);
|
|
|
- ttStatus.setReservedPhysicalMemory(0);
|
|
|
|
|
|
// create scheduler
|
|
|
ArrayList<FakeQueueInfo> queues = new ArrayList<FakeQueueInfo>();
|
|
@@ -1915,14 +1684,17 @@ public class TestCapacityScheduler extends TestCase {
|
|
|
resConf.setFakeQueues(queues);
|
|
|
scheduler.setTaskTrackerManager(taskTrackerManager);
|
|
|
// enabled memory-based scheduling
|
|
|
- LOG.debug("By default, jobs get 0.5 GB per task vmem" +
|
|
|
- " and 2 GB max vmem, with 50% of it for RAM");
|
|
|
- scheduler.getConf().setLong(JobConf.MAPRED_TASK_DEFAULT_MAXVMEM_PROPERTY,
|
|
|
- 512 * 1024 * 1024L);
|
|
|
- scheduler.getConf().setLong(JobConf.UPPER_LIMIT_ON_TASK_VMEM_PROPERTY,
|
|
|
- 2 * 1024 * 1024 * 1024L);
|
|
|
- resConf.setDefaultPercentOfPmemInVmem(50.0f);
|
|
|
- resConf.setLimitMaxPmemForTasks(1 * 1024 * 1024 * 1024L);
|
|
|
+ LOG.debug("Assume TT has 2GB for maps and 2GB for reduces");
|
|
|
+ scheduler.getConf().setLong(
|
|
|
+ JobTracker.MAPRED_CLUSTER_MAX_MAP_MEMORY_MB_PROPERTY,
|
|
|
+ 2 * 1024L);
|
|
|
+ scheduler.getConf().setLong(
|
|
|
+ JobTracker.MAPRED_CLUSTER_MAP_MEMORY_MB_PROPERTY, 512);
|
|
|
+ scheduler.getConf().setLong(
|
|
|
+ JobTracker.MAPRED_CLUSTER_MAX_REDUCE_MEMORY_MB_PROPERTY,
|
|
|
+ 2 * 1024L);
|
|
|
+ scheduler.getConf().setLong(
|
|
|
+ JobTracker.MAPRED_CLUSTER_REDUCE_MEMORY_MB_PROPERTY, 512);
|
|
|
scheduler.setResourceManagerConf(resConf);
|
|
|
scheduler.start();
|
|
|
|
|
@@ -1931,6 +1703,8 @@ public class TestCapacityScheduler extends TestCase {
|
|
|
JobConf jConf = new JobConf();
|
|
|
jConf.setNumMapTasks(2);
|
|
|
jConf.setNumReduceTasks(2);
|
|
|
+ jConf.setMemoryForMapTask(512);
|
|
|
+ jConf.setMemoryForReduceTask(512);
|
|
|
jConf.setQueueName("default");
|
|
|
jConf.setUser("u1");
|
|
|
FakeJobInProgress job1 = submitJobAndInit(JobStatus.PREP, jConf);
|
|
@@ -1949,6 +1723,8 @@ public class TestCapacityScheduler extends TestCase {
|
|
|
jConf = new JobConf();
|
|
|
jConf.setNumMapTasks(1);
|
|
|
jConf.setNumReduceTasks(1);
|
|
|
+ jConf.setMemoryForMapTask(512);
|
|
|
+ jConf.setMemoryForReduceTask(512);
|
|
|
jConf.setQueueName("default");
|
|
|
jConf.setUser("u1");
|
|
|
FakeJobInProgress job2 = submitJobAndInit(JobStatus.PREP, jConf);
|
|
@@ -2396,21 +2172,8 @@ public class TestCapacityScheduler extends TestCase {
|
|
|
*/
|
|
|
public void testHighRamJobWithSpeculativeExecution() throws IOException {
|
|
|
// 2 map and 2 reduce slots
|
|
|
- taskTrackerManager = new FakeTaskTrackerManager(2, 2, 2);
|
|
|
-
|
|
|
- //task tracker memory configurations.
|
|
|
- TaskTrackerStatus.ResourceStatus ttStatus =
|
|
|
- taskTrackerManager.getTaskTracker("tt1").getResourceStatus();
|
|
|
- ttStatus.setTotalVirtualMemory(3 * 1024 * 1024 * 1024L);
|
|
|
- ttStatus.setReservedVirtualMemory(0);
|
|
|
- ttStatus.setTotalPhysicalMemory(1536 * 1024 * 1024L);
|
|
|
- ttStatus.setReservedPhysicalMemory(0);
|
|
|
- ttStatus = taskTrackerManager.getTaskTracker("tt2").getResourceStatus();
|
|
|
- ttStatus.setTotalVirtualMemory(3 * 1024 * 1024 * 1024L);
|
|
|
- ttStatus.setReservedVirtualMemory(0);
|
|
|
- ttStatus.setTotalPhysicalMemory(1536 * 1024 * 1024L);
|
|
|
- ttStatus.setReservedPhysicalMemory(0);
|
|
|
-
|
|
|
+ taskTrackerManager = new FakeTaskTrackerManager(2, 3, 3);
|
|
|
+ // 1GB for each map, 1GB for each reduce
|
|
|
|
|
|
taskTrackerManager.addQueues(new String[] { "default" });
|
|
|
ArrayList<FakeQueueInfo> queues = new ArrayList<FakeQueueInfo>();
|
|
@@ -2418,19 +2181,23 @@ public class TestCapacityScheduler extends TestCase {
|
|
|
resConf.setFakeQueues(queues);
|
|
|
scheduler.setTaskTrackerManager(taskTrackerManager);
|
|
|
// enabled memory-based scheduling
|
|
|
- scheduler.getConf().setLong(JobConf.MAPRED_TASK_DEFAULT_MAXVMEM_PROPERTY,
|
|
|
- 1 * 1024 * 1024 * 1024L);
|
|
|
- scheduler.getConf().setLong(JobConf.UPPER_LIMIT_ON_TASK_VMEM_PROPERTY,
|
|
|
- 3 * 1024 * 1024 * 1024L);
|
|
|
- resConf.setDefaultPercentOfPmemInVmem(33.3f);
|
|
|
- resConf.setLimitMaxPmemForTasks(1536 * 1024 * 1024L);
|
|
|
+ scheduler.getConf().setLong(
|
|
|
+ JobTracker.MAPRED_CLUSTER_MAX_MAP_MEMORY_MB_PROPERTY,
|
|
|
+ 3 * 1024L);
|
|
|
+ scheduler.getConf().setLong(
|
|
|
+ JobTracker.MAPRED_CLUSTER_MAP_MEMORY_MB_PROPERTY, 1 * 1024L);
|
|
|
+ scheduler.getConf().setLong(
|
|
|
+ JobTracker.MAPRED_CLUSTER_MAX_REDUCE_MEMORY_MB_PROPERTY,
|
|
|
+ 3 * 1024L);
|
|
|
+ scheduler.getConf().setLong(
|
|
|
+ JobTracker.MAPRED_CLUSTER_REDUCE_MEMORY_MB_PROPERTY, 1 * 1024L);
|
|
|
scheduler.setResourceManagerConf(resConf);
|
|
|
scheduler.start();
|
|
|
|
|
|
-
|
|
|
+ //Submit a high memory job with speculative tasks.
|
|
|
JobConf jConf = new JobConf();
|
|
|
- jConf.setMaxVirtualMemoryForTask(2 * 1024 * 1024 * 1024L); // 2GB vmem
|
|
|
- jConf.setMaxPhysicalMemoryForTask(400 * 1024 * 1024L); // 400MB pmem
|
|
|
+ jConf.setMemoryForMapTask(2 * 1024);
|
|
|
+ jConf.setMemoryForReduceTask(0);
|
|
|
jConf.setNumMapTasks(1);
|
|
|
jConf.setNumReduceTasks(0);
|
|
|
jConf.setQueueName("default");
|
|
@@ -2439,20 +2206,18 @@ public class TestCapacityScheduler extends TestCase {
|
|
|
jConf.setReduceSpeculativeExecution(false);
|
|
|
FakeJobInProgress job1 = new FakeJobInProgress(new JobID("test", ++jobCounter),
|
|
|
jConf, taskTrackerManager,"u1");
|
|
|
-
|
|
|
- //Submit a high memory job with speculative tasks.
|
|
|
taskTrackerManager.submitJob(job1);
|
|
|
-
|
|
|
+
|
|
|
+ //Submit normal job
|
|
|
jConf = new JobConf();
|
|
|
- jConf.setMaxVirtualMemoryForTask(100 * 1024 * 1024L); // 100MB vmem
|
|
|
- jConf.setMaxPhysicalMemoryForTask(50 * 1024 * 1024L); // 50MB pmem
|
|
|
+ jConf.setMemoryForMapTask(1 * 1024);
|
|
|
+ jConf.setMemoryForReduceTask(0);
|
|
|
jConf.setNumMapTasks(1);
|
|
|
jConf.setNumReduceTasks(0);
|
|
|
jConf.setQueueName("default");
|
|
|
jConf.setUser("u1");
|
|
|
jConf.setMapSpeculativeExecution(false);
|
|
|
jConf.setReduceSpeculativeExecution(false);
|
|
|
- //Submit normal job
|
|
|
FakeJobInProgress job2 = submitJob(JobStatus.PREP, jConf);
|
|
|
|
|
|
controlledInitializationPoller.selectJobsToInitialize();
|
|
@@ -2487,8 +2252,8 @@ public class TestCapacityScheduler extends TestCase {
|
|
|
|
|
|
//Now submit high ram job with speculative reduce and check.
|
|
|
jConf = new JobConf();
|
|
|
- jConf.setMaxVirtualMemoryForTask(2 * 1024 * 1024 * 1024L); // 2GB vmem
|
|
|
- jConf.setMaxPhysicalMemoryForTask(400 * 1024 * 1024L); // 400MB pmem
|
|
|
+ jConf.setMemoryForMapTask(2 * 1024);
|
|
|
+ jConf.setMemoryForReduceTask(2 * 1024L);
|
|
|
jConf.setNumMapTasks(1);
|
|
|
jConf.setNumReduceTasks(1);
|
|
|
jConf.setQueueName("default");
|
|
@@ -2497,76 +2262,43 @@ public class TestCapacityScheduler extends TestCase {
|
|
|
jConf.setReduceSpeculativeExecution(true);
|
|
|
FakeJobInProgress job3 = new FakeJobInProgress(new JobID("test", ++jobCounter),
|
|
|
jConf, taskTrackerManager,"u1");
|
|
|
-
|
|
|
- //Submit a high memory job with speculative reduce tasks.
|
|
|
taskTrackerManager.submitJob(job3);
|
|
|
-
|
|
|
+
|
|
|
+ //Submit normal job w.r.t reduces
|
|
|
jConf = new JobConf();
|
|
|
- jConf.setMaxVirtualMemoryForTask(100 * 1024 * 1024L); // 100MB vmem
|
|
|
- jConf.setMaxPhysicalMemoryForTask(50 * 1024 * 1024L); // 50MB pmem
|
|
|
+ jConf.setMemoryForMapTask(2 * 1024L);
|
|
|
+ jConf.setMemoryForReduceTask(1 * 104L);
|
|
|
jConf.setNumMapTasks(1);
|
|
|
jConf.setNumReduceTasks(1);
|
|
|
jConf.setQueueName("default");
|
|
|
jConf.setUser("u1");
|
|
|
jConf.setMapSpeculativeExecution(false);
|
|
|
jConf.setReduceSpeculativeExecution(false);
|
|
|
- //Submit normal job
|
|
|
FakeJobInProgress job4 = submitJob(JobStatus.PREP, jConf);
|
|
|
|
|
|
controlledInitializationPoller.selectJobsToInitialize();
|
|
|
raiseStatusChangeEvents(scheduler.jobQueuesManager);
|
|
|
- //all maps of jobs get assigned to same task tracker as
|
|
|
- //job does not have speculative map and same tracker sends two heart
|
|
|
- //beat back to back.
|
|
|
+
|
|
|
+ // Finish up the map scheduler
|
|
|
checkAssignment("tt1", "attempt_test_0003_m_000001_0 on tt1");
|
|
|
- //first map slot gets attention on this tracker.
|
|
|
checkAssignment("tt2", "attempt_test_0004_m_000001_0 on tt2");
|
|
|
- //now first reduce of the job3 would be scheduled on tt2 since it has
|
|
|
- //memory.
|
|
|
- //assigntasks() would check for free reduce slot is greater than
|
|
|
- //map slots. Seeing there is more free reduce slot it would try scheduling
|
|
|
- //reduce of job1 but would block as in it is a high memory task.
|
|
|
- assertNull(scheduler.assignTasks(tracker("tt1")));
|
|
|
- //TT2 would get the reduce task from high memory job as the tt is running
|
|
|
- //normal jobs map. which is low mem.
|
|
|
- checkAssignment("tt2", "attempt_test_0003_r_000001_0 on tt2");
|
|
|
- // now if either TT comes back, it will block because all maps
|
|
|
- // are done, and the first jobs reduce has a speculative task.
|
|
|
+
|
|
|
+ // first, a reduce from j3 will run
|
|
|
+ // at this point, there is a speculative task for the same job to be
|
|
|
+ //scheduled. This task would be scheduled. Till the tasks from job3 gets
|
|
|
+ //complete none of the tasks from other jobs would be scheduled.
|
|
|
+ checkAssignment("tt1", "attempt_test_0003_r_000001_0 on tt1");
|
|
|
+ assertEquals("pending reduces greater than zero " , job3.pendingMaps(), 0);
|
|
|
+ //make same tracker get back, check if you are blocking. Your job
|
|
|
+ //has speculative reduce task so tracker should be blocked even tho' it
|
|
|
+ //can run job4's reduce.
|
|
|
assertNull(scheduler.assignTasks(tracker("tt1")));
|
|
|
- assertNull(scheduler.assignTasks(tracker("tt2")));
|
|
|
- //finish maps.
|
|
|
- taskTrackerManager.finishTask("tt1", "attempt_test_0003_m_000001_0",
|
|
|
- job3);
|
|
|
- taskTrackerManager.finishTask("tt2", "attempt_test_0004_m_000001_0",
|
|
|
- job4);
|
|
|
- //check speculative reduce code path is covered.
|
|
|
- assertEquals("Pending reduces not zero for high " +
|
|
|
- "ram job with speculative reduce.", 0, job3.pendingReduces());
|
|
|
- //if tt2 returns back it is not given any task even if it can schedule
|
|
|
- //job2 reduce.
|
|
|
- assertNull(scheduler.assignTasks(tracker("tt2")));
|
|
|
- //speculative reduce of the job3 would be scheduled.
|
|
|
- checkAssignment("tt1", "attempt_test_0003_r_000001_1 on tt1");
|
|
|
- //now both speculative and actual task have been scheduled for job3.
|
|
|
- //Normal task of Job4 would now be scheduled on TT1 as it has free space
|
|
|
- //to run.
|
|
|
+ //TT2 now gets speculative map of the job1
|
|
|
+ checkAssignment("tt2", "attempt_test_0003_r_000001_1 on tt2");
|
|
|
+
|
|
|
+ // Now since j3 has no more speculative reduces, it can schedule
|
|
|
+ // the j4.
|
|
|
checkAssignment("tt1", "attempt_test_0004_r_000001_0 on tt1");
|
|
|
- //No more tasks.
|
|
|
- assertNull(scheduler.assignTasks(tracker("tt2")));
|
|
|
- assertNull(scheduler.assignTasks(tracker("tt1")));
|
|
|
-
|
|
|
- //finish all the reduces.
|
|
|
- taskTrackerManager.finishTask("tt1", "attempt_test_0003_r_000001_1",
|
|
|
- job3);
|
|
|
- taskTrackerManager.finishTask("tt2", "attempt_test_0003_r_000001_0",
|
|
|
- job3);
|
|
|
- //finish the job
|
|
|
- taskTrackerManager.finalizeJob(job3);
|
|
|
- //finish the task and the job.
|
|
|
- taskTrackerManager.finishTask("tt1", "attempt_test_0004_r_000001_0",
|
|
|
- job4);
|
|
|
- taskTrackerManager.finalizeJob(job4);
|
|
|
-
|
|
|
}
|
|
|
|
|
|
private void checkFailedInitializedJobMovement() throws IOException {
|