public void testAbort() throws IOException {
    JobConf job = new JobConf();
    setConfForFileOutputCommitter(job);
    JobContext jContext = new JobContextImpl(job, taskID.getJobID());
    TaskAttemptContext tContext = new TaskAttemptContextImpl(job, taskID);
    FileOutputCommitter committer = new FileOutputCommitter();
    FileOutputFormat.setWorkOutputPath(job, committer.getTempTaskOutputPath(tContext));

    // do setup
    committer.setupJob(jContext);
    committer.setupTask(tContext);
    String file = "test.txt";

    // A reporter that does nothing
    Reporter reporter = Reporter.NULL;
    // write output
    FileSystem localFs = FileSystem.getLocal(job);
    TextOutputFormat theOutputFormat = new TextOutputFormat();
    RecordWriter theRecordWriter = theOutputFormat.getRecordWriter(localFs, job, file, reporter);
    writeOutput(theRecordWriter, reporter);

    // do abort
    committer.abortTask(tContext);
    File expectedFile =
        new File(new Path(committer.getTempTaskOutputPath(tContext), file).toString());
    assertFalse("task temp dir still exists", expectedFile.exists());

    committer.abortJob(jContext, JobStatus.State.FAILED);
    expectedFile = new File(new Path(outDir, FileOutputCommitter.TEMP_DIR_NAME).toString());
    assertFalse("job temp dir still exists", expectedFile.exists());
    assertEquals("Output directory not empty", 0, new File(outDir.toString()).listFiles().length);
    FileUtil.fullyDelete(new File(outDir.toString()));
  }
  private void testAbortInternal(int version) throws IOException, InterruptedException {
    JobConf conf = new JobConf();
    FileOutputFormat.setOutputPath(conf, outDir);
    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());
    TaskAttemptContext tContext = new TaskAttemptContextImpl(conf, taskID);
    FileOutputCommitter committer = new FileOutputCommitter();

    // do setup
    committer.setupJob(jContext);
    committer.setupTask(tContext);

    // write output
    TextOutputFormat theOutputFormat = new TextOutputFormat();
    RecordWriter theRecordWriter = theOutputFormat.getRecordWriter(null, conf, partFile, null);
    writeOutput(theRecordWriter, tContext);

    // do abort
    committer.abortTask(tContext);
    File out = new File(outDir.toUri().getPath());
    Path workPath = committer.getWorkPath(tContext, outDir);
    File wp = new File(workPath.toUri().getPath());
    File expectedFile = new File(wp, partFile);
    assertFalse("task temp dir still exists", expectedFile.exists());

    committer.abortJob(jContext, JobStatus.State.FAILED);
    expectedFile = new File(out, FileOutputCommitter.TEMP_DIR_NAME);
    assertFalse("job temp dir still exists", expectedFile.exists());
    assertEquals("Output directory not empty", 0, out.listFiles().length);
    FileUtil.fullyDelete(out);
  }
  private void testCommitterInternal(int version) throws Exception {
    JobConf conf = new JobConf();
    FileOutputFormat.setOutputPath(conf, outDir);
    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());
    TaskAttemptContext tContext = new TaskAttemptContextImpl(conf, taskID);
    FileOutputCommitter committer = new FileOutputCommitter();

    // setup
    committer.setupJob(jContext);
    committer.setupTask(tContext);

    // write output
    TextOutputFormat theOutputFormat = new TextOutputFormat();
    RecordWriter theRecordWriter = theOutputFormat.getRecordWriter(null, conf, partFile, null);
    writeOutput(theRecordWriter, tContext);

    // do commit
    if (committer.needsTaskCommit(tContext)) {
      committer.commitTask(tContext);
    }
    committer.commitJob(jContext);

    // validate output
    validateContent(outDir);
    FileUtil.fullyDelete(new File(outDir.toString()));
  }
  public void testFailAbort() throws IOException {
    JobConf job = new JobConf();
    job.set(FileSystem.FS_DEFAULT_NAME_KEY, "faildel:///");
    job.setClass("fs.faildel.impl", FakeFileSystem.class, FileSystem.class);
    setConfForFileOutputCommitter(job);
    JobContext jContext = new JobContextImpl(job, taskID.getJobID());
    TaskAttemptContext tContext = new TaskAttemptContextImpl(job, taskID);
    FileOutputCommitter committer = new FileOutputCommitter();
    FileOutputFormat.setWorkOutputPath(job, committer.getTempTaskOutputPath(tContext));

    // do setup
    committer.setupJob(jContext);
    committer.setupTask(tContext);
    String file = "test.txt";

    // A reporter that does nothing
    Reporter reporter = Reporter.NULL;
    // write output
    FileSystem localFs = new FakeFileSystem();
    TextOutputFormat theOutputFormat = new TextOutputFormat();
    RecordWriter theRecordWriter = theOutputFormat.getRecordWriter(localFs, job, file, reporter);
    writeOutput(theRecordWriter, reporter);

    // do abort
    Throwable th = null;
    try {
      committer.abortTask(tContext);
    } catch (IOException ie) {
      th = ie;
    }
    assertNotNull(th);
    assertTrue(th instanceof IOException);
    assertTrue(th.getMessage().contains("fake delete failed"));
    File jobTmpDir = new File(new Path(outDir, FileOutputCommitter.TEMP_DIR_NAME).toString());
    File taskTmpDir = new File(jobTmpDir, "_" + taskID);
    File expectedFile = new File(taskTmpDir, file);
    assertTrue(expectedFile + " does not exists", expectedFile.exists());

    th = null;
    try {
      committer.abortJob(jContext, JobStatus.State.FAILED);
    } catch (IOException ie) {
      th = ie;
    }
    assertNotNull(th);
    assertTrue(th instanceof IOException);
    assertTrue(th.getMessage().contains("fake delete failed"));
    assertTrue("job temp dir does not exists", jobTmpDir.exists());
  }
  @SuppressWarnings("unchecked")
  public void testCommitter() throws Exception {
    JobConf job = new JobConf();
    setConfForFileOutputCommitter(job);
    JobContext jContext = new JobContextImpl(job, taskID.getJobID());
    TaskAttemptContext tContext = new TaskAttemptContextImpl(job, taskID);
    FileOutputCommitter committer = new FileOutputCommitter();
    FileOutputFormat.setWorkOutputPath(job, committer.getTempTaskOutputPath(tContext));

    committer.setupJob(jContext);
    committer.setupTask(tContext);
    String file = "test.txt";

    // A reporter that does nothing
    Reporter reporter = Reporter.NULL;
    // write output
    FileSystem localFs = FileSystem.getLocal(job);
    TextOutputFormat theOutputFormat = new TextOutputFormat();
    RecordWriter theRecordWriter = theOutputFormat.getRecordWriter(localFs, job, file, reporter);
    writeOutput(theRecordWriter, reporter);

    // do commit
    committer.commitTask(tContext);
    committer.commitJob(jContext);

    // validate output
    File expectedFile = new File(new Path(outDir, file).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 = UtilsForTests.slurp(expectedFile);
    assertEquals(output, expectedOutput.toString());

    FileUtil.fullyDelete(new File(outDir.toString()));
  }
  private void testRecoveryInternal(int commitVersion, int recoveryVersion) throws Exception {
    JobConf conf = new JobConf();
    FileOutputFormat.setOutputPath(conf, outDir);
    conf.set(JobContext.TASK_ATTEMPT_ID, attempt);
    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());
    TaskAttemptContext tContext = new TaskAttemptContextImpl(conf, taskID);
    FileOutputCommitter committer = new FileOutputCommitter();

    // setup
    committer.setupJob(jContext);
    committer.setupTask(tContext);

    // write output
    TextOutputFormat theOutputFormat = new TextOutputFormat();
    RecordWriter theRecordWriter = theOutputFormat.getRecordWriter(null, conf, partFile, null);
    writeOutput(theRecordWriter, tContext);

    // do commit
    if (committer.needsTaskCommit(tContext)) {
      committer.commitTask(tContext);
    }

    Path jobTempDir1 = committer.getCommittedTaskPath(tContext);
    File jtd1 = new File(jobTempDir1.toUri().getPath());
    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
    JobConf conf2 = new JobConf(conf);
    conf2.set(JobContext.TASK_ATTEMPT_ID, attempt);
    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());
    TaskAttemptContext tContext2 = new TaskAttemptContextImpl(conf2, taskID);
    FileOutputCommitter committer2 = new FileOutputCommitter();
    committer2.setupJob(jContext2);

    committer2.recoverTask(tContext2);

    Path jobTempDir2 = committer2.getCommittedTaskPath(tContext2);
    File jtd2 = new File(jobTempDir2.toUri().getPath());
    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);
    validateContent(outDir);
    FileUtil.fullyDelete(new File(outDir.toString()));
  }
  private void testFailAbortInternal(int version) throws IOException, InterruptedException {
    JobConf conf = new JobConf();
    conf.set(FileSystem.FS_DEFAULT_NAME_KEY, "faildel:///");
    conf.setClass("fs.faildel.impl", FakeFileSystem.class, FileSystem.class);
    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);
    FileOutputFormat.setOutputPath(conf, outDir);
    JobContext jContext = new JobContextImpl(conf, taskID.getJobID());
    TaskAttemptContext tContext = new TaskAttemptContextImpl(conf, taskID);
    FileOutputCommitter committer = new FileOutputCommitter();

    // do setup
    committer.setupJob(jContext);
    committer.setupTask(tContext);

    // write output
    File jobTmpDir =
        new File(
            new Path(
                    outDir,
                    FileOutputCommitter.TEMP_DIR_NAME
                        + Path.SEPARATOR
                        + conf.getInt(MRConstants.APPLICATION_ATTEMPT_ID, 0)
                        + Path.SEPARATOR
                        + FileOutputCommitter.TEMP_DIR_NAME)
                .toString());
    File taskTmpDir = new File(jobTmpDir, "_" + taskID);
    File expectedFile = new File(taskTmpDir, partFile);
    TextOutputFormat<?, ?> theOutputFormat = new TextOutputFormat();
    RecordWriter<?, ?> theRecordWriter =
        theOutputFormat.getRecordWriter(null, conf, expectedFile.getAbsolutePath(), null);
    writeOutput(theRecordWriter, tContext);

    // do abort
    Throwable th = null;
    try {
      committer.abortTask(tContext);
    } catch (IOException ie) {
      th = ie;
    }
    assertNotNull(th);
    assertTrue(th instanceof IOException);
    assertTrue(th.getMessage().contains("fake delete failed"));
    assertTrue(expectedFile + " does not exists", expectedFile.exists());

    th = null;
    try {
      committer.abortJob(jContext, JobStatus.State.FAILED);
    } catch (IOException ie) {
      th = ie;
    }
    assertNotNull(th);
    assertTrue(th instanceof IOException);
    assertTrue(th.getMessage().contains("fake delete failed"));
    assertTrue("job temp dir does not exists", jobTmpDir.exists());
    FileUtil.fullyDelete(new File(outDir.toString()));
  }