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