001    /**
002     * Licensed to the Apache Software Foundation (ASF) under one
003     * or more contributor license agreements.  See the NOTICE file
004     * distributed with this work for additional information
005     * regarding copyright ownership.  The ASF licenses this file
006     * to you under the Apache License, Version 2.0 (the
007     * "License"); you may not use this file except in compliance
008     * with the License.  You may obtain a copy of the License at
009     *
010     *     http://www.apache.org/licenses/LICENSE-2.0
011     *
012     * Unless required by applicable law or agreed to in writing, software
013     * distributed under the License is distributed on an "AS IS" BASIS,
014     * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015     * See the License for the specific language governing permissions and
016     * limitations under the License.
017     */
018    
019    package org.apache.hadoop.mapred;
020    
021    import java.io.IOException;
022    import java.text.NumberFormat;
023    
024    import org.apache.hadoop.classification.InterfaceAudience;
025    import org.apache.hadoop.classification.InterfaceAudience.Private;
026    import org.apache.hadoop.classification.InterfaceStability;
027    import org.apache.hadoop.fs.FileSystem;
028    import org.apache.hadoop.fs.Path;
029    import org.apache.hadoop.io.compress.CompressionCodec;
030    import org.apache.hadoop.mapred.FileAlreadyExistsException;
031    import org.apache.hadoop.mapreduce.security.TokenCache;
032    import org.apache.hadoop.util.Progressable;
033    
034    /** A base class for {@link OutputFormat}. */
035    @InterfaceAudience.Public
036    @InterfaceStability.Stable
037    public abstract class FileOutputFormat<K, V> implements OutputFormat<K, V> {
038    
039      @Deprecated
040      public static enum Counter { 
041        BYTES_WRITTEN
042      }
043      
044      /**
045       * Set whether the output of the job is compressed.
046       * @param conf the {@link JobConf} to modify
047       * @param compress should the output of the job be compressed?
048       */
049      public static void setCompressOutput(JobConf conf, boolean compress) {
050        conf.setBoolean(org.apache.hadoop.mapreduce.lib.output.
051          FileOutputFormat.COMPRESS, compress);
052      }
053      
054      /**
055       * Is the job output compressed?
056       * @param conf the {@link JobConf} to look in
057       * @return <code>true</code> if the job output should be compressed,
058       *         <code>false</code> otherwise
059       */
060      public static boolean getCompressOutput(JobConf conf) {
061        return conf.getBoolean(org.apache.hadoop.mapreduce.lib.output.
062          FileOutputFormat.COMPRESS, false);
063      }
064      
065      /**
066       * Set the {@link CompressionCodec} to be used to compress job outputs.
067       * @param conf the {@link JobConf} to modify
068       * @param codecClass the {@link CompressionCodec} to be used to
069       *                   compress the job outputs
070       */
071      public static void 
072      setOutputCompressorClass(JobConf conf, 
073                               Class<? extends CompressionCodec> codecClass) {
074        setCompressOutput(conf, true);
075        conf.setClass(org.apache.hadoop.mapreduce.lib.output.
076          FileOutputFormat.COMPRESS_CODEC, codecClass, 
077                      CompressionCodec.class);
078      }
079      
080      /**
081       * Get the {@link CompressionCodec} for compressing the job outputs.
082       * @param conf the {@link JobConf} to look in
083       * @param defaultValue the {@link CompressionCodec} to return if not set
084       * @return the {@link CompressionCodec} to be used to compress the 
085       *         job outputs
086       * @throws IllegalArgumentException if the class was specified, but not found
087       */
088      public static Class<? extends CompressionCodec> 
089      getOutputCompressorClass(JobConf conf, 
090                                           Class<? extends CompressionCodec> defaultValue) {
091        Class<? extends CompressionCodec> codecClass = defaultValue;
092        
093        String name = conf.get(org.apache.hadoop.mapreduce.lib.output.
094          FileOutputFormat.COMPRESS_CODEC);
095        if (name != null) {
096          try {
097            codecClass = 
098                    conf.getClassByName(name).asSubclass(CompressionCodec.class);
099          } catch (ClassNotFoundException e) {
100            throw new IllegalArgumentException("Compression codec " + name + 
101                                               " was not found.", e);
102          }
103        }
104        return codecClass;
105      }
106      
107      public abstract RecordWriter<K, V> getRecordWriter(FileSystem ignored,
108                                                   JobConf job, String name,
109                                                   Progressable progress)
110        throws IOException;
111    
112      public void checkOutputSpecs(FileSystem ignored, JobConf job) 
113        throws FileAlreadyExistsException, 
114               InvalidJobConfException, IOException {
115        // Ensure that the output directory is set and not already there
116        Path outDir = getOutputPath(job);
117        if (outDir == null && job.getNumReduceTasks() != 0) {
118          throw new InvalidJobConfException("Output directory not set in JobConf.");
119        }
120        if (outDir != null) {
121          FileSystem fs = outDir.getFileSystem(job);
122          // normalize the output directory
123          outDir = fs.makeQualified(outDir);
124          setOutputPath(job, outDir);
125          
126          // get delegation token for the outDir's file system
127          TokenCache.obtainTokensForNamenodes(job.getCredentials(), 
128                                              new Path[] {outDir}, job);
129          
130          // check its existence
131          if (fs.exists(outDir)) {
132            throw new FileAlreadyExistsException("Output directory " + outDir + 
133                                                 " already exists");
134          }
135        }
136      }
137    
138      /**
139       * Set the {@link Path} of the output directory for the map-reduce job.
140       *
141       * @param conf The configuration of the job.
142       * @param outputDir the {@link Path} of the output directory for 
143       * the map-reduce job.
144       */
145      public static void setOutputPath(JobConf conf, Path outputDir) {
146        outputDir = new Path(conf.getWorkingDirectory(), outputDir);
147        conf.set(org.apache.hadoop.mapreduce.lib.output.
148          FileOutputFormat.OUTDIR, outputDir.toString());
149      }
150    
151      /**
152       * Set the {@link Path} of the task's temporary output directory 
153       * for the map-reduce job.
154       * 
155       * <p><i>Note</i>: Task output path is set by the framework.
156       * </p>
157       * @param conf The configuration of the job.
158       * @param outputDir the {@link Path} of the output directory 
159       * for the map-reduce job.
160       */
161      @Private
162      public static void setWorkOutputPath(JobConf conf, Path outputDir) {
163        outputDir = new Path(conf.getWorkingDirectory(), outputDir);
164        conf.set(JobContext.TASK_OUTPUT_DIR, outputDir.toString());
165      }
166      
167      /**
168       * Get the {@link Path} to the output directory for the map-reduce job.
169       * 
170       * @return the {@link Path} to the output directory for the map-reduce job.
171       * @see FileOutputFormat#getWorkOutputPath(JobConf)
172       */
173      public static Path getOutputPath(JobConf conf) {
174        String name = conf.get(org.apache.hadoop.mapreduce.lib.output.
175          FileOutputFormat.OUTDIR);
176        return name == null ? null: new Path(name);
177      }
178      
179      /**
180       *  Get the {@link Path} to the task's temporary output directory 
181       *  for the map-reduce job
182       *  
183       * <h4 id="SideEffectFiles">Tasks' Side-Effect Files</h4>
184       * 
185       * <p><i>Note:</i> The following is valid only if the {@link OutputCommitter}
186       *  is {@link FileOutputCommitter}. If <code>OutputCommitter</code> is not 
187       *  a <code>FileOutputCommitter</code>, the task's temporary output
188       *  directory is same as {@link #getOutputPath(JobConf)} i.e.
189       *  <tt>${mapreduce.output.fileoutputformat.outputdir}$</tt></p>
190       *  
191       * <p>Some applications need to create/write-to side-files, which differ from
192       * the actual job-outputs.
193       * 
194       * <p>In such cases there could be issues with 2 instances of the same TIP 
195       * (running simultaneously e.g. speculative tasks) trying to open/write-to the
196       * same file (path) on HDFS. Hence the application-writer will have to pick 
197       * unique names per task-attempt (e.g. using the attemptid, say 
198       * <tt>attempt_200709221812_0001_m_000000_0</tt>), not just per TIP.</p> 
199       * 
200       * <p>To get around this the Map-Reduce framework helps the application-writer 
201       * out by maintaining a special 
202       * <tt>${mapreduce.output.fileoutputformat.outputdir}/_temporary/_${taskid}</tt> 
203       * sub-directory for each task-attempt on HDFS where the output of the 
204       * task-attempt goes. On successful completion of the task-attempt the files 
205       * in the <tt>${mapreduce.output.fileoutputformat.outputdir}/_temporary/_${taskid}</tt> (only) 
206       * are <i>promoted</i> to <tt>${mapreduce.output.fileoutputformat.outputdir}</tt>. Of course, the 
207       * framework discards the sub-directory of unsuccessful task-attempts. This 
208       * is completely transparent to the application.</p>
209       * 
210       * <p>The application-writer can take advantage of this by creating any 
211       * side-files required in <tt>${mapreduce.task.output.dir}</tt> during execution 
212       * of his reduce-task i.e. via {@link #getWorkOutputPath(JobConf)}, and the 
213       * framework will move them out similarly - thus she doesn't have to pick 
214       * unique paths per task-attempt.</p>
215       * 
216       * <p><i>Note</i>: the value of <tt>${mapreduce.task.output.dir}</tt> during 
217       * execution of a particular task-attempt is actually 
218       * <tt>${mapreduce.output.fileoutputformat.outputdir}/_temporary/_{$taskid}</tt>, and this value is 
219       * set by the map-reduce framework. So, just create any side-files in the 
220       * path  returned by {@link #getWorkOutputPath(JobConf)} from map/reduce 
221       * task to take advantage of this feature.</p>
222       * 
223       * <p>The entire discussion holds true for maps of jobs with 
224       * reducer=NONE (i.e. 0 reduces) since output of the map, in that case, 
225       * goes directly to HDFS.</p> 
226       * 
227       * @return the {@link Path} to the task's temporary output directory 
228       * for the map-reduce job.
229       */
230      public static Path getWorkOutputPath(JobConf conf) {
231        String name = conf.get(JobContext.TASK_OUTPUT_DIR);
232        return name == null ? null: new Path(name);
233      }
234    
235      /**
236       * Helper function to create the task's temporary output directory and 
237       * return the path to the task's output file.
238       * 
239       * @param conf job-configuration
240       * @param name temporary task-output filename
241       * @return path to the task's temporary output file
242       * @throws IOException
243       */
244      public static Path getTaskOutputPath(JobConf conf, String name) 
245      throws IOException {
246        // ${mapred.out.dir}
247        Path outputPath = getOutputPath(conf);
248        if (outputPath == null) {
249          throw new IOException("Undefined job output-path");
250        }
251    
252        OutputCommitter committer = conf.getOutputCommitter();
253        Path workPath = outputPath;
254        TaskAttemptContext context = 
255          new TaskAttemptContextImpl(conf,
256                                     TaskAttemptID.forName(conf.get(
257                                         JobContext.TASK_ATTEMPT_ID)));
258        if (committer instanceof FileOutputCommitter) {
259          workPath = ((FileOutputCommitter)committer).getWorkPath(context,
260                                                                  outputPath);
261        }
262        
263        // ${mapred.out.dir}/_temporary/_${taskid}/${name}
264        return new Path(workPath, name);
265      } 
266    
267      /**
268       * Helper function to generate a name that is unique for the task.
269       *
270       * <p>The generated name can be used to create custom files from within the
271       * different tasks for the job, the names for different tasks will not collide
272       * with each other.</p>
273       *
274       * <p>The given name is postfixed with the task type, 'm' for maps, 'r' for
275       * reduces and the task partition number. For example, give a name 'test'
276       * running on the first map o the job the generated name will be
277       * 'test-m-00000'.</p>
278       *
279       * @param conf the configuration for the job.
280       * @param name the name to make unique.
281       * @return a unique name accross all tasks of the job.
282       */
283      public static String getUniqueName(JobConf conf, String name) {
284        int partition = conf.getInt(JobContext.TASK_PARTITION, -1);
285        if (partition == -1) {
286          throw new IllegalArgumentException(
287            "This method can only be called from within a Job");
288        }
289    
290        String taskType = (conf.getBoolean(JobContext.TASK_ISMAP, true)) ? "m" : "r";
291    
292        NumberFormat numberFormat = NumberFormat.getInstance();
293        numberFormat.setMinimumIntegerDigits(5);
294        numberFormat.setGroupingUsed(false);
295    
296        return name + "-" + taskType + "-" + numberFormat.format(partition);
297      }
298    
299      /**
300       * Helper function to generate a {@link Path} for a file that is unique for
301       * the task within the job output directory.
302       *
303       * <p>The path can be used to create custom files from within the map and
304       * reduce tasks. The path name will be unique for each task. The path parent
305       * will be the job output directory.</p>ls
306       *
307       * <p>This method uses the {@link #getUniqueName} method to make the file name
308       * unique for the task.</p>
309       *
310       * @param conf the configuration for the job.
311       * @param name the name for the file.
312       * @return a unique path accross all tasks of the job.
313       */
314      public static Path getPathForCustomFile(JobConf conf, String name) {
315        return new Path(getWorkOutputPath(conf), getUniqueName(conf, name));
316      }
317    }
318