|
@@ -18,6 +18,9 @@
|
|
|
|
|
|
package org.apache.hadoop.mapreduce.v2.app;
|
|
|
|
|
|
+import java.io.File;
|
|
|
+import java.io.FileInputStream;
|
|
|
+import java.io.IOException;
|
|
|
import java.util.Iterator;
|
|
|
|
|
|
import junit.framework.Assert;
|
|
@@ -25,10 +28,21 @@ import junit.framework.Assert;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.fs.Path;
|
|
|
+import org.apache.hadoop.io.NullWritable;
|
|
|
+import org.apache.hadoop.io.Text;
|
|
|
import org.apache.hadoop.mapreduce.MRJobConfig;
|
|
|
+import org.apache.hadoop.mapreduce.OutputCommitter;
|
|
|
+import org.apache.hadoop.mapreduce.OutputFormat;
|
|
|
+import org.apache.hadoop.mapreduce.RecordWriter;
|
|
|
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
|
|
+import org.apache.hadoop.mapreduce.TypeConverter;
|
|
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
|
|
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler;
|
|
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.AMInfo;
|
|
|
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
|
|
+import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
|
|
|
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.JobState;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
|
|
@@ -37,20 +51,34 @@ 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.TaskAttemptEvent;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
|
|
|
+import org.apache.hadoop.util.ReflectionUtils;
|
|
|
import org.apache.hadoop.yarn.event.EventHandler;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
public class TestRecovery {
|
|
|
|
|
|
private static final Log LOG = LogFactory.getLog(TestRecovery.class);
|
|
|
+ private static Path outputDir = new Path(new File("target",
|
|
|
+ TestRecovery.class.getName()).getAbsolutePath() +
|
|
|
+ Path.SEPARATOR + "out");
|
|
|
+ private static String partFile = "part-r-00000";
|
|
|
+ private Text key1 = new Text("key1");
|
|
|
+ private Text key2 = new Text("key2");
|
|
|
+ private Text val1 = new Text("val1");
|
|
|
+ private Text val2 = new Text("val2");
|
|
|
+
|
|
|
|
|
|
@Test
|
|
|
public void testCrashed() throws Exception {
|
|
|
+
|
|
|
int runCount = 0;
|
|
|
long am1StartTimeEst = System.currentTimeMillis();
|
|
|
MRApp app = new MRAppWithHistory(2, 1, false, this.getClass().getName(), true, ++runCount);
|
|
|
Configuration conf = new Configuration();
|
|
|
+ conf.setBoolean("mapred.mapper.new-api", true);
|
|
|
+ conf.setBoolean("mapred.reducer.new-api", true);
|
|
|
conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
|
|
|
+ conf.set(FileOutputFormat.OUTDIR, outputDir.toString());
|
|
|
Job job = app.submit(conf);
|
|
|
app.waitForState(job, JobState.RUNNING);
|
|
|
long jobStartTime = job.getReport().getStartTime();
|
|
@@ -135,6 +163,9 @@ public class TestRecovery {
|
|
|
app = new MRAppWithHistory(2, 1, false, this.getClass().getName(), false, ++runCount);
|
|
|
conf = new Configuration();
|
|
|
conf.setBoolean(MRJobConfig.MR_AM_JOB_RECOVERY_ENABLE, true);
|
|
|
+ conf.setBoolean("mapred.mapper.new-api", true);
|
|
|
+ conf.setBoolean("mapred.reducer.new-api", true);
|
|
|
+ conf.set(FileOutputFormat.OUTDIR, outputDir.toString());
|
|
|
conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
|
|
|
job = app.submit(conf);
|
|
|
app.waitForState(job, JobState.RUNNING);
|
|
@@ -201,7 +232,165 @@ public class TestRecovery {
|
|
|
// TODO Add verification of additional data from jobHistory - whatever was
|
|
|
// available in the failed attempt should be available here
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testOutputRecovery() throws Exception {
|
|
|
+ int runCount = 0;
|
|
|
+ MRApp app = new MRAppWithHistory(1, 2, false, this.getClass().getName(),
|
|
|
+ true, ++runCount);
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ conf.setBoolean("mapred.mapper.new-api", true);
|
|
|
+ conf.setBoolean("mapred.reducer.new-api", true);
|
|
|
+ conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
|
|
|
+ conf.set(FileOutputFormat.OUTDIR, outputDir.toString());
|
|
|
+ Job job = app.submit(conf);
|
|
|
+ app.waitForState(job, JobState.RUNNING);
|
|
|
+ Assert.assertEquals("No of tasks not correct",
|
|
|
+ 3, job.getTasks().size());
|
|
|
+ Iterator<Task> it = job.getTasks().values().iterator();
|
|
|
+ Task mapTask1 = it.next();
|
|
|
+ Task reduceTask1 = it.next();
|
|
|
+
|
|
|
+ // all maps must be running
|
|
|
+ app.waitForState(mapTask1, TaskState.RUNNING);
|
|
|
+
|
|
|
+ TaskAttempt task1Attempt1 = mapTask1.getAttempts().values().iterator()
|
|
|
+ .next();
|
|
|
+
|
|
|
+ //before sending the TA_DONE, event make sure attempt has come to
|
|
|
+ //RUNNING state
|
|
|
+ app.waitForState(task1Attempt1, TaskAttemptState.RUNNING);
|
|
|
|
|
|
+ //send the done signal to the map
|
|
|
+ app.getContext().getEventHandler().handle(
|
|
|
+ new TaskAttemptEvent(
|
|
|
+ task1Attempt1.getID(),
|
|
|
+ TaskAttemptEventType.TA_DONE));
|
|
|
+
|
|
|
+ //wait for map task to complete
|
|
|
+ app.waitForState(mapTask1, TaskState.SUCCEEDED);
|
|
|
+
|
|
|
+ app.waitForState(reduceTask1, TaskState.RUNNING);
|
|
|
+ TaskAttempt reduce1Attempt1 = reduceTask1.getAttempts().values().iterator().next();
|
|
|
+
|
|
|
+ // write output corresponding to reduce1
|
|
|
+ writeOutput(reduce1Attempt1, conf);
|
|
|
+
|
|
|
+ //send the done signal to the 1st reduce
|
|
|
+ app.getContext().getEventHandler().handle(
|
|
|
+ new TaskAttemptEvent(
|
|
|
+ reduce1Attempt1.getID(),
|
|
|
+ TaskAttemptEventType.TA_DONE));
|
|
|
+
|
|
|
+ //wait for first reduce task to complete
|
|
|
+ app.waitForState(reduceTask1, TaskState.SUCCEEDED);
|
|
|
+
|
|
|
+ //stop the app before the job completes.
|
|
|
+ app.stop();
|
|
|
+
|
|
|
+ //rerun
|
|
|
+ //in rerun the map will be recovered from previous run
|
|
|
+ app = new MRAppWithHistory(1, 2, false, this.getClass().getName(), false, ++runCount);
|
|
|
+ conf = new Configuration();
|
|
|
+ conf.setBoolean(MRJobConfig.MR_AM_JOB_RECOVERY_ENABLE, true);
|
|
|
+ conf.setBoolean("mapred.mapper.new-api", true);
|
|
|
+ conf.setBoolean("mapred.reducer.new-api", true);
|
|
|
+ conf.set(FileOutputFormat.OUTDIR, outputDir.toString());
|
|
|
+ conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
|
|
|
+ job = app.submit(conf);
|
|
|
+ app.waitForState(job, JobState.RUNNING);
|
|
|
+ Assert.assertEquals("No of tasks not correct",
|
|
|
+ 3, job.getTasks().size());
|
|
|
+ it = job.getTasks().values().iterator();
|
|
|
+ mapTask1 = it.next();
|
|
|
+ reduceTask1 = it.next();
|
|
|
+ Task reduceTask2 = it.next();
|
|
|
+
|
|
|
+ // map will be recovered, no need to send done
|
|
|
+ app.waitForState(mapTask1, TaskState.SUCCEEDED);
|
|
|
+
|
|
|
+ // first reduce will be recovered, no need to send done
|
|
|
+ app.waitForState(reduceTask1, TaskState.SUCCEEDED);
|
|
|
+
|
|
|
+ app.waitForState(reduceTask2, TaskState.RUNNING);
|
|
|
+
|
|
|
+ TaskAttempt reduce2Attempt = reduceTask2.getAttempts().values()
|
|
|
+ .iterator().next();
|
|
|
+ //before sending the TA_DONE, event make sure attempt has come to
|
|
|
+ //RUNNING state
|
|
|
+ app.waitForState(reduce2Attempt, TaskAttemptState.RUNNING);
|
|
|
+
|
|
|
+ //send the done signal to the 2nd reduce task
|
|
|
+ app.getContext().getEventHandler().handle(
|
|
|
+ new TaskAttemptEvent(
|
|
|
+ reduce2Attempt.getID(),
|
|
|
+ TaskAttemptEventType.TA_DONE));
|
|
|
+
|
|
|
+ //wait to get it completed
|
|
|
+ app.waitForState(reduceTask2, TaskState.SUCCEEDED);
|
|
|
+
|
|
|
+ app.waitForState(job, JobState.SUCCEEDED);
|
|
|
+ app.verifyCompleted();
|
|
|
+ validateOutput();
|
|
|
+ }
|
|
|
+
|
|
|
+ private void writeOutput(TaskAttempt attempt, Configuration conf)
|
|
|
+ throws Exception {
|
|
|
+ TaskAttemptContext tContext = new TaskAttemptContextImpl(conf,
|
|
|
+ TypeConverter.fromYarn(attempt.getID()));
|
|
|
+
|
|
|
+ TextOutputFormat<?, ?> theOutputFormat = new TextOutputFormat();
|
|
|
+ RecordWriter theRecordWriter = theOutputFormat
|
|
|
+ .getRecordWriter(tContext);
|
|
|
+
|
|
|
+ NullWritable nullWritable = NullWritable.get();
|
|
|
+ try {
|
|
|
+ theRecordWriter.write(key1, val1);
|
|
|
+ theRecordWriter.write(null, nullWritable);
|
|
|
+ theRecordWriter.write(null, val1);
|
|
|
+ theRecordWriter.write(nullWritable, val2);
|
|
|
+ theRecordWriter.write(key2, nullWritable);
|
|
|
+ theRecordWriter.write(key1, null);
|
|
|
+ theRecordWriter.write(null, null);
|
|
|
+ theRecordWriter.write(key2, val2);
|
|
|
+ } finally {
|
|
|
+ theRecordWriter.close(tContext);
|
|
|
+ }
|
|
|
+
|
|
|
+ OutputFormat outputFormat = ReflectionUtils.newInstance(
|
|
|
+ tContext.getOutputFormatClass(), conf);
|
|
|
+ OutputCommitter committer = outputFormat.getOutputCommitter(tContext);
|
|
|
+ committer.commitTask(tContext);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void validateOutput() throws IOException {
|
|
|
+ File expectedFile = new File(new Path(outputDir, partFile).toString());
|
|
|
+ StringBuffer expectedOutput = new StringBuffer();
|
|
|
+ expectedOutput.append(key1).append('\t').append(val1).append("\n");
|
|
|
+ expectedOutput.append(val1).append("\n");
|
|
|
+ expectedOutput.append(val2).append("\n");
|
|
|
+ expectedOutput.append(key2).append("\n");
|
|
|
+ expectedOutput.append(key1).append("\n");
|
|
|
+ expectedOutput.append(key2).append('\t').append(val2).append("\n");
|
|
|
+ String output = slurp(expectedFile);
|
|
|
+ Assert.assertEquals(output, expectedOutput.toString());
|
|
|
+ }
|
|
|
+
|
|
|
+ public static String slurp(File f) throws IOException {
|
|
|
+ int len = (int) f.length();
|
|
|
+ byte[] buf = new byte[len];
|
|
|
+ FileInputStream in = new FileInputStream(f);
|
|
|
+ String contents = null;
|
|
|
+ try {
|
|
|
+ in.read(buf, 0, len);
|
|
|
+ contents = new String(buf, "UTF-8");
|
|
|
+ } finally {
|
|
|
+ in.close();
|
|
|
+ }
|
|
|
+ return contents;
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
class MRAppWithHistory extends MRApp {
|
|
|
public MRAppWithHistory(int maps, int reduces, boolean autoComplete,
|
|
|
String testName, boolean cleanOnStart, int startCount) {
|