001/* 002 * Copyright (c) 2016-2018 Chris K Wensel <chris@wensel.net>. All Rights Reserved. 003 * Copyright (c) 2007-2017 Xplenty, Inc. All Rights Reserved. 004 * 005 * Project and contact information: http://www.cascading.org/ 006 * 007 * This file is part of the Cascading project. 008 * 009 * Licensed under the Apache License, Version 2.0 (the "License"); 010 * you may not use this file except in compliance with the License. 011 * You may obtain a copy of the License at 012 * 013 * http://www.apache.org/licenses/LICENSE-2.0 014 * 015 * Unless required by applicable law or agreed to in writing, software 016 * distributed under the License is distributed on an "AS IS" BASIS, 017 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 018 * See the License for the specific language governing permissions and 019 * limitations under the License. 020 */ 021 022package cascading.tap.hadoop.util; 023 024import java.io.IOException; 025import java.net.URI; 026import java.util.HashMap; 027import java.util.Map; 028import java.util.concurrent.atomic.AtomicInteger; 029 030import cascading.flow.hadoop.util.HadoopUtil; 031import cascading.tap.Tap; 032import org.apache.hadoop.conf.Configuration; 033import org.apache.hadoop.fs.FileStatus; 034import org.apache.hadoop.fs.FileSystem; 035import org.apache.hadoop.fs.Path; 036import org.apache.hadoop.mapred.FileOutputFormat; 037import org.slf4j.Logger; 038import org.slf4j.LoggerFactory; 039 040import static cascading.flow.hadoop.util.HadoopUtil.asJobConfInstance; 041 042public class Hadoop18TapUtil 043 { 044 /** Field LOG */ 045 private static final Logger LOG = LoggerFactory.getLogger( Hadoop18TapUtil.class ); 046 047 /** The Hadoop temporary path used to prevent collisions */ 048 public static final String TEMPORARY_PATH = "_temporary"; 049 050 private static final Map<String, AtomicInteger> pathCounts = new HashMap<String, AtomicInteger>(); 051 052 /** 053 * should only be called if not in a Flow 054 * 055 * @param conf 056 * @throws IOException 057 */ 058 public static void setupJob( Configuration conf ) throws IOException 059 { 060 Path outputPath = FileOutputFormat.getOutputPath( asJobConfInstance( conf ) ); 061 062 if( outputPath == null ) 063 return; 064 065 if( getFSSafe( conf, outputPath ) == null ) 066 return; 067 068 String taskID = conf.get( "mapred.task.id", conf.get( "mapreduce.task.id" ) ); 069 070 if( taskID == null ) // need to stuff a fake id 071 { 072 String mapper = conf.getBoolean( "mapred.task.is.map", conf.getBoolean( "mapreduce.task.is.map", true ) ) ? "m" : "r"; 073 String value = String.format( "attempt_%012d_0000_%s_000000_0", (int) Math.rint( System.currentTimeMillis() ), mapper ); 074 conf.set( "mapred.task.id", value ); 075 conf.set( "mapreduce.task.id", value ); 076 } 077 078 makeTempPath( conf ); 079 080 if( writeDirectlyToWorkingPath( conf, outputPath ) ) 081 { 082 LOG.info( "writing directly to output path: {}", outputPath ); 083 setWorkOutputPath( conf, outputPath ); 084 return; 085 } 086 087 // "mapred.work.output.dir" 088 Path taskOutputPath = getTaskOutputPath( conf ); 089 setWorkOutputPath( conf, taskOutputPath ); 090 } 091 092 public static synchronized void setupTask( Configuration conf ) throws IOException 093 { 094 String workpath = conf.get( "mapred.work.output.dir" ); 095 096 if( workpath == null ) 097 return; 098 099 FileSystem fs = getFSSafe( conf, new Path( workpath ) ); 100 101 if( fs == null ) 102 return; 103 104 String taskId = conf.get( "mapred.task.id", conf.get( "mapreduce.task.id" ) ); 105 106 LOG.info( "setting up task: '{}' - {}", taskId, workpath ); 107 108 AtomicInteger integer = pathCounts.get( workpath ); 109 110 if( integer == null ) 111 { 112 integer = new AtomicInteger(); 113 pathCounts.put( workpath, integer ); 114 } 115 116 integer.incrementAndGet(); 117 } 118 119 public static boolean needsTaskCommit( Configuration conf ) throws IOException 120 { 121 String workpath = conf.get( "mapred.work.output.dir" ); 122 123 if( workpath == null ) 124 return false; 125 126 Path taskOutputPath = new Path( workpath ); 127 128 if( taskOutputPath != null ) 129 { 130 FileSystem fs = getFSSafe( conf, taskOutputPath ); 131 132 if( fs == null ) 133 return false; 134 135 if( fs.exists( taskOutputPath ) ) 136 return true; 137 } 138 139 return false; 140 } 141 142 /** 143 * copies all files from the taskoutputpath to the outputpath 144 * 145 * @param conf 146 */ 147 public static boolean commitTask( Configuration conf ) throws IOException 148 { 149 Path taskOutputPath = new Path( conf.get( "mapred.work.output.dir" ) ); 150 151 FileSystem fs = getFSSafe( conf, taskOutputPath ); 152 153 if( fs == null ) 154 return false; 155 156 AtomicInteger integer = pathCounts.get( taskOutputPath.toString() ); 157 158 if( integer.decrementAndGet() != 0 ) 159 return false; 160 161 String taskId = conf.get( "mapred.task.id", conf.get( "mapreduce.task.id" ) ); 162 163 LOG.info( "committing task: '{}' - {}", taskId, taskOutputPath ); 164 165 if( taskOutputPath != null ) 166 { 167 if( writeDirectlyToWorkingPath( conf, taskOutputPath ) ) 168 return true; 169 170 if( fs.exists( taskOutputPath ) ) 171 { 172 Path jobOutputPath = taskOutputPath.getParent().getParent(); 173 // Move the task outputs to their final place 174 moveTaskOutputs( conf, fs, jobOutputPath, taskOutputPath ); 175 176 // Delete the temporary task-specific output directory 177 if( !fs.delete( taskOutputPath, true ) ) 178 LOG.info( "failed to delete the temporary output directory of task: '{}' - {}", taskId, taskOutputPath ); 179 180 LOG.info( "saved output of task '{}' to {}", taskId, jobOutputPath ); 181 } 182 } 183 184 return true; 185 } 186 187 /** 188 * Called from flow step to remove temp dirs 189 * 190 * @param conf 191 * @throws IOException 192 */ 193 public static void cleanupTapMetaData( Configuration conf, Tap tap ) throws IOException 194 { 195 cleanTempPath( conf, new Path( tap.getIdentifier() ) ); 196 } 197 198 public static void writeSuccessMarker( Configuration conf ) throws IOException 199 { 200 writeSuccessMarker( conf, FileOutputFormat.getOutputPath( asJobConfInstance( conf ) ) ); 201 } 202 203 public static void writeSuccessMarker( Configuration conf, Path outputPath ) throws IOException 204 { 205 if( conf.getBoolean( "mapreduce.fileoutputcommitter.marksuccessfuljobs", true ) ) 206 { 207 LOG.info( "writing success marker to {}", outputPath ); 208 209 Path markerPath = new Path( outputPath, "_SUCCESS" ); 210 FileSystem fs = markerPath.getFileSystem( conf ); 211 212 fs.create( markerPath ).close(); 213 } 214 } 215 216 /** 217 * May only be called once. should only be called if not in a flow 218 * 219 * @param conf 220 */ 221 public static void cleanupJob( Configuration conf ) throws IOException 222 { 223 if( HadoopUtil.isInflow( conf ) ) 224 return; 225 226 Path outputPath = FileOutputFormat.getOutputPath( asJobConfInstance( conf ) ); 227 228 cleanTempPath( conf, outputPath ); 229 } 230 231 private static synchronized void cleanTempPath( Configuration conf, Path outputPath ) throws IOException 232 { 233 // do the clean up of temporary directory 234 235 if( outputPath != null ) 236 { 237 FileSystem fileSys = getFSSafe( conf, outputPath ); 238 239 if( fileSys == null ) 240 return; 241 242 if( !fileSys.exists( outputPath ) ) 243 return; 244 245 Path tmpDir = new Path( outputPath, TEMPORARY_PATH ); 246 247 LOG.info( "deleting temp path {}", tmpDir ); 248 249 if( fileSys.exists( tmpDir ) ) 250 fileSys.delete( tmpDir, true ); 251 } 252 } 253 254 private static FileSystem getFSSafe( Configuration conf, Path tmpDir ) 255 { 256 try 257 { 258 return tmpDir.getFileSystem( conf ); 259 } 260 catch( IOException e ) 261 { 262 // ignore 263 } 264 265 return null; 266 } 267 268 private static Path getTaskOutputPath( Configuration conf ) 269 { 270 String taskId = conf.get( "mapred.task.id", conf.get( "mapreduce.task.id" ) ); 271 272 Path p = new Path( FileOutputFormat.getOutputPath( asJobConfInstance( conf ) ), TEMPORARY_PATH + Path.SEPARATOR + "_" + taskId ); 273 274 try 275 { 276 FileSystem fs = p.getFileSystem( conf ); 277 return p.makeQualified( fs ); 278 } 279 catch( IOException ie ) 280 { 281 return p; 282 } 283 } 284 285 static void setWorkOutputPath( Configuration conf, Path outputDir ) 286 { 287 outputDir = new Path( asJobConfInstance( conf ).getWorkingDirectory(), outputDir ); 288 conf.set( "mapred.work.output.dir", outputDir.toString() ); 289 } 290 291 public static void makeTempPath( Configuration conf ) throws IOException 292 { 293 // create job specific temporary directory in output path 294 Path outputPath = FileOutputFormat.getOutputPath( asJobConfInstance( conf ) ); 295 296 if( outputPath != null ) 297 { 298 Path tmpDir = new Path( outputPath, TEMPORARY_PATH ); 299 FileSystem fileSys = tmpDir.getFileSystem( conf ); 300 301 if( !fileSys.exists( tmpDir ) && !fileSys.mkdirs( tmpDir ) ) 302 LOG.error( "mkdirs failed to create {}", tmpDir ); 303 } 304 } 305 306 private static void moveTaskOutputs( Configuration conf, FileSystem fs, Path jobOutputDir, Path taskOutput ) throws IOException 307 { 308 String taskId = conf.get( "mapred.task.id", conf.get( "mapreduce.task.id" ) ); 309 310 if( fs.isFile( taskOutput ) ) 311 { 312 Path finalOutputPath = getFinalPath( jobOutputDir, taskOutput, getTaskOutputPath( conf ) ); 313 if( !fs.rename( taskOutput, finalOutputPath ) ) 314 { 315 if( !fs.delete( finalOutputPath, true ) ) 316 throw new IOException( "Failed to delete earlier output of task: " + taskId ); 317 318 if( !fs.rename( taskOutput, finalOutputPath ) ) 319 throw new IOException( "Failed to save output of task: " + taskId ); 320 } 321 322 LOG.debug( "Moved {} to {}", taskOutput, finalOutputPath ); 323 } 324 else if( fs.getFileStatus( taskOutput ).isDir() ) 325 { 326 FileStatus[] paths = fs.listStatus( taskOutput ); 327 Path finalOutputPath = getFinalPath( jobOutputDir, taskOutput, getTaskOutputPath( conf ) ); 328 fs.mkdirs( finalOutputPath ); 329 if( paths != null ) 330 { 331 for( FileStatus path : paths ) 332 moveTaskOutputs( conf, fs, jobOutputDir, path.getPath() ); 333 } 334 } 335 } 336 337 private static Path getFinalPath( Path jobOutputDir, Path taskOutput, Path taskOutputPath ) throws IOException 338 { 339 URI taskOutputUri = taskOutput.toUri(); 340 URI relativePath = taskOutputPath.toUri().relativize( taskOutputUri ); 341 if( taskOutputUri == relativePath ) 342 {//taskOutputPath is not a parent of taskOutput 343 throw new IOException( "Can not get the relative path: base = " + taskOutputPath + " child = " + taskOutput ); 344 } 345 if( relativePath.getPath().length() > 0 ) 346 { 347 return new Path( jobOutputDir, relativePath.getPath() ); 348 } 349 else 350 { 351 return jobOutputDir; 352 } 353 } 354 355 /** used in AWS EMR to disable temp paths on some file systems, s3. */ 356 private static boolean writeDirectlyToWorkingPath( Configuration conf, Path path ) 357 { 358 FileSystem fs = getFSSafe( conf, path ); 359 360 if( fs == null ) 361 return false; 362 363 boolean result = conf.getBoolean( "mapred.output.direct." + fs.getClass().getSimpleName(), false ); 364 365 if( result ) 366 LOG.info( "output direct is enabled for this fs: " + fs.getName() ); 367 368 return result; 369 } 370 }