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..41e9724 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 @@ -29,7 +29,9 @@ 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.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 +115,13 @@ public void run() { switch(hookContext.getHookType()) { case PRE_EXEC_HOOK: - ExplainTask explain = new ExplainTask(); + ExplainWork work = new ExplainWork(null, null, plan.getRootTasks(), + plan.getFetchTask(), null, null, false, true, false, false, false, false, null); + @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;