Browse Source

HADOOP-3041. Deprecate JobConf.setOutputPath and JobConf.getOutputPath. Deprecate OutputFormatBase. Add FileOutputFormat. Existing output formats extending OutputFormatBase, now extend FileOutputFormat. Add the following APIs in FileOutputFormat: setOutputPath, getOutputPath, getWorkOutputPath. Contributed by Amareshwari Sriramadasu.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@644868 13f79535-47bb-0310-9956-ffa450edef68
Nigel Daley 17 years ago
parent
commit
7f68eebd2f
71 changed files with 206 additions and 188 deletions
  1. 6 0
      CHANGES.txt
  2. 2 1
      src/contrib/data_join/src/java/org/apache/hadoop/contrib/utils/join/DataJoinJob.java
  3. 2 1
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/main/UpdateIndex.java
  4. 4 4
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/IndexUpdateOutputFormat.java
  5. 4 2
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/IndexUpdater.java
  6. 2 1
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java
  7. 21 7
      src/docs/src/documentation/content/xdocs/mapred_tutorial.xml
  8. 5 2
      src/docs/src/documentation/content/xdocs/site.xml
  9. 2 2
      src/examples/org/apache/hadoop/examples/Grep.java
  10. 2 1
      src/examples/org/apache/hadoop/examples/Join.java
  11. 2 1
      src/examples/org/apache/hadoop/examples/PiEstimator.java
  12. 2 1
      src/examples/org/apache/hadoop/examples/RandomTextWriter.java
  13. 3 2
      src/examples/org/apache/hadoop/examples/RandomWriter.java
  14. 3 2
      src/examples/org/apache/hadoop/examples/Sort.java
  15. 2 1
      src/examples/org/apache/hadoop/examples/WordCount.java
  16. 1 1
      src/examples/org/apache/hadoop/examples/dancing/DistributedPentomino.java
  17. 1 1
      src/examples/pipes/impl/wordcount-nopipe.cc
  18. 20 46
      src/java/org/apache/hadoop/mapred/JobConf.java
  19. 1 1
      src/java/org/apache/hadoop/mapred/JobHistory.java
  20. 2 2
      src/java/org/apache/hadoop/mapred/JobInProgress.java
  21. 1 1
      src/java/org/apache/hadoop/mapred/LocalJobRunner.java
  22. 4 4
      src/java/org/apache/hadoop/mapred/MapFileOutputFormat.java
  23. 5 2
      src/java/org/apache/hadoop/mapred/OutputFormatBase.java
  24. 6 4
      src/java/org/apache/hadoop/mapred/SequenceFileOutputFormat.java
  25. 4 4
      src/java/org/apache/hadoop/mapred/Task.java
  26. 1 1
      src/java/org/apache/hadoop/mapred/TaskTracker.java
  27. 2 2
      src/java/org/apache/hadoop/mapred/TextOutputFormat.java
  28. 4 3
      src/java/org/apache/hadoop/mapred/lib/MultipleOutputFormat.java
  29. 10 2
      src/java/org/apache/hadoop/mapred/lib/MultipleSequenceFileOutputFormat.java
  30. 2 1
      src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorJob.java
  31. 3 1
      src/java/org/apache/hadoop/mapred/pipes/Submitter.java
  32. 2 1
      src/java/org/apache/hadoop/tools/Logalyzer.java
  33. 2 1
      src/java/org/apache/hadoop/util/CopyFiles.java
  34. 2 1
      src/test/org/apache/hadoop/dfs/NNBench.java
  35. 1 1
      src/test/org/apache/hadoop/fs/DFSCIOTest.java
  36. 1 1
      src/test/org/apache/hadoop/fs/DistributedFSCheck.java
  37. 1 1
      src/test/org/apache/hadoop/fs/TestDFSIO.java
  38. 4 3
      src/test/org/apache/hadoop/fs/TestFileSystem.java
  39. 2 7
      src/test/org/apache/hadoop/io/FileBench.java
  40. 1 1
      src/test/org/apache/hadoop/mapred/BigMapOutput.java
  41. 2 2
      src/test/org/apache/hadoop/mapred/GenericMRLoadGenerator.java
  42. 3 5
      src/test/org/apache/hadoop/mapred/MRBench.java
  43. 1 4
      src/test/org/apache/hadoop/mapred/MRCaching.java
  44. 1 5
      src/test/org/apache/hadoop/mapred/NotificationTestCase.java
  45. 1 1
      src/test/org/apache/hadoop/mapred/PiEstimator.java
  46. 10 6
      src/test/org/apache/hadoop/mapred/SortValidator.java
  47. 1 1
      src/test/org/apache/hadoop/mapred/TestClusterMapReduceTestCase.java
  48. 1 1
      src/test/org/apache/hadoop/mapred/TestCollect.java
  49. 1 2
      src/test/org/apache/hadoop/mapred/TestComparators.java
  50. 1 1
      src/test/org/apache/hadoop/mapred/TestEmptyJobWithDFS.java
  51. 1 3
      src/test/org/apache/hadoop/mapred/TestFieldSelection.java
  52. 3 3
      src/test/org/apache/hadoop/mapred/TestJavaSerialization.java
  53. 1 1
      src/test/org/apache/hadoop/mapred/TestJobStatusPersistency.java
  54. 1 1
      src/test/org/apache/hadoop/mapred/TestMapOutputType.java
  55. 5 5
      src/test/org/apache/hadoop/mapred/TestMapRed.java
  56. 2 4
      src/test/org/apache/hadoop/mapred/TestMiniMRClasspath.java
  57. 1 1
      src/test/org/apache/hadoop/mapred/TestMiniMRMapRedDebugScript.java
  58. 1 1
      src/test/org/apache/hadoop/mapred/TestMiniMRTaskTempDir.java
  59. 1 1
      src/test/org/apache/hadoop/mapred/TestMiniMRWithDFS.java
  60. 1 1
      src/test/org/apache/hadoop/mapred/TestMultipleTextOutputFormat.java
  61. 1 1
      src/test/org/apache/hadoop/mapred/TestRackAwareTaskPlacement.java
  62. 4 3
      src/test/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java
  63. 1 1
      src/test/org/apache/hadoop/mapred/TestTextOutputFormat.java
  64. 3 3
      src/test/org/apache/hadoop/mapred/ThreadedMapBenchmark.java
  65. 2 1
      src/test/org/apache/hadoop/mapred/jobcontrol/JobControlTestUtils.java
  66. 2 1
      src/test/org/apache/hadoop/mapred/join/TestDatamerge.java
  67. 1 1
      src/test/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
  68. 1 1
      src/test/org/apache/hadoop/mapred/lib/aggregate/TestAggregates.java
  69. 2 1
      src/test/org/apache/hadoop/mapred/pipes/TestPipes.java
  70. 3 3
      src/test/org/apache/hadoop/record/TestRecordMR.java
  71. 2 2
      src/test/testshell/ExternalMapReduce.java

+ 6 - 0
CHANGES.txt

@@ -488,6 +488,12 @@ Trunk (unreleased changes)
     HADOOP-3138. DFS mkdirs() should not throw an exception if the directory
     already exists. (rangadi)
 
+    HADOOP-3041. Deprecate JobConf.setOutputPath and JobConf.getOutputPath.
+    Deprecate OutputFormatBase. Add FileOutputFormat. Existing output formats
+    extending OutputFormatBase, now extend FileOutputFormat. Add the following
+    APIs in FileOutputFormat: setOutputPath, getOutputPath, getWorkOutputPath.
+    (Amareshwari Sriramadasu via nigel)
+
 Release 0.16.2 - 2008-04-02
 
   BUG FIXES

+ 2 - 1
src/contrib/data_join/src/java/org/apache/hadoop/contrib/utils/join/DataJoinJob.java

