|
@@ -26,6 +26,7 @@ import static org.mockito.Mockito.when;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
|
|
|
+import junit.framework.Assert;
|
|
|
import junit.framework.TestCase;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
@@ -35,12 +36,21 @@ import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.mapreduce.MRJobConfig;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
|
|
+import org.apache.hadoop.mapreduce.v2.api.records.JobState;
|
|
|
+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.event.JobFinishEvent;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocator;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent;
|
|
|
+import org.apache.hadoop.security.UserGroupInformation;
|
|
|
+import org.apache.hadoop.yarn.YarnException;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.event.EventHandler;
|
|
|
import org.apache.hadoop.yarn.factories.RecordFactory;
|
|
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
|
|
+import org.apache.hadoop.yarn.service.AbstractService;
|
|
|
import org.apache.hadoop.yarn.util.BuilderUtils;
|
|
|
import org.junit.Test;
|
|
|
|
|
@@ -103,4 +113,89 @@ import org.junit.Test;
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private final class MRAppTestCleanup extends MRApp {
|
|
|
+ boolean stoppedContainerAllocator;
|
|
|
+ boolean cleanedBeforeContainerAllocatorStopped;
|
|
|
+
|
|
|
+ public MRAppTestCleanup(int maps, int reduces, boolean autoComplete,
|
|
|
+ String testName, boolean cleanOnStart) {
|
|
|
+ super(maps, reduces, autoComplete, testName, cleanOnStart);
|
|
|
+ stoppedContainerAllocator = false;
|
|
|
+ cleanedBeforeContainerAllocatorStopped = 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
|
|
|
+ protected ContainerAllocator createContainerAllocator(
|
|
|
+ ClientService clientService, AppContext context) {
|
|
|
+ return new TestCleanupContainerAllocator();
|
|
|
+ }
|
|
|
+
|
|
|
+ private class TestCleanupContainerAllocator extends AbstractService
|
|
|
+ implements ContainerAllocator {
|
|
|
+ private MRAppContainerAllocator allocator;
|
|
|
+
|
|
|
+ TestCleanupContainerAllocator() {
|
|
|
+ super(TestCleanupContainerAllocator.class.getName());
|
|
|
+ allocator = new MRAppContainerAllocator();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void handle(ContainerAllocatorEvent event) {
|
|
|
+ allocator.handle(event);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public synchronized void stop() {
|
|
|
+ stoppedContainerAllocator = true;
|
|
|
+ super.stop();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void cleanupStagingDir() throws IOException {
|
|
|
+ cleanedBeforeContainerAllocatorStopped = !stoppedContainerAllocator;
|
|
|
+ }
|
|
|
+
|
|
|
+ @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.cleanedBeforeContainerAllocatorStopped) {
|
|
|
+ Thread.sleep(100);
|
|
|
+ waitTime -= 100;
|
|
|
+ }
|
|
|
+ Assert.assertTrue("Staging directory not cleaned before notifying RM",
|
|
|
+ app.cleanedBeforeContainerAllocatorStopped);
|
|
|
+ }
|
|
|
}
|