MAPREDUCE-5911. Terasort TeraOutputFormat does not check for output directory existance. Contributed by Bruno P. Kinoshita.

(cherry picked from commit 7bbda6ef92)
This commit is contained in:
Ivan Mitic 2014-10-19 18:57:48 -04:00
parent 9915d52185
commit 767a3fd0b7
2 changed files with 9 additions and 0 deletions

View File

@ -219,6 +219,9 @@ Release 2.6.0 - UNRELEASED
MAPREDUCE-5542. Killing a job just as it finishes can generate an NPE in
client (Rohith via jlowe)
MAPREDUCE-5911. Terasort TeraOutputFormat does not check for output
directory existance (Bruno P. Kinoshita via ivanmi)
Release 2.5.1 - 2014-09-05
INCOMPATIBLE CHANGES

View File

@ -24,6 +24,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.FileAlreadyExistsException;
import org.apache.hadoop.mapred.InvalidJobConfException;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.OutputCommitter;
@ -90,6 +91,11 @@ public class TeraOutputFormat extends FileOutputFormat<Text,Text> {
// get delegation token for outDir's file system
TokenCache.obtainTokensForNamenodes(job.getCredentials(),
new Path[] { outDir }, job.getConfiguration());
if (outDir.getFileSystem(job.getConfiguration()).exists(outDir)) {
throw new FileAlreadyExistsException("Output directory " + outDir +
" already exists");
}
}
public RecordWriter<Text,Text> getRecordWriter(TaskAttemptContext job