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.mapreduce.lib.output; 020 021 import java.io.IOException; 022 import java.net.URI; 023 024 import org.apache.commons.logging.Log; 025 import org.apache.commons.logging.LogFactory; 026 import org.apache.hadoop.classification.InterfaceAudience; 027 import org.apache.hadoop.classification.InterfaceStability; 028 import org.apache.hadoop.conf.Configuration; 029 import org.apache.hadoop.fs.FileStatus; 030 import org.apache.hadoop.fs.FileSystem; 031 import org.apache.hadoop.fs.Path; 032 import org.apache.hadoop.mapreduce.JobContext; 033 import org.apache.hadoop.mapreduce.JobStatus; 034 import org.apache.hadoop.mapreduce.MRJobConfig; 035 import org.apache.hadoop.mapreduce.OutputCommitter; 036 import org.apache.hadoop.mapreduce.TaskAttemptContext; 037 import org.apache.hadoop.mapreduce.TaskAttemptID; 038 039 /** An {@link OutputCommitter} that commits files specified 040 * in job output directory i.e. ${mapreduce.output.fileoutputformat.outputdir}. 041 **/ 042 @InterfaceAudience.Public 043 @InterfaceStability.Stable 044 public class FileOutputCommitter extends OutputCommitter { 045 046 private static final Log LOG = LogFactory.getLog(FileOutputCommitter.class); 047 048 /** 049 * Temporary directory name 050 */ 051 protected static final String TEMP_DIR_NAME = "_temporary"; 052 public static final String SUCCEEDED_FILE_NAME = "_SUCCESS"; 053 static final String SUCCESSFUL_JOB_OUTPUT_DIR_MARKER = 054 "mapreduce.fileoutputcommitter.marksuccessfuljobs"; 055 private FileSystem outputFileSystem = null; 056 private Path outputPath = null; 057 private Path workPath = null; 058 059 /** 060 * Create a file output committer 061 * @param outputPath the job's output path 062 * @param context the task's context 063 * @throws IOException 064 */ 065 public FileOutputCommitter(Path outputPath, 066 TaskAttemptContext context) throws IOException { 067 if (outputPath != null) { 068 this.outputPath = outputPath; 069 outputFileSystem = outputPath.getFileSystem(context.getConfiguration()); 070 workPath = new Path(outputPath, 071 getTaskAttemptBaseDirName(context)) 072 .makeQualified(outputFileSystem); 073 } 074 } 075 076 /** 077 * Create the temporary directory that is the root of all of the task 078 * work directories. 079 * @param context the job's context 080 */ 081 public void setupJob(JobContext context) throws IOException { 082 if (outputPath != null) { 083 Path tmpDir = new Path(outputPath, getJobAttemptBaseDirName(context) + 084 Path.SEPARATOR + FileOutputCommitter.TEMP_DIR_NAME); 085 FileSystem fileSys = tmpDir.getFileSystem(context.getConfiguration()); 086 if (!fileSys.mkdirs(tmpDir)) { 087 LOG.error("Mkdirs failed to create " + tmpDir.toString()); 088 } 089 } 090 } 091 092 // True if the job requires output.dir marked on successful job. 093 // Note that by default it is set to true. 094 private boolean shouldMarkOutputDir(Configuration conf) { 095 return conf.getBoolean(SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, true); 096 } 097 098 // Create a _success file in the job's output dir 099 private void markOutputDirSuccessful(MRJobConfig context) throws IOException { 100 if (outputPath != null) { 101 // create a file in the output folder to mark the job completion 102 Path filePath = new Path(outputPath, SUCCEEDED_FILE_NAME); 103 outputFileSystem.create(filePath).close(); 104 } 105 } 106 107 /** 108 * Move all job output to the final place. 109 * Delete the temporary directory, including all of the work directories. 110 * Create a _SUCCESS file to make it as successful. 111 * @param context the job's context 112 */ 113 public void commitJob(JobContext context) throws IOException { 114 if (outputPath != null) { 115 //delete the task temp directory from the current jobtempdir 116 Path tmpDir = new Path(outputPath, getJobAttemptBaseDirName(context) + 117 Path.SEPARATOR + FileOutputCommitter.TEMP_DIR_NAME); 118 FileSystem fileSys = tmpDir.getFileSystem(context.getConfiguration()); 119 if (fileSys.exists(tmpDir)) { 120 fileSys.delete(tmpDir, true); 121 } else { 122 LOG.warn("Task temp dir could not be deleted " + tmpDir); 123 } 124 125 //move the job output to final place 126 Path jobOutputPath = 127 new Path(outputPath, getJobAttemptBaseDirName(context)); 128 moveJobOutputs(outputFileSystem, jobOutputPath, outputPath, jobOutputPath); 129 130 // delete the _temporary folder and create a _done file in the o/p folder 131 cleanupJob(context); 132 if (shouldMarkOutputDir(context.getConfiguration())) { 133 markOutputDirSuccessful(context); 134 } 135 } 136 } 137 138 /** 139 * Move job output to final location 140 * @param fs Filesystem handle 141 * @param origJobOutputPath The original location of the job output 142 * Required to generate the relative path for correct moving of data. 143 * @param finalOutputDir The final output directory to which the job output 144 * needs to be moved 145 * @param jobOutput The current job output directory being moved 146 * @throws IOException 147 */ 148 private void moveJobOutputs(FileSystem fs, final Path origJobOutputPath, 149 Path finalOutputDir, Path jobOutput) throws IOException { 150 LOG.debug("Told to move job output from " + jobOutput 151 + " to " + finalOutputDir + 152 " and orig job output path is " + origJobOutputPath); 153 if (fs.isFile(jobOutput)) { 154 Path finalOutputPath = 155 getFinalPath(finalOutputDir, jobOutput, origJobOutputPath); 156 if (!fs.rename(jobOutput, finalOutputPath)) { 157 if (!fs.delete(finalOutputPath, true)) { 158 throw new IOException("Failed to delete earlier output of job"); 159 } 160 if (!fs.rename(jobOutput, finalOutputPath)) { 161 throw new IOException("Failed to save output of job"); 162 } 163 } 164 LOG.debug("Moved job output file from " + jobOutput + " to " + 165 finalOutputPath); 166 } else if (fs.getFileStatus(jobOutput).isDirectory()) { 167 LOG.debug("Job output file " + jobOutput + " is a dir"); 168 FileStatus[] paths = fs.listStatus(jobOutput); 169 Path finalOutputPath = 170 getFinalPath(finalOutputDir, jobOutput, origJobOutputPath); 171 fs.mkdirs(finalOutputPath); 172 LOG.debug("Creating dirs along job output path " + finalOutputPath); 173 if (paths != null) { 174 for (FileStatus path : paths) { 175 moveJobOutputs(fs, origJobOutputPath, finalOutputDir, path.getPath()); 176 } 177 } 178 } 179 } 180 181 @Override 182 @Deprecated 183 public void cleanupJob(JobContext context) throws IOException { 184 if (outputPath != null) { 185 Path tmpDir = new Path(outputPath, FileOutputCommitter.TEMP_DIR_NAME); 186 FileSystem fileSys = tmpDir.getFileSystem(context.getConfiguration()); 187 if (fileSys.exists(tmpDir)) { 188 fileSys.delete(tmpDir, true); 189 } 190 } else { 191 LOG.warn("Output Path is null in cleanup"); 192 } 193 } 194 195 /** 196 * Delete the temporary directory, including all of the work directories. 197 * @param context the job's context 198 */ 199 @Override 200 public void abortJob(JobContext context, JobStatus.State state) 201 throws IOException { 202 // delete the _temporary folder 203 cleanupJob(context); 204 } 205 206 /** 207 * No task setup required. 208 */ 209 @Override 210 public void setupTask(TaskAttemptContext context) throws IOException { 211 // FileOutputCommitter's setupTask doesn't do anything. Because the 212 // temporary task directory is created on demand when the 213 // task is writing. 214 } 215 216 /** 217 * Move the files from the work directory to the job output directory 218 * @param context the task context 219 */ 220 public void commitTask(TaskAttemptContext context) 221 throws IOException { 222 TaskAttemptID attemptId = context.getTaskAttemptID(); 223 if (workPath != null) { 224 context.progress(); 225 if (outputFileSystem.exists(workPath)) { 226 // Move the task outputs to the current job attempt output dir 227 Path jobOutputPath = 228 new Path(outputPath, getJobAttemptBaseDirName(context)); 229 moveTaskOutputs(context, outputFileSystem, jobOutputPath, workPath); 230 // Delete the temporary task-specific output directory 231 if (!outputFileSystem.delete(workPath, true)) { 232 LOG.warn("Failed to delete the temporary output" + 233 " directory of task: " + attemptId + " - " + workPath); 234 } 235 LOG.info("Saved output of task '" + attemptId + "' to " + 236 outputPath); 237 } 238 } 239 } 240 241 /** 242 * Move all of the files from the work directory to the final output 243 * @param context the task context 244 * @param fs the output file system 245 * @param jobOutputDir the final output direcotry 246 * @param taskOutput the work path 247 * @throws IOException 248 */ 249 private void moveTaskOutputs(TaskAttemptContext context, 250 FileSystem fs, 251 Path jobOutputDir, 252 Path taskOutput) 253 throws IOException { 254 TaskAttemptID attemptId = context.getTaskAttemptID(); 255 context.progress(); 256 LOG.debug("Told to move taskoutput from " + taskOutput 257 + " to " + jobOutputDir); 258 if (fs.isFile(taskOutput)) { 259 Path finalOutputPath = getFinalPath(jobOutputDir, taskOutput, 260 workPath); 261 if (!fs.rename(taskOutput, finalOutputPath)) { 262 if (!fs.delete(finalOutputPath, true)) { 263 throw new IOException("Failed to delete earlier output of task: " + 264 attemptId); 265 } 266 if (!fs.rename(taskOutput, finalOutputPath)) { 267 throw new IOException("Failed to save output of task: " + 268 attemptId); 269 } 270 } 271 LOG.debug("Moved " + taskOutput + " to " + finalOutputPath); 272 } else if(fs.getFileStatus(taskOutput).isDirectory()) { 273 LOG.debug("Taskoutput " + taskOutput + " is a dir"); 274 FileStatus[] paths = fs.listStatus(taskOutput); 275 Path finalOutputPath = getFinalPath(jobOutputDir, taskOutput, workPath); 276 fs.mkdirs(finalOutputPath); 277 LOG.debug("Creating dirs along path " + finalOutputPath); 278 if (paths != null) { 279 for (FileStatus path : paths) { 280 moveTaskOutputs(context, fs, jobOutputDir, path.getPath()); 281 } 282 } 283 } 284 } 285 286 /** 287 * Delete the work directory 288 * @throws IOException 289 */ 290 @Override 291 public void abortTask(TaskAttemptContext context) throws IOException { 292 if (workPath != null) { 293 context.progress(); 294 outputFileSystem.delete(workPath, true); 295 } 296 } 297 298 /** 299 * Find the final name of a given output file, given the job output directory 300 * and the work directory. 301 * @param jobOutputDir the job's output directory 302 * @param taskOutput the specific task output file 303 * @param taskOutputPath the job's work directory 304 * @return the final path for the specific output file 305 * @throws IOException 306 */ 307 private Path getFinalPath(Path jobOutputDir, Path taskOutput, 308 Path taskOutputPath) throws IOException { 309 URI taskOutputUri = taskOutput.makeQualified(outputFileSystem.getUri(), 310 outputFileSystem.getWorkingDirectory()).toUri(); 311 URI taskOutputPathUri = 312 taskOutputPath.makeQualified( 313 outputFileSystem.getUri(), 314 outputFileSystem.getWorkingDirectory()).toUri(); 315 URI relativePath = taskOutputPathUri.relativize(taskOutputUri); 316 if (taskOutputUri == relativePath) { 317 throw new IOException("Can not get the relative path: base = " + 318 taskOutputPathUri + " child = " + taskOutputUri); 319 } 320 if (relativePath.getPath().length() > 0) { 321 return new Path(jobOutputDir, relativePath.getPath()); 322 } else { 323 return jobOutputDir; 324 } 325 } 326 327 /** 328 * Did this task write any files in the work directory? 329 * @param context the task's context 330 */ 331 @Override 332 public boolean needsTaskCommit(TaskAttemptContext context 333 ) throws IOException { 334 return workPath != null && outputFileSystem.exists(workPath); 335 } 336 337 /** 338 * Get the directory that the task should write results into 339 * @return the work directory 340 * @throws IOException 341 */ 342 public Path getWorkPath() throws IOException { 343 return workPath; 344 } 345 346 @Override 347 public boolean isRecoverySupported() { 348 return true; 349 } 350 351 @Override 352 public void recoverTask(TaskAttemptContext context) 353 throws IOException { 354 context.progress(); 355 Path jobOutputPath = 356 new Path(outputPath, getJobAttemptBaseDirName(context)); 357 int previousAttempt = 358 context.getConfiguration().getInt( 359 MRJobConfig.APPLICATION_ATTEMPT_ID, 0) - 1; 360 if (previousAttempt < 0) { 361 throw new IOException ("Cannot recover task output for first attempt..."); 362 } 363 364 Path pathToRecover = 365 new Path(outputPath, getJobAttemptBaseDirName(previousAttempt)); 366 LOG.debug("Trying to recover task from " + pathToRecover 367 + " into " + jobOutputPath); 368 if (outputFileSystem.exists(pathToRecover)) { 369 // Move the task outputs to their final place 370 moveJobOutputs(outputFileSystem, 371 pathToRecover, jobOutputPath, pathToRecover); 372 LOG.info("Saved output of job to " + jobOutputPath); 373 } 374 } 375 376 protected static String getJobAttemptBaseDirName(JobContext context) { 377 int appAttemptId = 378 context.getConfiguration().getInt( 379 MRJobConfig.APPLICATION_ATTEMPT_ID, 0); 380 return getJobAttemptBaseDirName(appAttemptId); 381 } 382 383 protected static String getJobAttemptBaseDirName(int appAttemptId) { 384 return FileOutputCommitter.TEMP_DIR_NAME + Path.SEPARATOR + 385 + appAttemptId; 386 } 387 388 protected static String getTaskAttemptBaseDirName( 389 TaskAttemptContext context) { 390 return getJobAttemptBaseDirName(context) + Path.SEPARATOR + 391 FileOutputCommitter.TEMP_DIR_NAME + Path.SEPARATOR + 392 "_" + context.getTaskAttemptID().toString(); 393 } 394 }