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 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 void run() { 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> 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;