HIVE-10853 : Create ExplainTask in ATS hook through ExplainWork (Pengcheng Xiong via Gunther Hagleitner)
Signed-off-by: Ashutosh Chauhan <hashut...@apache.org> Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/4f8de20a Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/4f8de20a Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/4f8de20a Branch: refs/heads/spark Commit: 4f8de20ace77036a212debaf5aeed0ec7e4b479a Parents: 3ea0359 Author: Pengcheng Xiong <pxi...@hortonworks.com> Authored: Fri May 29 11:19:00 2015 -0700 Committer: Ashutosh Chauhan <hashut...@apache.org> Committed: Sat Jun 6 14:22:04 2015 -0700 ---------------------------------------------------------------------- .../apache/hadoop/hive/ql/hooks/ATSHook.java | 27 +++++++++++++++++--- 1 file changed, 23 insertions(+), 4 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hive/blob/4f8de20a/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java ---------------------------------------------------------------------- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java index 53d169d..87638da 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hive.ql.hooks; +import java.io.Serializable; import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -25,11 +26,16 @@ import java.util.concurrent.TimeUnit; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.QueryPlan; import org.apache.hadoop.hive.ql.exec.ExplainTask; import org.apache.hadoop.hive.ql.exec.Task; +import org.apache.hadoop.hive.ql.exec.TaskFactory; import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer; +import org.apache.hadoop.hive.ql.parse.ParseContext; +import org.apache.hadoop.hive.ql.plan.ExplainWork; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity; import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent; @@ -113,12 +119,25 @@ public class ATSHook implements ExecuteWithHookContext { switch(hookContext.getHookType()) { case PRE_EXEC_HOOK: - ExplainTask explain = new ExplainTask(); + ExplainWork work = new ExplainWork(null,// resFile + null,// pCtx + plan.getRootTasks(),// RootTasks + plan.getFetchTask(),// FetchTask + null,// astStringTree + null,// analyzer + false,// extended + true,// formatted + false,// dependency + false,// logical + false,// authorize + false,// userLevelExplain + null// cboInfo + ); + @SuppressWarnings("unchecked") + ExplainTask explain = (ExplainTask) TaskFactory.get(work, conf); explain.initialize(conf, plan, null); String query = plan.getQueryStr(); - List<Task<?>> rootTasks = plan.getRootTasks(); - JSONObject explainPlan = explain.getJSONPlan(null, null, rootTasks, - plan.getFetchTask(), true, false, false); + JSONObject explainPlan = explain.getJSONPlan(null, work); fireAndForget(conf, createPreHookEvent(queryId, query, explainPlan, queryStartTime, user, requestuser, numMrJobs, numTezJobs, opId)); break;