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