MAPREDUCE-5911. Terasort TeraOutputFormat does not check for output directory existance. Contributed by Bruno P. Kinoshita.
This commit is contained in:
parent
8256766498
commit
7bbda6ef92
@ -438,6 +438,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
|
||||
|
@ -24,6 +24,7 @@
|
||||
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 void checkOutputSpecs(JobContext job
|
||||
// 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
|
||||
|
Loading…
Reference in New Issue
Block a user