001    /*
002     * Copyright (c) 2007-2014 Concurrent, Inc. All Rights Reserved.
003     *
004     * Project and contact information: http://www.cascading.org/
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    
021    package cascading.flow.hadoop;
022    
023    import java.io.IOException;
024    import java.util.Collections;
025    import java.util.HashMap;
026    import java.util.HashSet;
027    import java.util.Iterator;
028    import java.util.Map;
029    import java.util.Set;
030    
031    import cascading.flow.FlowException;
032    import cascading.flow.FlowProcess;
033    import cascading.flow.hadoop.planner.HadoopFlowStepJob;
034    import cascading.flow.hadoop.util.HadoopUtil;
035    import cascading.flow.planner.BaseFlowStep;
036    import cascading.flow.planner.FlowStepJob;
037    import cascading.flow.planner.Scope;
038    import cascading.property.ConfigDef;
039    import cascading.tap.Tap;
040    import cascading.tap.hadoop.io.MultiInputFormat;
041    import cascading.tap.hadoop.util.Hadoop18TapUtil;
042    import cascading.tap.hadoop.util.TempHfs;
043    import cascading.tuple.Fields;
044    import cascading.tuple.Tuple;
045    import cascading.tuple.hadoop.TupleSerialization;
046    import cascading.tuple.hadoop.util.CoGroupingComparator;
047    import cascading.tuple.hadoop.util.CoGroupingPartitioner;
048    import cascading.tuple.hadoop.util.GroupingComparator;
049    import cascading.tuple.hadoop.util.GroupingPartitioner;
050    import cascading.tuple.hadoop.util.GroupingSortingComparator;
051    import cascading.tuple.hadoop.util.IndexTupleCoGroupingComparator;
052    import cascading.tuple.hadoop.util.ReverseGroupingSortingComparator;
053    import cascading.tuple.hadoop.util.ReverseTupleComparator;
054    import cascading.tuple.hadoop.util.TupleComparator;
055    import cascading.tuple.io.IndexTuple;
056    import cascading.tuple.io.TuplePair;
057    import cascading.util.Util;
058    import cascading.util.Version;
059    import org.apache.hadoop.fs.Path;
060    import org.apache.hadoop.mapred.FileOutputFormat;
061    import org.apache.hadoop.mapred.JobConf;
062    
063    import static cascading.flow.hadoop.util.HadoopUtil.serializeBase64;
064    import static cascading.flow.hadoop.util.HadoopUtil.writeStateToDistCache;
065    
066    /**
067     *
068     */
069    public class HadoopFlowStep extends BaseFlowStep<JobConf>
070      {
071      /** Field mapperTraps */
072      private final Map<String, Tap> mapperTraps = new HashMap<String, Tap>();
073      /** Field reducerTraps */
074      private final Map<String, Tap> reducerTraps = new HashMap<String, Tap>();
075    
076      public HadoopFlowStep( String name, int stepNum )
077        {
078        super( name, stepNum );
079        }
080    
081      public JobConf getInitializedConfig( FlowProcess<JobConf> flowProcess, JobConf parentConfig )
082        {
083        JobConf conf = parentConfig == null ? new JobConf() : new JobConf( parentConfig );
084    
085        // disable warning
086        conf.setBoolean( "mapred.used.genericoptionsparser", true );
087    
088        conf.setJobName( getStepDisplayName( conf.getInt( "cascading.step.display.id.truncate", Util.ID_LENGTH ) ) );
089    
090        conf.setOutputKeyClass( Tuple.class );
091        conf.setOutputValueClass( Tuple.class );
092    
093        conf.setMapRunnerClass( FlowMapper.class );
094        conf.setReducerClass( FlowReducer.class );
095    
096        // set for use by the shuffling phase
097        TupleSerialization.setSerializations( conf );
098    
099        initFromSources( flowProcess, conf );
100    
101        initFromSink( flowProcess, conf );
102    
103        initFromTraps( flowProcess, conf );
104    
105        initFromProcessConfigDef( conf );
106    
107        if( getSink().getScheme().getNumSinkParts() != 0 )
108          {
109          // if no reducer, set num map tasks to control parts
110          if( getGroup() != null )
111            conf.setNumReduceTasks( getSink().getScheme().getNumSinkParts() );
112          else
113            conf.setNumMapTasks( getSink().getScheme().getNumSinkParts() );
114          }
115    
116        conf.setOutputKeyComparatorClass( TupleComparator.class );
117    
118        if( getGroup() == null )
119          {
120          conf.setNumReduceTasks( 0 ); // disable reducers
121          }
122        else
123          {
124          // must set map output defaults when performing a reduce
125          conf.setMapOutputKeyClass( Tuple.class );
126          conf.setMapOutputValueClass( Tuple.class );
127    
128          // handles the case the groupby sort should be reversed
129          if( getGroup().isSortReversed() )
130            conf.setOutputKeyComparatorClass( ReverseTupleComparator.class );
131    
132          addComparators( conf, "cascading.group.comparator", getGroup().getKeySelectors() );
133    
134          if( getGroup().isGroupBy() )
135            addComparators( conf, "cascading.sort.comparator", getGroup().getSortingSelectors() );
136    
137          if( !getGroup().isGroupBy() )
138            {
139            conf.setPartitionerClass( CoGroupingPartitioner.class );
140            conf.setMapOutputKeyClass( IndexTuple.class ); // allows groups to be sorted by index
141            conf.setMapOutputValueClass( IndexTuple.class );
142            conf.setOutputKeyComparatorClass( IndexTupleCoGroupingComparator.class ); // sorts by group, then by index
143            conf.setOutputValueGroupingComparator( CoGroupingComparator.class );
144            }
145    
146          if( getGroup().isSorted() )
147            {
148            conf.setPartitionerClass( GroupingPartitioner.class );
149            conf.setMapOutputKeyClass( TuplePair.class );
150    
151            if( getGroup().isSortReversed() )
152              conf.setOutputKeyComparatorClass( ReverseGroupingSortingComparator.class );
153            else
154              conf.setOutputKeyComparatorClass( GroupingSortingComparator.class );
155    
156            // no need to supply a reverse comparator, only equality is checked
157            conf.setOutputValueGroupingComparator( GroupingComparator.class );
158            }
159          }
160    
161        // perform last so init above will pass to tasks
162        String versionString = Version.getRelease();
163    
164        if( versionString != null )
165          conf.set( "cascading.version", versionString );
166    
167        conf.set( CASCADING_FLOW_STEP_ID, getID() );
168        conf.set( "cascading.flow.step.num", Integer.toString( getStepNum() ) );
169    
170        String stepState = pack( this, conf );
171    
172        // hadoop 20.2 doesn't like dist cache when using local mode
173        int maxSize = Short.MAX_VALUE;
174        if( isHadoopLocalMode( conf ) || stepState.length() < maxSize ) // seems safe
175          conf.set( "cascading.flow.step", stepState );
176        else
177          conf.set( "cascading.flow.step.path", writeStateToDistCache( conf, getID(), stepState ) );
178    
179        return conf;
180        }
181    
182      public boolean isHadoopLocalMode( JobConf conf )
183        {
184        return HadoopUtil.isLocal( conf );
185        }
186    
187      private String pack( Object object, JobConf conf )
188        {
189        try
190          {
191          return serializeBase64( object, conf, true );
192          }
193        catch( IOException exception )
194          {
195          throw new FlowException( "unable to pack object: " + object.getClass().getCanonicalName(), exception );
196          }
197        }
198    
199      protected FlowStepJob<JobConf> createFlowStepJob( FlowProcess<JobConf> flowProcess, JobConf parentConfig )
200        {
201        JobConf initializedConfig = getInitializedConfig( flowProcess, parentConfig );
202    
203        setConf( initializedConfig );
204    
205        return new HadoopFlowStepJob( createClientState( flowProcess ), this, initializedConfig );
206        }
207    
208      /**
209       * Method clean removes any temporary files used by this FlowStep instance. It will log any IOExceptions thrown.
210       *
211       * @param config of type JobConf
212       */
213      public void clean( JobConf config )
214        {
215        String stepStatePath = config.get( "cascading.flow.step.path" );
216    
217        if( stepStatePath != null )
218          {
219          try
220            {
221            HadoopUtil.removeStateFromDistCache( config, stepStatePath );
222            }
223          catch( IOException exception )
224            {
225            logWarn( "unable to remove step state file: " + stepStatePath, exception );
226            }
227          }
228    
229        if( tempSink != null )
230          {
231          try
232            {
233            tempSink.deleteResource( config );
234            }
235          catch( Exception exception )
236            {
237            // sink all exceptions, don't fail app
238            logWarn( "unable to remove temporary file: " + tempSink, exception );
239            }
240          }
241    
242        if( getSink() instanceof TempHfs &&
243          ( getFlow().getFlowStats().isSuccessful() || getFlow().getRunID() == null ) )
244          {
245          try
246            {
247            getSink().deleteResource( config );
248            }
249          catch( Exception exception )
250            {
251            // sink all exceptions, don't fail app
252            logWarn( "unable to remove temporary file: " + getSink(), exception );
253            }
254          }
255        else
256          {
257          cleanTapMetaData( config, getSink() );
258          }
259    
260        for( Tap tap : getMapperTraps().values() )
261          cleanTapMetaData( config, tap );
262    
263        for( Tap tap : getReducerTraps().values() )
264          cleanTapMetaData( config, tap );
265    
266        }
267    
268      private void cleanTapMetaData( JobConf jobConf, Tap tap )
269        {
270        try
271          {
272          Hadoop18TapUtil.cleanupTapMetaData( jobConf, tap );
273          }
274        catch( IOException exception )
275          {
276          // ignore exception
277          }
278        }
279    
280      private void addComparators( JobConf conf, String property, Map<String, Fields> map )
281        {
282        Iterator<Fields> fieldsIterator = map.values().iterator();
283    
284        if( !fieldsIterator.hasNext() )
285          return;
286    
287        Fields fields = fieldsIterator.next();
288    
289        if( fields.hasComparators() )
290          {
291          conf.set( property, pack( fields, conf ) );
292          return;
293          }
294    
295        // use resolved fields if there are no comparators.
296        Set<Scope> previousScopes = getPreviousScopes( getGroup() );
297    
298        fields = previousScopes.iterator().next().getOutValuesFields();
299    
300        if( fields.size() != 0 ) // allows fields.UNKNOWN to be used
301          conf.setInt( property + ".size", fields.size() );
302        }
303    
304      private void initFromTraps( FlowProcess<JobConf> flowProcess, JobConf conf, Map<String, Tap> traps )
305        {
306        if( !traps.isEmpty() )
307          {
308          JobConf trapConf = new JobConf( conf );
309    
310          for( Tap tap : traps.values() )
311            tap.sinkConfInit( flowProcess, trapConf );
312          }
313        }
314    
315      protected void initFromSources( FlowProcess<JobConf> flowProcess, JobConf conf )
316        {
317        // handles case where same tap is used on multiple branches
318        // we do not want to init the same tap multiple times
319        Set<Tap> uniqueSources = getUniqueStreamedSources();
320    
321        JobConf[] streamedJobs = new JobConf[ uniqueSources.size() ];
322        int i = 0;
323    
324        for( Tap tap : uniqueSources )
325          {
326          if( tap.getIdentifier() == null )
327            throw new IllegalStateException( "tap may not have null identifier: " + tap.toString() );
328    
329          streamedJobs[ i ] = flowProcess.copyConfig( conf );
330          tap.sourceConfInit( flowProcess, streamedJobs[ i ] );
331          streamedJobs[ i ].set( "cascading.step.source", Tap.id( tap ) );
332          i++;
333          }
334    
335        Set<Tap> accumulatedSources = getAllAccumulatedSources();
336    
337        for( Tap tap : accumulatedSources )
338          {
339          JobConf accumulatedJob = flowProcess.copyConfig( conf );
340          tap.sourceConfInit( flowProcess, accumulatedJob );
341          Map<String, String> map = flowProcess.diffConfigIntoMap( conf, accumulatedJob );
342          conf.set( "cascading.step.accumulated.source.conf." + Tap.id( tap ), pack( map, conf ) );
343          }
344    
345        MultiInputFormat.addInputFormat( conf, streamedJobs ); //must come last
346        }
347    
348      public Tap getTapForID( Set<Tap> taps, String id )
349        {
350        for( Tap tap : taps )
351          {
352          if( Tap.id( tap ).equals( id ) )
353            return tap;
354          }
355    
356        return null;
357        }
358    
359      private void initFromProcessConfigDef( final JobConf conf )
360        {
361        initConfFromProcessConfigDef( getSetterFor( conf ) );
362        }
363    
364      private ConfigDef.Setter getSetterFor( final JobConf conf )
365        {
366        return new ConfigDef.Setter()
367        {
368        @Override
369        public String set( String key, String value )
370          {
371          String oldValue = get( key );
372    
373          conf.set( key, value );
374    
375          return oldValue;
376          }
377    
378        @Override
379        public String update( String key, String value )
380          {
381          String oldValue = get( key );
382    
383          if( oldValue == null )
384            conf.set( key, value );
385          else if( !oldValue.contains( value ) )
386            conf.set( key, oldValue + "," + value );
387    
388          return oldValue;
389          }
390    
391        @Override
392        public String get( String key )
393          {
394          String value = conf.get( key );
395    
396          if( value == null || value.isEmpty() )
397            return null;
398    
399          return value;
400          }
401        };
402        }
403    
404      /**
405       * sources are specific to step, remove all known accumulated sources, if any
406       *
407       * @return
408       */
409      private Set<Tap> getUniqueStreamedSources()
410        {
411        HashSet<Tap> set = new HashSet<Tap>( sources.keySet() );
412    
413        set.removeAll( getAllAccumulatedSources() );
414    
415        return set;
416        }
417    
418      protected void initFromSink( FlowProcess<JobConf> flowProcess, JobConf conf )
419        {
420        // init sink first so tempSink can take precedence
421        if( getSink() != null )
422          getSink().sinkConfInit( flowProcess, conf );
423    
424        if( FileOutputFormat.getOutputPath( conf ) == null )
425          tempSink = new TempHfs( conf, "tmp:/" + new Path( getSink().getIdentifier() ).toUri().getPath(), true );
426    
427        // tempSink exists because sink is writeDirect
428        if( tempSink != null )
429          tempSink.sinkConfInit( flowProcess, conf );
430        }
431    
432      protected void initFromTraps( FlowProcess<JobConf> flowProcess, JobConf conf )
433        {
434        initFromTraps( flowProcess, conf, getMapperTraps() );
435        initFromTraps( flowProcess, conf, getReducerTraps() );
436        }
437    
438      @Override
439      public Set<Tap> getTraps()
440        {
441        Set<Tap> set = new HashSet<Tap>();
442    
443        set.addAll( mapperTraps.values() );
444        set.addAll( reducerTraps.values() );
445    
446        return Collections.unmodifiableSet( set );
447        }
448    
449      @Override
450      public Tap getTrap( String name )
451        {
452        Tap trap = getMapperTrap( name );
453    
454        if( trap == null )
455          trap = getReducerTrap( name );
456    
457        return trap;
458        }
459    
460      public Map<String, Tap> getMapperTraps()
461        {
462        return mapperTraps;
463        }
464    
465      public Map<String, Tap> getReducerTraps()
466        {
467        return reducerTraps;
468        }
469    
470      public Tap getMapperTrap( String name )
471        {
472        return getMapperTraps().get( name );
473        }
474    
475      public Tap getReducerTrap( String name )
476        {
477        return getReducerTraps().get( name );
478        }
479      }