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.Map;
026
027import cascading.flow.BaseFlow;
028import cascading.flow.Flow;
029import cascading.flow.FlowDef;
030import cascading.flow.FlowException;
031import cascading.flow.FlowProcess;
032import cascading.flow.FlowStep;
033import cascading.flow.hadoop.util.HadoopMRUtil;
034import cascading.flow.hadoop.util.HadoopUtil;
035import cascading.flow.planner.BaseFlowStep;
036import cascading.flow.planner.PlatformInfo;
037import cascading.property.PropertyUtil;
038import cascading.tap.hadoop.io.HttpFileSystem;
039import cascading.util.ShutdownUtil;
040import org.apache.hadoop.fs.Path;
041import org.apache.hadoop.mapred.JobConf;
042import riffle.process.ProcessConfiguration;
043
044import static cascading.flow.FlowProps.MAX_CONCURRENT_STEPS;
045import static cascading.flow.FlowProps.PRESERVE_TEMPORARY_FILES;
046
047/**
048 * Class HadoopFlow is the Apache Hadoop specific implementation of a {@link Flow}.
049 * <p>
050 * HadoopFlow must be created through a {@link cascading.flow.FlowConnector} sub-class instance.
051 * <p>
052 * If classpath paths are provided on the {@link FlowDef}, the Hadoop distributed cache mechanism will be used
053 * to augment the remote classpath.
054 * <p>
055 * Any path elements that are relative will be uploaded to HDFS, and the HDFS URI will be used on the JobConf. Note
056 * all paths are added as "files" to the JobConf, not archives, so they aren't needlessly uncompressed cluster side.
057 *
058 * @see cascading.flow.FlowConnector
059 */
060public class HadoopFlow extends BaseFlow<JobConf>
061  {
062  /** Field hdfsShutdown */
063  private static Thread hdfsShutdown = null;
064  /** Field shutdownHook */
065  private static ShutdownUtil.Hook shutdownHook;
066  /** Field jobConf */
067  private transient JobConf jobConf;
068  /** Field preserveTemporaryFiles */
069  private boolean preserveTemporaryFiles = false;
070  /** Field syncPaths */
071  private transient Map<Path, Path> syncPaths;
072
073  protected HadoopFlow()
074    {
075    }
076
077  /**
078   * Returns property preserveTemporaryFiles.
079   *
080   * @param properties of type Map
081   * @return a boolean
082   */
083  static boolean getPreserveTemporaryFiles( Map<Object, Object> properties )
084    {
085    return Boolean.parseBoolean( PropertyUtil.getProperty( properties, PRESERVE_TEMPORARY_FILES, "false" ) );
086    }
087
088  static int getMaxConcurrentSteps( JobConf jobConf )
089    {
090    return jobConf.getInt( MAX_CONCURRENT_STEPS, 0 );
091    }
092
093  protected HadoopFlow( PlatformInfo platformInfo, Map<Object, Object> properties, JobConf jobConf, String name, Map<String, String> flowDescriptor )
094    {
095    super( platformInfo, properties, jobConf, name, flowDescriptor );
096
097    initFromProperties( properties );
098    }
099
100  public HadoopFlow( PlatformInfo platformInfo, Map<Object, Object> properties, JobConf jobConf, FlowDef flowDef )
101    {
102    super( platformInfo, properties, jobConf, flowDef );
103
104    initFromProperties( properties );
105    }
106
107  @Override
108  protected void initFromProperties( Map<Object, Object> properties )
109    {
110    super.initFromProperties( properties );
111    preserveTemporaryFiles = getPreserveTemporaryFiles( properties );
112    }
113
114  protected void initConfig( Map<Object, Object> properties, JobConf parentConfig )
115    {
116    if( properties != null )
117      parentConfig = createConfig( properties, parentConfig );
118
119    if( parentConfig == null ) // this is ok, getJobConf will pass a default parent in
120      return;
121
122    jobConf = HadoopUtil.copyJobConf( parentConfig ); // prevent local values from being shared
123    jobConf.set( "fs.http.impl", HttpFileSystem.class.getName() );
124    jobConf.set( "fs.https.impl", HttpFileSystem.class.getName() );
125
126    syncPaths = HadoopMRUtil.addToClassPath( jobConf, getClassPath() );
127    }
128
129  @Override
130  protected void setConfigProperty( JobConf config, Object key, Object value )
131    {
132    // don't let these objects pass, even though toString is called below.
133    if( value instanceof Class || value instanceof JobConf || value == null )
134      return;
135
136    config.set( key.toString(), value.toString() );
137    }
138
139  @Override
140  protected JobConf newConfig( JobConf defaultConfig )
141    {
142    return defaultConfig == null ? new JobConf() : HadoopUtil.copyJobConf( defaultConfig );
143    }
144
145  @ProcessConfiguration
146  @Override
147  public JobConf getConfig()
148    {
149    if( jobConf == null )
150      initConfig( null, new JobConf() );
151
152    return jobConf;
153    }
154
155  @Override
156  public JobConf getConfigCopy()
157    {
158    return HadoopUtil.copyJobConf( getConfig() );
159    }
160
161  @Override
162  public Map<Object, Object> getConfigAsProperties()
163    {
164    return HadoopUtil.createProperties( getConfig() );
165    }
166
167  /**
168   * Method getProperty returns the value associated with the given key from the underlying properties system.
169   *
170   * @param key of type String
171   * @return String
172   */
173  public String getProperty( String key )
174    {
175    return getConfig().get( key );
176    }
177
178  @Override
179  public FlowProcess<JobConf> getFlowProcess()
180    {
181    return new HadoopFlowProcess( getFlowSession(), getConfig() );
182    }
183
184  /**
185   * Method isPreserveTemporaryFiles returns false if temporary files will be cleaned when this Flow completes.
186   *
187   * @return the preserveTemporaryFiles (type boolean) of this Flow object.
188   */
189  public boolean isPreserveTemporaryFiles()
190    {
191    return preserveTemporaryFiles;
192    }
193
194  @Override
195  protected void internalStart()
196    {
197    try
198      {
199      copyToDistributedCache();
200      deleteSinksIfReplace();
201      deleteTrapsIfReplace();
202      deleteCheckpointsIfReplace();
203      }
204    catch( IOException exception )
205      {
206      throw new FlowException( "unable to delete sinks", exception );
207      }
208
209    registerHadoopShutdownHook();
210    }
211
212  protected void registerHadoopShutdownHook()
213    {
214    registerHadoopShutdownHook( this );
215    }
216
217  protected void copyToDistributedCache()
218    {
219    HadoopUtil.syncPaths( jobConf, syncPaths, true );
220    }
221
222  @Override
223  public boolean stepsAreLocal()
224    {
225    return HadoopUtil.isLocal( getConfig() );
226    }
227
228  private void cleanTemporaryFiles( boolean stop )
229    {
230    if( stop ) // unstable to call fs operations during shutdown
231      return;
232
233    // use step config so cascading.flow.step.path property is properly used
234    for( FlowStep<JobConf> step : getFlowSteps() )
235      ( (BaseFlowStep<JobConf>) step ).clean();
236    }
237
238  private static synchronized void registerHadoopShutdownHook( Flow flow )
239    {
240    if( !flow.isStopJobsOnExit() )
241      return;
242
243    // guaranteed singleton here
244    if( shutdownHook != null )
245      return;
246
247    getHdfsShutdownHook();
248
249    shutdownHook = new ShutdownUtil.Hook()
250      {
251      @Override
252      public Priority priority()
253        {
254        return Priority.LAST; // very last thing to happen
255        }
256
257      @Override
258      public void execute()
259        {
260        callHdfsShutdownHook();
261        }
262      };
263
264    ShutdownUtil.addHook( shutdownHook );
265    }
266
267  private synchronized static void callHdfsShutdownHook()
268    {
269    if( hdfsShutdown != null )
270      hdfsShutdown.start();
271    }
272
273  private synchronized static void getHdfsShutdownHook()
274    {
275    if( hdfsShutdown == null )
276      hdfsShutdown = HadoopUtil.getHDFSShutdownHook();
277    }
278
279  protected void internalClean( boolean stop )
280    {
281    if( !isPreserveTemporaryFiles() )
282      cleanTemporaryFiles( stop );
283    }
284
285  protected void internalShutdown()
286    {
287    }
288
289  protected int getMaxNumParallelSteps()
290    {
291    return stepsAreLocal() ? 1 : getMaxConcurrentSteps( getConfig() );
292    }
293
294  @Override
295  protected long getTotalSliceCPUMilliSeconds()
296    {
297    // this is a hadoop2 MR specific counter/value
298    long counterValue = flowStats.getCounterValue( "org.apache.hadoop.mapreduce.TaskCounter", "CPU_MILLISECONDS" );
299
300    if( counterValue == 0 )
301      return -1;
302
303    return counterValue;
304    }
305  }