|
@@ -20,6 +20,7 @@ package org.apache.hadoop.mapreduce.lib.output;
|
|
|
|
|
|
import java.io.File;
|
|
|
import java.io.FileInputStream;
|
|
|
+import java.io.FileNotFoundException;
|
|
|
import java.io.IOException;
|
|
|
import java.net.URI;
|
|
|
import java.util.concurrent.Callable;
|
|
@@ -28,6 +29,7 @@ import java.util.concurrent.Executors;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
import junit.framework.TestCase;
|
|
|
+import org.junit.Assert;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
@@ -283,6 +285,174 @@ public class TestFileOutputCommitter extends TestCase {
|
|
|
public void testCommitterV2() throws Exception {
|
|
|
testCommitterInternal(2);
|
|
|
}
|
|
|
+
|
|
|
+ public void testCommitterWithDuplicatedCommitV1() throws Exception {
|
|
|
+ testCommitterWithDuplicatedCommitInternal(1);
|
|
|
+ }
|
|
|
+
|
|
|
+ public void testCommitterWithDuplicatedCommitV2() throws Exception {
|
|
|
+ testCommitterWithDuplicatedCommitInternal(2);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void testCommitterWithDuplicatedCommitInternal(int version) throws
|
|
|
+ Exception {
|
|
|
+ Job job = Job.getInstance();
|
|
|
+ FileOutputFormat.setOutputPath(job, outDir);
|
|
|
+ Configuration conf = job.getConfiguration();
|
|
|
+ conf.set(MRJobConfig.TASK_ATTEMPT_ID, attempt);
|
|
|
+ conf.setInt(FileOutputCommitter.FILEOUTPUTCOMMITTER_ALGORITHM_VERSION,
|
|
|
+ version);
|
|
|
+ JobContext jContext = new JobContextImpl(conf, taskID.getJobID());
|
|
|
+ TaskAttemptContext tContext = new TaskAttemptContextImpl(conf, taskID);
|
|
|
+ FileOutputCommitter committer = new FileOutputCommitter(outDir, tContext);
|
|
|
+
|
|
|
+ // setup
|
|
|
+ committer.setupJob(jContext);
|
|
|
+ committer.setupTask(tContext);
|
|
|
+
|
|
|
+ // write output
|
|
|
+ TextOutputFormat theOutputFormat = new TextOutputFormat();
|
|
|
+ RecordWriter theRecordWriter = theOutputFormat.getRecordWriter(tContext);
|
|
|
+ writeOutput(theRecordWriter, tContext);
|
|
|
+
|
|
|
+ // do commit
|
|
|
+ committer.commitTask(tContext);
|
|
|
+ committer.commitJob(jContext);
|
|
|
+
|
|
|
+ // validate output
|
|
|
+ validateContent(outDir);
|
|
|
+
|
|
|
+ // commit job again on a successful commit job.
|
|
|
+ try {
|
|
|
+ committer.commitJob(jContext);
|
|
|
+ if (version == 1) {
|
|
|
+ Assert.fail("Duplicate commit success: wrong behavior for version 1.");
|
|
|
+ }
|
|
|
+ } catch (IOException e) {
|
|
|
+ if (version == 2) {
|
|
|
+ Assert.fail("Duplicate commit failed: wrong behavior for version 2.");
|
|
|
+ }
|
|
|
+ }
|
|
|
+ FileUtil.fullyDelete(new File(outDir.toString()));
|
|
|
+ }
|
|
|
+
|
|
|
+ public void testCommitterWithFailureV1() throws Exception {
|
|
|
+ testCommitterWithFailureInternal(1, 1);
|
|
|
+ testCommitterWithFailureInternal(1, 2);
|
|
|
+ }
|
|
|
+
|
|
|
+ public void testCommitterWithFailureV2() throws Exception {
|
|
|
+ testCommitterWithFailureInternal(2, 1);
|
|
|
+ testCommitterWithFailureInternal(2, 2);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void testCommitterWithFailureInternal(int version, int maxAttempts)
|
|
|
+ throws Exception {
|
|
|
+ Job job = Job.getInstance();
|
|
|
+ FileOutputFormat.setOutputPath(job, outDir);
|
|
|
+ Configuration conf = job.getConfiguration();
|
|
|
+ conf.set(MRJobConfig.TASK_ATTEMPT_ID, attempt);
|
|
|
+ conf.setInt(FileOutputCommitter.FILEOUTPUTCOMMITTER_ALGORITHM_VERSION,
|
|
|
+ version);
|
|
|
+ conf.setInt(FileOutputCommitter.FILEOUTPUTCOMMITTER_FAILURE_ATTEMPTS,
|
|
|
+ maxAttempts);
|
|
|
+
|
|
|
+ JobContext jContext = new JobContextImpl(conf, taskID.getJobID());
|
|
|
+ TaskAttemptContext tContext = new TaskAttemptContextImpl(conf, taskID);
|
|
|
+ FileOutputCommitter committer = new CommitterWithFailedThenSucceed(outDir,
|
|
|
+ tContext);
|
|
|
+
|
|
|
+ // setup
|
|
|
+ committer.setupJob(jContext);
|
|
|
+ committer.setupTask(tContext);
|
|
|
+
|
|
|
+ // write output
|
|
|
+ TextOutputFormat theOutputFormat = new TextOutputFormat();
|
|
|
+ RecordWriter theRecordWriter = theOutputFormat.getRecordWriter(tContext);
|
|
|
+ writeOutput(theRecordWriter, tContext);
|
|
|
+
|
|
|
+ // do commit
|
|
|
+ committer.commitTask(tContext);
|
|
|
+
|
|
|
+ try {
|
|
|
+ committer.commitJob(jContext);
|
|
|
+ // (1,1), (1,2), (2,1) shouldn't reach to here.
|
|
|
+ if (version == 1 || maxAttempts <= 1) {
|
|
|
+ Assert.fail("Commit successful: wrong behavior for version 1.");
|
|
|
+ }
|
|
|
+ } catch (IOException e) {
|
|
|
+ // (2,2) shouldn't reach to here.
|
|
|
+ if (version == 2 && maxAttempts > 2) {
|
|
|
+ Assert.fail("Commit failed: wrong behavior for version 2.");
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ FileUtil.fullyDelete(new File(outDir.toString()));
|
|
|
+ }
|
|
|
+
|
|
|
+ public void testCommitterRepeatableV1() throws Exception {
|
|
|
+ testCommitterRetryInternal(1);
|
|
|
+ }
|
|
|
+
|
|
|
+ public void testCommitterRepeatableV2() throws Exception {
|
|
|
+ testCommitterRetryInternal(2);
|
|
|
+ }
|
|
|
+
|
|
|
+ // retry committer for 2 times.
|
|
|
+ private void testCommitterRetryInternal(int version)
|
|
|
+ throws Exception {
|
|
|
+ Job job = Job.getInstance();
|
|
|
+ FileOutputFormat.setOutputPath(job, outDir);
|
|
|
+ Configuration conf = job.getConfiguration();
|
|
|
+ conf.set(MRJobConfig.TASK_ATTEMPT_ID, attempt);
|
|
|
+ conf.setInt(FileOutputCommitter.FILEOUTPUTCOMMITTER_ALGORITHM_VERSION,
|
|
|
+ version);
|
|
|
+ // only attempt for 1 time.
|
|
|
+ conf.setInt(FileOutputCommitter.FILEOUTPUTCOMMITTER_FAILURE_ATTEMPTS,
|
|
|
+ 1);
|
|
|
+
|
|
|
+ JobContext jContext = new JobContextImpl(conf, taskID.getJobID());
|
|
|
+ TaskAttemptContext tContext = new TaskAttemptContextImpl(conf, taskID);
|
|
|
+ FileOutputCommitter committer = new CommitterWithFailedThenSucceed(outDir,
|
|
|
+ tContext);
|
|
|
+
|
|
|
+ // setup
|
|
|
+ committer.setupJob(jContext);
|
|
|
+ committer.setupTask(tContext);
|
|
|
+
|
|
|
+ // write output
|
|
|
+ TextOutputFormat theOutputFormat = new TextOutputFormat();
|
|
|
+ RecordWriter theRecordWriter = theOutputFormat.getRecordWriter(tContext);
|
|
|
+ writeOutput(theRecordWriter, tContext);
|
|
|
+
|
|
|
+ // do commit
|
|
|
+ committer.commitTask(tContext);
|
|
|
+
|
|
|
+ try {
|
|
|
+ committer.commitJob(jContext);
|
|
|
+ Assert.fail("Commit successful: wrong behavior for the first time " +
|
|
|
+ "commit.");
|
|
|
+ } catch (IOException e) {
|
|
|
+ // commit again.
|
|
|
+ try {
|
|
|
+ committer.commitJob(jContext);
|
|
|
+ // version 1 shouldn't reach to here.
|
|
|
+ if (version == 1) {
|
|
|
+ Assert.fail("Commit successful after retry: wrong behavior for " +
|
|
|
+ "version 1.");
|
|
|
+ }
|
|
|
+ } catch (FileNotFoundException ex) {
|
|
|
+ if (version == 2) {
|
|
|
+ Assert.fail("Commit failed after retry: wrong behavior for" +
|
|
|
+ " version 2.");
|
|
|
+ }
|
|
|
+ assertTrue(ex.getMessage().contains(committer.getJobAttemptPath(
|
|
|
+ jContext).toString() + " does not exist"));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ FileUtil.fullyDelete(new File(outDir.toString()));
|
|
|
+ }
|
|
|
|
|
|
private void testMapFileOutputCommitterInternal(int version)
|
|
|
throws Exception {
|
|
@@ -292,7 +462,7 @@ public class TestFileOutputCommitter extends TestCase {
|
|
|
conf.set(MRJobConfig.TASK_ATTEMPT_ID, attempt);
|
|
|
conf.setInt(FileOutputCommitter.FILEOUTPUTCOMMITTER_ALGORITHM_VERSION,
|
|
|
version);
|
|
|
- JobContext jContext = new JobContextImpl(conf, taskID.getJobID());
|
|
|
+ JobContext jContext = new JobContextImpl(conf, taskID.getJobID());
|
|
|
TaskAttemptContext tContext = new TaskAttemptContextImpl(conf, taskID);
|
|
|
FileOutputCommitter committer = new FileOutputCommitter(outDir, tContext);
|
|
|
|
|
@@ -584,4 +754,29 @@ public class TestFileOutputCommitter extends TestCase {
|
|
|
return contents;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * The class provides a overrided implementation of commitJobInternal which
|
|
|
+ * causes the commit failed for the first time then succeed.
|
|
|
+ */
|
|
|
+ public static class CommitterWithFailedThenSucceed extends
|
|
|
+ FileOutputCommitter {
|
|
|
+ boolean firstTimeFail = true;
|
|
|
+
|
|
|
+ public CommitterWithFailedThenSucceed(Path outputPath,
|
|
|
+ JobContext context) throws IOException {
|
|
|
+ super(outputPath, context);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ protected void commitJobInternal(JobContext context) throws IOException {
|
|
|
+ super.commitJobInternal(context);
|
|
|
+ if (firstTimeFail) {
|
|
|
+ firstTimeFail = false;
|
|
|
+ throw new IOException();
|
|
|
+ } else {
|
|
|
+ // succeed then, nothing to do
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
}
|