From 9b9d2857cb32db36e63bedeb62232b4a5d85b58e Mon Sep 17 00:00:00 2001 From: Sumant Date: Wed, 13 Dec 2017 12:45:20 -0800 Subject: [PATCH 1/5] EA Tez Working Commit --- app-conf/AggregatorConf.xml | 4 + app-conf/FetcherConf.xml | 7 + app-conf/HeuristicConf.xml | 117 +++++ app-conf/JobTypeConf.xml | 6 + .../drelephant/tez/TezMetricsAggregator.java | 94 ++++ .../tez/TezTaskLevelAggregatedMetrics.java | 139 ++++++ .../tez/data/TezApplicationData.java | 122 +++++ .../drelephant/tez/data/TezCounterData.java | 180 +++++++ .../drelephant/tez/data/TezTaskData.java | 122 +++++ .../drelephant/tez/fetchers/TezFetcher.java | 466 ++++++++++++++++++ .../heuristics/GenericDataSkewHeuristic.java | 199 ++++++++ .../tez/heuristics/GenericGCHeuristic.java | 126 +++++ .../heuristics/GenericMemoryHeuristic.java | 169 +++++++ .../heuristics/MapperDataSkewHeuristic.java | 31 ++ .../tez/heuristics/MapperGCHeuristic.java | 35 ++ .../tez/heuristics/MapperMemoryHeuristic.java | 29 ++ .../tez/heuristics/MapperSpeedHeuristic.java | 149 ++++++ .../tez/heuristics/MapperSpillHeuristic.java | 128 +++++ .../tez/heuristics/MapperTimeHeuristic.java | 166 +++++++ .../heuristics/ReducerDataSkewHeuristic.java | 26 + .../tez/heuristics/ReducerGCHeuristic.java | 25 + .../heuristics/ReducerMemoryHeuristic.java | 31 ++ .../tez/heuristics/ReducerTimeHeuristic.java | 151 ++++++ .../drelephant/util/InfoExtractor.java | 8 + .../TezTaskLevelAggregatedMetricsTest.java | 46 ++ .../tez/fetchers/TezFetcherTest.java | 17 + .../MapperDataSkewHeuristicTest.java | 130 +++++ .../tez/heuristics/MapperGCHeuristicTest.java | 68 +++ .../heuristics/MapperMemoryHeuristicTest.java | 78 +++ .../heuristics/MapperSpeedHeuristicTest.java | 84 ++++ .../heuristics/MapperSpillHeuristicTest.java | 74 +++ .../heuristics/MapperTimeHeuristicTest.java | 95 ++++ .../ReducerDataSkewHeuristicTest.java | 126 +++++ .../heuristics/ReducerGCHeuristicTest.java | 67 +++ .../ReducerMemoryHeuristicTest.java | 77 +++ .../heuristics/ReducerTimeHeuristicTest.java | 81 +++ 36 files changed, 3473 insertions(+) create mode 100644 app/com/linkedin/drelephant/tez/TezMetricsAggregator.java create mode 100644 app/com/linkedin/drelephant/tez/TezTaskLevelAggregatedMetrics.java create mode 100644 app/com/linkedin/drelephant/tez/data/TezApplicationData.java create mode 100644 app/com/linkedin/drelephant/tez/data/TezCounterData.java create mode 100644 app/com/linkedin/drelephant/tez/data/TezTaskData.java create mode 100644 app/com/linkedin/drelephant/tez/fetchers/TezFetcher.java create mode 100644 app/com/linkedin/drelephant/tez/heuristics/GenericDataSkewHeuristic.java create mode 100644 app/com/linkedin/drelephant/tez/heuristics/GenericGCHeuristic.java create mode 100644 app/com/linkedin/drelephant/tez/heuristics/GenericMemoryHeuristic.java create mode 100644 app/com/linkedin/drelephant/tez/heuristics/MapperDataSkewHeuristic.java create mode 100644 app/com/linkedin/drelephant/tez/heuristics/MapperGCHeuristic.java create mode 100644 app/com/linkedin/drelephant/tez/heuristics/MapperMemoryHeuristic.java create mode 100644 app/com/linkedin/drelephant/tez/heuristics/MapperSpeedHeuristic.java create mode 100644 app/com/linkedin/drelephant/tez/heuristics/MapperSpillHeuristic.java create mode 100644 app/com/linkedin/drelephant/tez/heuristics/MapperTimeHeuristic.java create mode 100644 app/com/linkedin/drelephant/tez/heuristics/ReducerDataSkewHeuristic.java create mode 100644 app/com/linkedin/drelephant/tez/heuristics/ReducerGCHeuristic.java create mode 100644 app/com/linkedin/drelephant/tez/heuristics/ReducerMemoryHeuristic.java create mode 100644 app/com/linkedin/drelephant/tez/heuristics/ReducerTimeHeuristic.java create mode 100644 test/com/linkedin/drelephant/tez/TezTaskLevelAggregatedMetricsTest.java create mode 100644 test/com/linkedin/drelephant/tez/fetchers/TezFetcherTest.java create mode 100644 test/com/linkedin/drelephant/tez/heuristics/MapperDataSkewHeuristicTest.java create mode 100644 test/com/linkedin/drelephant/tez/heuristics/MapperGCHeuristicTest.java create mode 100644 test/com/linkedin/drelephant/tez/heuristics/MapperMemoryHeuristicTest.java create mode 100644 test/com/linkedin/drelephant/tez/heuristics/MapperSpeedHeuristicTest.java create mode 100644 test/com/linkedin/drelephant/tez/heuristics/MapperSpillHeuristicTest.java create mode 100644 test/com/linkedin/drelephant/tez/heuristics/MapperTimeHeuristicTest.java create mode 100644 test/com/linkedin/drelephant/tez/heuristics/ReducerDataSkewHeuristicTest.java create mode 100644 test/com/linkedin/drelephant/tez/heuristics/ReducerGCHeuristicTest.java create mode 100644 test/com/linkedin/drelephant/tez/heuristics/ReducerMemoryHeuristicTest.java create mode 100644 test/com/linkedin/drelephant/tez/heuristics/ReducerTimeHeuristicTest.java diff --git a/app-conf/AggregatorConf.xml b/app-conf/AggregatorConf.xml index 23586d587..1536b2d96 100644 --- a/app-conf/AggregatorConf.xml +++ b/app-conf/AggregatorConf.xml @@ -33,6 +33,10 @@ mapreduce com.linkedin.drelephant.mapreduce.MapReduceMetricsAggregator + + tez + com.linkedin.drelephant.tez.TezMetricsAggregator + spark com.linkedin.drelephant.spark.SparkMetricsAggregator diff --git a/app-conf/FetcherConf.xml b/app-conf/FetcherConf.xml index f81aad5e6..5448c9f40 100644 --- a/app-conf/FetcherConf.xml +++ b/app-conf/FetcherConf.xml @@ -29,6 +29,13 @@ --> + + + tez + com.linkedin.drelephant.tez.fetchers.TezFetcher + + + + + tez + Mapper Data Skew + com.linkedin.drelephant.tez.heuristics.MapperDataSkewHeuristic + views.html.help.mapreduce.helpMapperSpill + + + + + tez + Mapper GC + com.linkedin.drelephant.tez.heuristics.MapperGCHeuristic + views.html.help.mapreduce.helpMapperSpill + + + + + tez + Mapper Time + com.linkedin.drelephant.tez.heuristics.MapperTimeHeuristic + views.html.help.mapreduce.helpMapperSpill + + + + + tez + Mapper Speed + com.linkedin.drelephant.tez.heuristics.MapperSpeedHeuristic + views.html.help.mapreduce.helpMapperSpill + + + + + tez + Mapper Memory + com.linkedin.drelephant.tez.heuristics.MapperMemoryHeuristic + views.html.help.mapreduce.helpMapperSpill + + + + + tez + Mapper Spill + com.linkedin.drelephant.tez.heuristics.MapperSpillHeuristic + views.html.help.mapreduce.helpMapperSpill + + + + + tez + Reducer Data Skew + com.linkedin.drelephant.tez.heuristics.ReducerDataSkewHeuristic + views.html.help.mapreduce.helpMapperSpill + + + + + tez + Reducer GC + com.linkedin.drelephant.tez.heuristics.ReducerGCHeuristic + views.html.help.mapreduce.helpMapperSpill + + + + + tez + Reducer Time + com.linkedin.drelephant.tez.heuristics.ReducerTimeHeuristic + views.html.help.mapreduce.helpMapperSpill + + + + + tez + Reducer Memory + com.linkedin.drelephant.tez.heuristics.ReducerMemoryHeuristic + views.html.help.mapreduce.helpMapperSpill + + + diff --git a/app-conf/JobTypeConf.xml b/app-conf/JobTypeConf.xml index 8a4cae3eb..9dc4aa5ba 100644 --- a/app-conf/JobTypeConf.xml +++ b/app-conf/JobTypeConf.xml @@ -42,6 +42,12 @@ mapreduce pig.script + + Tez + tez + hive.mapred.mode + + Hive mapreduce diff --git a/app/com/linkedin/drelephant/tez/TezMetricsAggregator.java b/app/com/linkedin/drelephant/tez/TezMetricsAggregator.java new file mode 100644 index 000000000..2963b1c0f --- /dev/null +++ b/app/com/linkedin/drelephant/tez/TezMetricsAggregator.java @@ -0,0 +1,94 @@ +package com.linkedin.drelephant.tez; + +import com.linkedin.drelephant.analysis.*; +import com.linkedin.drelephant.configurations.aggregator.AggregatorConfigurationData; +import com.linkedin.drelephant.tez.data.TezApplicationData; +import org.apache.commons.io.FileUtils; +import org.apache.log4j.Logger; + +/** + * Aggregates task level metrics to application + */ + +public class TezMetricsAggregator implements HadoopMetricsAggregator { + + private static final Logger logger = Logger.getLogger(TezMetricsAggregator.class); + + private static final String TEZ_CONTAINER_CONFIG = "hive.tez.container.size"; + private static final String MAP_CONTAINER_CONFIG = "mapreduce.map.memory.mb"; + private static final String REDUCER_CONTAINER_CONFIG = "mapreduce.reduce.memory.mb"; + private static final String REDUCER_SLOW_START_CONFIG = "mapreduce.job.reduce.slowstart.completedmaps"; + private static final long CONTAINER_MEMORY_DEFAULT_BYTES = 2048L * FileUtils.ONE_MB; + + private HadoopAggregatedData _hadoopAggregatedData = null; + private TezTaskLevelAggregatedMetrics _mapTasks; + private TezTaskLevelAggregatedMetrics _reduceTasks; + + private AggregatorConfigurationData _aggregatorConfigurationData; + + public TezMetricsAggregator(AggregatorConfigurationData _aggregatorConfigurationData) { + this._aggregatorConfigurationData = _aggregatorConfigurationData; + _hadoopAggregatedData = new HadoopAggregatedData(); + } + + @Override + public void aggregate(HadoopApplicationData hadoopData) { + + TezApplicationData data = (TezApplicationData) hadoopData; + + long mapTaskContainerSize = getMapContainerSize(data); + long reduceTaskContainerSize = getReducerContainerSize(data); + + int reduceTaskSlowStartPercentage = + (int) (Double.parseDouble(data.getConf().getProperty(REDUCER_SLOW_START_CONFIG)) * 100); + + + //overwrite reduceTaskSlowStartPercentage to 100%. TODO: make use of the slow start percent + reduceTaskSlowStartPercentage = 100; + + _mapTasks = new TezTaskLevelAggregatedMetrics(data.getMapTaskData(), mapTaskContainerSize, data.getStartTime()); + + long reduceIdealStartTime = _mapTasks.getNthPercentileFinishTime(reduceTaskSlowStartPercentage); + + // Mappers list is empty + if(reduceIdealStartTime == -1) { + // ideal start time for reducer is infinite since it cannot start + reduceIdealStartTime = Long.MAX_VALUE; + } + + _reduceTasks = new TezTaskLevelAggregatedMetrics(data.getReduceTaskData(), reduceTaskContainerSize, reduceIdealStartTime); + + _hadoopAggregatedData.setResourceUsed(_mapTasks.getResourceUsed() + _reduceTasks.getResourceUsed()); + _hadoopAggregatedData.setTotalDelay(_mapTasks.getDelay() + _reduceTasks.getDelay()); + _hadoopAggregatedData.setResourceWasted(_mapTasks.getResourceWasted() + _reduceTasks.getResourceWasted()); + } + + @Override + public HadoopAggregatedData getResult() { + return _hadoopAggregatedData; + } + + private long getMapContainerSize(HadoopApplicationData data) { + try { + long mapContainerSize = Long.parseLong(data.getConf().getProperty(TEZ_CONTAINER_CONFIG)); + if (mapContainerSize > 0) + return mapContainerSize; + else + return Long.parseLong(data.getConf().getProperty(MAP_CONTAINER_CONFIG)); + } catch ( NumberFormatException ex) { + return CONTAINER_MEMORY_DEFAULT_BYTES; + } + } + + private long getReducerContainerSize(HadoopApplicationData data) { + try { + long reducerContainerSize = Long.parseLong(data.getConf().getProperty(TEZ_CONTAINER_CONFIG)); + if (reducerContainerSize > 0) + return reducerContainerSize; + else + return Long.parseLong(data.getConf().getProperty(REDUCER_CONTAINER_CONFIG)); + } catch ( NumberFormatException ex) { + return CONTAINER_MEMORY_DEFAULT_BYTES; + } + } +} diff --git a/app/com/linkedin/drelephant/tez/TezTaskLevelAggregatedMetrics.java b/app/com/linkedin/drelephant/tez/TezTaskLevelAggregatedMetrics.java new file mode 100644 index 000000000..65a7f27b4 --- /dev/null +++ b/app/com/linkedin/drelephant/tez/TezTaskLevelAggregatedMetrics.java @@ -0,0 +1,139 @@ +package com.linkedin.drelephant.tez; + +import com.linkedin.drelephant.tez.data.TezCounterData; +import com.linkedin.drelephant.tez.data.TezTaskData; +import com.linkedin.drelephant.math.Statistics; +import java.util.ArrayList; +import java.util.List; +import org.apache.commons.io.FileUtils; +import org.apache.log4j.Logger; + +/** + * Aggregation functionality for task level metrics + */ + +public class TezTaskLevelAggregatedMetrics { + + private static final Logger logger = Logger.getLogger(TezTaskLevelAggregatedMetrics.class); + + private long _delay = 0; + private long _resourceWasted = 0; + private long _resourceUsed = 0; + + private List finishTimes = new ArrayList(); + private List durations = new ArrayList(); + + private static final double MEMORY_BUFFER = 1.5; + private static final double CLUSTER_MEMORY_FACTOR = 2.1; + + /** + * Returns the nth percentile finish job + * @param percentile The percentile of finish job to return + * @return The nth percentile finish job + */ + public long getNthPercentileFinishTime(int percentile) + { + if(finishTimes == null || finishTimes.size() == 0 ) { + return -1; + } + return Statistics.percentile(finishTimes, percentile); + } + + /** + * Constructor for TaskLevelAggregatedMetrics + * @param taskData Array containing the task data for mappers and/or reducers + * @param containerSize The container size of the tasks + * @param idealStartTime The ideal start time for the task. For mappers it is the submit time, for + * reducers, it is the time when the number of completed maps become more than + * the slow start time. + */ + public TezTaskLevelAggregatedMetrics(TezTaskData[] taskData, long containerSize, long idealStartTime) { + compute(taskData, containerSize, idealStartTime); + } + + /** + * Returns the overall delay for the tasks. + * @return The delay of the tasks. + */ + public long getDelay() { + return _delay; + } + + /** + * Retruns the resources wasted by all the tasks in MB Seconds + * @return The wasted resources of all the tasks in MB Seconds + */ + public long getResourceWasted() { + return _resourceWasted; + } + + /** + * Returns the resource used by all the tasks in MB Seconds + * @return The total resources used by all tasks in MB Seconds + */ + public long getResourceUsed() { + return _resourceUsed; + } + + /** + * Computes the aggregated metrics -> peakMemory, delay, total task duration, wasted resources and memory usage. + * @param taskDatas + * @param containerSize + * @param idealStartTime + */ + private void compute(TezTaskData[] taskDatas, long containerSize, long idealStartTime) { + + long peakMemoryNeed = 0; + long taskFinishTimeMax = 0; + long taskDurationMax = 0; + + // if there are zero tasks, then nothing to compute. + if(taskDatas == null || taskDatas.length == 0) { + return; + } + + for (TezTaskData taskData: taskDatas) { + if (!taskData.isSampled()) { + continue; + } + long taskMemory = taskData.getCounters().get(TezCounterData.CounterName.PHYSICAL_MEMORY_BYTES)/ FileUtils.ONE_MB; // MB + long taskVM = taskData.getCounters().get(TezCounterData.CounterName.VIRTUAL_MEMORY_BYTES)/ FileUtils.ONE_MB; // MB + long taskDuration = taskData.getFinishTime() - taskData.getStartTime(); // Milliseconds + long taskCost = (containerSize) * (taskDuration / Statistics.SECOND_IN_MS); // MB Seconds + + durations.add(taskDuration); + finishTimes.add(taskData.getFinishTime()); + + //peak Memory usage + long memoryRequiredForVM = (long) (taskVM/CLUSTER_MEMORY_FACTOR); + long biggerMemoryRequirement = memoryRequiredForVM > taskMemory ? memoryRequiredForVM : taskMemory; + peakMemoryNeed = biggerMemoryRequirement > peakMemoryNeed ? biggerMemoryRequirement : peakMemoryNeed; + + if(taskFinishTimeMax < taskData.getFinishTime()) { + taskFinishTimeMax = taskData.getFinishTime(); + } + + if(taskDurationMax < taskDuration) { + taskDurationMax = taskDuration; + } + _resourceUsed += taskCost; + } + + // Compute the delay in starting the task. + _delay = taskFinishTimeMax - (idealStartTime + taskDurationMax); + + // invalid delay + if(_delay < 0) { + _delay = 0; + } + + // wastedResources + long wastedMemory = containerSize - (long) (peakMemoryNeed * MEMORY_BUFFER); + if(wastedMemory > 0) { + for (long duration : durations) { + _resourceWasted += (wastedMemory) * (duration / Statistics.SECOND_IN_MS); // MB Seconds + } + } + } + +} \ No newline at end of file diff --git a/app/com/linkedin/drelephant/tez/data/TezApplicationData.java b/app/com/linkedin/drelephant/tez/data/TezApplicationData.java new file mode 100644 index 000000000..0d4547746 --- /dev/null +++ b/app/com/linkedin/drelephant/tez/data/TezApplicationData.java @@ -0,0 +1,122 @@ +package com.linkedin.drelephant.tez.data; + +import com.linkedin.drelephant.analysis.ApplicationType; +import com.linkedin.drelephant.analysis.HadoopApplicationData; + +import java.util.Properties; + +/** + * Tez Application level data structure which hold all task data + */ +public class TezApplicationData implements HadoopApplicationData { + + private static final ApplicationType APPLICATION_TYPE = new ApplicationType("TEZ"); + + private String _appId = ""; + private Properties _conf; + private boolean _succeeded = true; + private TezTaskData[] _redudeTasks; + private TezTaskData[] _mapTasks; + private TezCounterData _counterHolder; + + private long _submitTime = 0; + private long _startTime = 0; + private long _finishTime = 0; + + public boolean getSucceeded() { + return _succeeded; + } + + @Override + public String getAppId() { + return _appId; + } + + @Override + public Properties getConf() { + return _conf; + } + + @Override + public ApplicationType getApplicationType() { + return APPLICATION_TYPE; + } + + @Override + public boolean isEmpty() { + return _succeeded && getMapTaskData().length == 0 && getReduceTaskData().length == 0; + } + + public TezTaskData[] getReduceTaskData() { + return _redudeTasks; + } + + public TezTaskData[] getMapTaskData() { + return _mapTasks; + } + + public long getSubmitTime() { + return _submitTime; + } + + public long getStartTime() { + return _startTime; + } + + public long getFinishTime() { + return _finishTime; + } + + public TezCounterData getCounters() { + return _counterHolder; + } + + public TezApplicationData setCounters(TezCounterData counterHolder) { + this._counterHolder = counterHolder; + return this; + } + + public TezApplicationData setAppId(String appId) { + this._appId = appId; + return this; + } + + public TezApplicationData setConf(Properties conf) { + this._conf = conf; + return this; + } + + public TezApplicationData setSucceeded(boolean succeeded) { + this._succeeded = succeeded; + return this; + } + + public TezApplicationData setReduceTaskData(TezTaskData[] reduceTasks) { + this._redudeTasks = reduceTasks; + return this; + } + + public TezApplicationData setMapTaskData(TezTaskData[] mapTasks) { + this._mapTasks = mapTasks; + return this; + } + + public TezApplicationData setSubmitTime(long submitTime) { + this._submitTime = submitTime; + return this; + } + + public TezApplicationData setStartTime(long startTime) { + this._startTime = startTime; + return this; + } + + public TezApplicationData setFinishTime(long finishTime) { + this._finishTime = finishTime; + return this; + } + + public String toString(){ + return APPLICATION_TYPE.toString() + " " + _appId; + } +} \ No newline at end of file diff --git a/app/com/linkedin/drelephant/tez/data/TezCounterData.java b/app/com/linkedin/drelephant/tez/data/TezCounterData.java new file mode 100644 index 000000000..6e53e0744 --- /dev/null +++ b/app/com/linkedin/drelephant/tez/data/TezCounterData.java @@ -0,0 +1,180 @@ +package com.linkedin.drelephant.tez.data; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +/** + * Tez Counter Data defining data structure. + */ +public class TezCounterData { + // Map to group counters into DAG, Task and application levels. + private final Map> _pubCounters; + + public String toString() { + return _pubCounters.toString(); + } + + public TezCounterData() { + _pubCounters = new HashMap>(8); + } + + public long get(CounterName counterName) { + for(Map counterGrp : _pubCounters.values()) { + if(counterGrp.containsKey(counterName._name)) { + return counterGrp.get(counterName._name); + } + } + return 0; + } + + public void set(CounterName counterName, long value) { + set(counterName.getGroupName(), counterName.getName(), value); + } + + public void set(String groupName, String counterName, long value) { + Map counterMap = _pubCounters.get(groupName); + if (counterMap == null) { + counterMap = new HashMap(4); + _pubCounters.put(groupName, counterMap); + } + counterMap.put(counterName, value); + } + + public Set getGroupNames() { + Set groupNames = _pubCounters.keySet(); + return Collections.unmodifiableSet(groupNames); + } + + public Map getAllCountersInGroup(String groupName) { + Map counterMap = _pubCounters.get(groupName); + if (counterMap == null) { + counterMap = new HashMap(1); + } + return counterMap; + } + + public static enum GroupName { + FileSystemCounters("org.apache.tez.common.counters.FileSystemCounter"), + TezTask("org.apache.tez.common.counters.TaskCounter"), + TezDag("org.apache.tez.common.counters.DAGCounter"); + + + String _name; + GroupName(String name) { + _name = name; + } + } + + public static enum CounterName { + + NUM_SUCCEEDED_TASKS(GroupName.TezDag, "NUM_SUCCEEDED_TASKS", "NUM_SUCCEEDED_TASKS"), + TOTAL_LAUNCHED_TASKS(GroupName.TezDag, "TOTAL_LAUNCHED_TASKS", "TOTAL_LAUNCHED_TASKS"), + RACK_LOCAL_TASKS(GroupName.TezDag, "RACK_LOCAL_TASKS", "RACK_LOCAL_TASKS"), + AM_CPU_MILLISECONDS(GroupName.TezDag, "AM_CPU_MILLISECONDS", "AM_CPU_MILLISECONDS"), + AM_GC_TIME_MILLIS(GroupName.TezDag, "AM_GC_TIME_MILLIS", "AM_GC_TIME_MILLIS"), + + FILE_BYTES_READ(GroupName.FileSystemCounters, "FILE_BYTES_READ", "FILE_BYTES_READ"), + FILE_BYTES_WRITTEN(GroupName.FileSystemCounters, "FILE_BYTES_WRITTEN", "FILE_BYTES_WRITTEN"), + FILE_READ_OPS(GroupName.FileSystemCounters, "FILE_READ_OPS", "FILE_READ_OPS"), + FILE_LARGE_READ_OPS(GroupName.FileSystemCounters, "FILE_LARGE_READ_OPS", "FILE_LARGE_READ_OPS"), + FILE_WRITE_OPS(GroupName.FileSystemCounters, "FILE_WRITE_OPS", "FILE_WRITE_OPS"), + HDFS_BYTES_READ(GroupName.FileSystemCounters, "HDFS_BYTES_READ", "HDFS_BYTES_READ"), + HDFS_BYTES_WRITTEN(GroupName.FileSystemCounters, "HDFS_BYTES_WRITTEN", "HDFS_BYTES_WRITTEN"), + HDFS_READ_OPS(GroupName.FileSystemCounters, "HDFS_READ_OPS", "HDFS_READ_OPS"), + HDFS_LARGE_READ_OPS(GroupName.FileSystemCounters, "HDFS_LARGE_READ_OPS", "HDFS_LARGE_READ_OPS"), + HDFS_WRITE_OPS(GroupName.FileSystemCounters, "HDFS_WRITE_OPS", "HDFS_WRITE_OPS"), + S3A_BYTES_READ(GroupName.FileSystemCounters, "S3A_BYTES_READ", "S3A_BYTES_READ"), + S3A_BYTES_WRITTEN(GroupName.FileSystemCounters, "S3A_BYTES_WRITTEN", "S3A_BYTES_WRITTEN"), + S3A_READ_OPS(GroupName.FileSystemCounters, "S3A_READ_OPS", "S3A_READ_OPS"), + S3A_LARGE_READ_OPS(GroupName.FileSystemCounters, "S3A_LARGE_READ_OPS", "S3A_LARGE_READ_OPS"), + S3A_WRITE_OPS(GroupName.FileSystemCounters, "S3A_WRITE_OPS", "S3_WRITE_OPS"), + S3N_BYTES_READ(GroupName.FileSystemCounters, "S3N_BYTES_READ", "S3N_BYTES_READ"), + S3N_BYTES_WRITTEN(GroupName.FileSystemCounters, "S3N_BYTES_WRITTEN", "S3N_BYTES_WRITTEN"), + S3N_READ_OPS(GroupName.FileSystemCounters, "S3N_READ_OPS", "S3N_READ_OPS"), + S3N_LARGE_READ_OPS(GroupName.FileSystemCounters, "S3N_LARGE_READ_OPS", "S3N_LARGE_READ_OPS"), + S3N_WRITE_OPS(GroupName.FileSystemCounters, "S3N_WRITE_OPS", "S3N_WRITE_OPS"), + + REDUCE_INPUT_GROUPS(GroupName.TezTask, "REDUCE_INPUT_GROUPS", "REDUCE_INPUT_GROUPS"), + REDUCE_INPUT_RECORDS(GroupName.TezTask, "REDUCE_INPUT_RECORDS", "REDUCE_INPUT_RECORDS"), + COMBINE_INPUT_RECORDS(GroupName.TezTask, "COMBINE_INPUT_RECORDS", "COMBINE_INPUT_RECORDS"), + SPILLED_RECORDS(GroupName.TezTask, "SPILLED_RECORDS", "SPILLED_RECORDS"), + NUM_SHUFFLED_INPUTS(GroupName.TezTask, "NUM_SHUFFLED_INPUTS", "NUM_SHUFFLED_INPUTS"), + NUM_SKIPPED_INPUTS(GroupName.TezTask, "NUM_SKIPPED_INPUTS", "NUM_SKIPPED_INPUTS"), + NUM_FAILED_SHUFFLE_INPUTS(GroupName.TezTask, "NUM_FAILED_SHUFFLE_INPUTS", "NUM_FAILED_SHUFFLE_INPUTS"), + MERGED_MAP_OUTPUTS(GroupName.TezTask, "MERGED_MAP_OUTPUTS", "MERGED_MAP_OUTPUTS"), + GC_TIME_MILLIS(GroupName.TezTask, "GC_TIME_MILLIS", "GC_TIME_MILLIS"), + COMMITTED_HEAP_BYTES(GroupName.TezTask, "COMMITTED_HEAP_BYTES", "COMMITTED_HEAP_BYTES"), + INPUT_RECORDS_PROCESSED(GroupName.TezTask, "INPUT_RECORDS_PROCESSED", "INPUT_RECORDS_PROCESSED"), + OUTPUT_RECORDS(GroupName.TezTask, "OUTPUT_RECORDS", "OUTPUT_RECORDS"), + OUTPUT_BYTES(GroupName.TezTask, "OUTPUT_BYTES", "OUTPUT_BYTES"), + OUTPUT_BYTES_WITH_OVERHEAD(GroupName.TezTask, "OUTPUT_BYTES_WITH_OVERHEAD", "OUTPUT_BYTES_WITH_OVERHEAD"), + OUTPUT_BYTES_PHYSICAL(GroupName.TezTask, "OUTPUT_BYTES_PHYSICAL", "OUTPUT_BYTES_PHYSICAL"), + ADDITIONAL_SPILLS_BYTES_WRITTEN(GroupName.TezTask, "ADDITIONAL_SPILLS_BYTES_WRITTEN", "ADDITIONAL_SPILLS_BYTES_WRITTEN"), + ADDITIONAL_SPILLS_BYTES_READ(GroupName.TezTask, "ADDITIONAL_SPILLS_BYTES_READ", "ADDITIONAL_SPILLS_BYTES_READ"), + ADDITIONAL_SPILL_COUNT(GroupName.TezTask, "ADDITIONAL_SPILL_COUNT", "ADDITIONAL_SPILL_COUNT"), + SHUFFLE_BYTES(GroupName.TezTask, "SHUFFLE_BYTES", "SHUFFLE_BYTES"), + SHUFFLE_BYTES_DECOMPRESSED(GroupName.TezTask, "SHUFFLE_BYTES_DECOMPRESSED", "SHUFFLE_BYTES_DECOMPRESSED"), + SHUFFLE_BYTES_TO_MEM(GroupName.TezTask, "SHUFFLE_BYTES_TO_MEM", "SHUFFLE_BYTES_TO_MEM"), + SHUFFLE_BYTES_TO_DISK(GroupName.TezTask, "SHUFFLE_BYTES_TO_DISK", "SHUFFLE_BYTES_TO_DISK"), + SHUFFLE_BYTES_DISK_DIRECT(GroupName.TezTask, "SHUFFLE_BYTES_DISK_DIRECT", "SHUFFLE_BYTES_DISK_DIRECT"), + NUM_MEM_TO_DISK_MERGES(GroupName.TezTask, "NUM_MEM_TO_DISK_MERGES", "NUM_MEM_TO_DISK_MERGES"), + CPU_MILLISECONDS(GroupName.TezTask,"CPU_MILLISECONDS","CPU_MILLISECONDS"), + PHYSICAL_MEMORY_BYTES(GroupName.TezTask,"PHYSICAL_MEMORY_BYTES","PHYSICAL_MEMORY_BYTES"), + VIRTUAL_MEMORY_BYTES(GroupName.TezTask,"VIRTUAL_MEMORY_BYTES","VIRTUAL_MEMORY_BYTES"), + NUM_DISK_TO_DISK_MERGES(GroupName.TezTask, "NUM_DISK_TO_DISK_MERGES", "NUM_DISK_TO_DISK_MERGES"), + SHUFFLE_PHASE_TIME(GroupName.TezTask, "SHUFFLE_PHASE_TIME", "SHUFFLE_PHASE_TIME"), + MERGE_PHASE_TIME(GroupName.TezTask, "MERGE_PHASE_TIME", "MERGE_PHASE_TIME"), + FIRST_EVENT_RECEIVED(GroupName.TezTask, "FIRST_EVENT_RECEIVED", "FIRST_EVENT_RECEIVED"), + LAST_EVENT_RECEIVED(GroupName.TezTask, "LAST_EVENT_RECEIVED", "LAST_EVENT_RECEIVED"); + + + GroupName _group; + String _name; + String _displayName; + + CounterName(GroupName group, String name, String displayName) { + this._group = group; + this._name = name; + this._displayName = displayName; + } + + static Map _counterDisplayNameMap; + static Map _counterNameMap; + static { + _counterDisplayNameMap = new HashMap(); + _counterNameMap = new HashMap(); + for (CounterName cn : CounterName.values()) { + _counterDisplayNameMap.put(cn._displayName, cn); + _counterNameMap.put(cn._name, cn); + } + } + + public static CounterName getCounterFromName(String name) { + if (_counterNameMap.containsKey(name)) { + return _counterNameMap.get(name); + } + return null; + } + + public static CounterName getCounterFromDisplayName(String displayName) { + if (_counterDisplayNameMap.containsKey(displayName)) { + return _counterDisplayNameMap.get(displayName); + } + return null; + } + + public String getName() { + return _name; + } + + public String getDisplayName() { + return _displayName; + } + + public String getGroupName() { + return _group.name(); + } + } +} \ No newline at end of file diff --git a/app/com/linkedin/drelephant/tez/data/TezTaskData.java b/app/com/linkedin/drelephant/tez/data/TezTaskData.java new file mode 100644 index 000000000..1e170a642 --- /dev/null +++ b/app/com/linkedin/drelephant/tez/data/TezTaskData.java @@ -0,0 +1,122 @@ +package com.linkedin.drelephant.tez.data; + +/** + * Tez Task Level metadata holding data structure + */ + +public class TezTaskData { + private TezCounterData _counterHolder; + private String _taskId; + // The successful attempt id + private String _attemptId; + + private long _totalTimeMs = 0; + private long _shuffleTimeMs = 0; + private long _sortTimeMs = 0; + private long _startTime = 0; + private long _finishTime = 0; + private boolean _isSampled = false; + + //Constructor used only in Test Cases , if provided to partially assign needed time values while ignoring the others + public TezTaskData(TezCounterData counterHolder, long[] time) { + if(time == null || time.length<3){ + time = new long[5]; + } + this._counterHolder = counterHolder; + this._totalTimeMs = time[0]; + this._shuffleTimeMs = time[1]; + this._sortTimeMs = time[2]; + if (time.length > 3) + this._startTime = time[3]; + if (time.length > 4) + this._finishTime = time[4]; + this._isSampled = true; + } + + public TezTaskData(TezCounterData counterHolder) { + this._counterHolder = counterHolder; + } + + public TezTaskData(String taskId, String taskAttemptId) { + this._taskId = taskId; + this._attemptId = taskAttemptId; + } + + public void setCounter(TezCounterData counterHolder) { + this._counterHolder = counterHolder; + this._isSampled = true; + } + + public void setTime(long[] time) { + //No Validation needed here as time array will always be of fixed length 5 from upstream methods. + this._totalTimeMs = time[0]; + this._shuffleTimeMs = time[1]; + this._sortTimeMs = time[2]; + this._startTime = time[3]; + this._finishTime = time[4]; + this._isSampled = true; + } + + //Used only in Test Cases + public void setTimeAndCounter(long[] time, TezCounterData counterHolder){ + if(time == null || time.length<3){ + time = new long[5]; + } + this._totalTimeMs = time[0]; + this._shuffleTimeMs = time[1]; + this._sortTimeMs = time[2]; + if (time.length > 3) + this._startTime = time[3]; + if (time.length > 4) + this._finishTime = time[4]; + this._isSampled = true; + this._counterHolder = counterHolder; + + + } + + public TezCounterData getCounters() { + return _counterHolder; + } + + public long getTotalRunTimeMs() { + return _totalTimeMs; + } + + public long getCodeExecutionTimeMs() { + return _totalTimeMs - _shuffleTimeMs - _sortTimeMs; + } + + public long getShuffleTimeMs() { + return _shuffleTimeMs; + } + + public long getSortTimeMs() { + return _sortTimeMs; + } + + public boolean isSampled() { + return _isSampled; + } + + public String getTaskId() { + return _taskId; + } + + public String getAttemptId() { + return _attemptId; + } + + public long getStartTime() { + return _startTime; + } + + public long getFinishTime() { + return _finishTime; + } + + public void setTotalTimeMs(long totalTimeMs, boolean isSampled) { + this._totalTimeMs = totalTimeMs; + this._isSampled = isSampled; + } +} diff --git a/app/com/linkedin/drelephant/tez/fetchers/TezFetcher.java b/app/com/linkedin/drelephant/tez/fetchers/TezFetcher.java new file mode 100644 index 000000000..db38586fe --- /dev/null +++ b/app/com/linkedin/drelephant/tez/fetchers/TezFetcher.java @@ -0,0 +1,466 @@ +package com.linkedin.drelephant.tez.fetchers; + +import com.linkedin.drelephant.analysis.AnalyticJob; +import com.linkedin.drelephant.analysis.ElephantFetcher; +import com.linkedin.drelephant.configurations.fetcher.FetcherConfigurationData; +import com.linkedin.drelephant.math.Statistics; +import com.linkedin.drelephant.tez.data.TezApplicationData; +import com.linkedin.drelephant.tez.data.TezCounterData; +import com.linkedin.drelephant.tez.data.TezTaskData; +import org.apache.log4j.Logger; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.authentication.client.AuthenticatedURL; +import org.apache.hadoop.security.authentication.client.AuthenticationException; +import org.codehaus.jackson.JsonNode; +import org.codehaus.jackson.map.ObjectMapper; + +import java.io.IOException; +import java.net.HttpURLConnection; +import java.net.MalformedURLException; +import java.net.URL; +import java.net.URLConnection; +import java.util.*; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Task level data mining for Tez Tasks from timeline server API + */ + +public class TezFetcher implements ElephantFetcher { + + private static final Logger logger = Logger.getLogger(TezFetcher.class); + private static final int MAX_SAMPLE_SIZE = 1000; + + private URLFactory _urlFactory; + private JSONFactory _jsonFactory; + private String _timelineWebAddr; + + private FetcherConfigurationData _fetcherConfigurationData; + + public TezFetcher(FetcherConfigurationData fetcherConfData) throws IOException { + this._fetcherConfigurationData = fetcherConfData; + final String applicationHistoryAddr = new Configuration().get("yarn.timeline-service.webapp.address"); + + //Connection validity checked using method verifyURL(_timelineWebAddr) inside URLFactory constructor; + _urlFactory = new URLFactory(applicationHistoryAddr); + logger.info("Connection success."); + + _jsonFactory = new JSONFactory(); + _timelineWebAddr = "http://" + _timelineWebAddr + "/ws/v1/timeline/"; + + } + + public TezApplicationData fetchData(AnalyticJob analyticJob) throws IOException, AuthenticationException { + + int maxSize = 0; + String appId = analyticJob.getAppId(); + TezApplicationData jobData = new TezApplicationData(); + jobData.setAppId(appId); + Properties jobConf = _jsonFactory.getProperties(_urlFactory.getApplicationURL(appId)); + jobData.setConf(jobConf); + URL dagIdsUrl = _urlFactory.getDagURLByTezApplicationId(appId); + + List dagIdsByApplicationId = _jsonFactory.getDagIdsByApplicationId(dagIdsUrl); + + List mapperListAggregate = new ArrayList(); + List reducerListAggregate = new ArrayList(); + + //Iterate over dagIds and choose the dagId with the highest no. of tasks/highest impact as settings changes can be made only at DAG level. + for(String dagId : dagIdsByApplicationId){ + try { + //set job task independent properties + + URL dagUrl = _urlFactory.getDagURL(dagId); + String state = _jsonFactory.getState(dagUrl); + + jobData.setStartTime(_jsonFactory.getDagStartTime(dagUrl)); + jobData.setFinishTime(_jsonFactory.getDagEndTime(dagUrl)); + + if (state.equals("SUCCEEDED")) { + jobData.setSucceeded(true); + + List mapperList = new ArrayList(); + List reducerList = new ArrayList(); + + // Fetch task data + URL vertexListUrl = _urlFactory.getVertexListURL(dagId); + _jsonFactory.getTaskDataAll(vertexListUrl, dagId, mapperList, reducerList); + + if(mapperList.size() + reducerList.size() > maxSize){ + mapperListAggregate = mapperList; + reducerListAggregate = reducerList; + maxSize = mapperList.size() + reducerList.size(); + } + + } + } + finally { + ThreadContextMR2.updateAuthToken(); + } + } + + if(mapperListAggregate.isEmpty() && reducerListAggregate.isEmpty()){ + jobData.setSucceeded(false); + } + + TezTaskData[] mapperData = mapperListAggregate.toArray(new TezTaskData[mapperListAggregate.size()]); + TezTaskData[] reducerData = reducerListAggregate.toArray(new TezTaskData[reducerListAggregate.size()]); + + TezCounterData dagCounter = _jsonFactory.getDagCounter(_urlFactory.getDagURL(_jsonFactory.getDagIdsByApplicationId(dagIdsUrl).get(0))); + + jobData.setCounters(dagCounter).setMapTaskData(mapperData).setReduceTaskData(reducerData); + + return jobData; + } + + private URL getTaskListByVertexURL(String dagId, String vertexId) throws MalformedURLException { + return _urlFactory.getTaskListByVertexURL(dagId, vertexId); + } + + private URL getTaskURL(String taskId) throws MalformedURLException { + return _urlFactory.getTasksURL(taskId); + } + + private URL getTaskAttemptURL(String dagId, String taskId, String attemptId) throws MalformedURLException { + return _urlFactory.getTaskAttemptURL(dagId, taskId, attemptId); + } + + private class URLFactory { + + private String _timelineWebAddr; + + private URLFactory(String hserverAddr) throws IOException { + _timelineWebAddr = "http://" + hserverAddr + "/ws/v1/timeline"; + verifyURL(_timelineWebAddr); + } + + private void verifyURL(String url) throws IOException { + final URLConnection connection = new URL(url).openConnection(); + // Check service availability + connection.connect(); + return; + } + + private URL getDagURLByTezApplicationId(String applicationId) throws MalformedURLException { + return new URL(_timelineWebAddr + "/TEZ_DAG_ID?primaryFilter=applicationId:" + applicationId); + } + + private URL getApplicationURL(String applicationId) throws MalformedURLException { + return new URL(_timelineWebAddr + "/TEZ_APPLICATION/tez_" + applicationId); + } + + private URL getDagURL(String dagId) throws MalformedURLException { + return new URL(_timelineWebAddr + "/TEZ_DAG_ID/" + dagId); + } + + private URL getVertexListURL(String dagId) throws MalformedURLException { + return new URL(_timelineWebAddr + "/TEZ_VERTEX_ID?primaryFilter=TEZ_DAG_ID:" + dagId); + } + + private URL getTaskListByVertexURL(String dagId, String vertexId) throws MalformedURLException { + return new URL(_timelineWebAddr + "/TEZ_TASK_ID?primaryFilter=TEZ_DAG_ID:" + dagId + + "&secondaryFilter=TEZ_VERTEX_ID:" + vertexId); + } + + private URL getTasksURL(String taskId) throws MalformedURLException { + return new URL(_timelineWebAddr + "/TEZ_TASK_ID/" + taskId); + } + + private URL getTaskAllAttemptsURL(String dagId, String taskId) throws MalformedURLException { + return new URL(_timelineWebAddr + "/TEZ_TASK_ATTEMPT_ID?primaryFilter=TEZ_DAG_ID:" + dagId + + "&secondaryFilter=TEZ_TASK_ID:" + taskId); + } + + private URL getTaskAttemptURL(String dagId, String taskId, String attemptId) throws MalformedURLException { + return new URL(_timelineWebAddr + "/TEZ_TASK_ATTEMPT_ID/" + attemptId); + } + + } + + /** + * JSONFactory class provides functionality to parse mined job data from timeline server. + */ + + private class JSONFactory { + + private String getState(URL url) throws IOException, AuthenticationException { + JsonNode rootNode = ThreadContextMR2.readJsonNode(url); + return rootNode.path("otherinfo").path("status").getTextValue(); + } + + private Properties getProperties(URL url) throws IOException, AuthenticationException { + Properties jobConf = new Properties(); + JsonNode rootNode = ThreadContextMR2.readJsonNode(url); + JsonNode configs = rootNode.path("otherinfo").path("config"); + Iterator keys = configs.getFieldNames(); + String key = ""; + String value = ""; + while (keys.hasNext()) { + key = keys.next(); + value = configs.get(key).getTextValue(); + jobConf.put(key, value); + } + return jobConf; + } + + private List getDagIdsByApplicationId(URL dagIdsUrl) throws IOException, AuthenticationException { + List dagIds = new ArrayList(); + JsonNode nodes = ThreadContextMR2.readJsonNode(dagIdsUrl).get("entities"); + + for (JsonNode node : nodes) { + String dagId = node.get("entity").getTextValue(); + dagIds.add(dagId); + } + + return dagIds; + } + + private TezCounterData getDagCounter(URL url) throws IOException, AuthenticationException { + TezCounterData holder = new TezCounterData(); + JsonNode rootNode = ThreadContextMR2.readJsonNode(url); + JsonNode groups = rootNode.path("otherinfo").path("counters").path("counterGroups"); + + for (JsonNode group : groups) { + for (JsonNode counter : group.path("counters")) { + String name = counter.get("counterName").getTextValue(); + String groupName = group.get("counterGroupName").getTextValue(); + Long value = counter.get("counterValue").getLongValue(); + holder.set(groupName, name, value); + } + } + + return holder; + } + + private long getDagStartTime(URL url) throws IOException, AuthenticationException { + JsonNode rootNode = ThreadContextMR2.readJsonNode(url); + long startTime = rootNode.path("otherinfo").get("startTime").getLongValue(); + return startTime; + } + + private long getDagEndTime(URL url) throws IOException, AuthenticationException { + JsonNode rootNode = ThreadContextMR2.readJsonNode(url); + long endTime = rootNode.path("otherinfo").get("endTime").getLongValue(); + return endTime; + } + + private void getTaskDataAll(URL vertexListUrl, String dagId, List mapperList, + List reducerList) throws IOException, AuthenticationException { + + JsonNode rootVertexNode = ThreadContextMR2.readJsonNode(vertexListUrl); + JsonNode vertices = rootVertexNode.path("entities"); + boolean isMapVertex = false; + + for (JsonNode vertex : vertices) { + String vertexId = vertex.get("entity").getTextValue(); + String vertexClass = vertex.path("otherinfo").path("processorClassName").getTextValue(); + + if (vertexClass.equals("org.apache.hadoop.hive.ql.exec.tez.MapTezProcessor")) + isMapVertex = true; + else if (vertexClass.equals("org.apache.hadoop.hive.ql.exec.tez.ReduceTezProcessor")) + isMapVertex = false; + + URL tasksByVertexURL = getTaskListByVertexURL(dagId, vertexId); + if(isMapVertex) + getTaskDataByVertexId(tasksByVertexURL, dagId, vertexId, mapperList, true); + else + getTaskDataByVertexId(tasksByVertexURL, dagId, vertexId, reducerList, false); + } + } + + private void getTaskDataByVertexId(URL url, String dagId, String vertexId, List taskList, + boolean isMapVertex) throws IOException, AuthenticationException { + + JsonNode rootNode = ThreadContextMR2.readJsonNode(url); + JsonNode tasks = rootNode.path("entities"); + for (JsonNode task : tasks) { + String state = task.path("otherinfo").path("status").getTextValue(); + String taskId = task.get("entity").getValueAsText(); + String attemptId = task.path("otherinfo").path("successfulAttemptId").getTextValue(); + if (state.equals("SUCCEEDED")) { + attemptId = task.path("otherinfo").path("successfulAttemptId").getTextValue(); + } + else{ + JsonNode firstAttempt = getTaskFirstFailedAttempt(_urlFactory.getTaskAllAttemptsURL(dagId,taskId)); + if(firstAttempt != null){ + attemptId = firstAttempt.get("entity").getTextValue(); + } + } + + taskList.add(new TezTaskData(taskId, attemptId)); + } + + getTaskData(dagId, taskList, isMapVertex); + + } + + private JsonNode getTaskFirstFailedAttempt(URL taskAllAttemptsUrl) throws IOException, AuthenticationException { + JsonNode rootNode = ThreadContextMR2.readJsonNode(taskAllAttemptsUrl); + long firstAttemptFinishTime = Long.MAX_VALUE; + JsonNode firstAttempt = null; + JsonNode taskAttempts = rootNode.path("entities"); + for (JsonNode taskAttempt : taskAttempts) { + String state = taskAttempt.path("otherinfo").path("counters").path("status").getTextValue(); + if (state.equals("SUCCEEDED")) { + continue; + } + long finishTime = taskAttempt.path("otherinfo").path("counters").path("endTime").getLongValue(); + if( finishTime < firstAttemptFinishTime) { + firstAttempt = taskAttempt; + firstAttemptFinishTime = finishTime; + } + } + return firstAttempt; + } + + + + private void getTaskData(String dagId, List taskList, boolean isMapTask) + throws IOException, AuthenticationException { + + if (taskList.size() > MAX_SAMPLE_SIZE) { + logger.info(dagId + " needs sampling."); + Collections.shuffle(taskList); + } + + int sampleSize = Math.min(taskList.size(), MAX_SAMPLE_SIZE); + for (int i=0; i _LOCAL_THREAD_ID = new ThreadLocal() { + @Override + public Integer initialValue() { + return THREAD_ID.getAndIncrement(); + } + }; + + private static final ThreadLocal _LOCAL_LAST_UPDATED = new ThreadLocal(); + private static final ThreadLocal _LOCAL_UPDATE_INTERVAL = new ThreadLocal(); + + private static final ThreadLocal _LOCAL_DIAGNOSTIC_PATTERN = new ThreadLocal() { + @Override + public Pattern initialValue() { + // Example: "Task task_1443068695259_9143_m_000475 failed 1 times" + return Pattern.compile( + "Task[\\s\\u00A0]+(.*)[\\s\\u00A0]+failed[\\s\\u00A0]+([0-9])[\\s\\u00A0]+times[\\s\\u00A0]+"); + } + }; + + private static final ThreadLocal _LOCAL_AUTH_TOKEN = + new ThreadLocal() { + @Override + public AuthenticatedURL.Token initialValue() { + _LOCAL_LAST_UPDATED.set(System.currentTimeMillis()); + // Random an interval for each executor to avoid update token at the same time + _LOCAL_UPDATE_INTERVAL.set(Statistics.MINUTE_IN_MS * 30 + new Random().nextLong() + % (3 * Statistics.MINUTE_IN_MS)); + logger.info("Executor " + _LOCAL_THREAD_ID.get() + " update interval " + _LOCAL_UPDATE_INTERVAL.get() * 1.0 + / Statistics.MINUTE_IN_MS); + return new AuthenticatedURL.Token(); + } + }; + + private static final ThreadLocal _LOCAL_AUTH_URL = new ThreadLocal() { + @Override + public AuthenticatedURL initialValue() { + return new AuthenticatedURL(); + } + }; + + private static final ThreadLocal _LOCAL_MAPPER = new ThreadLocal() { + @Override + public ObjectMapper initialValue() { + return new ObjectMapper(); + } + }; + + private ThreadContextMR2() { + // Empty on purpose + } + + public static Matcher getDiagnosticMatcher(String diagnosticInfo) { + return _LOCAL_DIAGNOSTIC_PATTERN.get().matcher(diagnosticInfo); + } + + public static JsonNode readJsonNode(URL url) throws IOException, AuthenticationException { + HttpURLConnection conn = _LOCAL_AUTH_URL.get().openConnection(url, _LOCAL_AUTH_TOKEN.get()); + return _LOCAL_MAPPER.get().readTree(conn.getInputStream()); + } + + public static void updateAuthToken() { + long curTime = System.currentTimeMillis(); + if (curTime - _LOCAL_LAST_UPDATED.get() > _LOCAL_UPDATE_INTERVAL.get()) { + logger.info("Executor " + _LOCAL_THREAD_ID.get() + " updates its AuthenticatedToken."); + _LOCAL_AUTH_TOKEN.set(new AuthenticatedURL.Token()); + _LOCAL_AUTH_URL.set(new AuthenticatedURL()); + _LOCAL_LAST_UPDATED.set(curTime); + } + } +} \ No newline at end of file diff --git a/app/com/linkedin/drelephant/tez/heuristics/GenericDataSkewHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/GenericDataSkewHeuristic.java new file mode 100644 index 000000000..61a115296 --- /dev/null +++ b/app/com/linkedin/drelephant/tez/heuristics/GenericDataSkewHeuristic.java @@ -0,0 +1,199 @@ +package com.linkedin.drelephant.tez.heuristics; + + +import com.google.common.primitives.Longs; +import com.linkedin.drelephant.analysis.HDFSContext; +import com.linkedin.drelephant.analysis.Heuristic; +import com.linkedin.drelephant.analysis.HeuristicResult; +import com.linkedin.drelephant.analysis.Severity; +import com.linkedin.drelephant.math.Statistics; + +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; +import com.linkedin.drelephant.tez.data.TezApplicationData; +import com.linkedin.drelephant.tez.data.TezCounterData; +import com.linkedin.drelephant.tez.data.TezTaskData; +import com.linkedin.drelephant.util.Utils; + +import java.util.Arrays; +import java.util.ArrayList; +import java.util.Map; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang.time.DurationFormatUtils; +import org.apache.log4j.Logger; + + +/** + * This Heuristic analyses the skewness in the task input data + */ +public abstract class GenericDataSkewHeuristic implements Heuristic { + private static final Logger logger = Logger.getLogger(GenericDataSkewHeuristic.class); + + // Severity Parameters + private static final String NUM_TASKS_SEVERITY = "num_tasks_severity"; + private static final String DEVIATION_SEVERITY = "deviation_severity"; + private static final String FILES_SEVERITY = "files_severity"; + + // Default value of parameters + private double[] numTasksLimits = {10, 50, 100, 200}; // Number of map or reduce tasks + private double[] deviationLimits = {2, 4, 8, 16}; // Deviation in i/p bytes btw 2 groups + private double[] filesLimits = {1d / 8, 1d / 4, 1d / 2, 1d}; // Fraction of HDFS Block Size + + private HeuristicConfigurationData _heuristicConfData; + private List _counterNames; + + private void loadParameters() { + Map paramMap = _heuristicConfData.getParamMap(); + String heuristicName = _heuristicConfData.getHeuristicName(); + + double[] confNumTasksThreshold = Utils.getParam(paramMap.get(NUM_TASKS_SEVERITY), numTasksLimits.length); + if (confNumTasksThreshold != null) { + numTasksLimits = confNumTasksThreshold; + } + logger.info(heuristicName + " will use " + NUM_TASKS_SEVERITY + " with the following threshold settings: " + + Arrays.toString(numTasksLimits)); + + double[] confDeviationThreshold = Utils.getParam(paramMap.get(DEVIATION_SEVERITY), deviationLimits.length); + if (confDeviationThreshold != null) { + deviationLimits = confDeviationThreshold; + } + logger.info(heuristicName + " will use " + DEVIATION_SEVERITY + " with the following threshold settings: " + + Arrays.toString(deviationLimits)); + + double[] confFilesThreshold = Utils.getParam(paramMap.get(FILES_SEVERITY), filesLimits.length); + if (confFilesThreshold != null) { + filesLimits = confFilesThreshold; + } + logger.info( + heuristicName + " will use " + FILES_SEVERITY + " with the following threshold settings: " + Arrays.toString( + filesLimits)); + for (int i = 0; i < filesLimits.length; i++) { + filesLimits[i] = filesLimits[i] * HDFSContext.HDFS_BLOCK_SIZE; + } + } + + public GenericDataSkewHeuristic(List counterNames, HeuristicConfigurationData heuristicConfData) { + this._counterNames = counterNames; + this._heuristicConfData = heuristicConfData; + loadParameters(); + } + + protected abstract TezTaskData[] getTasks(TezApplicationData data); + + public HeuristicConfigurationData getHeuristicConfData() { + return _heuristicConfData; + } + + public HeuristicResult apply(TezApplicationData data) { + + if (!data.getSucceeded()) { + return null; + } + + TezTaskData[] tasks = getTasks(data); + + //Gathering data for checking time skew + List timeTaken = new ArrayList(); + + for(int i = 0; i < tasks.length; i++) { + if (tasks[i].isSampled()) { + timeTaken.add(tasks[i].getTotalRunTimeMs()); + } + } + + long[][] groupsTime = Statistics.findTwoGroups(Longs.toArray(timeTaken)); + + long timeAvg1 = Statistics.average(groupsTime[0]); + long timeAvg2 = Statistics.average(groupsTime[1]); + + //seconds are used for calculating deviation as they provide a better idea than millisecond. + long timeAvgSec1 = TimeUnit.MILLISECONDS.toSeconds(timeAvg1); + long timeAvgSec2 = TimeUnit.MILLISECONDS.toSeconds(timeAvg2); + + long minTime = Math.min(timeAvgSec1, timeAvgSec2); + long diffTime = Math.abs(timeAvgSec1 - timeAvgSec2); + + //using the same deviation limits for time skew as for data skew. It can be changed in the fututre. + Severity severityTime = getDeviationSeverity(minTime, diffTime); + + //This reduces severity if number of tasks is insignificant + severityTime = Severity.min(severityTime, + Severity.getSeverityAscending(groupsTime[0].length, numTasksLimits[0], numTasksLimits[1], numTasksLimits[2], + numTasksLimits[3])); + + //Gather data + List inputSizes = new ArrayList(); + + for (int i = 0; i < tasks.length; i++) { + if (tasks[i].isSampled()) { + + long inputByte = 0; + for (TezCounterData.CounterName counterName : _counterNames) { + inputByte += tasks[i].getCounters().get(counterName); + } + + inputSizes.add(inputByte); + } + } + + long[][] groups = Statistics.findTwoGroups(Longs.toArray(inputSizes)); + + long avg1 = Statistics.average(groups[0]); + long avg2 = Statistics.average(groups[1]); + + long min = Math.min(avg1, avg2); + long diff = Math.abs(avg2 - avg1); + + Severity severityData = getDeviationSeverity(min, diff); + + //This reduces severity if the largest file sizes are insignificant + severityData = Severity.min(severityData, getFilesSeverity(avg2)); + + //This reduces severity if number of tasks is insignificant + severityData = Severity.min(severityData, + Severity.getSeverityAscending(groups[0].length, numTasksLimits[0], numTasksLimits[1], numTasksLimits[2], + numTasksLimits[3])); + + Severity severity = Severity.max(severityData, severityTime); + + HeuristicResult result = + new HeuristicResult(_heuristicConfData.getClassName(), _heuristicConfData.getHeuristicName(), severity, + Utils.getHeuristicScore(severityData, tasks.length)); + + result.addResultDetail("Data skew (Number of tasks)", Integer.toString(tasks.length)); + result.addResultDetail("Data skew (Group A)", + groups[0].length + " tasks @ " + FileUtils.byteCountToDisplaySize(avg1) + " avg"); + result.addResultDetail("Data skew (Group B)", + groups[1].length + " tasks @ " + FileUtils.byteCountToDisplaySize(avg2) + " avg"); + + result.addResultDetail("Time skew (Number of tasks)", Integer.toString(tasks.length)); + result.addResultDetail("Time skew (Group A)", + groupsTime[0].length + " tasks @ " + convertTimeMs(timeAvg1) + " avg"); + result.addResultDetail("Time skew (Group B)", + groupsTime[1].length + " tasks @ " + convertTimeMs(timeAvg2) + " avg"); + + return result; + } + + private String convertTimeMs(long timeMs) { + if (timeMs < 1000) { + return Long.toString(timeMs) + " msec"; + } + return DurationFormatUtils.formatDuration(timeMs, "HH:mm:ss") + " HH:MM:SS"; + } + + private Severity getDeviationSeverity(long averageMin, long averageDiff) { + if (averageMin <= 0) { + averageMin = 1; + } + long value = averageDiff / averageMin; + return Severity.getSeverityAscending(value, deviationLimits[0], deviationLimits[1], deviationLimits[2], + deviationLimits[3]); + } + + private Severity getFilesSeverity(long value) { + return Severity.getSeverityAscending(value, filesLimits[0], filesLimits[1], filesLimits[2], filesLimits[3]); + } +} \ No newline at end of file diff --git a/app/com/linkedin/drelephant/tez/heuristics/GenericGCHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/GenericGCHeuristic.java new file mode 100644 index 000000000..f146a0386 --- /dev/null +++ b/app/com/linkedin/drelephant/tez/heuristics/GenericGCHeuristic.java @@ -0,0 +1,126 @@ +package com.linkedin.drelephant.tez.heuristics; + +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; +import com.linkedin.drelephant.tez.data.TezApplicationData; +import com.linkedin.drelephant.tez.data.TezCounterData; +import com.linkedin.drelephant.tez.data.TezTaskData; +import com.linkedin.drelephant.util.Utils; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import com.linkedin.drelephant.analysis.Heuristic; +import com.linkedin.drelephant.analysis.HeuristicResult; +import com.linkedin.drelephant.analysis.Severity; +import com.linkedin.drelephant.math.Statistics; +import java.util.Map; +import org.apache.log4j.Logger; + + +/** + * Analyses garbage collection efficiency + */ +public abstract class GenericGCHeuristic implements Heuristic { + private static final Logger logger = Logger.getLogger(GenericGCHeuristic.class); + + // Severity Parameters + private static final String GC_RATIO_SEVERITY = "gc_ratio_severity"; + private static final String RUNTIME_SEVERITY = "runtime_severity_in_min"; + + // Default value of parameters + private double[] gcRatioLimits = {0.01d, 0.02d, 0.03d, 0.04d}; // Garbage Collection Time / CPU Time + private double[] runtimeLimits = {5, 10, 12, 15}; // Task Runtime in milli sec + + private HeuristicConfigurationData _heuristicConfData; + + private void loadParameters() { + Map paramMap = _heuristicConfData.getParamMap(); + String heuristicName = _heuristicConfData.getHeuristicName(); + + double[] confGcRatioThreshold = Utils.getParam(paramMap.get(GC_RATIO_SEVERITY), gcRatioLimits.length); + if (confGcRatioThreshold != null) { + gcRatioLimits = confGcRatioThreshold; + } + logger.info(heuristicName + " will use " + GC_RATIO_SEVERITY + " with the following threshold settings: " + + Arrays.toString(gcRatioLimits)); + + double[] confRuntimeThreshold = Utils.getParam(paramMap.get(RUNTIME_SEVERITY), runtimeLimits.length); + if (confRuntimeThreshold != null) { + runtimeLimits = confRuntimeThreshold; + } + logger.info(heuristicName + " will use " + RUNTIME_SEVERITY + " with the following threshold settings: " + + Arrays.toString(runtimeLimits)); + for (int i = 0; i < runtimeLimits.length; i++) { + runtimeLimits[i] = runtimeLimits[i] * Statistics.MINUTE_IN_MS; + } + } + + public GenericGCHeuristic(HeuristicConfigurationData heuristicConfData) { + this._heuristicConfData = heuristicConfData; + loadParameters(); + } + + protected abstract TezTaskData[] getTasks(TezApplicationData data); + + public HeuristicConfigurationData getHeuristicConfData() { + return _heuristicConfData; + } + + public HeuristicResult apply(TezApplicationData data) { + + if(!data.getSucceeded()) { + return null; + } + + TezTaskData[] tasks = getTasks(data) ; + List gcMs = new ArrayList(); + List cpuMs = new ArrayList(); + List runtimesMs = new ArrayList(); + + for (TezTaskData task : tasks) { + if (task.isSampled()) { + runtimesMs.add(task.getTotalRunTimeMs()); + gcMs.add(task.getCounters().get(TezCounterData.CounterName.GC_TIME_MILLIS)); + cpuMs.add(task.getCounters().get(TezCounterData.CounterName.CPU_MILLISECONDS)); + } + } + + long avgRuntimeMs = Statistics.average(runtimesMs); + long avgCpuMs = Statistics.average(cpuMs); + long avgGcMs = Statistics.average(gcMs); + double ratio = avgCpuMs != 0 ? avgGcMs*(1.0)/avgCpuMs: 0; + + Severity severity; + if (tasks.length == 0) { + severity = Severity.NONE; + } else { + severity = getGcRatioSeverity(avgRuntimeMs, avgCpuMs, avgGcMs); + } + + HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), + _heuristicConfData.getHeuristicName(), severity, Utils.getHeuristicScore(severity, tasks.length)); + + result.addResultDetail("Number of tasks", Integer.toString(tasks.length)); + result.addResultDetail("Avg task runtime (ms)", Long.toString(avgRuntimeMs)); + result.addResultDetail("Avg task CPU time (ms)", Long.toString(avgCpuMs)); + result.addResultDetail("Avg task GC time (ms)", Long.toString(avgGcMs)); + result.addResultDetail("Task GC/CPU ratio", Double.toString(ratio)); + return result; + } + + private Severity getGcRatioSeverity(long runtimeMs, long cpuMs, long gcMs) { + double gcRatio = ((double)gcMs)/cpuMs; + Severity ratioSeverity = Severity.getSeverityAscending( + gcRatio, gcRatioLimits[0], gcRatioLimits[1], gcRatioLimits[2], gcRatioLimits[3]); + + // Severity is reduced if task runtime is insignificant + Severity runtimeSeverity = getRuntimeSeverity(runtimeMs); + + return Severity.min(ratioSeverity, runtimeSeverity); + } + + private Severity getRuntimeSeverity(long runtimeMs) { + return Severity.getSeverityAscending( + runtimeMs, runtimeLimits[0], runtimeLimits[1], runtimeLimits[2], runtimeLimits[3]); + } + +} \ No newline at end of file diff --git a/app/com/linkedin/drelephant/tez/heuristics/GenericMemoryHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/GenericMemoryHeuristic.java new file mode 100644 index 000000000..abc1a5189 --- /dev/null +++ b/app/com/linkedin/drelephant/tez/heuristics/GenericMemoryHeuristic.java @@ -0,0 +1,169 @@ +package com.linkedin.drelephant.tez.heuristics; + +import com.google.common.base.Strings; +import com.linkedin.drelephant.analysis.Heuristic; +import com.linkedin.drelephant.analysis.HeuristicResult; +import com.linkedin.drelephant.analysis.Severity; +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; +import com.linkedin.drelephant.math.Statistics; +import com.linkedin.drelephant.tez.data.TezApplicationData; +import com.linkedin.drelephant.tez.data.TezCounterData; +import com.linkedin.drelephant.tez.data.TezTaskData; +import com.linkedin.drelephant.util.Utils; +import org.apache.log4j.Logger; + +import org.apache.commons.io.FileUtils; + +import java.util.*; + +/** + * Analyzes mapper memory allocation and requirements + */ +public abstract class GenericMemoryHeuristic implements Heuristic { + + private static final Logger logger = Logger.getLogger(GenericMemoryHeuristic.class); + + //Severity Parameters + private static final String MEM_RATIO_SEVERITY = "memory_ratio_severity"; + private static final String DEFAULT_MAPPER_CONTAINER_SIZE = "2048"; + private static final String CONTAINER_MEM_DEFAULT_MB = "container_memory_default_mb"; + private String _containerMemConf; + + //Default Value of parameters + + private double [] memoryRatioLimits = {0.6d, 0.5d, 0.4d, 0.3d}; //Ratio of successful tasks + + private HeuristicConfigurationData _heuristicConfData; + + private String getContainerMemDefaultMBytes() { + Map paramMap = _heuristicConfData.getParamMap(); + if (paramMap.containsKey(CONTAINER_MEM_DEFAULT_MB)) { + String strValue = paramMap.get(CONTAINER_MEM_DEFAULT_MB); + try { + return strValue; + } + catch (NumberFormatException e) { + logger.warn(CONTAINER_MEM_DEFAULT_MB + ": expected number [" + strValue + "]"); + } + } + return DEFAULT_MAPPER_CONTAINER_SIZE; + } + + private void loadParameters() { + Map paramMap = _heuristicConfData.getParamMap(); + String heuristicName = _heuristicConfData.getHeuristicName(); + + double[] confSuccessRatioLimits = Utils.getParam(paramMap.get(MEM_RATIO_SEVERITY), memoryRatioLimits.length); + if (confSuccessRatioLimits != null) { + memoryRatioLimits = confSuccessRatioLimits; + } + logger.info(heuristicName + " will use " + MEM_RATIO_SEVERITY + " with the following threshold settings: " + + Arrays.toString(memoryRatioLimits)); + + + } + + public GenericMemoryHeuristic(String containerMemConf, HeuristicConfigurationData heuristicConfData) { + this._containerMemConf = containerMemConf; + this._heuristicConfData = heuristicConfData; + loadParameters(); + } + + public HeuristicConfigurationData getHeuristicConfData() { + return _heuristicConfData; + } + + protected abstract TezTaskData[] getTasks(TezApplicationData data); + + public HeuristicResult apply(TezApplicationData data) { + if(!data.getSucceeded()) { + return null; + } + TezTaskData[] tasks = getTasks(data); + + + List totalPhysicalMemory = new LinkedList(); + List totalVirtualMemory = new LinkedList(); + List runTime = new LinkedList(); + + for (TezTaskData task : tasks) { + + if (task.isSampled()) { + totalPhysicalMemory.add(task.getCounters().get(TezCounterData.CounterName.PHYSICAL_MEMORY_BYTES)); + totalVirtualMemory.add(task.getCounters().get(TezCounterData.CounterName.VIRTUAL_MEMORY_BYTES)); + runTime.add(task.getTotalRunTimeMs()); + } + + + } + + long averagePMem = Statistics.average(totalPhysicalMemory); + long averageVMem = Statistics.average(totalVirtualMemory); + long maxPMem; + long minPMem; + try{ + maxPMem = Collections.max(totalPhysicalMemory); + minPMem = Collections.min(totalPhysicalMemory); + + } + catch(Exception exception){ + maxPMem = 0; + minPMem = 0; + } + long averageRunTime = Statistics.average(runTime); + + String containerSizeStr; + + if(!Strings.isNullOrEmpty(data.getConf().getProperty(_containerMemConf))){ + containerSizeStr = data.getConf().getProperty(_containerMemConf); + } + else { + containerSizeStr = getContainerMemDefaultMBytes(); + } + + long containerSize = Long.valueOf(containerSizeStr) * FileUtils.ONE_MB; + + double averageMemMb = (double)((averagePMem) /FileUtils.ONE_MB) ; + + double ratio = averageMemMb / ((double)(containerSize / FileUtils.ONE_MB)); + + Severity severity ; + + if(tasks.length == 0){ + severity = Severity.NONE; + } + else{ + severity = getMemoryRatioSeverity(ratio); + } + + HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), + _heuristicConfData.getHeuristicName(), severity, Utils.getHeuristicScore(severity, tasks.length)); + + result.addResultDetail("Number of tasks", Integer.toString(tasks.length)); + result.addResultDetail("Maximum Physical Memory (MB)", + tasks.length == 0 ? "0" : Long.toString(maxPMem/FileUtils.ONE_MB)); + result.addResultDetail("Minimum Physical memory (MB)", + tasks.length == 0 ? "0" : Long.toString(minPMem/FileUtils.ONE_MB)); + result.addResultDetail("Average Physical Memory (MB)", + tasks.length == 0 ? "0" : Long.toString(averagePMem/FileUtils.ONE_MB)); + result.addResultDetail("Average Virtual Memory (MB)", + tasks.length == 0 ? "0" : Long.toString(averageVMem/FileUtils.ONE_MB)); + result.addResultDetail("Average Task RunTime", + tasks.length == 0 ? "0" : Statistics.readableTimespan(averageRunTime)); + result.addResultDetail("Requested Container Memory (MB)", + (tasks.length == 0 || containerSize == 0 || containerSize == -1) ? "0" : String.valueOf(containerSize / FileUtils.ONE_MB)); + + + return result; + + } + + private Severity getMemoryRatioSeverity(double ratio) { + return Severity.getSeverityDescending( + ratio, memoryRatioLimits[0], memoryRatioLimits[1], memoryRatioLimits[2], memoryRatioLimits[3]); + } + + + + +} \ No newline at end of file diff --git a/app/com/linkedin/drelephant/tez/heuristics/MapperDataSkewHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/MapperDataSkewHeuristic.java new file mode 100644 index 000000000..3fb4c431a --- /dev/null +++ b/app/com/linkedin/drelephant/tez/heuristics/MapperDataSkewHeuristic.java @@ -0,0 +1,31 @@ +package com.linkedin.drelephant.tez.heuristics; + + +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; +import com.linkedin.drelephant.tez.data.TezApplicationData; +import com.linkedin.drelephant.tez.data.TezCounterData; +import com.linkedin.drelephant.tez.data.TezTaskData; + +import java.util.Arrays; +import org.apache.log4j.Logger; + + +/** + * This Heuristic analyses the skewness in the task input data + */ +public class MapperDataSkewHeuristic extends GenericDataSkewHeuristic { + private static final Logger logger = Logger.getLogger(MapperDataSkewHeuristic.class); + + public MapperDataSkewHeuristic(HeuristicConfigurationData heuristicConfData) { + super(Arrays.asList( + TezCounterData.CounterName.HDFS_BYTES_READ, + TezCounterData.CounterName.S3A_BYTES_READ, + TezCounterData.CounterName.S3N_BYTES_READ + ), heuristicConfData); + } + + @Override + protected TezTaskData[] getTasks(TezApplicationData data) { + return data.getMapTaskData(); + } +} \ No newline at end of file diff --git a/app/com/linkedin/drelephant/tez/heuristics/MapperGCHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/MapperGCHeuristic.java new file mode 100644 index 000000000..b3f992adf --- /dev/null +++ b/app/com/linkedin/drelephant/tez/heuristics/MapperGCHeuristic.java @@ -0,0 +1,35 @@ +package com.linkedin.drelephant.tez.heuristics; + + +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; +import com.linkedin.drelephant.tez.data.TezApplicationData; +import com.linkedin.drelephant.tez.data.TezCounterData; +import com.linkedin.drelephant.tez.data.TezTaskData; +import com.linkedin.drelephant.util.Utils; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import com.linkedin.drelephant.analysis.Heuristic; +import com.linkedin.drelephant.analysis.HeuristicResult; +import com.linkedin.drelephant.analysis.Severity; +import com.linkedin.drelephant.math.Statistics; +import java.util.Map; +import org.apache.log4j.Logger; + + +/** + * Analyses garbage collection efficiency + */ +public class MapperGCHeuristic extends GenericGCHeuristic{ + private static final Logger logger = Logger.getLogger(MapperGCHeuristic.class); + + public MapperGCHeuristic(HeuristicConfigurationData heuristicConfData) { + super(heuristicConfData); + } + + @Override + protected TezTaskData[] getTasks(TezApplicationData data) { + return data.getMapTaskData(); + } + +} \ No newline at end of file diff --git a/app/com/linkedin/drelephant/tez/heuristics/MapperMemoryHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/MapperMemoryHeuristic.java new file mode 100644 index 000000000..6a9af98fd --- /dev/null +++ b/app/com/linkedin/drelephant/tez/heuristics/MapperMemoryHeuristic.java @@ -0,0 +1,29 @@ +package com.linkedin.drelephant.tez.heuristics; + +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; + +import com.linkedin.drelephant.tez.data.TezApplicationData; + +import com.linkedin.drelephant.tez.data.TezTaskData; + +import org.apache.log4j.Logger; + +/** + * Analyzes mapper memory allocation and requirements + */ +public class MapperMemoryHeuristic extends GenericMemoryHeuristic { + + private static final Logger logger = Logger.getLogger(MapperMemoryHeuristic.class); + + public static final String MAPPER_MEMORY_CONF = "mapreduce.map.memory.mb"; + + public MapperMemoryHeuristic(HeuristicConfigurationData __heuristicConfData){ + super(MAPPER_MEMORY_CONF, __heuristicConfData); + } + + @Override + protected TezTaskData[] getTasks(TezApplicationData data) { + return data.getMapTaskData(); + } + +} \ No newline at end of file diff --git a/app/com/linkedin/drelephant/tez/heuristics/MapperSpeedHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/MapperSpeedHeuristic.java new file mode 100644 index 000000000..af97ee5da --- /dev/null +++ b/app/com/linkedin/drelephant/tez/heuristics/MapperSpeedHeuristic.java @@ -0,0 +1,149 @@ +package com.linkedin.drelephant.tez.heuristics; + + +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; +import com.linkedin.drelephant.tez.data.TezApplicationData; +import com.linkedin.drelephant.tez.data.TezCounterData; +import com.linkedin.drelephant.tez.data.TezTaskData; +import com.linkedin.drelephant.util.Utils; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import com.linkedin.drelephant.analysis.HDFSContext; +import com.linkedin.drelephant.analysis.Heuristic; +import com.linkedin.drelephant.analysis.HeuristicResult; +import com.linkedin.drelephant.analysis.Severity; +import com.linkedin.drelephant.math.Statistics; + +import java.util.Map; + +import org.apache.commons.io.FileUtils; +import org.apache.log4j.Logger; + +/** + * Analyzes mapper task speed and efficiency + */ +public class MapperSpeedHeuristic implements Heuristic { + private static final Logger logger = Logger.getLogger(MapperSpeedHeuristic.class); + + // Severity parameters. + private static final String DISK_SPEED_SEVERITY = "disk_speed_severity"; + private static final String RUNTIME_SEVERITY = "runtime_severity_in_min"; + + // Default value of parameters + private double[] diskSpeedLimits = {1d/2, 1d/4, 1d/8, 1d/32}; // Fraction of HDFS block size + private double[] runtimeLimits = {5, 10, 15, 30}; // The Map task runtime in milli sec + + private List _counterNames = Arrays.asList( + TezCounterData.CounterName.HDFS_BYTES_READ, + TezCounterData.CounterName.S3A_BYTES_READ, + TezCounterData.CounterName.S3N_BYTES_READ + ); + + private HeuristicConfigurationData _heuristicConfData; + + private void loadParameters() { + Map paramMap = _heuristicConfData.getParamMap(); + String heuristicName = _heuristicConfData.getHeuristicName(); + + double[] confDiskSpeedThreshold = Utils.getParam(paramMap.get(DISK_SPEED_SEVERITY), diskSpeedLimits.length); + if (confDiskSpeedThreshold != null) { + diskSpeedLimits = confDiskSpeedThreshold; + } + logger.info(heuristicName + " will use " + DISK_SPEED_SEVERITY + " with the following threshold settings: " + + Arrays.toString(diskSpeedLimits)); + for (int i = 0; i < diskSpeedLimits.length; i++) { + diskSpeedLimits[i] = diskSpeedLimits[i] * HDFSContext.DISK_READ_SPEED; + } + + double[] confRuntimeThreshold = Utils.getParam(paramMap.get(RUNTIME_SEVERITY), runtimeLimits.length); + if (confRuntimeThreshold != null) { + runtimeLimits = confRuntimeThreshold; + } + logger.info(heuristicName + " will use " + RUNTIME_SEVERITY + " with the following threshold settings: " + Arrays + .toString(runtimeLimits)); + for (int i = 0; i < runtimeLimits.length; i++) { + runtimeLimits[i] = runtimeLimits[i] * Statistics.MINUTE_IN_MS; + } + } + + public MapperSpeedHeuristic(HeuristicConfigurationData heuristicConfData) { + this._heuristicConfData = heuristicConfData; + loadParameters(); + } + + public HeuristicConfigurationData getHeuristicConfData() { + return _heuristicConfData; + } + + public HeuristicResult apply(TezApplicationData data) { + + if(!data.getSucceeded()) { + return null; + } + + TezTaskData[] tasks = data.getMapTaskData(); + + List inputSizes = new ArrayList(); + List speeds = new ArrayList(); + List runtimesMs = new ArrayList(); + + for (TezTaskData task : tasks) { + + if (task.isSampled()) { + + long inputBytes = 0; + + for (TezCounterData.CounterName counterName: _counterNames) { + inputBytes += task.getCounters().get(counterName); + } + + long runtimeMs = task.getTotalRunTimeMs(); + inputSizes.add(inputBytes); + runtimesMs.add(runtimeMs); + //Speed is records per second + speeds.add((1000 * inputBytes) / (runtimeMs)); + } + } + + long medianSpeed; + long medianSize; + long medianRuntimeMs; + + if (tasks.length != 0) { + medianSpeed = Statistics.median(speeds); + medianSize = Statistics.median(inputSizes); + medianRuntimeMs = Statistics.median(runtimesMs); + } else { + medianSpeed = 0; + medianSize = 0; + medianRuntimeMs = 0; + } + + Severity severity = getDiskSpeedSeverity(medianSpeed); + + //This reduces severity if task runtime is insignificant + severity = Severity.min(severity, getRuntimeSeverity(medianRuntimeMs)); + + HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), + _heuristicConfData.getHeuristicName(), severity, Utils.getHeuristicScore(severity, tasks.length)); + + result.addResultDetail("Number of tasks", Integer.toString(tasks.length)); + result.addResultDetail("Median task input ", FileUtils.byteCountToDisplaySize(medianSize)); + result.addResultDetail("Median task runtime", Statistics.readableTimespan(medianRuntimeMs)); + result.addResultDetail("Median task speed", FileUtils.byteCountToDisplaySize(medianSpeed) + "/s"); + + return result; + } + + private Severity getDiskSpeedSeverity(long speed) { + return Severity.getSeverityDescending( + speed, diskSpeedLimits[0], diskSpeedLimits[1], diskSpeedLimits[2], diskSpeedLimits[3]); + } + + private Severity getRuntimeSeverity(long runtimeMs) { + return Severity.getSeverityAscending( + runtimeMs, runtimeLimits[0], runtimeLimits[1], runtimeLimits[2], runtimeLimits[3]); + } +} \ No newline at end of file diff --git a/app/com/linkedin/drelephant/tez/heuristics/MapperSpillHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/MapperSpillHeuristic.java new file mode 100644 index 000000000..270680925 --- /dev/null +++ b/app/com/linkedin/drelephant/tez/heuristics/MapperSpillHeuristic.java @@ -0,0 +1,128 @@ +package com.linkedin.drelephant.tez.heuristics; + +import com.linkedin.drelephant.analysis.Heuristic; +import com.linkedin.drelephant.analysis.HeuristicResult; +import com.linkedin.drelephant.analysis.Severity; +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; +import com.linkedin.drelephant.tez.data.TezCounterData; +import com.linkedin.drelephant.tez.data.TezApplicationData; +import com.linkedin.drelephant.tez.data.TezTaskData; +import com.linkedin.drelephant.util.Utils; +import org.apache.log4j.Logger; + + +import java.util.Arrays; +import java.util.Map; + +/** + * Analyzes mapper task data spill rates + */ +public class MapperSpillHeuristic implements Heuristic { + private static final Logger logger = Logger.getLogger(MapperSpillHeuristic.class); + private static final long THRESHOLD_SPILL_FACTOR = 10000; + + // Severity parameters. + private static final String SPILL_SEVERITY = "spill_severity"; + private static final String NUM_TASKS_SEVERITY = "num_tasks_severity"; + + // Default value of parameters + private double[] numTasksLimits = {50, 100, 500, 1000}; // Number of Map tasks. + private double[] spillLimits = {2.01d, 2.2d, 2.5d, 3.0d}; // Records spilled/total output records + + private HeuristicConfigurationData _heuristicConfData; + + private void loadParameters() { + Map paramMap = _heuristicConfData.getParamMap(); + String heuristicName = _heuristicConfData.getHeuristicName(); + + double[] confNumTasksThreshold = Utils.getParam(paramMap.get(NUM_TASKS_SEVERITY), numTasksLimits.length); + if (confNumTasksThreshold != null) { + numTasksLimits = confNumTasksThreshold; + } + logger.info(heuristicName + " will use " + NUM_TASKS_SEVERITY + " with the following threshold settings: " + + Arrays.toString(numTasksLimits)); + + double[] confSpillThreshold = Utils.getParam(paramMap.get(SPILL_SEVERITY), spillLimits.length); + if (confSpillThreshold != null) { + spillLimits = confSpillThreshold; + } + logger.info(heuristicName + " will use " + SPILL_SEVERITY + " with the following threshold settings: " + Arrays + .toString(spillLimits)); + for (int i = 0; i < spillLimits.length; i++) { + spillLimits[i] = spillLimits[i] * THRESHOLD_SPILL_FACTOR; + } + } + + public MapperSpillHeuristic(HeuristicConfigurationData heuristicConfData) { + this._heuristicConfData = heuristicConfData; + loadParameters(); + } + + @Override + public HeuristicConfigurationData getHeuristicConfData() { + return _heuristicConfData; + } + + @Override + public HeuristicResult apply(TezApplicationData data) { + if(!data.getSucceeded()) { + return null; + } + TezTaskData[] tasks = data.getMapTaskData(); + + long totalSpills = 0; + long totalOutputRecords = 0; + double ratioSpills = 0.0; + + for (TezTaskData task : tasks) { + + if (task.isSampled()) { + totalSpills += task.getCounters().get(TezCounterData.CounterName.SPILLED_RECORDS); + totalOutputRecords += task.getCounters().get(TezCounterData.CounterName.OUTPUT_RECORDS); + } + } + + //If both totalSpills and totalOutputRecords are zero then set ratioSpills to zero. + if (totalSpills == 0) { + ratioSpills = 0; + } else { + ratioSpills = (double) totalSpills / (double) totalOutputRecords; + } + + Severity severity = getSpillSeverity(ratioSpills); + + // Severity is reduced if number of tasks is small + Severity taskSeverity = getNumTasksSeverity(tasks.length); + severity = Severity.min(severity, taskSeverity); + + HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), + _heuristicConfData.getHeuristicName(), severity, Utils.getHeuristicScore(severity, tasks.length)); + + result.addResultDetail("Number of tasks", Integer.toString(tasks.length)); + result.addResultDetail("Avg spilled records per task", + tasks.length == 0 ? "0" : Long.toString(totalSpills / tasks.length)); + result.addResultDetail("Avg output records per task", + tasks.length == 0 ? "0" : Long.toString(totalOutputRecords / tasks.length)); + result.addResultDetail("Ratio of spilled records to output records", Double.toString(ratioSpills)); + + return result; + + } + + private Severity getSpillSeverity(double ratioSpills) { + + long normalizedSpillRatio = 0; + //Normalize the ratio to integer. + normalizedSpillRatio = (long) (ratioSpills * THRESHOLD_SPILL_FACTOR); + + return Severity.getSeverityAscending( + normalizedSpillRatio, spillLimits[0], spillLimits[1], spillLimits[2], spillLimits[3]); + } + + private Severity getNumTasksSeverity(long numTasks) { + return Severity.getSeverityAscending( + numTasks, numTasksLimits[0], numTasksLimits[1], numTasksLimits[2], numTasksLimits[3]); + } + + +} diff --git a/app/com/linkedin/drelephant/tez/heuristics/MapperTimeHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/MapperTimeHeuristic.java new file mode 100644 index 000000000..10e970299 --- /dev/null +++ b/app/com/linkedin/drelephant/tez/heuristics/MapperTimeHeuristic.java @@ -0,0 +1,166 @@ +package com.linkedin.drelephant.tez.heuristics; + + +import com.linkedin.drelephant.analysis.Heuristic; +import com.linkedin.drelephant.analysis.HeuristicResult; +import com.linkedin.drelephant.analysis.Severity; +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; +import com.linkedin.drelephant.tez.data.TezApplicationData; +import com.linkedin.drelephant.tez.data.TezCounterData; +import com.linkedin.drelephant.tez.data.TezTaskData; +import com.linkedin.drelephant.util.Utils; +import org.apache.commons.io.FileUtils; +import org.apache.log4j.Logger; + +import com.linkedin.drelephant.math.Statistics; + +import java.util.*; + +/** + * Analyzes mapper task runtimes + */ +public class MapperTimeHeuristic implements Heuristic { + + private static final Logger logger = Logger.getLogger(MapperTimeHeuristic.class); + + // Severity parameters. + private static final String SHORT_RUNTIME_SEVERITY = "short_runtime_severity_in_min"; + private static final String LONG_RUNTIME_SEVERITY = "long_runtime_severity_in_min"; + private static final String NUM_TASKS_SEVERITY = "num_tasks_severity"; + + // Default value of parameters + private double[] shortRuntimeLimits = {10, 4, 2, 1}; // Limits(ms) for tasks with shorter runtime + private double[] longRuntimeLimits = {15, 30, 60, 120}; // Limits(ms) for tasks with longer runtime + private double[] numTasksLimits = {50, 101, 500, 1000}; // Number of Map tasks. + + private List _counterNames = Arrays.asList( + TezCounterData.CounterName.HDFS_BYTES_READ, + TezCounterData.CounterName.S3A_BYTES_READ, + TezCounterData.CounterName.S3N_BYTES_READ + ); + + private HeuristicConfigurationData _heuristicConfData; + + private void loadParameters() { + Map paramMap = _heuristicConfData.getParamMap(); + String heuristicName = _heuristicConfData.getHeuristicName(); + + double[] confShortThreshold = Utils.getParam(paramMap.get(SHORT_RUNTIME_SEVERITY), shortRuntimeLimits.length); + if (confShortThreshold != null) { + shortRuntimeLimits = confShortThreshold; + } + logger.info(heuristicName + " will use " + SHORT_RUNTIME_SEVERITY + " with the following threshold settings: " + + Arrays.toString(shortRuntimeLimits)); + for (int i = 0; i < shortRuntimeLimits.length; i++) { + shortRuntimeLimits[i] = shortRuntimeLimits[i] * Statistics.MINUTE_IN_MS; + } + + double[] confLongThreshold = Utils.getParam(paramMap.get(LONG_RUNTIME_SEVERITY), longRuntimeLimits.length); + if (confLongThreshold != null) { + longRuntimeLimits = confLongThreshold; + } + logger.info(heuristicName + " will use " + LONG_RUNTIME_SEVERITY + " with the following threshold settings: " + + Arrays.toString(longRuntimeLimits)); + for (int i = 0; i < longRuntimeLimits.length; i++) { + longRuntimeLimits[i] = longRuntimeLimits[i] * Statistics.MINUTE_IN_MS; + } + + double[] confNumTasksThreshold = Utils.getParam(paramMap.get(NUM_TASKS_SEVERITY), numTasksLimits.length); + if (confNumTasksThreshold != null) { + numTasksLimits = confNumTasksThreshold; + } + logger.info(heuristicName + " will use " + NUM_TASKS_SEVERITY + " with the following threshold settings: " + Arrays + .toString(numTasksLimits)); + + + } + + public MapperTimeHeuristic(HeuristicConfigurationData heuristicConfData) { + this._heuristicConfData = heuristicConfData; + loadParameters(); + } + + public HeuristicConfigurationData getHeuristicConfData() { + return _heuristicConfData; + } + + public HeuristicResult apply(TezApplicationData data) { + if(!data.getSucceeded()) { + return null; + } + TezTaskData[] tasks = data.getMapTaskData(); + + List inputSizes = new ArrayList(); + List runtimesMs = new ArrayList(); + long taskMinMs = Long.MAX_VALUE; + long taskMaxMs = 0; + + for (TezTaskData task : tasks) { + + if (task.isSampled()) { + long inputByte = 0; + for (TezCounterData.CounterName counterName: _counterNames) { + inputByte += task.getCounters().get(counterName); + } + inputSizes.add(inputByte); + long taskTime = task.getTotalRunTimeMs(); + runtimesMs.add(taskTime); + taskMinMs = Math.min(taskMinMs, taskTime); + taskMaxMs = Math.max(taskMaxMs, taskTime); + } + } + + if(taskMinMs == Long.MAX_VALUE) { + taskMinMs = 0; + } + + long averageSize = Statistics.average(inputSizes); + long averageTimeMs = Statistics.average(runtimesMs); + + Severity shortTaskSeverity = shortTaskSeverity(tasks.length, averageTimeMs); + Severity longTaskSeverity = longTaskSeverity(tasks.length, averageTimeMs); + Severity severity = Severity.max(shortTaskSeverity, longTaskSeverity); + + HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), + _heuristicConfData.getHeuristicName(), severity, Utils.getHeuristicScore(severity, tasks.length)); + + result.addResultDetail("Number of tasks", Integer.toString(tasks.length)); + result.addResultDetail("Average task input size", FileUtils.byteCountToDisplaySize(averageSize)); + result.addResultDetail("Average task runtime", Statistics.readableTimespan(averageTimeMs)); + result.addResultDetail("Max task runtime", Statistics.readableTimespan(taskMaxMs)); + result.addResultDetail("Min task runtime", Statistics.readableTimespan(taskMinMs)); + + return result; + } + + private Severity shortTaskSeverity(long numTasks, long averageTimeMs) { + // We want to identify jobs with short task runtime + Severity severity = getShortRuntimeSeverity(averageTimeMs); + // Severity is reduced if number of tasks is small. + Severity numTaskSeverity = getNumTasksSeverity(numTasks); + return Severity.min(severity, numTaskSeverity); + } + + private Severity longTaskSeverity(long numTasks, long averageTimeMs) { + // We want to identify jobs with long task runtime. Severity is NOT reduced if num of tasks is large + return getLongRuntimeSeverity(averageTimeMs); + } + + private Severity getShortRuntimeSeverity(long runtimeMs) { + return Severity.getSeverityDescending( + runtimeMs, shortRuntimeLimits[0], shortRuntimeLimits[1], shortRuntimeLimits[2], shortRuntimeLimits[3]); + } + + private Severity getLongRuntimeSeverity(long runtimeMs) { + return Severity.getSeverityAscending( + runtimeMs, longRuntimeLimits[0], longRuntimeLimits[1], longRuntimeLimits[2], longRuntimeLimits[3]); + } + + private Severity getNumTasksSeverity(long numTasks) { + return Severity.getSeverityAscending( + numTasks, numTasksLimits[0], numTasksLimits[1], numTasksLimits[2], numTasksLimits[3]); + } + + + +} diff --git a/app/com/linkedin/drelephant/tez/heuristics/ReducerDataSkewHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/ReducerDataSkewHeuristic.java new file mode 100644 index 000000000..f05010f0d --- /dev/null +++ b/app/com/linkedin/drelephant/tez/heuristics/ReducerDataSkewHeuristic.java @@ -0,0 +1,26 @@ +package com.linkedin.drelephant.tez.heuristics; + +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; +import com.linkedin.drelephant.tez.data.TezApplicationData; +import com.linkedin.drelephant.tez.data.TezCounterData; +import com.linkedin.drelephant.tez.data.TezTaskData; +import org.apache.log4j.Logger; + +import java.util.Arrays; + + +/** + * This Heuristic analyses the skewness in the task input data + */ +public class ReducerDataSkewHeuristic extends GenericDataSkewHeuristic { + private static final Logger logger = Logger.getLogger(ReducerDataSkewHeuristic.class); + + public ReducerDataSkewHeuristic(HeuristicConfigurationData heuristicConfData) { + super(Arrays.asList(TezCounterData.CounterName.SHUFFLE_BYTES), heuristicConfData); + } + + @Override + protected TezTaskData[] getTasks(TezApplicationData data) { + return data.getReduceTaskData(); + } +} \ No newline at end of file diff --git a/app/com/linkedin/drelephant/tez/heuristics/ReducerGCHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/ReducerGCHeuristic.java new file mode 100644 index 000000000..af89e6e4d --- /dev/null +++ b/app/com/linkedin/drelephant/tez/heuristics/ReducerGCHeuristic.java @@ -0,0 +1,25 @@ +package com.linkedin.drelephant.tez.heuristics; + +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; +import com.linkedin.drelephant.tez.data.TezApplicationData; +import com.linkedin.drelephant.tez.data.TezTaskData; +import org.apache.log4j.Logger; + + +/** + * Analyses garbage collection efficiency + */ +public class ReducerGCHeuristic extends GenericGCHeuristic { + + private static final Logger logger = Logger.getLogger(ReducerGCHeuristic.class); + + public ReducerGCHeuristic(HeuristicConfigurationData heuristicConfData) { + super(heuristicConfData); + } + + @Override + protected TezTaskData[] getTasks(TezApplicationData data) { + return data.getReduceTaskData(); + } + +} diff --git a/app/com/linkedin/drelephant/tez/heuristics/ReducerMemoryHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/ReducerMemoryHeuristic.java new file mode 100644 index 000000000..a404f3202 --- /dev/null +++ b/app/com/linkedin/drelephant/tez/heuristics/ReducerMemoryHeuristic.java @@ -0,0 +1,31 @@ +package com.linkedin.drelephant.tez.heuristics; + + +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; +import com.linkedin.drelephant.tez.data.TezApplicationData; +import com.linkedin.drelephant.tez.data.TezTaskData; +import org.apache.log4j.Logger; + +/** + * Analyzes reducer memory allocation and requirements + */ + +public class ReducerMemoryHeuristic extends GenericMemoryHeuristic { + + private static final Logger logger = Logger.getLogger(ReducerMemoryHeuristic.class); + + public static final String REDUCER_MEMORY_CONF = "mapreduce.reduce.memory.mb"; + + public ReducerMemoryHeuristic(HeuristicConfigurationData __heuristicConfData){ + super(REDUCER_MEMORY_CONF, __heuristicConfData); + } + + @Override + protected TezTaskData[] getTasks(TezApplicationData data) { + return data.getReduceTaskData(); + } + + + + +} diff --git a/app/com/linkedin/drelephant/tez/heuristics/ReducerTimeHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/ReducerTimeHeuristic.java new file mode 100644 index 000000000..41de16a6c --- /dev/null +++ b/app/com/linkedin/drelephant/tez/heuristics/ReducerTimeHeuristic.java @@ -0,0 +1,151 @@ +package com.linkedin.drelephant.tez.heuristics; + + +import com.linkedin.drelephant.analysis.Heuristic; +import com.linkedin.drelephant.analysis.HeuristicResult; +import com.linkedin.drelephant.analysis.Severity; +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; +import com.linkedin.drelephant.tez.data.TezApplicationData; +import com.linkedin.drelephant.tez.data.TezTaskData; +import com.linkedin.drelephant.util.Utils; +import org.apache.log4j.Logger; + +import com.linkedin.drelephant.math.Statistics; + +import java.util.*; + +/** + * Analyzes reducer task runtimes + */ + +public class ReducerTimeHeuristic implements Heuristic { + + private static final Logger logger = Logger.getLogger(ReducerTimeHeuristic.class); + + // Severity parameters. + private static final String SHORT_RUNTIME_SEVERITY = "short_runtime_severity_in_min"; + private static final String LONG_RUNTIME_SEVERITY = "long_runtime_severity_in_min"; + private static final String NUM_TASKS_SEVERITY = "num_tasks_severity"; + + // Default value of parameters + private double[] shortRuntimeLimits = {10, 4, 2, 1}; // Limits(ms) for tasks with shorter runtime + private double[] longRuntimeLimits = {15, 30, 60, 120}; // Limits(ms) for tasks with longer runtime + private double[] numTasksLimits = {50, 101, 500, 1000}; // Number of Map tasks. + + private HeuristicConfigurationData _heuristicConfData; + + private void loadParameters() { + Map paramMap = _heuristicConfData.getParamMap(); + String heuristicName = _heuristicConfData.getHeuristicName(); + + double[] confShortThreshold = Utils.getParam(paramMap.get(SHORT_RUNTIME_SEVERITY), shortRuntimeLimits.length); + if (confShortThreshold != null) { + shortRuntimeLimits = confShortThreshold; + } + logger.info(heuristicName + " will use " + SHORT_RUNTIME_SEVERITY + " with the following threshold settings: " + + Arrays.toString(shortRuntimeLimits)); + for (int i = 0; i < shortRuntimeLimits.length; i++) { + shortRuntimeLimits[i] = shortRuntimeLimits[i] * Statistics.MINUTE_IN_MS; + } + + double[] confLongThreshold = Utils.getParam(paramMap.get(LONG_RUNTIME_SEVERITY), longRuntimeLimits.length); + if (confLongThreshold != null) { + longRuntimeLimits = confLongThreshold; + } + logger.info(heuristicName + " will use " + LONG_RUNTIME_SEVERITY + " with the following threshold settings: " + + Arrays.toString(longRuntimeLimits)); + for (int i = 0; i < longRuntimeLimits.length; i++) { + longRuntimeLimits[i] = longRuntimeLimits[i] * Statistics.MINUTE_IN_MS; + } + + double[] confNumTasksThreshold = Utils.getParam(paramMap.get(NUM_TASKS_SEVERITY), numTasksLimits.length); + if (confNumTasksThreshold != null) { + numTasksLimits = confNumTasksThreshold; + } + logger.info(heuristicName + " will use " + NUM_TASKS_SEVERITY + " with the following threshold settings: " + Arrays + .toString(numTasksLimits)); + + + } + + public ReducerTimeHeuristic(HeuristicConfigurationData heuristicConfData) { + this._heuristicConfData = heuristicConfData; + loadParameters(); + } + + public HeuristicConfigurationData getHeuristicConfData() { + return _heuristicConfData; + } + + public HeuristicResult apply(TezApplicationData data) { + if(!data.getSucceeded()) { + return null; + } + TezTaskData[] tasks = data.getReduceTaskData(); + + List runtimesMs = new ArrayList(); + long taskMinMs = Long.MAX_VALUE; + long taskMaxMs = 0; + + for (TezTaskData task : tasks) { + + if (task.isSampled()) { + long taskTime = task.getTotalRunTimeMs(); + runtimesMs.add(taskTime); + taskMinMs = Math.min(taskMinMs, taskTime); + taskMaxMs = Math.max(taskMaxMs, taskTime); + } + } + + if(taskMinMs == Long.MAX_VALUE) { + taskMinMs = 0; + } + + + long averageTimeMs = Statistics.average(runtimesMs); + + Severity shortTaskSeverity = shortTaskSeverity(tasks.length, averageTimeMs); + Severity longTaskSeverity = longTaskSeverity(tasks.length, averageTimeMs); + Severity severity = Severity.max(shortTaskSeverity, longTaskSeverity); + + HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), + _heuristicConfData.getHeuristicName(), severity, Utils.getHeuristicScore(severity, tasks.length)); + + result.addResultDetail("Number of tasks", Integer.toString(tasks.length)); + result.addResultDetail("Average task runtime", tasks.length == 0 ? "0" : (Statistics.readableTimespan(averageTimeMs).equals("") ? "0 sec" : Statistics.readableTimespan(averageTimeMs))); + result.addResultDetail("Max task runtime", tasks.length == 0 ? "0" : (Statistics.readableTimespan(taskMaxMs).equals("") ? "0 sec" : Statistics.readableTimespan(taskMaxMs)) ); + result.addResultDetail("Min task runtime", tasks.length == 0 ? "0" : (Statistics.readableTimespan(taskMinMs).equals("") ? "0 sec" :Statistics.readableTimespan(taskMinMs))) ; + + return result; + } + + private Severity shortTaskSeverity(long numTasks, long averageTimeMs) { + // We want to identify jobs with short task runtime + Severity severity = getShortRuntimeSeverity(averageTimeMs); + // Severity is reduced if number of tasks is small. + Severity numTaskSeverity = getNumTasksSeverity(numTasks); + return Severity.min(severity, numTaskSeverity); + } + + private Severity longTaskSeverity(long numTasks, long averageTimeMs) { + // We want to identify jobs with long task runtime. Severity is NOT reduced if num of tasks is large + return getLongRuntimeSeverity(averageTimeMs); + } + + private Severity getShortRuntimeSeverity(long runtimeMs) { + return Severity.getSeverityDescending( + runtimeMs, shortRuntimeLimits[0], shortRuntimeLimits[1], shortRuntimeLimits[2], shortRuntimeLimits[3]); + } + + private Severity getLongRuntimeSeverity(long runtimeMs) { + return Severity.getSeverityAscending( + runtimeMs, longRuntimeLimits[0], longRuntimeLimits[1], longRuntimeLimits[2], longRuntimeLimits[3]); + } + + private Severity getNumTasksSeverity(long numTasks) { + return Severity.getSeverityAscending( + numTasks, numTasksLimits[0], numTasksLimits[1], numTasksLimits[2], numTasksLimits[3]); + } + +} + diff --git a/app/com/linkedin/drelephant/util/InfoExtractor.java b/app/com/linkedin/drelephant/util/InfoExtractor.java index a81de19d9..b55cfe6be 100644 --- a/app/com/linkedin/drelephant/util/InfoExtractor.java +++ b/app/com/linkedin/drelephant/util/InfoExtractor.java @@ -19,6 +19,7 @@ import com.linkedin.drelephant.analysis.HadoopApplicationData; import com.linkedin.drelephant.configurations.scheduler.SchedulerConfiguration; import com.linkedin.drelephant.configurations.scheduler.SchedulerConfigurationData; +import com.linkedin.drelephant.tez.data.TezApplicationData; import com.linkedin.drelephant.exceptions.WorkflowClient; import com.linkedin.drelephant.mapreduce.data.MapReduceApplicationData; import com.linkedin.drelephant.schedulers.Scheduler; @@ -116,6 +117,9 @@ public static void loadInfo(AppResult result, HadoopApplicationData data) { } else if ( data instanceof SparkApplicationData) { properties = retrieveSparkProperties((SparkApplicationData) data); } + else if(data instanceof TezApplicationData){ + properties = retrieveTezProperties((TezApplicationData) data); + } Scheduler scheduler = getSchedulerInstance(data.getAppId(), properties); if (scheduler == null) { @@ -166,6 +170,10 @@ public static Properties retrieveMapreduceProperties(MapReduceApplicationData ap return appData.getConf(); } + public static Properties retrieveTezProperties(TezApplicationData appData) { + return appData.getConf(); + } + /** * Populates the given app result with the info from the given application data and scheduler. * diff --git a/test/com/linkedin/drelephant/tez/TezTaskLevelAggregatedMetricsTest.java b/test/com/linkedin/drelephant/tez/TezTaskLevelAggregatedMetricsTest.java new file mode 100644 index 000000000..0b98068a0 --- /dev/null +++ b/test/com/linkedin/drelephant/tez/TezTaskLevelAggregatedMetricsTest.java @@ -0,0 +1,46 @@ +package com.linkedin.drelephant.tez; + + +import com.linkedin.drelephant.tez.data.TezCounterData; +import com.linkedin.drelephant.tez.data.TezTaskData; +import org.junit.Assert; +import org.junit.Test; + +public class TezTaskLevelAggregatedMetricsTest { + + @Test + public void testZeroTasks() { + TezTaskData taskData[] = {}; + TezTaskLevelAggregatedMetrics taskMetrics = new TezTaskLevelAggregatedMetrics(taskData, 0, 0); + Assert.assertEquals(taskMetrics.getDelay(), 0); + Assert.assertEquals(taskMetrics.getResourceUsed(), 0); + Assert.assertEquals(taskMetrics.getResourceWasted(), 0); + } + + @Test + public void testNullTaskArray() { + TezTaskLevelAggregatedMetrics taskMetrics = new TezTaskLevelAggregatedMetrics(null, 0, 0); + Assert.assertEquals(taskMetrics.getDelay(), 0); + Assert.assertEquals(taskMetrics.getResourceUsed(), 0); + Assert.assertEquals(taskMetrics.getResourceWasted(), 0); + } + + @Test + public void testTaskLevelData() { + TezTaskData taskData[] = new TezTaskData[3]; + TezCounterData counterData = new TezCounterData(); + counterData.set(TezCounterData.CounterName.PHYSICAL_MEMORY_BYTES, 655577088L); + counterData.set(TezCounterData.CounterName.VIRTUAL_MEMORY_BYTES, 3051589632L); + long time[] = {0,0,0,1464218501117L, 1464218534148L}; + taskData[0] = new TezTaskData("task", "id"); + taskData[0].setTimeAndCounter(time,counterData); + taskData[1] = new TezTaskData("task", "id"); + taskData[1].setTimeAndCounter(new long[5],counterData); + // Non-sampled task, which does not contain time and counter data + taskData[2] = new TezTaskData("task", "id"); + TezTaskLevelAggregatedMetrics taskMetrics = new TezTaskLevelAggregatedMetrics(taskData, 4096L, 1463218501117L); + Assert.assertEquals(taskMetrics.getDelay(), 1000000000L); + Assert.assertEquals(taskMetrics.getResourceUsed(), 135168L); + Assert.assertEquals(taskMetrics.getResourceWasted(), 66627L); + } +} \ No newline at end of file diff --git a/test/com/linkedin/drelephant/tez/fetchers/TezFetcherTest.java b/test/com/linkedin/drelephant/tez/fetchers/TezFetcherTest.java new file mode 100644 index 000000000..2ab1953d0 --- /dev/null +++ b/test/com/linkedin/drelephant/tez/fetchers/TezFetcherTest.java @@ -0,0 +1,17 @@ +package com.linkedin.drelephant.tez.fetchers; + +import java.util.regex.Matcher; +import org.junit.Assert; +import org.junit.Test; + +public class TezFetcherTest { + + @Test + public void testDiagnosticMatcher() { + Matcher matcher = com.linkedin.drelephant.tez.fetchers.ThreadContextMR2.getDiagnosticMatcher("Task task_1443068695259_9143_m_000475 failed 1 time"); + Assert.assertEquals("Task[\\s\\u00A0]+(.*)[\\s\\u00A0]+failed[\\s\\u00A0]+([0-9])[\\s\\u00A0]+times[\\s\\u00A0]+", matcher.pattern().toString()); + Assert.assertEquals(false, matcher.matches()); + Assert.assertEquals(2, matcher.groupCount()); + } + +} \ No newline at end of file diff --git a/test/com/linkedin/drelephant/tez/heuristics/MapperDataSkewHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/MapperDataSkewHeuristicTest.java new file mode 100644 index 000000000..35d253bf0 --- /dev/null +++ b/test/com/linkedin/drelephant/tez/heuristics/MapperDataSkewHeuristicTest.java @@ -0,0 +1,130 @@ +package com.linkedin.drelephant.tez.heuristics; + +import com.linkedin.drelephant.analysis.*; +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import com.linkedin.drelephant.tez.data.TezApplicationData; +import com.linkedin.drelephant.tez.data.TezCounterData; +import com.linkedin.drelephant.tez.data.TezTaskData; +import junit.framework.TestCase; + + +public class MapperDataSkewHeuristicTest extends TestCase { + + private static final long UNITSIZE = HDFSContext.HDFS_BLOCK_SIZE / 64; //1MB + private static final long UNITSIZETIME = 1000000; //1000sec + + private static Map paramsMap = new HashMap(); + private static Heuristic _heuristic = new MapperDataSkewHeuristic(new HeuristicConfigurationData("test_heuristic", + "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); + + public void testCritical() throws IOException { + assertEquals(Severity.CRITICAL, analyzeJob(200, 200, 1 * UNITSIZE, 100 * UNITSIZE)); + } + + public void testSevere() throws IOException { + assertEquals(Severity.SEVERE, analyzeJob(200, 200, 10 * UNITSIZE, 100 * UNITSIZE)); + } + + public void testModerate() throws IOException { + assertEquals(Severity.MODERATE, analyzeJob(200, 200, 20 * UNITSIZE, 100 * UNITSIZE)); + } + + public void testLow() throws IOException { + assertEquals(Severity.LOW, analyzeJob(200, 200, 30 * UNITSIZE, 100 * UNITSIZE)); + } + + public void testNone() throws IOException { + assertEquals(Severity.NONE, analyzeJob(200, 200, 50 * UNITSIZE, 100 * UNITSIZE)); + } + + public void testSmallFiles() throws IOException { + assertEquals(Severity.NONE, analyzeJob(200, 200, 1 * UNITSIZE, 5 * UNITSIZE)); + } + + public void testSmallTasks() throws IOException { + assertEquals(Severity.NONE, analyzeJob(5, 5, 10 * UNITSIZE, 100 * UNITSIZE)); + } + + public void testCriticalTime() throws IOException { + assertEquals(Severity.CRITICAL, analyzeJobTime(200, 200, 1 * UNITSIZETIME, 100 * UNITSIZETIME)); + } + + public void testSevereTime() throws IOException { + assertEquals(Severity.SEVERE, analyzeJobTime(200, 200, 10 * UNITSIZETIME, 100 * UNITSIZETIME)); + } + + public void testModerateTime() throws IOException { + assertEquals(Severity.MODERATE, analyzeJobTime(200, 200, 20 * UNITSIZETIME, 100 * UNITSIZETIME)); + } + + public void testLowTime() throws IOException { + assertEquals(Severity.LOW, analyzeJobTime(200, 200, 30 * UNITSIZETIME, 100 * UNITSIZETIME)); + } + + public void testNoneTime() throws IOException { + assertEquals(Severity.NONE, analyzeJobTime(200, 200, 50 * UNITSIZETIME, 100 * UNITSIZETIME)); + } + + public void testSmallTasksTime() throws IOException { + assertEquals(Severity.NONE, analyzeJobTime(5, 5, 10 * UNITSIZETIME, 100 * UNITSIZETIME)); + } + + private Severity analyzeJob(int numSmallTasks, int numLargeTasks, long smallInputSize, long largeInputSize) + throws IOException { + TezCounterData jobCounter = new TezCounterData(); + TezTaskData[] mappers = new TezTaskData[numSmallTasks + numLargeTasks + 1]; + + TezCounterData smallCounter = new TezCounterData(); + smallCounter.set(TezCounterData.CounterName.HDFS_BYTES_READ, smallInputSize); + + TezCounterData largeCounter = new TezCounterData(); + largeCounter.set(TezCounterData.CounterName.S3A_BYTES_READ, largeInputSize); + + int i = 0; + for (; i < numSmallTasks; i++) { + mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + mappers[i].setTimeAndCounter(new long[5], smallCounter); + } + for (; i < numSmallTasks + numLargeTasks; i++) { + mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + mappers[i].setTimeAndCounter(new long[5], largeCounter); + } + // Non-sampled task, which does not contain time and counter data + mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + + TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setMapTaskData(mappers); + HeuristicResult result = _heuristic.apply(data); + return result.getSeverity(); + + } + + private Severity analyzeJobTime(int numSmallTasks, int numLongTasks, long smallTimeTaken, long longTimeTaken) + throws IOException { + TezCounterData jobCounter = new TezCounterData(); + TezTaskData[] mappers = new TezTaskData[numSmallTasks + numLongTasks + 1]; + + int i = 0; + for (; i < numSmallTasks; i++) { + mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + mappers[i].setTotalTimeMs(smallTimeTaken, true); + mappers[i].setCounter(jobCounter); + } + for (; i < numSmallTasks + numLongTasks; i++) { + mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + mappers[i].setTotalTimeMs(longTimeTaken, true); + mappers[i].setCounter(jobCounter); + } + // Non-sampled task, which does not contain time data + mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + + TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setMapTaskData(mappers); + HeuristicResult result = _heuristic.apply(data); + return result.getSeverity(); + + } +} \ No newline at end of file diff --git a/test/com/linkedin/drelephant/tez/heuristics/MapperGCHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/MapperGCHeuristicTest.java new file mode 100644 index 000000000..9f9713311 --- /dev/null +++ b/test/com/linkedin/drelephant/tez/heuristics/MapperGCHeuristicTest.java @@ -0,0 +1,68 @@ +package com.linkedin.drelephant.tez.heuristics; + +import com.linkedin.drelephant.analysis.ApplicationType; +import com.linkedin.drelephant.analysis.HeuristicResult; +import com.linkedin.drelephant.analysis.Severity; +import com.linkedin.drelephant.tez.data.TezApplicationData; +import com.linkedin.drelephant.tez.data.TezCounterData; +import com.linkedin.drelephant.tez.data.TezTaskData; +import com.linkedin.drelephant.analysis.Heuristic; +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; +import junit.framework.TestCase; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +public class MapperGCHeuristicTest extends TestCase { + + private static Map paramsMap = new HashMap(); + + private static Heuristic _heuristic = new MapperGCHeuristic(new HeuristicConfigurationData("test_heuristic", + "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); + + private static int NUMTASKS = 100; + + public void testGCCritical() throws IOException { + assertEquals(Severity.CRITICAL, analyzeJob(1000000, 50000, 2000)); + } + + public void testGCSevere() throws IOException { + assertEquals(Severity.SEVERE, analyzeJob(1000000, 50000, 1500)); + } + + public void testGCModerate() throws IOException { + assertEquals(Severity.MODERATE, analyzeJob(1000000, 50000, 1000)); + } + + public void testGCNone() throws IOException { + assertEquals(Severity.NONE, analyzeJob(1000000, 50000, 300)); + } + + public void testShortTasksNone() throws IOException { + assertEquals(Severity.NONE, analyzeJob(100000, 50000, 2000)); + } + + + private Severity analyzeJob(long runtimeMs, long cpuMs, long gcMs) throws IOException { + TezCounterData jobCounter = new TezCounterData(); + TezTaskData[] mappers = new TezTaskData[NUMTASKS + 1]; + + TezCounterData counter = new TezCounterData(); + counter.set(TezCounterData.CounterName.CPU_MILLISECONDS, cpuMs); + counter.set(TezCounterData.CounterName.GC_TIME_MILLIS, gcMs); + + int i = 0; + for (; i < NUMTASKS; i++) { + mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + mappers[i].setTimeAndCounter(new long[]{runtimeMs, 0 , 0, 0, 0}, counter); + } + // Non-sampled task, which does not contain time and counter data + mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + + TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setMapTaskData(mappers); + HeuristicResult result = _heuristic.apply(data); + return result.getSeverity(); + } + +} \ No newline at end of file diff --git a/test/com/linkedin/drelephant/tez/heuristics/MapperMemoryHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/MapperMemoryHeuristicTest.java new file mode 100644 index 000000000..3ed836ba1 --- /dev/null +++ b/test/com/linkedin/drelephant/tez/heuristics/MapperMemoryHeuristicTest.java @@ -0,0 +1,78 @@ +package com.linkedin.drelephant.tez.heuristics; + +import com.linkedin.drelephant.analysis.ApplicationType; +import com.linkedin.drelephant.analysis.Heuristic; +import com.linkedin.drelephant.analysis.HeuristicResult; +import com.linkedin.drelephant.analysis.Severity; +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; +import com.linkedin.drelephant.tez.data.TezApplicationData; +import com.linkedin.drelephant.tez.data.TezCounterData; +import com.linkedin.drelephant.tez.data.TezTaskData; +import junit.framework.TestCase; +import org.apache.commons.io.FileUtils; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +public class MapperMemoryHeuristicTest extends TestCase{ + + private static Map paramsMap = new HashMap(); + private static Heuristic _heuristic = new MapperMemoryHeuristic(new HeuristicConfigurationData("test_heuristic", + "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); + + private int NUMTASKS = 100; + + public void testLargeContainerSizeCritical() throws IOException { + assertEquals(Severity.CRITICAL, analyzeJob(2048, 8192)); + } + + public void testLargeContainerSizeSevere() throws IOException { + assertEquals(Severity.SEVERE, analyzeJob(3072, 8192)); + } + + public void testLargeContainerSizeModerate() throws IOException { + assertEquals(Severity.MODERATE, analyzeJob(4096, 8192)); + } + + public void testLargeContainerSizeNone() throws IOException { + assertEquals(Severity.NONE, analyzeJob(6144, 8192)); + } + + // If the task use default container size, it should not be flagged + // Not using Default Container param, will calculate severity irrespective of default container size Chaning NONE -> CRITICAL + public void testDefaultContainerNone() throws IOException { + assertEquals(Severity.CRITICAL, analyzeJob(256, 2048)); + } + + // Not using Default Container param, will calculate severity irrespective of default container size Chaning NONE -> MODERATE + public void testDefaultContainerNoneMore() throws IOException { + assertEquals(Severity.MODERATE, analyzeJob(1024, 2048)); + } + + private Severity analyzeJob(long taskAvgMemMB, long containerMemMB) throws IOException { + TezCounterData jobCounter = new TezCounterData(); + TezTaskData[] mappers = new TezTaskData[NUMTASKS + 1]; + + TezCounterData counter = new TezCounterData(); + counter.set(TezCounterData.CounterName.PHYSICAL_MEMORY_BYTES, taskAvgMemMB* FileUtils.ONE_MB); + + Properties p = new Properties(); + p.setProperty(MapperMemoryHeuristic.MAPPER_MEMORY_CONF, Long.toString(containerMemMB)); + + int i = 0; + for (; i < NUMTASKS; i++) { + mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + mappers[i].setTime(new long[5]); + mappers[i].setCounter(counter); + } + // Non-sampled task, which does not contain time and counter data + mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + + TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setMapTaskData(mappers); + data.setConf(p); + HeuristicResult result = _heuristic.apply(data); + return result.getSeverity(); + } +} \ No newline at end of file diff --git a/test/com/linkedin/drelephant/tez/heuristics/MapperSpeedHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/MapperSpeedHeuristicTest.java new file mode 100644 index 000000000..bf1faa34f --- /dev/null +++ b/test/com/linkedin/drelephant/tez/heuristics/MapperSpeedHeuristicTest.java @@ -0,0 +1,84 @@ +package com.linkedin.drelephant.tez.heuristics; + +import com.linkedin.drelephant.analysis.ApplicationType; +import com.linkedin.drelephant.analysis.Heuristic; +import com.linkedin.drelephant.analysis.HeuristicResult; +import com.linkedin.drelephant.analysis.Severity; +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; +import com.linkedin.drelephant.math.Statistics; +import com.linkedin.drelephant.tez.data.TezApplicationData; +import com.linkedin.drelephant.tez.data.TezCounterData; +import com.linkedin.drelephant.tez.data.TezTaskData; +import junit.framework.TestCase; +import org.apache.commons.io.FileUtils; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +public class MapperSpeedHeuristicTest extends TestCase { + + private static Map paramsMap = new HashMap(); + private static Heuristic _heuristic = new MapperSpeedHeuristic(new HeuristicConfigurationData("test_heuristic", + "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); + + private static final long MB_IN_BYTES = FileUtils.ONE_MB; + private static final long MINUTE_IN_MS = Statistics.MINUTE_IN_MS; + private static final int NUMTASKS = 100; + + public void testCritical() throws IOException { + long runtime = 120 * MINUTE_IN_MS; + long speed_factor = (runtime * MB_IN_BYTES) / 1000; + assertEquals(Severity.CRITICAL, analyzeJob(runtime, 1 * speed_factor)); + } + + public void testSevere() throws IOException { + long runtime = 120 * MINUTE_IN_MS; + long speed_factor = (runtime * MB_IN_BYTES) / 1000; + assertEquals(Severity.SEVERE, analyzeJob(runtime, 4 * speed_factor)); + } + + public void testModerate() throws IOException { + long runtime = 120 * MINUTE_IN_MS; + long speed_factor = (runtime * MB_IN_BYTES) / 1000; + assertEquals(Severity.MODERATE, analyzeJob(runtime, 13 * speed_factor)); + } + + public void testLow() throws IOException { + long runtime = 120 * MINUTE_IN_MS; + long speed_factor = (runtime * MB_IN_BYTES) / 1000; + assertEquals(Severity.LOW, analyzeJob(runtime, 50 * speed_factor)); + } + + public void testNone() throws IOException { + long runtime = 120 * MINUTE_IN_MS; + long speed_factor = (runtime * MB_IN_BYTES) / 1000; + assertEquals(Severity.NONE, analyzeJob(runtime, 51 * speed_factor)); + } + + public void testShortTask() throws IOException { + long runtime = 2 * MINUTE_IN_MS; + long speed_factor = (runtime * MB_IN_BYTES) / 1000; + assertEquals(Severity.NONE, analyzeJob(runtime, 1 * speed_factor)); + } + + private Severity analyzeJob(long runtimeMs, long readBytes) throws IOException { + TezCounterData jobCounter = new TezCounterData(); + TezTaskData[] mappers = new TezTaskData[NUMTASKS + 1]; + + TezCounterData counter = new TezCounterData(); + counter.set(TezCounterData.CounterName.HDFS_BYTES_READ, readBytes / 2); + counter.set(TezCounterData.CounterName.S3A_BYTES_READ, readBytes / 2); + + int i = 0; + for (; i < NUMTASKS; i++) { + mappers[i] = new TezTaskData(counter, new long[] { runtimeMs, 0, 0 ,0, 0}); + } + // Non-sampled task, which does not contain time and counter data + mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + + TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setMapTaskData(mappers); + HeuristicResult result = _heuristic.apply(data); + return result.getSeverity(); + } +} \ No newline at end of file diff --git a/test/com/linkedin/drelephant/tez/heuristics/MapperSpillHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/MapperSpillHeuristicTest.java new file mode 100644 index 000000000..9b11bba97 --- /dev/null +++ b/test/com/linkedin/drelephant/tez/heuristics/MapperSpillHeuristicTest.java @@ -0,0 +1,74 @@ +package com.linkedin.drelephant.tez.heuristics; + +import com.linkedin.drelephant.analysis.ApplicationType; +import com.linkedin.drelephant.analysis.Heuristic; +import com.linkedin.drelephant.analysis.HeuristicResult; +import com.linkedin.drelephant.analysis.Severity; +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; +import com.linkedin.drelephant.tez.data.TezApplicationData; +import com.linkedin.drelephant.tez.data.TezCounterData; +import com.linkedin.drelephant.tez.data.TezTaskData; +import junit.framework.TestCase; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + + +public class MapperSpillHeuristicTest extends TestCase{ + + private static Map paramsMap = new HashMap(); + private static Heuristic _heuristic = new MapperSpillHeuristic(new HeuristicConfigurationData("test_heuristic", + "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); + + public void testCritical() throws IOException { + // Spill ratio 3.0, 1000 tasks + assertEquals(Severity.CRITICAL, analyzeJob(3000, 1000, 1000)); + } + + public void testSevere() throws IOException { + // Spill ratio 2.5, 1000 tasks + assertEquals(Severity.SEVERE, analyzeJob(2500, 1000, 1000)); + } + + public void testModerate() throws IOException { + // Spill ratio 2.3, 1000 tasks + assertEquals(Severity.MODERATE, analyzeJob(2300, 1000, 1000)); + } + + public void testLow() throws IOException { + // Spill ratio 2.1, 1000 tasks + assertEquals(Severity.LOW, analyzeJob(2100, 1000, 1000)); + } + + public void testNone() throws IOException { + // Spill ratio 1.0, 1000 tasks + assertEquals(Severity.NONE, analyzeJob(1000, 1000, 1000)); + } + + public void testSmallNumTasks() throws IOException { + // Spill ratio 3.0, should be critical, but number of task is small(10), final result is NONE + assertEquals(Severity.NONE, analyzeJob(3000, 1000, 10)); + } + + private Severity analyzeJob(long spilledRecords, long mapRecords, int numTasks) throws IOException { + TezCounterData jobCounter = new TezCounterData(); + TezTaskData[] mappers = new TezTaskData[numTasks + 1]; + + TezCounterData counter = new TezCounterData(); + counter.set(TezCounterData.CounterName.SPILLED_RECORDS, spilledRecords); + counter.set(TezCounterData.CounterName.OUTPUT_RECORDS, mapRecords); + + int i = 0; + for (; i < numTasks; i++) { + mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + mappers[i].setTimeAndCounter(new long[5], counter); + } + // Non-sampled task, which does not contain time and counter data + mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + + TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setMapTaskData(mappers); + HeuristicResult result = _heuristic.apply(data); + return result.getSeverity(); + } +} \ No newline at end of file diff --git a/test/com/linkedin/drelephant/tez/heuristics/MapperTimeHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/MapperTimeHeuristicTest.java new file mode 100644 index 000000000..ff9ed8a2f --- /dev/null +++ b/test/com/linkedin/drelephant/tez/heuristics/MapperTimeHeuristicTest.java @@ -0,0 +1,95 @@ +package com.linkedin.drelephant.tez.heuristics; + +import com.linkedin.drelephant.analysis.ApplicationType; +import com.linkedin.drelephant.analysis.Heuristic; +import com.linkedin.drelephant.analysis.HeuristicResult; +import com.linkedin.drelephant.analysis.Severity; +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; +import com.linkedin.drelephant.math.Statistics; +import com.linkedin.drelephant.tez.data.TezApplicationData; +import com.linkedin.drelephant.tez.data.TezCounterData; +import com.linkedin.drelephant.tez.data.TezTaskData; +import junit.framework.TestCase; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + + +public class MapperTimeHeuristicTest extends TestCase { + + private static final long DUMMY_INPUT_SIZE = 0; + + private static Map paramsMap = new HashMap(); + private static Heuristic _heuristic = new MapperTimeHeuristic(new HeuristicConfigurationData("test_heuristic", + "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); + + // Test batch 1: Large runtime. Heuristic is not affected by various number of tasks */ + + public void testLongRuntimeTasksCritical() throws IOException { + // Should decrease split size and increase number of tasks + assertEquals(Severity.CRITICAL, analyzeJob(10, 120 * Statistics.MINUTE_IN_MS)); + } + + public void testLongRuntimeTasksCriticalMore() throws IOException { + // Should decrease split size and increase number of tasks + assertEquals(Severity.CRITICAL, analyzeJob(1000, 120 * Statistics.MINUTE_IN_MS)); + } + + public void testLongRuntimeTasksSevere() throws IOException { + // Should decrease split size and increase number of tasks + assertEquals(Severity.SEVERE, analyzeJob(10, 60 * Statistics.MINUTE_IN_MS)); + } + + public void testLongRuntimeTasksSevereMore() throws IOException { + // Should decrease split size and increase number of tasks + assertEquals(Severity.SEVERE, analyzeJob(1000, 60 * Statistics.MINUTE_IN_MS)); + } + + // Test batch 2: Short runtime and various number of tasks + + public void testShortRuntimeTasksCritical() throws IOException { + // Should increase split size and decrease number of tasks + assertEquals(Severity.CRITICAL, analyzeJob(1000, 1 * Statistics.MINUTE_IN_MS)); + } + + public void testShortRuntimeTasksSevere() throws IOException { + // Should increase split size and decrease number of tasks + assertEquals(Severity.SEVERE, analyzeJob(500, 1 * Statistics.MINUTE_IN_MS)); + } + + public void testShortRuntimeTasksModerate() throws IOException { + assertEquals(Severity.MODERATE, analyzeJob(101, 1 * Statistics.MINUTE_IN_MS)); + } + + public void testShortRuntimeTasksLow() throws IOException { + assertEquals(Severity.LOW, analyzeJob(50, 1 * Statistics.MINUTE_IN_MS)); + } + + public void testShortRuntimeTasksNone() throws IOException { + // Small file with small number of tasks and short runtime. This should be the common case. + assertEquals(Severity.NONE, analyzeJob(5, 1 * Statistics.MINUTE_IN_MS)); + } + + private Severity analyzeJob(int numTasks, long runtime) throws IOException { + TezCounterData jobCounter = new TezCounterData(); + TezTaskData[] mappers = new TezTaskData[numTasks + 1]; + + TezCounterData taskCounter = new TezCounterData(); + taskCounter.set(TezCounterData.CounterName.S3A_BYTES_READ, DUMMY_INPUT_SIZE / 4); + + + int i = 0; + for (; i < numTasks; i++) { + mappers[i] = new TezTaskData(jobCounter,new long[] { runtime, 0, 0, 0, 0 }); + } + // Non-sampled task, which does not contain time and counter data + mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + + TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setMapTaskData(mappers); + HeuristicResult result = _heuristic.apply(data); + return result.getSeverity(); + } + + +} \ No newline at end of file diff --git a/test/com/linkedin/drelephant/tez/heuristics/ReducerDataSkewHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/ReducerDataSkewHeuristicTest.java new file mode 100644 index 000000000..268625965 --- /dev/null +++ b/test/com/linkedin/drelephant/tez/heuristics/ReducerDataSkewHeuristicTest.java @@ -0,0 +1,126 @@ +package com.linkedin.drelephant.tez.heuristics; + +import com.linkedin.drelephant.analysis.*; +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; +import com.linkedin.drelephant.tez.data.TezApplicationData; +import com.linkedin.drelephant.tez.data.TezCounterData; +import com.linkedin.drelephant.tez.data.TezTaskData; +import junit.framework.TestCase; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +public class ReducerDataSkewHeuristicTest extends TestCase{ + + private static final long UNITSIZE = HDFSContext.HDFS_BLOCK_SIZE / 64; //1mb + private static final long UNITSIZETIME = 1000000; //1000sec + + private static Map paramsMap = new HashMap(); + private static Heuristic _heuristic = new ReducerDataSkewHeuristic(new HeuristicConfigurationData("test_heuristic", + "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); + + public void testCritical() throws IOException { + assertEquals(Severity.CRITICAL, analyzeJob(200, 200, 1 * UNITSIZE, 100 * UNITSIZE)); + } + + public void testSevere() throws IOException { + assertEquals(Severity.SEVERE, analyzeJob(200, 200, 10 * UNITSIZE, 100 * UNITSIZE)); + } + + public void testModerate() throws IOException { + assertEquals(Severity.MODERATE, analyzeJob(200, 200, 20 * UNITSIZE, 100 * UNITSIZE)); + } + + public void testLow() throws IOException { + assertEquals(Severity.LOW, analyzeJob(200, 200, 30 * UNITSIZE, 100 * UNITSIZE)); + } + + public void testNone() throws IOException { + assertEquals(Severity.NONE, analyzeJob(200, 200, 50 * UNITSIZE, 100 * UNITSIZE)); + } + + public void testSmallFiles() throws IOException { + assertEquals(Severity.NONE, analyzeJob(200, 200, 1 * UNITSIZE, 5 * UNITSIZE)); + } + + public void testSmallTasks() throws IOException { + assertEquals(Severity.NONE, analyzeJob(5, 5, 10 * UNITSIZE, 100 * UNITSIZE)); + } + + public void testCriticalTime() throws IOException { + assertEquals(Severity.CRITICAL, analyzeJobTime(200, 200, 1 * UNITSIZETIME, 100 * UNITSIZETIME)); + } + + public void testSevereTime() throws IOException { + assertEquals(Severity.SEVERE, analyzeJobTime(200, 200, 10 * UNITSIZETIME, 100 * UNITSIZETIME)); + } + + public void testModerateTime() throws IOException { + assertEquals(Severity.MODERATE, analyzeJobTime(200, 200, 20 * UNITSIZETIME, 100 * UNITSIZETIME)); + } + + public void testLowTime() throws IOException { + assertEquals(Severity.LOW, analyzeJobTime(200, 200, 30 * UNITSIZETIME, 100 * UNITSIZETIME)); + } + + public void testNoneTime() throws IOException { + assertEquals(Severity.NONE, analyzeJobTime(200, 200, 50 * UNITSIZETIME, 100 * UNITSIZETIME)); + } + + public void testSmallTasksTime() throws IOException { + assertEquals(Severity.NONE, analyzeJobTime(5, 5, 10 * UNITSIZETIME, 100 * UNITSIZETIME)); + } + + private Severity analyzeJob(int numSmallTasks, int numLargeTasks, long smallInputSize, long largeInputSize) + throws IOException { + TezCounterData jobCounter = new TezCounterData(); + TezTaskData[] reducers = new TezTaskData[numSmallTasks + numLargeTasks + 1]; + + TezCounterData smallCounter = new TezCounterData(); + smallCounter.set(TezCounterData.CounterName.SHUFFLE_BYTES, smallInputSize); + + TezCounterData largeCounter = new TezCounterData(); + largeCounter.set(TezCounterData.CounterName.SHUFFLE_BYTES, largeInputSize); + + int i = 0; + for (; i < numSmallTasks; i++) { + reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + reducers[i].setTimeAndCounter(new long[5], smallCounter); + } + for (; i < numSmallTasks + numLargeTasks; i++) { + reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + reducers[i].setTimeAndCounter(new long[5], largeCounter); + } + // Non-sampled task, which does not contain time and counter data + reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + + TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setReduceTaskData(reducers); + HeuristicResult result = _heuristic.apply(data); + return result.getSeverity(); + } + + private Severity analyzeJobTime(int numSmallTasks, int numLongTasks, long smallTimeTaken, long longTimeTaken) + throws IOException { + TezCounterData jobCounter = new TezCounterData(); + TezTaskData[] reducers = new TezTaskData[numSmallTasks + numLongTasks + 1]; + + int i = 0; + for (; i < numSmallTasks; i++) { + reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + reducers[i].setTotalTimeMs(smallTimeTaken, true); + reducers[i].setCounter(jobCounter); + } + for (; i < numSmallTasks + numLongTasks; i++) { + reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + reducers[i].setTotalTimeMs(longTimeTaken, true); + reducers[i].setCounter(jobCounter); + } + // Non-sampled task, which does not contain time data + reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setReduceTaskData(reducers); + HeuristicResult result = _heuristic.apply(data); + return result.getSeverity(); + + } +} \ No newline at end of file diff --git a/test/com/linkedin/drelephant/tez/heuristics/ReducerGCHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/ReducerGCHeuristicTest.java new file mode 100644 index 000000000..254459f28 --- /dev/null +++ b/test/com/linkedin/drelephant/tez/heuristics/ReducerGCHeuristicTest.java @@ -0,0 +1,67 @@ +package com.linkedin.drelephant.tez.heuristics; + +import com.linkedin.drelephant.analysis.ApplicationType; +import com.linkedin.drelephant.analysis.Heuristic; +import com.linkedin.drelephant.analysis.HeuristicResult; +import com.linkedin.drelephant.analysis.Severity; +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; +import com.linkedin.drelephant.tez.data.TezApplicationData; +import com.linkedin.drelephant.tez.data.TezCounterData; +import com.linkedin.drelephant.tez.data.TezTaskData; +import junit.framework.TestCase; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +public class ReducerGCHeuristicTest extends TestCase{ + + private static Map paramsMap = new HashMap(); + + private static Heuristic _heuristic = new ReducerGCHeuristic(new HeuristicConfigurationData("test_heuristic", + "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); + + private static int NUMTASKS = 100; + + public void testGCCritical() throws IOException { + assertEquals(Severity.CRITICAL, analyzeJob(1000000, 50000, 2000)); + } + + public void testGCSevere() throws IOException { + assertEquals(Severity.SEVERE, analyzeJob(1000000, 50000, 1500)); + } + + public void testGCModerate() throws IOException { + assertEquals(Severity.MODERATE, analyzeJob(1000000, 50000, 1000)); + } + + public void testGCNone() throws IOException { + assertEquals(Severity.NONE, analyzeJob(1000000, 50000, 300)); + } + + public void testShortTasksNone() throws IOException { + assertEquals(Severity.NONE, analyzeJob(100000, 50000, 2000)); + } + + + private Severity analyzeJob(long runtimeMs, long cpuMs, long gcMs) throws IOException { + TezCounterData jobCounter = new TezCounterData(); + TezTaskData[] reducers = new TezTaskData[NUMTASKS + 1]; + + TezCounterData counter = new TezCounterData(); + counter.set(TezCounterData.CounterName.CPU_MILLISECONDS, cpuMs); + counter.set(TezCounterData.CounterName.GC_TIME_MILLIS, gcMs); + + int i = 0; + for (; i < NUMTASKS; i++) { + reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + reducers[i].setTimeAndCounter(new long[] { runtimeMs, 0, 0, 0, 0 }, counter); + } + // Non-sampled task, which does not contain time and counter data + reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + + TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setReduceTaskData(reducers); + HeuristicResult result = _heuristic.apply(data); + return result.getSeverity(); + } +} \ No newline at end of file diff --git a/test/com/linkedin/drelephant/tez/heuristics/ReducerMemoryHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/ReducerMemoryHeuristicTest.java new file mode 100644 index 000000000..f6064d468 --- /dev/null +++ b/test/com/linkedin/drelephant/tez/heuristics/ReducerMemoryHeuristicTest.java @@ -0,0 +1,77 @@ +package com.linkedin.drelephant.tez.heuristics; + +import com.linkedin.drelephant.analysis.ApplicationType; +import com.linkedin.drelephant.analysis.Heuristic; +import com.linkedin.drelephant.analysis.HeuristicResult; +import com.linkedin.drelephant.analysis.Severity; +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; +import com.linkedin.drelephant.tez.data.TezApplicationData; +import com.linkedin.drelephant.tez.data.TezCounterData; +import com.linkedin.drelephant.tez.data.TezTaskData; +import junit.framework.TestCase; +import org.apache.commons.io.FileUtils; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +public class ReducerMemoryHeuristicTest extends TestCase { + + private static Map paramsMap = new HashMap(); + private static Heuristic _heuristic = new ReducerMemoryHeuristic(new HeuristicConfigurationData("test_heuristic", + "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); + + private int NUMTASKS = 100; + + public void testLargeContainerSizeCritical() throws IOException { + assertEquals(Severity.CRITICAL, analyzeJob(2048, 8192)); + } + + public void testLargeContainerSizeSevere() throws IOException { + assertEquals(Severity.SEVERE, analyzeJob(3072, 8192)); + } + + public void testLargeContainerSizeModerate() throws IOException { + assertEquals(Severity.MODERATE, analyzeJob(4096, 8192)); + } + + public void testLargeContainerSizeNone() throws IOException { + assertEquals(Severity.NONE, analyzeJob(6144, 8192)); + } + + // If the task use default container size, it should not be flagged + // Not using Default Container param, will calculate severity irrespective of default container size Chaning NONE -> CRITICAL + public void testDefaultContainerNone() throws IOException { + assertEquals(Severity.CRITICAL, analyzeJob(256, 2048)); + } + + // Not using Default Container param, will calculate severity irrespective of default container size Chaning NONE -> MODERATE + public void testDefaultContainerNoneMore() throws IOException { + assertEquals(Severity.MODERATE, analyzeJob(1024, 2048)); + } + + private Severity analyzeJob(long taskAvgMemMB, long containerMemMB) throws IOException { + TezCounterData jobCounter = new TezCounterData(); + TezTaskData[] reducers = new TezTaskData[NUMTASKS + 1]; + + TezCounterData counter = new TezCounterData(); + counter.set(TezCounterData.CounterName.PHYSICAL_MEMORY_BYTES, taskAvgMemMB* FileUtils.ONE_MB); + + Properties p = new Properties(); + p.setProperty(com.linkedin.drelephant.mapreduce.heuristics.ReducerMemoryHeuristic.REDUCER_MEMORY_CONF, Long.toString(containerMemMB)); + + int i = 0; + for (; i < NUMTASKS; i++) { + reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + reducers[i].setTimeAndCounter(new long[5], counter); + } + // Non-sampled task, which does not contain time and counter data + reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + + TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setReduceTaskData(reducers); + data.setConf(p); + HeuristicResult result = _heuristic.apply(data); + return result.getSeverity(); + } +} \ No newline at end of file diff --git a/test/com/linkedin/drelephant/tez/heuristics/ReducerTimeHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/ReducerTimeHeuristicTest.java new file mode 100644 index 000000000..e17db9146 --- /dev/null +++ b/test/com/linkedin/drelephant/tez/heuristics/ReducerTimeHeuristicTest.java @@ -0,0 +1,81 @@ +package com.linkedin.drelephant.tez.heuristics; + +import com.linkedin.drelephant.analysis.ApplicationType; +import com.linkedin.drelephant.analysis.Heuristic; +import com.linkedin.drelephant.analysis.HeuristicResult; +import com.linkedin.drelephant.analysis.Severity; +import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; +import com.linkedin.drelephant.math.Statistics; +import com.linkedin.drelephant.tez.data.TezApplicationData; +import com.linkedin.drelephant.tez.data.TezCounterData; +import com.linkedin.drelephant.tez.data.TezTaskData; +import junit.framework.TestCase; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + + +public class ReducerTimeHeuristicTest extends TestCase { + + private static Map paramsMap = new HashMap(); + private static Heuristic _heuristic = new ReducerTimeHeuristic(new HeuristicConfigurationData("test_heuristic", + "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); + + private static final long MINUTE_IN_MS = Statistics.MINUTE_IN_MS;; + + public void testShortRunetimeCritical() throws IOException { + assertEquals(Severity.CRITICAL, analyzeJob(1 * MINUTE_IN_MS, 1000)); + } + + public void testShortRunetimeSevere() throws IOException { + assertEquals(Severity.SEVERE, analyzeJob(1 * MINUTE_IN_MS, 500)); + } + + public void testShortRunetimeModerate() throws IOException { + assertEquals(Severity.MODERATE, analyzeJob(1 * MINUTE_IN_MS, 101)); + } + + public void testShortRunetimeLow() throws IOException { + assertEquals(Severity.LOW, analyzeJob(1 * MINUTE_IN_MS, 50)); + } + + public void testShortRunetimeNone() throws IOException { + assertEquals(Severity.NONE, analyzeJob(1 * MINUTE_IN_MS, 2)); + } + + public void testLongRunetimeCritical() throws IOException { + assertEquals(Severity.CRITICAL, analyzeJob(120 * MINUTE_IN_MS, 10)); + } + + // Long runtime severity is not affected by number of tasks + public void testLongRunetimeCriticalMore() throws IOException { + assertEquals(Severity.CRITICAL, analyzeJob(120 * MINUTE_IN_MS, 1000)); + } + + public void testLongRunetimeSevere() throws IOException { + assertEquals(Severity.SEVERE, analyzeJob(60 * MINUTE_IN_MS, 10)); + } + + public void testLongRunetimeSevereMore() throws IOException { + assertEquals(Severity.SEVERE, analyzeJob(60 * MINUTE_IN_MS, 1000)); + } + + private Severity analyzeJob(long runtimeMs, int numTasks) throws IOException { + TezCounterData dummyCounter = new TezCounterData(); + TezTaskData[] reducers = new TezTaskData[numTasks + 1]; + + int i = 0; + for (; i < numTasks; i++) { + reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + reducers[i].setTime(new long[] { runtimeMs, 0, 0, 0, 0 }); + reducers[i].setCounter(dummyCounter); + } + // Non-sampled task, which does not contain time and counter data + reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + + TezApplicationData data = new TezApplicationData().setCounters(dummyCounter).setReduceTaskData(reducers); + HeuristicResult result = _heuristic.apply(data); + return result.getSeverity(); + } +} \ No newline at end of file From 9cc52646cf747a11efd4a1c02697ef5f4115e54b Mon Sep 17 00:00:00 2001 From: Sumant Date: Thu, 14 Dec 2017 11:04:34 -0800 Subject: [PATCH 2/5] Added Apache License --- .../tez/heuristics/GenericDataSkewHeuristic.java | 7 +++++++ .../drelephant/tez/heuristics/GenericGCHeuristic.java | 6 ++++++ .../drelephant/tez/heuristics/GenericMemoryHeuristic.java | 6 ++++++ .../drelephant/tez/heuristics/MapperDataSkewHeuristic.java | 6 ++++++ .../drelephant/tez/heuristics/MapperGCHeuristic.java | 6 ++++++ .../drelephant/tez/heuristics/MapperMemoryHeuristic.java | 6 ++++++ .../drelephant/tez/heuristics/MapperSpeedHeuristic.java | 6 ++++++ .../drelephant/tez/heuristics/MapperTimeHeuristic.java | 6 ++++++ .../tez/heuristics/ReducerDataSkewHeuristic.java | 6 ++++++ .../drelephant/tez/heuristics/ReducerGCHeuristic.java | 6 ++++++ .../drelephant/tez/heuristics/ReducerMemoryHeuristic.java | 6 ++++++ .../drelephant/tez/heuristics/ReducerTimeHeuristic.java | 6 ++++++ .../drelephant/tez/TezTaskLevelAggregatedMetricsTest.java | 6 ++++++ .../linkedin/drelephant/tez/fetchers/TezFetcherTest.java | 6 ++++++ .../tez/heuristics/MapperDataSkewHeuristicTest.java | 6 ++++++ .../drelephant/tez/heuristics/MapperGCHeuristicTest.java | 6 ++++++ .../tez/heuristics/MapperMemoryHeuristicTest.java | 6 ++++++ .../tez/heuristics/MapperSpeedHeuristicTest.java | 6 ++++++ .../tez/heuristics/MapperSpillHeuristicTest.java | 6 ++++++ .../drelephant/tez/heuristics/MapperTimeHeuristicTest.java | 6 ++++++ .../tez/heuristics/ReducerDataSkewHeuristicTest.java | 6 ++++++ .../drelephant/tez/heuristics/ReducerGCHeuristicTest.java | 6 ++++++ .../tez/heuristics/ReducerMemoryHeuristicTest.java | 6 ++++++ .../tez/heuristics/ReducerTimeHeuristicTest.java | 6 ++++++ 24 files changed, 145 insertions(+) diff --git a/app/com/linkedin/drelephant/tez/heuristics/GenericDataSkewHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/GenericDataSkewHeuristic.java index 61a115296..107a84ee9 100644 --- a/app/com/linkedin/drelephant/tez/heuristics/GenericDataSkewHeuristic.java +++ b/app/com/linkedin/drelephant/tez/heuristics/GenericDataSkewHeuristic.java @@ -1,3 +1,10 @@ +/* + * Copyright 2017 Electronic Arts Inc. + * + * Licensed under the Apache License, Version 2.0 + * + */ + package com.linkedin.drelephant.tez.heuristics; diff --git a/app/com/linkedin/drelephant/tez/heuristics/GenericGCHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/GenericGCHeuristic.java index f146a0386..7e67edbd8 100644 --- a/app/com/linkedin/drelephant/tez/heuristics/GenericGCHeuristic.java +++ b/app/com/linkedin/drelephant/tez/heuristics/GenericGCHeuristic.java @@ -1,3 +1,9 @@ +/* + * Copyright 2017 Electronic Arts Inc. + * + * Licensed under the Apache License, Version 2.0 + * + */ package com.linkedin.drelephant.tez.heuristics; import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; diff --git a/app/com/linkedin/drelephant/tez/heuristics/GenericMemoryHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/GenericMemoryHeuristic.java index abc1a5189..a9c0104b4 100644 --- a/app/com/linkedin/drelephant/tez/heuristics/GenericMemoryHeuristic.java +++ b/app/com/linkedin/drelephant/tez/heuristics/GenericMemoryHeuristic.java @@ -1,3 +1,9 @@ +/* + * Copyright 2017 Electronic Arts Inc. + * + * Licensed under the Apache License, Version 2.0 + * + */ package com.linkedin.drelephant.tez.heuristics; import com.google.common.base.Strings; diff --git a/app/com/linkedin/drelephant/tez/heuristics/MapperDataSkewHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/MapperDataSkewHeuristic.java index 3fb4c431a..42c27b987 100644 --- a/app/com/linkedin/drelephant/tez/heuristics/MapperDataSkewHeuristic.java +++ b/app/com/linkedin/drelephant/tez/heuristics/MapperDataSkewHeuristic.java @@ -1,3 +1,9 @@ +/* + * Copyright 2017 Electronic Arts Inc. + * + * Licensed under the Apache License, Version 2.0 + * + */ package com.linkedin.drelephant.tez.heuristics; diff --git a/app/com/linkedin/drelephant/tez/heuristics/MapperGCHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/MapperGCHeuristic.java index b3f992adf..14a091d1a 100644 --- a/app/com/linkedin/drelephant/tez/heuristics/MapperGCHeuristic.java +++ b/app/com/linkedin/drelephant/tez/heuristics/MapperGCHeuristic.java @@ -1,3 +1,9 @@ +/* + * Copyright 2017 Electronic Arts Inc. + * + * Licensed under the Apache License, Version 2.0 + * + */ package com.linkedin.drelephant.tez.heuristics; diff --git a/app/com/linkedin/drelephant/tez/heuristics/MapperMemoryHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/MapperMemoryHeuristic.java index 6a9af98fd..cc1932a1f 100644 --- a/app/com/linkedin/drelephant/tez/heuristics/MapperMemoryHeuristic.java +++ b/app/com/linkedin/drelephant/tez/heuristics/MapperMemoryHeuristic.java @@ -1,3 +1,9 @@ +/* + * Copyright 2017 Electronic Arts Inc. + * + * Licensed under the Apache License, Version 2.0 + * + */ package com.linkedin.drelephant.tez.heuristics; import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; diff --git a/app/com/linkedin/drelephant/tez/heuristics/MapperSpeedHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/MapperSpeedHeuristic.java index af97ee5da..a8bb5ebe7 100644 --- a/app/com/linkedin/drelephant/tez/heuristics/MapperSpeedHeuristic.java +++ b/app/com/linkedin/drelephant/tez/heuristics/MapperSpeedHeuristic.java @@ -1,3 +1,9 @@ +/* + * Copyright 2017 Electronic Arts Inc. + * + * Licensed under the Apache License, Version 2.0 + * + */ package com.linkedin.drelephant.tez.heuristics; diff --git a/app/com/linkedin/drelephant/tez/heuristics/MapperTimeHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/MapperTimeHeuristic.java index 10e970299..0f71a1d20 100644 --- a/app/com/linkedin/drelephant/tez/heuristics/MapperTimeHeuristic.java +++ b/app/com/linkedin/drelephant/tez/heuristics/MapperTimeHeuristic.java @@ -1,3 +1,9 @@ +/* + * Copyright 2017 Electronic Arts Inc. + * + * Licensed under the Apache License, Version 2.0 + * + */ package com.linkedin.drelephant.tez.heuristics; diff --git a/app/com/linkedin/drelephant/tez/heuristics/ReducerDataSkewHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/ReducerDataSkewHeuristic.java index f05010f0d..7daa8ba0d 100644 --- a/app/com/linkedin/drelephant/tez/heuristics/ReducerDataSkewHeuristic.java +++ b/app/com/linkedin/drelephant/tez/heuristics/ReducerDataSkewHeuristic.java @@ -1,3 +1,9 @@ +/* + * Copyright 2017 Electronic Arts Inc. + * + * Licensed under the Apache License, Version 2.0 + * + */ package com.linkedin.drelephant.tez.heuristics; import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; diff --git a/app/com/linkedin/drelephant/tez/heuristics/ReducerGCHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/ReducerGCHeuristic.java index af89e6e4d..facda408b 100644 --- a/app/com/linkedin/drelephant/tez/heuristics/ReducerGCHeuristic.java +++ b/app/com/linkedin/drelephant/tez/heuristics/ReducerGCHeuristic.java @@ -1,3 +1,9 @@ +/* + * Copyright 2017 Electronic Arts Inc. + * + * Licensed under the Apache License, Version 2.0 + * + */ package com.linkedin.drelephant.tez.heuristics; import com.linkedin.drelephant.configurations.heuristic.HeuristicConfigurationData; diff --git a/app/com/linkedin/drelephant/tez/heuristics/ReducerMemoryHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/ReducerMemoryHeuristic.java index a404f3202..f34cf7e66 100644 --- a/app/com/linkedin/drelephant/tez/heuristics/ReducerMemoryHeuristic.java +++ b/app/com/linkedin/drelephant/tez/heuristics/ReducerMemoryHeuristic.java @@ -1,3 +1,9 @@ +/* + * Copyright 2017 Electronic Arts Inc. + * + * Licensed under the Apache License, Version 2.0 + * + */ package com.linkedin.drelephant.tez.heuristics; diff --git a/app/com/linkedin/drelephant/tez/heuristics/ReducerTimeHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/ReducerTimeHeuristic.java index 41de16a6c..f5f8edcaf 100644 --- a/app/com/linkedin/drelephant/tez/heuristics/ReducerTimeHeuristic.java +++ b/app/com/linkedin/drelephant/tez/heuristics/ReducerTimeHeuristic.java @@ -1,3 +1,9 @@ +/* + * Copyright 2017 Electronic Arts Inc. + * + * Licensed under the Apache License, Version 2.0 + * + */ package com.linkedin.drelephant.tez.heuristics; diff --git a/test/com/linkedin/drelephant/tez/TezTaskLevelAggregatedMetricsTest.java b/test/com/linkedin/drelephant/tez/TezTaskLevelAggregatedMetricsTest.java index 0b98068a0..256c9b9f2 100644 --- a/test/com/linkedin/drelephant/tez/TezTaskLevelAggregatedMetricsTest.java +++ b/test/com/linkedin/drelephant/tez/TezTaskLevelAggregatedMetricsTest.java @@ -1,3 +1,9 @@ +/* + * Copyright 2017 Electronic Arts Inc. + * + * Licensed under the Apache License, Version 2.0 + * + */ package com.linkedin.drelephant.tez; diff --git a/test/com/linkedin/drelephant/tez/fetchers/TezFetcherTest.java b/test/com/linkedin/drelephant/tez/fetchers/TezFetcherTest.java index 2ab1953d0..feaafad46 100644 --- a/test/com/linkedin/drelephant/tez/fetchers/TezFetcherTest.java +++ b/test/com/linkedin/drelephant/tez/fetchers/TezFetcherTest.java @@ -1,3 +1,9 @@ +/* + * Copyright 2017 Electronic Arts Inc. + * + * Licensed under the Apache License, Version 2.0 + * + */ package com.linkedin.drelephant.tez.fetchers; import java.util.regex.Matcher; diff --git a/test/com/linkedin/drelephant/tez/heuristics/MapperDataSkewHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/MapperDataSkewHeuristicTest.java index 35d253bf0..2b8c97728 100644 --- a/test/com/linkedin/drelephant/tez/heuristics/MapperDataSkewHeuristicTest.java +++ b/test/com/linkedin/drelephant/tez/heuristics/MapperDataSkewHeuristicTest.java @@ -1,3 +1,9 @@ +/* + * Copyright 2017 Electronic Arts Inc. + * + * Licensed under the Apache License, Version 2.0 + * + */ package com.linkedin.drelephant.tez.heuristics; import com.linkedin.drelephant.analysis.*; diff --git a/test/com/linkedin/drelephant/tez/heuristics/MapperGCHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/MapperGCHeuristicTest.java index 9f9713311..c4a47d89a 100644 --- a/test/com/linkedin/drelephant/tez/heuristics/MapperGCHeuristicTest.java +++ b/test/com/linkedin/drelephant/tez/heuristics/MapperGCHeuristicTest.java @@ -1,3 +1,9 @@ +/* + * Copyright 2017 Electronic Arts Inc. + * + * Licensed under the Apache License, Version 2.0 + * + */ package com.linkedin.drelephant.tez.heuristics; import com.linkedin.drelephant.analysis.ApplicationType; diff --git a/test/com/linkedin/drelephant/tez/heuristics/MapperMemoryHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/MapperMemoryHeuristicTest.java index 3ed836ba1..0a0bbb263 100644 --- a/test/com/linkedin/drelephant/tez/heuristics/MapperMemoryHeuristicTest.java +++ b/test/com/linkedin/drelephant/tez/heuristics/MapperMemoryHeuristicTest.java @@ -1,3 +1,9 @@ +/* + * Copyright 2017 Electronic Arts Inc. + * + * Licensed under the Apache License, Version 2.0 + * + */ package com.linkedin.drelephant.tez.heuristics; import com.linkedin.drelephant.analysis.ApplicationType; diff --git a/test/com/linkedin/drelephant/tez/heuristics/MapperSpeedHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/MapperSpeedHeuristicTest.java index bf1faa34f..a2dba6c64 100644 --- a/test/com/linkedin/drelephant/tez/heuristics/MapperSpeedHeuristicTest.java +++ b/test/com/linkedin/drelephant/tez/heuristics/MapperSpeedHeuristicTest.java @@ -1,3 +1,9 @@ +/* + * Copyright 2017 Electronic Arts Inc. + * + * Licensed under the Apache License, Version 2.0 + * + */ package com.linkedin.drelephant.tez.heuristics; import com.linkedin.drelephant.analysis.ApplicationType; diff --git a/test/com/linkedin/drelephant/tez/heuristics/MapperSpillHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/MapperSpillHeuristicTest.java index 9b11bba97..736fd9866 100644 --- a/test/com/linkedin/drelephant/tez/heuristics/MapperSpillHeuristicTest.java +++ b/test/com/linkedin/drelephant/tez/heuristics/MapperSpillHeuristicTest.java @@ -1,3 +1,9 @@ +/* + * Copyright 2017 Electronic Arts Inc. + * + * Licensed under the Apache License, Version 2.0 + * + */ package com.linkedin.drelephant.tez.heuristics; import com.linkedin.drelephant.analysis.ApplicationType; diff --git a/test/com/linkedin/drelephant/tez/heuristics/MapperTimeHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/MapperTimeHeuristicTest.java index ff9ed8a2f..824398800 100644 --- a/test/com/linkedin/drelephant/tez/heuristics/MapperTimeHeuristicTest.java +++ b/test/com/linkedin/drelephant/tez/heuristics/MapperTimeHeuristicTest.java @@ -1,3 +1,9 @@ +/* + * Copyright 2017 Electronic Arts Inc. + * + * Licensed under the Apache License, Version 2.0 + * + */ package com.linkedin.drelephant.tez.heuristics; import com.linkedin.drelephant.analysis.ApplicationType; diff --git a/test/com/linkedin/drelephant/tez/heuristics/ReducerDataSkewHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/ReducerDataSkewHeuristicTest.java index 268625965..9bd06838a 100644 --- a/test/com/linkedin/drelephant/tez/heuristics/ReducerDataSkewHeuristicTest.java +++ b/test/com/linkedin/drelephant/tez/heuristics/ReducerDataSkewHeuristicTest.java @@ -1,3 +1,9 @@ +/* + * Copyright 2017 Electronic Arts Inc. + * + * Licensed under the Apache License, Version 2.0 + * + */ package com.linkedin.drelephant.tez.heuristics; import com.linkedin.drelephant.analysis.*; diff --git a/test/com/linkedin/drelephant/tez/heuristics/ReducerGCHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/ReducerGCHeuristicTest.java index 254459f28..b157c6268 100644 --- a/test/com/linkedin/drelephant/tez/heuristics/ReducerGCHeuristicTest.java +++ b/test/com/linkedin/drelephant/tez/heuristics/ReducerGCHeuristicTest.java @@ -1,3 +1,9 @@ +/* + * Copyright 2017 Electronic Arts Inc. + * + * Licensed under the Apache License, Version 2.0 + * + */ package com.linkedin.drelephant.tez.heuristics; import com.linkedin.drelephant.analysis.ApplicationType; diff --git a/test/com/linkedin/drelephant/tez/heuristics/ReducerMemoryHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/ReducerMemoryHeuristicTest.java index f6064d468..fd4f5ef71 100644 --- a/test/com/linkedin/drelephant/tez/heuristics/ReducerMemoryHeuristicTest.java +++ b/test/com/linkedin/drelephant/tez/heuristics/ReducerMemoryHeuristicTest.java @@ -1,3 +1,9 @@ +/* + * Copyright 2017 Electronic Arts Inc. + * + * Licensed under the Apache License, Version 2.0 + * + */ package com.linkedin.drelephant.tez.heuristics; import com.linkedin.drelephant.analysis.ApplicationType; diff --git a/test/com/linkedin/drelephant/tez/heuristics/ReducerTimeHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/ReducerTimeHeuristicTest.java index e17db9146..aac3964a6 100644 --- a/test/com/linkedin/drelephant/tez/heuristics/ReducerTimeHeuristicTest.java +++ b/test/com/linkedin/drelephant/tez/heuristics/ReducerTimeHeuristicTest.java @@ -1,3 +1,9 @@ +/* + * Copyright 2017 Electronic Arts Inc. + * + * Licensed under the Apache License, Version 2.0 + * + */ package com.linkedin.drelephant.tez.heuristics; import com.linkedin.drelephant.analysis.ApplicationType; From 805eb6157909581b3a4e1d80690701839f9ed132 Mon Sep 17 00:00:00 2001 From: Sumant Date: Fri, 2 Mar 2018 11:40:52 -0800 Subject: [PATCH 3/5] Indent Change and License header update --- .../drelephant/tez/TezMetricsAggregator.java | 139 ++-- .../tez/TezTaskLevelAggregatedMetrics.java | 239 +++--- .../tez/data/TezApplicationData.java | 233 +++--- .../drelephant/tez/data/TezCounterData.java | 319 ++++---- .../drelephant/tez/data/TezTaskData.java | 243 ++++--- .../drelephant/tez/fetchers/TezFetcher.java | 686 +++++++++--------- .../heuristics/GenericDataSkewHeuristic.java | 261 +++---- .../tez/heuristics/GenericGCHeuristic.java | 192 ++--- .../heuristics/GenericMemoryHeuristic.java | 224 +++--- .../heuristics/MapperDataSkewHeuristic.java | 36 +- .../tez/heuristics/MapperGCHeuristic.java | 28 +- .../tez/heuristics/MapperMemoryHeuristic.java | 30 +- .../tez/heuristics/MapperSpeedHeuristic.java | 198 ++--- .../tez/heuristics/MapperSpillHeuristic.java | 188 ++--- .../tez/heuristics/MapperTimeHeuristic.java | 242 +++--- .../heuristics/ReducerDataSkewHeuristic.java | 28 +- .../tez/heuristics/ReducerGCHeuristic.java | 28 +- .../heuristics/ReducerMemoryHeuristic.java | 30 +- .../tez/heuristics/ReducerTimeHeuristic.java | 218 +++--- .../TezTaskLevelAggregatedMetricsTest.java | 78 +- .../tez/fetchers/TezFetcherTest.java | 26 +- .../MapperDataSkewHeuristicTest.java | 196 ++--- .../tez/heuristics/MapperGCHeuristicTest.java | 84 ++- .../heuristics/MapperMemoryHeuristicTest.java | 108 +-- .../heuristics/MapperSpeedHeuristicTest.java | 116 +-- .../heuristics/MapperSpillHeuristicTest.java | 100 +-- .../heuristics/MapperTimeHeuristicTest.java | 122 ++-- .../ReducerDataSkewHeuristicTest.java | 196 ++--- .../heuristics/ReducerGCHeuristicTest.java | 84 ++- .../ReducerMemoryHeuristicTest.java | 100 +-- .../heuristics/ReducerTimeHeuristicTest.java | 108 +-- 31 files changed, 2611 insertions(+), 2269 deletions(-) diff --git a/app/com/linkedin/drelephant/tez/TezMetricsAggregator.java b/app/com/linkedin/drelephant/tez/TezMetricsAggregator.java index 2963b1c0f..c04edd7fd 100644 --- a/app/com/linkedin/drelephant/tez/TezMetricsAggregator.java +++ b/app/com/linkedin/drelephant/tez/TezMetricsAggregator.java @@ -1,3 +1,18 @@ +/* + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + * + */ package com.linkedin.drelephant.tez; import com.linkedin.drelephant.analysis.*; @@ -12,83 +27,83 @@ public class TezMetricsAggregator implements HadoopMetricsAggregator { - private static final Logger logger = Logger.getLogger(TezMetricsAggregator.class); - - private static final String TEZ_CONTAINER_CONFIG = "hive.tez.container.size"; - private static final String MAP_CONTAINER_CONFIG = "mapreduce.map.memory.mb"; - private static final String REDUCER_CONTAINER_CONFIG = "mapreduce.reduce.memory.mb"; - private static final String REDUCER_SLOW_START_CONFIG = "mapreduce.job.reduce.slowstart.completedmaps"; - private static final long CONTAINER_MEMORY_DEFAULT_BYTES = 2048L * FileUtils.ONE_MB; + private static final Logger logger = Logger.getLogger(TezMetricsAggregator.class); - private HadoopAggregatedData _hadoopAggregatedData = null; - private TezTaskLevelAggregatedMetrics _mapTasks; - private TezTaskLevelAggregatedMetrics _reduceTasks; + private static final String TEZ_CONTAINER_CONFIG = "hive.tez.container.size"; + private static final String MAP_CONTAINER_CONFIG = "mapreduce.map.memory.mb"; + private static final String REDUCER_CONTAINER_CONFIG = "mapreduce.reduce.memory.mb"; + private static final String REDUCER_SLOW_START_CONFIG = "mapreduce.job.reduce.slowstart.completedmaps"; + private static final long CONTAINER_MEMORY_DEFAULT_BYTES = 2048L * FileUtils.ONE_MB; - private AggregatorConfigurationData _aggregatorConfigurationData; - - public TezMetricsAggregator(AggregatorConfigurationData _aggregatorConfigurationData) { - this._aggregatorConfigurationData = _aggregatorConfigurationData; - _hadoopAggregatedData = new HadoopAggregatedData(); - } + private HadoopAggregatedData _hadoopAggregatedData = null; + private TezTaskLevelAggregatedMetrics _mapTasks; + private TezTaskLevelAggregatedMetrics _reduceTasks; - @Override - public void aggregate(HadoopApplicationData hadoopData) { + private AggregatorConfigurationData _aggregatorConfigurationData; - TezApplicationData data = (TezApplicationData) hadoopData; + public TezMetricsAggregator(AggregatorConfigurationData _aggregatorConfigurationData) { + this._aggregatorConfigurationData = _aggregatorConfigurationData; + _hadoopAggregatedData = new HadoopAggregatedData(); + } - long mapTaskContainerSize = getMapContainerSize(data); - long reduceTaskContainerSize = getReducerContainerSize(data); + @Override + public void aggregate(HadoopApplicationData hadoopData) { - int reduceTaskSlowStartPercentage = - (int) (Double.parseDouble(data.getConf().getProperty(REDUCER_SLOW_START_CONFIG)) * 100); + TezApplicationData data = (TezApplicationData) hadoopData; + long mapTaskContainerSize = getMapContainerSize(data); + long reduceTaskContainerSize = getReducerContainerSize(data); - //overwrite reduceTaskSlowStartPercentage to 100%. TODO: make use of the slow start percent - reduceTaskSlowStartPercentage = 100; + int reduceTaskSlowStartPercentage = + (int) (Double.parseDouble(data.getConf().getProperty(REDUCER_SLOW_START_CONFIG)) * 100); - _mapTasks = new TezTaskLevelAggregatedMetrics(data.getMapTaskData(), mapTaskContainerSize, data.getStartTime()); - long reduceIdealStartTime = _mapTasks.getNthPercentileFinishTime(reduceTaskSlowStartPercentage); + //overwrite reduceTaskSlowStartPercentage to 100%. TODO: make use of the slow start percent + reduceTaskSlowStartPercentage = 100; - // Mappers list is empty - if(reduceIdealStartTime == -1) { - // ideal start time for reducer is infinite since it cannot start - reduceIdealStartTime = Long.MAX_VALUE; - } + _mapTasks = new TezTaskLevelAggregatedMetrics(data.getMapTaskData(), mapTaskContainerSize, data.getStartTime()); - _reduceTasks = new TezTaskLevelAggregatedMetrics(data.getReduceTaskData(), reduceTaskContainerSize, reduceIdealStartTime); + long reduceIdealStartTime = _mapTasks.getNthPercentileFinishTime(reduceTaskSlowStartPercentage); - _hadoopAggregatedData.setResourceUsed(_mapTasks.getResourceUsed() + _reduceTasks.getResourceUsed()); - _hadoopAggregatedData.setTotalDelay(_mapTasks.getDelay() + _reduceTasks.getDelay()); - _hadoopAggregatedData.setResourceWasted(_mapTasks.getResourceWasted() + _reduceTasks.getResourceWasted()); + // Mappers list is empty + if(reduceIdealStartTime == -1) { + // ideal start time for reducer is infinite since it cannot start + reduceIdealStartTime = Long.MAX_VALUE; } - @Override - public HadoopAggregatedData getResult() { - return _hadoopAggregatedData; + _reduceTasks = new TezTaskLevelAggregatedMetrics(data.getReduceTaskData(), reduceTaskContainerSize, reduceIdealStartTime); + + _hadoopAggregatedData.setResourceUsed(_mapTasks.getResourceUsed() + _reduceTasks.getResourceUsed()); + _hadoopAggregatedData.setTotalDelay(_mapTasks.getDelay() + _reduceTasks.getDelay()); + _hadoopAggregatedData.setResourceWasted(_mapTasks.getResourceWasted() + _reduceTasks.getResourceWasted()); + } + + @Override + public HadoopAggregatedData getResult() { + return _hadoopAggregatedData; + } + + private long getMapContainerSize(HadoopApplicationData data) { + try { + long mapContainerSize = Long.parseLong(data.getConf().getProperty(TEZ_CONTAINER_CONFIG)); + if (mapContainerSize > 0) + return mapContainerSize; + else + return Long.parseLong(data.getConf().getProperty(MAP_CONTAINER_CONFIG)); + } catch ( NumberFormatException ex) { + return CONTAINER_MEMORY_DEFAULT_BYTES; } - - private long getMapContainerSize(HadoopApplicationData data) { - try { - long mapContainerSize = Long.parseLong(data.getConf().getProperty(TEZ_CONTAINER_CONFIG)); - if (mapContainerSize > 0) - return mapContainerSize; - else - return Long.parseLong(data.getConf().getProperty(MAP_CONTAINER_CONFIG)); - } catch ( NumberFormatException ex) { - return CONTAINER_MEMORY_DEFAULT_BYTES; - } - } - - private long getReducerContainerSize(HadoopApplicationData data) { - try { - long reducerContainerSize = Long.parseLong(data.getConf().getProperty(TEZ_CONTAINER_CONFIG)); - if (reducerContainerSize > 0) - return reducerContainerSize; - else - return Long.parseLong(data.getConf().getProperty(REDUCER_CONTAINER_CONFIG)); - } catch ( NumberFormatException ex) { - return CONTAINER_MEMORY_DEFAULT_BYTES; - } + } + + private long getReducerContainerSize(HadoopApplicationData data) { + try { + long reducerContainerSize = Long.parseLong(data.getConf().getProperty(TEZ_CONTAINER_CONFIG)); + if (reducerContainerSize > 0) + return reducerContainerSize; + else + return Long.parseLong(data.getConf().getProperty(REDUCER_CONTAINER_CONFIG)); + } catch ( NumberFormatException ex) { + return CONTAINER_MEMORY_DEFAULT_BYTES; } + } } diff --git a/app/com/linkedin/drelephant/tez/TezTaskLevelAggregatedMetrics.java b/app/com/linkedin/drelephant/tez/TezTaskLevelAggregatedMetrics.java index 65a7f27b4..0f5eaa5f6 100644 --- a/app/com/linkedin/drelephant/tez/TezTaskLevelAggregatedMetrics.java +++ b/app/com/linkedin/drelephant/tez/TezTaskLevelAggregatedMetrics.java @@ -1,3 +1,18 @@ +/* + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + * + */ package com.linkedin.drelephant.tez; import com.linkedin.drelephant.tez.data.TezCounterData; @@ -14,126 +29,126 @@ public class TezTaskLevelAggregatedMetrics { - private static final Logger logger = Logger.getLogger(TezTaskLevelAggregatedMetrics.class); - - private long _delay = 0; - private long _resourceWasted = 0; - private long _resourceUsed = 0; - - private List finishTimes = new ArrayList(); - private List durations = new ArrayList(); - - private static final double MEMORY_BUFFER = 1.5; - private static final double CLUSTER_MEMORY_FACTOR = 2.1; - - /** - * Returns the nth percentile finish job - * @param percentile The percentile of finish job to return - * @return The nth percentile finish job - */ - public long getNthPercentileFinishTime(int percentile) - { - if(finishTimes == null || finishTimes.size() == 0 ) { - return -1; - } - return Statistics.percentile(finishTimes, percentile); - } + private static final Logger logger = Logger.getLogger(TezTaskLevelAggregatedMetrics.class); - /** - * Constructor for TaskLevelAggregatedMetrics - * @param taskData Array containing the task data for mappers and/or reducers - * @param containerSize The container size of the tasks - * @param idealStartTime The ideal start time for the task. For mappers it is the submit time, for - * reducers, it is the time when the number of completed maps become more than - * the slow start time. - */ - public TezTaskLevelAggregatedMetrics(TezTaskData[] taskData, long containerSize, long idealStartTime) { - compute(taskData, containerSize, idealStartTime); - } + private long _delay = 0; + private long _resourceWasted = 0; + private long _resourceUsed = 0; + + private List finishTimes = new ArrayList(); + private List durations = new ArrayList(); - /** - * Returns the overall delay for the tasks. - * @return The delay of the tasks. - */ - public long getDelay() { - return _delay; + private static final double MEMORY_BUFFER = 1.5; + private static final double CLUSTER_MEMORY_FACTOR = 2.1; + + /** + * Returns the nth percentile finish job + * @param percentile The percentile of finish job to return + * @return The nth percentile finish job + */ + public long getNthPercentileFinishTime(int percentile) + { + if(finishTimes == null || finishTimes.size() == 0 ) { + return -1; + } + return Statistics.percentile(finishTimes, percentile); + } + + /** + * Constructor for TaskLevelAggregatedMetrics + * @param taskData Array containing the task data for mappers and/or reducers + * @param containerSize The container size of the tasks + * @param idealStartTime The ideal start time for the task. For mappers it is the submit time, for + * reducers, it is the time when the number of completed maps become more than + * the slow start time. + */ + public TezTaskLevelAggregatedMetrics(TezTaskData[] taskData, long containerSize, long idealStartTime) { + compute(taskData, containerSize, idealStartTime); + } + + /** + * Returns the overall delay for the tasks. + * @return The delay of the tasks. + */ + public long getDelay() { + return _delay; + } + + /** + * Retruns the resources wasted by all the tasks in MB Seconds + * @return The wasted resources of all the tasks in MB Seconds + */ + public long getResourceWasted() { + return _resourceWasted; + } + + /** + * Returns the resource used by all the tasks in MB Seconds + * @return The total resources used by all tasks in MB Seconds + */ + public long getResourceUsed() { + return _resourceUsed; + } + + /** + * Computes the aggregated metrics -> peakMemory, delay, total task duration, wasted resources and memory usage. + * @param taskDatas + * @param containerSize + * @param idealStartTime + */ + private void compute(TezTaskData[] taskDatas, long containerSize, long idealStartTime) { + + long peakMemoryNeed = 0; + long taskFinishTimeMax = 0; + long taskDurationMax = 0; + + // if there are zero tasks, then nothing to compute. + if(taskDatas == null || taskDatas.length == 0) { + return; } - /** - * Retruns the resources wasted by all the tasks in MB Seconds - * @return The wasted resources of all the tasks in MB Seconds - */ - public long getResourceWasted() { - return _resourceWasted; + for (TezTaskData taskData: taskDatas) { + if (!taskData.isSampled()) { + continue; + } + long taskMemory = taskData.getCounters().get(TezCounterData.CounterName.PHYSICAL_MEMORY_BYTES)/ FileUtils.ONE_MB; // MB + long taskVM = taskData.getCounters().get(TezCounterData.CounterName.VIRTUAL_MEMORY_BYTES)/ FileUtils.ONE_MB; // MB + long taskDuration = taskData.getFinishTime() - taskData.getStartTime(); // Milliseconds + long taskCost = (containerSize) * (taskDuration / Statistics.SECOND_IN_MS); // MB Seconds + + durations.add(taskDuration); + finishTimes.add(taskData.getFinishTime()); + + //peak Memory usage + long memoryRequiredForVM = (long) (taskVM/CLUSTER_MEMORY_FACTOR); + long biggerMemoryRequirement = memoryRequiredForVM > taskMemory ? memoryRequiredForVM : taskMemory; + peakMemoryNeed = biggerMemoryRequirement > peakMemoryNeed ? biggerMemoryRequirement : peakMemoryNeed; + + if(taskFinishTimeMax < taskData.getFinishTime()) { + taskFinishTimeMax = taskData.getFinishTime(); + } + + if(taskDurationMax < taskDuration) { + taskDurationMax = taskDuration; + } + _resourceUsed += taskCost; } - /** - * Returns the resource used by all the tasks in MB Seconds - * @return The total resources used by all tasks in MB Seconds - */ - public long getResourceUsed() { - return _resourceUsed; + // Compute the delay in starting the task. + _delay = taskFinishTimeMax - (idealStartTime + taskDurationMax); + + // invalid delay + if(_delay < 0) { + _delay = 0; } - /** - * Computes the aggregated metrics -> peakMemory, delay, total task duration, wasted resources and memory usage. - * @param taskDatas - * @param containerSize - * @param idealStartTime - */ - private void compute(TezTaskData[] taskDatas, long containerSize, long idealStartTime) { - - long peakMemoryNeed = 0; - long taskFinishTimeMax = 0; - long taskDurationMax = 0; - - // if there are zero tasks, then nothing to compute. - if(taskDatas == null || taskDatas.length == 0) { - return; - } - - for (TezTaskData taskData: taskDatas) { - if (!taskData.isSampled()) { - continue; - } - long taskMemory = taskData.getCounters().get(TezCounterData.CounterName.PHYSICAL_MEMORY_BYTES)/ FileUtils.ONE_MB; // MB - long taskVM = taskData.getCounters().get(TezCounterData.CounterName.VIRTUAL_MEMORY_BYTES)/ FileUtils.ONE_MB; // MB - long taskDuration = taskData.getFinishTime() - taskData.getStartTime(); // Milliseconds - long taskCost = (containerSize) * (taskDuration / Statistics.SECOND_IN_MS); // MB Seconds - - durations.add(taskDuration); - finishTimes.add(taskData.getFinishTime()); - - //peak Memory usage - long memoryRequiredForVM = (long) (taskVM/CLUSTER_MEMORY_FACTOR); - long biggerMemoryRequirement = memoryRequiredForVM > taskMemory ? memoryRequiredForVM : taskMemory; - peakMemoryNeed = biggerMemoryRequirement > peakMemoryNeed ? biggerMemoryRequirement : peakMemoryNeed; - - if(taskFinishTimeMax < taskData.getFinishTime()) { - taskFinishTimeMax = taskData.getFinishTime(); - } - - if(taskDurationMax < taskDuration) { - taskDurationMax = taskDuration; - } - _resourceUsed += taskCost; - } - - // Compute the delay in starting the task. - _delay = taskFinishTimeMax - (idealStartTime + taskDurationMax); - - // invalid delay - if(_delay < 0) { - _delay = 0; - } - - // wastedResources - long wastedMemory = containerSize - (long) (peakMemoryNeed * MEMORY_BUFFER); - if(wastedMemory > 0) { - for (long duration : durations) { - _resourceWasted += (wastedMemory) * (duration / Statistics.SECOND_IN_MS); // MB Seconds - } - } + // wastedResources + long wastedMemory = containerSize - (long) (peakMemoryNeed * MEMORY_BUFFER); + if(wastedMemory > 0) { + for (long duration : durations) { + _resourceWasted += (wastedMemory) * (duration / Statistics.SECOND_IN_MS); // MB Seconds + } } + } } \ No newline at end of file diff --git a/app/com/linkedin/drelephant/tez/data/TezApplicationData.java b/app/com/linkedin/drelephant/tez/data/TezApplicationData.java index 0d4547746..19cd1d9e2 100644 --- a/app/com/linkedin/drelephant/tez/data/TezApplicationData.java +++ b/app/com/linkedin/drelephant/tez/data/TezApplicationData.java @@ -1,3 +1,18 @@ +/* + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + * + */ package com.linkedin.drelephant.tez.data; import com.linkedin.drelephant.analysis.ApplicationType; @@ -10,113 +25,113 @@ */ public class TezApplicationData implements HadoopApplicationData { - private static final ApplicationType APPLICATION_TYPE = new ApplicationType("TEZ"); - - private String _appId = ""; - private Properties _conf; - private boolean _succeeded = true; - private TezTaskData[] _redudeTasks; - private TezTaskData[] _mapTasks; - private TezCounterData _counterHolder; - - private long _submitTime = 0; - private long _startTime = 0; - private long _finishTime = 0; - - public boolean getSucceeded() { - return _succeeded; - } - - @Override - public String getAppId() { - return _appId; - } - - @Override - public Properties getConf() { - return _conf; - } - - @Override - public ApplicationType getApplicationType() { - return APPLICATION_TYPE; - } - - @Override - public boolean isEmpty() { - return _succeeded && getMapTaskData().length == 0 && getReduceTaskData().length == 0; - } - - public TezTaskData[] getReduceTaskData() { - return _redudeTasks; - } - - public TezTaskData[] getMapTaskData() { - return _mapTasks; - } - - public long getSubmitTime() { - return _submitTime; - } - - public long getStartTime() { - return _startTime; - } - - public long getFinishTime() { - return _finishTime; - } - - public TezCounterData getCounters() { - return _counterHolder; - } - - public TezApplicationData setCounters(TezCounterData counterHolder) { - this._counterHolder = counterHolder; - return this; - } - - public TezApplicationData setAppId(String appId) { - this._appId = appId; - return this; - } - - public TezApplicationData setConf(Properties conf) { - this._conf = conf; - return this; - } - - public TezApplicationData setSucceeded(boolean succeeded) { - this._succeeded = succeeded; - return this; - } - - public TezApplicationData setReduceTaskData(TezTaskData[] reduceTasks) { - this._redudeTasks = reduceTasks; - return this; - } - - public TezApplicationData setMapTaskData(TezTaskData[] mapTasks) { - this._mapTasks = mapTasks; - return this; - } - - public TezApplicationData setSubmitTime(long submitTime) { - this._submitTime = submitTime; - return this; - } - - public TezApplicationData setStartTime(long startTime) { - this._startTime = startTime; - return this; - } - - public TezApplicationData setFinishTime(long finishTime) { - this._finishTime = finishTime; - return this; - } - - public String toString(){ - return APPLICATION_TYPE.toString() + " " + _appId; - } + private static final ApplicationType APPLICATION_TYPE = new ApplicationType("TEZ"); + + private String _appId = ""; + private Properties _conf; + private boolean _succeeded = true; + private TezTaskData[] _redudeTasks; + private TezTaskData[] _mapTasks; + private TezCounterData _counterHolder; + + private long _submitTime = 0; + private long _startTime = 0; + private long _finishTime = 0; + + public boolean getSucceeded() { + return _succeeded; + } + + @Override + public String getAppId() { + return _appId; + } + + @Override + public Properties getConf() { + return _conf; + } + + @Override + public ApplicationType getApplicationType() { + return APPLICATION_TYPE; + } + + @Override + public boolean isEmpty() { + return _succeeded && getMapTaskData().length == 0 && getReduceTaskData().length == 0; + } + + public TezTaskData[] getReduceTaskData() { + return _redudeTasks; + } + + public TezTaskData[] getMapTaskData() { + return _mapTasks; + } + + public long getSubmitTime() { + return _submitTime; + } + + public long getStartTime() { + return _startTime; + } + + public long getFinishTime() { + return _finishTime; + } + + public TezCounterData getCounters() { + return _counterHolder; + } + + public TezApplicationData setCounters(TezCounterData counterHolder) { + this._counterHolder = counterHolder; + return this; + } + + public TezApplicationData setAppId(String appId) { + this._appId = appId; + return this; + } + + public TezApplicationData setConf(Properties conf) { + this._conf = conf; + return this; + } + + public TezApplicationData setSucceeded(boolean succeeded) { + this._succeeded = succeeded; + return this; + } + + public TezApplicationData setReduceTaskData(TezTaskData[] reduceTasks) { + this._redudeTasks = reduceTasks; + return this; + } + + public TezApplicationData setMapTaskData(TezTaskData[] mapTasks) { + this._mapTasks = mapTasks; + return this; + } + + public TezApplicationData setSubmitTime(long submitTime) { + this._submitTime = submitTime; + return this; + } + + public TezApplicationData setStartTime(long startTime) { + this._startTime = startTime; + return this; + } + + public TezApplicationData setFinishTime(long finishTime) { + this._finishTime = finishTime; + return this; + } + + public String toString(){ + return APPLICATION_TYPE.toString() + " " + _appId; + } } \ No newline at end of file diff --git a/app/com/linkedin/drelephant/tez/data/TezCounterData.java b/app/com/linkedin/drelephant/tez/data/TezCounterData.java index 6e53e0744..9aac00117 100644 --- a/app/com/linkedin/drelephant/tez/data/TezCounterData.java +++ b/app/com/linkedin/drelephant/tez/data/TezCounterData.java @@ -1,3 +1,18 @@ +/* + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + * + */ package com.linkedin.drelephant.tez.data; import java.util.Collections; @@ -9,172 +24,172 @@ * Tez Counter Data defining data structure. */ public class TezCounterData { - // Map to group counters into DAG, Task and application levels. - private final Map> _pubCounters; - - public String toString() { - return _pubCounters.toString(); + // Map to group counters into DAG, Task and application levels. + private final Map> _pubCounters; + + public String toString() { + return _pubCounters.toString(); + } + + public TezCounterData() { + _pubCounters = new HashMap>(8); + } + + public long get(CounterName counterName) { + for(Map counterGrp : _pubCounters.values()) { + if(counterGrp.containsKey(counterName._name)) { + return counterGrp.get(counterName._name); + } } - - public TezCounterData() { - _pubCounters = new HashMap>(8); + return 0; + } + + public void set(CounterName counterName, long value) { + set(counterName.getGroupName(), counterName.getName(), value); + } + + public void set(String groupName, String counterName, long value) { + Map counterMap = _pubCounters.get(groupName); + if (counterMap == null) { + counterMap = new HashMap(4); + _pubCounters.put(groupName, counterMap); } - - public long get(CounterName counterName) { - for(Map counterGrp : _pubCounters.values()) { - if(counterGrp.containsKey(counterName._name)) { - return counterGrp.get(counterName._name); - } - } - return 0; + counterMap.put(counterName, value); + } + + public Set getGroupNames() { + Set groupNames = _pubCounters.keySet(); + return Collections.unmodifiableSet(groupNames); + } + + public Map getAllCountersInGroup(String groupName) { + Map counterMap = _pubCounters.get(groupName); + if (counterMap == null) { + counterMap = new HashMap(1); } + return counterMap; + } - public void set(CounterName counterName, long value) { - set(counterName.getGroupName(), counterName.getName(), value); - } + public static enum GroupName { + FileSystemCounters("org.apache.tez.common.counters.FileSystemCounter"), + TezTask("org.apache.tez.common.counters.TaskCounter"), + TezDag("org.apache.tez.common.counters.DAGCounter"); - public void set(String groupName, String counterName, long value) { - Map counterMap = _pubCounters.get(groupName); - if (counterMap == null) { - counterMap = new HashMap(4); - _pubCounters.put(groupName, counterMap); - } - counterMap.put(counterName, value); + + String _name; + GroupName(String name) { + _name = name; + } + } + + public static enum CounterName { + + NUM_SUCCEEDED_TASKS(GroupName.TezDag, "NUM_SUCCEEDED_TASKS", "NUM_SUCCEEDED_TASKS"), + TOTAL_LAUNCHED_TASKS(GroupName.TezDag, "TOTAL_LAUNCHED_TASKS", "TOTAL_LAUNCHED_TASKS"), + RACK_LOCAL_TASKS(GroupName.TezDag, "RACK_LOCAL_TASKS", "RACK_LOCAL_TASKS"), + AM_CPU_MILLISECONDS(GroupName.TezDag, "AM_CPU_MILLISECONDS", "AM_CPU_MILLISECONDS"), + AM_GC_TIME_MILLIS(GroupName.TezDag, "AM_GC_TIME_MILLIS", "AM_GC_TIME_MILLIS"), + + FILE_BYTES_READ(GroupName.FileSystemCounters, "FILE_BYTES_READ", "FILE_BYTES_READ"), + FILE_BYTES_WRITTEN(GroupName.FileSystemCounters, "FILE_BYTES_WRITTEN", "FILE_BYTES_WRITTEN"), + FILE_READ_OPS(GroupName.FileSystemCounters, "FILE_READ_OPS", "FILE_READ_OPS"), + FILE_LARGE_READ_OPS(GroupName.FileSystemCounters, "FILE_LARGE_READ_OPS", "FILE_LARGE_READ_OPS"), + FILE_WRITE_OPS(GroupName.FileSystemCounters, "FILE_WRITE_OPS", "FILE_WRITE_OPS"), + HDFS_BYTES_READ(GroupName.FileSystemCounters, "HDFS_BYTES_READ", "HDFS_BYTES_READ"), + HDFS_BYTES_WRITTEN(GroupName.FileSystemCounters, "HDFS_BYTES_WRITTEN", "HDFS_BYTES_WRITTEN"), + HDFS_READ_OPS(GroupName.FileSystemCounters, "HDFS_READ_OPS", "HDFS_READ_OPS"), + HDFS_LARGE_READ_OPS(GroupName.FileSystemCounters, "HDFS_LARGE_READ_OPS", "HDFS_LARGE_READ_OPS"), + HDFS_WRITE_OPS(GroupName.FileSystemCounters, "HDFS_WRITE_OPS", "HDFS_WRITE_OPS"), + S3A_BYTES_READ(GroupName.FileSystemCounters, "S3A_BYTES_READ", "S3A_BYTES_READ"), + S3A_BYTES_WRITTEN(GroupName.FileSystemCounters, "S3A_BYTES_WRITTEN", "S3A_BYTES_WRITTEN"), + S3A_READ_OPS(GroupName.FileSystemCounters, "S3A_READ_OPS", "S3A_READ_OPS"), + S3A_LARGE_READ_OPS(GroupName.FileSystemCounters, "S3A_LARGE_READ_OPS", "S3A_LARGE_READ_OPS"), + S3A_WRITE_OPS(GroupName.FileSystemCounters, "S3A_WRITE_OPS", "S3_WRITE_OPS"), + S3N_BYTES_READ(GroupName.FileSystemCounters, "S3N_BYTES_READ", "S3N_BYTES_READ"), + S3N_BYTES_WRITTEN(GroupName.FileSystemCounters, "S3N_BYTES_WRITTEN", "S3N_BYTES_WRITTEN"), + S3N_READ_OPS(GroupName.FileSystemCounters, "S3N_READ_OPS", "S3N_READ_OPS"), + S3N_LARGE_READ_OPS(GroupName.FileSystemCounters, "S3N_LARGE_READ_OPS", "S3N_LARGE_READ_OPS"), + S3N_WRITE_OPS(GroupName.FileSystemCounters, "S3N_WRITE_OPS", "S3N_WRITE_OPS"), + + REDUCE_INPUT_GROUPS(GroupName.TezTask, "REDUCE_INPUT_GROUPS", "REDUCE_INPUT_GROUPS"), + REDUCE_INPUT_RECORDS(GroupName.TezTask, "REDUCE_INPUT_RECORDS", "REDUCE_INPUT_RECORDS"), + COMBINE_INPUT_RECORDS(GroupName.TezTask, "COMBINE_INPUT_RECORDS", "COMBINE_INPUT_RECORDS"), + SPILLED_RECORDS(GroupName.TezTask, "SPILLED_RECORDS", "SPILLED_RECORDS"), + NUM_SHUFFLED_INPUTS(GroupName.TezTask, "NUM_SHUFFLED_INPUTS", "NUM_SHUFFLED_INPUTS"), + NUM_SKIPPED_INPUTS(GroupName.TezTask, "NUM_SKIPPED_INPUTS", "NUM_SKIPPED_INPUTS"), + NUM_FAILED_SHUFFLE_INPUTS(GroupName.TezTask, "NUM_FAILED_SHUFFLE_INPUTS", "NUM_FAILED_SHUFFLE_INPUTS"), + MERGED_MAP_OUTPUTS(GroupName.TezTask, "MERGED_MAP_OUTPUTS", "MERGED_MAP_OUTPUTS"), + GC_TIME_MILLIS(GroupName.TezTask, "GC_TIME_MILLIS", "GC_TIME_MILLIS"), + COMMITTED_HEAP_BYTES(GroupName.TezTask, "COMMITTED_HEAP_BYTES", "COMMITTED_HEAP_BYTES"), + INPUT_RECORDS_PROCESSED(GroupName.TezTask, "INPUT_RECORDS_PROCESSED", "INPUT_RECORDS_PROCESSED"), + OUTPUT_RECORDS(GroupName.TezTask, "OUTPUT_RECORDS", "OUTPUT_RECORDS"), + OUTPUT_BYTES(GroupName.TezTask, "OUTPUT_BYTES", "OUTPUT_BYTES"), + OUTPUT_BYTES_WITH_OVERHEAD(GroupName.TezTask, "OUTPUT_BYTES_WITH_OVERHEAD", "OUTPUT_BYTES_WITH_OVERHEAD"), + OUTPUT_BYTES_PHYSICAL(GroupName.TezTask, "OUTPUT_BYTES_PHYSICAL", "OUTPUT_BYTES_PHYSICAL"), + ADDITIONAL_SPILLS_BYTES_WRITTEN(GroupName.TezTask, "ADDITIONAL_SPILLS_BYTES_WRITTEN", "ADDITIONAL_SPILLS_BYTES_WRITTEN"), + ADDITIONAL_SPILLS_BYTES_READ(GroupName.TezTask, "ADDITIONAL_SPILLS_BYTES_READ", "ADDITIONAL_SPILLS_BYTES_READ"), + ADDITIONAL_SPILL_COUNT(GroupName.TezTask, "ADDITIONAL_SPILL_COUNT", "ADDITIONAL_SPILL_COUNT"), + SHUFFLE_BYTES(GroupName.TezTask, "SHUFFLE_BYTES", "SHUFFLE_BYTES"), + SHUFFLE_BYTES_DECOMPRESSED(GroupName.TezTask, "SHUFFLE_BYTES_DECOMPRESSED", "SHUFFLE_BYTES_DECOMPRESSED"), + SHUFFLE_BYTES_TO_MEM(GroupName.TezTask, "SHUFFLE_BYTES_TO_MEM", "SHUFFLE_BYTES_TO_MEM"), + SHUFFLE_BYTES_TO_DISK(GroupName.TezTask, "SHUFFLE_BYTES_TO_DISK", "SHUFFLE_BYTES_TO_DISK"), + SHUFFLE_BYTES_DISK_DIRECT(GroupName.TezTask, "SHUFFLE_BYTES_DISK_DIRECT", "SHUFFLE_BYTES_DISK_DIRECT"), + NUM_MEM_TO_DISK_MERGES(GroupName.TezTask, "NUM_MEM_TO_DISK_MERGES", "NUM_MEM_TO_DISK_MERGES"), + CPU_MILLISECONDS(GroupName.TezTask,"CPU_MILLISECONDS","CPU_MILLISECONDS"), + PHYSICAL_MEMORY_BYTES(GroupName.TezTask,"PHYSICAL_MEMORY_BYTES","PHYSICAL_MEMORY_BYTES"), + VIRTUAL_MEMORY_BYTES(GroupName.TezTask,"VIRTUAL_MEMORY_BYTES","VIRTUAL_MEMORY_BYTES"), + NUM_DISK_TO_DISK_MERGES(GroupName.TezTask, "NUM_DISK_TO_DISK_MERGES", "NUM_DISK_TO_DISK_MERGES"), + SHUFFLE_PHASE_TIME(GroupName.TezTask, "SHUFFLE_PHASE_TIME", "SHUFFLE_PHASE_TIME"), + MERGE_PHASE_TIME(GroupName.TezTask, "MERGE_PHASE_TIME", "MERGE_PHASE_TIME"), + FIRST_EVENT_RECEIVED(GroupName.TezTask, "FIRST_EVENT_RECEIVED", "FIRST_EVENT_RECEIVED"), + LAST_EVENT_RECEIVED(GroupName.TezTask, "LAST_EVENT_RECEIVED", "LAST_EVENT_RECEIVED"); + + + GroupName _group; + String _name; + String _displayName; + + CounterName(GroupName group, String name, String displayName) { + this._group = group; + this._name = name; + this._displayName = displayName; } - public Set getGroupNames() { - Set groupNames = _pubCounters.keySet(); - return Collections.unmodifiableSet(groupNames); + static Map _counterDisplayNameMap; + static Map _counterNameMap; + static { + _counterDisplayNameMap = new HashMap(); + _counterNameMap = new HashMap(); + for (CounterName cn : CounterName.values()) { + _counterDisplayNameMap.put(cn._displayName, cn); + _counterNameMap.put(cn._name, cn); + } } - public Map getAllCountersInGroup(String groupName) { - Map counterMap = _pubCounters.get(groupName); - if (counterMap == null) { - counterMap = new HashMap(1); - } - return counterMap; + public static CounterName getCounterFromName(String name) { + if (_counterNameMap.containsKey(name)) { + return _counterNameMap.get(name); + } + return null; } - public static enum GroupName { - FileSystemCounters("org.apache.tez.common.counters.FileSystemCounter"), - TezTask("org.apache.tez.common.counters.TaskCounter"), - TezDag("org.apache.tez.common.counters.DAGCounter"); + public static CounterName getCounterFromDisplayName(String displayName) { + if (_counterDisplayNameMap.containsKey(displayName)) { + return _counterDisplayNameMap.get(displayName); + } + return null; + } + public String getName() { + return _name; + } - String _name; - GroupName(String name) { - _name = name; - } + public String getDisplayName() { + return _displayName; } - public static enum CounterName { - - NUM_SUCCEEDED_TASKS(GroupName.TezDag, "NUM_SUCCEEDED_TASKS", "NUM_SUCCEEDED_TASKS"), - TOTAL_LAUNCHED_TASKS(GroupName.TezDag, "TOTAL_LAUNCHED_TASKS", "TOTAL_LAUNCHED_TASKS"), - RACK_LOCAL_TASKS(GroupName.TezDag, "RACK_LOCAL_TASKS", "RACK_LOCAL_TASKS"), - AM_CPU_MILLISECONDS(GroupName.TezDag, "AM_CPU_MILLISECONDS", "AM_CPU_MILLISECONDS"), - AM_GC_TIME_MILLIS(GroupName.TezDag, "AM_GC_TIME_MILLIS", "AM_GC_TIME_MILLIS"), - - FILE_BYTES_READ(GroupName.FileSystemCounters, "FILE_BYTES_READ", "FILE_BYTES_READ"), - FILE_BYTES_WRITTEN(GroupName.FileSystemCounters, "FILE_BYTES_WRITTEN", "FILE_BYTES_WRITTEN"), - FILE_READ_OPS(GroupName.FileSystemCounters, "FILE_READ_OPS", "FILE_READ_OPS"), - FILE_LARGE_READ_OPS(GroupName.FileSystemCounters, "FILE_LARGE_READ_OPS", "FILE_LARGE_READ_OPS"), - FILE_WRITE_OPS(GroupName.FileSystemCounters, "FILE_WRITE_OPS", "FILE_WRITE_OPS"), - HDFS_BYTES_READ(GroupName.FileSystemCounters, "HDFS_BYTES_READ", "HDFS_BYTES_READ"), - HDFS_BYTES_WRITTEN(GroupName.FileSystemCounters, "HDFS_BYTES_WRITTEN", "HDFS_BYTES_WRITTEN"), - HDFS_READ_OPS(GroupName.FileSystemCounters, "HDFS_READ_OPS", "HDFS_READ_OPS"), - HDFS_LARGE_READ_OPS(GroupName.FileSystemCounters, "HDFS_LARGE_READ_OPS", "HDFS_LARGE_READ_OPS"), - HDFS_WRITE_OPS(GroupName.FileSystemCounters, "HDFS_WRITE_OPS", "HDFS_WRITE_OPS"), - S3A_BYTES_READ(GroupName.FileSystemCounters, "S3A_BYTES_READ", "S3A_BYTES_READ"), - S3A_BYTES_WRITTEN(GroupName.FileSystemCounters, "S3A_BYTES_WRITTEN", "S3A_BYTES_WRITTEN"), - S3A_READ_OPS(GroupName.FileSystemCounters, "S3A_READ_OPS", "S3A_READ_OPS"), - S3A_LARGE_READ_OPS(GroupName.FileSystemCounters, "S3A_LARGE_READ_OPS", "S3A_LARGE_READ_OPS"), - S3A_WRITE_OPS(GroupName.FileSystemCounters, "S3A_WRITE_OPS", "S3_WRITE_OPS"), - S3N_BYTES_READ(GroupName.FileSystemCounters, "S3N_BYTES_READ", "S3N_BYTES_READ"), - S3N_BYTES_WRITTEN(GroupName.FileSystemCounters, "S3N_BYTES_WRITTEN", "S3N_BYTES_WRITTEN"), - S3N_READ_OPS(GroupName.FileSystemCounters, "S3N_READ_OPS", "S3N_READ_OPS"), - S3N_LARGE_READ_OPS(GroupName.FileSystemCounters, "S3N_LARGE_READ_OPS", "S3N_LARGE_READ_OPS"), - S3N_WRITE_OPS(GroupName.FileSystemCounters, "S3N_WRITE_OPS", "S3N_WRITE_OPS"), - - REDUCE_INPUT_GROUPS(GroupName.TezTask, "REDUCE_INPUT_GROUPS", "REDUCE_INPUT_GROUPS"), - REDUCE_INPUT_RECORDS(GroupName.TezTask, "REDUCE_INPUT_RECORDS", "REDUCE_INPUT_RECORDS"), - COMBINE_INPUT_RECORDS(GroupName.TezTask, "COMBINE_INPUT_RECORDS", "COMBINE_INPUT_RECORDS"), - SPILLED_RECORDS(GroupName.TezTask, "SPILLED_RECORDS", "SPILLED_RECORDS"), - NUM_SHUFFLED_INPUTS(GroupName.TezTask, "NUM_SHUFFLED_INPUTS", "NUM_SHUFFLED_INPUTS"), - NUM_SKIPPED_INPUTS(GroupName.TezTask, "NUM_SKIPPED_INPUTS", "NUM_SKIPPED_INPUTS"), - NUM_FAILED_SHUFFLE_INPUTS(GroupName.TezTask, "NUM_FAILED_SHUFFLE_INPUTS", "NUM_FAILED_SHUFFLE_INPUTS"), - MERGED_MAP_OUTPUTS(GroupName.TezTask, "MERGED_MAP_OUTPUTS", "MERGED_MAP_OUTPUTS"), - GC_TIME_MILLIS(GroupName.TezTask, "GC_TIME_MILLIS", "GC_TIME_MILLIS"), - COMMITTED_HEAP_BYTES(GroupName.TezTask, "COMMITTED_HEAP_BYTES", "COMMITTED_HEAP_BYTES"), - INPUT_RECORDS_PROCESSED(GroupName.TezTask, "INPUT_RECORDS_PROCESSED", "INPUT_RECORDS_PROCESSED"), - OUTPUT_RECORDS(GroupName.TezTask, "OUTPUT_RECORDS", "OUTPUT_RECORDS"), - OUTPUT_BYTES(GroupName.TezTask, "OUTPUT_BYTES", "OUTPUT_BYTES"), - OUTPUT_BYTES_WITH_OVERHEAD(GroupName.TezTask, "OUTPUT_BYTES_WITH_OVERHEAD", "OUTPUT_BYTES_WITH_OVERHEAD"), - OUTPUT_BYTES_PHYSICAL(GroupName.TezTask, "OUTPUT_BYTES_PHYSICAL", "OUTPUT_BYTES_PHYSICAL"), - ADDITIONAL_SPILLS_BYTES_WRITTEN(GroupName.TezTask, "ADDITIONAL_SPILLS_BYTES_WRITTEN", "ADDITIONAL_SPILLS_BYTES_WRITTEN"), - ADDITIONAL_SPILLS_BYTES_READ(GroupName.TezTask, "ADDITIONAL_SPILLS_BYTES_READ", "ADDITIONAL_SPILLS_BYTES_READ"), - ADDITIONAL_SPILL_COUNT(GroupName.TezTask, "ADDITIONAL_SPILL_COUNT", "ADDITIONAL_SPILL_COUNT"), - SHUFFLE_BYTES(GroupName.TezTask, "SHUFFLE_BYTES", "SHUFFLE_BYTES"), - SHUFFLE_BYTES_DECOMPRESSED(GroupName.TezTask, "SHUFFLE_BYTES_DECOMPRESSED", "SHUFFLE_BYTES_DECOMPRESSED"), - SHUFFLE_BYTES_TO_MEM(GroupName.TezTask, "SHUFFLE_BYTES_TO_MEM", "SHUFFLE_BYTES_TO_MEM"), - SHUFFLE_BYTES_TO_DISK(GroupName.TezTask, "SHUFFLE_BYTES_TO_DISK", "SHUFFLE_BYTES_TO_DISK"), - SHUFFLE_BYTES_DISK_DIRECT(GroupName.TezTask, "SHUFFLE_BYTES_DISK_DIRECT", "SHUFFLE_BYTES_DISK_DIRECT"), - NUM_MEM_TO_DISK_MERGES(GroupName.TezTask, "NUM_MEM_TO_DISK_MERGES", "NUM_MEM_TO_DISK_MERGES"), - CPU_MILLISECONDS(GroupName.TezTask,"CPU_MILLISECONDS","CPU_MILLISECONDS"), - PHYSICAL_MEMORY_BYTES(GroupName.TezTask,"PHYSICAL_MEMORY_BYTES","PHYSICAL_MEMORY_BYTES"), - VIRTUAL_MEMORY_BYTES(GroupName.TezTask,"VIRTUAL_MEMORY_BYTES","VIRTUAL_MEMORY_BYTES"), - NUM_DISK_TO_DISK_MERGES(GroupName.TezTask, "NUM_DISK_TO_DISK_MERGES", "NUM_DISK_TO_DISK_MERGES"), - SHUFFLE_PHASE_TIME(GroupName.TezTask, "SHUFFLE_PHASE_TIME", "SHUFFLE_PHASE_TIME"), - MERGE_PHASE_TIME(GroupName.TezTask, "MERGE_PHASE_TIME", "MERGE_PHASE_TIME"), - FIRST_EVENT_RECEIVED(GroupName.TezTask, "FIRST_EVENT_RECEIVED", "FIRST_EVENT_RECEIVED"), - LAST_EVENT_RECEIVED(GroupName.TezTask, "LAST_EVENT_RECEIVED", "LAST_EVENT_RECEIVED"); - - - GroupName _group; - String _name; - String _displayName; - - CounterName(GroupName group, String name, String displayName) { - this._group = group; - this._name = name; - this._displayName = displayName; - } - - static Map _counterDisplayNameMap; - static Map _counterNameMap; - static { - _counterDisplayNameMap = new HashMap(); - _counterNameMap = new HashMap(); - for (CounterName cn : CounterName.values()) { - _counterDisplayNameMap.put(cn._displayName, cn); - _counterNameMap.put(cn._name, cn); - } - } - - public static CounterName getCounterFromName(String name) { - if (_counterNameMap.containsKey(name)) { - return _counterNameMap.get(name); - } - return null; - } - - public static CounterName getCounterFromDisplayName(String displayName) { - if (_counterDisplayNameMap.containsKey(displayName)) { - return _counterDisplayNameMap.get(displayName); - } - return null; - } - - public String getName() { - return _name; - } - - public String getDisplayName() { - return _displayName; - } - - public String getGroupName() { - return _group.name(); - } + public String getGroupName() { + return _group.name(); } + } } \ No newline at end of file diff --git a/app/com/linkedin/drelephant/tez/data/TezTaskData.java b/app/com/linkedin/drelephant/tez/data/TezTaskData.java index 1e170a642..eb6aa832c 100644 --- a/app/com/linkedin/drelephant/tez/data/TezTaskData.java +++ b/app/com/linkedin/drelephant/tez/data/TezTaskData.java @@ -1,3 +1,18 @@ +/* + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + * + */ package com.linkedin.drelephant.tez.data; /** @@ -5,118 +20,118 @@ */ public class TezTaskData { - private TezCounterData _counterHolder; - private String _taskId; - // The successful attempt id - private String _attemptId; - - private long _totalTimeMs = 0; - private long _shuffleTimeMs = 0; - private long _sortTimeMs = 0; - private long _startTime = 0; - private long _finishTime = 0; - private boolean _isSampled = false; - - //Constructor used only in Test Cases , if provided to partially assign needed time values while ignoring the others - public TezTaskData(TezCounterData counterHolder, long[] time) { - if(time == null || time.length<3){ - time = new long[5]; - } - this._counterHolder = counterHolder; - this._totalTimeMs = time[0]; - this._shuffleTimeMs = time[1]; - this._sortTimeMs = time[2]; - if (time.length > 3) - this._startTime = time[3]; - if (time.length > 4) - this._finishTime = time[4]; - this._isSampled = true; - } - - public TezTaskData(TezCounterData counterHolder) { - this._counterHolder = counterHolder; - } - - public TezTaskData(String taskId, String taskAttemptId) { - this._taskId = taskId; - this._attemptId = taskAttemptId; - } - - public void setCounter(TezCounterData counterHolder) { - this._counterHolder = counterHolder; - this._isSampled = true; - } - - public void setTime(long[] time) { - //No Validation needed here as time array will always be of fixed length 5 from upstream methods. - this._totalTimeMs = time[0]; - this._shuffleTimeMs = time[1]; - this._sortTimeMs = time[2]; - this._startTime = time[3]; - this._finishTime = time[4]; - this._isSampled = true; - } - - //Used only in Test Cases - public void setTimeAndCounter(long[] time, TezCounterData counterHolder){ - if(time == null || time.length<3){ - time = new long[5]; - } - this._totalTimeMs = time[0]; - this._shuffleTimeMs = time[1]; - this._sortTimeMs = time[2]; - if (time.length > 3) - this._startTime = time[3]; - if (time.length > 4) - this._finishTime = time[4]; - this._isSampled = true; - this._counterHolder = counterHolder; - - - } - - public TezCounterData getCounters() { - return _counterHolder; - } - - public long getTotalRunTimeMs() { - return _totalTimeMs; - } - - public long getCodeExecutionTimeMs() { - return _totalTimeMs - _shuffleTimeMs - _sortTimeMs; - } - - public long getShuffleTimeMs() { - return _shuffleTimeMs; - } - - public long getSortTimeMs() { - return _sortTimeMs; - } - - public boolean isSampled() { - return _isSampled; - } - - public String getTaskId() { - return _taskId; - } - - public String getAttemptId() { - return _attemptId; - } - - public long getStartTime() { - return _startTime; - } - - public long getFinishTime() { - return _finishTime; - } - - public void setTotalTimeMs(long totalTimeMs, boolean isSampled) { - this._totalTimeMs = totalTimeMs; - this._isSampled = isSampled; - } + private TezCounterData _counterHolder; + private String _taskId; + // The successful attempt id + private String _attemptId; + + private long _totalTimeMs = 0; + private long _shuffleTimeMs = 0; + private long _sortTimeMs = 0; + private long _startTime = 0; + private long _finishTime = 0; + private boolean _isSampled = false; + + //Constructor used only in Test Cases , if provided to partially assign needed time values while ignoring the others + public TezTaskData(TezCounterData counterHolder, long[] time) { + if(time == null || time.length<3){ + time = new long[5]; + } + this._counterHolder = counterHolder; + this._totalTimeMs = time[0]; + this._shuffleTimeMs = time[1]; + this._sortTimeMs = time[2]; + if (time.length > 3) + this._startTime = time[3]; + if (time.length > 4) + this._finishTime = time[4]; + this._isSampled = true; + } + + public TezTaskData(TezCounterData counterHolder) { + this._counterHolder = counterHolder; + } + + public TezTaskData(String taskId, String taskAttemptId) { + this._taskId = taskId; + this._attemptId = taskAttemptId; + } + + public void setCounter(TezCounterData counterHolder) { + this._counterHolder = counterHolder; + this._isSampled = true; + } + + public void setTime(long[] time) { + //No Validation needed here as time array will always be of fixed length 5 from upstream methods. + this._totalTimeMs = time[0]; + this._shuffleTimeMs = time[1]; + this._sortTimeMs = time[2]; + this._startTime = time[3]; + this._finishTime = time[4]; + this._isSampled = true; + } + + //Used only in Test Cases + public void setTimeAndCounter(long[] time, TezCounterData counterHolder){ + if(time == null || time.length<3){ + time = new long[5]; + } + this._totalTimeMs = time[0]; + this._shuffleTimeMs = time[1]; + this._sortTimeMs = time[2]; + if (time.length > 3) + this._startTime = time[3]; + if (time.length > 4) + this._finishTime = time[4]; + this._isSampled = true; + this._counterHolder = counterHolder; + + + } + + public TezCounterData getCounters() { + return _counterHolder; + } + + public long getTotalRunTimeMs() { + return _totalTimeMs; + } + + public long getCodeExecutionTimeMs() { + return _totalTimeMs - _shuffleTimeMs - _sortTimeMs; + } + + public long getShuffleTimeMs() { + return _shuffleTimeMs; + } + + public long getSortTimeMs() { + return _sortTimeMs; + } + + public boolean isSampled() { + return _isSampled; + } + + public String getTaskId() { + return _taskId; + } + + public String getAttemptId() { + return _attemptId; + } + + public long getStartTime() { + return _startTime; + } + + public long getFinishTime() { + return _finishTime; + } + + public void setTotalTimeMs(long totalTimeMs, boolean isSampled) { + this._totalTimeMs = totalTimeMs; + this._isSampled = isSampled; + } } diff --git a/app/com/linkedin/drelephant/tez/fetchers/TezFetcher.java b/app/com/linkedin/drelephant/tez/fetchers/TezFetcher.java index db38586fe..aed07aa6c 100644 --- a/app/com/linkedin/drelephant/tez/fetchers/TezFetcher.java +++ b/app/com/linkedin/drelephant/tez/fetchers/TezFetcher.java @@ -1,3 +1,17 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + * + */ package com.linkedin.drelephant.tez.fetchers; import com.linkedin.drelephant.analysis.AnalyticJob; @@ -31,436 +45,436 @@ public class TezFetcher implements ElephantFetcher { - private static final Logger logger = Logger.getLogger(TezFetcher.class); - private static final int MAX_SAMPLE_SIZE = 1000; + private static final Logger logger = Logger.getLogger(TezFetcher.class); + private static final int MAX_SAMPLE_SIZE = 1000; - private URLFactory _urlFactory; - private JSONFactory _jsonFactory; - private String _timelineWebAddr; + private URLFactory _urlFactory; + private JSONFactory _jsonFactory; + private String _timelineWebAddr; - private FetcherConfigurationData _fetcherConfigurationData; + private FetcherConfigurationData _fetcherConfigurationData; - public TezFetcher(FetcherConfigurationData fetcherConfData) throws IOException { - this._fetcherConfigurationData = fetcherConfData; - final String applicationHistoryAddr = new Configuration().get("yarn.timeline-service.webapp.address"); + public TezFetcher(FetcherConfigurationData fetcherConfData) throws IOException { + this._fetcherConfigurationData = fetcherConfData; + final String applicationHistoryAddr = new Configuration().get("yarn.timeline-service.webapp.address"); - //Connection validity checked using method verifyURL(_timelineWebAddr) inside URLFactory constructor; - _urlFactory = new URLFactory(applicationHistoryAddr); - logger.info("Connection success."); + //Connection validity checked using method verifyURL(_timelineWebAddr) inside URLFactory constructor; + _urlFactory = new URLFactory(applicationHistoryAddr); + logger.info("Connection success."); - _jsonFactory = new JSONFactory(); - _timelineWebAddr = "http://" + _timelineWebAddr + "/ws/v1/timeline/"; + _jsonFactory = new JSONFactory(); + _timelineWebAddr = "http://" + _timelineWebAddr + "/ws/v1/timeline/"; - } + } - public TezApplicationData fetchData(AnalyticJob analyticJob) throws IOException, AuthenticationException { + public TezApplicationData fetchData(AnalyticJob analyticJob) throws IOException, AuthenticationException { - int maxSize = 0; - String appId = analyticJob.getAppId(); - TezApplicationData jobData = new TezApplicationData(); - jobData.setAppId(appId); - Properties jobConf = _jsonFactory.getProperties(_urlFactory.getApplicationURL(appId)); - jobData.setConf(jobConf); - URL dagIdsUrl = _urlFactory.getDagURLByTezApplicationId(appId); + int maxSize = 0; + String appId = analyticJob.getAppId(); + TezApplicationData jobData = new TezApplicationData(); + jobData.setAppId(appId); + Properties jobConf = _jsonFactory.getProperties(_urlFactory.getApplicationURL(appId)); + jobData.setConf(jobConf); + URL dagIdsUrl = _urlFactory.getDagURLByTezApplicationId(appId); - List dagIdsByApplicationId = _jsonFactory.getDagIdsByApplicationId(dagIdsUrl); + List dagIdsByApplicationId = _jsonFactory.getDagIdsByApplicationId(dagIdsUrl); - List mapperListAggregate = new ArrayList(); - List reducerListAggregate = new ArrayList(); + List mapperListAggregate = new ArrayList(); + List reducerListAggregate = new ArrayList(); - //Iterate over dagIds and choose the dagId with the highest no. of tasks/highest impact as settings changes can be made only at DAG level. - for(String dagId : dagIdsByApplicationId){ - try { - //set job task independent properties + //Iterate over dagIds and choose the dagId with the highest no. of tasks/highest impact as settings changes can be made only at DAG level. + for(String dagId : dagIdsByApplicationId){ + try { + //set job task independent properties - URL dagUrl = _urlFactory.getDagURL(dagId); - String state = _jsonFactory.getState(dagUrl); + URL dagUrl = _urlFactory.getDagURL(dagId); + String state = _jsonFactory.getState(dagUrl); - jobData.setStartTime(_jsonFactory.getDagStartTime(dagUrl)); - jobData.setFinishTime(_jsonFactory.getDagEndTime(dagUrl)); + jobData.setStartTime(_jsonFactory.getDagStartTime(dagUrl)); + jobData.setFinishTime(_jsonFactory.getDagEndTime(dagUrl)); - if (state.equals("SUCCEEDED")) { - jobData.setSucceeded(true); + if (state.equals("SUCCEEDED")) { + jobData.setSucceeded(true); - List mapperList = new ArrayList(); - List reducerList = new ArrayList(); + List mapperList = new ArrayList(); + List reducerList = new ArrayList(); - // Fetch task data - URL vertexListUrl = _urlFactory.getVertexListURL(dagId); - _jsonFactory.getTaskDataAll(vertexListUrl, dagId, mapperList, reducerList); + // Fetch task data + URL vertexListUrl = _urlFactory.getVertexListURL(dagId); + _jsonFactory.getTaskDataAll(vertexListUrl, dagId, mapperList, reducerList); - if(mapperList.size() + reducerList.size() > maxSize){ - mapperListAggregate = mapperList; - reducerListAggregate = reducerList; - maxSize = mapperList.size() + reducerList.size(); - } + if(mapperList.size() + reducerList.size() > maxSize){ + mapperListAggregate = mapperList; + reducerListAggregate = reducerList; + maxSize = mapperList.size() + reducerList.size(); + } - } - } - finally { - ThreadContextMR2.updateAuthToken(); - } } + } + finally { + ThreadContextMR2.updateAuthToken(); + } + } - if(mapperListAggregate.isEmpty() && reducerListAggregate.isEmpty()){ - jobData.setSucceeded(false); - } + if(mapperListAggregate.isEmpty() && reducerListAggregate.isEmpty()){ + jobData.setSucceeded(false); + } - TezTaskData[] mapperData = mapperListAggregate.toArray(new TezTaskData[mapperListAggregate.size()]); - TezTaskData[] reducerData = reducerListAggregate.toArray(new TezTaskData[reducerListAggregate.size()]); + TezTaskData[] mapperData = mapperListAggregate.toArray(new TezTaskData[mapperListAggregate.size()]); + TezTaskData[] reducerData = reducerListAggregate.toArray(new TezTaskData[reducerListAggregate.size()]); - TezCounterData dagCounter = _jsonFactory.getDagCounter(_urlFactory.getDagURL(_jsonFactory.getDagIdsByApplicationId(dagIdsUrl).get(0))); + TezCounterData dagCounter = _jsonFactory.getDagCounter(_urlFactory.getDagURL(_jsonFactory.getDagIdsByApplicationId(dagIdsUrl).get(0))); - jobData.setCounters(dagCounter).setMapTaskData(mapperData).setReduceTaskData(reducerData); + jobData.setCounters(dagCounter).setMapTaskData(mapperData).setReduceTaskData(reducerData); - return jobData; - } + return jobData; + } - private URL getTaskListByVertexURL(String dagId, String vertexId) throws MalformedURLException { - return _urlFactory.getTaskListByVertexURL(dagId, vertexId); - } + private URL getTaskListByVertexURL(String dagId, String vertexId) throws MalformedURLException { + return _urlFactory.getTaskListByVertexURL(dagId, vertexId); + } - private URL getTaskURL(String taskId) throws MalformedURLException { - return _urlFactory.getTasksURL(taskId); + private URL getTaskURL(String taskId) throws MalformedURLException { + return _urlFactory.getTasksURL(taskId); + } + + private URL getTaskAttemptURL(String dagId, String taskId, String attemptId) throws MalformedURLException { + return _urlFactory.getTaskAttemptURL(dagId, taskId, attemptId); + } + + private class URLFactory { + + private String _timelineWebAddr; + + private URLFactory(String hserverAddr) throws IOException { + _timelineWebAddr = "http://" + hserverAddr + "/ws/v1/timeline"; + verifyURL(_timelineWebAddr); } - private URL getTaskAttemptURL(String dagId, String taskId, String attemptId) throws MalformedURLException { - return _urlFactory.getTaskAttemptURL(dagId, taskId, attemptId); + private void verifyURL(String url) throws IOException { + final URLConnection connection = new URL(url).openConnection(); + // Check service availability + connection.connect(); + return; } - private class URLFactory { + private URL getDagURLByTezApplicationId(String applicationId) throws MalformedURLException { + return new URL(_timelineWebAddr + "/TEZ_DAG_ID?primaryFilter=applicationId:" + applicationId); + } - private String _timelineWebAddr; + private URL getApplicationURL(String applicationId) throws MalformedURLException { + return new URL(_timelineWebAddr + "/TEZ_APPLICATION/tez_" + applicationId); + } - private URLFactory(String hserverAddr) throws IOException { - _timelineWebAddr = "http://" + hserverAddr + "/ws/v1/timeline"; - verifyURL(_timelineWebAddr); - } + private URL getDagURL(String dagId) throws MalformedURLException { + return new URL(_timelineWebAddr + "/TEZ_DAG_ID/" + dagId); + } - private void verifyURL(String url) throws IOException { - final URLConnection connection = new URL(url).openConnection(); - // Check service availability - connection.connect(); - return; - } + private URL getVertexListURL(String dagId) throws MalformedURLException { + return new URL(_timelineWebAddr + "/TEZ_VERTEX_ID?primaryFilter=TEZ_DAG_ID:" + dagId); + } - private URL getDagURLByTezApplicationId(String applicationId) throws MalformedURLException { - return new URL(_timelineWebAddr + "/TEZ_DAG_ID?primaryFilter=applicationId:" + applicationId); - } + private URL getTaskListByVertexURL(String dagId, String vertexId) throws MalformedURLException { + return new URL(_timelineWebAddr + "/TEZ_TASK_ID?primaryFilter=TEZ_DAG_ID:" + dagId + + "&secondaryFilter=TEZ_VERTEX_ID:" + vertexId); + } - private URL getApplicationURL(String applicationId) throws MalformedURLException { - return new URL(_timelineWebAddr + "/TEZ_APPLICATION/tez_" + applicationId); - } + private URL getTasksURL(String taskId) throws MalformedURLException { + return new URL(_timelineWebAddr + "/TEZ_TASK_ID/" + taskId); + } - private URL getDagURL(String dagId) throws MalformedURLException { - return new URL(_timelineWebAddr + "/TEZ_DAG_ID/" + dagId); - } + private URL getTaskAllAttemptsURL(String dagId, String taskId) throws MalformedURLException { + return new URL(_timelineWebAddr + "/TEZ_TASK_ATTEMPT_ID?primaryFilter=TEZ_DAG_ID:" + dagId + + "&secondaryFilter=TEZ_TASK_ID:" + taskId); + } - private URL getVertexListURL(String dagId) throws MalformedURLException { - return new URL(_timelineWebAddr + "/TEZ_VERTEX_ID?primaryFilter=TEZ_DAG_ID:" + dagId); - } + private URL getTaskAttemptURL(String dagId, String taskId, String attemptId) throws MalformedURLException { + return new URL(_timelineWebAddr + "/TEZ_TASK_ATTEMPT_ID/" + attemptId); + } - private URL getTaskListByVertexURL(String dagId, String vertexId) throws MalformedURLException { - return new URL(_timelineWebAddr + "/TEZ_TASK_ID?primaryFilter=TEZ_DAG_ID:" + dagId + - "&secondaryFilter=TEZ_VERTEX_ID:" + vertexId); - } + } - private URL getTasksURL(String taskId) throws MalformedURLException { - return new URL(_timelineWebAddr + "/TEZ_TASK_ID/" + taskId); - } + /** + * JSONFactory class provides functionality to parse mined job data from timeline server. + */ - private URL getTaskAllAttemptsURL(String dagId, String taskId) throws MalformedURLException { - return new URL(_timelineWebAddr + "/TEZ_TASK_ATTEMPT_ID?primaryFilter=TEZ_DAG_ID:" + dagId + - "&secondaryFilter=TEZ_TASK_ID:" + taskId); - } + private class JSONFactory { - private URL getTaskAttemptURL(String dagId, String taskId, String attemptId) throws MalformedURLException { - return new URL(_timelineWebAddr + "/TEZ_TASK_ATTEMPT_ID/" + attemptId); - } + private String getState(URL url) throws IOException, AuthenticationException { + JsonNode rootNode = ThreadContextMR2.readJsonNode(url); + return rootNode.path("otherinfo").path("status").getTextValue(); + } + private Properties getProperties(URL url) throws IOException, AuthenticationException { + Properties jobConf = new Properties(); + JsonNode rootNode = ThreadContextMR2.readJsonNode(url); + JsonNode configs = rootNode.path("otherinfo").path("config"); + Iterator keys = configs.getFieldNames(); + String key = ""; + String value = ""; + while (keys.hasNext()) { + key = keys.next(); + value = configs.get(key).getTextValue(); + jobConf.put(key, value); + } + return jobConf; } - /** - * JSONFactory class provides functionality to parse mined job data from timeline server. - */ + private List getDagIdsByApplicationId(URL dagIdsUrl) throws IOException, AuthenticationException { + List dagIds = new ArrayList(); + JsonNode nodes = ThreadContextMR2.readJsonNode(dagIdsUrl).get("entities"); - private class JSONFactory { + for (JsonNode node : nodes) { + String dagId = node.get("entity").getTextValue(); + dagIds.add(dagId); + } - private String getState(URL url) throws IOException, AuthenticationException { - JsonNode rootNode = ThreadContextMR2.readJsonNode(url); - return rootNode.path("otherinfo").path("status").getTextValue(); - } + return dagIds; + } - private Properties getProperties(URL url) throws IOException, AuthenticationException { - Properties jobConf = new Properties(); - JsonNode rootNode = ThreadContextMR2.readJsonNode(url); - JsonNode configs = rootNode.path("otherinfo").path("config"); - Iterator keys = configs.getFieldNames(); - String key = ""; - String value = ""; - while (keys.hasNext()) { - key = keys.next(); - value = configs.get(key).getTextValue(); - jobConf.put(key, value); - } - return jobConf; + private TezCounterData getDagCounter(URL url) throws IOException, AuthenticationException { + TezCounterData holder = new TezCounterData(); + JsonNode rootNode = ThreadContextMR2.readJsonNode(url); + JsonNode groups = rootNode.path("otherinfo").path("counters").path("counterGroups"); + + for (JsonNode group : groups) { + for (JsonNode counter : group.path("counters")) { + String name = counter.get("counterName").getTextValue(); + String groupName = group.get("counterGroupName").getTextValue(); + Long value = counter.get("counterValue").getLongValue(); + holder.set(groupName, name, value); } + } - private List getDagIdsByApplicationId(URL dagIdsUrl) throws IOException, AuthenticationException { - List dagIds = new ArrayList(); - JsonNode nodes = ThreadContextMR2.readJsonNode(dagIdsUrl).get("entities"); + return holder; + } - for (JsonNode node : nodes) { - String dagId = node.get("entity").getTextValue(); - dagIds.add(dagId); - } + private long getDagStartTime(URL url) throws IOException, AuthenticationException { + JsonNode rootNode = ThreadContextMR2.readJsonNode(url); + long startTime = rootNode.path("otherinfo").get("startTime").getLongValue(); + return startTime; + } - return dagIds; - } + private long getDagEndTime(URL url) throws IOException, AuthenticationException { + JsonNode rootNode = ThreadContextMR2.readJsonNode(url); + long endTime = rootNode.path("otherinfo").get("endTime").getLongValue(); + return endTime; + } - private TezCounterData getDagCounter(URL url) throws IOException, AuthenticationException { - TezCounterData holder = new TezCounterData(); - JsonNode rootNode = ThreadContextMR2.readJsonNode(url); - JsonNode groups = rootNode.path("otherinfo").path("counters").path("counterGroups"); - - for (JsonNode group : groups) { - for (JsonNode counter : group.path("counters")) { - String name = counter.get("counterName").getTextValue(); - String groupName = group.get("counterGroupName").getTextValue(); - Long value = counter.get("counterValue").getLongValue(); - holder.set(groupName, name, value); - } - } - - return holder; - } + private void getTaskDataAll(URL vertexListUrl, String dagId, List mapperList, + List reducerList) throws IOException, AuthenticationException { - private long getDagStartTime(URL url) throws IOException, AuthenticationException { - JsonNode rootNode = ThreadContextMR2.readJsonNode(url); - long startTime = rootNode.path("otherinfo").get("startTime").getLongValue(); - return startTime; - } + JsonNode rootVertexNode = ThreadContextMR2.readJsonNode(vertexListUrl); + JsonNode vertices = rootVertexNode.path("entities"); + boolean isMapVertex = false; - private long getDagEndTime(URL url) throws IOException, AuthenticationException { - JsonNode rootNode = ThreadContextMR2.readJsonNode(url); - long endTime = rootNode.path("otherinfo").get("endTime").getLongValue(); - return endTime; - } + for (JsonNode vertex : vertices) { + String vertexId = vertex.get("entity").getTextValue(); + String vertexClass = vertex.path("otherinfo").path("processorClassName").getTextValue(); - private void getTaskDataAll(URL vertexListUrl, String dagId, List mapperList, - List reducerList) throws IOException, AuthenticationException { + if (vertexClass.equals("org.apache.hadoop.hive.ql.exec.tez.MapTezProcessor")) + isMapVertex = true; + else if (vertexClass.equals("org.apache.hadoop.hive.ql.exec.tez.ReduceTezProcessor")) + isMapVertex = false; - JsonNode rootVertexNode = ThreadContextMR2.readJsonNode(vertexListUrl); - JsonNode vertices = rootVertexNode.path("entities"); - boolean isMapVertex = false; + URL tasksByVertexURL = getTaskListByVertexURL(dagId, vertexId); + if(isMapVertex) + getTaskDataByVertexId(tasksByVertexURL, dagId, vertexId, mapperList, true); + else + getTaskDataByVertexId(tasksByVertexURL, dagId, vertexId, reducerList, false); + } + } - for (JsonNode vertex : vertices) { - String vertexId = vertex.get("entity").getTextValue(); - String vertexClass = vertex.path("otherinfo").path("processorClassName").getTextValue(); + private void getTaskDataByVertexId(URL url, String dagId, String vertexId, List taskList, + boolean isMapVertex) throws IOException, AuthenticationException { + + JsonNode rootNode = ThreadContextMR2.readJsonNode(url); + JsonNode tasks = rootNode.path("entities"); + for (JsonNode task : tasks) { + String state = task.path("otherinfo").path("status").getTextValue(); + String taskId = task.get("entity").getValueAsText(); + String attemptId = task.path("otherinfo").path("successfulAttemptId").getTextValue(); + if (state.equals("SUCCEEDED")) { + attemptId = task.path("otherinfo").path("successfulAttemptId").getTextValue(); + } + else{ + JsonNode firstAttempt = getTaskFirstFailedAttempt(_urlFactory.getTaskAllAttemptsURL(dagId,taskId)); + if(firstAttempt != null){ + attemptId = firstAttempt.get("entity").getTextValue(); + } + } - if (vertexClass.equals("org.apache.hadoop.hive.ql.exec.tez.MapTezProcessor")) - isMapVertex = true; - else if (vertexClass.equals("org.apache.hadoop.hive.ql.exec.tez.ReduceTezProcessor")) - isMapVertex = false; + taskList.add(new TezTaskData(taskId, attemptId)); + } - URL tasksByVertexURL = getTaskListByVertexURL(dagId, vertexId); - if(isMapVertex) - getTaskDataByVertexId(tasksByVertexURL, dagId, vertexId, mapperList, true); - else - getTaskDataByVertexId(tasksByVertexURL, dagId, vertexId, reducerList, false); - } - } + getTaskData(dagId, taskList, isMapVertex); - private void getTaskDataByVertexId(URL url, String dagId, String vertexId, List taskList, - boolean isMapVertex) throws IOException, AuthenticationException { - - JsonNode rootNode = ThreadContextMR2.readJsonNode(url); - JsonNode tasks = rootNode.path("entities"); - for (JsonNode task : tasks) { - String state = task.path("otherinfo").path("status").getTextValue(); - String taskId = task.get("entity").getValueAsText(); - String attemptId = task.path("otherinfo").path("successfulAttemptId").getTextValue(); - if (state.equals("SUCCEEDED")) { - attemptId = task.path("otherinfo").path("successfulAttemptId").getTextValue(); - } - else{ - JsonNode firstAttempt = getTaskFirstFailedAttempt(_urlFactory.getTaskAllAttemptsURL(dagId,taskId)); - if(firstAttempt != null){ - attemptId = firstAttempt.get("entity").getTextValue(); - } - } - - taskList.add(new TezTaskData(taskId, attemptId)); - } - - getTaskData(dagId, taskList, isMapVertex); + } + private JsonNode getTaskFirstFailedAttempt(URL taskAllAttemptsUrl) throws IOException, AuthenticationException { + JsonNode rootNode = ThreadContextMR2.readJsonNode(taskAllAttemptsUrl); + long firstAttemptFinishTime = Long.MAX_VALUE; + JsonNode firstAttempt = null; + JsonNode taskAttempts = rootNode.path("entities"); + for (JsonNode taskAttempt : taskAttempts) { + String state = taskAttempt.path("otherinfo").path("counters").path("status").getTextValue(); + if (state.equals("SUCCEEDED")) { + continue; } - - private JsonNode getTaskFirstFailedAttempt(URL taskAllAttemptsUrl) throws IOException, AuthenticationException { - JsonNode rootNode = ThreadContextMR2.readJsonNode(taskAllAttemptsUrl); - long firstAttemptFinishTime = Long.MAX_VALUE; - JsonNode firstAttempt = null; - JsonNode taskAttempts = rootNode.path("entities"); - for (JsonNode taskAttempt : taskAttempts) { - String state = taskAttempt.path("otherinfo").path("counters").path("status").getTextValue(); - if (state.equals("SUCCEEDED")) { - continue; - } - long finishTime = taskAttempt.path("otherinfo").path("counters").path("endTime").getLongValue(); - if( finishTime < firstAttemptFinishTime) { - firstAttempt = taskAttempt; - firstAttemptFinishTime = finishTime; - } - } - return firstAttempt; + long finishTime = taskAttempt.path("otherinfo").path("counters").path("endTime").getLongValue(); + if( finishTime < firstAttemptFinishTime) { + firstAttempt = taskAttempt; + firstAttemptFinishTime = finishTime; } + } + return firstAttempt; + } - private void getTaskData(String dagId, List taskList, boolean isMapTask) - throws IOException, AuthenticationException { + private void getTaskData(String dagId, List taskList, boolean isMapTask) + throws IOException, AuthenticationException { - if (taskList.size() > MAX_SAMPLE_SIZE) { - logger.info(dagId + " needs sampling."); - Collections.shuffle(taskList); - } + if (taskList.size() > MAX_SAMPLE_SIZE) { + logger.info(dagId + " needs sampling."); + Collections.shuffle(taskList); + } - int sampleSize = Math.min(taskList.size(), MAX_SAMPLE_SIZE); - for (int i=0; i _LOCAL_THREAD_ID = new ThreadLocal() { - @Override - public Integer initialValue() { - return THREAD_ID.getAndIncrement(); - } - }; + private static final ThreadLocal _LOCAL_THREAD_ID = new ThreadLocal() { + @Override + public Integer initialValue() { + return THREAD_ID.getAndIncrement(); + } + }; - private static final ThreadLocal _LOCAL_LAST_UPDATED = new ThreadLocal(); - private static final ThreadLocal _LOCAL_UPDATE_INTERVAL = new ThreadLocal(); + private static final ThreadLocal _LOCAL_LAST_UPDATED = new ThreadLocal(); + private static final ThreadLocal _LOCAL_UPDATE_INTERVAL = new ThreadLocal(); - private static final ThreadLocal _LOCAL_DIAGNOSTIC_PATTERN = new ThreadLocal() { - @Override - public Pattern initialValue() { - // Example: "Task task_1443068695259_9143_m_000475 failed 1 times" - return Pattern.compile( - "Task[\\s\\u00A0]+(.*)[\\s\\u00A0]+failed[\\s\\u00A0]+([0-9])[\\s\\u00A0]+times[\\s\\u00A0]+"); - } - }; - - private static final ThreadLocal _LOCAL_AUTH_TOKEN = - new ThreadLocal() { - @Override - public AuthenticatedURL.Token initialValue() { - _LOCAL_LAST_UPDATED.set(System.currentTimeMillis()); - // Random an interval for each executor to avoid update token at the same time - _LOCAL_UPDATE_INTERVAL.set(Statistics.MINUTE_IN_MS * 30 + new Random().nextLong() - % (3 * Statistics.MINUTE_IN_MS)); - logger.info("Executor " + _LOCAL_THREAD_ID.get() + " update interval " + _LOCAL_UPDATE_INTERVAL.get() * 1.0 - / Statistics.MINUTE_IN_MS); - return new AuthenticatedURL.Token(); - } - }; - - private static final ThreadLocal _LOCAL_AUTH_URL = new ThreadLocal() { - @Override - public AuthenticatedURL initialValue() { - return new AuthenticatedURL(); - } - }; + private static final ThreadLocal _LOCAL_DIAGNOSTIC_PATTERN = new ThreadLocal() { + @Override + public Pattern initialValue() { + // Example: "Task task_1443068695259_9143_m_000475 failed 1 times" + return Pattern.compile( + "Task[\\s\\u00A0]+(.*)[\\s\\u00A0]+failed[\\s\\u00A0]+([0-9])[\\s\\u00A0]+times[\\s\\u00A0]+"); + } + }; - private static final ThreadLocal _LOCAL_MAPPER = new ThreadLocal() { + private static final ThreadLocal _LOCAL_AUTH_TOKEN = + new ThreadLocal() { @Override - public ObjectMapper initialValue() { - return new ObjectMapper(); + public AuthenticatedURL.Token initialValue() { + _LOCAL_LAST_UPDATED.set(System.currentTimeMillis()); + // Random an interval for each executor to avoid update token at the same time + _LOCAL_UPDATE_INTERVAL.set(Statistics.MINUTE_IN_MS * 30 + new Random().nextLong() + % (3 * Statistics.MINUTE_IN_MS)); + logger.info("Executor " + _LOCAL_THREAD_ID.get() + " update interval " + _LOCAL_UPDATE_INTERVAL.get() * 1.0 + / Statistics.MINUTE_IN_MS); + return new AuthenticatedURL.Token(); } - }; + }; - private ThreadContextMR2() { - // Empty on purpose + private static final ThreadLocal _LOCAL_AUTH_URL = new ThreadLocal() { + @Override + public AuthenticatedURL initialValue() { + return new AuthenticatedURL(); } + }; - public static Matcher getDiagnosticMatcher(String diagnosticInfo) { - return _LOCAL_DIAGNOSTIC_PATTERN.get().matcher(diagnosticInfo); + private static final ThreadLocal _LOCAL_MAPPER = new ThreadLocal() { + @Override + public ObjectMapper initialValue() { + return new ObjectMapper(); } - - public static JsonNode readJsonNode(URL url) throws IOException, AuthenticationException { - HttpURLConnection conn = _LOCAL_AUTH_URL.get().openConnection(url, _LOCAL_AUTH_TOKEN.get()); - return _LOCAL_MAPPER.get().readTree(conn.getInputStream()); - } - - public static void updateAuthToken() { - long curTime = System.currentTimeMillis(); - if (curTime - _LOCAL_LAST_UPDATED.get() > _LOCAL_UPDATE_INTERVAL.get()) { - logger.info("Executor " + _LOCAL_THREAD_ID.get() + " updates its AuthenticatedToken."); - _LOCAL_AUTH_TOKEN.set(new AuthenticatedURL.Token()); - _LOCAL_AUTH_URL.set(new AuthenticatedURL()); - _LOCAL_LAST_UPDATED.set(curTime); - } + }; + + private ThreadContextMR2() { + // Empty on purpose + } + + public static Matcher getDiagnosticMatcher(String diagnosticInfo) { + return _LOCAL_DIAGNOSTIC_PATTERN.get().matcher(diagnosticInfo); + } + + public static JsonNode readJsonNode(URL url) throws IOException, AuthenticationException { + HttpURLConnection conn = _LOCAL_AUTH_URL.get().openConnection(url, _LOCAL_AUTH_TOKEN.get()); + return _LOCAL_MAPPER.get().readTree(conn.getInputStream()); + } + + public static void updateAuthToken() { + long curTime = System.currentTimeMillis(); + if (curTime - _LOCAL_LAST_UPDATED.get() > _LOCAL_UPDATE_INTERVAL.get()) { + logger.info("Executor " + _LOCAL_THREAD_ID.get() + " updates its AuthenticatedToken."); + _LOCAL_AUTH_TOKEN.set(new AuthenticatedURL.Token()); + _LOCAL_AUTH_URL.set(new AuthenticatedURL()); + _LOCAL_LAST_UPDATED.set(curTime); } + } } \ No newline at end of file diff --git a/app/com/linkedin/drelephant/tez/heuristics/GenericDataSkewHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/GenericDataSkewHeuristic.java index 107a84ee9..6970c3294 100644 --- a/app/com/linkedin/drelephant/tez/heuristics/GenericDataSkewHeuristic.java +++ b/app/com/linkedin/drelephant/tez/heuristics/GenericDataSkewHeuristic.java @@ -1,10 +1,19 @@ /* * Copyright 2017 Electronic Arts Inc. * - * Licensed under the Apache License, Version 2.0 + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. * */ - package com.linkedin.drelephant.tez.heuristics; @@ -36,171 +45,171 @@ * This Heuristic analyses the skewness in the task input data */ public abstract class GenericDataSkewHeuristic implements Heuristic { - private static final Logger logger = Logger.getLogger(GenericDataSkewHeuristic.class); + private static final Logger logger = Logger.getLogger(GenericDataSkewHeuristic.class); - // Severity Parameters - private static final String NUM_TASKS_SEVERITY = "num_tasks_severity"; - private static final String DEVIATION_SEVERITY = "deviation_severity"; - private static final String FILES_SEVERITY = "files_severity"; + // Severity Parameters + private static final String NUM_TASKS_SEVERITY = "num_tasks_severity"; + private static final String DEVIATION_SEVERITY = "deviation_severity"; + private static final String FILES_SEVERITY = "files_severity"; - // Default value of parameters - private double[] numTasksLimits = {10, 50, 100, 200}; // Number of map or reduce tasks - private double[] deviationLimits = {2, 4, 8, 16}; // Deviation in i/p bytes btw 2 groups - private double[] filesLimits = {1d / 8, 1d / 4, 1d / 2, 1d}; // Fraction of HDFS Block Size + // Default value of parameters + private double[] numTasksLimits = {10, 50, 100, 200}; // Number of map or reduce tasks + private double[] deviationLimits = {2, 4, 8, 16}; // Deviation in i/p bytes btw 2 groups + private double[] filesLimits = {1d / 8, 1d / 4, 1d / 2, 1d}; // Fraction of HDFS Block Size - private HeuristicConfigurationData _heuristicConfData; - private List _counterNames; + private HeuristicConfigurationData _heuristicConfData; + private List _counterNames; - private void loadParameters() { - Map paramMap = _heuristicConfData.getParamMap(); - String heuristicName = _heuristicConfData.getHeuristicName(); - - double[] confNumTasksThreshold = Utils.getParam(paramMap.get(NUM_TASKS_SEVERITY), numTasksLimits.length); - if (confNumTasksThreshold != null) { - numTasksLimits = confNumTasksThreshold; - } - logger.info(heuristicName + " will use " + NUM_TASKS_SEVERITY + " with the following threshold settings: " - + Arrays.toString(numTasksLimits)); + private void loadParameters() { + Map paramMap = _heuristicConfData.getParamMap(); + String heuristicName = _heuristicConfData.getHeuristicName(); - double[] confDeviationThreshold = Utils.getParam(paramMap.get(DEVIATION_SEVERITY), deviationLimits.length); - if (confDeviationThreshold != null) { - deviationLimits = confDeviationThreshold; - } - logger.info(heuristicName + " will use " + DEVIATION_SEVERITY + " with the following threshold settings: " - + Arrays.toString(deviationLimits)); + double[] confNumTasksThreshold = Utils.getParam(paramMap.get(NUM_TASKS_SEVERITY), numTasksLimits.length); + if (confNumTasksThreshold != null) { + numTasksLimits = confNumTasksThreshold; + } + logger.info(heuristicName + " will use " + NUM_TASKS_SEVERITY + " with the following threshold settings: " + + Arrays.toString(numTasksLimits)); - double[] confFilesThreshold = Utils.getParam(paramMap.get(FILES_SEVERITY), filesLimits.length); - if (confFilesThreshold != null) { - filesLimits = confFilesThreshold; - } - logger.info( - heuristicName + " will use " + FILES_SEVERITY + " with the following threshold settings: " + Arrays.toString( - filesLimits)); - for (int i = 0; i < filesLimits.length; i++) { - filesLimits[i] = filesLimits[i] * HDFSContext.HDFS_BLOCK_SIZE; - } + double[] confDeviationThreshold = Utils.getParam(paramMap.get(DEVIATION_SEVERITY), deviationLimits.length); + if (confDeviationThreshold != null) { + deviationLimits = confDeviationThreshold; } + logger.info(heuristicName + " will use " + DEVIATION_SEVERITY + " with the following threshold settings: " + + Arrays.toString(deviationLimits)); - public GenericDataSkewHeuristic(List counterNames, HeuristicConfigurationData heuristicConfData) { - this._counterNames = counterNames; - this._heuristicConfData = heuristicConfData; - loadParameters(); + double[] confFilesThreshold = Utils.getParam(paramMap.get(FILES_SEVERITY), filesLimits.length); + if (confFilesThreshold != null) { + filesLimits = confFilesThreshold; + } + logger.info( + heuristicName + " will use " + FILES_SEVERITY + " with the following threshold settings: " + Arrays.toString( + filesLimits)); + for (int i = 0; i < filesLimits.length; i++) { + filesLimits[i] = filesLimits[i] * HDFSContext.HDFS_BLOCK_SIZE; } + } - protected abstract TezTaskData[] getTasks(TezApplicationData data); + public GenericDataSkewHeuristic(List counterNames, HeuristicConfigurationData heuristicConfData) { + this._counterNames = counterNames; + this._heuristicConfData = heuristicConfData; + loadParameters(); + } - public HeuristicConfigurationData getHeuristicConfData() { - return _heuristicConfData; - } + protected abstract TezTaskData[] getTasks(TezApplicationData data); - public HeuristicResult apply(TezApplicationData data) { + public HeuristicConfigurationData getHeuristicConfData() { + return _heuristicConfData; + } - if (!data.getSucceeded()) { - return null; - } + public HeuristicResult apply(TezApplicationData data) { - TezTaskData[] tasks = getTasks(data); + if (!data.getSucceeded()) { + return null; + } - //Gathering data for checking time skew - List timeTaken = new ArrayList(); + TezTaskData[] tasks = getTasks(data); - for(int i = 0; i < tasks.length; i++) { - if (tasks[i].isSampled()) { - timeTaken.add(tasks[i].getTotalRunTimeMs()); - } - } + //Gathering data for checking time skew + List timeTaken = new ArrayList(); - long[][] groupsTime = Statistics.findTwoGroups(Longs.toArray(timeTaken)); + for(int i = 0; i < tasks.length; i++) { + if (tasks[i].isSampled()) { + timeTaken.add(tasks[i].getTotalRunTimeMs()); + } + } - long timeAvg1 = Statistics.average(groupsTime[0]); - long timeAvg2 = Statistics.average(groupsTime[1]); + long[][] groupsTime = Statistics.findTwoGroups(Longs.toArray(timeTaken)); - //seconds are used for calculating deviation as they provide a better idea than millisecond. - long timeAvgSec1 = TimeUnit.MILLISECONDS.toSeconds(timeAvg1); - long timeAvgSec2 = TimeUnit.MILLISECONDS.toSeconds(timeAvg2); + long timeAvg1 = Statistics.average(groupsTime[0]); + long timeAvg2 = Statistics.average(groupsTime[1]); - long minTime = Math.min(timeAvgSec1, timeAvgSec2); - long diffTime = Math.abs(timeAvgSec1 - timeAvgSec2); + //seconds are used for calculating deviation as they provide a better idea than millisecond. + long timeAvgSec1 = TimeUnit.MILLISECONDS.toSeconds(timeAvg1); + long timeAvgSec2 = TimeUnit.MILLISECONDS.toSeconds(timeAvg2); - //using the same deviation limits for time skew as for data skew. It can be changed in the fututre. - Severity severityTime = getDeviationSeverity(minTime, diffTime); + long minTime = Math.min(timeAvgSec1, timeAvgSec2); + long diffTime = Math.abs(timeAvgSec1 - timeAvgSec2); - //This reduces severity if number of tasks is insignificant - severityTime = Severity.min(severityTime, - Severity.getSeverityAscending(groupsTime[0].length, numTasksLimits[0], numTasksLimits[1], numTasksLimits[2], - numTasksLimits[3])); + //using the same deviation limits for time skew as for data skew. It can be changed in the fututre. + Severity severityTime = getDeviationSeverity(minTime, diffTime); - //Gather data - List inputSizes = new ArrayList(); + //This reduces severity if number of tasks is insignificant + severityTime = Severity.min(severityTime, + Severity.getSeverityAscending(groupsTime[0].length, numTasksLimits[0], numTasksLimits[1], numTasksLimits[2], + numTasksLimits[3])); - for (int i = 0; i < tasks.length; i++) { - if (tasks[i].isSampled()) { + //Gather data + List inputSizes = new ArrayList(); - long inputByte = 0; - for (TezCounterData.CounterName counterName : _counterNames) { - inputByte += tasks[i].getCounters().get(counterName); - } + for (int i = 0; i < tasks.length; i++) { + if (tasks[i].isSampled()) { - inputSizes.add(inputByte); - } + long inputByte = 0; + for (TezCounterData.CounterName counterName : _counterNames) { + inputByte += tasks[i].getCounters().get(counterName); } - long[][] groups = Statistics.findTwoGroups(Longs.toArray(inputSizes)); + inputSizes.add(inputByte); + } + } - long avg1 = Statistics.average(groups[0]); - long avg2 = Statistics.average(groups[1]); + long[][] groups = Statistics.findTwoGroups(Longs.toArray(inputSizes)); - long min = Math.min(avg1, avg2); - long diff = Math.abs(avg2 - avg1); + long avg1 = Statistics.average(groups[0]); + long avg2 = Statistics.average(groups[1]); - Severity severityData = getDeviationSeverity(min, diff); + long min = Math.min(avg1, avg2); + long diff = Math.abs(avg2 - avg1); - //This reduces severity if the largest file sizes are insignificant - severityData = Severity.min(severityData, getFilesSeverity(avg2)); + Severity severityData = getDeviationSeverity(min, diff); - //This reduces severity if number of tasks is insignificant - severityData = Severity.min(severityData, - Severity.getSeverityAscending(groups[0].length, numTasksLimits[0], numTasksLimits[1], numTasksLimits[2], - numTasksLimits[3])); + //This reduces severity if the largest file sizes are insignificant + severityData = Severity.min(severityData, getFilesSeverity(avg2)); - Severity severity = Severity.max(severityData, severityTime); + //This reduces severity if number of tasks is insignificant + severityData = Severity.min(severityData, + Severity.getSeverityAscending(groups[0].length, numTasksLimits[0], numTasksLimits[1], numTasksLimits[2], + numTasksLimits[3])); - HeuristicResult result = - new HeuristicResult(_heuristicConfData.getClassName(), _heuristicConfData.getHeuristicName(), severity, - Utils.getHeuristicScore(severityData, tasks.length)); + Severity severity = Severity.max(severityData, severityTime); - result.addResultDetail("Data skew (Number of tasks)", Integer.toString(tasks.length)); - result.addResultDetail("Data skew (Group A)", - groups[0].length + " tasks @ " + FileUtils.byteCountToDisplaySize(avg1) + " avg"); - result.addResultDetail("Data skew (Group B)", - groups[1].length + " tasks @ " + FileUtils.byteCountToDisplaySize(avg2) + " avg"); + HeuristicResult result = + new HeuristicResult(_heuristicConfData.getClassName(), _heuristicConfData.getHeuristicName(), severity, + Utils.getHeuristicScore(severityData, tasks.length)); - result.addResultDetail("Time skew (Number of tasks)", Integer.toString(tasks.length)); - result.addResultDetail("Time skew (Group A)", - groupsTime[0].length + " tasks @ " + convertTimeMs(timeAvg1) + " avg"); - result.addResultDetail("Time skew (Group B)", - groupsTime[1].length + " tasks @ " + convertTimeMs(timeAvg2) + " avg"); + result.addResultDetail("Data skew (Number of tasks)", Integer.toString(tasks.length)); + result.addResultDetail("Data skew (Group A)", + groups[0].length + " tasks @ " + FileUtils.byteCountToDisplaySize(avg1) + " avg"); + result.addResultDetail("Data skew (Group B)", + groups[1].length + " tasks @ " + FileUtils.byteCountToDisplaySize(avg2) + " avg"); - return result; - } + result.addResultDetail("Time skew (Number of tasks)", Integer.toString(tasks.length)); + result.addResultDetail("Time skew (Group A)", + groupsTime[0].length + " tasks @ " + convertTimeMs(timeAvg1) + " avg"); + result.addResultDetail("Time skew (Group B)", + groupsTime[1].length + " tasks @ " + convertTimeMs(timeAvg2) + " avg"); - private String convertTimeMs(long timeMs) { - if (timeMs < 1000) { - return Long.toString(timeMs) + " msec"; - } - return DurationFormatUtils.formatDuration(timeMs, "HH:mm:ss") + " HH:MM:SS"; - } + return result; + } - private Severity getDeviationSeverity(long averageMin, long averageDiff) { - if (averageMin <= 0) { - averageMin = 1; - } - long value = averageDiff / averageMin; - return Severity.getSeverityAscending(value, deviationLimits[0], deviationLimits[1], deviationLimits[2], - deviationLimits[3]); + private String convertTimeMs(long timeMs) { + if (timeMs < 1000) { + return Long.toString(timeMs) + " msec"; } + return DurationFormatUtils.formatDuration(timeMs, "HH:mm:ss") + " HH:MM:SS"; + } - private Severity getFilesSeverity(long value) { - return Severity.getSeverityAscending(value, filesLimits[0], filesLimits[1], filesLimits[2], filesLimits[3]); + private Severity getDeviationSeverity(long averageMin, long averageDiff) { + if (averageMin <= 0) { + averageMin = 1; } + long value = averageDiff / averageMin; + return Severity.getSeverityAscending(value, deviationLimits[0], deviationLimits[1], deviationLimits[2], + deviationLimits[3]); + } + + private Severity getFilesSeverity(long value) { + return Severity.getSeverityAscending(value, filesLimits[0], filesLimits[1], filesLimits[2], filesLimits[3]); + } } \ No newline at end of file diff --git a/app/com/linkedin/drelephant/tez/heuristics/GenericGCHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/GenericGCHeuristic.java index 7e67edbd8..d5726ef71 100644 --- a/app/com/linkedin/drelephant/tez/heuristics/GenericGCHeuristic.java +++ b/app/com/linkedin/drelephant/tez/heuristics/GenericGCHeuristic.java @@ -1,7 +1,17 @@ /* * Copyright 2017 Electronic Arts Inc. * - * Licensed under the Apache License, Version 2.0 + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. * */ package com.linkedin.drelephant.tez.heuristics; @@ -26,107 +36,107 @@ * Analyses garbage collection efficiency */ public abstract class GenericGCHeuristic implements Heuristic { - private static final Logger logger = Logger.getLogger(GenericGCHeuristic.class); - - // Severity Parameters - private static final String GC_RATIO_SEVERITY = "gc_ratio_severity"; - private static final String RUNTIME_SEVERITY = "runtime_severity_in_min"; - - // Default value of parameters - private double[] gcRatioLimits = {0.01d, 0.02d, 0.03d, 0.04d}; // Garbage Collection Time / CPU Time - private double[] runtimeLimits = {5, 10, 12, 15}; // Task Runtime in milli sec - - private HeuristicConfigurationData _heuristicConfData; - - private void loadParameters() { - Map paramMap = _heuristicConfData.getParamMap(); - String heuristicName = _heuristicConfData.getHeuristicName(); - - double[] confGcRatioThreshold = Utils.getParam(paramMap.get(GC_RATIO_SEVERITY), gcRatioLimits.length); - if (confGcRatioThreshold != null) { - gcRatioLimits = confGcRatioThreshold; - } - logger.info(heuristicName + " will use " + GC_RATIO_SEVERITY + " with the following threshold settings: " - + Arrays.toString(gcRatioLimits)); - - double[] confRuntimeThreshold = Utils.getParam(paramMap.get(RUNTIME_SEVERITY), runtimeLimits.length); - if (confRuntimeThreshold != null) { - runtimeLimits = confRuntimeThreshold; - } - logger.info(heuristicName + " will use " + RUNTIME_SEVERITY + " with the following threshold settings: " - + Arrays.toString(runtimeLimits)); - for (int i = 0; i < runtimeLimits.length; i++) { - runtimeLimits[i] = runtimeLimits[i] * Statistics.MINUTE_IN_MS; - } - } + private static final Logger logger = Logger.getLogger(GenericGCHeuristic.class); - public GenericGCHeuristic(HeuristicConfigurationData heuristicConfData) { - this._heuristicConfData = heuristicConfData; - loadParameters(); - } + // Severity Parameters + private static final String GC_RATIO_SEVERITY = "gc_ratio_severity"; + private static final String RUNTIME_SEVERITY = "runtime_severity_in_min"; + + // Default value of parameters + private double[] gcRatioLimits = {0.01d, 0.02d, 0.03d, 0.04d}; // Garbage Collection Time / CPU Time + private double[] runtimeLimits = {5, 10, 12, 15}; // Task Runtime in milli sec - protected abstract TezTaskData[] getTasks(TezApplicationData data); + private HeuristicConfigurationData _heuristicConfData; - public HeuristicConfigurationData getHeuristicConfData() { - return _heuristicConfData; + private void loadParameters() { + Map paramMap = _heuristicConfData.getParamMap(); + String heuristicName = _heuristicConfData.getHeuristicName(); + + double[] confGcRatioThreshold = Utils.getParam(paramMap.get(GC_RATIO_SEVERITY), gcRatioLimits.length); + if (confGcRatioThreshold != null) { + gcRatioLimits = confGcRatioThreshold; } + logger.info(heuristicName + " will use " + GC_RATIO_SEVERITY + " with the following threshold settings: " + + Arrays.toString(gcRatioLimits)); - public HeuristicResult apply(TezApplicationData data) { - - if(!data.getSucceeded()) { - return null; - } - - TezTaskData[] tasks = getTasks(data) ; - List gcMs = new ArrayList(); - List cpuMs = new ArrayList(); - List runtimesMs = new ArrayList(); - - for (TezTaskData task : tasks) { - if (task.isSampled()) { - runtimesMs.add(task.getTotalRunTimeMs()); - gcMs.add(task.getCounters().get(TezCounterData.CounterName.GC_TIME_MILLIS)); - cpuMs.add(task.getCounters().get(TezCounterData.CounterName.CPU_MILLISECONDS)); - } - } - - long avgRuntimeMs = Statistics.average(runtimesMs); - long avgCpuMs = Statistics.average(cpuMs); - long avgGcMs = Statistics.average(gcMs); - double ratio = avgCpuMs != 0 ? avgGcMs*(1.0)/avgCpuMs: 0; - - Severity severity; - if (tasks.length == 0) { - severity = Severity.NONE; - } else { - severity = getGcRatioSeverity(avgRuntimeMs, avgCpuMs, avgGcMs); - } - - HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), - _heuristicConfData.getHeuristicName(), severity, Utils.getHeuristicScore(severity, tasks.length)); - - result.addResultDetail("Number of tasks", Integer.toString(tasks.length)); - result.addResultDetail("Avg task runtime (ms)", Long.toString(avgRuntimeMs)); - result.addResultDetail("Avg task CPU time (ms)", Long.toString(avgCpuMs)); - result.addResultDetail("Avg task GC time (ms)", Long.toString(avgGcMs)); - result.addResultDetail("Task GC/CPU ratio", Double.toString(ratio)); - return result; + double[] confRuntimeThreshold = Utils.getParam(paramMap.get(RUNTIME_SEVERITY), runtimeLimits.length); + if (confRuntimeThreshold != null) { + runtimeLimits = confRuntimeThreshold; + } + logger.info(heuristicName + " will use " + RUNTIME_SEVERITY + " with the following threshold settings: " + + Arrays.toString(runtimeLimits)); + for (int i = 0; i < runtimeLimits.length; i++) { + runtimeLimits[i] = runtimeLimits[i] * Statistics.MINUTE_IN_MS; } + } + + public GenericGCHeuristic(HeuristicConfigurationData heuristicConfData) { + this._heuristicConfData = heuristicConfData; + loadParameters(); + } + + protected abstract TezTaskData[] getTasks(TezApplicationData data); - private Severity getGcRatioSeverity(long runtimeMs, long cpuMs, long gcMs) { - double gcRatio = ((double)gcMs)/cpuMs; - Severity ratioSeverity = Severity.getSeverityAscending( - gcRatio, gcRatioLimits[0], gcRatioLimits[1], gcRatioLimits[2], gcRatioLimits[3]); + public HeuristicConfigurationData getHeuristicConfData() { + return _heuristicConfData; + } - // Severity is reduced if task runtime is insignificant - Severity runtimeSeverity = getRuntimeSeverity(runtimeMs); + public HeuristicResult apply(TezApplicationData data) { - return Severity.min(ratioSeverity, runtimeSeverity); + if(!data.getSucceeded()) { + return null; } - private Severity getRuntimeSeverity(long runtimeMs) { - return Severity.getSeverityAscending( - runtimeMs, runtimeLimits[0], runtimeLimits[1], runtimeLimits[2], runtimeLimits[3]); + TezTaskData[] tasks = getTasks(data) ; + List gcMs = new ArrayList(); + List cpuMs = new ArrayList(); + List runtimesMs = new ArrayList(); + + for (TezTaskData task : tasks) { + if (task.isSampled()) { + runtimesMs.add(task.getTotalRunTimeMs()); + gcMs.add(task.getCounters().get(TezCounterData.CounterName.GC_TIME_MILLIS)); + cpuMs.add(task.getCounters().get(TezCounterData.CounterName.CPU_MILLISECONDS)); + } } + long avgRuntimeMs = Statistics.average(runtimesMs); + long avgCpuMs = Statistics.average(cpuMs); + long avgGcMs = Statistics.average(gcMs); + double ratio = avgCpuMs != 0 ? avgGcMs*(1.0)/avgCpuMs: 0; + + Severity severity; + if (tasks.length == 0) { + severity = Severity.NONE; + } else { + severity = getGcRatioSeverity(avgRuntimeMs, avgCpuMs, avgGcMs); + } + + HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), + _heuristicConfData.getHeuristicName(), severity, Utils.getHeuristicScore(severity, tasks.length)); + + result.addResultDetail("Number of tasks", Integer.toString(tasks.length)); + result.addResultDetail("Avg task runtime (ms)", Long.toString(avgRuntimeMs)); + result.addResultDetail("Avg task CPU time (ms)", Long.toString(avgCpuMs)); + result.addResultDetail("Avg task GC time (ms)", Long.toString(avgGcMs)); + result.addResultDetail("Task GC/CPU ratio", Double.toString(ratio)); + return result; + } + + private Severity getGcRatioSeverity(long runtimeMs, long cpuMs, long gcMs) { + double gcRatio = ((double)gcMs)/cpuMs; + Severity ratioSeverity = Severity.getSeverityAscending( + gcRatio, gcRatioLimits[0], gcRatioLimits[1], gcRatioLimits[2], gcRatioLimits[3]); + + // Severity is reduced if task runtime is insignificant + Severity runtimeSeverity = getRuntimeSeverity(runtimeMs); + + return Severity.min(ratioSeverity, runtimeSeverity); + } + + private Severity getRuntimeSeverity(long runtimeMs) { + return Severity.getSeverityAscending( + runtimeMs, runtimeLimits[0], runtimeLimits[1], runtimeLimits[2], runtimeLimits[3]); + } + } \ No newline at end of file diff --git a/app/com/linkedin/drelephant/tez/heuristics/GenericMemoryHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/GenericMemoryHeuristic.java index a9c0104b4..3a575092b 100644 --- a/app/com/linkedin/drelephant/tez/heuristics/GenericMemoryHeuristic.java +++ b/app/com/linkedin/drelephant/tez/heuristics/GenericMemoryHeuristic.java @@ -1,7 +1,17 @@ /* * Copyright 2017 Electronic Arts Inc. * - * Licensed under the Apache License, Version 2.0 + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. * */ package com.linkedin.drelephant.tez.heuristics; @@ -27,147 +37,147 @@ */ public abstract class GenericMemoryHeuristic implements Heuristic { - private static final Logger logger = Logger.getLogger(GenericMemoryHeuristic.class); + private static final Logger logger = Logger.getLogger(GenericMemoryHeuristic.class); - //Severity Parameters - private static final String MEM_RATIO_SEVERITY = "memory_ratio_severity"; - private static final String DEFAULT_MAPPER_CONTAINER_SIZE = "2048"; - private static final String CONTAINER_MEM_DEFAULT_MB = "container_memory_default_mb"; - private String _containerMemConf; + //Severity Parameters + private static final String MEM_RATIO_SEVERITY = "memory_ratio_severity"; + private static final String DEFAULT_MAPPER_CONTAINER_SIZE = "2048"; + private static final String CONTAINER_MEM_DEFAULT_MB = "container_memory_default_mb"; + private String _containerMemConf; - //Default Value of parameters + //Default Value of parameters - private double [] memoryRatioLimits = {0.6d, 0.5d, 0.4d, 0.3d}; //Ratio of successful tasks + private double [] memoryRatioLimits = {0.6d, 0.5d, 0.4d, 0.3d}; //Ratio of successful tasks - private HeuristicConfigurationData _heuristicConfData; + private HeuristicConfigurationData _heuristicConfData; - private String getContainerMemDefaultMBytes() { - Map paramMap = _heuristicConfData.getParamMap(); - if (paramMap.containsKey(CONTAINER_MEM_DEFAULT_MB)) { - String strValue = paramMap.get(CONTAINER_MEM_DEFAULT_MB); - try { - return strValue; - } - catch (NumberFormatException e) { - logger.warn(CONTAINER_MEM_DEFAULT_MB + ": expected number [" + strValue + "]"); - } - } - return DEFAULT_MAPPER_CONTAINER_SIZE; + private String getContainerMemDefaultMBytes() { + Map paramMap = _heuristicConfData.getParamMap(); + if (paramMap.containsKey(CONTAINER_MEM_DEFAULT_MB)) { + String strValue = paramMap.get(CONTAINER_MEM_DEFAULT_MB); + try { + return strValue; + } + catch (NumberFormatException e) { + logger.warn(CONTAINER_MEM_DEFAULT_MB + ": expected number [" + strValue + "]"); + } } + return DEFAULT_MAPPER_CONTAINER_SIZE; + } - private void loadParameters() { - Map paramMap = _heuristicConfData.getParamMap(); - String heuristicName = _heuristicConfData.getHeuristicName(); + private void loadParameters() { + Map paramMap = _heuristicConfData.getParamMap(); + String heuristicName = _heuristicConfData.getHeuristicName(); - double[] confSuccessRatioLimits = Utils.getParam(paramMap.get(MEM_RATIO_SEVERITY), memoryRatioLimits.length); - if (confSuccessRatioLimits != null) { - memoryRatioLimits = confSuccessRatioLimits; - } - logger.info(heuristicName + " will use " + MEM_RATIO_SEVERITY + " with the following threshold settings: " - + Arrays.toString(memoryRatioLimits)); + double[] confSuccessRatioLimits = Utils.getParam(paramMap.get(MEM_RATIO_SEVERITY), memoryRatioLimits.length); + if (confSuccessRatioLimits != null) { + memoryRatioLimits = confSuccessRatioLimits; + } + logger.info(heuristicName + " will use " + MEM_RATIO_SEVERITY + " with the following threshold settings: " + + Arrays.toString(memoryRatioLimits)); - } + } - public GenericMemoryHeuristic(String containerMemConf, HeuristicConfigurationData heuristicConfData) { - this._containerMemConf = containerMemConf; - this._heuristicConfData = heuristicConfData; - loadParameters(); - } + public GenericMemoryHeuristic(String containerMemConf, HeuristicConfigurationData heuristicConfData) { + this._containerMemConf = containerMemConf; + this._heuristicConfData = heuristicConfData; + loadParameters(); + } - public HeuristicConfigurationData getHeuristicConfData() { - return _heuristicConfData; - } + public HeuristicConfigurationData getHeuristicConfData() { + return _heuristicConfData; + } - protected abstract TezTaskData[] getTasks(TezApplicationData data); + protected abstract TezTaskData[] getTasks(TezApplicationData data); - public HeuristicResult apply(TezApplicationData data) { - if(!data.getSucceeded()) { - return null; - } - TezTaskData[] tasks = getTasks(data); + public HeuristicResult apply(TezApplicationData data) { + if(!data.getSucceeded()) { + return null; + } + TezTaskData[] tasks = getTasks(data); - List totalPhysicalMemory = new LinkedList(); - List totalVirtualMemory = new LinkedList(); - List runTime = new LinkedList(); + List totalPhysicalMemory = new LinkedList(); + List totalVirtualMemory = new LinkedList(); + List runTime = new LinkedList(); - for (TezTaskData task : tasks) { + for (TezTaskData task : tasks) { - if (task.isSampled()) { - totalPhysicalMemory.add(task.getCounters().get(TezCounterData.CounterName.PHYSICAL_MEMORY_BYTES)); - totalVirtualMemory.add(task.getCounters().get(TezCounterData.CounterName.VIRTUAL_MEMORY_BYTES)); - runTime.add(task.getTotalRunTimeMs()); - } + if (task.isSampled()) { + totalPhysicalMemory.add(task.getCounters().get(TezCounterData.CounterName.PHYSICAL_MEMORY_BYTES)); + totalVirtualMemory.add(task.getCounters().get(TezCounterData.CounterName.VIRTUAL_MEMORY_BYTES)); + runTime.add(task.getTotalRunTimeMs()); + } - } + } - long averagePMem = Statistics.average(totalPhysicalMemory); - long averageVMem = Statistics.average(totalVirtualMemory); - long maxPMem; - long minPMem; - try{ - maxPMem = Collections.max(totalPhysicalMemory); - minPMem = Collections.min(totalPhysicalMemory); + long averagePMem = Statistics.average(totalPhysicalMemory); + long averageVMem = Statistics.average(totalVirtualMemory); + long maxPMem; + long minPMem; + try{ + maxPMem = Collections.max(totalPhysicalMemory); + minPMem = Collections.min(totalPhysicalMemory); - } - catch(Exception exception){ - maxPMem = 0; - minPMem = 0; - } - long averageRunTime = Statistics.average(runTime); + } + catch(Exception exception){ + maxPMem = 0; + minPMem = 0; + } + long averageRunTime = Statistics.average(runTime); - String containerSizeStr; + String containerSizeStr; - if(!Strings.isNullOrEmpty(data.getConf().getProperty(_containerMemConf))){ - containerSizeStr = data.getConf().getProperty(_containerMemConf); - } - else { - containerSizeStr = getContainerMemDefaultMBytes(); - } + if(!Strings.isNullOrEmpty(data.getConf().getProperty(_containerMemConf))){ + containerSizeStr = data.getConf().getProperty(_containerMemConf); + } + else { + containerSizeStr = getContainerMemDefaultMBytes(); + } - long containerSize = Long.valueOf(containerSizeStr) * FileUtils.ONE_MB; + long containerSize = Long.valueOf(containerSizeStr) * FileUtils.ONE_MB; - double averageMemMb = (double)((averagePMem) /FileUtils.ONE_MB) ; + double averageMemMb = (double)((averagePMem) /FileUtils.ONE_MB) ; - double ratio = averageMemMb / ((double)(containerSize / FileUtils.ONE_MB)); + double ratio = averageMemMb / ((double)(containerSize / FileUtils.ONE_MB)); - Severity severity ; + Severity severity ; - if(tasks.length == 0){ - severity = Severity.NONE; - } - else{ - severity = getMemoryRatioSeverity(ratio); - } + if(tasks.length == 0){ + severity = Severity.NONE; + } + else{ + severity = getMemoryRatioSeverity(ratio); + } - HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), - _heuristicConfData.getHeuristicName(), severity, Utils.getHeuristicScore(severity, tasks.length)); + HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), + _heuristicConfData.getHeuristicName(), severity, Utils.getHeuristicScore(severity, tasks.length)); - result.addResultDetail("Number of tasks", Integer.toString(tasks.length)); - result.addResultDetail("Maximum Physical Memory (MB)", - tasks.length == 0 ? "0" : Long.toString(maxPMem/FileUtils.ONE_MB)); - result.addResultDetail("Minimum Physical memory (MB)", - tasks.length == 0 ? "0" : Long.toString(minPMem/FileUtils.ONE_MB)); - result.addResultDetail("Average Physical Memory (MB)", - tasks.length == 0 ? "0" : Long.toString(averagePMem/FileUtils.ONE_MB)); - result.addResultDetail("Average Virtual Memory (MB)", - tasks.length == 0 ? "0" : Long.toString(averageVMem/FileUtils.ONE_MB)); - result.addResultDetail("Average Task RunTime", - tasks.length == 0 ? "0" : Statistics.readableTimespan(averageRunTime)); - result.addResultDetail("Requested Container Memory (MB)", - (tasks.length == 0 || containerSize == 0 || containerSize == -1) ? "0" : String.valueOf(containerSize / FileUtils.ONE_MB)); + result.addResultDetail("Number of tasks", Integer.toString(tasks.length)); + result.addResultDetail("Maximum Physical Memory (MB)", + tasks.length == 0 ? "0" : Long.toString(maxPMem/FileUtils.ONE_MB)); + result.addResultDetail("Minimum Physical memory (MB)", + tasks.length == 0 ? "0" : Long.toString(minPMem/FileUtils.ONE_MB)); + result.addResultDetail("Average Physical Memory (MB)", + tasks.length == 0 ? "0" : Long.toString(averagePMem/FileUtils.ONE_MB)); + result.addResultDetail("Average Virtual Memory (MB)", + tasks.length == 0 ? "0" : Long.toString(averageVMem/FileUtils.ONE_MB)); + result.addResultDetail("Average Task RunTime", + tasks.length == 0 ? "0" : Statistics.readableTimespan(averageRunTime)); + result.addResultDetail("Requested Container Memory (MB)", + (tasks.length == 0 || containerSize == 0 || containerSize == -1) ? "0" : String.valueOf(containerSize / FileUtils.ONE_MB)); - return result; + return result; - } + } - private Severity getMemoryRatioSeverity(double ratio) { - return Severity.getSeverityDescending( - ratio, memoryRatioLimits[0], memoryRatioLimits[1], memoryRatioLimits[2], memoryRatioLimits[3]); - } + private Severity getMemoryRatioSeverity(double ratio) { + return Severity.getSeverityDescending( + ratio, memoryRatioLimits[0], memoryRatioLimits[1], memoryRatioLimits[2], memoryRatioLimits[3]); + } diff --git a/app/com/linkedin/drelephant/tez/heuristics/MapperDataSkewHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/MapperDataSkewHeuristic.java index 42c27b987..b17ebe3b2 100644 --- a/app/com/linkedin/drelephant/tez/heuristics/MapperDataSkewHeuristic.java +++ b/app/com/linkedin/drelephant/tez/heuristics/MapperDataSkewHeuristic.java @@ -1,7 +1,17 @@ /* * Copyright 2017 Electronic Arts Inc. * - * Licensed under the Apache License, Version 2.0 + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. * */ package com.linkedin.drelephant.tez.heuristics; @@ -20,18 +30,18 @@ * This Heuristic analyses the skewness in the task input data */ public class MapperDataSkewHeuristic extends GenericDataSkewHeuristic { - private static final Logger logger = Logger.getLogger(MapperDataSkewHeuristic.class); + private static final Logger logger = Logger.getLogger(MapperDataSkewHeuristic.class); - public MapperDataSkewHeuristic(HeuristicConfigurationData heuristicConfData) { - super(Arrays.asList( - TezCounterData.CounterName.HDFS_BYTES_READ, - TezCounterData.CounterName.S3A_BYTES_READ, - TezCounterData.CounterName.S3N_BYTES_READ - ), heuristicConfData); - } + public MapperDataSkewHeuristic(HeuristicConfigurationData heuristicConfData) { + super(Arrays.asList( + TezCounterData.CounterName.HDFS_BYTES_READ, + TezCounterData.CounterName.S3A_BYTES_READ, + TezCounterData.CounterName.S3N_BYTES_READ + ), heuristicConfData); + } - @Override - protected TezTaskData[] getTasks(TezApplicationData data) { - return data.getMapTaskData(); - } + @Override + protected TezTaskData[] getTasks(TezApplicationData data) { + return data.getMapTaskData(); + } } \ No newline at end of file diff --git a/app/com/linkedin/drelephant/tez/heuristics/MapperGCHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/MapperGCHeuristic.java index 14a091d1a..3a82ae5f9 100644 --- a/app/com/linkedin/drelephant/tez/heuristics/MapperGCHeuristic.java +++ b/app/com/linkedin/drelephant/tez/heuristics/MapperGCHeuristic.java @@ -1,7 +1,17 @@ /* * Copyright 2017 Electronic Arts Inc. * - * Licensed under the Apache License, Version 2.0 + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. * */ package com.linkedin.drelephant.tez.heuristics; @@ -27,15 +37,15 @@ * Analyses garbage collection efficiency */ public class MapperGCHeuristic extends GenericGCHeuristic{ - private static final Logger logger = Logger.getLogger(MapperGCHeuristic.class); + private static final Logger logger = Logger.getLogger(MapperGCHeuristic.class); - public MapperGCHeuristic(HeuristicConfigurationData heuristicConfData) { - super(heuristicConfData); - } + public MapperGCHeuristic(HeuristicConfigurationData heuristicConfData) { + super(heuristicConfData); + } - @Override - protected TezTaskData[] getTasks(TezApplicationData data) { - return data.getMapTaskData(); - } + @Override + protected TezTaskData[] getTasks(TezApplicationData data) { + return data.getMapTaskData(); + } } \ No newline at end of file diff --git a/app/com/linkedin/drelephant/tez/heuristics/MapperMemoryHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/MapperMemoryHeuristic.java index cc1932a1f..96d5bb12e 100644 --- a/app/com/linkedin/drelephant/tez/heuristics/MapperMemoryHeuristic.java +++ b/app/com/linkedin/drelephant/tez/heuristics/MapperMemoryHeuristic.java @@ -1,7 +1,17 @@ /* * Copyright 2017 Electronic Arts Inc. * - * Licensed under the Apache License, Version 2.0 + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. * */ package com.linkedin.drelephant.tez.heuristics; @@ -19,17 +29,17 @@ */ public class MapperMemoryHeuristic extends GenericMemoryHeuristic { - private static final Logger logger = Logger.getLogger(MapperMemoryHeuristic.class); + private static final Logger logger = Logger.getLogger(MapperMemoryHeuristic.class); - public static final String MAPPER_MEMORY_CONF = "mapreduce.map.memory.mb"; + public static final String MAPPER_MEMORY_CONF = "mapreduce.map.memory.mb"; - public MapperMemoryHeuristic(HeuristicConfigurationData __heuristicConfData){ - super(MAPPER_MEMORY_CONF, __heuristicConfData); - } + public MapperMemoryHeuristic(HeuristicConfigurationData __heuristicConfData){ + super(MAPPER_MEMORY_CONF, __heuristicConfData); + } - @Override - protected TezTaskData[] getTasks(TezApplicationData data) { - return data.getMapTaskData(); - } + @Override + protected TezTaskData[] getTasks(TezApplicationData data) { + return data.getMapTaskData(); + } } \ No newline at end of file diff --git a/app/com/linkedin/drelephant/tez/heuristics/MapperSpeedHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/MapperSpeedHeuristic.java index a8bb5ebe7..1694e32ea 100644 --- a/app/com/linkedin/drelephant/tez/heuristics/MapperSpeedHeuristic.java +++ b/app/com/linkedin/drelephant/tez/heuristics/MapperSpeedHeuristic.java @@ -1,7 +1,17 @@ /* * Copyright 2017 Electronic Arts Inc. * - * Licensed under the Apache License, Version 2.0 + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. * */ package com.linkedin.drelephant.tez.heuristics; @@ -31,125 +41,125 @@ * Analyzes mapper task speed and efficiency */ public class MapperSpeedHeuristic implements Heuristic { - private static final Logger logger = Logger.getLogger(MapperSpeedHeuristic.class); - - // Severity parameters. - private static final String DISK_SPEED_SEVERITY = "disk_speed_severity"; - private static final String RUNTIME_SEVERITY = "runtime_severity_in_min"; + private static final Logger logger = Logger.getLogger(MapperSpeedHeuristic.class); - // Default value of parameters - private double[] diskSpeedLimits = {1d/2, 1d/4, 1d/8, 1d/32}; // Fraction of HDFS block size - private double[] runtimeLimits = {5, 10, 15, 30}; // The Map task runtime in milli sec + // Severity parameters. + private static final String DISK_SPEED_SEVERITY = "disk_speed_severity"; + private static final String RUNTIME_SEVERITY = "runtime_severity_in_min"; - private List _counterNames = Arrays.asList( - TezCounterData.CounterName.HDFS_BYTES_READ, - TezCounterData.CounterName.S3A_BYTES_READ, - TezCounterData.CounterName.S3N_BYTES_READ - ); + // Default value of parameters + private double[] diskSpeedLimits = {1d/2, 1d/4, 1d/8, 1d/32}; // Fraction of HDFS block size + private double[] runtimeLimits = {5, 10, 15, 30}; // The Map task runtime in milli sec - private HeuristicConfigurationData _heuristicConfData; + private List _counterNames = Arrays.asList( + TezCounterData.CounterName.HDFS_BYTES_READ, + TezCounterData.CounterName.S3A_BYTES_READ, + TezCounterData.CounterName.S3N_BYTES_READ + ); - private void loadParameters() { - Map paramMap = _heuristicConfData.getParamMap(); - String heuristicName = _heuristicConfData.getHeuristicName(); + private HeuristicConfigurationData _heuristicConfData; - double[] confDiskSpeedThreshold = Utils.getParam(paramMap.get(DISK_SPEED_SEVERITY), diskSpeedLimits.length); - if (confDiskSpeedThreshold != null) { - diskSpeedLimits = confDiskSpeedThreshold; - } - logger.info(heuristicName + " will use " + DISK_SPEED_SEVERITY + " with the following threshold settings: " - + Arrays.toString(diskSpeedLimits)); - for (int i = 0; i < diskSpeedLimits.length; i++) { - diskSpeedLimits[i] = diskSpeedLimits[i] * HDFSContext.DISK_READ_SPEED; - } + private void loadParameters() { + Map paramMap = _heuristicConfData.getParamMap(); + String heuristicName = _heuristicConfData.getHeuristicName(); - double[] confRuntimeThreshold = Utils.getParam(paramMap.get(RUNTIME_SEVERITY), runtimeLimits.length); - if (confRuntimeThreshold != null) { - runtimeLimits = confRuntimeThreshold; - } - logger.info(heuristicName + " will use " + RUNTIME_SEVERITY + " with the following threshold settings: " + Arrays - .toString(runtimeLimits)); - for (int i = 0; i < runtimeLimits.length; i++) { - runtimeLimits[i] = runtimeLimits[i] * Statistics.MINUTE_IN_MS; - } + double[] confDiskSpeedThreshold = Utils.getParam(paramMap.get(DISK_SPEED_SEVERITY), diskSpeedLimits.length); + if (confDiskSpeedThreshold != null) { + diskSpeedLimits = confDiskSpeedThreshold; } - - public MapperSpeedHeuristic(HeuristicConfigurationData heuristicConfData) { - this._heuristicConfData = heuristicConfData; - loadParameters(); + logger.info(heuristicName + " will use " + DISK_SPEED_SEVERITY + " with the following threshold settings: " + + Arrays.toString(diskSpeedLimits)); + for (int i = 0; i < diskSpeedLimits.length; i++) { + diskSpeedLimits[i] = diskSpeedLimits[i] * HDFSContext.DISK_READ_SPEED; } - public HeuristicConfigurationData getHeuristicConfData() { - return _heuristicConfData; + double[] confRuntimeThreshold = Utils.getParam(paramMap.get(RUNTIME_SEVERITY), runtimeLimits.length); + if (confRuntimeThreshold != null) { + runtimeLimits = confRuntimeThreshold; } + logger.info(heuristicName + " will use " + RUNTIME_SEVERITY + " with the following threshold settings: " + Arrays + .toString(runtimeLimits)); + for (int i = 0; i < runtimeLimits.length; i++) { + runtimeLimits[i] = runtimeLimits[i] * Statistics.MINUTE_IN_MS; + } + } - public HeuristicResult apply(TezApplicationData data) { + public MapperSpeedHeuristic(HeuristicConfigurationData heuristicConfData) { + this._heuristicConfData = heuristicConfData; + loadParameters(); + } - if(!data.getSucceeded()) { - return null; - } + public HeuristicConfigurationData getHeuristicConfData() { + return _heuristicConfData; + } - TezTaskData[] tasks = data.getMapTaskData(); + public HeuristicResult apply(TezApplicationData data) { - List inputSizes = new ArrayList(); - List speeds = new ArrayList(); - List runtimesMs = new ArrayList(); + if(!data.getSucceeded()) { + return null; + } - for (TezTaskData task : tasks) { + TezTaskData[] tasks = data.getMapTaskData(); - if (task.isSampled()) { + List inputSizes = new ArrayList(); + List speeds = new ArrayList(); + List runtimesMs = new ArrayList(); - long inputBytes = 0; + for (TezTaskData task : tasks) { - for (TezCounterData.CounterName counterName: _counterNames) { - inputBytes += task.getCounters().get(counterName); - } + if (task.isSampled()) { - long runtimeMs = task.getTotalRunTimeMs(); - inputSizes.add(inputBytes); - runtimesMs.add(runtimeMs); - //Speed is records per second - speeds.add((1000 * inputBytes) / (runtimeMs)); - } - } + long inputBytes = 0; - long medianSpeed; - long medianSize; - long medianRuntimeMs; - - if (tasks.length != 0) { - medianSpeed = Statistics.median(speeds); - medianSize = Statistics.median(inputSizes); - medianRuntimeMs = Statistics.median(runtimesMs); - } else { - medianSpeed = 0; - medianSize = 0; - medianRuntimeMs = 0; + for (TezCounterData.CounterName counterName: _counterNames) { + inputBytes += task.getCounters().get(counterName); } - Severity severity = getDiskSpeedSeverity(medianSpeed); + long runtimeMs = task.getTotalRunTimeMs(); + inputSizes.add(inputBytes); + runtimesMs.add(runtimeMs); + //Speed is records per second + speeds.add((1000 * inputBytes) / (runtimeMs)); + } + } - //This reduces severity if task runtime is insignificant - severity = Severity.min(severity, getRuntimeSeverity(medianRuntimeMs)); + long medianSpeed; + long medianSize; + long medianRuntimeMs; + + if (tasks.length != 0) { + medianSpeed = Statistics.median(speeds); + medianSize = Statistics.median(inputSizes); + medianRuntimeMs = Statistics.median(runtimesMs); + } else { + medianSpeed = 0; + medianSize = 0; + medianRuntimeMs = 0; + } - HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), - _heuristicConfData.getHeuristicName(), severity, Utils.getHeuristicScore(severity, tasks.length)); + Severity severity = getDiskSpeedSeverity(medianSpeed); - result.addResultDetail("Number of tasks", Integer.toString(tasks.length)); - result.addResultDetail("Median task input ", FileUtils.byteCountToDisplaySize(medianSize)); - result.addResultDetail("Median task runtime", Statistics.readableTimespan(medianRuntimeMs)); - result.addResultDetail("Median task speed", FileUtils.byteCountToDisplaySize(medianSpeed) + "/s"); + //This reduces severity if task runtime is insignificant + severity = Severity.min(severity, getRuntimeSeverity(medianRuntimeMs)); - return result; - } + HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), + _heuristicConfData.getHeuristicName(), severity, Utils.getHeuristicScore(severity, tasks.length)); - private Severity getDiskSpeedSeverity(long speed) { - return Severity.getSeverityDescending( - speed, diskSpeedLimits[0], diskSpeedLimits[1], diskSpeedLimits[2], diskSpeedLimits[3]); - } + result.addResultDetail("Number of tasks", Integer.toString(tasks.length)); + result.addResultDetail("Median task input ", FileUtils.byteCountToDisplaySize(medianSize)); + result.addResultDetail("Median task runtime", Statistics.readableTimespan(medianRuntimeMs)); + result.addResultDetail("Median task speed", FileUtils.byteCountToDisplaySize(medianSpeed) + "/s"); - private Severity getRuntimeSeverity(long runtimeMs) { - return Severity.getSeverityAscending( - runtimeMs, runtimeLimits[0], runtimeLimits[1], runtimeLimits[2], runtimeLimits[3]); - } + return result; + } + + private Severity getDiskSpeedSeverity(long speed) { + return Severity.getSeverityDescending( + speed, diskSpeedLimits[0], diskSpeedLimits[1], diskSpeedLimits[2], diskSpeedLimits[3]); + } + + private Severity getRuntimeSeverity(long runtimeMs) { + return Severity.getSeverityAscending( + runtimeMs, runtimeLimits[0], runtimeLimits[1], runtimeLimits[2], runtimeLimits[3]); + } } \ No newline at end of file diff --git a/app/com/linkedin/drelephant/tez/heuristics/MapperSpillHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/MapperSpillHeuristic.java index 270680925..f7ea997e5 100644 --- a/app/com/linkedin/drelephant/tez/heuristics/MapperSpillHeuristic.java +++ b/app/com/linkedin/drelephant/tez/heuristics/MapperSpillHeuristic.java @@ -1,3 +1,17 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + * + */ package com.linkedin.drelephant.tez.heuristics; import com.linkedin.drelephant.analysis.Heuristic; @@ -18,111 +32,111 @@ * Analyzes mapper task data spill rates */ public class MapperSpillHeuristic implements Heuristic { - private static final Logger logger = Logger.getLogger(MapperSpillHeuristic.class); - private static final long THRESHOLD_SPILL_FACTOR = 10000; - - // Severity parameters. - private static final String SPILL_SEVERITY = "spill_severity"; - private static final String NUM_TASKS_SEVERITY = "num_tasks_severity"; - - // Default value of parameters - private double[] numTasksLimits = {50, 100, 500, 1000}; // Number of Map tasks. - private double[] spillLimits = {2.01d, 2.2d, 2.5d, 3.0d}; // Records spilled/total output records - - private HeuristicConfigurationData _heuristicConfData; - - private void loadParameters() { - Map paramMap = _heuristicConfData.getParamMap(); - String heuristicName = _heuristicConfData.getHeuristicName(); - - double[] confNumTasksThreshold = Utils.getParam(paramMap.get(NUM_TASKS_SEVERITY), numTasksLimits.length); - if (confNumTasksThreshold != null) { - numTasksLimits = confNumTasksThreshold; - } - logger.info(heuristicName + " will use " + NUM_TASKS_SEVERITY + " with the following threshold settings: " - + Arrays.toString(numTasksLimits)); - - double[] confSpillThreshold = Utils.getParam(paramMap.get(SPILL_SEVERITY), spillLimits.length); - if (confSpillThreshold != null) { - spillLimits = confSpillThreshold; - } - logger.info(heuristicName + " will use " + SPILL_SEVERITY + " with the following threshold settings: " + Arrays - .toString(spillLimits)); - for (int i = 0; i < spillLimits.length; i++) { - spillLimits[i] = spillLimits[i] * THRESHOLD_SPILL_FACTOR; - } - } + private static final Logger logger = Logger.getLogger(MapperSpillHeuristic.class); + private static final long THRESHOLD_SPILL_FACTOR = 10000; - public MapperSpillHeuristic(HeuristicConfigurationData heuristicConfData) { - this._heuristicConfData = heuristicConfData; - loadParameters(); - } + // Severity parameters. + private static final String SPILL_SEVERITY = "spill_severity"; + private static final String NUM_TASKS_SEVERITY = "num_tasks_severity"; + + // Default value of parameters + private double[] numTasksLimits = {50, 100, 500, 1000}; // Number of Map tasks. + private double[] spillLimits = {2.01d, 2.2d, 2.5d, 3.0d}; // Records spilled/total output records - @Override - public HeuristicConfigurationData getHeuristicConfData() { - return _heuristicConfData; + private HeuristicConfigurationData _heuristicConfData; + + private void loadParameters() { + Map paramMap = _heuristicConfData.getParamMap(); + String heuristicName = _heuristicConfData.getHeuristicName(); + + double[] confNumTasksThreshold = Utils.getParam(paramMap.get(NUM_TASKS_SEVERITY), numTasksLimits.length); + if (confNumTasksThreshold != null) { + numTasksLimits = confNumTasksThreshold; } + logger.info(heuristicName + " will use " + NUM_TASKS_SEVERITY + " with the following threshold settings: " + + Arrays.toString(numTasksLimits)); - @Override - public HeuristicResult apply(TezApplicationData data) { - if(!data.getSucceeded()) { - return null; - } - TezTaskData[] tasks = data.getMapTaskData(); + double[] confSpillThreshold = Utils.getParam(paramMap.get(SPILL_SEVERITY), spillLimits.length); + if (confSpillThreshold != null) { + spillLimits = confSpillThreshold; + } + logger.info(heuristicName + " will use " + SPILL_SEVERITY + " with the following threshold settings: " + Arrays + .toString(spillLimits)); + for (int i = 0; i < spillLimits.length; i++) { + spillLimits[i] = spillLimits[i] * THRESHOLD_SPILL_FACTOR; + } + } + + public MapperSpillHeuristic(HeuristicConfigurationData heuristicConfData) { + this._heuristicConfData = heuristicConfData; + loadParameters(); + } + + @Override + public HeuristicConfigurationData getHeuristicConfData() { + return _heuristicConfData; + } + + @Override + public HeuristicResult apply(TezApplicationData data) { + if(!data.getSucceeded()) { + return null; + } + TezTaskData[] tasks = data.getMapTaskData(); - long totalSpills = 0; - long totalOutputRecords = 0; - double ratioSpills = 0.0; + long totalSpills = 0; + long totalOutputRecords = 0; + double ratioSpills = 0.0; - for (TezTaskData task : tasks) { + for (TezTaskData task : tasks) { - if (task.isSampled()) { - totalSpills += task.getCounters().get(TezCounterData.CounterName.SPILLED_RECORDS); - totalOutputRecords += task.getCounters().get(TezCounterData.CounterName.OUTPUT_RECORDS); - } - } + if (task.isSampled()) { + totalSpills += task.getCounters().get(TezCounterData.CounterName.SPILLED_RECORDS); + totalOutputRecords += task.getCounters().get(TezCounterData.CounterName.OUTPUT_RECORDS); + } + } - //If both totalSpills and totalOutputRecords are zero then set ratioSpills to zero. - if (totalSpills == 0) { - ratioSpills = 0; - } else { - ratioSpills = (double) totalSpills / (double) totalOutputRecords; - } + //If both totalSpills and totalOutputRecords are zero then set ratioSpills to zero. + if (totalSpills == 0) { + ratioSpills = 0; + } else { + ratioSpills = (double) totalSpills / (double) totalOutputRecords; + } - Severity severity = getSpillSeverity(ratioSpills); + Severity severity = getSpillSeverity(ratioSpills); - // Severity is reduced if number of tasks is small - Severity taskSeverity = getNumTasksSeverity(tasks.length); - severity = Severity.min(severity, taskSeverity); + // Severity is reduced if number of tasks is small + Severity taskSeverity = getNumTasksSeverity(tasks.length); + severity = Severity.min(severity, taskSeverity); - HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), - _heuristicConfData.getHeuristicName(), severity, Utils.getHeuristicScore(severity, tasks.length)); + HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), + _heuristicConfData.getHeuristicName(), severity, Utils.getHeuristicScore(severity, tasks.length)); - result.addResultDetail("Number of tasks", Integer.toString(tasks.length)); - result.addResultDetail("Avg spilled records per task", - tasks.length == 0 ? "0" : Long.toString(totalSpills / tasks.length)); - result.addResultDetail("Avg output records per task", - tasks.length == 0 ? "0" : Long.toString(totalOutputRecords / tasks.length)); - result.addResultDetail("Ratio of spilled records to output records", Double.toString(ratioSpills)); + result.addResultDetail("Number of tasks", Integer.toString(tasks.length)); + result.addResultDetail("Avg spilled records per task", + tasks.length == 0 ? "0" : Long.toString(totalSpills / tasks.length)); + result.addResultDetail("Avg output records per task", + tasks.length == 0 ? "0" : Long.toString(totalOutputRecords / tasks.length)); + result.addResultDetail("Ratio of spilled records to output records", Double.toString(ratioSpills)); - return result; + return result; - } + } - private Severity getSpillSeverity(double ratioSpills) { + private Severity getSpillSeverity(double ratioSpills) { - long normalizedSpillRatio = 0; - //Normalize the ratio to integer. - normalizedSpillRatio = (long) (ratioSpills * THRESHOLD_SPILL_FACTOR); + long normalizedSpillRatio = 0; + //Normalize the ratio to integer. + normalizedSpillRatio = (long) (ratioSpills * THRESHOLD_SPILL_FACTOR); - return Severity.getSeverityAscending( - normalizedSpillRatio, spillLimits[0], spillLimits[1], spillLimits[2], spillLimits[3]); - } + return Severity.getSeverityAscending( + normalizedSpillRatio, spillLimits[0], spillLimits[1], spillLimits[2], spillLimits[3]); + } - private Severity getNumTasksSeverity(long numTasks) { - return Severity.getSeverityAscending( - numTasks, numTasksLimits[0], numTasksLimits[1], numTasksLimits[2], numTasksLimits[3]); - } + private Severity getNumTasksSeverity(long numTasks) { + return Severity.getSeverityAscending( + numTasks, numTasksLimits[0], numTasksLimits[1], numTasksLimits[2], numTasksLimits[3]); + } } diff --git a/app/com/linkedin/drelephant/tez/heuristics/MapperTimeHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/MapperTimeHeuristic.java index 0f71a1d20..838bd7a59 100644 --- a/app/com/linkedin/drelephant/tez/heuristics/MapperTimeHeuristic.java +++ b/app/com/linkedin/drelephant/tez/heuristics/MapperTimeHeuristic.java @@ -1,7 +1,17 @@ /* * Copyright 2017 Electronic Arts Inc. * - * Licensed under the Apache License, Version 2.0 + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. * */ package com.linkedin.drelephant.tez.heuristics; @@ -27,145 +37,145 @@ */ public class MapperTimeHeuristic implements Heuristic { - private static final Logger logger = Logger.getLogger(MapperTimeHeuristic.class); - - // Severity parameters. - private static final String SHORT_RUNTIME_SEVERITY = "short_runtime_severity_in_min"; - private static final String LONG_RUNTIME_SEVERITY = "long_runtime_severity_in_min"; - private static final String NUM_TASKS_SEVERITY = "num_tasks_severity"; + private static final Logger logger = Logger.getLogger(MapperTimeHeuristic.class); - // Default value of parameters - private double[] shortRuntimeLimits = {10, 4, 2, 1}; // Limits(ms) for tasks with shorter runtime - private double[] longRuntimeLimits = {15, 30, 60, 120}; // Limits(ms) for tasks with longer runtime - private double[] numTasksLimits = {50, 101, 500, 1000}; // Number of Map tasks. + // Severity parameters. + private static final String SHORT_RUNTIME_SEVERITY = "short_runtime_severity_in_min"; + private static final String LONG_RUNTIME_SEVERITY = "long_runtime_severity_in_min"; + private static final String NUM_TASKS_SEVERITY = "num_tasks_severity"; - private List _counterNames = Arrays.asList( - TezCounterData.CounterName.HDFS_BYTES_READ, - TezCounterData.CounterName.S3A_BYTES_READ, - TezCounterData.CounterName.S3N_BYTES_READ - ); + // Default value of parameters + private double[] shortRuntimeLimits = {10, 4, 2, 1}; // Limits(ms) for tasks with shorter runtime + private double[] longRuntimeLimits = {15, 30, 60, 120}; // Limits(ms) for tasks with longer runtime + private double[] numTasksLimits = {50, 101, 500, 1000}; // Number of Map tasks. - private HeuristicConfigurationData _heuristicConfData; + private List _counterNames = Arrays.asList( + TezCounterData.CounterName.HDFS_BYTES_READ, + TezCounterData.CounterName.S3A_BYTES_READ, + TezCounterData.CounterName.S3N_BYTES_READ + ); - private void loadParameters() { - Map paramMap = _heuristicConfData.getParamMap(); - String heuristicName = _heuristicConfData.getHeuristicName(); - - double[] confShortThreshold = Utils.getParam(paramMap.get(SHORT_RUNTIME_SEVERITY), shortRuntimeLimits.length); - if (confShortThreshold != null) { - shortRuntimeLimits = confShortThreshold; - } - logger.info(heuristicName + " will use " + SHORT_RUNTIME_SEVERITY + " with the following threshold settings: " - + Arrays.toString(shortRuntimeLimits)); - for (int i = 0; i < shortRuntimeLimits.length; i++) { - shortRuntimeLimits[i] = shortRuntimeLimits[i] * Statistics.MINUTE_IN_MS; - } - - double[] confLongThreshold = Utils.getParam(paramMap.get(LONG_RUNTIME_SEVERITY), longRuntimeLimits.length); - if (confLongThreshold != null) { - longRuntimeLimits = confLongThreshold; - } - logger.info(heuristicName + " will use " + LONG_RUNTIME_SEVERITY + " with the following threshold settings: " - + Arrays.toString(longRuntimeLimits)); - for (int i = 0; i < longRuntimeLimits.length; i++) { - longRuntimeLimits[i] = longRuntimeLimits[i] * Statistics.MINUTE_IN_MS; - } - - double[] confNumTasksThreshold = Utils.getParam(paramMap.get(NUM_TASKS_SEVERITY), numTasksLimits.length); - if (confNumTasksThreshold != null) { - numTasksLimits = confNumTasksThreshold; - } - logger.info(heuristicName + " will use " + NUM_TASKS_SEVERITY + " with the following threshold settings: " + Arrays - .toString(numTasksLimits)); + private HeuristicConfigurationData _heuristicConfData; + private void loadParameters() { + Map paramMap = _heuristicConfData.getParamMap(); + String heuristicName = _heuristicConfData.getHeuristicName(); + double[] confShortThreshold = Utils.getParam(paramMap.get(SHORT_RUNTIME_SEVERITY), shortRuntimeLimits.length); + if (confShortThreshold != null) { + shortRuntimeLimits = confShortThreshold; } - - public MapperTimeHeuristic(HeuristicConfigurationData heuristicConfData) { - this._heuristicConfData = heuristicConfData; - loadParameters(); + logger.info(heuristicName + " will use " + SHORT_RUNTIME_SEVERITY + " with the following threshold settings: " + + Arrays.toString(shortRuntimeLimits)); + for (int i = 0; i < shortRuntimeLimits.length; i++) { + shortRuntimeLimits[i] = shortRuntimeLimits[i] * Statistics.MINUTE_IN_MS; } - public HeuristicConfigurationData getHeuristicConfData() { - return _heuristicConfData; + double[] confLongThreshold = Utils.getParam(paramMap.get(LONG_RUNTIME_SEVERITY), longRuntimeLimits.length); + if (confLongThreshold != null) { + longRuntimeLimits = confLongThreshold; + } + logger.info(heuristicName + " will use " + LONG_RUNTIME_SEVERITY + " with the following threshold settings: " + + Arrays.toString(longRuntimeLimits)); + for (int i = 0; i < longRuntimeLimits.length; i++) { + longRuntimeLimits[i] = longRuntimeLimits[i] * Statistics.MINUTE_IN_MS; } - public HeuristicResult apply(TezApplicationData data) { - if(!data.getSucceeded()) { - return null; - } - TezTaskData[] tasks = data.getMapTaskData(); - - List inputSizes = new ArrayList(); - List runtimesMs = new ArrayList(); - long taskMinMs = Long.MAX_VALUE; - long taskMaxMs = 0; - - for (TezTaskData task : tasks) { - - if (task.isSampled()) { - long inputByte = 0; - for (TezCounterData.CounterName counterName: _counterNames) { - inputByte += task.getCounters().get(counterName); - } - inputSizes.add(inputByte); - long taskTime = task.getTotalRunTimeMs(); - runtimesMs.add(taskTime); - taskMinMs = Math.min(taskMinMs, taskTime); - taskMaxMs = Math.max(taskMaxMs, taskTime); - } - } - - if(taskMinMs == Long.MAX_VALUE) { - taskMinMs = 0; - } + double[] confNumTasksThreshold = Utils.getParam(paramMap.get(NUM_TASKS_SEVERITY), numTasksLimits.length); + if (confNumTasksThreshold != null) { + numTasksLimits = confNumTasksThreshold; + } + logger.info(heuristicName + " will use " + NUM_TASKS_SEVERITY + " with the following threshold settings: " + Arrays + .toString(numTasksLimits)); - long averageSize = Statistics.average(inputSizes); - long averageTimeMs = Statistics.average(runtimesMs); - Severity shortTaskSeverity = shortTaskSeverity(tasks.length, averageTimeMs); - Severity longTaskSeverity = longTaskSeverity(tasks.length, averageTimeMs); - Severity severity = Severity.max(shortTaskSeverity, longTaskSeverity); + } - HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), - _heuristicConfData.getHeuristicName(), severity, Utils.getHeuristicScore(severity, tasks.length)); + public MapperTimeHeuristic(HeuristicConfigurationData heuristicConfData) { + this._heuristicConfData = heuristicConfData; + loadParameters(); + } - result.addResultDetail("Number of tasks", Integer.toString(tasks.length)); - result.addResultDetail("Average task input size", FileUtils.byteCountToDisplaySize(averageSize)); - result.addResultDetail("Average task runtime", Statistics.readableTimespan(averageTimeMs)); - result.addResultDetail("Max task runtime", Statistics.readableTimespan(taskMaxMs)); - result.addResultDetail("Min task runtime", Statistics.readableTimespan(taskMinMs)); + public HeuristicConfigurationData getHeuristicConfData() { + return _heuristicConfData; + } - return result; + public HeuristicResult apply(TezApplicationData data) { + if(!data.getSucceeded()) { + return null; } + TezTaskData[] tasks = data.getMapTaskData(); - private Severity shortTaskSeverity(long numTasks, long averageTimeMs) { - // We want to identify jobs with short task runtime - Severity severity = getShortRuntimeSeverity(averageTimeMs); - // Severity is reduced if number of tasks is small. - Severity numTaskSeverity = getNumTasksSeverity(numTasks); - return Severity.min(severity, numTaskSeverity); - } + List inputSizes = new ArrayList(); + List runtimesMs = new ArrayList(); + long taskMinMs = Long.MAX_VALUE; + long taskMaxMs = 0; - private Severity longTaskSeverity(long numTasks, long averageTimeMs) { - // We want to identify jobs with long task runtime. Severity is NOT reduced if num of tasks is large - return getLongRuntimeSeverity(averageTimeMs); - } + for (TezTaskData task : tasks) { - private Severity getShortRuntimeSeverity(long runtimeMs) { - return Severity.getSeverityDescending( - runtimeMs, shortRuntimeLimits[0], shortRuntimeLimits[1], shortRuntimeLimits[2], shortRuntimeLimits[3]); + if (task.isSampled()) { + long inputByte = 0; + for (TezCounterData.CounterName counterName: _counterNames) { + inputByte += task.getCounters().get(counterName); + } + inputSizes.add(inputByte); + long taskTime = task.getTotalRunTimeMs(); + runtimesMs.add(taskTime); + taskMinMs = Math.min(taskMinMs, taskTime); + taskMaxMs = Math.max(taskMaxMs, taskTime); + } } - private Severity getLongRuntimeSeverity(long runtimeMs) { - return Severity.getSeverityAscending( - runtimeMs, longRuntimeLimits[0], longRuntimeLimits[1], longRuntimeLimits[2], longRuntimeLimits[3]); + if(taskMinMs == Long.MAX_VALUE) { + taskMinMs = 0; } - private Severity getNumTasksSeverity(long numTasks) { - return Severity.getSeverityAscending( - numTasks, numTasksLimits[0], numTasksLimits[1], numTasksLimits[2], numTasksLimits[3]); - } + long averageSize = Statistics.average(inputSizes); + long averageTimeMs = Statistics.average(runtimesMs); + + Severity shortTaskSeverity = shortTaskSeverity(tasks.length, averageTimeMs); + Severity longTaskSeverity = longTaskSeverity(tasks.length, averageTimeMs); + Severity severity = Severity.max(shortTaskSeverity, longTaskSeverity); + + HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), + _heuristicConfData.getHeuristicName(), severity, Utils.getHeuristicScore(severity, tasks.length)); + + result.addResultDetail("Number of tasks", Integer.toString(tasks.length)); + result.addResultDetail("Average task input size", FileUtils.byteCountToDisplaySize(averageSize)); + result.addResultDetail("Average task runtime", Statistics.readableTimespan(averageTimeMs)); + result.addResultDetail("Max task runtime", Statistics.readableTimespan(taskMaxMs)); + result.addResultDetail("Min task runtime", Statistics.readableTimespan(taskMinMs)); + + return result; + } + + private Severity shortTaskSeverity(long numTasks, long averageTimeMs) { + // We want to identify jobs with short task runtime + Severity severity = getShortRuntimeSeverity(averageTimeMs); + // Severity is reduced if number of tasks is small. + Severity numTaskSeverity = getNumTasksSeverity(numTasks); + return Severity.min(severity, numTaskSeverity); + } + + private Severity longTaskSeverity(long numTasks, long averageTimeMs) { + // We want to identify jobs with long task runtime. Severity is NOT reduced if num of tasks is large + return getLongRuntimeSeverity(averageTimeMs); + } + + private Severity getShortRuntimeSeverity(long runtimeMs) { + return Severity.getSeverityDescending( + runtimeMs, shortRuntimeLimits[0], shortRuntimeLimits[1], shortRuntimeLimits[2], shortRuntimeLimits[3]); + } + + private Severity getLongRuntimeSeverity(long runtimeMs) { + return Severity.getSeverityAscending( + runtimeMs, longRuntimeLimits[0], longRuntimeLimits[1], longRuntimeLimits[2], longRuntimeLimits[3]); + } + + private Severity getNumTasksSeverity(long numTasks) { + return Severity.getSeverityAscending( + numTasks, numTasksLimits[0], numTasksLimits[1], numTasksLimits[2], numTasksLimits[3]); + } diff --git a/app/com/linkedin/drelephant/tez/heuristics/ReducerDataSkewHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/ReducerDataSkewHeuristic.java index 7daa8ba0d..3a01448d4 100644 --- a/app/com/linkedin/drelephant/tez/heuristics/ReducerDataSkewHeuristic.java +++ b/app/com/linkedin/drelephant/tez/heuristics/ReducerDataSkewHeuristic.java @@ -1,7 +1,17 @@ /* * Copyright 2017 Electronic Arts Inc. * - * Licensed under the Apache License, Version 2.0 + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. * */ package com.linkedin.drelephant.tez.heuristics; @@ -19,14 +29,14 @@ * This Heuristic analyses the skewness in the task input data */ public class ReducerDataSkewHeuristic extends GenericDataSkewHeuristic { - private static final Logger logger = Logger.getLogger(ReducerDataSkewHeuristic.class); + private static final Logger logger = Logger.getLogger(ReducerDataSkewHeuristic.class); - public ReducerDataSkewHeuristic(HeuristicConfigurationData heuristicConfData) { - super(Arrays.asList(TezCounterData.CounterName.SHUFFLE_BYTES), heuristicConfData); - } + public ReducerDataSkewHeuristic(HeuristicConfigurationData heuristicConfData) { + super(Arrays.asList(TezCounterData.CounterName.SHUFFLE_BYTES), heuristicConfData); + } - @Override - protected TezTaskData[] getTasks(TezApplicationData data) { - return data.getReduceTaskData(); - } + @Override + protected TezTaskData[] getTasks(TezApplicationData data) { + return data.getReduceTaskData(); + } } \ No newline at end of file diff --git a/app/com/linkedin/drelephant/tez/heuristics/ReducerGCHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/ReducerGCHeuristic.java index facda408b..3ea3bdac4 100644 --- a/app/com/linkedin/drelephant/tez/heuristics/ReducerGCHeuristic.java +++ b/app/com/linkedin/drelephant/tez/heuristics/ReducerGCHeuristic.java @@ -1,7 +1,17 @@ /* * Copyright 2017 Electronic Arts Inc. * - * Licensed under the Apache License, Version 2.0 + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. * */ package com.linkedin.drelephant.tez.heuristics; @@ -17,15 +27,15 @@ */ public class ReducerGCHeuristic extends GenericGCHeuristic { - private static final Logger logger = Logger.getLogger(ReducerGCHeuristic.class); + private static final Logger logger = Logger.getLogger(ReducerGCHeuristic.class); - public ReducerGCHeuristic(HeuristicConfigurationData heuristicConfData) { - super(heuristicConfData); - } + public ReducerGCHeuristic(HeuristicConfigurationData heuristicConfData) { + super(heuristicConfData); + } - @Override - protected TezTaskData[] getTasks(TezApplicationData data) { - return data.getReduceTaskData(); - } + @Override + protected TezTaskData[] getTasks(TezApplicationData data) { + return data.getReduceTaskData(); + } } diff --git a/app/com/linkedin/drelephant/tez/heuristics/ReducerMemoryHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/ReducerMemoryHeuristic.java index f34cf7e66..41fc6fa1d 100644 --- a/app/com/linkedin/drelephant/tez/heuristics/ReducerMemoryHeuristic.java +++ b/app/com/linkedin/drelephant/tez/heuristics/ReducerMemoryHeuristic.java @@ -1,7 +1,17 @@ /* * Copyright 2017 Electronic Arts Inc. * - * Licensed under the Apache License, Version 2.0 + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. * */ package com.linkedin.drelephant.tez.heuristics; @@ -18,18 +28,18 @@ public class ReducerMemoryHeuristic extends GenericMemoryHeuristic { - private static final Logger logger = Logger.getLogger(ReducerMemoryHeuristic.class); + private static final Logger logger = Logger.getLogger(ReducerMemoryHeuristic.class); - public static final String REDUCER_MEMORY_CONF = "mapreduce.reduce.memory.mb"; + public static final String REDUCER_MEMORY_CONF = "mapreduce.reduce.memory.mb"; - public ReducerMemoryHeuristic(HeuristicConfigurationData __heuristicConfData){ - super(REDUCER_MEMORY_CONF, __heuristicConfData); - } + public ReducerMemoryHeuristic(HeuristicConfigurationData __heuristicConfData){ + super(REDUCER_MEMORY_CONF, __heuristicConfData); + } - @Override - protected TezTaskData[] getTasks(TezApplicationData data) { - return data.getReduceTaskData(); - } + @Override + protected TezTaskData[] getTasks(TezApplicationData data) { + return data.getReduceTaskData(); + } diff --git a/app/com/linkedin/drelephant/tez/heuristics/ReducerTimeHeuristic.java b/app/com/linkedin/drelephant/tez/heuristics/ReducerTimeHeuristic.java index f5f8edcaf..5e0dbfe65 100644 --- a/app/com/linkedin/drelephant/tez/heuristics/ReducerTimeHeuristic.java +++ b/app/com/linkedin/drelephant/tez/heuristics/ReducerTimeHeuristic.java @@ -1,7 +1,17 @@ /* * Copyright 2017 Electronic Arts Inc. * - * Licensed under the Apache License, Version 2.0 + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. * */ package com.linkedin.drelephant.tez.heuristics; @@ -26,132 +36,132 @@ public class ReducerTimeHeuristic implements Heuristic { - private static final Logger logger = Logger.getLogger(ReducerTimeHeuristic.class); - - // Severity parameters. - private static final String SHORT_RUNTIME_SEVERITY = "short_runtime_severity_in_min"; - private static final String LONG_RUNTIME_SEVERITY = "long_runtime_severity_in_min"; - private static final String NUM_TASKS_SEVERITY = "num_tasks_severity"; - - // Default value of parameters - private double[] shortRuntimeLimits = {10, 4, 2, 1}; // Limits(ms) for tasks with shorter runtime - private double[] longRuntimeLimits = {15, 30, 60, 120}; // Limits(ms) for tasks with longer runtime - private double[] numTasksLimits = {50, 101, 500, 1000}; // Number of Map tasks. - - private HeuristicConfigurationData _heuristicConfData; - - private void loadParameters() { - Map paramMap = _heuristicConfData.getParamMap(); - String heuristicName = _heuristicConfData.getHeuristicName(); - - double[] confShortThreshold = Utils.getParam(paramMap.get(SHORT_RUNTIME_SEVERITY), shortRuntimeLimits.length); - if (confShortThreshold != null) { - shortRuntimeLimits = confShortThreshold; - } - logger.info(heuristicName + " will use " + SHORT_RUNTIME_SEVERITY + " with the following threshold settings: " - + Arrays.toString(shortRuntimeLimits)); - for (int i = 0; i < shortRuntimeLimits.length; i++) { - shortRuntimeLimits[i] = shortRuntimeLimits[i] * Statistics.MINUTE_IN_MS; - } - - double[] confLongThreshold = Utils.getParam(paramMap.get(LONG_RUNTIME_SEVERITY), longRuntimeLimits.length); - if (confLongThreshold != null) { - longRuntimeLimits = confLongThreshold; - } - logger.info(heuristicName + " will use " + LONG_RUNTIME_SEVERITY + " with the following threshold settings: " - + Arrays.toString(longRuntimeLimits)); - for (int i = 0; i < longRuntimeLimits.length; i++) { - longRuntimeLimits[i] = longRuntimeLimits[i] * Statistics.MINUTE_IN_MS; - } - - double[] confNumTasksThreshold = Utils.getParam(paramMap.get(NUM_TASKS_SEVERITY), numTasksLimits.length); - if (confNumTasksThreshold != null) { - numTasksLimits = confNumTasksThreshold; - } - logger.info(heuristicName + " will use " + NUM_TASKS_SEVERITY + " with the following threshold settings: " + Arrays - .toString(numTasksLimits)); + private static final Logger logger = Logger.getLogger(ReducerTimeHeuristic.class); + + // Severity parameters. + private static final String SHORT_RUNTIME_SEVERITY = "short_runtime_severity_in_min"; + private static final String LONG_RUNTIME_SEVERITY = "long_runtime_severity_in_min"; + private static final String NUM_TASKS_SEVERITY = "num_tasks_severity"; + + // Default value of parameters + private double[] shortRuntimeLimits = {10, 4, 2, 1}; // Limits(ms) for tasks with shorter runtime + private double[] longRuntimeLimits = {15, 30, 60, 120}; // Limits(ms) for tasks with longer runtime + private double[] numTasksLimits = {50, 101, 500, 1000}; // Number of Map tasks. + + private HeuristicConfigurationData _heuristicConfData; + private void loadParameters() { + Map paramMap = _heuristicConfData.getParamMap(); + String heuristicName = _heuristicConfData.getHeuristicName(); + double[] confShortThreshold = Utils.getParam(paramMap.get(SHORT_RUNTIME_SEVERITY), shortRuntimeLimits.length); + if (confShortThreshold != null) { + shortRuntimeLimits = confShortThreshold; + } + logger.info(heuristicName + " will use " + SHORT_RUNTIME_SEVERITY + " with the following threshold settings: " + + Arrays.toString(shortRuntimeLimits)); + for (int i = 0; i < shortRuntimeLimits.length; i++) { + shortRuntimeLimits[i] = shortRuntimeLimits[i] * Statistics.MINUTE_IN_MS; } - public ReducerTimeHeuristic(HeuristicConfigurationData heuristicConfData) { - this._heuristicConfData = heuristicConfData; - loadParameters(); + double[] confLongThreshold = Utils.getParam(paramMap.get(LONG_RUNTIME_SEVERITY), longRuntimeLimits.length); + if (confLongThreshold != null) { + longRuntimeLimits = confLongThreshold; + } + logger.info(heuristicName + " will use " + LONG_RUNTIME_SEVERITY + " with the following threshold settings: " + + Arrays.toString(longRuntimeLimits)); + for (int i = 0; i < longRuntimeLimits.length; i++) { + longRuntimeLimits[i] = longRuntimeLimits[i] * Statistics.MINUTE_IN_MS; } - public HeuristicConfigurationData getHeuristicConfData() { - return _heuristicConfData; + double[] confNumTasksThreshold = Utils.getParam(paramMap.get(NUM_TASKS_SEVERITY), numTasksLimits.length); + if (confNumTasksThreshold != null) { + numTasksLimits = confNumTasksThreshold; } + logger.info(heuristicName + " will use " + NUM_TASKS_SEVERITY + " with the following threshold settings: " + Arrays + .toString(numTasksLimits)); + + + } + + public ReducerTimeHeuristic(HeuristicConfigurationData heuristicConfData) { + this._heuristicConfData = heuristicConfData; + loadParameters(); + } - public HeuristicResult apply(TezApplicationData data) { - if(!data.getSucceeded()) { - return null; - } - TezTaskData[] tasks = data.getReduceTaskData(); + public HeuristicConfigurationData getHeuristicConfData() { + return _heuristicConfData; + } - List runtimesMs = new ArrayList(); - long taskMinMs = Long.MAX_VALUE; - long taskMaxMs = 0; + public HeuristicResult apply(TezApplicationData data) { + if(!data.getSucceeded()) { + return null; + } + TezTaskData[] tasks = data.getReduceTaskData(); - for (TezTaskData task : tasks) { + List runtimesMs = new ArrayList(); + long taskMinMs = Long.MAX_VALUE; + long taskMaxMs = 0; - if (task.isSampled()) { - long taskTime = task.getTotalRunTimeMs(); - runtimesMs.add(taskTime); - taskMinMs = Math.min(taskMinMs, taskTime); - taskMaxMs = Math.max(taskMaxMs, taskTime); - } - } + for (TezTaskData task : tasks) { - if(taskMinMs == Long.MAX_VALUE) { - taskMinMs = 0; - } + if (task.isSampled()) { + long taskTime = task.getTotalRunTimeMs(); + runtimesMs.add(taskTime); + taskMinMs = Math.min(taskMinMs, taskTime); + taskMaxMs = Math.max(taskMaxMs, taskTime); + } + } + if(taskMinMs == Long.MAX_VALUE) { + taskMinMs = 0; + } - long averageTimeMs = Statistics.average(runtimesMs); - Severity shortTaskSeverity = shortTaskSeverity(tasks.length, averageTimeMs); - Severity longTaskSeverity = longTaskSeverity(tasks.length, averageTimeMs); - Severity severity = Severity.max(shortTaskSeverity, longTaskSeverity); + long averageTimeMs = Statistics.average(runtimesMs); - HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), - _heuristicConfData.getHeuristicName(), severity, Utils.getHeuristicScore(severity, tasks.length)); + Severity shortTaskSeverity = shortTaskSeverity(tasks.length, averageTimeMs); + Severity longTaskSeverity = longTaskSeverity(tasks.length, averageTimeMs); + Severity severity = Severity.max(shortTaskSeverity, longTaskSeverity); - result.addResultDetail("Number of tasks", Integer.toString(tasks.length)); - result.addResultDetail("Average task runtime", tasks.length == 0 ? "0" : (Statistics.readableTimespan(averageTimeMs).equals("") ? "0 sec" : Statistics.readableTimespan(averageTimeMs))); - result.addResultDetail("Max task runtime", tasks.length == 0 ? "0" : (Statistics.readableTimespan(taskMaxMs).equals("") ? "0 sec" : Statistics.readableTimespan(taskMaxMs)) ); - result.addResultDetail("Min task runtime", tasks.length == 0 ? "0" : (Statistics.readableTimespan(taskMinMs).equals("") ? "0 sec" :Statistics.readableTimespan(taskMinMs))) ; + HeuristicResult result = new HeuristicResult(_heuristicConfData.getClassName(), + _heuristicConfData.getHeuristicName(), severity, Utils.getHeuristicScore(severity, tasks.length)); - return result; - } + result.addResultDetail("Number of tasks", Integer.toString(tasks.length)); + result.addResultDetail("Average task runtime", tasks.length == 0 ? "0" : (Statistics.readableTimespan(averageTimeMs).equals("") ? "0 sec" : Statistics.readableTimespan(averageTimeMs))); + result.addResultDetail("Max task runtime", tasks.length == 0 ? "0" : (Statistics.readableTimespan(taskMaxMs).equals("") ? "0 sec" : Statistics.readableTimespan(taskMaxMs)) ); + result.addResultDetail("Min task runtime", tasks.length == 0 ? "0" : (Statistics.readableTimespan(taskMinMs).equals("") ? "0 sec" :Statistics.readableTimespan(taskMinMs))) ; - private Severity shortTaskSeverity(long numTasks, long averageTimeMs) { - // We want to identify jobs with short task runtime - Severity severity = getShortRuntimeSeverity(averageTimeMs); - // Severity is reduced if number of tasks is small. - Severity numTaskSeverity = getNumTasksSeverity(numTasks); - return Severity.min(severity, numTaskSeverity); - } + return result; + } - private Severity longTaskSeverity(long numTasks, long averageTimeMs) { - // We want to identify jobs with long task runtime. Severity is NOT reduced if num of tasks is large - return getLongRuntimeSeverity(averageTimeMs); - } + private Severity shortTaskSeverity(long numTasks, long averageTimeMs) { + // We want to identify jobs with short task runtime + Severity severity = getShortRuntimeSeverity(averageTimeMs); + // Severity is reduced if number of tasks is small. + Severity numTaskSeverity = getNumTasksSeverity(numTasks); + return Severity.min(severity, numTaskSeverity); + } - private Severity getShortRuntimeSeverity(long runtimeMs) { - return Severity.getSeverityDescending( - runtimeMs, shortRuntimeLimits[0], shortRuntimeLimits[1], shortRuntimeLimits[2], shortRuntimeLimits[3]); - } + private Severity longTaskSeverity(long numTasks, long averageTimeMs) { + // We want to identify jobs with long task runtime. Severity is NOT reduced if num of tasks is large + return getLongRuntimeSeverity(averageTimeMs); + } - private Severity getLongRuntimeSeverity(long runtimeMs) { - return Severity.getSeverityAscending( - runtimeMs, longRuntimeLimits[0], longRuntimeLimits[1], longRuntimeLimits[2], longRuntimeLimits[3]); - } + private Severity getShortRuntimeSeverity(long runtimeMs) { + return Severity.getSeverityDescending( + runtimeMs, shortRuntimeLimits[0], shortRuntimeLimits[1], shortRuntimeLimits[2], shortRuntimeLimits[3]); + } - private Severity getNumTasksSeverity(long numTasks) { - return Severity.getSeverityAscending( - numTasks, numTasksLimits[0], numTasksLimits[1], numTasksLimits[2], numTasksLimits[3]); - } + private Severity getLongRuntimeSeverity(long runtimeMs) { + return Severity.getSeverityAscending( + runtimeMs, longRuntimeLimits[0], longRuntimeLimits[1], longRuntimeLimits[2], longRuntimeLimits[3]); + } + + private Severity getNumTasksSeverity(long numTasks) { + return Severity.getSeverityAscending( + numTasks, numTasksLimits[0], numTasksLimits[1], numTasksLimits[2], numTasksLimits[3]); + } } diff --git a/test/com/linkedin/drelephant/tez/TezTaskLevelAggregatedMetricsTest.java b/test/com/linkedin/drelephant/tez/TezTaskLevelAggregatedMetricsTest.java index 256c9b9f2..4a42a8617 100644 --- a/test/com/linkedin/drelephant/tez/TezTaskLevelAggregatedMetricsTest.java +++ b/test/com/linkedin/drelephant/tez/TezTaskLevelAggregatedMetricsTest.java @@ -1,7 +1,17 @@ /* * Copyright 2017 Electronic Arts Inc. * - * Licensed under the Apache License, Version 2.0 + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. * */ package com.linkedin.drelephant.tez; @@ -14,39 +24,39 @@ public class TezTaskLevelAggregatedMetricsTest { - @Test - public void testZeroTasks() { - TezTaskData taskData[] = {}; - TezTaskLevelAggregatedMetrics taskMetrics = new TezTaskLevelAggregatedMetrics(taskData, 0, 0); - Assert.assertEquals(taskMetrics.getDelay(), 0); - Assert.assertEquals(taskMetrics.getResourceUsed(), 0); - Assert.assertEquals(taskMetrics.getResourceWasted(), 0); - } + @Test + public void testZeroTasks() { + TezTaskData taskData[] = {}; + TezTaskLevelAggregatedMetrics taskMetrics = new TezTaskLevelAggregatedMetrics(taskData, 0, 0); + Assert.assertEquals(taskMetrics.getDelay(), 0); + Assert.assertEquals(taskMetrics.getResourceUsed(), 0); + Assert.assertEquals(taskMetrics.getResourceWasted(), 0); + } - @Test - public void testNullTaskArray() { - TezTaskLevelAggregatedMetrics taskMetrics = new TezTaskLevelAggregatedMetrics(null, 0, 0); - Assert.assertEquals(taskMetrics.getDelay(), 0); - Assert.assertEquals(taskMetrics.getResourceUsed(), 0); - Assert.assertEquals(taskMetrics.getResourceWasted(), 0); - } + @Test + public void testNullTaskArray() { + TezTaskLevelAggregatedMetrics taskMetrics = new TezTaskLevelAggregatedMetrics(null, 0, 0); + Assert.assertEquals(taskMetrics.getDelay(), 0); + Assert.assertEquals(taskMetrics.getResourceUsed(), 0); + Assert.assertEquals(taskMetrics.getResourceWasted(), 0); + } - @Test - public void testTaskLevelData() { - TezTaskData taskData[] = new TezTaskData[3]; - TezCounterData counterData = new TezCounterData(); - counterData.set(TezCounterData.CounterName.PHYSICAL_MEMORY_BYTES, 655577088L); - counterData.set(TezCounterData.CounterName.VIRTUAL_MEMORY_BYTES, 3051589632L); - long time[] = {0,0,0,1464218501117L, 1464218534148L}; - taskData[0] = new TezTaskData("task", "id"); - taskData[0].setTimeAndCounter(time,counterData); - taskData[1] = new TezTaskData("task", "id"); - taskData[1].setTimeAndCounter(new long[5],counterData); - // Non-sampled task, which does not contain time and counter data - taskData[2] = new TezTaskData("task", "id"); - TezTaskLevelAggregatedMetrics taskMetrics = new TezTaskLevelAggregatedMetrics(taskData, 4096L, 1463218501117L); - Assert.assertEquals(taskMetrics.getDelay(), 1000000000L); - Assert.assertEquals(taskMetrics.getResourceUsed(), 135168L); - Assert.assertEquals(taskMetrics.getResourceWasted(), 66627L); - } + @Test + public void testTaskLevelData() { + TezTaskData taskData[] = new TezTaskData[3]; + TezCounterData counterData = new TezCounterData(); + counterData.set(TezCounterData.CounterName.PHYSICAL_MEMORY_BYTES, 655577088L); + counterData.set(TezCounterData.CounterName.VIRTUAL_MEMORY_BYTES, 3051589632L); + long time[] = {0,0,0,1464218501117L, 1464218534148L}; + taskData[0] = new TezTaskData("task", "id"); + taskData[0].setTimeAndCounter(time,counterData); + taskData[1] = new TezTaskData("task", "id"); + taskData[1].setTimeAndCounter(new long[5],counterData); + // Non-sampled task, which does not contain time and counter data + taskData[2] = new TezTaskData("task", "id"); + TezTaskLevelAggregatedMetrics taskMetrics = new TezTaskLevelAggregatedMetrics(taskData, 4096L, 1463218501117L); + Assert.assertEquals(taskMetrics.getDelay(), 1000000000L); + Assert.assertEquals(taskMetrics.getResourceUsed(), 135168L); + Assert.assertEquals(taskMetrics.getResourceWasted(), 66627L); + } } \ No newline at end of file diff --git a/test/com/linkedin/drelephant/tez/fetchers/TezFetcherTest.java b/test/com/linkedin/drelephant/tez/fetchers/TezFetcherTest.java index feaafad46..77c16a9ce 100644 --- a/test/com/linkedin/drelephant/tez/fetchers/TezFetcherTest.java +++ b/test/com/linkedin/drelephant/tez/fetchers/TezFetcherTest.java @@ -1,7 +1,17 @@ /* * Copyright 2017 Electronic Arts Inc. * - * Licensed under the Apache License, Version 2.0 + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. * */ package com.linkedin.drelephant.tez.fetchers; @@ -12,12 +22,12 @@ public class TezFetcherTest { - @Test - public void testDiagnosticMatcher() { - Matcher matcher = com.linkedin.drelephant.tez.fetchers.ThreadContextMR2.getDiagnosticMatcher("Task task_1443068695259_9143_m_000475 failed 1 time"); - Assert.assertEquals("Task[\\s\\u00A0]+(.*)[\\s\\u00A0]+failed[\\s\\u00A0]+([0-9])[\\s\\u00A0]+times[\\s\\u00A0]+", matcher.pattern().toString()); - Assert.assertEquals(false, matcher.matches()); - Assert.assertEquals(2, matcher.groupCount()); - } + @Test + public void testDiagnosticMatcher() { + Matcher matcher = com.linkedin.drelephant.tez.fetchers.ThreadContextMR2.getDiagnosticMatcher("Task task_1443068695259_9143_m_000475 failed 1 time"); + Assert.assertEquals("Task[\\s\\u00A0]+(.*)[\\s\\u00A0]+failed[\\s\\u00A0]+([0-9])[\\s\\u00A0]+times[\\s\\u00A0]+", matcher.pattern().toString()); + Assert.assertEquals(false, matcher.matches()); + Assert.assertEquals(2, matcher.groupCount()); + } } \ No newline at end of file diff --git a/test/com/linkedin/drelephant/tez/heuristics/MapperDataSkewHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/MapperDataSkewHeuristicTest.java index 2b8c97728..8792e70f4 100644 --- a/test/com/linkedin/drelephant/tez/heuristics/MapperDataSkewHeuristicTest.java +++ b/test/com/linkedin/drelephant/tez/heuristics/MapperDataSkewHeuristicTest.java @@ -1,7 +1,17 @@ /* * Copyright 2017 Electronic Arts Inc. * - * Licensed under the Apache License, Version 2.0 + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. * */ package com.linkedin.drelephant.tez.heuristics; @@ -21,116 +31,116 @@ public class MapperDataSkewHeuristicTest extends TestCase { - private static final long UNITSIZE = HDFSContext.HDFS_BLOCK_SIZE / 64; //1MB - private static final long UNITSIZETIME = 1000000; //1000sec + private static final long UNITSIZE = HDFSContext.HDFS_BLOCK_SIZE / 64; //1MB + private static final long UNITSIZETIME = 1000000; //1000sec - private static Map paramsMap = new HashMap(); - private static Heuristic _heuristic = new MapperDataSkewHeuristic(new HeuristicConfigurationData("test_heuristic", - "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); + private static Map paramsMap = new HashMap(); + private static Heuristic _heuristic = new MapperDataSkewHeuristic(new HeuristicConfigurationData("test_heuristic", + "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); - public void testCritical() throws IOException { - assertEquals(Severity.CRITICAL, analyzeJob(200, 200, 1 * UNITSIZE, 100 * UNITSIZE)); - } + public void testCritical() throws IOException { + assertEquals(Severity.CRITICAL, analyzeJob(200, 200, 1 * UNITSIZE, 100 * UNITSIZE)); + } - public void testSevere() throws IOException { - assertEquals(Severity.SEVERE, analyzeJob(200, 200, 10 * UNITSIZE, 100 * UNITSIZE)); - } + public void testSevere() throws IOException { + assertEquals(Severity.SEVERE, analyzeJob(200, 200, 10 * UNITSIZE, 100 * UNITSIZE)); + } - public void testModerate() throws IOException { - assertEquals(Severity.MODERATE, analyzeJob(200, 200, 20 * UNITSIZE, 100 * UNITSIZE)); - } + public void testModerate() throws IOException { + assertEquals(Severity.MODERATE, analyzeJob(200, 200, 20 * UNITSIZE, 100 * UNITSIZE)); + } - public void testLow() throws IOException { - assertEquals(Severity.LOW, analyzeJob(200, 200, 30 * UNITSIZE, 100 * UNITSIZE)); - } + public void testLow() throws IOException { + assertEquals(Severity.LOW, analyzeJob(200, 200, 30 * UNITSIZE, 100 * UNITSIZE)); + } - public void testNone() throws IOException { - assertEquals(Severity.NONE, analyzeJob(200, 200, 50 * UNITSIZE, 100 * UNITSIZE)); - } + public void testNone() throws IOException { + assertEquals(Severity.NONE, analyzeJob(200, 200, 50 * UNITSIZE, 100 * UNITSIZE)); + } - public void testSmallFiles() throws IOException { - assertEquals(Severity.NONE, analyzeJob(200, 200, 1 * UNITSIZE, 5 * UNITSIZE)); - } + public void testSmallFiles() throws IOException { + assertEquals(Severity.NONE, analyzeJob(200, 200, 1 * UNITSIZE, 5 * UNITSIZE)); + } - public void testSmallTasks() throws IOException { - assertEquals(Severity.NONE, analyzeJob(5, 5, 10 * UNITSIZE, 100 * UNITSIZE)); - } + public void testSmallTasks() throws IOException { + assertEquals(Severity.NONE, analyzeJob(5, 5, 10 * UNITSIZE, 100 * UNITSIZE)); + } - public void testCriticalTime() throws IOException { - assertEquals(Severity.CRITICAL, analyzeJobTime(200, 200, 1 * UNITSIZETIME, 100 * UNITSIZETIME)); - } + public void testCriticalTime() throws IOException { + assertEquals(Severity.CRITICAL, analyzeJobTime(200, 200, 1 * UNITSIZETIME, 100 * UNITSIZETIME)); + } - public void testSevereTime() throws IOException { - assertEquals(Severity.SEVERE, analyzeJobTime(200, 200, 10 * UNITSIZETIME, 100 * UNITSIZETIME)); - } + public void testSevereTime() throws IOException { + assertEquals(Severity.SEVERE, analyzeJobTime(200, 200, 10 * UNITSIZETIME, 100 * UNITSIZETIME)); + } - public void testModerateTime() throws IOException { - assertEquals(Severity.MODERATE, analyzeJobTime(200, 200, 20 * UNITSIZETIME, 100 * UNITSIZETIME)); - } + public void testModerateTime() throws IOException { + assertEquals(Severity.MODERATE, analyzeJobTime(200, 200, 20 * UNITSIZETIME, 100 * UNITSIZETIME)); + } - public void testLowTime() throws IOException { - assertEquals(Severity.LOW, analyzeJobTime(200, 200, 30 * UNITSIZETIME, 100 * UNITSIZETIME)); - } + public void testLowTime() throws IOException { + assertEquals(Severity.LOW, analyzeJobTime(200, 200, 30 * UNITSIZETIME, 100 * UNITSIZETIME)); + } - public void testNoneTime() throws IOException { - assertEquals(Severity.NONE, analyzeJobTime(200, 200, 50 * UNITSIZETIME, 100 * UNITSIZETIME)); - } + public void testNoneTime() throws IOException { + assertEquals(Severity.NONE, analyzeJobTime(200, 200, 50 * UNITSIZETIME, 100 * UNITSIZETIME)); + } - public void testSmallTasksTime() throws IOException { - assertEquals(Severity.NONE, analyzeJobTime(5, 5, 10 * UNITSIZETIME, 100 * UNITSIZETIME)); - } + public void testSmallTasksTime() throws IOException { + assertEquals(Severity.NONE, analyzeJobTime(5, 5, 10 * UNITSIZETIME, 100 * UNITSIZETIME)); + } - private Severity analyzeJob(int numSmallTasks, int numLargeTasks, long smallInputSize, long largeInputSize) - throws IOException { - TezCounterData jobCounter = new TezCounterData(); - TezTaskData[] mappers = new TezTaskData[numSmallTasks + numLargeTasks + 1]; - - TezCounterData smallCounter = new TezCounterData(); - smallCounter.set(TezCounterData.CounterName.HDFS_BYTES_READ, smallInputSize); - - TezCounterData largeCounter = new TezCounterData(); - largeCounter.set(TezCounterData.CounterName.S3A_BYTES_READ, largeInputSize); - - int i = 0; - for (; i < numSmallTasks; i++) { - mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); - mappers[i].setTimeAndCounter(new long[5], smallCounter); - } - for (; i < numSmallTasks + numLargeTasks; i++) { - mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); - mappers[i].setTimeAndCounter(new long[5], largeCounter); - } - // Non-sampled task, which does not contain time and counter data - mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); - - TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setMapTaskData(mappers); - HeuristicResult result = _heuristic.apply(data); - return result.getSeverity(); + private Severity analyzeJob(int numSmallTasks, int numLargeTasks, long smallInputSize, long largeInputSize) + throws IOException { + TezCounterData jobCounter = new TezCounterData(); + TezTaskData[] mappers = new TezTaskData[numSmallTasks + numLargeTasks + 1]; + TezCounterData smallCounter = new TezCounterData(); + smallCounter.set(TezCounterData.CounterName.HDFS_BYTES_READ, smallInputSize); + + TezCounterData largeCounter = new TezCounterData(); + largeCounter.set(TezCounterData.CounterName.S3A_BYTES_READ, largeInputSize); + + int i = 0; + for (; i < numSmallTasks; i++) { + mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + mappers[i].setTimeAndCounter(new long[5], smallCounter); + } + for (; i < numSmallTasks + numLargeTasks; i++) { + mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + mappers[i].setTimeAndCounter(new long[5], largeCounter); } + // Non-sampled task, which does not contain time and counter data + mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); - private Severity analyzeJobTime(int numSmallTasks, int numLongTasks, long smallTimeTaken, long longTimeTaken) - throws IOException { - TezCounterData jobCounter = new TezCounterData(); - TezTaskData[] mappers = new TezTaskData[numSmallTasks + numLongTasks + 1]; - - int i = 0; - for (; i < numSmallTasks; i++) { - mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); - mappers[i].setTotalTimeMs(smallTimeTaken, true); - mappers[i].setCounter(jobCounter); - } - for (; i < numSmallTasks + numLongTasks; i++) { - mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); - mappers[i].setTotalTimeMs(longTimeTaken, true); - mappers[i].setCounter(jobCounter); - } - // Non-sampled task, which does not contain time data - mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); - - TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setMapTaskData(mappers); - HeuristicResult result = _heuristic.apply(data); - return result.getSeverity(); + TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setMapTaskData(mappers); + HeuristicResult result = _heuristic.apply(data); + return result.getSeverity(); + } + + private Severity analyzeJobTime(int numSmallTasks, int numLongTasks, long smallTimeTaken, long longTimeTaken) + throws IOException { + TezCounterData jobCounter = new TezCounterData(); + TezTaskData[] mappers = new TezTaskData[numSmallTasks + numLongTasks + 1]; + + int i = 0; + for (; i < numSmallTasks; i++) { + mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + mappers[i].setTotalTimeMs(smallTimeTaken, true); + mappers[i].setCounter(jobCounter); + } + for (; i < numSmallTasks + numLongTasks; i++) { + mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + mappers[i].setTotalTimeMs(longTimeTaken, true); + mappers[i].setCounter(jobCounter); } + // Non-sampled task, which does not contain time data + mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + + TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setMapTaskData(mappers); + HeuristicResult result = _heuristic.apply(data); + return result.getSeverity(); + + } } \ No newline at end of file diff --git a/test/com/linkedin/drelephant/tez/heuristics/MapperGCHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/MapperGCHeuristicTest.java index c4a47d89a..52cea2035 100644 --- a/test/com/linkedin/drelephant/tez/heuristics/MapperGCHeuristicTest.java +++ b/test/com/linkedin/drelephant/tez/heuristics/MapperGCHeuristicTest.java @@ -1,7 +1,17 @@ /* * Copyright 2017 Electronic Arts Inc. * - * Licensed under the Apache License, Version 2.0 + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. * */ package com.linkedin.drelephant.tez.heuristics; @@ -22,53 +32,53 @@ public class MapperGCHeuristicTest extends TestCase { - private static Map paramsMap = new HashMap(); + private static Map paramsMap = new HashMap(); - private static Heuristic _heuristic = new MapperGCHeuristic(new HeuristicConfigurationData("test_heuristic", - "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); + private static Heuristic _heuristic = new MapperGCHeuristic(new HeuristicConfigurationData("test_heuristic", + "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); - private static int NUMTASKS = 100; + private static int NUMTASKS = 100; - public void testGCCritical() throws IOException { - assertEquals(Severity.CRITICAL, analyzeJob(1000000, 50000, 2000)); - } + public void testGCCritical() throws IOException { + assertEquals(Severity.CRITICAL, analyzeJob(1000000, 50000, 2000)); + } - public void testGCSevere() throws IOException { - assertEquals(Severity.SEVERE, analyzeJob(1000000, 50000, 1500)); - } + public void testGCSevere() throws IOException { + assertEquals(Severity.SEVERE, analyzeJob(1000000, 50000, 1500)); + } - public void testGCModerate() throws IOException { - assertEquals(Severity.MODERATE, analyzeJob(1000000, 50000, 1000)); - } + public void testGCModerate() throws IOException { + assertEquals(Severity.MODERATE, analyzeJob(1000000, 50000, 1000)); + } - public void testGCNone() throws IOException { - assertEquals(Severity.NONE, analyzeJob(1000000, 50000, 300)); - } - - public void testShortTasksNone() throws IOException { - assertEquals(Severity.NONE, analyzeJob(100000, 50000, 2000)); - } + public void testGCNone() throws IOException { + assertEquals(Severity.NONE, analyzeJob(1000000, 50000, 300)); + } + public void testShortTasksNone() throws IOException { + assertEquals(Severity.NONE, analyzeJob(100000, 50000, 2000)); + } - private Severity analyzeJob(long runtimeMs, long cpuMs, long gcMs) throws IOException { - TezCounterData jobCounter = new TezCounterData(); - TezTaskData[] mappers = new TezTaskData[NUMTASKS + 1]; - TezCounterData counter = new TezCounterData(); - counter.set(TezCounterData.CounterName.CPU_MILLISECONDS, cpuMs); - counter.set(TezCounterData.CounterName.GC_TIME_MILLIS, gcMs); + private Severity analyzeJob(long runtimeMs, long cpuMs, long gcMs) throws IOException { + TezCounterData jobCounter = new TezCounterData(); + TezTaskData[] mappers = new TezTaskData[NUMTASKS + 1]; - int i = 0; - for (; i < NUMTASKS; i++) { - mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); - mappers[i].setTimeAndCounter(new long[]{runtimeMs, 0 , 0, 0, 0}, counter); - } - // Non-sampled task, which does not contain time and counter data - mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + TezCounterData counter = new TezCounterData(); + counter.set(TezCounterData.CounterName.CPU_MILLISECONDS, cpuMs); + counter.set(TezCounterData.CounterName.GC_TIME_MILLIS, gcMs); - TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setMapTaskData(mappers); - HeuristicResult result = _heuristic.apply(data); - return result.getSeverity(); + int i = 0; + for (; i < NUMTASKS; i++) { + mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + mappers[i].setTimeAndCounter(new long[]{runtimeMs, 0 , 0, 0, 0}, counter); } + // Non-sampled task, which does not contain time and counter data + mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + + TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setMapTaskData(mappers); + HeuristicResult result = _heuristic.apply(data); + return result.getSeverity(); + } } \ No newline at end of file diff --git a/test/com/linkedin/drelephant/tez/heuristics/MapperMemoryHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/MapperMemoryHeuristicTest.java index 0a0bbb263..72e426743 100644 --- a/test/com/linkedin/drelephant/tez/heuristics/MapperMemoryHeuristicTest.java +++ b/test/com/linkedin/drelephant/tez/heuristics/MapperMemoryHeuristicTest.java @@ -1,7 +1,17 @@ /* * Copyright 2017 Electronic Arts Inc. * - * Licensed under the Apache License, Version 2.0 + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. * */ package com.linkedin.drelephant.tez.heuristics; @@ -24,61 +34,61 @@ public class MapperMemoryHeuristicTest extends TestCase{ - private static Map paramsMap = new HashMap(); - private static Heuristic _heuristic = new MapperMemoryHeuristic(new HeuristicConfigurationData("test_heuristic", - "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); + private static Map paramsMap = new HashMap(); + private static Heuristic _heuristic = new MapperMemoryHeuristic(new HeuristicConfigurationData("test_heuristic", + "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); - private int NUMTASKS = 100; + private int NUMTASKS = 100; - public void testLargeContainerSizeCritical() throws IOException { - assertEquals(Severity.CRITICAL, analyzeJob(2048, 8192)); - } + public void testLargeContainerSizeCritical() throws IOException { + assertEquals(Severity.CRITICAL, analyzeJob(2048, 8192)); + } - public void testLargeContainerSizeSevere() throws IOException { - assertEquals(Severity.SEVERE, analyzeJob(3072, 8192)); - } + public void testLargeContainerSizeSevere() throws IOException { + assertEquals(Severity.SEVERE, analyzeJob(3072, 8192)); + } - public void testLargeContainerSizeModerate() throws IOException { - assertEquals(Severity.MODERATE, analyzeJob(4096, 8192)); - } + public void testLargeContainerSizeModerate() throws IOException { + assertEquals(Severity.MODERATE, analyzeJob(4096, 8192)); + } - public void testLargeContainerSizeNone() throws IOException { - assertEquals(Severity.NONE, analyzeJob(6144, 8192)); - } + public void testLargeContainerSizeNone() throws IOException { + assertEquals(Severity.NONE, analyzeJob(6144, 8192)); + } - // If the task use default container size, it should not be flagged - // Not using Default Container param, will calculate severity irrespective of default container size Chaning NONE -> CRITICAL - public void testDefaultContainerNone() throws IOException { - assertEquals(Severity.CRITICAL, analyzeJob(256, 2048)); - } + // If the task use default container size, it should not be flagged + // Not using Default Container param, will calculate severity irrespective of default container size Chaning NONE -> CRITICAL + public void testDefaultContainerNone() throws IOException { + assertEquals(Severity.CRITICAL, analyzeJob(256, 2048)); + } - // Not using Default Container param, will calculate severity irrespective of default container size Chaning NONE -> MODERATE - public void testDefaultContainerNoneMore() throws IOException { - assertEquals(Severity.MODERATE, analyzeJob(1024, 2048)); - } + // Not using Default Container param, will calculate severity irrespective of default container size Chaning NONE -> MODERATE + public void testDefaultContainerNoneMore() throws IOException { + assertEquals(Severity.MODERATE, analyzeJob(1024, 2048)); + } + + private Severity analyzeJob(long taskAvgMemMB, long containerMemMB) throws IOException { + TezCounterData jobCounter = new TezCounterData(); + TezTaskData[] mappers = new TezTaskData[NUMTASKS + 1]; + + TezCounterData counter = new TezCounterData(); + counter.set(TezCounterData.CounterName.PHYSICAL_MEMORY_BYTES, taskAvgMemMB* FileUtils.ONE_MB); + + Properties p = new Properties(); + p.setProperty(MapperMemoryHeuristic.MAPPER_MEMORY_CONF, Long.toString(containerMemMB)); - private Severity analyzeJob(long taskAvgMemMB, long containerMemMB) throws IOException { - TezCounterData jobCounter = new TezCounterData(); - TezTaskData[] mappers = new TezTaskData[NUMTASKS + 1]; - - TezCounterData counter = new TezCounterData(); - counter.set(TezCounterData.CounterName.PHYSICAL_MEMORY_BYTES, taskAvgMemMB* FileUtils.ONE_MB); - - Properties p = new Properties(); - p.setProperty(MapperMemoryHeuristic.MAPPER_MEMORY_CONF, Long.toString(containerMemMB)); - - int i = 0; - for (; i < NUMTASKS; i++) { - mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); - mappers[i].setTime(new long[5]); - mappers[i].setCounter(counter); - } - // Non-sampled task, which does not contain time and counter data - mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); - - TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setMapTaskData(mappers); - data.setConf(p); - HeuristicResult result = _heuristic.apply(data); - return result.getSeverity(); + int i = 0; + for (; i < NUMTASKS; i++) { + mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + mappers[i].setTime(new long[5]); + mappers[i].setCounter(counter); } + // Non-sampled task, which does not contain time and counter data + mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + + TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setMapTaskData(mappers); + data.setConf(p); + HeuristicResult result = _heuristic.apply(data); + return result.getSeverity(); + } } \ No newline at end of file diff --git a/test/com/linkedin/drelephant/tez/heuristics/MapperSpeedHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/MapperSpeedHeuristicTest.java index a2dba6c64..2d21f856a 100644 --- a/test/com/linkedin/drelephant/tez/heuristics/MapperSpeedHeuristicTest.java +++ b/test/com/linkedin/drelephant/tez/heuristics/MapperSpeedHeuristicTest.java @@ -1,7 +1,17 @@ /* * Copyright 2017 Electronic Arts Inc. * - * Licensed under the Apache License, Version 2.0 + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. * */ package com.linkedin.drelephant.tez.heuristics; @@ -24,67 +34,67 @@ public class MapperSpeedHeuristicTest extends TestCase { - private static Map paramsMap = new HashMap(); - private static Heuristic _heuristic = new MapperSpeedHeuristic(new HeuristicConfigurationData("test_heuristic", - "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); + private static Map paramsMap = new HashMap(); + private static Heuristic _heuristic = new MapperSpeedHeuristic(new HeuristicConfigurationData("test_heuristic", + "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); - private static final long MB_IN_BYTES = FileUtils.ONE_MB; - private static final long MINUTE_IN_MS = Statistics.MINUTE_IN_MS; - private static final int NUMTASKS = 100; + private static final long MB_IN_BYTES = FileUtils.ONE_MB; + private static final long MINUTE_IN_MS = Statistics.MINUTE_IN_MS; + private static final int NUMTASKS = 100; - public void testCritical() throws IOException { - long runtime = 120 * MINUTE_IN_MS; - long speed_factor = (runtime * MB_IN_BYTES) / 1000; - assertEquals(Severity.CRITICAL, analyzeJob(runtime, 1 * speed_factor)); - } - - public void testSevere() throws IOException { - long runtime = 120 * MINUTE_IN_MS; - long speed_factor = (runtime * MB_IN_BYTES) / 1000; - assertEquals(Severity.SEVERE, analyzeJob(runtime, 4 * speed_factor)); - } + public void testCritical() throws IOException { + long runtime = 120 * MINUTE_IN_MS; + long speed_factor = (runtime * MB_IN_BYTES) / 1000; + assertEquals(Severity.CRITICAL, analyzeJob(runtime, 1 * speed_factor)); + } - public void testModerate() throws IOException { - long runtime = 120 * MINUTE_IN_MS; - long speed_factor = (runtime * MB_IN_BYTES) / 1000; - assertEquals(Severity.MODERATE, analyzeJob(runtime, 13 * speed_factor)); - } + public void testSevere() throws IOException { + long runtime = 120 * MINUTE_IN_MS; + long speed_factor = (runtime * MB_IN_BYTES) / 1000; + assertEquals(Severity.SEVERE, analyzeJob(runtime, 4 * speed_factor)); + } - public void testLow() throws IOException { - long runtime = 120 * MINUTE_IN_MS; - long speed_factor = (runtime * MB_IN_BYTES) / 1000; - assertEquals(Severity.LOW, analyzeJob(runtime, 50 * speed_factor)); - } + public void testModerate() throws IOException { + long runtime = 120 * MINUTE_IN_MS; + long speed_factor = (runtime * MB_IN_BYTES) / 1000; + assertEquals(Severity.MODERATE, analyzeJob(runtime, 13 * speed_factor)); + } - public void testNone() throws IOException { - long runtime = 120 * MINUTE_IN_MS; - long speed_factor = (runtime * MB_IN_BYTES) / 1000; - assertEquals(Severity.NONE, analyzeJob(runtime, 51 * speed_factor)); - } + public void testLow() throws IOException { + long runtime = 120 * MINUTE_IN_MS; + long speed_factor = (runtime * MB_IN_BYTES) / 1000; + assertEquals(Severity.LOW, analyzeJob(runtime, 50 * speed_factor)); + } - public void testShortTask() throws IOException { - long runtime = 2 * MINUTE_IN_MS; - long speed_factor = (runtime * MB_IN_BYTES) / 1000; - assertEquals(Severity.NONE, analyzeJob(runtime, 1 * speed_factor)); - } + public void testNone() throws IOException { + long runtime = 120 * MINUTE_IN_MS; + long speed_factor = (runtime * MB_IN_BYTES) / 1000; + assertEquals(Severity.NONE, analyzeJob(runtime, 51 * speed_factor)); + } - private Severity analyzeJob(long runtimeMs, long readBytes) throws IOException { - TezCounterData jobCounter = new TezCounterData(); - TezTaskData[] mappers = new TezTaskData[NUMTASKS + 1]; + public void testShortTask() throws IOException { + long runtime = 2 * MINUTE_IN_MS; + long speed_factor = (runtime * MB_IN_BYTES) / 1000; + assertEquals(Severity.NONE, analyzeJob(runtime, 1 * speed_factor)); + } - TezCounterData counter = new TezCounterData(); - counter.set(TezCounterData.CounterName.HDFS_BYTES_READ, readBytes / 2); - counter.set(TezCounterData.CounterName.S3A_BYTES_READ, readBytes / 2); + private Severity analyzeJob(long runtimeMs, long readBytes) throws IOException { + TezCounterData jobCounter = new TezCounterData(); + TezTaskData[] mappers = new TezTaskData[NUMTASKS + 1]; - int i = 0; - for (; i < NUMTASKS; i++) { - mappers[i] = new TezTaskData(counter, new long[] { runtimeMs, 0, 0 ,0, 0}); - } - // Non-sampled task, which does not contain time and counter data - mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + TezCounterData counter = new TezCounterData(); + counter.set(TezCounterData.CounterName.HDFS_BYTES_READ, readBytes / 2); + counter.set(TezCounterData.CounterName.S3A_BYTES_READ, readBytes / 2); - TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setMapTaskData(mappers); - HeuristicResult result = _heuristic.apply(data); - return result.getSeverity(); + int i = 0; + for (; i < NUMTASKS; i++) { + mappers[i] = new TezTaskData(counter, new long[] { runtimeMs, 0, 0 ,0, 0}); } + // Non-sampled task, which does not contain time and counter data + mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + + TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setMapTaskData(mappers); + HeuristicResult result = _heuristic.apply(data); + return result.getSeverity(); + } } \ No newline at end of file diff --git a/test/com/linkedin/drelephant/tez/heuristics/MapperSpillHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/MapperSpillHeuristicTest.java index 736fd9866..62831a727 100644 --- a/test/com/linkedin/drelephant/tez/heuristics/MapperSpillHeuristicTest.java +++ b/test/com/linkedin/drelephant/tez/heuristics/MapperSpillHeuristicTest.java @@ -1,7 +1,17 @@ /* * Copyright 2017 Electronic Arts Inc. * - * Licensed under the Apache License, Version 2.0 + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. * */ package com.linkedin.drelephant.tez.heuristics; @@ -23,58 +33,58 @@ public class MapperSpillHeuristicTest extends TestCase{ - private static Map paramsMap = new HashMap(); - private static Heuristic _heuristic = new MapperSpillHeuristic(new HeuristicConfigurationData("test_heuristic", - "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); + private static Map paramsMap = new HashMap(); + private static Heuristic _heuristic = new MapperSpillHeuristic(new HeuristicConfigurationData("test_heuristic", + "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); - public void testCritical() throws IOException { - // Spill ratio 3.0, 1000 tasks - assertEquals(Severity.CRITICAL, analyzeJob(3000, 1000, 1000)); - } - - public void testSevere() throws IOException { - // Spill ratio 2.5, 1000 tasks - assertEquals(Severity.SEVERE, analyzeJob(2500, 1000, 1000)); - } + public void testCritical() throws IOException { + // Spill ratio 3.0, 1000 tasks + assertEquals(Severity.CRITICAL, analyzeJob(3000, 1000, 1000)); + } - public void testModerate() throws IOException { - // Spill ratio 2.3, 1000 tasks - assertEquals(Severity.MODERATE, analyzeJob(2300, 1000, 1000)); - } + public void testSevere() throws IOException { + // Spill ratio 2.5, 1000 tasks + assertEquals(Severity.SEVERE, analyzeJob(2500, 1000, 1000)); + } - public void testLow() throws IOException { - // Spill ratio 2.1, 1000 tasks - assertEquals(Severity.LOW, analyzeJob(2100, 1000, 1000)); - } + public void testModerate() throws IOException { + // Spill ratio 2.3, 1000 tasks + assertEquals(Severity.MODERATE, analyzeJob(2300, 1000, 1000)); + } - public void testNone() throws IOException { - // Spill ratio 1.0, 1000 tasks - assertEquals(Severity.NONE, analyzeJob(1000, 1000, 1000)); - } + public void testLow() throws IOException { + // Spill ratio 2.1, 1000 tasks + assertEquals(Severity.LOW, analyzeJob(2100, 1000, 1000)); + } - public void testSmallNumTasks() throws IOException { - // Spill ratio 3.0, should be critical, but number of task is small(10), final result is NONE - assertEquals(Severity.NONE, analyzeJob(3000, 1000, 10)); - } + public void testNone() throws IOException { + // Spill ratio 1.0, 1000 tasks + assertEquals(Severity.NONE, analyzeJob(1000, 1000, 1000)); + } - private Severity analyzeJob(long spilledRecords, long mapRecords, int numTasks) throws IOException { - TezCounterData jobCounter = new TezCounterData(); - TezTaskData[] mappers = new TezTaskData[numTasks + 1]; + public void testSmallNumTasks() throws IOException { + // Spill ratio 3.0, should be critical, but number of task is small(10), final result is NONE + assertEquals(Severity.NONE, analyzeJob(3000, 1000, 10)); + } - TezCounterData counter = new TezCounterData(); - counter.set(TezCounterData.CounterName.SPILLED_RECORDS, spilledRecords); - counter.set(TezCounterData.CounterName.OUTPUT_RECORDS, mapRecords); + private Severity analyzeJob(long spilledRecords, long mapRecords, int numTasks) throws IOException { + TezCounterData jobCounter = new TezCounterData(); + TezTaskData[] mappers = new TezTaskData[numTasks + 1]; - int i = 0; - for (; i < numTasks; i++) { - mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); - mappers[i].setTimeAndCounter(new long[5], counter); - } - // Non-sampled task, which does not contain time and counter data - mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + TezCounterData counter = new TezCounterData(); + counter.set(TezCounterData.CounterName.SPILLED_RECORDS, spilledRecords); + counter.set(TezCounterData.CounterName.OUTPUT_RECORDS, mapRecords); - TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setMapTaskData(mappers); - HeuristicResult result = _heuristic.apply(data); - return result.getSeverity(); + int i = 0; + for (; i < numTasks; i++) { + mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + mappers[i].setTimeAndCounter(new long[5], counter); } + // Non-sampled task, which does not contain time and counter data + mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + + TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setMapTaskData(mappers); + HeuristicResult result = _heuristic.apply(data); + return result.getSeverity(); + } } \ No newline at end of file diff --git a/test/com/linkedin/drelephant/tez/heuristics/MapperTimeHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/MapperTimeHeuristicTest.java index 824398800..39c6ec891 100644 --- a/test/com/linkedin/drelephant/tez/heuristics/MapperTimeHeuristicTest.java +++ b/test/com/linkedin/drelephant/tez/heuristics/MapperTimeHeuristicTest.java @@ -1,7 +1,17 @@ /* * Copyright 2017 Electronic Arts Inc. * - * Licensed under the Apache License, Version 2.0 + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. * */ package com.linkedin.drelephant.tez.heuristics; @@ -24,78 +34,78 @@ public class MapperTimeHeuristicTest extends TestCase { - private static final long DUMMY_INPUT_SIZE = 0; + private static final long DUMMY_INPUT_SIZE = 0; - private static Map paramsMap = new HashMap(); - private static Heuristic _heuristic = new MapperTimeHeuristic(new HeuristicConfigurationData("test_heuristic", - "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); + private static Map paramsMap = new HashMap(); + private static Heuristic _heuristic = new MapperTimeHeuristic(new HeuristicConfigurationData("test_heuristic", + "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); - // Test batch 1: Large runtime. Heuristic is not affected by various number of tasks */ + // Test batch 1: Large runtime. Heuristic is not affected by various number of tasks */ - public void testLongRuntimeTasksCritical() throws IOException { - // Should decrease split size and increase number of tasks - assertEquals(Severity.CRITICAL, analyzeJob(10, 120 * Statistics.MINUTE_IN_MS)); - } + public void testLongRuntimeTasksCritical() throws IOException { + // Should decrease split size and increase number of tasks + assertEquals(Severity.CRITICAL, analyzeJob(10, 120 * Statistics.MINUTE_IN_MS)); + } - public void testLongRuntimeTasksCriticalMore() throws IOException { - // Should decrease split size and increase number of tasks - assertEquals(Severity.CRITICAL, analyzeJob(1000, 120 * Statistics.MINUTE_IN_MS)); - } + public void testLongRuntimeTasksCriticalMore() throws IOException { + // Should decrease split size and increase number of tasks + assertEquals(Severity.CRITICAL, analyzeJob(1000, 120 * Statistics.MINUTE_IN_MS)); + } - public void testLongRuntimeTasksSevere() throws IOException { - // Should decrease split size and increase number of tasks - assertEquals(Severity.SEVERE, analyzeJob(10, 60 * Statistics.MINUTE_IN_MS)); - } + public void testLongRuntimeTasksSevere() throws IOException { + // Should decrease split size and increase number of tasks + assertEquals(Severity.SEVERE, analyzeJob(10, 60 * Statistics.MINUTE_IN_MS)); + } - public void testLongRuntimeTasksSevereMore() throws IOException { - // Should decrease split size and increase number of tasks - assertEquals(Severity.SEVERE, analyzeJob(1000, 60 * Statistics.MINUTE_IN_MS)); - } + public void testLongRuntimeTasksSevereMore() throws IOException { + // Should decrease split size and increase number of tasks + assertEquals(Severity.SEVERE, analyzeJob(1000, 60 * Statistics.MINUTE_IN_MS)); + } - // Test batch 2: Short runtime and various number of tasks + // Test batch 2: Short runtime and various number of tasks - public void testShortRuntimeTasksCritical() throws IOException { - // Should increase split size and decrease number of tasks - assertEquals(Severity.CRITICAL, analyzeJob(1000, 1 * Statistics.MINUTE_IN_MS)); - } + public void testShortRuntimeTasksCritical() throws IOException { + // Should increase split size and decrease number of tasks + assertEquals(Severity.CRITICAL, analyzeJob(1000, 1 * Statistics.MINUTE_IN_MS)); + } - public void testShortRuntimeTasksSevere() throws IOException { - // Should increase split size and decrease number of tasks - assertEquals(Severity.SEVERE, analyzeJob(500, 1 * Statistics.MINUTE_IN_MS)); - } - - public void testShortRuntimeTasksModerate() throws IOException { - assertEquals(Severity.MODERATE, analyzeJob(101, 1 * Statistics.MINUTE_IN_MS)); - } + public void testShortRuntimeTasksSevere() throws IOException { + // Should increase split size and decrease number of tasks + assertEquals(Severity.SEVERE, analyzeJob(500, 1 * Statistics.MINUTE_IN_MS)); + } - public void testShortRuntimeTasksLow() throws IOException { - assertEquals(Severity.LOW, analyzeJob(50, 1 * Statistics.MINUTE_IN_MS)); - } + public void testShortRuntimeTasksModerate() throws IOException { + assertEquals(Severity.MODERATE, analyzeJob(101, 1 * Statistics.MINUTE_IN_MS)); + } - public void testShortRuntimeTasksNone() throws IOException { - // Small file with small number of tasks and short runtime. This should be the common case. - assertEquals(Severity.NONE, analyzeJob(5, 1 * Statistics.MINUTE_IN_MS)); - } + public void testShortRuntimeTasksLow() throws IOException { + assertEquals(Severity.LOW, analyzeJob(50, 1 * Statistics.MINUTE_IN_MS)); + } - private Severity analyzeJob(int numTasks, long runtime) throws IOException { - TezCounterData jobCounter = new TezCounterData(); - TezTaskData[] mappers = new TezTaskData[numTasks + 1]; + public void testShortRuntimeTasksNone() throws IOException { + // Small file with small number of tasks and short runtime. This should be the common case. + assertEquals(Severity.NONE, analyzeJob(5, 1 * Statistics.MINUTE_IN_MS)); + } - TezCounterData taskCounter = new TezCounterData(); - taskCounter.set(TezCounterData.CounterName.S3A_BYTES_READ, DUMMY_INPUT_SIZE / 4); + private Severity analyzeJob(int numTasks, long runtime) throws IOException { + TezCounterData jobCounter = new TezCounterData(); + TezTaskData[] mappers = new TezTaskData[numTasks + 1]; + TezCounterData taskCounter = new TezCounterData(); + taskCounter.set(TezCounterData.CounterName.S3A_BYTES_READ, DUMMY_INPUT_SIZE / 4); - int i = 0; - for (; i < numTasks; i++) { - mappers[i] = new TezTaskData(jobCounter,new long[] { runtime, 0, 0, 0, 0 }); - } - // Non-sampled task, which does not contain time and counter data - mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); - TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setMapTaskData(mappers); - HeuristicResult result = _heuristic.apply(data); - return result.getSeverity(); + int i = 0; + for (; i < numTasks; i++) { + mappers[i] = new TezTaskData(jobCounter,new long[] { runtime, 0, 0, 0, 0 }); } + // Non-sampled task, which does not contain time and counter data + mappers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + + TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setMapTaskData(mappers); + HeuristicResult result = _heuristic.apply(data); + return result.getSeverity(); + } } \ No newline at end of file diff --git a/test/com/linkedin/drelephant/tez/heuristics/ReducerDataSkewHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/ReducerDataSkewHeuristicTest.java index 9bd06838a..ff95e862a 100644 --- a/test/com/linkedin/drelephant/tez/heuristics/ReducerDataSkewHeuristicTest.java +++ b/test/com/linkedin/drelephant/tez/heuristics/ReducerDataSkewHeuristicTest.java @@ -1,7 +1,17 @@ /* * Copyright 2017 Electronic Arts Inc. * - * Licensed under the Apache License, Version 2.0 + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. * */ package com.linkedin.drelephant.tez.heuristics; @@ -19,114 +29,114 @@ public class ReducerDataSkewHeuristicTest extends TestCase{ - private static final long UNITSIZE = HDFSContext.HDFS_BLOCK_SIZE / 64; //1mb - private static final long UNITSIZETIME = 1000000; //1000sec + private static final long UNITSIZE = HDFSContext.HDFS_BLOCK_SIZE / 64; //1mb + private static final long UNITSIZETIME = 1000000; //1000sec - private static Map paramsMap = new HashMap(); - private static Heuristic _heuristic = new ReducerDataSkewHeuristic(new HeuristicConfigurationData("test_heuristic", - "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); + private static Map paramsMap = new HashMap(); + private static Heuristic _heuristic = new ReducerDataSkewHeuristic(new HeuristicConfigurationData("test_heuristic", + "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); - public void testCritical() throws IOException { - assertEquals(Severity.CRITICAL, analyzeJob(200, 200, 1 * UNITSIZE, 100 * UNITSIZE)); - } + public void testCritical() throws IOException { + assertEquals(Severity.CRITICAL, analyzeJob(200, 200, 1 * UNITSIZE, 100 * UNITSIZE)); + } - public void testSevere() throws IOException { - assertEquals(Severity.SEVERE, analyzeJob(200, 200, 10 * UNITSIZE, 100 * UNITSIZE)); - } + public void testSevere() throws IOException { + assertEquals(Severity.SEVERE, analyzeJob(200, 200, 10 * UNITSIZE, 100 * UNITSIZE)); + } - public void testModerate() throws IOException { - assertEquals(Severity.MODERATE, analyzeJob(200, 200, 20 * UNITSIZE, 100 * UNITSIZE)); - } + public void testModerate() throws IOException { + assertEquals(Severity.MODERATE, analyzeJob(200, 200, 20 * UNITSIZE, 100 * UNITSIZE)); + } - public void testLow() throws IOException { - assertEquals(Severity.LOW, analyzeJob(200, 200, 30 * UNITSIZE, 100 * UNITSIZE)); - } + public void testLow() throws IOException { + assertEquals(Severity.LOW, analyzeJob(200, 200, 30 * UNITSIZE, 100 * UNITSIZE)); + } - public void testNone() throws IOException { - assertEquals(Severity.NONE, analyzeJob(200, 200, 50 * UNITSIZE, 100 * UNITSIZE)); - } + public void testNone() throws IOException { + assertEquals(Severity.NONE, analyzeJob(200, 200, 50 * UNITSIZE, 100 * UNITSIZE)); + } - public void testSmallFiles() throws IOException { - assertEquals(Severity.NONE, analyzeJob(200, 200, 1 * UNITSIZE, 5 * UNITSIZE)); - } + public void testSmallFiles() throws IOException { + assertEquals(Severity.NONE, analyzeJob(200, 200, 1 * UNITSIZE, 5 * UNITSIZE)); + } - public void testSmallTasks() throws IOException { - assertEquals(Severity.NONE, analyzeJob(5, 5, 10 * UNITSIZE, 100 * UNITSIZE)); - } + public void testSmallTasks() throws IOException { + assertEquals(Severity.NONE, analyzeJob(5, 5, 10 * UNITSIZE, 100 * UNITSIZE)); + } - public void testCriticalTime() throws IOException { - assertEquals(Severity.CRITICAL, analyzeJobTime(200, 200, 1 * UNITSIZETIME, 100 * UNITSIZETIME)); - } + public void testCriticalTime() throws IOException { + assertEquals(Severity.CRITICAL, analyzeJobTime(200, 200, 1 * UNITSIZETIME, 100 * UNITSIZETIME)); + } - public void testSevereTime() throws IOException { - assertEquals(Severity.SEVERE, analyzeJobTime(200, 200, 10 * UNITSIZETIME, 100 * UNITSIZETIME)); - } + public void testSevereTime() throws IOException { + assertEquals(Severity.SEVERE, analyzeJobTime(200, 200, 10 * UNITSIZETIME, 100 * UNITSIZETIME)); + } - public void testModerateTime() throws IOException { - assertEquals(Severity.MODERATE, analyzeJobTime(200, 200, 20 * UNITSIZETIME, 100 * UNITSIZETIME)); - } + public void testModerateTime() throws IOException { + assertEquals(Severity.MODERATE, analyzeJobTime(200, 200, 20 * UNITSIZETIME, 100 * UNITSIZETIME)); + } - public void testLowTime() throws IOException { - assertEquals(Severity.LOW, analyzeJobTime(200, 200, 30 * UNITSIZETIME, 100 * UNITSIZETIME)); - } + public void testLowTime() throws IOException { + assertEquals(Severity.LOW, analyzeJobTime(200, 200, 30 * UNITSIZETIME, 100 * UNITSIZETIME)); + } - public void testNoneTime() throws IOException { - assertEquals(Severity.NONE, analyzeJobTime(200, 200, 50 * UNITSIZETIME, 100 * UNITSIZETIME)); - } + public void testNoneTime() throws IOException { + assertEquals(Severity.NONE, analyzeJobTime(200, 200, 50 * UNITSIZETIME, 100 * UNITSIZETIME)); + } - public void testSmallTasksTime() throws IOException { - assertEquals(Severity.NONE, analyzeJobTime(5, 5, 10 * UNITSIZETIME, 100 * UNITSIZETIME)); - } + public void testSmallTasksTime() throws IOException { + assertEquals(Severity.NONE, analyzeJobTime(5, 5, 10 * UNITSIZETIME, 100 * UNITSIZETIME)); + } - private Severity analyzeJob(int numSmallTasks, int numLargeTasks, long smallInputSize, long largeInputSize) - throws IOException { - TezCounterData jobCounter = new TezCounterData(); - TezTaskData[] reducers = new TezTaskData[numSmallTasks + numLargeTasks + 1]; - - TezCounterData smallCounter = new TezCounterData(); - smallCounter.set(TezCounterData.CounterName.SHUFFLE_BYTES, smallInputSize); - - TezCounterData largeCounter = new TezCounterData(); - largeCounter.set(TezCounterData.CounterName.SHUFFLE_BYTES, largeInputSize); - - int i = 0; - for (; i < numSmallTasks; i++) { - reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); - reducers[i].setTimeAndCounter(new long[5], smallCounter); - } - for (; i < numSmallTasks + numLargeTasks; i++) { - reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); - reducers[i].setTimeAndCounter(new long[5], largeCounter); - } - // Non-sampled task, which does not contain time and counter data - reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); - - TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setReduceTaskData(reducers); - HeuristicResult result = _heuristic.apply(data); - return result.getSeverity(); - } + private Severity analyzeJob(int numSmallTasks, int numLargeTasks, long smallInputSize, long largeInputSize) + throws IOException { + TezCounterData jobCounter = new TezCounterData(); + TezTaskData[] reducers = new TezTaskData[numSmallTasks + numLargeTasks + 1]; - private Severity analyzeJobTime(int numSmallTasks, int numLongTasks, long smallTimeTaken, long longTimeTaken) - throws IOException { - TezCounterData jobCounter = new TezCounterData(); - TezTaskData[] reducers = new TezTaskData[numSmallTasks + numLongTasks + 1]; - - int i = 0; - for (; i < numSmallTasks; i++) { - reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); - reducers[i].setTotalTimeMs(smallTimeTaken, true); - reducers[i].setCounter(jobCounter); - } - for (; i < numSmallTasks + numLongTasks; i++) { - reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); - reducers[i].setTotalTimeMs(longTimeTaken, true); - reducers[i].setCounter(jobCounter); - } - // Non-sampled task, which does not contain time data - reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); - TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setReduceTaskData(reducers); - HeuristicResult result = _heuristic.apply(data); - return result.getSeverity(); + TezCounterData smallCounter = new TezCounterData(); + smallCounter.set(TezCounterData.CounterName.SHUFFLE_BYTES, smallInputSize); + TezCounterData largeCounter = new TezCounterData(); + largeCounter.set(TezCounterData.CounterName.SHUFFLE_BYTES, largeInputSize); + + int i = 0; + for (; i < numSmallTasks; i++) { + reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + reducers[i].setTimeAndCounter(new long[5], smallCounter); + } + for (; i < numSmallTasks + numLargeTasks; i++) { + reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + reducers[i].setTimeAndCounter(new long[5], largeCounter); + } + // Non-sampled task, which does not contain time and counter data + reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + + TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setReduceTaskData(reducers); + HeuristicResult result = _heuristic.apply(data); + return result.getSeverity(); + } + + private Severity analyzeJobTime(int numSmallTasks, int numLongTasks, long smallTimeTaken, long longTimeTaken) + throws IOException { + TezCounterData jobCounter = new TezCounterData(); + TezTaskData[] reducers = new TezTaskData[numSmallTasks + numLongTasks + 1]; + + int i = 0; + for (; i < numSmallTasks; i++) { + reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + reducers[i].setTotalTimeMs(smallTimeTaken, true); + reducers[i].setCounter(jobCounter); } + for (; i < numSmallTasks + numLongTasks; i++) { + reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + reducers[i].setTotalTimeMs(longTimeTaken, true); + reducers[i].setCounter(jobCounter); + } + // Non-sampled task, which does not contain time data + reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setReduceTaskData(reducers); + HeuristicResult result = _heuristic.apply(data); + return result.getSeverity(); + + } } \ No newline at end of file diff --git a/test/com/linkedin/drelephant/tez/heuristics/ReducerGCHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/ReducerGCHeuristicTest.java index b157c6268..4a86fbefb 100644 --- a/test/com/linkedin/drelephant/tez/heuristics/ReducerGCHeuristicTest.java +++ b/test/com/linkedin/drelephant/tez/heuristics/ReducerGCHeuristicTest.java @@ -1,7 +1,17 @@ /* * Copyright 2017 Electronic Arts Inc. * - * Licensed under the Apache License, Version 2.0 + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. * */ package com.linkedin.drelephant.tez.heuristics; @@ -22,52 +32,52 @@ public class ReducerGCHeuristicTest extends TestCase{ - private static Map paramsMap = new HashMap(); + private static Map paramsMap = new HashMap(); - private static Heuristic _heuristic = new ReducerGCHeuristic(new HeuristicConfigurationData("test_heuristic", - "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); + private static Heuristic _heuristic = new ReducerGCHeuristic(new HeuristicConfigurationData("test_heuristic", + "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); - private static int NUMTASKS = 100; + private static int NUMTASKS = 100; - public void testGCCritical() throws IOException { - assertEquals(Severity.CRITICAL, analyzeJob(1000000, 50000, 2000)); - } + public void testGCCritical() throws IOException { + assertEquals(Severity.CRITICAL, analyzeJob(1000000, 50000, 2000)); + } - public void testGCSevere() throws IOException { - assertEquals(Severity.SEVERE, analyzeJob(1000000, 50000, 1500)); - } + public void testGCSevere() throws IOException { + assertEquals(Severity.SEVERE, analyzeJob(1000000, 50000, 1500)); + } - public void testGCModerate() throws IOException { - assertEquals(Severity.MODERATE, analyzeJob(1000000, 50000, 1000)); - } + public void testGCModerate() throws IOException { + assertEquals(Severity.MODERATE, analyzeJob(1000000, 50000, 1000)); + } - public void testGCNone() throws IOException { - assertEquals(Severity.NONE, analyzeJob(1000000, 50000, 300)); - } - - public void testShortTasksNone() throws IOException { - assertEquals(Severity.NONE, analyzeJob(100000, 50000, 2000)); - } + public void testGCNone() throws IOException { + assertEquals(Severity.NONE, analyzeJob(1000000, 50000, 300)); + } + public void testShortTasksNone() throws IOException { + assertEquals(Severity.NONE, analyzeJob(100000, 50000, 2000)); + } - private Severity analyzeJob(long runtimeMs, long cpuMs, long gcMs) throws IOException { - TezCounterData jobCounter = new TezCounterData(); - TezTaskData[] reducers = new TezTaskData[NUMTASKS + 1]; - TezCounterData counter = new TezCounterData(); - counter.set(TezCounterData.CounterName.CPU_MILLISECONDS, cpuMs); - counter.set(TezCounterData.CounterName.GC_TIME_MILLIS, gcMs); + private Severity analyzeJob(long runtimeMs, long cpuMs, long gcMs) throws IOException { + TezCounterData jobCounter = new TezCounterData(); + TezTaskData[] reducers = new TezTaskData[NUMTASKS + 1]; - int i = 0; - for (; i < NUMTASKS; i++) { - reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); - reducers[i].setTimeAndCounter(new long[] { runtimeMs, 0, 0, 0, 0 }, counter); - } - // Non-sampled task, which does not contain time and counter data - reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + TezCounterData counter = new TezCounterData(); + counter.set(TezCounterData.CounterName.CPU_MILLISECONDS, cpuMs); + counter.set(TezCounterData.CounterName.GC_TIME_MILLIS, gcMs); - TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setReduceTaskData(reducers); - HeuristicResult result = _heuristic.apply(data); - return result.getSeverity(); + int i = 0; + for (; i < NUMTASKS; i++) { + reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + reducers[i].setTimeAndCounter(new long[] { runtimeMs, 0, 0, 0, 0 }, counter); } + // Non-sampled task, which does not contain time and counter data + reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + + TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setReduceTaskData(reducers); + HeuristicResult result = _heuristic.apply(data); + return result.getSeverity(); + } } \ No newline at end of file diff --git a/test/com/linkedin/drelephant/tez/heuristics/ReducerMemoryHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/ReducerMemoryHeuristicTest.java index fd4f5ef71..66c3e193c 100644 --- a/test/com/linkedin/drelephant/tez/heuristics/ReducerMemoryHeuristicTest.java +++ b/test/com/linkedin/drelephant/tez/heuristics/ReducerMemoryHeuristicTest.java @@ -1,7 +1,17 @@ /* * Copyright 2017 Electronic Arts Inc. * - * Licensed under the Apache License, Version 2.0 + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. * */ package com.linkedin.drelephant.tez.heuristics; @@ -24,60 +34,60 @@ public class ReducerMemoryHeuristicTest extends TestCase { - private static Map paramsMap = new HashMap(); - private static Heuristic _heuristic = new ReducerMemoryHeuristic(new HeuristicConfigurationData("test_heuristic", - "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); + private static Map paramsMap = new HashMap(); + private static Heuristic _heuristic = new ReducerMemoryHeuristic(new HeuristicConfigurationData("test_heuristic", + "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); - private int NUMTASKS = 100; + private int NUMTASKS = 100; - public void testLargeContainerSizeCritical() throws IOException { - assertEquals(Severity.CRITICAL, analyzeJob(2048, 8192)); - } + public void testLargeContainerSizeCritical() throws IOException { + assertEquals(Severity.CRITICAL, analyzeJob(2048, 8192)); + } - public void testLargeContainerSizeSevere() throws IOException { - assertEquals(Severity.SEVERE, analyzeJob(3072, 8192)); - } + public void testLargeContainerSizeSevere() throws IOException { + assertEquals(Severity.SEVERE, analyzeJob(3072, 8192)); + } - public void testLargeContainerSizeModerate() throws IOException { - assertEquals(Severity.MODERATE, analyzeJob(4096, 8192)); - } + public void testLargeContainerSizeModerate() throws IOException { + assertEquals(Severity.MODERATE, analyzeJob(4096, 8192)); + } - public void testLargeContainerSizeNone() throws IOException { - assertEquals(Severity.NONE, analyzeJob(6144, 8192)); - } - - // If the task use default container size, it should not be flagged - // Not using Default Container param, will calculate severity irrespective of default container size Chaning NONE -> CRITICAL - public void testDefaultContainerNone() throws IOException { - assertEquals(Severity.CRITICAL, analyzeJob(256, 2048)); - } + public void testLargeContainerSizeNone() throws IOException { + assertEquals(Severity.NONE, analyzeJob(6144, 8192)); + } - // Not using Default Container param, will calculate severity irrespective of default container size Chaning NONE -> MODERATE - public void testDefaultContainerNoneMore() throws IOException { - assertEquals(Severity.MODERATE, analyzeJob(1024, 2048)); - } + // If the task use default container size, it should not be flagged + // Not using Default Container param, will calculate severity irrespective of default container size Chaning NONE -> CRITICAL + public void testDefaultContainerNone() throws IOException { + assertEquals(Severity.CRITICAL, analyzeJob(256, 2048)); + } - private Severity analyzeJob(long taskAvgMemMB, long containerMemMB) throws IOException { - TezCounterData jobCounter = new TezCounterData(); - TezTaskData[] reducers = new TezTaskData[NUMTASKS + 1]; + // Not using Default Container param, will calculate severity irrespective of default container size Chaning NONE -> MODERATE + public void testDefaultContainerNoneMore() throws IOException { + assertEquals(Severity.MODERATE, analyzeJob(1024, 2048)); + } - TezCounterData counter = new TezCounterData(); - counter.set(TezCounterData.CounterName.PHYSICAL_MEMORY_BYTES, taskAvgMemMB* FileUtils.ONE_MB); + private Severity analyzeJob(long taskAvgMemMB, long containerMemMB) throws IOException { + TezCounterData jobCounter = new TezCounterData(); + TezTaskData[] reducers = new TezTaskData[NUMTASKS + 1]; - Properties p = new Properties(); - p.setProperty(com.linkedin.drelephant.mapreduce.heuristics.ReducerMemoryHeuristic.REDUCER_MEMORY_CONF, Long.toString(containerMemMB)); + TezCounterData counter = new TezCounterData(); + counter.set(TezCounterData.CounterName.PHYSICAL_MEMORY_BYTES, taskAvgMemMB* FileUtils.ONE_MB); - int i = 0; - for (; i < NUMTASKS; i++) { - reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); - reducers[i].setTimeAndCounter(new long[5], counter); - } - // Non-sampled task, which does not contain time and counter data - reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + Properties p = new Properties(); + p.setProperty(com.linkedin.drelephant.mapreduce.heuristics.ReducerMemoryHeuristic.REDUCER_MEMORY_CONF, Long.toString(containerMemMB)); - TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setReduceTaskData(reducers); - data.setConf(p); - HeuristicResult result = _heuristic.apply(data); - return result.getSeverity(); + int i = 0; + for (; i < NUMTASKS; i++) { + reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + reducers[i].setTimeAndCounter(new long[5], counter); } + // Non-sampled task, which does not contain time and counter data + reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + + TezApplicationData data = new TezApplicationData().setCounters(jobCounter).setReduceTaskData(reducers); + data.setConf(p); + HeuristicResult result = _heuristic.apply(data); + return result.getSeverity(); + } } \ No newline at end of file diff --git a/test/com/linkedin/drelephant/tez/heuristics/ReducerTimeHeuristicTest.java b/test/com/linkedin/drelephant/tez/heuristics/ReducerTimeHeuristicTest.java index aac3964a6..ab0abbe38 100644 --- a/test/com/linkedin/drelephant/tez/heuristics/ReducerTimeHeuristicTest.java +++ b/test/com/linkedin/drelephant/tez/heuristics/ReducerTimeHeuristicTest.java @@ -1,7 +1,17 @@ /* * Copyright 2017 Electronic Arts Inc. * - * Licensed under the Apache License, Version 2.0 + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. * */ package com.linkedin.drelephant.tez.heuristics; @@ -24,64 +34,64 @@ public class ReducerTimeHeuristicTest extends TestCase { - private static Map paramsMap = new HashMap(); - private static Heuristic _heuristic = new ReducerTimeHeuristic(new HeuristicConfigurationData("test_heuristic", - "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); + private static Map paramsMap = new HashMap(); + private static Heuristic _heuristic = new ReducerTimeHeuristic(new HeuristicConfigurationData("test_heuristic", + "test_class", "test_view", new ApplicationType("test_apptype"), paramsMap)); - private static final long MINUTE_IN_MS = Statistics.MINUTE_IN_MS;; + private static final long MINUTE_IN_MS = Statistics.MINUTE_IN_MS;; - public void testShortRunetimeCritical() throws IOException { - assertEquals(Severity.CRITICAL, analyzeJob(1 * MINUTE_IN_MS, 1000)); - } + public void testShortRunetimeCritical() throws IOException { + assertEquals(Severity.CRITICAL, analyzeJob(1 * MINUTE_IN_MS, 1000)); + } - public void testShortRunetimeSevere() throws IOException { - assertEquals(Severity.SEVERE, analyzeJob(1 * MINUTE_IN_MS, 500)); - } + public void testShortRunetimeSevere() throws IOException { + assertEquals(Severity.SEVERE, analyzeJob(1 * MINUTE_IN_MS, 500)); + } - public void testShortRunetimeModerate() throws IOException { - assertEquals(Severity.MODERATE, analyzeJob(1 * MINUTE_IN_MS, 101)); - } + public void testShortRunetimeModerate() throws IOException { + assertEquals(Severity.MODERATE, analyzeJob(1 * MINUTE_IN_MS, 101)); + } - public void testShortRunetimeLow() throws IOException { - assertEquals(Severity.LOW, analyzeJob(1 * MINUTE_IN_MS, 50)); - } + public void testShortRunetimeLow() throws IOException { + assertEquals(Severity.LOW, analyzeJob(1 * MINUTE_IN_MS, 50)); + } - public void testShortRunetimeNone() throws IOException { - assertEquals(Severity.NONE, analyzeJob(1 * MINUTE_IN_MS, 2)); - } + public void testShortRunetimeNone() throws IOException { + assertEquals(Severity.NONE, analyzeJob(1 * MINUTE_IN_MS, 2)); + } - public void testLongRunetimeCritical() throws IOException { - assertEquals(Severity.CRITICAL, analyzeJob(120 * MINUTE_IN_MS, 10)); - } + public void testLongRunetimeCritical() throws IOException { + assertEquals(Severity.CRITICAL, analyzeJob(120 * MINUTE_IN_MS, 10)); + } - // Long runtime severity is not affected by number of tasks - public void testLongRunetimeCriticalMore() throws IOException { - assertEquals(Severity.CRITICAL, analyzeJob(120 * MINUTE_IN_MS, 1000)); - } + // Long runtime severity is not affected by number of tasks + public void testLongRunetimeCriticalMore() throws IOException { + assertEquals(Severity.CRITICAL, analyzeJob(120 * MINUTE_IN_MS, 1000)); + } - public void testLongRunetimeSevere() throws IOException { - assertEquals(Severity.SEVERE, analyzeJob(60 * MINUTE_IN_MS, 10)); - } + public void testLongRunetimeSevere() throws IOException { + assertEquals(Severity.SEVERE, analyzeJob(60 * MINUTE_IN_MS, 10)); + } - public void testLongRunetimeSevereMore() throws IOException { - assertEquals(Severity.SEVERE, analyzeJob(60 * MINUTE_IN_MS, 1000)); - } + public void testLongRunetimeSevereMore() throws IOException { + assertEquals(Severity.SEVERE, analyzeJob(60 * MINUTE_IN_MS, 1000)); + } + + private Severity analyzeJob(long runtimeMs, int numTasks) throws IOException { + TezCounterData dummyCounter = new TezCounterData(); + TezTaskData[] reducers = new TezTaskData[numTasks + 1]; - private Severity analyzeJob(long runtimeMs, int numTasks) throws IOException { - TezCounterData dummyCounter = new TezCounterData(); - TezTaskData[] reducers = new TezTaskData[numTasks + 1]; - - int i = 0; - for (; i < numTasks; i++) { - reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); - reducers[i].setTime(new long[] { runtimeMs, 0, 0, 0, 0 }); - reducers[i].setCounter(dummyCounter); - } - // Non-sampled task, which does not contain time and counter data - reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); - - TezApplicationData data = new TezApplicationData().setCounters(dummyCounter).setReduceTaskData(reducers); - HeuristicResult result = _heuristic.apply(data); - return result.getSeverity(); + int i = 0; + for (; i < numTasks; i++) { + reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + reducers[i].setTime(new long[] { runtimeMs, 0, 0, 0, 0 }); + reducers[i].setCounter(dummyCounter); } + // Non-sampled task, which does not contain time and counter data + reducers[i] = new TezTaskData("task-id-"+i, "task-attempt-id-"+i); + + TezApplicationData data = new TezApplicationData().setCounters(dummyCounter).setReduceTaskData(reducers); + HeuristicResult result = _heuristic.apply(data); + return result.getSeverity(); + } } \ No newline at end of file From fb6b83c6ac1f62e6633f21944445032ac733d613 Mon Sep 17 00:00:00 2001 From: Sumant Date: Thu, 15 Mar 2018 13:30:12 -0700 Subject: [PATCH 4/5] Tez fetcher variable change --- app/com/linkedin/drelephant/tez/fetchers/TezFetcher.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/app/com/linkedin/drelephant/tez/fetchers/TezFetcher.java b/app/com/linkedin/drelephant/tez/fetchers/TezFetcher.java index aed07aa6c..5f04dcbb3 100644 --- a/app/com/linkedin/drelephant/tez/fetchers/TezFetcher.java +++ b/app/com/linkedin/drelephant/tez/fetchers/TezFetcher.java @@ -47,6 +47,7 @@ public class TezFetcher implements ElephantFetcher { private static final Logger logger = Logger.getLogger(TezFetcher.class); private static final int MAX_SAMPLE_SIZE = 1000; + private static final String TIMELINE_SERVER_URL = "yarn.timeline-service.webapp.address"; private URLFactory _urlFactory; private JSONFactory _jsonFactory; @@ -56,7 +57,7 @@ public class TezFetcher implements ElephantFetcher { public TezFetcher(FetcherConfigurationData fetcherConfData) throws IOException { this._fetcherConfigurationData = fetcherConfData; - final String applicationHistoryAddr = new Configuration().get("yarn.timeline-service.webapp.address"); + final String applicationHistoryAddr = new Configuration().get(TIMELINE_SERVER_URL); //Connection validity checked using method verifyURL(_timelineWebAddr) inside URLFactory constructor; _urlFactory = new URLFactory(applicationHistoryAddr); From 74af5c0eb381368f9b63359382ab5097d918069e Mon Sep 17 00:00:00 2001 From: Sumant Date: Fri, 23 Mar 2018 12:19:11 -0700 Subject: [PATCH 5/5] Removed default 100 Limit for Vertex URL task Fetch and code review changes --- .../fetchers/MapReduceFetcherHadoop2.java | 84 +------------- .../tez/data/TezApplicationData.java | 6 +- .../drelephant/tez/fetchers/TezFetcher.java | 104 ++---------------- .../drelephant/util/ThreadContextMR2.java | 92 ++++++++++++++++ .../fetchers/MapReduceFetcherHadoop2Test.java | 2 + .../tez/fetchers/TezFetcherTest.java | 10 +- 6 files changed, 111 insertions(+), 187 deletions(-) create mode 100644 app/com/linkedin/drelephant/util/ThreadContextMR2.java diff --git a/app/com/linkedin/drelephant/mapreduce/fetchers/MapReduceFetcherHadoop2.java b/app/com/linkedin/drelephant/mapreduce/fetchers/MapReduceFetcherHadoop2.java index 4165971aa..5a8a64425 100644 --- a/app/com/linkedin/drelephant/mapreduce/fetchers/MapReduceFetcherHadoop2.java +++ b/app/com/linkedin/drelephant/mapreduce/fetchers/MapReduceFetcherHadoop2.java @@ -20,30 +20,23 @@ import com.linkedin.drelephant.mapreduce.data.MapReduceApplicationData; import com.linkedin.drelephant.mapreduce.data.MapReduceCounterData; import com.linkedin.drelephant.mapreduce.data.MapReduceTaskData; -import com.linkedin.drelephant.math.Statistics; import com.linkedin.drelephant.configurations.fetcher.FetcherConfigurationData; +import com.linkedin.drelephant.util.ThreadContextMR2; import com.linkedin.drelephant.util.Utils; import java.io.IOException; -import java.lang.Integer; -import java.net.HttpURLConnection; import java.net.MalformedURLException; import java.net.URL; import java.net.URLConnection; import java.util.ArrayList; import java.util.List; import java.util.Properties; -import java.util.Random; -import java.util.concurrent.atomic.AtomicInteger; import java.util.regex.Matcher; -import java.util.regex.Pattern; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.security.authentication.client.AuthenticatedURL; import org.apache.hadoop.security.authentication.client.AuthenticationException; import org.apache.log4j.Logger; import org.codehaus.jackson.JsonNode; -import org.codehaus.jackson.map.ObjectMapper; /** @@ -387,78 +380,3 @@ private JsonNode getTaskFirstFailedAttempt(URL taskAllAttemptsUrl) throws IOExce } } } - -final class ThreadContextMR2 { - private static final Logger logger = Logger.getLogger(ThreadContextMR2.class); - private static final AtomicInteger THREAD_ID = new AtomicInteger(1); - - private static final ThreadLocal _LOCAL_THREAD_ID = new ThreadLocal() { - @Override - public Integer initialValue() { - return THREAD_ID.getAndIncrement(); - } - }; - - private static final ThreadLocal _LOCAL_LAST_UPDATED = new ThreadLocal(); - private static final ThreadLocal _LOCAL_UPDATE_INTERVAL = new ThreadLocal(); - - private static final ThreadLocal _LOCAL_DIAGNOSTIC_PATTERN = new ThreadLocal() { - @Override - public Pattern initialValue() { - // Example: "Task task_1443068695259_9143_m_000475 failed 1 times" - return Pattern.compile( - ".*[\\s\\u00A0]+(task_[0-9]+_[0-9]+_[m|r]_[0-9]+)[\\s\\u00A0]+.*"); - } - }; - - private static final ThreadLocal _LOCAL_AUTH_TOKEN = - new ThreadLocal() { - @Override - public AuthenticatedURL.Token initialValue() { - _LOCAL_LAST_UPDATED.set(System.currentTimeMillis()); - // Random an interval for each executor to avoid update token at the same time - _LOCAL_UPDATE_INTERVAL.set(Statistics.MINUTE_IN_MS * 30 + new Random().nextLong() - % (3 * Statistics.MINUTE_IN_MS)); - logger.info("Executor " + _LOCAL_THREAD_ID.get() + " update interval " + _LOCAL_UPDATE_INTERVAL.get() * 1.0 - / Statistics.MINUTE_IN_MS); - return new AuthenticatedURL.Token(); - } - }; - - private static final ThreadLocal _LOCAL_AUTH_URL = new ThreadLocal() { - @Override - public AuthenticatedURL initialValue() { - return new AuthenticatedURL(); - } - }; - - private static final ThreadLocal _LOCAL_MAPPER = new ThreadLocal() { - @Override - public ObjectMapper initialValue() { - return new ObjectMapper(); - } - }; - - private ThreadContextMR2() { - // Empty on purpose - } - - public static Matcher getDiagnosticMatcher(String diagnosticInfo) { - return _LOCAL_DIAGNOSTIC_PATTERN.get().matcher(diagnosticInfo); - } - - public static JsonNode readJsonNode(URL url) throws IOException, AuthenticationException { - HttpURLConnection conn = _LOCAL_AUTH_URL.get().openConnection(url, _LOCAL_AUTH_TOKEN.get()); - return _LOCAL_MAPPER.get().readTree(conn.getInputStream()); - } - - public static void updateAuthToken() { - long curTime = System.currentTimeMillis(); - if (curTime - _LOCAL_LAST_UPDATED.get() > _LOCAL_UPDATE_INTERVAL.get()) { - logger.info("Executor " + _LOCAL_THREAD_ID.get() + " updates its AuthenticatedToken."); - _LOCAL_AUTH_TOKEN.set(new AuthenticatedURL.Token()); - _LOCAL_AUTH_URL.set(new AuthenticatedURL()); - _LOCAL_LAST_UPDATED.set(curTime); - } - } -} diff --git a/app/com/linkedin/drelephant/tez/data/TezApplicationData.java b/app/com/linkedin/drelephant/tez/data/TezApplicationData.java index 19cd1d9e2..f38f527bc 100644 --- a/app/com/linkedin/drelephant/tez/data/TezApplicationData.java +++ b/app/com/linkedin/drelephant/tez/data/TezApplicationData.java @@ -30,7 +30,7 @@ public class TezApplicationData implements HadoopApplicationData { private String _appId = ""; private Properties _conf; private boolean _succeeded = true; - private TezTaskData[] _redudeTasks; + private TezTaskData[] _reduceTasks; private TezTaskData[] _mapTasks; private TezCounterData _counterHolder; @@ -63,7 +63,7 @@ public boolean isEmpty() { } public TezTaskData[] getReduceTaskData() { - return _redudeTasks; + return _reduceTasks; } public TezTaskData[] getMapTaskData() { @@ -107,7 +107,7 @@ public TezApplicationData setSucceeded(boolean succeeded) { } public TezApplicationData setReduceTaskData(TezTaskData[] reduceTasks) { - this._redudeTasks = reduceTasks; + this._reduceTasks = reduceTasks; return this; } diff --git a/app/com/linkedin/drelephant/tez/fetchers/TezFetcher.java b/app/com/linkedin/drelephant/tez/fetchers/TezFetcher.java index 5f04dcbb3..294fe20e0 100644 --- a/app/com/linkedin/drelephant/tez/fetchers/TezFetcher.java +++ b/app/com/linkedin/drelephant/tez/fetchers/TezFetcher.java @@ -17,27 +17,21 @@ import com.linkedin.drelephant.analysis.AnalyticJob; import com.linkedin.drelephant.analysis.ElephantFetcher; import com.linkedin.drelephant.configurations.fetcher.FetcherConfigurationData; -import com.linkedin.drelephant.math.Statistics; import com.linkedin.drelephant.tez.data.TezApplicationData; import com.linkedin.drelephant.tez.data.TezCounterData; import com.linkedin.drelephant.tez.data.TezTaskData; +import com.linkedin.drelephant.util.ThreadContextMR2; import org.apache.log4j.Logger; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.security.authentication.client.AuthenticatedURL; import org.apache.hadoop.security.authentication.client.AuthenticationException; import org.codehaus.jackson.JsonNode; -import org.codehaus.jackson.map.ObjectMapper; import java.io.IOException; -import java.net.HttpURLConnection; import java.net.MalformedURLException; import java.net.URL; import java.net.URLConnection; import java.util.*; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.regex.Matcher; -import java.util.regex.Pattern; /** * Task level data mining for Tez Tasks from timeline server API @@ -46,7 +40,7 @@ public class TezFetcher implements ElephantFetcher { private static final Logger logger = Logger.getLogger(TezFetcher.class); - private static final int MAX_SAMPLE_SIZE = 1000; + private static final String TIMELINE_SERVER_URL = "yarn.timeline-service.webapp.address"; private URLFactory _urlFactory; @@ -109,7 +103,9 @@ public TezApplicationData fetchData(AnalyticJob analyticJob) throws IOException, reducerListAggregate = reducerList; maxSize = mapperList.size() + reducerList.size(); } - + } + if (state.equals("FAILED")) { + jobData.setSucceeded(false); } } finally { @@ -117,10 +113,6 @@ public TezApplicationData fetchData(AnalyticJob analyticJob) throws IOException, } } - if(mapperListAggregate.isEmpty() && reducerListAggregate.isEmpty()){ - jobData.setSucceeded(false); - } - TezTaskData[] mapperData = mapperListAggregate.toArray(new TezTaskData[mapperListAggregate.size()]); TezTaskData[] reducerData = reducerListAggregate.toArray(new TezTaskData[reducerListAggregate.size()]); @@ -177,7 +169,7 @@ private URL getVertexListURL(String dagId) throws MalformedURLException { private URL getTaskListByVertexURL(String dagId, String vertexId) throws MalformedURLException { return new URL(_timelineWebAddr + "/TEZ_TASK_ID?primaryFilter=TEZ_DAG_ID:" + dagId + - "&secondaryFilter=TEZ_VERTEX_ID:" + vertexId); + "&secondaryFilter=TEZ_VERTEX_ID:" + vertexId + "&limit=500000"); } private URL getTasksURL(String taskId) throws MalformedURLException { @@ -336,13 +328,7 @@ private JsonNode getTaskFirstFailedAttempt(URL taskAllAttemptsUrl) throws IOExce private void getTaskData(String dagId, List taskList, boolean isMapTask) throws IOException, AuthenticationException { - if (taskList.size() > MAX_SAMPLE_SIZE) { - logger.info(dagId + " needs sampling."); - Collections.shuffle(taskList); - } - - int sampleSize = Math.min(taskList.size(), MAX_SAMPLE_SIZE); - for (int i=0; i _LOCAL_THREAD_ID = new ThreadLocal() { - @Override - public Integer initialValue() { - return THREAD_ID.getAndIncrement(); - } - }; - - private static final ThreadLocal _LOCAL_LAST_UPDATED = new ThreadLocal(); - private static final ThreadLocal _LOCAL_UPDATE_INTERVAL = new ThreadLocal(); - - private static final ThreadLocal _LOCAL_DIAGNOSTIC_PATTERN = new ThreadLocal() { - @Override - public Pattern initialValue() { - // Example: "Task task_1443068695259_9143_m_000475 failed 1 times" - return Pattern.compile( - "Task[\\s\\u00A0]+(.*)[\\s\\u00A0]+failed[\\s\\u00A0]+([0-9])[\\s\\u00A0]+times[\\s\\u00A0]+"); - } - }; - - private static final ThreadLocal _LOCAL_AUTH_TOKEN = - new ThreadLocal() { - @Override - public AuthenticatedURL.Token initialValue() { - _LOCAL_LAST_UPDATED.set(System.currentTimeMillis()); - // Random an interval for each executor to avoid update token at the same time - _LOCAL_UPDATE_INTERVAL.set(Statistics.MINUTE_IN_MS * 30 + new Random().nextLong() - % (3 * Statistics.MINUTE_IN_MS)); - logger.info("Executor " + _LOCAL_THREAD_ID.get() + " update interval " + _LOCAL_UPDATE_INTERVAL.get() * 1.0 - / Statistics.MINUTE_IN_MS); - return new AuthenticatedURL.Token(); - } - }; - - private static final ThreadLocal _LOCAL_AUTH_URL = new ThreadLocal() { - @Override - public AuthenticatedURL initialValue() { - return new AuthenticatedURL(); - } - }; - - private static final ThreadLocal _LOCAL_MAPPER = new ThreadLocal() { - @Override - public ObjectMapper initialValue() { - return new ObjectMapper(); - } - }; - - private ThreadContextMR2() { - // Empty on purpose - } - - public static Matcher getDiagnosticMatcher(String diagnosticInfo) { - return _LOCAL_DIAGNOSTIC_PATTERN.get().matcher(diagnosticInfo); - } - - public static JsonNode readJsonNode(URL url) throws IOException, AuthenticationException { - HttpURLConnection conn = _LOCAL_AUTH_URL.get().openConnection(url, _LOCAL_AUTH_TOKEN.get()); - return _LOCAL_MAPPER.get().readTree(conn.getInputStream()); - } - - public static void updateAuthToken() { - long curTime = System.currentTimeMillis(); - if (curTime - _LOCAL_LAST_UPDATED.get() > _LOCAL_UPDATE_INTERVAL.get()) { - logger.info("Executor " + _LOCAL_THREAD_ID.get() + " updates its AuthenticatedToken."); - _LOCAL_AUTH_TOKEN.set(new AuthenticatedURL.Token()); - _LOCAL_AUTH_URL.set(new AuthenticatedURL()); - _LOCAL_LAST_UPDATED.set(curTime); - } - } } \ No newline at end of file diff --git a/app/com/linkedin/drelephant/util/ThreadContextMR2.java b/app/com/linkedin/drelephant/util/ThreadContextMR2.java new file mode 100644 index 000000000..a448a24fe --- /dev/null +++ b/app/com/linkedin/drelephant/util/ThreadContextMR2.java @@ -0,0 +1,92 @@ +package com.linkedin.drelephant.util; + +import com.linkedin.drelephant.math.Statistics; +import org.apache.hadoop.security.authentication.client.AuthenticatedURL; +import org.apache.hadoop.security.authentication.client.AuthenticationException; +import org.apache.log4j.Logger; +import org.codehaus.jackson.JsonNode; +import org.codehaus.jackson.map.ObjectMapper; + +import java.io.IOException; +import java.net.HttpURLConnection; +import java.net.URL; +import java.util.Random; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +public final class ThreadContextMR2 { + private static final Logger logger = Logger.getLogger(com.linkedin.drelephant.util.ThreadContextMR2.class); + + private static final AtomicInteger THREAD_ID = new AtomicInteger(1); + + private static final ThreadLocal _LOCAL_THREAD_ID = new ThreadLocal() { + @Override + public Integer initialValue() { + return THREAD_ID.getAndIncrement(); + } + }; + + private static final ThreadLocal _LOCAL_LAST_UPDATED = new ThreadLocal(); + private static final ThreadLocal _LOCAL_UPDATE_INTERVAL = new ThreadLocal(); + + private static final ThreadLocal _LOCAL_DIAGNOSTIC_PATTERN = new ThreadLocal() { + @Override + public Pattern initialValue() { + // Example: "Task task_1443068695259_9143_m_000475 failed 1 times" + return Pattern.compile( + ".*[\\s\\u00A0]+(task_[0-9]+_[0-9]+_[m|r]_[0-9]+)[\\s\\u00A0]+.*"); + } + }; + + private static final ThreadLocal _LOCAL_AUTH_TOKEN = + new ThreadLocal() { + @Override + public AuthenticatedURL.Token initialValue() { + _LOCAL_LAST_UPDATED.set(System.currentTimeMillis()); + // Random an interval for each executor to avoid update token at the same time + _LOCAL_UPDATE_INTERVAL.set(Statistics.MINUTE_IN_MS * 30 + new Random().nextLong() + % (3 * Statistics.MINUTE_IN_MS)); + logger.info("Executor " + _LOCAL_THREAD_ID.get() + " update interval " + _LOCAL_UPDATE_INTERVAL.get() * 1.0 + / Statistics.MINUTE_IN_MS); + return new AuthenticatedURL.Token(); + } + }; + + private static final ThreadLocal _LOCAL_AUTH_URL = new ThreadLocal() { + @Override + public AuthenticatedURL initialValue() { + return new AuthenticatedURL(); + } + }; + + private static final ThreadLocal _LOCAL_MAPPER = new ThreadLocal() { + @Override + public ObjectMapper initialValue() { + return new ObjectMapper(); + } + }; + + private ThreadContextMR2() { + // Empty on purpose + } + + public static Matcher getDiagnosticMatcher(String diagnosticInfo) { + return _LOCAL_DIAGNOSTIC_PATTERN.get().matcher(diagnosticInfo); + } + + public static JsonNode readJsonNode(URL url) throws IOException, AuthenticationException { + HttpURLConnection conn = _LOCAL_AUTH_URL.get().openConnection(url, _LOCAL_AUTH_TOKEN.get()); + return _LOCAL_MAPPER.get().readTree(conn.getInputStream()); + } + + public static void updateAuthToken() { + long curTime = System.currentTimeMillis(); + if (curTime - _LOCAL_LAST_UPDATED.get() > _LOCAL_UPDATE_INTERVAL.get()) { + logger.info("Executor " + _LOCAL_THREAD_ID.get() + " updates its AuthenticatedToken."); + _LOCAL_AUTH_TOKEN.set(new AuthenticatedURL.Token()); + _LOCAL_AUTH_URL.set(new AuthenticatedURL()); + _LOCAL_LAST_UPDATED.set(curTime); + } + } +} \ No newline at end of file diff --git a/test/com/linkedin/drelephant/mapreduce/fetchers/MapReduceFetcherHadoop2Test.java b/test/com/linkedin/drelephant/mapreduce/fetchers/MapReduceFetcherHadoop2Test.java index 531bb3724..8ce84bec1 100644 --- a/test/com/linkedin/drelephant/mapreduce/fetchers/MapReduceFetcherHadoop2Test.java +++ b/test/com/linkedin/drelephant/mapreduce/fetchers/MapReduceFetcherHadoop2Test.java @@ -18,6 +18,8 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; + +import com.linkedin.drelephant.util.ThreadContextMR2; import org.junit.Assert; import org.junit.Test; diff --git a/test/com/linkedin/drelephant/tez/fetchers/TezFetcherTest.java b/test/com/linkedin/drelephant/tez/fetchers/TezFetcherTest.java index 77c16a9ce..bf38f10c3 100644 --- a/test/com/linkedin/drelephant/tez/fetchers/TezFetcherTest.java +++ b/test/com/linkedin/drelephant/tez/fetchers/TezFetcherTest.java @@ -19,15 +19,17 @@ import java.util.regex.Matcher; import org.junit.Assert; import org.junit.Test; +import com.linkedin.drelephant.util.ThreadContextMR2; public class TezFetcherTest { @Test public void testDiagnosticMatcher() { - Matcher matcher = com.linkedin.drelephant.tez.fetchers.ThreadContextMR2.getDiagnosticMatcher("Task task_1443068695259_9143_m_000475 failed 1 time"); - Assert.assertEquals("Task[\\s\\u00A0]+(.*)[\\s\\u00A0]+failed[\\s\\u00A0]+([0-9])[\\s\\u00A0]+times[\\s\\u00A0]+", matcher.pattern().toString()); - Assert.assertEquals(false, matcher.matches()); - Assert.assertEquals(2, matcher.groupCount()); + Matcher matcher = ThreadContextMR2.getDiagnosticMatcher("Task task_1443068695259_9143_m_000475 failed 1 time"); + Assert.assertEquals(".*[\\s\\u00A0]+(task_[0-9]+_[0-9]+_[m|r]_[0-9]+)[\\s\\u00A0]+.*", matcher.pattern().toString()); + Assert.assertEquals(true, matcher.matches()); + Assert.assertEquals(1, matcher.groupCount()); + Assert.assertEquals("task_1443068695259_9143_m_000475", matcher.group(1)); } } \ No newline at end of file