@@ -25,6 +25,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RunningJob;
@@ -100,7 +101,7 @@ public class DataJoinJob {
     job.setInputFormat(inputFormat);
 
     job.setMapperClass(mapper);
-    job.setOutputPath(new Path(outputDir));
+    FileOutputFormat.setOutputPath(job, new Path(outputDir));
     job.setOutputFormat(outputFormat);
     SequenceFileOutputFormat.setOutputCompressionType(job,
             SequenceFile.CompressionType.BLOCK);

+ 2 - 1
src/contrib/index/src/java/org/apache/hadoop/contrib/index/main/UpdateIndex.java

@@ -31,6 +31,7 @@ import org.apache.hadoop.contrib.index.mapred.Shard;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.util.ReflectionUtils;
 
@@ -210,7 +211,7 @@ public class UpdateIndex {
     }
 
     if (outputPath == null) {
-      outputPath = jobConf.getOutputPath();
+      outputPath = FileOutputFormat.getOutputPath(jobConf);
     }
 
     if (inputPaths == null || outputPath == null) {

+ 4 - 4
src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/IndexUpdateOutputFormat.java

@@ -23,8 +23,8 @@ import java.io.IOException;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.OutputFormatBase;
 import org.apache.hadoop.mapred.RecordWriter;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.util.Progressable;
@@ -33,16 +33,16 @@ import org.apache.hadoop.util.Progressable;
  * The record writer of this output format simply puts a message in an output
  * path when a shard update is done.
  */
-public class IndexUpdateOutputFormat extends OutputFormatBase<Shard, Text> {
+public class IndexUpdateOutputFormat extends FileOutputFormat<Shard, Text> {
 
   /* (non-Javadoc)
-   * @see org.apache.hadoop.mapred.OutputFormatBase#getRecordWriter(org.apache.hadoop.fs.FileSystem, org.apache.hadoop.mapred.JobConf, java.lang.String, org.apache.hadoop.util.Progressable)
+   * @see FileOutputFormat#getRecordWriter(FileSystem, JobConf, String, Progressable)
    */
   public RecordWriter<Shard, Text> getRecordWriter(final FileSystem fs,
       JobConf job, String name, final Progressable progress)
       throws IOException {
 
-    final Path perm = new Path(job.getOutputPath(), name);
+    final Path perm = new Path(getWorkOutputPath(job), name);
 
     return new RecordWriter<Shard, Text>() {
       public void write(Shard key, Text value) throws IOException {

+ 4 - 2
src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/IndexUpdater.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.contrib.index.lucene.FileSystemDirectory;
 import org.apache.hadoop.contrib.index.lucene.LuceneUtil;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
 
@@ -79,7 +80,7 @@ public class IndexUpdater implements IIndexUpdater {
       jobConf.addInputPath(inputPaths[i]);
     }
 
-    jobConf.setOutputPath(outputPath);
+    FileOutputFormat.setOutputPath(jobConf, outputPath);
 
     jobConf.setNumMapTasks(numMapTasks);
 
@@ -95,7 +96,8 @@ public class IndexUpdater implements IIndexUpdater {
       buffer.append(inputs[i].toString());
     }
     LOG.info("mapred.input.dir = " + buffer.toString());
-    LOG.info("mapred.output.dir = " + jobConf.getOutputPath().toString());
+    LOG.info("mapred.output.dir = " + 
+             FileOutputFormat.getOutputPath(jobConf).toString());
     LOG.info("mapred.map.tasks = " + jobConf.getNumMapTasks());
     LOG.info("mapred.reduce.tasks = " + jobConf.getNumReduceTasks());
     LOG.info(shards.length + " shards = " + iconf.getIndexShards());

+ 2 - 1
src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java

@@ -52,6 +52,7 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.fs.Path;
 
 import org.apache.hadoop.mapred.FileAlreadyExistsException;
+import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.InvalidJobConfException;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobClient;
@@ -796,7 +797,7 @@ public class StreamJob {
     }
     
     setUserJobConfProps(false);
-    jobConf_.setOutputPath(new Path(output_));
+    FileOutputFormat.setOutputPath(jobConf_, new Path(output_));
     fmt = null;
     if (outputFormatSpec_!= null) {
       c = StreamUtil.goodClassOrNull(outputFormatSpec_, defaultPackage);

+ 21 - 7
src/docs/src/documentation/content/xdocs/mapred_tutorial.xml

@@ -510,7 +510,7 @@
             <td>53.</td>
             <td>
               &nbsp;&nbsp;&nbsp;&nbsp;
-              <code>conf.setOutputPath(new Path(args[1]));</code>
+              <code>FileOutputFormat.setOutputPath(conf, new Path(args[1]));</code>
             </td>
           </tr>
           <tr>
@@ -907,7 +907,7 @@
  
             <p>In this case the outputs of the map-tasks go directly to the
             <code>FileSystem</code>, into the output path set by 
-            <a href="ext:api/org/apache/hadoop/mapred/jobconf/setoutputpath">
+            <a href="ext:api/org/apache/hadoop/mapred/fileoutputformat/setoutputpath">
             setOutputPath(Path)</a>. The framework does not sort the 
             map-outputs before writing them out to the <code>FileSystem</code>.
             </p>
@@ -1002,7 +1002,7 @@
         indicates the set of input files 
         (<a href="ext:api/org/apache/hadoop/mapred/jobconf/setinputpath">setInputPath(Path)</a>/<a href="ext:api/org/apache/hadoop/mapred/jobconf/addinputpath">addInputPath(Path)</a>)
         and where the output files should be written
-        (<a href="ext:api/org/apache/hadoop/mapred/jobconf/setoutputpath">setOutputPath(Path)</a>).</p>
+        (<a href="ext:api/org/apache/hadoop/mapred/fileoutputformat/setoutputpath">setOutputPath(Path)</a>).</p>
 
         <p>Optionally, <code>JobConf</code> is used to specify other advanced 
         facets of the job such as the <code>Comparator</code> to be used, files 
@@ -1319,6 +1319,7 @@
  
           <p>To avoid these issues the Map-Reduce framework maintains a special 
           <code>${mapred.output.dir}/_temporary/_${taskid}</code> sub-directory
+          accessible via <code>${mapred.work.output.dir}</code>
           for each task-attempt on the <code>FileSystem</code> where the output
           of the task-attempt is stored. On successful completion of the 
           task-attempt, the files in the 
@@ -1328,12 +1329,25 @@
           This process is completely transparent to the application.</p>
  
           <p>The application-writer can take advantage of this feature by 
-          creating any side-files required in <code>${mapred.output.dir}</code> 
+          creating any side-files required in <code>${mapred.work.output.dir}</code>
           during execution of a task via 
-          <a href="ext:api/org/apache/hadoop/mapred/jobconf/getoutputpath">
-          JobConf.getOutputPath()</a>, and the framework will promote them 
+          <a href="ext:api/org/apache/hadoop/mapred/fileoutputformat/getworkoutputpath">
+          FileOutputFormat.getWorkOutputPath()</a>, and the framework will promote them 
           similarly for succesful task-attempts, thus eliminating the need to 
           pick unique paths per task-attempt.</p>
+          
+          <p>Note: The value of <code>${mapred.work.output.dir}</code> during 
+          execution of a particular task-attempt is actually 
+          <code>${mapred.output.dir}/_temporary/_{$taskid}</code>, and this value is 
+          set by the map-reduce framework. So, just create any side-files in the 
+          path  returned by
+          <a href="ext:api/org/apache/hadoop/mapred/fileoutputformat/getworkoutputpath">
+          FileOutputFormat.getWorkOutputPath() </a>from map/reduce 
+          task to take advantage of this feature.</p>
+          
+          <p>The entire discussion holds true for maps of jobs with 
+           reducer=NONE (i.e. 0 reduces) since output of the map, in that case, 
+           goes directly to HDFS.</p> 
         </section>
         
         <section>
@@ -2381,7 +2395,7 @@
             <td>112.</td>
             <td>
               &nbsp;&nbsp;&nbsp;&nbsp;
-              <code>conf.setOutputPath(new Path(other_args.get(1)));</code>
+              <code>FileOutputFormat.setOutputPath(conf, new Path(other_args.get(1)));</code>
             </td>
           </tr>
           <tr>

+ 5 - 2
src/docs/src/documentation/content/xdocs/site.xml

@@ -126,6 +126,11 @@ See http://forrest.apache.org/docs/linking.html for more info.
               <clusterstatus href="ClusterStatus.html" />
               <counters href="Counters.html" />
               <fileinputformat href="FileInputFormat.html" />
+              <fileoutputformat href="FileInputFormat.html">
+                <getoutputpath href="#getOutputPath(org.apache.hadoop.mapred.JobConf)" />
+                <getworkoutputpath href="#getWorkOutputPath(org.apache.hadoop.mapred.JobConf)" />
+                <setoutputpath href="#setOutputPath(org.apache.hadoop.mapred.JobConf,%20org.apache.hadoop.fs.Path)" />
+              </fileoutputformat>
               <filesplit href="FileSplit.html" />
               <inputformat href="InputFormat.html" />
               <inputsplit href="InputSplit.html" />
@@ -141,8 +146,6 @@ See http://forrest.apache.org/docs/linking.html for more info.
                 <setoutputvaluegroupingcomparator href="#setOutputValueGroupingComparator(java.lang.Class)" />
                 <setinputpath href="#setInputPath(org.apache.hadoop.fs.Path)" />
                 <addinputpath href="#addInputPath(org.apache.hadoop.fs.Path)" />
-                <getoutputpath href="#getOutputPath()" />
-                <setoutputpath href="#setOutputPath(org.apache.hadoop.fs.Path)" />
                 <setcombinerclass href="#setCombinerClass(java.lang.Class)" />
                 <setmapdebugscript href="#setMapDebugScript(java.lang.String)" />
                 <setreducedebugscript href="#setReduceDebugScript(java.lang.String)" />

+ 2 - 2
src/examples/org/apache/hadoop/examples/Grep.java

@@ -61,7 +61,7 @@ public class Grep extends Configured implements Tool {
       grepJob.setCombinerClass(LongSumReducer.class);
       grepJob.setReducerClass(LongSumReducer.class);
 
-      grepJob.setOutputPath(tempDir);
+      FileOutputFormat.setOutputPath(grepJob, tempDir);
       grepJob.setOutputFormat(SequenceFileOutputFormat.class);
       grepJob.setOutputKeyClass(Text.class);
       grepJob.setOutputValueClass(LongWritable.class);
@@ -77,7 +77,7 @@ public class Grep extends Configured implements Tool {
       sortJob.setMapperClass(InverseMapper.class);
 
       sortJob.setNumReduceTasks(1);                 // write a single file
-      sortJob.setOutputPath(new Path(args[1]));
+      FileOutputFormat.setOutputPath(sortJob, new Path(args[1]));
       sortJob.setOutputKeyComparatorClass           // sort by decreasing freq
       (LongWritable.DecreasingComparator.class);
 

+ 2 - 1
src/examples/org/apache/hadoop/examples/Join.java

@@ -134,7 +134,8 @@ public class Join extends Configured implements Tool {
       return printUsage();
     }
 
-    jobConf.setOutputPath(new Path(otherArgs.remove(otherArgs.size() - 1)));
+    FileOutputFormat.setOutputPath(jobConf, 
+      new Path(otherArgs.remove(otherArgs.size() - 1)));
     List<Path> plist = new ArrayList<Path>(otherArgs.size());
     for (String s : otherArgs) {
       plist.add(new Path(s));

+ 2 - 1
src/examples/org/apache/hadoop/examples/PiEstimator.java

@@ -31,6 +31,7 @@ import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.SequenceFile.CompressionType;
+import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.MapReduceBase;
@@ -189,7 +190,7 @@ public class PiEstimator extends Configured implements Tool {
     }
     
     jobConf.setInputPath(inDir);
-    jobConf.setOutputPath(outDir);
+    FileOutputFormat.setOutputPath(jobConf, outDir);
     
     jobConf.setNumMapTasks(numMaps);
     

+ 2 - 1
src/examples/org/apache/hadoop/examples/RandomTextWriter.java

@@ -29,6 +29,7 @@ import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.ClusterStatus;
+import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.MapReduceBase;
@@ -224,7 +225,7 @@ public class RandomTextWriter extends Configured implements Tool {
     }
 
     job.setOutputFormat(outputFormatClass);
-    job.setOutputPath(new Path(otherArgs.get(0)));
+    FileOutputFormat.setOutputPath(job, new Path(otherArgs.get(0)));
     
     job.setNumMapTasks(numMaps);
     System.out.println("Running " + numMaps + " maps.");

+ 3 - 2
src/examples/org/apache/hadoop/examples/RandomWriter.java

@@ -30,6 +30,7 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapred.ClusterStatus;
+import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.FileSplit;
 import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.InputSplit;
@@ -105,7 +106,7 @@ public class RandomWriter extends Configured implements Tool {
     public InputSplit[] getSplits(JobConf job, 
                                   int numSplits) throws IOException {
       InputSplit[] result = new InputSplit[numSplits];
-      Path outDir = job.getOutputPath();
+      Path outDir = FileOutputFormat.getOutputPath(job);
       for(int i=0; i < result.length; ++i) {
         result[i] = new FileSplit(new Path(outDir, "dummy-split-" + i), 0, 1, 
                                   (String[])null);
@@ -237,7 +238,7 @@ public class RandomWriter extends Configured implements Tool {
     
     job.setJarByClass(RandomWriter.class);
     job.setJobName("random-writer");
-    job.setOutputPath(outDir);
+    FileOutputFormat.setOutputPath(job, outDir);
     
     job.setOutputKeyClass(BytesWritable.class);
     job.setOutputValueClass(BytesWritable.class);

+ 3 - 2
src/examples/org/apache/hadoop/examples/Sort.java

@@ -134,13 +134,14 @@ public class Sort extends Configured implements Tool {
       return printUsage();
     }
     jobConf.setInputPath(new Path(otherArgs.get(0)));
-    jobConf.setOutputPath(new Path(otherArgs.get(1)));
+    FileOutputFormat.setOutputPath(jobConf, new Path(otherArgs.get(1)));
 
     System.out.println("Running on " +
         cluster.getTaskTrackers() +
         " nodes to sort from " + 
         jobConf.getInputPaths()[0] + " into " +
-        jobConf.getOutputPath() + " with " + num_reduces + " reduces.");
+        FileOutputFormat.getOutputPath(jobConf) +
+        " with " + num_reduces + " reduces.");
     Date startTime = new Date();
     System.out.println("Job started: " + startTime);
     JobClient.runJob(jobConf);

+ 2 - 1
src/examples/org/apache/hadoop/examples/WordCount.java

@@ -30,6 +30,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.MapReduceBase;
@@ -142,7 +143,7 @@ public class WordCount extends Configured implements Tool {
       return printUsage();
     }
     conf.setInputPath(new Path(other_args.get(0)));
-    conf.setOutputPath(new Path(other_args.get(1)));
+    FileOutputFormat.setOutputPath(conf, new Path(other_args.get(1)));
         
     JobClient.runJob(conf);
     return 0;

+ 1 - 1
src/examples/org/apache/hadoop/examples/dancing/DistributedPentomino.java

@@ -178,7 +178,7 @@ public class DistributedPentomino extends Configured implements Tool {
     FileSystem fileSys = FileSystem.get(conf);
     try {
       conf.setInputPath(input);
-      conf.setOutputPath(output);
+      FileOutputFormat.setOutputPath(conf, output);
       conf.setJarByClass(PentMap.class);
       
       conf.setJobName("dancingElephant");

+ 1 - 1
src/examples/pipes/impl/wordcount-nopipe.cc

@@ -103,7 +103,7 @@ public:
   WordCountWriter(HadoopPipes::ReduceContext& context) {
     const HadoopPipes::JobConf* job = context.getJobConf();
     int part = job->getInt("mapred.task.partition");
-    std::string outDir = job->get("mapred.output.dir");
+    std::string outDir = job->get("mapred.work.output.dir");
     // remove the file: schema substring
     std::string::size_type posn = outDir.find(":");
     HADOOP_ASSERT(posn != std::string::npos, 

+ 20 - 46
src/java/org/apache/hadoop/mapred/JobConf.java

@@ -72,7 +72,8 @@ import org.apache.hadoop.util.Tool;
  * (if any), {@link Partitioner}, {@link Reducer}, {@link InputFormat} and 
  * {@link OutputFormat} implementations to be used etc. It also indicates the 
  * set of input files ({@link #setInputPath(Path)}/{@link #addInputPath(Path)}), 
- * and where the output files should be written ({@link #setOutputPath(Path)}).
+ * and where the output files should be written via
+ * {@link FileOutputFormat#setOutputPath(JobConf, Path)}.
  *
  * <p>Optionally <code>JobConf</code> is used to specify other advanced facets 
  * of the job such as <code>Comparator</code>s to be used, files to be put in  
@@ -91,7 +92,7 @@ import org.apache.hadoop.util.Tool;
  *     job.setJobName("myjob");
  *     
  *     job.setInputPath(new Path("in"));
- *     job.setOutputPath(new Path("out"));
+ *     FileOutputFormat.setOutputPath(job, new Path("out"));
  *     
  *     job.setMapperClass(MyJob.MyMapper.class);
  *     job.setCombinerClass(MyJob.MyReducer.class);
@@ -356,63 +357,35 @@ public class JobConf extends Configuration {
   }
   
   /**
+   * @deprecated Use {@link FileOutputFormat#getOutputPath(JobConf)} or
+   *                 {@link FileOutputFormat#getWorkOutputPath(JobConf)}
    * Get the {@link Path} to the output directory for the map-reduce job.
    * 
-   * <h4 id="SideEffectFiles">Tasks' Side-Effect Files</h4>
-   * 
-   * <p>Some applications need to create/write-to side-files, which differ from
-   * the actual job-outputs.
-   * 
-   * <p>In such cases there could be issues with 2 instances of the same TIP 
-   * (running simultaneously e.g. speculative tasks) trying to open/write-to the
-   * same file (path) on HDFS. Hence the application-writer will have to pick 
-   * unique names per task-attempt (e.g. using the taskid, say 
-   * <tt>task_200709221812_0001_m_000000_0</tt>), not just per TIP.</p> 
-   * 
-   * <p>To get around this the Map-Reduce framework helps the application-writer 
-   * out by maintaining a special 
-   * <tt>${mapred.output.dir}/_temporary/_${taskid}</tt> 
-   * sub-directory for each task-attempt on HDFS where the output of the 
-   * task-attempt goes. On successful completion of the task-attempt the files 
-   * in the <tt>${mapred.output.dir}/_temporary/_${taskid}</tt> (only) 
-   * are <i>promoted</i> to <tt>${mapred.output.dir}</tt>. Of course, the 
-   * framework discards the sub-directory of unsuccessful task-attempts. This 
-   * is completely transparent to the application.</p>
-   * 
-   * <p>The application-writer can take advantage of this by creating any 
-   * side-files required in <tt>${mapred.output.dir}</tt> during execution of his 
-   * reduce-task i.e. via {@link #getOutputPath()}, and the framework will move 
-   * them out similarly - thus she doesn't have to pick unique paths per 
-   * task-attempt.</p>
-   * 
-   * <p><i>Note</i>: the value of <tt>${mapred.output.dir}</tt> during execution 
-   * of a particular task-attempt is actually 
-   * <tt>${mapred.output.dir}/_temporary/_{$taskid}</tt>, not the value set by 
-   * {@link #setOutputPath(Path)}. So, just create any side-files in the path 
-   * returned by {@link #getOutputPath()} from map/reduce task to take 
-   * advantage of this feature.</p>
-   * 
-   * <p>The entire discussion holds true for maps of jobs with 
-   * reducer=NONE (i.e. 0 reduces) since output of the map, in that case, 
-   * goes directly to HDFS.</p> 
-   * 
    * @return the {@link Path} to the output directory for the map-reduce job.
    */
-  public Path getOutputPath() { 
-    String name = get("mapred.output.dir");
-    return name == null ? null: new Path(name);
+  @Deprecated
+  public Path getOutputPath() {
+    // this return context sensitive value for output path
+    // Returns task's temporary output path while task's execution
+    // Otherwise returns the output path that was set.
+    Path workOutputDir = FileOutputFormat.getWorkOutputPath(this);
+    if (workOutputDir != null) {
+      return workOutputDir;
+    }
+    else return FileOutputFormat.getOutputPath(this);
   }
 
   /**
+   * @deprecated Use {@link FileOutputFormat#setOutputPath(JobConf, Path)} 
    * Set the {@link Path} of the output directory for the map-reduce job.
    * 
    * <p><i>Note</i>:
    * </p>
    * @param dir the {@link Path} of the output directory for the map-reduce job.
    */
+  @Deprecated
   public void setOutputPath(Path dir) {
-    dir = new Path(getWorkingDirectory(), dir);
-    set("mapred.output.dir", dir.toString());
+    FileOutputFormat.setOutputPath(this, dir);
   }
 
   /**
@@ -964,7 +937,8 @@ public class JobConf extends Configuration {
    * <p>It is legal to set the number of reduce-tasks to <code>zero</code>.</p>
    * 
    * <p>In this case the output of the map-tasks directly go to distributed 
-   * file-system, to the path set by {@link #setOutputPath(Path)}. Also, the 
+   * file-system, to the path set by 
+   * {@link FileOutputFormat#setOutputPath(JobConf, Path)}. Also, the 
    * framework doesn't sort the map-outputs before writing it out to HDFS.</p>
    * 
    * @param n the number of reduce tasks for this job.

+ 1 - 1
src/java/org/apache/hadoop/mapred/JobHistory.java

@@ -375,7 +375,7 @@ public class JobHistory {
         }
 
         // find user log directory 
-        Path outputPath = jobConf.getOutputPath();
+        Path outputPath = FileOutputFormat.getOutputPath(jobConf);
         userLogDir = jobConf.get("hadoop.job.history.user.location",
         		outputPath == null ? null : outputPath.toString());
         if ("none".equals(userLogDir)) {

+ 2 - 2
src/java/org/apache/hadoop/mapred/JobInProgress.java

@@ -352,7 +352,7 @@ class JobInProgress {
     }
 
     // create job specific temporary directory in output path
-    Path outputPath = conf.getOutputPath();
+    Path outputPath = FileOutputFormat.getOutputPath(conf);
     if (outputPath != null) {
       Path tmpDir = new Path(outputPath, MRConstants.TEMP_DIR_NAME);
       FileSystem fileSys = tmpDir.getFileSystem(conf);
@@ -1642,7 +1642,7 @@ class JobInProgress {
       fs.delete(tempDir, true); 
 
       // delete the temporary directory in output directory
-      Path outputPath = conf.getOutputPath();
+      Path outputPath = FileOutputFormat.getOutputPath(conf);
       if (outputPath != null) {
         Path tmpDir = new Path(outputPath, MRConstants.TEMP_DIR_NAME);
         FileSystem fileSys = tmpDir.getFileSystem(conf);

+ 1 - 1
src/java/org/apache/hadoop/mapred/LocalJobRunner.java

@@ -114,7 +114,7 @@ class LocalJobRunner implements JobSubmissionProtocol {
           job.setNumReduceTasks(1);
         }
         // create job specific temp directory in output path
-        Path outputPath = job.getOutputPath();
+        Path outputPath = FileOutputFormat.getOutputPath(job);
         FileSystem outputFs = null;
         Path tmpDir = null;
         if (outputPath != null) {

+ 4 - 4
src/java/org/apache/hadoop/mapred/MapFileOutputFormat.java

@@ -25,7 +25,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileUtil;
 
-import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.MapFile;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.Writable;
@@ -37,13 +36,14 @@ import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
 
 /** An {@link OutputFormat} that writes {@link MapFile}s. */
-public class MapFileOutputFormat extends OutputFormatBase {
+public class MapFileOutputFormat 
+extends FileOutputFormat<WritableComparable, Writable> {
 
-  public RecordWriter getRecordWriter(FileSystem ignored, JobConf job,
+  public RecordWriter<WritableComparable, Writable> getRecordWriter(FileSystem ignored, JobConf job,
                                       String name, Progressable progress)
     throws IOException {
 
-    Path outputPath = job.getOutputPath();
+    Path outputPath = getWorkOutputPath(job);
     FileSystem fs = outputPath.getFileSystem(job);
     if (!fs.exists(outputPath)) {
       throw new IOException("Output directory doesnt exist");

+ 5 - 2
src/java/org/apache/hadoop/mapred/OutputFormatBase.java

@@ -25,7 +25,10 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.util.Progressable;
 
-/** A base class for {@link OutputFormat}. */
+/** A base class for {@link OutputFormat}.
+ * @deprecated Use {@link FileOutputFormat}
+ */
+@Deprecated
 public abstract class OutputFormatBase<K, V> implements OutputFormat<K, V> {
 
   /**
@@ -96,7 +99,7 @@ public abstract class OutputFormatBase<K, V> implements OutputFormat<K, V> {
     throws FileAlreadyExistsException, 
            InvalidJobConfException, IOException {
     // Ensure that the output directory is set and not already there
-    Path outDir = job.getOutputPath();
+    Path outDir = FileOutputFormat.getOutputPath(job);
     if (outDir == null && job.getNumReduceTasks() != 0) {
       throw new InvalidJobConfException("Output directory not set in JobConf.");
     }

+ 6 - 4
src/java/org/apache/hadoop/mapred/SequenceFileOutputFormat.java

@@ -35,13 +35,15 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.*;
 
 /** An {@link OutputFormat} that writes {@link SequenceFile}s. */
-public class SequenceFileOutputFormat extends OutputFormatBase {
+public class SequenceFileOutputFormat 
+extends FileOutputFormat<WritableComparable, Writable> {
 
-  public RecordWriter getRecordWriter(FileSystem ignored, JobConf job,
-                                      String name, Progressable progress)
+  public RecordWriter<WritableComparable, Writable> getRecordWriter(
+                                          FileSystem ignored, JobConf job,
+                                          String name, Progressable progress)
     throws IOException {
 
-    Path outputPath = job.getOutputPath();
+    Path outputPath = getWorkOutputPath(job);
     FileSystem fs = outputPath.getFileSystem(job);
     if (!fs.exists(outputPath)) {
       throw new IOException("Output directory doesnt exist");

+ 4 - 4
src/java/org/apache/hadoop/mapred/Task.java

@@ -211,7 +211,7 @@ abstract class Task implements Writable, Configurable {
   public String toString() { return taskId; }
 
   private Path getTaskOutputPath(JobConf conf) {
-    Path p = new Path(conf.getOutputPath(), 
+    Path p = new Path(FileOutputFormat.getOutputPath(conf), 
       (MRConstants.TEMP_DIR_NAME + Path.SEPARATOR + "_" + taskId));
     try {
       FileSystem fs = p.getFileSystem(conf);
@@ -233,9 +233,9 @@ abstract class Task implements Writable, Configurable {
     conf.set("mapred.job.id", jobId);
     
     // The task-specific output path
-    if (conf.getOutputPath() != null) {
+    if (FileOutputFormat.getOutputPath(conf) != null) {
       taskOutputPath = getTaskOutputPath(conf);
-      conf.setOutputPath(taskOutputPath);
+      FileOutputFormat.setWorkOutputPath(conf, taskOutputPath);
     }
   }
   
@@ -513,7 +513,7 @@ abstract class Task implements Writable, Configurable {
       this.conf = (JobConf) conf;
 
       if (taskId != null && taskOutputPath == null && 
-              this.conf.getOutputPath() != null) {
+          FileOutputFormat.getOutputPath(this.conf) != null) {
         taskOutputPath = getTaskOutputPath(this.conf);
       }
     } else {

+ 1 - 1
src/java/org/apache/hadoop/mapred/TaskTracker.java

@@ -1425,7 +1425,7 @@ public class TaskTracker
       keepFailedTaskFiles = localJobConf.getKeepFailedTaskFiles();
 
       // create _taskid directory in output path temporary directory.
-      Path outputPath = localJobConf.getOutputPath();
+      Path outputPath = FileOutputFormat.getOutputPath(localJobConf);
       if (outputPath != null) {
         Path jobTmpDir = new Path(outputPath, MRConstants.TEMP_DIR_NAME);
         FileSystem fs = jobTmpDir.getFileSystem(localJobConf);

+ 2 - 2
src/java/org/apache/hadoop/mapred/TextOutputFormat.java

@@ -33,7 +33,7 @@ import org.apache.hadoop.io.compress.GzipCodec;
 import org.apache.hadoop.util.*;
 
 /** An {@link OutputFormat} that writes plain text files. */
-public class TextOutputFormat<K, V> extends OutputFormatBase<K, V> {
+public class TextOutputFormat<K, V> extends FileOutputFormat<K, V> {
 
   protected static class LineRecordWriter<K, V>
     implements RecordWriter<K, V> {
@@ -101,7 +101,7 @@ public class TextOutputFormat<K, V> extends OutputFormatBase<K, V> {
                                                   Progressable progress)
     throws IOException {
 
-    Path dir = job.getOutputPath();
+    Path dir = getWorkOutputPath(job);
     FileSystem fs = dir.getFileSystem(job);
     if (!fs.exists(dir)) {
       throw new IOException("Output directory doesnt exist");

+ 4 - 3
src/java/org/apache/hadoop/mapred/lib/MultipleOutputFormat.java

@@ -27,7 +27,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.OutputFormatBase;
+import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.RecordWriter;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.util.Progressable;
@@ -50,8 +50,9 @@ import org.apache.hadoop.util.Progressable;
  * output file name that depends on both the keys and the input file name,
  * 
  */
-public abstract class MultipleOutputFormat<K extends WritableComparable, V extends Writable>
-    extends OutputFormatBase<K, V> {
+public abstract class MultipleOutputFormat<K extends WritableComparable,
+                                           V extends Writable>
+extends FileOutputFormat<K, V> {
 
   /**
    * Create a composite record writer that can write key/value data to different

+ 10 - 2
src/java/org/apache/hadoop/mapred/lib/MultipleSequenceFileOutputFormat.java

@@ -21,6 +21,8 @@ package org.apache.hadoop.mapred.lib;
 import java.io.IOException;
 
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RecordWriter;
 import org.apache.hadoop.mapred.SequenceFileOutputFormat;
@@ -30,12 +32,18 @@ import org.apache.hadoop.util.Progressable;
  * This class extends the MultipleOutputFormat, allowing to write the output data 
  * to different output files in sequence file output format. 
  */
-public class MultipleSequenceFileOutputFormat extends MultipleOutputFormat {
+public class MultipleSequenceFileOutputFormat 
+extends MultipleOutputFormat<WritableComparable, Writable> {
 
   private SequenceFileOutputFormat theSequenceFileOutputFormat = null;
   
   @Override
-  protected RecordWriter getBaseRecordWriter(FileSystem fs, JobConf job, String name, Progressable arg3) throws IOException {
+  protected RecordWriter<WritableComparable, Writable> getBaseRecordWriter(
+                                                         FileSystem fs,
+                                                         JobConf job,
+                                                         String name,
+                                                         Progressable arg3) 
+  throws IOException {
     if (theSequenceFileOutputFormat == null) {
       theSequenceFileOutputFormat = new SequenceFileOutputFormat();
     }

+ 2 - 1
src/java/org/apache/hadoop/mapred/lib/aggregate/ValueAggregatorJob.java

@@ -24,6 +24,7 @@ import java.util.ArrayList;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
@@ -168,7 +169,7 @@ public class ValueAggregatorJob {
     theJob.setInputFormat(theInputFormat);
     
     theJob.setMapperClass(ValueAggregatorMapper.class);
-    theJob.setOutputPath(new Path(outputDir));
+    FileOutputFormat.setOutputPath(theJob, new Path(outputDir));
     theJob.setOutputFormat(TextOutputFormat.class);
     theJob.setMapOutputKeyClass(Text.class);
     theJob.setMapOutputValueClass(Text.class);

+ 3 - 1
src/java/org/apache/hadoop/mapred/pipes/Submitter.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.filecache.DistributedCache;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
@@ -356,7 +357,8 @@ public class Submitter {
         conf.setInputPath(new Path((String) results.getValue("-input")));
       }
       if (results.hasOption("-output")) {
-        conf.setOutputPath(new Path((String) results.getValue("-output")));
+        FileOutputFormat.setOutputPath(conf, 
+          new Path((String) results.getValue("-output")));
       }
       if (results.hasOption("-jar")) {
         conf.setJar((String) results.getValue("-jar"));

+ 2 - 1
src/java/org/apache/hadoop/tools/Logalyzer.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.MapReduceBase;
@@ -223,7 +224,7 @@ public class Logalyzer {
     grepJob.setCombinerClass(LongSumReducer.class);
     grepJob.setReducerClass(LongSumReducer.class);
     
-    grepJob.setOutputPath(analysisOutput);
+    FileOutputFormat.setOutputPath(grepJob, analysisOutput);
     grepJob.setOutputFormat(TextOutputFormat.class);
     grepJob.setOutputKeyClass(Text.class);
     grepJob.setOutputValueClass(LongWritable.class);

+ 2 - 1
src/java/org/apache/hadoop/util/CopyFiles.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.FileSplit;
 import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.InputSplit;
@@ -881,7 +882,7 @@ public class CopyFiles implements Tool {
         logPath = new Path(destPath, filename);
       }
     }
-    jobConf.setOutputPath(logPath);
+    FileOutputFormat.setOutputPath(jobConf, logPath);
     
     // create src list, dst list
     FileSystem jobfs = jobDirectory.getFileSystem(jobConf);

+ 2 - 1
src/test/org/apache/hadoop/dfs/NNBench.java

@@ -47,6 +47,7 @@ import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.SequenceFile.CompressionType;
 import org.apache.hadoop.io.SequenceFile;
 
+import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.Mapper;
 import org.apache.hadoop.mapred.SequenceFileInputFormat;
 import org.apache.hadoop.mapred.JobClient;
@@ -475,7 +476,7 @@ public class NNBench {
     job.setMapperClass(NNBenchMapper.class);
     job.setReducerClass(NNBenchReducer.class);
 
-    job.setOutputPath(new Path(baseDir, OUTPUT_DIR_NAME));
+    FileOutputFormat.setOutputPath(job, new Path(baseDir, OUTPUT_DIR_NAME));
     job.setOutputKeyClass(Text.class);
     job.setOutputValueClass(Text.class);
     job.setNumReduceTasks((int) numberOfReduces);

+ 1 - 1
src/test/org/apache/hadoop/fs/DFSCIOTest.java

@@ -273,7 +273,7 @@ public class DFSCIOTest extends TestCase {
     job.setMapperClass(mapperClass);
     job.setReducerClass(AccumulatingReducer.class);
 
-    job.setOutputPath(outputDir);
+    FileOutputFormat.setOutputPath(job, outputDir);
     job.setOutputKeyClass(UTF8.class);
     job.setOutputValueClass(UTF8.class);
     job.setNumReduceTasks(1);

+ 1 - 1
src/test/org/apache/hadoop/fs/DistributedFSCheck.java

@@ -194,7 +194,7 @@ public class DistributedFSCheck extends TestCase {
     job.setMapperClass(DistributedFSCheckMapper.class);
     job.setReducerClass(AccumulatingReducer.class);
 
-    job.setOutputPath(READ_DIR);
+    FileOutputFormat.setOutputPath(job, READ_DIR);
     job.setOutputKeyClass(UTF8.class);
     job.setOutputValueClass(UTF8.class);
     job.setNumReduceTasks(1);

+ 1 - 1
src/test/org/apache/hadoop/fs/TestDFSIO.java

@@ -229,7 +229,7 @@ public class TestDFSIO extends TestCase {
     job.setMapperClass(mapperClass);
     job.setReducerClass(AccumulatingReducer.class);
 
-    job.setOutputPath(outputDir);
+    FileOutputFormat.setOutputPath(job, outputDir);
     job.setOutputKeyClass(UTF8.class);
     job.setOutputValueClass(UTF8.class);
     job.setNumReduceTasks(1);

+ 4 - 3
src/test/org/apache/hadoop/fs/TestFileSystem.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.UTF8;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.SequenceFile.CompressionType;
+import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.Mapper;
@@ -210,7 +211,7 @@ public class TestFileSystem extends TestCase {
     job.setMapperClass(WriteMapper.class);
     job.setReducerClass(LongSumReducer.class);
 
-    job.setOutputPath(WRITE_DIR);
+    FileOutputFormat.setOutputPath(job, WRITE_DIR);
     job.setOutputKeyClass(UTF8.class);
     job.setOutputValueClass(LongWritable.class);
     job.setNumReduceTasks(1);
@@ -308,7 +309,7 @@ public class TestFileSystem extends TestCase {
     job.setMapperClass(ReadMapper.class);
     job.setReducerClass(LongSumReducer.class);
 
-    job.setOutputPath(READ_DIR);
+    FileOutputFormat.setOutputPath(job, READ_DIR);
     job.setOutputKeyClass(UTF8.class);
     job.setOutputValueClass(LongWritable.class);
     job.setNumReduceTasks(1);
@@ -404,7 +405,7 @@ public class TestFileSystem extends TestCase {
     job.setMapperClass(SeekMapper.class);
     job.setReducerClass(LongSumReducer.class);
 
-    job.setOutputPath(READ_DIR);
+    FileOutputFormat.setOutputPath(job, READ_DIR);
     job.setOutputKeyClass(UTF8.class);
     job.setOutputValueClass(LongWritable.class);
     job.setNumReduceTasks(1);

+ 2 - 7
src/test/org/apache/hadoop/io/FileBench.java

@@ -19,12 +19,10 @@
 package org.apache.hadoop.io;
 
 import java.io.IOException;
-import java.net.URI;
 import java.util.ArrayList;
 import java.util.Date;
 import java.util.EnumSet;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 import java.util.Random;
 
@@ -37,10 +35,7 @@ import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.LzoCodec;
 import org.apache.hadoop.io.compress.GzipCodec;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapred.*;
-import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
@@ -112,7 +107,7 @@ public class FileBench extends Configured implements Tool {
     Text val = new Text();
 
     final String fn = conf.get("test.filebench.name", "");
-    final Path outd = conf.getOutputPath();
+    final Path outd = FileOutputFormat.getOutputPath(conf);
     OutputFormat outf = conf.getOutputFormat();
     RecordWriter<Text,Text> rw =
       outf.getRecordWriter(outd.getFileSystem(conf), conf, fn,
@@ -203,7 +198,7 @@ public class FileBench extends Configured implements Tool {
     job.setOutputKeyClass(Text.class);
     job.setOutputValueClass(Text.class);
     job.setInputPath(root);
-    job.setOutputPath(root);
+    FileOutputFormat.setOutputPath(job, root);
 
     if (null == cc) cc = EnumSet.allOf(CCodec.class);
     if (null == ct) ct = EnumSet.allOf(CType.class);

+ 1 - 1
src/test/org/apache/hadoop/mapred/BigMapOutput.java

@@ -135,7 +135,7 @@ public class BigMapOutput extends Configured implements Tool {
     if (fs.exists(outputPath)) {
       fs.delete(outputPath, true);
     }
-    jobConf.setOutputPath(outputPath);
+    FileOutputFormat.setOutputPath(jobConf, outputPath);
     jobConf.setMapperClass(IdentityMapper.class);
     jobConf.setReducerClass(IdentityReducer.class);
     jobConf.setOutputKeyClass(BytesWritable.class);

+ 2 - 2
src/test/org/apache/hadoop/mapred/GenericMRLoadGenerator.java

@@ -109,7 +109,7 @@ public class GenericMRLoadGenerator extends Configured implements Tool {
         } else if ("-keepred".equals(argv[i])) {
           job.set("hadoop.sort.reduce.keep.percent", argv[++i]);
         } else if ("-outdir".equals(argv[i])) {
-          job.setOutputPath(new Path(argv[++i]));
+          FileOutputFormat.setOutputPath(job, new Path(argv[++i]));
         } else if ("-indir".equals(argv[i])) {
           job.addInputPath(new Path(argv[++i]));
         } else if ("-inFormatIndirect".equals(argv[i])) {
@@ -140,7 +140,7 @@ public class GenericMRLoadGenerator extends Configured implements Tool {
       return -1;
     }
 
-    if (null == job.getOutputPath()) {
+    if (null == FileOutputFormat.getOutputPath(job)) {
       // No output dir? No writes
       job.setOutputFormat(NullOutputFormat.class);
     }

+ 3 - 5
src/test/org/apache/hadoop/mapred/MRBench.java

@@ -26,12 +26,9 @@ import java.util.Random;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.UTF8;
-import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.Text;
 
@@ -179,11 +176,12 @@ public class MRBench {
       // reset the job jar because the copy constructor doesn't
       jobConf.setJar(masterJobConf.getJar());
       // give a new random name to output of the mapred tasks
-      jobConf.setOutputPath(new Path(OUTPUT_DIR, "output_" + rand.nextInt()));
+      FileOutputFormat.setOutputPath(jobConf, 
+                         new Path(OUTPUT_DIR, "output_" + rand.nextInt()));
 
       LOG.info("Running job " + i + ":" +
                " input=" + jobConf.getInputPaths()[0] + 
-               " output=" + jobConf.getOutputPath());
+               " output=" + FileOutputFormat.getOutputPath(jobConf));
       
       // run the mapred task now 
       long curTime = System.currentTimeMillis();

+ 1 - 4
src/test/org/apache/hadoop/mapred/MRCaching.java

@@ -25,8 +25,6 @@ import org.apache.hadoop.fs.*;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.Mapper;
@@ -37,7 +35,6 @@ import org.apache.hadoop.util.*;
 import org.apache.hadoop.mapred.MapReduceBase;
 import org.apache.hadoop.filecache.*;
 import java.net.URI;
-import java.net.URISyntaxException;
 
 public class MRCaching {
   static String testStr = "This is a test file " + "used for testing caching "
@@ -170,7 +167,7 @@ public class MRCaching {
     conf.setCombinerClass(MRCaching.ReduceClass.class);
     conf.setReducerClass(MRCaching.ReduceClass.class);
     conf.setInputPath(inDir);
-    conf.setOutputPath(outDir);
+    FileOutputFormat.setOutputPath(conf, outDir);
     conf.setNumMapTasks(1);
     conf.setNumReduceTasks(1);
     conf.setSpeculativeExecution(false);

+ 1 - 5
src/test/org/apache/hadoop/mapred/NotificationTestCase.java

@@ -22,10 +22,8 @@ import org.mortbay.http.HttpServer;
 import org.mortbay.http.SocketListener;
 import org.mortbay.http.HttpContext;
 import org.mortbay.jetty.servlet.ServletHandler;
-import org.apache.log4j.net.SocketServer;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.examples.WordCount;
@@ -37,8 +35,6 @@ import javax.servlet.ServletException;
 import java.io.IOException;
 import java.io.DataOutputStream;
 import java.util.Date;
-import java.net.Socket;
-import java.net.InetSocketAddress;
 
 /**
  * Base class to test Job end notification in local and cluster mode.
@@ -207,7 +203,7 @@ public abstract class NotificationTestCase extends HadoopTestCase {
     conf.setReducerClass(WordCount.Reduce.class);
 
     conf.setInputPath(inDir);
-    conf.setOutputPath(outDir);
+    FileOutputFormat.setOutputPath(conf, outDir);
     conf.setNumMapTasks(numMaps);
     conf.setNumReduceTasks(numReduces);
     JobClient.runJob(conf);

+ 1 - 1
src/test/org/apache/hadoop/mapred/PiEstimator.java

@@ -158,7 +158,7 @@ public class PiEstimator {
       throw new IOException("Mkdirs failed to create " + inDir.toString());
     }
     jobConf.setInputPath(inDir);
-    jobConf.setOutputPath(outDir);
+    FileOutputFormat.setOutputPath(jobConf, outDir);
     
     jobConf.setNumMapTasks(numMaps);
     

+ 10 - 6
src/test/org/apache/hadoop/mapred/SortValidator.java

@@ -342,7 +342,7 @@ public class SortValidator extends Configured implements Tool {
       if (fs.exists(outputPath)) {
         fs.delete(outputPath, true);
       }
-      jobConf.setOutputPath(outputPath);
+      FileOutputFormat.setOutputPath(jobConf, outputPath);
       
       // Uncomment to run locally in a single process
       //job_conf.set("mapred.job.tracker", "local");
@@ -350,8 +350,10 @@ public class SortValidator extends Configured implements Tool {
       System.out.println("\nSortValidator.RecordStatsChecker: Validate sort " +
                          "from " + jobConf.getInputPaths()[0] + " (" + 
                          noSortInputpaths + " files), " + 
-                         jobConf.getInputPaths()[1] + " (" + noSortReduceTasks + 
-                         " files) into " + jobConf.getOutputPath() + 
+                         jobConf.getInputPaths()[1] + " (" + 
+                         noSortReduceTasks + 
+                         " files) into " + 
+                         FileOutputFormat.getOutputPath(jobConf) + 
                          " with 1 reducer.");
       Date startTime = new Date();
       System.out.println("Job started: " + startTime);
@@ -484,15 +486,17 @@ public class SortValidator extends Configured implements Tool {
       if (fs.exists(outputPath)) {
         fs.delete(outputPath, true);
       }
-      jobConf.setOutputPath(outputPath);
+      FileOutputFormat.setOutputPath(jobConf, outputPath);
       
       // Uncomment to run locally in a single process
       //job_conf.set("mapred.job.tracker", "local");
       
       System.out.println("\nSortValidator.RecordChecker: Running on " +
                          cluster.getTaskTrackers() +
-                         " nodes to validate sort from " + jobConf.getInputPaths()[0] + ", " + 
-                         jobConf.getInputPaths()[1] + " into " + jobConf.getOutputPath() + 
+                         " nodes to validate sort from " +
+                         jobConf.getInputPaths()[0] + ", " + 
+                         jobConf.getInputPaths()[1] + " into " +
+                         FileOutputFormat.getOutputPath(jobConf) + 
                          " with " + noReduces + " reduces.");
       Date startTime = new Date();
       System.out.println("Job started: " + startTime);

+ 1 - 1
src/test/org/apache/hadoop/mapred/TestClusterMapReduceTestCase.java

@@ -57,7 +57,7 @@ public class TestClusterMapReduceTestCase extends ClusterMapReduceTestCase {
 
     conf.setInputPath(getInputDir());
 
-    conf.setOutputPath(getOutputDir());
+    FileOutputFormat.setOutputPath(conf, getOutputDir());
 
 
     JobClient.runJob(conf);

+ 1 - 1
src/test/org/apache/hadoop/mapred/TestCollect.java

@@ -118,7 +118,7 @@ public class TestCollect extends TestCase
     conf.setInputFormat(RandomInputFormat.class); // for self data generation
     conf.setOutputKeyClass(IntWritable.class);
     conf.setOutputValueClass(IntWritable.class);
-    conf.setOutputPath(OUTPUT_DIR);
+    FileOutputFormat.setOutputPath(conf, OUTPUT_DIR);
     
     conf.setMapperClass(Map.class);
     conf.setReducerClass(Reduce.class);

+ 1 - 2
src/test/org/apache/hadoop/mapred/TestComparators.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.mapred;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.io.BooleanWritable.Comparator;
-import org.apache.hadoop.mapred.lib.*;
 import junit.framework.TestCase;
 import java.io.*;
 import java.util.*;
@@ -298,7 +297,7 @@ public class TestComparators extends TestCase
     fs.delete(testdir, true);
     conf.setInputFormat(SequenceFileInputFormat.class);
     conf.setInputPath(inDir);
-    conf.setOutputPath(outDir);
+    FileOutputFormat.setOutputPath(conf, outDir);
     conf.setOutputKeyClass(IntWritable.class);
     conf.setOutputValueClass(Text.class);
     conf.setMapOutputValueClass(IntWritable.class);

+ 1 - 1
src/test/org/apache/hadoop/mapred/TestEmptyJobWithDFS.java

@@ -77,7 +77,7 @@ public class TestEmptyJobWithDFS extends TestCase {
     conf.setMapperClass(IdentityMapper.class);        
     conf.setReducerClass(IdentityReducer.class);
     conf.setInputPath(inDir);
-    conf.setOutputPath(outDir);
+    FileOutputFormat.setOutputPath(conf, outDir);
     conf.setNumMapTasks(numMaps);
     conf.setNumReduceTasks(numReduces);
       

+ 1 - 3
src/test/org/apache/hadoop/mapred/TestFieldSelection.java

@@ -21,8 +21,6 @@ import org.apache.hadoop.fs.*;
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.mapred.lib.*;
 import junit.framework.TestCase;
-import java.io.*;
-import java.util.*;
 import java.text.NumberFormat;
 
 public class TestFieldSelection extends TestCase {
@@ -96,7 +94,7 @@ private static NumberFormat idFormat = NumberFormat.getInstance();
     job.setMapperClass(FieldSelectionMapReduce.class);
     job.setReducerClass(FieldSelectionMapReduce.class);
 
-    job.setOutputPath(OUTPUT_DIR);
+    FileOutputFormat.setOutputPath(job, OUTPUT_DIR);
     job.setOutputKeyClass(Text.class);
     job.setOutputValueClass(Text.class);
     job.setOutputFormat(TextOutputFormat.class);

+ 3 - 3
src/test/org/apache/hadoop/mapred/TestJavaSerialization.java

@@ -49,7 +49,7 @@ public class TestJavaSerialization extends ClusterMapReduceTestCase {
 
   }
   
-  static class StringOutputFormat<K, V> extends OutputFormatBase<K, V> {
+  static class StringOutputFormat<K, V> extends FileOutputFormat<K, V> {
     
     static class LineRecordWriter<K, V> implements RecordWriter<K, V> {
       
@@ -80,7 +80,7 @@ public class TestJavaSerialization extends ClusterMapReduceTestCase {
     public RecordWriter<K, V> getRecordWriter(FileSystem ignored, JobConf job,
         String name, Progressable progress) throws IOException {
 
-      Path dir = job.getOutputPath();
+      Path dir = getWorkOutputPath(job);
       FileSystem fs = dir.getFileSystem(job);
       FSDataOutputStream fileOut = fs.create(new Path(dir, name), progress);
       return new LineRecordWriter<K, V>(fileOut);
@@ -119,7 +119,7 @@ public class TestJavaSerialization extends ClusterMapReduceTestCase {
 
     conf.setInputPath(getInputDir());
 
-    conf.setOutputPath(getOutputDir());
+    FileOutputFormat.setOutputPath(conf, getOutputDir());
 
     JobClient.runJob(conf);
 

+ 1 - 1
src/test/org/apache/hadoop/mapred/TestJobStatusPersistency.java

@@ -51,7 +51,7 @@ public class TestJobStatusPersistency extends ClusterMapReduceTestCase {
 
     conf.setInputPath(getInputDir());
 
-    conf.setOutputPath(getOutputDir());
+    FileOutputFormat.setOutputPath(conf, getOutputDir());
 
     return JobClient.runJob(conf).getJobID();
   }

+ 1 - 1
src/test/org/apache/hadoop/mapred/TestMapOutputType.java

@@ -84,7 +84,7 @@ public class TestMapOutputType extends TestCase
     conf.setInt("io.sort.mb", 1);
     conf.setInputFormat(SequenceFileInputFormat.class);
     conf.setInputPath(inDir);
-    conf.setOutputPath(outDir);
+    FileOutputFormat.setOutputPath(conf, outDir);
     conf.setMapperClass(TextGen.class);
     conf.setReducerClass(TextReduce.class);
     conf.setOutputKeyClass(Text.class);

+ 5 - 5
src/test/org/apache/hadoop/mapred/TestMapRed.java

@@ -298,7 +298,7 @@ public class TestMapRed extends TestCase {
     FileSystem fs = FileSystem.get(conf);
     fs.delete(testdir, true);
     conf.setInputPath(inDir);
-    conf.setOutputPath(outDir);
+    FileOutputFormat.setOutputPath(conf, outDir);
     conf.setMapperClass(MyMap.class);
     conf.setReducerClass(MyReduce.class);
     conf.setOutputKeyClass(Text.class);
@@ -427,7 +427,7 @@ public class TestMapRed extends TestCase {
     genJob.setInputFormat(SequenceFileInputFormat.class);
     genJob.setMapperClass(RandomGenMapper.class);
 
-    genJob.setOutputPath(randomOuts);
+    FileOutputFormat.setOutputPath(genJob, randomOuts);
     genJob.setOutputKeyClass(IntWritable.class);
     genJob.setOutputValueClass(IntWritable.class);
     genJob.setOutputFormat(TextOutputFormat.class);
@@ -472,7 +472,7 @@ public class TestMapRed extends TestCase {
     checkJob.setInputFormat(TextInputFormat.class);
     checkJob.setMapperClass(RandomCheckMapper.class);
 
-    checkJob.setOutputPath(intermediateOuts);
+    FileOutputFormat.setOutputPath(checkJob, intermediateOuts);
     checkJob.setOutputKeyClass(IntWritable.class);
     checkJob.setOutputValueClass(IntWritable.class);
     checkJob.setOutputFormat(MapFileOutputFormat.class);
@@ -495,7 +495,7 @@ public class TestMapRed extends TestCase {
     mergeJob.setInputFormat(SequenceFileInputFormat.class);
     mergeJob.setMapperClass(MergeMapper.class);
         
-    mergeJob.setOutputPath(finalOuts);
+    FileOutputFormat.setOutputPath(mergeJob, finalOuts);
     mergeJob.setOutputKeyClass(IntWritable.class);
     mergeJob.setOutputValueClass(IntWritable.class);
     mergeJob.setOutputFormat(SequenceFileOutputFormat.class);
@@ -599,7 +599,7 @@ public class TestMapRed extends TestCase {
       conf.setInt("io.sort.mb", 1);
       conf.setInputFormat(SequenceFileInputFormat.class);
       conf.setInputPath(inDir);
-      conf.setOutputPath(outDir);
+      FileOutputFormat.setOutputPath(conf, outDir);
       conf.setMapperClass(IdentityMapper.class);
       conf.setReducerClass(IdentityReducer.class);
       conf.setOutputKeyClass(Text.class);

+ 2 - 4
src/test/org/apache/hadoop/mapred/TestMiniMRClasspath.java

@@ -19,13 +19,11 @@
 package org.apache.hadoop.mapred;
 
 import java.io.*;
-import java.util.*;
 import junit.framework.TestCase;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.dfs.MiniDFSCluster;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Text;
@@ -69,7 +67,7 @@ public class TestMiniMRClasspath extends TestCase {
     conf.set("mapred.combine.class", "testjar.ClassWordCount$Reduce");
     conf.set("mapred.reducer.class", "testjar.ClassWordCount$Reduce");
     conf.setInputPath(inDir);
-    conf.setOutputPath(outDir);
+    FileOutputFormat.setOutputPath(conf, outDir);
     conf.setNumMapTasks(numMaps);
     conf.setNumReduceTasks(numReduces);
     //pass a job.jar already included in the hadoop build
@@ -122,7 +120,7 @@ public class TestMiniMRClasspath extends TestCase {
     conf.set("mapred.output.key.class", "testjar.ExternalWritable");
 
     conf.setInputPath(inDir);
-    conf.setOutputPath(outDir);
+    FileOutputFormat.setOutputPath(conf, outDir);
     conf.setNumMapTasks(numMaps);
     conf.setNumReduceTasks(numReduces);
     

+ 1 - 1
src/test/org/apache/hadoop/mapred/TestMiniMRMapRedDebugScript.java

@@ -134,7 +134,7 @@ public class TestMiniMRMapRedDebugScript extends TestCase {
     conf.setNumReduceTasks(0);
     conf.setMapDebugScript(debugScript);
     conf.setInputPath(inDir);
-    conf.setOutputPath(outDir);
+    FileOutputFormat.setOutputPath(conf, outDir);
     String TEST_ROOT_DIR = new Path(System.getProperty("test.build.data",
                                       "/tmp")).toString().replace(' ', '+');
     conf.set("test.build.data", TEST_ROOT_DIR);

+ 1 - 1
src/test/org/apache/hadoop/mapred/TestMiniMRTaskTempDir.java

@@ -114,7 +114,7 @@ public class TestMiniMRTaskTempDir extends TestCase {
     conf.setNumMapTasks(1);
     conf.setNumReduceTasks(0);
     conf.setInputPath(inDir);
-    conf.setOutputPath(outDir);
+    FileOutputFormat.setOutputPath(conf, outDir);
     String TEST_ROOT_DIR = new Path(System.getProperty("test.build.data",
                                       "/tmp")).toString().replace(' ', '+');
     conf.set("test.build.data", TEST_ROOT_DIR);

+ 1 - 1
src/test/org/apache/hadoop/mapred/TestMiniMRWithDFS.java

@@ -80,7 +80,7 @@ public class TestMiniMRWithDFS extends TestCase {
     conf.setCombinerClass(WordCount.Reduce.class);
     conf.setReducerClass(WordCount.Reduce.class);
     conf.setInputPath(inDir);
-    conf.setOutputPath(outDir);
+    FileOutputFormat.setOutputPath(conf, outDir);
     conf.setNumMapTasks(numMaps);
     conf.setNumReduceTasks(numReduces);
     RunningJob job = JobClient.runJob(conf);

+ 1 - 1
src/test/org/apache/hadoop/mapred/TestMultipleTextOutputFormat.java

@@ -81,7 +81,7 @@ public class TestMultipleTextOutputFormat extends TestCase {
   
   public void testFormat() throws Exception {
     JobConf job = new JobConf();
-    job.setOutputPath(workDir);
+    FileOutputFormat.setWorkOutputPath(job, workDir);
     FileSystem fs = workDir.getFileSystem(job);
     if (!fs.mkdirs(workDir)) {
       fail("Failed to create output directory");

+ 1 - 1
src/test/org/apache/hadoop/mapred/TestRackAwareTaskPlacement.java

@@ -178,7 +178,7 @@ public class TestRackAwareTaskPlacement extends TestCase {
     jobConf.setInputFormat(NonSplitableSequenceFileInputFormat.class);
     jobConf.setOutputFormat(SequenceFileOutputFormat.class);
     jobConf.setInputPath(inDir);
-    jobConf.setOutputPath(outputPath);
+    FileOutputFormat.setOutputPath(jobConf, outputPath);
     jobConf.setMapperClass(IdentityMapper.class);
     jobConf.setReducerClass(IdentityReducer.class);
     jobConf.setOutputKeyClass(BytesWritable.class);

+ 4 - 3
src/test/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java

@@ -75,7 +75,7 @@ public class TestSpecialCharactersInOutputPath extends TestCase {
     conf.setMapperClass(IdentityMapper.class);        
     conf.setReducerClass(IdentityReducer.class);
     conf.setInputPath(inDir);
-    conf.setOutputPath(outDir);
+    FileOutputFormat.setOutputPath(conf, outDir);
     conf.setNumMapTasks(numMaps);
     conf.setNumReduceTasks(numReduces);
       
@@ -123,9 +123,10 @@ public class TestSpecialCharactersInOutputPath extends TestCase {
   }
 
   /** generates output filenames with special characters */
-  static class SpecialTextOutputFormat extends TextOutputFormat {
+  static class SpecialTextOutputFormat<K,V> extends TextOutputFormat<K,V> {
     @Override
-    public RecordWriter getRecordWriter(FileSystem ignored, JobConf job, String name, Progressable progress) throws IOException {
+    public RecordWriter<K,V> getRecordWriter(FileSystem ignored, JobConf job,
+             String name, Progressable progress) throws IOException {
         return super.getRecordWriter(ignored, job, OUTPUT_FILENAME, progress);
     }
   }

+ 1 - 1
src/test/org/apache/hadoop/mapred/TestTextOutputFormat.java

@@ -43,7 +43,7 @@ public class TestTextOutputFormat extends TestCase {
   @SuppressWarnings("unchecked")
   public void testFormat() throws Exception {
     JobConf job = new JobConf();
-    job.setOutputPath(workDir);
+    FileOutputFormat.setWorkOutputPath(job, workDir);
     FileSystem fs = workDir.getFileSystem(job);
     if (!fs.mkdirs(workDir)) {
       fail("Failed to create output directory");

+ 3 - 3
src/test/org/apache/hadoop/mapred/ThreadedMapBenchmark.java

@@ -78,7 +78,7 @@ public class ThreadedMapBenchmark extends Configured implements Tool {
     public InputSplit[] getSplits(JobConf job, 
                                   int numSplits) throws IOException {
       InputSplit[] result = new InputSplit[numSplits];
-      Path outDir = job.getOutputPath();
+      Path outDir = FileOutputFormat.getOutputPath(job);
       for(int i=0; i < result.length; ++i) {
         result[i] = new FileSplit(new Path(outDir, "dummy-split-" + i),
                                   0, 1, (String[])null);
@@ -215,7 +215,7 @@ public class ThreadedMapBenchmark extends Configured implements Tool {
             String.valueOf(dataSizePerMap * 1024 * 1024));
     job.setNumReduceTasks(0); // none reduce
     job.setNumMapTasks(numMapsPerHost * cluster.getTaskTrackers());
-    job.setOutputPath(INPUT_DIR);
+    FileOutputFormat.setOutputPath(job, INPUT_DIR);
     
     FileSystem fs = FileSystem.get(job);
     fs.delete(BASE_DIR, true);
@@ -296,7 +296,7 @@ public class ThreadedMapBenchmark extends Configured implements Tool {
       job.setReducerClass(IdentityReducer.class);
       
       job.addInputPath(INPUT_DIR);
-      job.setOutputPath(OUTPUT_DIR);
+      FileOutputFormat.setOutputPath(job, OUTPUT_DIR);
       
       JobClient client = new JobClient(job);
       ClusterStatus cluster = client.getClusterStatus();

+ 2 - 1
src/test/org/apache/hadoop/mapred/jobcontrol/JobControlTestUtils.java

@@ -30,6 +30,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.MapReduceBase;
 import org.apache.hadoop.mapred.Mapper;
@@ -124,7 +125,7 @@ public class JobControlTestUtils {
       }
     }
     theJob.setMapperClass(DataCopy.class);
-    theJob.setOutputPath(outdir);
+    FileOutputFormat.setOutputPath(theJob, outdir);
     theJob.setOutputKeyClass(Text.class);
     theJob.setOutputValueClass(Text.class);
     theJob.setReducerClass(DataCopy.class);

+ 2 - 1
src/test/org/apache/hadoop/mapred/join/TestDatamerge.java

@@ -32,6 +32,7 @@ import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.Mapper;
@@ -221,7 +222,7 @@ public class TestDatamerge extends TestCase {
         SequenceFileInputFormat.class, src));
     job.setInt("testdatamerge.sources", srcs);
     job.setInputFormat(CompositeInputFormat.class);
-    job.setOutputPath(new Path(base, "out"));
+    FileOutputFormat.setOutputPath(job, new Path(base, "out"));
 
     job.setMapperClass(c);
     job.setReducerClass(c);

+ 1 - 1
src/test/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java

@@ -88,7 +88,7 @@ public class TestMultithreadedMapRunner extends HadoopTestCase {
     conf.setReducerClass(IDReduce.class);
 
     conf.setInputPath(inDir);
-    conf.setOutputPath(outDir);
+    FileOutputFormat.setOutputPath(conf, outDir);
 
     conf.setMapRunnerClass(MultithreadedMapRunner.class);
     

+ 1 - 1
src/test/org/apache/hadoop/mapred/lib/aggregate/TestAggregates.java

@@ -81,7 +81,7 @@ public class TestAggregates extends TestCase {
     job.setInputPath(INPUT_DIR);
     job.setInputFormat(TextInputFormat.class);
 
-    job.setOutputPath(OUTPUT_DIR);
+    FileOutputFormat.setOutputPath(job, OUTPUT_DIR);
     job.setOutputFormat(TextOutputFormat.class);
     job.setMapOutputKeyClass(Text.class);
     job.setMapOutputValueClass(Text.class);

+ 2 - 1
src/test/org/apache/hadoop/mapred/pipes/TestPipes.java

@@ -30,6 +30,7 @@ import org.apache.hadoop.dfs.MiniDFSCluster;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.MiniMRCluster;
 import org.apache.hadoop.mapred.OutputLogFilter;
@@ -146,7 +147,7 @@ public class TestPipes extends TestCase {
       Submitter.setIsJavaRecordReader(job, true);
       Submitter.setIsJavaRecordWriter(job, true);
       job.setInputPath(inputPath);
-      job.setOutputPath(outputPath);
+      FileOutputFormat.setOutputPath(job, outputPath);
       RunningJob result = Submitter.submitJob(job);
       assertTrue("pipes job failed", result.isSuccessful());
     }

+ 3 - 3
src/test/org/apache/hadoop/record/TestRecordMR.java

@@ -312,7 +312,7 @@ public class TestRecordMR extends TestCase {
     genJob.setInputFormat(SequenceFileInputFormat.class);
     genJob.setMapperClass(RandomGenMapper.class);
 
-    genJob.setOutputPath(randomOuts);
+    FileOutputFormat.setOutputPath(genJob, randomOuts);
     genJob.setOutputKeyClass(RecInt.class);
     genJob.setOutputValueClass(RecString.class);
     genJob.setOutputFormat(SequenceFileOutputFormat.class);
@@ -357,7 +357,7 @@ public class TestRecordMR extends TestCase {
     checkJob.setInputFormat(SequenceFileInputFormat.class);
     checkJob.setMapperClass(RandomCheckMapper.class);
 
-    checkJob.setOutputPath(intermediateOuts);
+    FileOutputFormat.setOutputPath(checkJob, intermediateOuts);
     checkJob.setOutputKeyClass(RecInt.class);
     checkJob.setOutputValueClass(RecString.class);
     checkJob.setOutputFormat(SequenceFileOutputFormat.class);
@@ -380,7 +380,7 @@ public class TestRecordMR extends TestCase {
     mergeJob.setInputFormat(SequenceFileInputFormat.class);
     mergeJob.setMapperClass(MergeMapper.class);
         
-    mergeJob.setOutputPath(finalOuts);
+    FileOutputFormat.setOutputPath(mergeJob, finalOuts);
     mergeJob.setOutputKeyClass(RecInt.class);
     mergeJob.setOutputValueClass(RecInt.class);
     mergeJob.setOutputFormat(SequenceFileOutputFormat.class);

+ 2 - 2
src/test/testshell/ExternalMapReduce.java

@@ -25,9 +25,9 @@ import java.util.Iterator;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.Mapper;
@@ -88,7 +88,7 @@ public class ExternalMapReduce
     JobConf testConf = new JobConf(commandConf, ExternalMapReduce.class);
     testConf.setJobName("external job");
     testConf.setInputPath(input);
-    testConf.setOutputPath(outDir);
+    FileOutputFormat.setOutputPath(testConf, outDir);
     testConf.setMapperClass(ExternalMapReduce.class);
     testConf.setReducerClass(ExternalMapReduce.class);
     testConf.setNumReduceTasks(1);