001/* 002 * Copyright (c) 2016-2017 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.flow.hadoop; 023 024import java.util.ArrayList; 025import java.util.Collection; 026import java.util.HashMap; 027import java.util.LinkedList; 028import java.util.List; 029import java.util.Map; 030import java.util.Properties; 031import java.util.concurrent.ExecutionException; 032 033import cascading.flow.hadoop.util.HadoopUtil; 034import cascading.flow.planner.PlatformInfo; 035import cascading.flow.planner.process.FlowStepGraph; 036import cascading.tap.SinkMode; 037import cascading.tap.Tap; 038import cascading.tap.hadoop.Hfs; 039import org.apache.hadoop.fs.Path; 040import org.apache.hadoop.mapred.JobConf; 041 042import static cascading.flow.planner.graph.ElementGraphs.asFlowElementGraph; 043import static cascading.util.Util.asList; 044 045/** 046 * Class MultiMapReduceFlow is a {@link cascading.flow.hadoop.HadoopFlow} subclass that supports custom MapReduce jobs 047 * pre-configured via one or more {@link JobConf} objects. 048 * <p> 049 * Use this class to group multiple JobConf instances together as a single Flow. MultiMapReduceFlow will automatically 050 * topologically order the JobConf instances and schedule them on the cluster once {@link #start()} or {@link #complete()} 051 * are called. 052 * <p> 053 * If you have a single JobConf instance, see {@link MapReduceFlow} as a alternative to this class. 054 * <p> 055 * This class will not delete any sinks before execution, it is up to the developer to make sure any intermediate and 056 * sink paths be removed/deleted before calling {@link #start()} or {@link #complete()}, otherwise Hadoop will throw 057 * an exception. 058 * <p> 059 * JobConf instances can be incrementally added at any point before the {@link #complete()} method is called. But they must 060 * logically (topologically) come after any previously provided JobConf instances. In practice the Flow will fail if 061 * the input source path is missing because a prior JobConf was not provided before the Flow was started. 062 * <p> 063 * The ordering is done by comparing the input and output paths of the given JobConf instances. By default, this class 064 * only works with JobConf instances that read and write from the Hadoop FileSystem (HDFS) (any path that would work 065 * with the {@link Hfs} Tap. 066 * <p> 067 * If the configured JobConf instance uses some other identifier instead of Hadoop FS paths, you should override the 068 * {@link #createSources(org.apache.hadoop.mapred.JobConf)}, {@link #createSinks(org.apache.hadoop.mapred.JobConf)}, and 069 * {@link #createTraps(org.apache.hadoop.mapred.JobConf)} methods to properly resolve the configured paths into 070 * usable {@link Tap} instances. By default createTraps returns an empty collection and should probably be left alone. 071 * <p> 072 * MultiMapReduceFlow supports both org.apache.hadoop.mapred.* and org.apache.hadoop.mapreduce.* API Jobs. 073 */ 074public class MultiMapReduceFlow extends BaseMapReduceFlow 075 { 076 /** Field tapCache */ 077 private Map<String, Tap> tapCache = new HashMap<>(); 078 /** Field queuedSteps */ 079 private List<MapReduceFlowStep> queuedSteps = new LinkedList<>(); 080 /** Field completeCalled */ 081 private volatile boolean completeCalled = false; 082 /** Field block */ 083 private final Object lock = new Object(); 084 085 /** 086 * Constructor MultiMapReduceFlow creates a new MultiMapReduceFlow instance. 087 * 088 * @param name of String 089 * @param jobConf of JobConf 090 * @param jobConfs of JobConf... 091 */ 092 public MultiMapReduceFlow( String name, JobConf jobConf, JobConf... jobConfs ) 093 { 094 this( HadoopUtil.getPlatformInfo( JobConf.class, "org/apache/hadoop", "Hadoop MR" ), new Properties(), name ); 095 096 initializeFrom( asList( jobConf, jobConfs ) ); 097 } 098 099 /** 100 * Constructor MultiMapReduceFlow creates a new MultiMapReduceFlow instance. 101 * 102 * @param properties of Map 103 * @param name of String 104 * @param jobConf of JobConf 105 * @param jobConfs of JobConf... 106 */ 107 public MultiMapReduceFlow( Map<Object, Object> properties, String name, JobConf jobConf, JobConf... jobConfs ) 108 { 109 this( HadoopUtil.getPlatformInfo( JobConf.class, "org/apache/hadoop", "Hadoop MR" ), properties, name, null ); 110 111 initializeFrom( asList( jobConf, jobConfs ) ); 112 } 113 114 /** 115 * Constructor MultiMapReduceFlow creates a new MultiMapReduceFlow instance. 116 * 117 * @param properties of Map 118 * @param name of String 119 * @param flowDescriptor of Map 120 * @param jobConf of JobConf 121 * @param jobConfs of JobConf... 122 */ 123 public MultiMapReduceFlow( Map<Object, Object> properties, String name, Map<String, String> flowDescriptor, JobConf jobConf, JobConf... jobConfs ) 124 { 125 this( HadoopUtil.getPlatformInfo( JobConf.class, "org/apache/hadoop", "Hadoop MR" ), properties, name, flowDescriptor ); 126 127 initializeFrom( asList( jobConf, jobConfs ) ); 128 } 129 130 /** 131 * Constructor MultiMapReduceFlow creates a new MultiMapReduceFlow instance. 132 * 133 * @param properties of Map 134 * @param name of String 135 * @param flowDescriptor of Map 136 * @param stopJobsOnExit of boolean 137 * @param jobConf of JobConf 138 * @param jobConfs of JobConf... 139 */ 140 public MultiMapReduceFlow( Map<Object, Object> properties, String name, Map<String, String> flowDescriptor, boolean stopJobsOnExit, JobConf jobConf, JobConf... jobConfs ) 141 { 142 this( HadoopUtil.getPlatformInfo( JobConf.class, "org/apache/hadoop", "Hadoop MR" ), properties, name, flowDescriptor ); 143 this.stopJobsOnExit = stopJobsOnExit; 144 145 initializeFrom( asList( jobConf, jobConfs ) ); 146 } 147 148 /** 149 * Constructor MultiMapReduceFlow creates a new MultiMapReduceFlow instance. 150 * 151 * @param platformInfo of PlatformInfo 152 * @param properties of Map 153 * @param name of String 154 */ 155 protected MultiMapReduceFlow( PlatformInfo platformInfo, Map<Object, Object> properties, String name ) 156 { 157 this( platformInfo, properties, name, null ); 158 } 159 160 /** 161 * Constructor MultiMapReduceFlow creates a new MultiMapReduceFlow instance. 162 * 163 * @param platformInfo of PlatformInfo 164 * @param properties of Map 165 * @param name of String 166 * @param flowDescriptor of Map 167 */ 168 protected MultiMapReduceFlow( PlatformInfo platformInfo, Map<Object, Object> properties, String name, Map<String, String> flowDescriptor ) 169 { 170 super( platformInfo, properties, name, flowDescriptor, false ); 171 } 172 173 protected void initializeFrom( List<JobConf> jobConfs ) 174 { 175 List<MapReduceFlowStep> steps = new ArrayList<>(); 176 177 for( JobConf jobConf : jobConfs ) 178 steps.add( createMapReduceFlowStep( jobConf ) ); 179 180 updateWithFlowSteps( steps ); 181 } 182 183 protected MapReduceFlowStep createMapReduceFlowStep( JobConf jobConf ) 184 { 185 return new MapReduceFlowStep( this, jobConf ); 186 } 187 188 public void notifyComplete() 189 { 190 completeCalled = true; 191 192 synchronized( lock ) 193 { 194 // forces blockingContinuePollingSteps to stop blocking 195 lock.notifyAll(); 196 } 197 } 198 199 @Override 200 public void complete() 201 { 202 notifyComplete(); 203 204 super.complete(); 205 } 206 207 @Override 208 protected boolean spawnSteps() throws InterruptedException, ExecutionException 209 { 210 // continue to spawn jobs until no longer required 211 while( !stop && throwable == null ) 212 { 213 if( !blockingContinuePollingSteps() ) 214 return true; 215 216 if( isInfoEnabled() ) 217 { 218 logInfo( "updated" ); 219 220 for( Tap source : getSourcesCollection() ) 221 logInfo( " source: " + source ); 222 for( Tap sink : getSinksCollection() ) 223 logInfo( " sink: " + sink ); 224 } 225 226 // will not return until all current steps are complete, or one failed 227 if( !super.spawnSteps() ) 228 return false; 229 } 230 231 return true; 232 } 233 234 protected boolean blockingContinuePollingSteps() 235 { 236 synchronized( lock ) 237 { 238 // block until queue has items, or complete is called 239 while( queuedSteps.isEmpty() && !completeCalled ) 240 { 241 try 242 { 243 lock.wait(); 244 } 245 catch( InterruptedException exception ) 246 { 247 // do nothing 248 } 249 } 250 251 updateWithFlowSteps( queuedSteps ).clear(); 252 } 253 254 if( getEligibleJobsSize() != 0 ) // new ones were added 255 return true; 256 257 return !completeCalled; 258 } 259 260 @Override 261 protected Tap createTap( JobConf jobConf, Path path, SinkMode sinkMode ) 262 { 263 Tap tap = tapCache.get( path.toString() ); 264 265 if( tap == null ) 266 { 267 tap = super.createTap( jobConf, path, sinkMode ); 268 tapCache.put( path.toString(), tap ); 269 } 270 271 return tap; 272 } 273 274 public void attachFlowStep( JobConf jobConf ) 275 { 276 if( completeCalled ) 277 throw new IllegalStateException( "cannot attach new FlowStep after complete() has been called" ); 278 279 addFlowStep( createMapReduceFlowStep( jobConf ) ); 280 } 281 282 protected void addFlowStep( MapReduceFlowStep flowStep ) 283 { 284 synchronized( lock ) 285 { 286 queuedSteps.add( flowStep ); 287 lock.notifyAll(); 288 } 289 } 290 291 protected FlowStepGraph getOrCreateFlowStepGraph() 292 { 293 FlowStepGraph flowStepGraph = getFlowStepGraph(); 294 295 if( flowStepGraph == null ) 296 { 297 flowStepGraph = new FlowStepGraph(); 298 setFlowStepGraph( flowStepGraph ); 299 } 300 301 return flowStepGraph; 302 } 303 304 protected Collection<MapReduceFlowStep> updateWithFlowSteps( Collection<MapReduceFlowStep> flowSteps ) 305 { 306 if( flowSteps.isEmpty() ) 307 return flowSteps; 308 309 FlowStepGraph flowStepGraph = getOrCreateFlowStepGraph(); 310 311 updateFlowStepGraph( flowStepGraph, flowSteps ); 312 313 setFlowElementGraph( asFlowElementGraph( platformInfo, flowStepGraph ) ); 314 315 removeListeners( getSourcesCollection() ); 316 removeListeners( getSinksCollection() ); 317 removeListeners( getTrapsCollection() ); 318 319 // re-adds listeners 320 setSources( flowStepGraph.getSourceTapsMap() ); 321 setSinks( flowStepGraph.getSinkTapsMap() ); 322 setTraps( flowStepGraph.getTrapsMap() ); 323 324 // this mirrors BaseFlow#initialize() 325 326 initSteps(); 327 328 if( flowStats == null ) 329 flowStats = createPrepareFlowStats(); // must be last 330 331 if( !isJobsMapInitialized() ) 332 initializeNewJobsMap(); 333 else 334 updateJobsMap(); 335 336 initializeChildStats(); 337 338 return flowSteps; 339 } 340 341 protected FlowStepGraph updateFlowStepGraph( FlowStepGraph flowStepGraph, Collection<MapReduceFlowStep> flowSteps ) 342 { 343 for( MapReduceFlowStep flowStep : flowSteps ) 344 flowStepGraph.addVertex( flowStep ); 345 346 flowStepGraph.bindEdges(); 347 348 return flowStepGraph; 349 } 350 }