001/*
002 * Copyright (c) 2007-2022 The Cascading Authors. All Rights Reserved.
003 *
004 * Project and contact information: https://cascading.wensel.net/
005 *
006 * This file is part of the Cascading project.
007 *
008 * Licensed under the Apache License, Version 2.0 (the "License");
009 * you may not use this file except in compliance with the License.
010 * You may obtain a copy of the License at
011 *
012 *     http://www.apache.org/licenses/LICENSE-2.0
013 *
014 * Unless required by applicable law or agreed to in writing, software
015 * distributed under the License is distributed on an "AS IS" BASIS,
016 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
017 * See the License for the specific language governing permissions and
018 * limitations under the License.
019 */
020
021package cascading.flow.hadoop;
022
023import java.io.IOException;
024import java.util.Arrays;
025import java.util.HashSet;
026import java.util.Iterator;
027import java.util.Map;
028import java.util.Set;
029
030import cascading.CascadingException;
031import cascading.flow.FlowElement;
032import cascading.flow.FlowException;
033import cascading.flow.FlowNode;
034import cascading.flow.FlowProcess;
035import cascading.flow.FlowRuntimeProps;
036import cascading.flow.hadoop.planner.HadoopFlowStepJob;
037import cascading.flow.hadoop.util.HadoopMRUtil;
038import cascading.flow.hadoop.util.HadoopUtil;
039import cascading.flow.planner.BaseFlowStep;
040import cascading.flow.planner.FlowStepJob;
041import cascading.flow.planner.PlatformInfo;
042import cascading.flow.planner.graph.ElementGraph;
043import cascading.flow.planner.process.FlowNodeGraph;
044import cascading.flow.planner.process.ProcessEdge;
045import cascading.management.state.ClientState;
046import cascading.pipe.CoGroup;
047import cascading.tap.Tap;
048import cascading.tap.hadoop.io.MultiInputFormat;
049import cascading.tap.hadoop.util.Hadoop18TapUtil;
050import cascading.tap.hadoop.util.TempHfs;
051import cascading.tuple.Fields;
052import cascading.tuple.hadoop.TupleSerialization;
053import cascading.tuple.hadoop.util.CoGroupingComparator;
054import cascading.tuple.hadoop.util.CoGroupingPartitioner;
055import cascading.tuple.hadoop.util.GroupingComparator;
056import cascading.tuple.hadoop.util.GroupingPartitioner;
057import cascading.tuple.hadoop.util.GroupingSortingComparator;
058import cascading.tuple.hadoop.util.GroupingSortingPartitioner;
059import cascading.tuple.hadoop.util.IndexTupleCoGroupingComparator;
060import cascading.tuple.hadoop.util.ReverseGroupingSortingComparator;
061import cascading.tuple.hadoop.util.ReverseTupleComparator;
062import cascading.tuple.hadoop.util.TupleComparator;
063import cascading.tuple.io.KeyIndexTuple;
064import cascading.tuple.io.KeyTuple;
065import cascading.tuple.io.TuplePair;
066import cascading.tuple.io.ValueIndexTuple;
067import cascading.tuple.io.ValueTuple;
068import cascading.util.ProcessLogger;
069import cascading.util.Util;
070import cascading.util.Version;
071import org.apache.hadoop.filecache.DistributedCache;
072import org.apache.hadoop.fs.Path;
073import org.apache.hadoop.io.serializer.Serialization;
074import org.apache.hadoop.mapred.FileOutputFormat;
075import org.apache.hadoop.mapred.JobConf;
076import org.apache.hadoop.mapred.OutputFormat;
077
078import static cascading.flow.hadoop.util.HadoopUtil.*;
079
080/**
081 *
082 */
083public class HadoopFlowStep extends BaseFlowStep<JobConf>
084  {
085  protected HadoopFlowStep()
086    {
087    }
088
089  protected HadoopFlowStep( String name, int ordinal )
090    {
091    super( name, ordinal );
092    }
093
094  public HadoopFlowStep( ElementGraph elementGraph, FlowNodeGraph flowNodeGraph )
095    {
096    super( elementGraph, flowNodeGraph );
097    }
098
099  @Override
100  public Map<Object, Object> getConfigAsProperties()
101    {
102    return HadoopUtil.createProperties( getConfig() );
103    }
104
105  public JobConf createInitializedConfig( FlowProcess<JobConf> flowProcess, JobConf parentConfig )
106    {
107    JobConf conf = parentConfig == null ? new JobConf() : HadoopUtil.copyJobConf( parentConfig );
108
109    // disable warning
110    conf.setBoolean( "mapred.used.genericoptionsparser", true );
111
112    conf.setJobName( getStepDisplayName( conf.getInt( "cascading.display.id.truncate", Util.ID_LENGTH ) ) );
113
114    conf.setOutputKeyClass( KeyTuple.class );
115    conf.setOutputValueClass( ValueTuple.class );
116
117    conf.setMapRunnerClass( FlowMapper.class );
118    conf.setReducerClass( FlowReducer.class );
119
120    Set<String> serializations = getFieldDeclaredSerializations( Serialization.class );
121
122    // set for use by the shuffling phase
123    TupleSerialization.setSerializations( conf, serializations );
124
125    initFromSources( flowProcess, conf );
126
127    initFromSink( flowProcess, conf );
128
129    initFromTraps( flowProcess, conf );
130
131    initFromStepConfigDef( conf );
132
133    int numSinkParts = getSink().getScheme().getNumSinkParts();
134
135    if( numSinkParts != 0 )
136      {
137      // if no reducer, set num map tasks to control parts
138      if( getGroup() != null )
139        conf.setNumReduceTasks( numSinkParts );
140      else
141        conf.setNumMapTasks( numSinkParts );
142      }
143    else if( getGroup() != null )
144      {
145      int gatherPartitions = conf.getNumReduceTasks();
146
147      if( gatherPartitions == 0 )
148        gatherPartitions = conf.getInt( FlowRuntimeProps.GATHER_PARTITIONS, 0 );
149
150      if( gatherPartitions == 0 )
151        throw new FlowException( getName(), "a default number of gather partitions must be set, see FlowRuntimeProps" );
152
153      conf.setNumReduceTasks( gatherPartitions );
154      }
155
156    conf.setOutputKeyComparatorClass( TupleComparator.class );
157
158    ProcessEdge processEdge = Util.getFirst( getFlowNodeGraph().edgeSet() );
159
160    if( getGroup() == null )
161      {
162      conf.setNumReduceTasks( 0 ); // disable reducers
163      }
164    else
165      {
166      // must set map output defaults when performing a reduce
167      conf.setMapOutputKeyClass( KeyTuple.class );
168      conf.setMapOutputValueClass( ValueTuple.class );
169      conf.setPartitionerClass( GroupingPartitioner.class );
170
171      // handles the case the groupby sort should be reversed
172      if( getGroup().isSortReversed() )
173        conf.setOutputKeyComparatorClass( ReverseTupleComparator.class );
174
175      Integer ordinal = (Integer) Util.getFirst( processEdge.getSinkExpectedOrdinals() );
176
177      addComparators( conf, "cascading.group.comparator", getGroup().getKeySelectors(), (Fields) processEdge.getResolvedKeyFields().get( ordinal ) );
178
179      if( getGroup().isGroupBy() )
180        addComparators( conf, "cascading.sort.comparator", getGroup().getSortingSelectors(), (Fields) processEdge.getResolvedSortFields().get( ordinal ) );
181
182      if( !getGroup().isGroupBy() )
183        {
184        conf.setPartitionerClass( CoGroupingPartitioner.class );
185        conf.setMapOutputKeyClass( KeyIndexTuple.class ); // allows groups to be sorted by index
186        conf.setMapOutputValueClass( ValueIndexTuple.class );
187        conf.setOutputKeyComparatorClass( IndexTupleCoGroupingComparator.class ); // sorts by group, then by index
188        conf.setOutputValueGroupingComparator( CoGroupingComparator.class );
189        }
190
191      if( getGroup().isSorted() )
192        {
193        conf.setPartitionerClass( GroupingSortingPartitioner.class );
194        conf.setMapOutputKeyClass( TuplePair.class );
195
196        if( getGroup().isSortReversed() )
197          conf.setOutputKeyComparatorClass( ReverseGroupingSortingComparator.class );
198        else
199          conf.setOutputKeyComparatorClass( GroupingSortingComparator.class );
200
201        // no need to supply a reverse comparator, only equality is checked
202        conf.setOutputValueGroupingComparator( GroupingComparator.class );
203        }
204      }
205
206    // if we write type information into the stream, we can perform comparisons in indexed tuples
207    // thus, if the edge is a CoGroup and they keys are not common types, force writing of type information
208    if( processEdge != null && ifCoGroupAndKeysHaveCommonTypes( this, processEdge.getFlowElement(), processEdge.getResolvedKeyFields() ) )
209      {
210      conf.set( "cascading.node.ordinals", Util.join( processEdge.getSinkExpectedOrdinals(), "," ) );
211      addFields( conf, "cascading.node.key.fields", processEdge.getResolvedKeyFields() );
212      addFields( conf, "cascading.node.sort.fields", processEdge.getResolvedSortFields() );
213      addFields( conf, "cascading.node.value.fields", processEdge.getResolvedValueFields() );
214      }
215
216    // perform last so init above will pass to tasks
217    String versionString = Version.getRelease();
218
219    if( versionString != null )
220      conf.set( "cascading.version", versionString );
221
222    conf.set( CASCADING_FLOW_STEP_ID, getID() );
223    conf.set( "cascading.flow.step.num", Integer.toString( getOrdinal() ) );
224
225    HadoopUtil.setIsInflow( conf );
226
227    Iterator<FlowNode> iterator = getFlowNodeGraph().getTopologicalIterator();
228
229    FlowNode mapperNode = iterator.next();
230    FlowNode reducerNode = iterator.hasNext() ? iterator.next() : null;
231
232    if( reducerNode != null )
233      reducerNode.addProcessAnnotation( FlowRuntimeProps.GATHER_PARTITIONS, Integer.toString( conf.getNumReduceTasks() ) );
234
235    String mapState = pack( mapperNode, conf );
236    String reduceState = pack( reducerNode, conf );
237
238    // hadoop 20.2 doesn't like dist cache when using local mode
239    int maxSize = Short.MAX_VALUE;
240
241    int length = mapState.length() + reduceState.length();
242
243    if( isHadoopLocalMode( conf ) || length < maxSize ) // seems safe
244      {
245      conf.set( "cascading.flow.step.node.map", mapState );
246
247      if( !Util.isEmpty( reduceState ) )
248        conf.set( "cascading.flow.step.node.reduce", reduceState );
249      }
250    else
251      {
252      conf.set( "cascading.flow.step.node.map.path", HadoopMRUtil.writeStateToDistCache( conf, getID(), "map", mapState ) );
253
254      if( !Util.isEmpty( reduceState ) )
255        conf.set( "cascading.flow.step.node.reduce.path", HadoopMRUtil.writeStateToDistCache( conf, getID(), "reduce", reduceState ) );
256      }
257
258    return conf;
259    }
260
261  private static boolean ifCoGroupAndKeysHaveCommonTypes( ProcessLogger processLogger, FlowElement flowElement, Map<Integer, Fields> resolvedKeyFields )
262    {
263    if( !( flowElement instanceof CoGroup ) )
264      return true;
265
266    if( resolvedKeyFields == null || resolvedKeyFields.size() < 2 )
267      return true;
268
269    Iterator<Map.Entry<Integer, Fields>> iterator = resolvedKeyFields.entrySet().iterator();
270
271    Fields fields = iterator.next().getValue();
272
273    while( iterator.hasNext() )
274      {
275      Fields next = iterator.next().getValue();
276
277      if( !Arrays.equals( fields.getTypesClasses(), next.getTypesClasses() ) )
278        {
279        processLogger.logWarn( "unable to perform: {}, on mismatched join types and optimize serialization with type exclusion, fields: {} & {}", flowElement, fields, next );
280        return false;
281        }
282      }
283
284    return true;
285    }
286
287  public boolean isHadoopLocalMode( JobConf conf )
288    {
289    return HadoopUtil.isLocal( conf );
290    }
291
292  protected FlowStepJob<JobConf> createFlowStepJob( ClientState clientState, FlowProcess<JobConf> flowProcess, JobConf initializedStepConfig )
293    {
294    try
295      {
296      return new HadoopFlowStepJob( clientState, this, initializedStepConfig );
297      }
298    catch( NoClassDefFoundError error )
299      {
300      PlatformInfo platformInfo = HadoopUtil.getPlatformInfo( JobConf.class, "org/apache/hadoop", "Hadoop MR" );
301      String message = "unable to load platform specific class, please verify Hadoop cluster version: '%s', matches the Hadoop platform build dependency and associated FlowConnector, cascading-hadoop or cascading-hadoop3-mr1";
302
303      logError( String.format( message, platformInfo.toString() ), error );
304
305      throw error;
306      }
307    }
308
309  /**
310   * Method clean removes any temporary files used by this FlowStep instance. It will log any IOExceptions thrown.
311   *
312   * @param config of type JobConf
313   */
314  public void clean( JobConf config )
315    {
316    String stepStatePath = config.get( "cascading.flow.step.path" );
317
318    if( stepStatePath != null )
319      {
320      try
321        {
322        HadoopUtil.removeStateFromDistCache( config, stepStatePath );
323        }
324      catch( IOException exception )
325        {
326        logWarn( "unable to remove step state file: " + stepStatePath, exception );
327        }
328      }
329
330    if( tempSink != null )
331      {
332      try
333        {
334        tempSink.deleteResource( config );
335        }
336      catch( Exception exception )
337        {
338        // sink all exceptions, don't fail app
339        logWarn( "unable to remove temporary file: " + tempSink, exception );
340        }
341      }
342
343    // safe way to handle zero sinks case
344    for( Tap sink : getSinkTaps() )
345      cleanIntermediateData( config, sink );
346
347    for( Tap tap : getTraps() )
348      cleanTapMetaData( config, tap );
349    }
350
351  protected void cleanIntermediateData( JobConf config, Tap sink )
352    {
353    if( sink.isTemporary() && ( getFlow().getFlowStats().isSuccessful() || getFlow().getRunID() == null ) )
354      {
355      try
356        {
357        sink.deleteResource( config );
358        }
359      catch( Exception exception )
360        {
361        // sink all exceptions, don't fail app
362        logWarn( "unable to remove temporary file: " + sink, exception );
363        }
364      }
365    else
366      {
367      cleanTapMetaData( config, sink );
368      }
369    }
370
371  private void cleanTapMetaData( JobConf jobConf, Tap tap )
372    {
373    try
374      {
375      Hadoop18TapUtil.cleanupTapMetaData( jobConf, tap );
376      }
377    catch( IOException exception )
378      {
379      // ignore exception
380      }
381    }
382
383  private void initFromTraps( FlowProcess<JobConf> flowProcess, JobConf conf, Map<String, Tap> traps )
384    {
385    if( !traps.isEmpty() )
386      {
387      JobConf trapConf = HadoopUtil.copyJobConf( conf );
388
389      for( Tap tap : traps.values() )
390        tap.sinkConfInit( flowProcess, trapConf );
391      }
392    }
393
394  protected void initFromSources( FlowProcess<JobConf> flowProcess, JobConf conf )
395    {
396    // handles case where same tap is used on multiple branches
397    // we do not want to init the same tap multiple times
398    Set<Tap> uniqueSources = getUniqueStreamedSources();
399
400    JobConf[] streamedJobs = new JobConf[ uniqueSources.size() ];
401    int i = 0;
402
403    for( Tap tap : uniqueSources )
404      {
405      if( tap.getIdentifier() == null )
406        throw new IllegalStateException( "tap may not have null identifier: " + tap.toString() );
407
408      streamedJobs[ i ] = flowProcess.copyConfig( conf );
409
410      streamedJobs[ i ].set( "cascading.step.source", Tap.id( tap ) );
411
412      tap.sourceConfInit( flowProcess, streamedJobs[ i ] );
413
414      i++;
415      }
416
417    Set<Tap> accumulatedSources = getAllAccumulatedSources();
418
419    for( Tap tap : accumulatedSources )
420      {
421      JobConf accumulatedJob = flowProcess.copyConfig( conf );
422
423      tap.sourceConfInit( flowProcess, accumulatedJob );
424
425      Map<String, String> map = flowProcess.diffConfigIntoMap( conf, accumulatedJob );
426      conf.set( "cascading.node.accumulated.source.conf." + Tap.id( tap ), pack( map, conf ) );
427
428      try
429        {
430        if( DistributedCache.getCacheFiles( accumulatedJob ) != null )
431          DistributedCache.setCacheFiles( DistributedCache.getCacheFiles( accumulatedJob ), conf );
432        }
433      catch( IOException exception )
434        {
435        throw new CascadingException( exception );
436        }
437      }
438
439    MultiInputFormat.addInputFormat( conf, streamedJobs ); //must come last
440    }
441
442  private void initFromStepConfigDef( final JobConf conf )
443    {
444    initConfFromStepConfigDef( new ConfigurationSetter( conf ) );
445    }
446
447  /**
448   * sources are specific to step, remove all known accumulated sources, if any
449   */
450  private Set<Tap> getUniqueStreamedSources()
451    {
452    Set<Tap> allAccumulatedSources = getAllAccumulatedSources();
453
454    // if a source is dual accumulated and streamed, honor the streamed annotation
455    allAccumulatedSources.removeAll( getAllStreamedSources() );
456
457    // start with the full source declaration and removed undesired taps. the above methods are dependent on
458    // annotations which may not exist, so we are safeguarding a declared tap is treated streamed by default
459    HashSet<Tap> set = new HashSet<>( sources.keySet() );
460
461    set.removeAll( allAccumulatedSources );
462
463    return set;
464    }
465
466  protected void initFromSink( FlowProcess<JobConf> flowProcess, JobConf conf )
467    {
468    // init sink first so tempSink can take precedence
469    if( getSink() != null )
470      getSink().sinkConfInit( flowProcess, conf );
471
472    Class<? extends OutputFormat> outputFormat = conf.getClass( "mapred.output.format.class", null, OutputFormat.class );
473    boolean isFileOutputFormat = false;
474
475    if( outputFormat != null )
476      isFileOutputFormat = FileOutputFormat.class.isAssignableFrom( outputFormat );
477
478    Path outputPath = FileOutputFormat.getOutputPath( conf );
479
480    // if no output path is set, we need to substitute an alternative if the OutputFormat is file based
481    // PartitionTap won't set the output, but will set an OutputFormat
482    // MultiSinkTap won't set the output or set the OutputFormat
483    // Non file based OutputFormats don't have an output path, but do have an OutputFormat set (JDBCTap..)
484    if( outputPath == null && ( isFileOutputFormat || outputFormat == null ) )
485      tempSink = new TempHfs( conf, "tmp:/" + new Path( getSink().getIdentifier() ).toUri().getPath(), true );
486
487    // tempSink exists because sink is writeDirect
488    if( tempSink != null )
489      tempSink.sinkConfInit( flowProcess, conf );
490    }
491
492  protected void initFromTraps( FlowProcess<JobConf> flowProcess, JobConf conf )
493    {
494    initFromTraps( flowProcess, conf, getTrapMap() );
495    }
496  }