From 767a3fd0b718a1d0b38c030438f0be4f426ba5b5 Mon Sep 17 00:00:00 2001 From: Ivan Mitic Date: Sun, 19 Oct 2014 18:57:48 -0400 Subject: [PATCH] MAPREDUCE-5911. Terasort TeraOutputFormat does not check for output directory existance. Contributed by Bruno P. Kinoshita. (cherry picked from commit 7bbda6ef92e9bf4a28e67b8736067b38defab51e) --- hadoop-mapreduce-project/CHANGES.txt | 3 +++ .../apache/hadoop/examples/terasort/TeraOutputFormat.java | 6 ++++++ 2 files changed, 9 insertions(+) diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt index ba8c673f615..05280f94bc9 100644 --- a/hadoop-mapreduce-project/CHANGES.txt +++ b/hadoop-mapreduce-project/CHANGES.txt @@ -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 diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraOutputFormat.java index a4a67fbe695..ab43d25b797 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraOutputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraOutputFormat.java @@ -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 getRecordWriter(TaskAttemptContext job