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/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/TezMetricsAggregator.java b/app/com/linkedin/drelephant/tez/TezMetricsAggregator.java
new file mode 100644
index 000000000..c04edd7fd
--- /dev/null
+++ b/app/com/linkedin/drelephant/tez/TezMetricsAggregator.java
@@ -0,0 +1,109 @@
+/*
+ *
+ * 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.*;
+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..0f5eaa5f6
--- /dev/null
+++ b/app/com/linkedin/drelephant/tez/TezTaskLevelAggregatedMetrics.java
@@ -0,0 +1,154 @@
+/*
+ *
+ * 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;
+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..f38f527bc
--- /dev/null
+++ b/app/com/linkedin/drelephant/tez/data/TezApplicationData.java
@@ -0,0 +1,137 @@
+/*
+ *
+ * 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;
+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[] _reduceTasks;
+ 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 _reduceTasks;
+ }
+
+ 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._reduceTasks = 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..9aac00117
--- /dev/null
+++ b/app/com/linkedin/drelephant/tez/data/TezCounterData.java
@@ -0,0 +1,195 @@
+/*
+ *
+ * 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;
+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..eb6aa832c
--- /dev/null
+++ b/app/com/linkedin/drelephant/tez/data/TezTaskData.java
@@ -0,0 +1,137 @@
+/*
+ *
+ * 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;
+
+/**
+ * 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..294fe20e0
--- /dev/null
+++ b/app/com/linkedin/drelephant/tez/fetchers/TezFetcher.java
@@ -0,0 +1,391 @@
+/*
+ * 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;
+import com.linkedin.drelephant.analysis.ElephantFetcher;
+import com.linkedin.drelephant.configurations.fetcher.FetcherConfigurationData;
+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.AuthenticationException;
+import org.codehaus.jackson.JsonNode;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLConnection;
+import java.util.*;
+
+/**
+ * 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 String TIMELINE_SERVER_URL = "yarn.timeline-service.webapp.address";
+
+ 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(TIMELINE_SERVER_URL);
+
+ //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();
+ }
+ }
+ if (state.equals("FAILED")) {
+ jobData.setSucceeded(false);
+ }
+ }
+ finally {
+ ThreadContextMR2.updateAuthToken();
+ }
+ }
+
+ 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 + "&limit=500000");
+ }
+
+ 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 {
+
+ for(int i=0; i {
+ 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..d5726ef71
--- /dev/null
+++ b/app/com/linkedin/drelephant/tez/heuristics/GenericGCHeuristic.java
@@ -0,0 +1,142 @@
+/*
+ * Copyright 2017 Electronic Arts Inc.
+ *
+ * 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.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..3a575092b
--- /dev/null
+++ b/app/com/linkedin/drelephant/tez/heuristics/GenericMemoryHeuristic.java
@@ -0,0 +1,185 @@
+/*
+ * Copyright 2017 Electronic Arts Inc.
+ *
+ * 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.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..b17ebe3b2
--- /dev/null
+++ b/app/com/linkedin/drelephant/tez/heuristics/MapperDataSkewHeuristic.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2017 Electronic Arts Inc.
+ *
+ * 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.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..3a82ae5f9
--- /dev/null
+++ b/app/com/linkedin/drelephant/tez/heuristics/MapperGCHeuristic.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2017 Electronic Arts Inc.
+ *
+ * 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.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..96d5bb12e
--- /dev/null
+++ b/app/com/linkedin/drelephant/tez/heuristics/MapperMemoryHeuristic.java
@@ -0,0 +1,45 @@
+/*
+ * Copyright 2017 Electronic Arts Inc.
+ *
+ * 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.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..1694e32ea
--- /dev/null
+++ b/app/com/linkedin/drelephant/tez/heuristics/MapperSpeedHeuristic.java
@@ -0,0 +1,165 @@
+/*
+ * Copyright 2017 Electronic Arts Inc.
+ *
+ * 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.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..f7ea997e5
--- /dev/null
+++ b/app/com/linkedin/drelephant/tez/heuristics/MapperSpillHeuristic.java
@@ -0,0 +1,142 @@
+/*
+ * 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;
+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..838bd7a59
--- /dev/null
+++ b/app/com/linkedin/drelephant/tez/heuristics/MapperTimeHeuristic.java
@@ -0,0 +1,182 @@
+/*
+ * Copyright 2017 Electronic Arts Inc.
+ *
+ * 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;
+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..3a01448d4
--- /dev/null
+++ b/app/com/linkedin/drelephant/tez/heuristics/ReducerDataSkewHeuristic.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2017 Electronic Arts Inc.
+ *
+ * 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.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..3ea3bdac4
--- /dev/null
+++ b/app/com/linkedin/drelephant/tez/heuristics/ReducerGCHeuristic.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2017 Electronic Arts Inc.
+ *
+ * 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.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..41fc6fa1d
--- /dev/null
+++ b/app/com/linkedin/drelephant/tez/heuristics/ReducerMemoryHeuristic.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2017 Electronic Arts Inc.
+ *
+ * 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.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..5e0dbfe65
--- /dev/null
+++ b/app/com/linkedin/drelephant/tez/heuristics/ReducerTimeHeuristic.java
@@ -0,0 +1,167 @@
+/*
+ * Copyright 2017 Electronic Arts Inc.
+ *
+ * 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;
+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 bbd8b92da..48433a277 100644
--- a/app/com/linkedin/drelephant/util/InfoExtractor.java
+++ b/app/com/linkedin/drelephant/util/InfoExtractor.java
@@ -20,6 +20,10 @@
import com.linkedin.drelephant.clients.WorkflowClient;
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.clients.WorkflowClient;
+
import com.linkedin.drelephant.mapreduce.data.MapReduceApplicationData;
import com.linkedin.drelephant.schedulers.Scheduler;
import com.linkedin.drelephant.spark.data.SparkApplicationData;
@@ -116,6 +120,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 +173,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/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/TezTaskLevelAggregatedMetricsTest.java b/test/com/linkedin/drelephant/tez/TezTaskLevelAggregatedMetricsTest.java
new file mode 100644
index 000000000..4a42a8617
--- /dev/null
+++ b/test/com/linkedin/drelephant/tez/TezTaskLevelAggregatedMetricsTest.java
@@ -0,0 +1,62 @@
+/*
+ * Copyright 2017 Electronic Arts Inc.
+ *
+ * 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;
+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..bf38f10c3
--- /dev/null
+++ b/test/com/linkedin/drelephant/tez/fetchers/TezFetcherTest.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2017 Electronic Arts Inc.
+ *
+ * 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 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 = 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
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..8792e70f4
--- /dev/null
+++ b/test/com/linkedin/drelephant/tez/heuristics/MapperDataSkewHeuristicTest.java
@@ -0,0 +1,146 @@
+/*
+ * Copyright 2017 Electronic Arts Inc.
+ *
+ * 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.*;
+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..52cea2035
--- /dev/null
+++ b/test/com/linkedin/drelephant/tez/heuristics/MapperGCHeuristicTest.java
@@ -0,0 +1,84 @@
+/*
+ * Copyright 2017 Electronic Arts Inc.
+ *
+ * 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.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..72e426743
--- /dev/null
+++ b/test/com/linkedin/drelephant/tez/heuristics/MapperMemoryHeuristicTest.java
@@ -0,0 +1,94 @@
+/*
+ * Copyright 2017 Electronic Arts Inc.
+ *
+ * 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.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..2d21f856a
--- /dev/null
+++ b/test/com/linkedin/drelephant/tez/heuristics/MapperSpeedHeuristicTest.java
@@ -0,0 +1,100 @@
+/*
+ * Copyright 2017 Electronic Arts Inc.
+ *
+ * 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.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..62831a727
--- /dev/null
+++ b/test/com/linkedin/drelephant/tez/heuristics/MapperSpillHeuristicTest.java
@@ -0,0 +1,90 @@
+/*
+ * Copyright 2017 Electronic Arts Inc.
+ *
+ * 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.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..39c6ec891
--- /dev/null
+++ b/test/com/linkedin/drelephant/tez/heuristics/MapperTimeHeuristicTest.java
@@ -0,0 +1,111 @@
+/*
+ * Copyright 2017 Electronic Arts Inc.
+ *
+ * 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.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..ff95e862a
--- /dev/null
+++ b/test/com/linkedin/drelephant/tez/heuristics/ReducerDataSkewHeuristicTest.java
@@ -0,0 +1,142 @@
+/*
+ * Copyright 2017 Electronic Arts Inc.
+ *
+ * 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.*;
+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..4a86fbefb
--- /dev/null
+++ b/test/com/linkedin/drelephant/tez/heuristics/ReducerGCHeuristicTest.java
@@ -0,0 +1,83 @@
+/*
+ * Copyright 2017 Electronic Arts Inc.
+ *
+ * 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.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..66c3e193c
--- /dev/null
+++ b/test/com/linkedin/drelephant/tez/heuristics/ReducerMemoryHeuristicTest.java
@@ -0,0 +1,93 @@
+/*
+ * Copyright 2017 Electronic Arts Inc.
+ *
+ * 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.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..ab0abbe38
--- /dev/null
+++ b/test/com/linkedin/drelephant/tez/heuristics/ReducerTimeHeuristicTest.java
@@ -0,0 +1,97 @@
+/*
+ * Copyright 2017 Electronic Arts Inc.
+ *
+ * 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.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