|
@@ -50,7 +50,6 @@ public class TestFileOutputCommitter extends TestCase {
|
|
private Text val1 = new Text("val1");
|
|
private Text val1 = new Text("val1");
|
|
private Text val2 = new Text("val2");
|
|
private Text val2 = new Text("val2");
|
|
|
|
|
|
-
|
|
|
|
private void writeOutput(RecordWriter theRecordWriter,
|
|
private void writeOutput(RecordWriter theRecordWriter,
|
|
TaskAttemptContext context) throws IOException, InterruptedException {
|
|
TaskAttemptContext context) throws IOException, InterruptedException {
|
|
NullWritable nullWritable = NullWritable.get();
|
|
NullWritable nullWritable = NullWritable.get();
|
|
@@ -83,12 +82,16 @@ public class TestFileOutputCommitter extends TestCase {
|
|
theRecordWriter.close(null);
|
|
theRecordWriter.close(null);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
- public void testRecovery() throws Exception {
|
|
|
|
- JobConf conf = new JobConf();
|
|
|
|
|
|
+
|
|
|
|
+ private void testRecoveryInternal(int commitVersion, int recoveryVersion)
|
|
|
|
+ throws Exception {
|
|
|
|
+ JobConf conf = new JobConf();
|
|
FileOutputFormat.setOutputPath(conf, outDir);
|
|
FileOutputFormat.setOutputPath(conf, outDir);
|
|
conf.set(JobContext.TASK_ATTEMPT_ID, attempt);
|
|
conf.set(JobContext.TASK_ATTEMPT_ID, attempt);
|
|
conf.setInt(MRConstants.APPLICATION_ATTEMPT_ID, 1);
|
|
conf.setInt(MRConstants.APPLICATION_ATTEMPT_ID, 1);
|
|
|
|
+ conf.setInt(org.apache.hadoop.mapreduce.lib.output.
|
|
|
|
+ FileOutputCommitter.FILEOUTPUTCOMMITTER_ALGORITHM_VERSION,
|
|
|
|
+ commitVersion);
|
|
JobContext jContext = new JobContextImpl(conf, taskID.getJobID());
|
|
JobContext jContext = new JobContextImpl(conf, taskID.getJobID());
|
|
TaskAttemptContext tContext = new TaskAttemptContextImpl(conf, taskID);
|
|
TaskAttemptContext tContext = new TaskAttemptContextImpl(conf, taskID);
|
|
FileOutputCommitter committer = new FileOutputCommitter();
|
|
FileOutputCommitter committer = new FileOutputCommitter();
|
|
@@ -99,7 +102,7 @@ public class TestFileOutputCommitter extends TestCase {
|
|
|
|
|
|
// write output
|
|
// write output
|
|
TextOutputFormat theOutputFormat = new TextOutputFormat();
|
|
TextOutputFormat theOutputFormat = new TextOutputFormat();
|
|
- RecordWriter theRecordWriter =
|
|
|
|
|
|
+ RecordWriter theRecordWriter =
|
|
theOutputFormat.getRecordWriter(null, conf, partFile, null);
|
|
theOutputFormat.getRecordWriter(null, conf, partFile, null);
|
|
writeOutput(theRecordWriter, tContext);
|
|
writeOutput(theRecordWriter, tContext);
|
|
|
|
|
|
@@ -107,31 +110,59 @@ public class TestFileOutputCommitter extends TestCase {
|
|
if(committer.needsTaskCommit(tContext)) {
|
|
if(committer.needsTaskCommit(tContext)) {
|
|
committer.commitTask(tContext);
|
|
committer.commitTask(tContext);
|
|
}
|
|
}
|
|
|
|
+
|
|
Path jobTempDir1 = committer.getCommittedTaskPath(tContext);
|
|
Path jobTempDir1 = committer.getCommittedTaskPath(tContext);
|
|
File jtd1 = new File(jobTempDir1.toUri().getPath());
|
|
File jtd1 = new File(jobTempDir1.toUri().getPath());
|
|
- assertTrue(jtd1.exists());
|
|
|
|
- validateContent(jobTempDir1);
|
|
|
|
-
|
|
|
|
- //now while running the second app attempt,
|
|
|
|
|
|
+ if (commitVersion == 1) {
|
|
|
|
+ assertTrue("Version 1 commits to temporary dir " + jtd1, jtd1.exists());
|
|
|
|
+ validateContent(jobTempDir1);
|
|
|
|
+ } else {
|
|
|
|
+ assertFalse("Version 2 commits to output dir " + jtd1, jtd1.exists());
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ //now while running the second app attempt,
|
|
//recover the task output from first attempt
|
|
//recover the task output from first attempt
|
|
JobConf conf2 = new JobConf(conf);
|
|
JobConf conf2 = new JobConf(conf);
|
|
conf2.set(JobContext.TASK_ATTEMPT_ID, attempt);
|
|
conf2.set(JobContext.TASK_ATTEMPT_ID, attempt);
|
|
conf2.setInt(MRConstants.APPLICATION_ATTEMPT_ID, 2);
|
|
conf2.setInt(MRConstants.APPLICATION_ATTEMPT_ID, 2);
|
|
|
|
+ conf2.setInt(org.apache.hadoop.mapreduce.lib.output.
|
|
|
|
+ FileOutputCommitter.FILEOUTPUTCOMMITTER_ALGORITHM_VERSION,
|
|
|
|
+ recoveryVersion);
|
|
JobContext jContext2 = new JobContextImpl(conf2, taskID.getJobID());
|
|
JobContext jContext2 = new JobContextImpl(conf2, taskID.getJobID());
|
|
TaskAttemptContext tContext2 = new TaskAttemptContextImpl(conf2, taskID);
|
|
TaskAttemptContext tContext2 = new TaskAttemptContextImpl(conf2, taskID);
|
|
FileOutputCommitter committer2 = new FileOutputCommitter();
|
|
FileOutputCommitter committer2 = new FileOutputCommitter();
|
|
committer2.setupJob(jContext2);
|
|
committer2.setupJob(jContext2);
|
|
- Path jobTempDir2 = committer2.getCommittedTaskPath(tContext2);
|
|
|
|
-
|
|
|
|
|
|
+
|
|
committer2.recoverTask(tContext2);
|
|
committer2.recoverTask(tContext2);
|
|
|
|
+
|
|
|
|
+ Path jobTempDir2 = committer2.getCommittedTaskPath(tContext2);
|
|
File jtd2 = new File(jobTempDir2.toUri().getPath());
|
|
File jtd2 = new File(jobTempDir2.toUri().getPath());
|
|
- assertTrue(jtd2.exists());
|
|
|
|
- validateContent(jobTempDir2);
|
|
|
|
-
|
|
|
|
|
|
+ if (recoveryVersion == 1) {
|
|
|
|
+ assertTrue("Version 1 recovers to " + jtd2, jtd2.exists());
|
|
|
|
+ validateContent(jobTempDir2);
|
|
|
|
+ } else {
|
|
|
|
+ assertFalse("Version 2 commits to output dir " + jtd2, jtd2.exists());
|
|
|
|
+ if (commitVersion == 1) {
|
|
|
|
+ assertTrue("Version 2 recovery moves to output dir from "
|
|
|
|
+ + jtd1 , jtd1.list().length == 0);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
committer2.commitJob(jContext2);
|
|
committer2.commitJob(jContext2);
|
|
validateContent(outDir);
|
|
validateContent(outDir);
|
|
FileUtil.fullyDelete(new File(outDir.toString()));
|
|
FileUtil.fullyDelete(new File(outDir.toString()));
|
|
}
|
|
}
|
|
|
|
+ public void testRecoveryV1() throws Exception {
|
|
|
|
+ testRecoveryInternal(1, 1);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public void testRecoveryV2() throws Exception {
|
|
|
|
+ testRecoveryInternal(2, 2);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public void testRecoveryUpgradeV1V2() throws Exception {
|
|
|
|
+ testRecoveryInternal(1, 2);
|
|
|
|
+ }
|
|
|
|
|
|
private void validateContent(Path dir) throws IOException {
|
|
private void validateContent(Path dir) throws IOException {
|
|
File fdir = new File(dir.toUri().getPath());
|
|
File fdir = new File(dir.toUri().getPath());
|
|
@@ -170,11 +201,13 @@ public class TestFileOutputCommitter extends TestCase {
|
|
assert(fileCount > 0);
|
|
assert(fileCount > 0);
|
|
assert(dataFileFound && indexFileFound);
|
|
assert(dataFileFound && indexFileFound);
|
|
}
|
|
}
|
|
-
|
|
|
|
- public void testCommitter() throws Exception {
|
|
|
|
|
|
+
|
|
|
|
+ private void testCommitterInternal(int version) throws Exception {
|
|
JobConf conf = new JobConf();
|
|
JobConf conf = new JobConf();
|
|
FileOutputFormat.setOutputPath(conf, outDir);
|
|
FileOutputFormat.setOutputPath(conf, outDir);
|
|
conf.set(JobContext.TASK_ATTEMPT_ID, attempt);
|
|
conf.set(JobContext.TASK_ATTEMPT_ID, attempt);
|
|
|
|
+ conf.setInt(org.apache.hadoop.mapreduce.lib.output.
|
|
|
|
+ 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);
|
|
TaskAttemptContext tContext = new TaskAttemptContextImpl(conf, taskID);
|
|
FileOutputCommitter committer = new FileOutputCommitter();
|
|
FileOutputCommitter committer = new FileOutputCommitter();
|
|
@@ -200,21 +233,33 @@ public class TestFileOutputCommitter extends TestCase {
|
|
FileUtil.fullyDelete(new File(outDir.toString()));
|
|
FileUtil.fullyDelete(new File(outDir.toString()));
|
|
}
|
|
}
|
|
|
|
|
|
- public void testMapFileOutputCommitter() throws Exception {
|
|
|
|
|
|
+ public void testCommitterV1() throws Exception {
|
|
|
|
+ testCommitterInternal(1);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public void testCommitterV2() throws Exception {
|
|
|
|
+ testCommitterInternal(2);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void testMapFileOutputCommitterInternal(int version)
|
|
|
|
+ throws Exception {
|
|
JobConf conf = new JobConf();
|
|
JobConf conf = new JobConf();
|
|
FileOutputFormat.setOutputPath(conf, outDir);
|
|
FileOutputFormat.setOutputPath(conf, outDir);
|
|
conf.set(JobContext.TASK_ATTEMPT_ID, attempt);
|
|
conf.set(JobContext.TASK_ATTEMPT_ID, attempt);
|
|
|
|
+ conf.setInt(org.apache.hadoop.mapreduce.lib.output.
|
|
|
|
+ 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);
|
|
TaskAttemptContext tContext = new TaskAttemptContextImpl(conf, taskID);
|
|
- FileOutputCommitter committer = new FileOutputCommitter();
|
|
|
|
-
|
|
|
|
|
|
+ FileOutputCommitter committer = new FileOutputCommitter();
|
|
|
|
+
|
|
// setup
|
|
// setup
|
|
committer.setupJob(jContext);
|
|
committer.setupJob(jContext);
|
|
committer.setupTask(tContext);
|
|
committer.setupTask(tContext);
|
|
|
|
|
|
// write output
|
|
// write output
|
|
MapFileOutputFormat theOutputFormat = new MapFileOutputFormat();
|
|
MapFileOutputFormat theOutputFormat = new MapFileOutputFormat();
|
|
- RecordWriter theRecordWriter = theOutputFormat.getRecordWriter(null, conf, partFile, null);
|
|
|
|
|
|
+ RecordWriter theRecordWriter =
|
|
|
|
+ theOutputFormat.getRecordWriter(null, conf, partFile, null);
|
|
writeMapFileOutput(theRecordWriter, tContext);
|
|
writeMapFileOutput(theRecordWriter, tContext);
|
|
|
|
|
|
// do commit
|
|
// do commit
|
|
@@ -227,11 +272,29 @@ public class TestFileOutputCommitter extends TestCase {
|
|
validateMapFileOutputContent(FileSystem.get(conf), outDir);
|
|
validateMapFileOutputContent(FileSystem.get(conf), outDir);
|
|
FileUtil.fullyDelete(new File(outDir.toString()));
|
|
FileUtil.fullyDelete(new File(outDir.toString()));
|
|
}
|
|
}
|
|
-
|
|
|
|
- public void testMapOnlyNoOutput() throws Exception {
|
|
|
|
|
|
+
|
|
|
|
+ public void testMapFileOutputCommitterV1() throws Exception {
|
|
|
|
+ testMapFileOutputCommitterInternal(1);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public void testMapFileOutputCommitterV2() throws Exception {
|
|
|
|
+ testMapFileOutputCommitterInternal(2);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public void testMapOnlyNoOutputV1() throws Exception {
|
|
|
|
+ testMapOnlyNoOutputInternal(1);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public void testMapOnlyNoOutputV2() throws Exception {
|
|
|
|
+ testMapOnlyNoOutputInternal(2);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void testMapOnlyNoOutputInternal(int version) throws Exception {
|
|
JobConf conf = new JobConf();
|
|
JobConf conf = new JobConf();
|
|
//This is not set on purpose. FileOutputFormat.setOutputPath(conf, outDir);
|
|
//This is not set on purpose. FileOutputFormat.setOutputPath(conf, outDir);
|
|
conf.set(JobContext.TASK_ATTEMPT_ID, attempt);
|
|
conf.set(JobContext.TASK_ATTEMPT_ID, attempt);
|
|
|
|
+ conf.setInt(org.apache.hadoop.mapreduce.lib.output.
|
|
|
|
+ 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);
|
|
TaskAttemptContext tContext = new TaskAttemptContextImpl(conf, taskID);
|
|
FileOutputCommitter committer = new FileOutputCommitter();
|
|
FileOutputCommitter committer = new FileOutputCommitter();
|
|
@@ -249,11 +312,14 @@ public class TestFileOutputCommitter extends TestCase {
|
|
// validate output
|
|
// validate output
|
|
FileUtil.fullyDelete(new File(outDir.toString()));
|
|
FileUtil.fullyDelete(new File(outDir.toString()));
|
|
}
|
|
}
|
|
-
|
|
|
|
- public void testAbort() throws IOException, InterruptedException {
|
|
|
|
|
|
+
|
|
|
|
+ private void testAbortInternal(int version)
|
|
|
|
+ throws IOException, InterruptedException {
|
|
JobConf conf = new JobConf();
|
|
JobConf conf = new JobConf();
|
|
FileOutputFormat.setOutputPath(conf, outDir);
|
|
FileOutputFormat.setOutputPath(conf, outDir);
|
|
conf.set(JobContext.TASK_ATTEMPT_ID, attempt);
|
|
conf.set(JobContext.TASK_ATTEMPT_ID, attempt);
|
|
|
|
+ conf.setInt(org.apache.hadoop.mapreduce.lib.output.
|
|
|
|
+ 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);
|
|
TaskAttemptContext tContext = new TaskAttemptContextImpl(conf, taskID);
|
|
FileOutputCommitter committer = new FileOutputCommitter();
|
|
FileOutputCommitter committer = new FileOutputCommitter();
|
|
@@ -283,6 +349,14 @@ public class TestFileOutputCommitter extends TestCase {
|
|
FileUtil.fullyDelete(out);
|
|
FileUtil.fullyDelete(out);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ public void testAbortV1() throws Exception {
|
|
|
|
+ testAbortInternal(1);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public void testAbortV2() throws Exception {
|
|
|
|
+ testAbortInternal(2);
|
|
|
|
+ }
|
|
|
|
+
|
|
public static class FakeFileSystem extends RawLocalFileSystem {
|
|
public static class FakeFileSystem extends RawLocalFileSystem {
|
|
public FakeFileSystem() {
|
|
public FakeFileSystem() {
|
|
super();
|
|
super();
|
|
@@ -299,11 +373,14 @@ public class TestFileOutputCommitter extends TestCase {
|
|
}
|
|
}
|
|
|
|
|
|
|
|
|
|
- public void testFailAbort() throws IOException, InterruptedException {
|
|
|
|
|
|
+ private void testFailAbortInternal(int version)
|
|
|
|
+ throws IOException, InterruptedException {
|
|
JobConf conf = new JobConf();
|
|
JobConf conf = new JobConf();
|
|
conf.set(FileSystem.FS_DEFAULT_NAME_KEY, "faildel:///");
|
|
conf.set(FileSystem.FS_DEFAULT_NAME_KEY, "faildel:///");
|
|
conf.setClass("fs.faildel.impl", FakeFileSystem.class, FileSystem.class);
|
|
conf.setClass("fs.faildel.impl", FakeFileSystem.class, FileSystem.class);
|
|
conf.set(JobContext.TASK_ATTEMPT_ID, attempt);
|
|
conf.set(JobContext.TASK_ATTEMPT_ID, attempt);
|
|
|
|
+ conf.setInt(org.apache.hadoop.mapreduce.lib.output.
|
|
|
|
+ FileOutputCommitter.FILEOUTPUTCOMMITTER_ALGORITHM_VERSION, version);
|
|
conf.setInt(MRConstants.APPLICATION_ATTEMPT_ID, 1);
|
|
conf.setInt(MRConstants.APPLICATION_ATTEMPT_ID, 1);
|
|
FileOutputFormat.setOutputPath(conf, outDir);
|
|
FileOutputFormat.setOutputPath(conf, outDir);
|
|
JobContext jContext = new JobContextImpl(conf, taskID.getJobID());
|
|
JobContext jContext = new JobContextImpl(conf, taskID.getJobID());
|
|
@@ -353,6 +430,13 @@ public class TestFileOutputCommitter extends TestCase {
|
|
FileUtil.fullyDelete(new File(outDir.toString()));
|
|
FileUtil.fullyDelete(new File(outDir.toString()));
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ public void testFailAbortV1() throws Exception {
|
|
|
|
+ testFailAbortInternal(1);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public void testFailAbortV2() throws Exception {
|
|
|
|
+ testFailAbortInternal(2);
|
|
|
|
+ }
|
|
public static String slurp(File f) throws IOException {
|
|
public static String slurp(File f) throws IOException {
|
|
int len = (int) f.length();
|
|
int len = (int) f.length();
|
|
byte[] buf = new byte[len];
|
|
byte[] buf = new byte[len];
|