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.stats.hadoop;
023
024import java.io.IOException;
025import java.util.HashMap;
026import java.util.Map;
027
028import cascading.flow.FlowNode;
029import cascading.management.state.ClientState;
030import cascading.stats.BaseCachedNodeStats;
031import cascading.stats.CounterCache;
032import cascading.stats.FlowNodeStats;
033import cascading.stats.FlowSliceStats;
034import cascading.util.Util;
035import org.apache.hadoop.conf.Configuration;
036import org.apache.hadoop.mapred.RunningJob;
037import org.apache.hadoop.mapreduce.Job;
038import org.apache.hadoop.mapreduce.TaskCompletionEvent;
039import org.apache.hadoop.mapreduce.TaskID;
040import org.apache.hadoop.mapreduce.TaskReport;
041import org.apache.hadoop.mapreduce.TaskType;
042
043import static cascading.util.Util.formatDurationFromMillis;
044
045/**
046 *
047 */
048public class HadoopNodeStats extends BaseCachedNodeStats<Configuration, FlowNodeStats, Map<String, Map<String, Long>>>
049  {
050  private Map<TaskID, String> sliceIDCache = new HashMap<TaskID, String>( 4999 ); // caching for ids
051
052  private HadoopStepStats parentStepStats;
053  private HadoopSliceStats.Kind kind;
054
055  /**
056   * Constructor CascadingStats creates a new CascadingStats instance.
057   *
058   * @param parentStepStats
059   * @param configuration
060   * @param kind
061   * @param flowNode
062   * @param clientState
063   */
064  protected HadoopNodeStats( final HadoopStepStats parentStepStats, Configuration configuration, HadoopSliceStats.Kind kind, FlowNode flowNode, ClientState clientState )
065    {
066    super( flowNode, clientState );
067    this.parentStepStats = parentStepStats;
068    this.kind = kind;
069
070    this.counterCache = new HadoopNodeCounterCache( this, configuration );
071    }
072
073  @Override
074  public String getKind()
075    {
076    if( kind == null )
077      return null;
078
079    return kind.name();
080    }
081
082  private Status getParentStatus()
083    {
084    return parentStepStats.getStatus();
085    }
086
087  private RunningJob getJobStatusClient()
088    {
089    return parentStepStats.getJobStatusClient();
090    }
091
092  /**
093   * Retrieves the TaskReports via the mapreduce API.
094   *
095   * @param kind The kind of TaskReport to retrieve.
096   * @return An array of TaskReports, but never {@code nul}.
097   * @throws IOException
098   */
099  private TaskReport[] retrieveTaskReports( HadoopSliceStats.Kind kind ) throws IOException, InterruptedException
100    {
101    Job job = HadoopStepStats.getJob( getJobStatusClient() );
102
103    if( job == null )
104      return new TaskReport[ 0 ];
105
106    switch( kind )
107      {
108      case MAPPER:
109        return job.getTaskReports( TaskType.MAP );
110      case REDUCER:
111        return job.getTaskReports( TaskType.REDUCE );
112      case SETUP:
113        return job.getTaskReports( TaskType.JOB_SETUP );
114      case CLEANUP:
115        return job.getTaskReports( TaskType.JOB_CLEANUP );
116      default:
117        return new TaskReport[ 0 ];
118      }
119    }
120
121  @Override
122  protected boolean captureChildDetailInternal()
123    {
124    if( allChildrenFinished )
125      return true;
126
127    Job job = HadoopStepStats.getJob( getJobStatusClient() );
128
129    if( job == null )
130      return false;
131
132    try
133      {
134      TaskReport[] taskReports = retrieveTaskReports( kind );
135
136      if( taskReports.length == 0 )
137        return false;
138
139      addTaskStats( taskReports, false );
140
141      return true;
142      }
143    catch( IOException exception )
144      {
145      logWarn( "unable to retrieve slice stats via task reports", exception );
146      }
147    catch( InterruptedException exception )
148      {
149      logWarn( "retrieving task reports timed out, consider increasing timeout delay in CounterCache via: '{}', message: {}", CounterCache.COUNTER_TIMEOUT_PROPERTY, exception.getMessage() );
150      }
151
152    return false;
153    }
154
155  protected void addTaskStats( TaskReport[] taskReports, boolean skipLast )
156    {
157    logInfo( "retrieved task reports: {}", taskReports.length );
158
159    long lastFetch = System.currentTimeMillis();
160    boolean fetchedAreFinished = true;
161
162    synchronized( sliceStatsMap )
163      {
164      int added = 0;
165      int updated = 0;
166
167      for( int i = 0; i < taskReports.length - ( skipLast ? 1 : 0 ); i++ )
168        {
169        TaskReport taskReport = taskReports[ i ];
170
171        if( taskReport == null )
172          {
173          logWarn( "found empty task report" );
174          continue;
175          }
176
177        String id = getSliceIDFor( taskReport.getTaskID() );
178        HadoopSliceStats sliceStats = (HadoopSliceStats) sliceStatsMap.get( id );
179
180        if( sliceStats != null )
181          {
182          sliceStats.update( getParentStatus(), kind, taskReport, lastFetch );
183          updated++;
184          }
185        else
186          {
187          sliceStats = new HadoopSliceStats( id, getParentStatus(), kind, taskReport, lastFetch );
188          sliceStatsMap.put( id, sliceStats );
189          added++;
190          }
191
192        if( !sliceStats.getStatus().isFinished() )
193          fetchedAreFinished = false;
194        }
195
196      int total = sliceStatsMap.size();
197      String duration = formatDurationFromMillis( System.currentTimeMillis() - lastFetch );
198
199      logInfo( "added {}, updated: {} slices, with duration: {}, total fetched: {}", added, updated, duration, total );
200      }
201
202    allChildrenFinished = taskReports.length != 0 && fetchedAreFinished;
203    }
204
205  protected void addAttempt( TaskCompletionEvent event )
206    {
207    // the event could be a housekeeping task, which we are not tracking
208    String sliceID = sliceIDCache.get( event.getTaskAttemptId().getTaskID() );
209
210    if( sliceID == null )
211      return;
212
213    FlowSliceStats stats;
214
215    synchronized( sliceStatsMap )
216      {
217      stats = sliceStatsMap.get( sliceID );
218      }
219
220    if( stats == null )
221      return;
222
223    ( (HadoopSliceStats) stats ).addAttempt( event );
224    }
225
226  private String getSliceIDFor( TaskID taskID )
227    {
228    // using taskID instance as #toString is quite painful
229    String id = sliceIDCache.get( taskID );
230
231    if( id == null )
232      {
233      id = Util.createUniqueID();
234      sliceIDCache.put( taskID, id );
235      }
236
237    return id;
238    }
239  }