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