|
@@ -22,6 +22,7 @@ import static org.mockito.Mockito.times;
|
|
|
import static org.mockito.Mockito.verify;
|
|
|
import static org.mockito.Mockito.spy;
|
|
|
|
|
|
+import java.io.IOException;
|
|
|
import java.util.Iterator;
|
|
|
|
|
|
import junit.framework.Assert;
|
|
@@ -35,11 +36,14 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
|
|
|
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.JobFinishEvent;
|
|
|
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.job.event.TaskEvent;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
|
|
|
+import org.apache.hadoop.security.UserGroupInformation;
|
|
|
+import org.apache.hadoop.yarn.YarnException;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
/**
|
|
@@ -233,6 +237,71 @@ public class TestMRApp {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private final class MRAppTestCleanup extends MRApp {
|
|
|
+ boolean hasStopped;
|
|
|
+ boolean cleanedBeforeStopped;
|
|
|
+
|
|
|
+ public MRAppTestCleanup(int maps, int reduces, boolean autoComplete,
|
|
|
+ String testName, boolean cleanOnStart) {
|
|
|
+ super(maps, reduces, autoComplete, testName, cleanOnStart);
|
|
|
+ hasStopped = false;
|
|
|
+ cleanedBeforeStopped = false;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ protected Job createJob(Configuration conf) {
|
|
|
+ UserGroupInformation currentUser = null;
|
|
|
+ try {
|
|
|
+ currentUser = UserGroupInformation.getCurrentUser();
|
|
|
+ } catch (IOException e) {
|
|
|
+ throw new YarnException(e);
|
|
|
+ }
|
|
|
+ Job newJob = new TestJob(getJobId(), getAttemptID(), conf,
|
|
|
+ getDispatcher().getEventHandler(),
|
|
|
+ getTaskAttemptListener(), getContext().getClock(),
|
|
|
+ getCommitter(), isNewApiCommitter(),
|
|
|
+ currentUser.getUserName(), getContext());
|
|
|
+ ((AppContext) getContext()).getAllJobs().put(newJob.getID(), newJob);
|
|
|
+
|
|
|
+ getDispatcher().register(JobFinishEvent.Type.class,
|
|
|
+ createJobFinishEventHandler());
|
|
|
+
|
|
|
+ return newJob;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void cleanupStagingDir() throws IOException {
|
|
|
+ cleanedBeforeStopped = !hasStopped;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public synchronized void stop() {
|
|
|
+ hasStopped = true;
|
|
|
+ super.stop();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ protected void sysexit() {
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testStagingCleanupOrder() throws Exception {
|
|
|
+ MRAppTestCleanup app = new MRAppTestCleanup(1, 1, true,
|
|
|
+ this.getClass().getName(), true);
|
|
|
+ JobImpl job = (JobImpl)app.submit(new Configuration());
|
|
|
+ app.waitForState(job, JobState.SUCCEEDED);
|
|
|
+ app.verifyCompleted();
|
|
|
+
|
|
|
+ int waitTime = 20 * 1000;
|
|
|
+ while (waitTime > 0 && !app.cleanedBeforeStopped) {
|
|
|
+ Thread.sleep(100);
|
|
|
+ waitTime -= 100;
|
|
|
+ }
|
|
|
+ Assert.assertTrue("Staging directory not cleaned before notifying RM",
|
|
|
+ app.cleanedBeforeStopped);
|
|
|
+ }
|
|
|
+
|
|
|
public static void main(String[] args) throws Exception {
|
|
|
TestMRApp t = new TestMRApp();
|
|
|
t.testMapReduce();
|
|
@@ -241,5 +310,6 @@ public class TestMRApp {
|
|
|
t.testCompletedMapsForReduceSlowstart();
|
|
|
t.testJobError();
|
|
|
t.testCountersOnJobFinish();
|
|
|
+ t.testStagingCleanupOrder();
|
|
|
}
|
|
|
}
|