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  }