You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by we...@apache.org on 2017/05/08 22:17:32 UTC

[15/50] [abbrv] hive git commit: HIVE-16389: Allow HookContext to access SQLOperationDisplay (Sahil Takiar, reviewed by Sergio Pena)

HIVE-16389: Allow HookContext to access SQLOperationDisplay (Sahil Takiar, reviewed by Sergio Pena)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/1fecb81f
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/1fecb81f
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/1fecb81f

Branch: refs/heads/hive-14535
Commit: 1fecb81f942ffa1ce91a2ea343f266ce82f6e7ef
Parents: 44804d8
Author: Sahil Takiar <ta...@gmail.com>
Authored: Fri May 5 09:46:09 2017 -0500
Committer: Sergio Pena <se...@cloudera.com>
Committed: Fri May 5 09:46:09 2017 -0500

----------------------------------------------------------------------
 .../service/cli/session/TestQueryDisplay.java   |  40 +++----
 .../java/org/apache/hadoop/hive/ql/Driver.java  |  16 ++-
 .../org/apache/hadoop/hive/ql/QueryInfo.java    | 102 ++++++++++++++++
 .../hadoop/hive/ql/hooks/HookContext.java       |  10 +-
 .../org/apache/hive/tmpl/QueryProfileTmpl.jamon |  68 +++++------
 .../service/cli/operation/OperationManager.java |  47 ++++----
 .../service/cli/operation/QueryInfoCache.java   |  41 +++++++
 .../service/cli/operation/SQLOperation.java     |  33 +++---
 .../cli/operation/SQLOperationDisplay.java      | 108 -----------------
 .../cli/operation/SQLOperationDisplayCache.java |  39 -------
 .../service/servlet/QueryProfileServlet.java    |   8 +-
 .../hive-webapps/hiveserver2/hiveserver2.jsp    |  10 +-
 .../TestQueryLifeTimeHooksWithSQLOperation.java | 115 +++++++++++++++++++
 13 files changed, 380 insertions(+), 257 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/1fecb81f/itests/hive-unit/src/test/java/org/apache/hive/service/cli/session/TestQueryDisplay.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/session/TestQueryDisplay.java b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/session/TestQueryDisplay.java
index 6c60125..155c65d 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/session/TestQueryDisplay.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/session/TestQueryDisplay.java
@@ -19,11 +19,11 @@ package org.apache.hive.service.cli.session;
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.QueryDisplay;
+import org.apache.hadoop.hive.ql.QueryInfo;
 import org.apache.hadoop.hive.ql.plan.api.StageType;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hive.service.cli.OperationHandle;
 import org.apache.hive.service.cli.SessionHandle;
-import org.apache.hive.service.cli.operation.SQLOperationDisplay;
 import org.apache.hive.service.rpc.thrift.TProtocolVersion;
 import org.apache.hive.service.server.HiveServer2;
 import org.apache.hive.tmpl.QueryProfileTmpl;
@@ -69,25 +69,25 @@ public class TestQueryDisplay {
     OperationHandle opHandle2 = session.executeStatement("show tables", null);
 
 
-    List<SQLOperationDisplay> liveSqlOperations, historicSqlOperations;
-    liveSqlOperations = sessionManager.getOperationManager().getLiveSqlOperations();
-    historicSqlOperations = sessionManager.getOperationManager().getHistoricalSQLOperations();
+    List<QueryInfo> liveSqlOperations, historicSqlOperations;
+    liveSqlOperations = sessionManager.getOperationManager().getLiveQueryInfos();
+    historicSqlOperations = sessionManager.getOperationManager().getHistoricalQueryInfos();
     Assert.assertEquals(liveSqlOperations.size(), 2);
     Assert.assertEquals(historicSqlOperations.size(), 0);
     verifyDDL(liveSqlOperations.get(0), "show databases", opHandle1.getHandleIdentifier().toString(), false);
     verifyDDL(liveSqlOperations.get(1),"show tables", opHandle2.getHandleIdentifier().toString(), false);
 
     session.closeOperation(opHandle1);
-    liveSqlOperations = sessionManager.getOperationManager().getLiveSqlOperations();
-    historicSqlOperations = sessionManager.getOperationManager().getHistoricalSQLOperations();
+    liveSqlOperations = sessionManager.getOperationManager().getLiveQueryInfos();
+    historicSqlOperations = sessionManager.getOperationManager().getHistoricalQueryInfos();
     Assert.assertEquals(liveSqlOperations.size(), 1);
     Assert.assertEquals(historicSqlOperations.size(), 1);
     verifyDDL(historicSqlOperations.get(0),"show databases", opHandle1.getHandleIdentifier().toString(), true);
     verifyDDL(liveSqlOperations.get(0),"show tables", opHandle2.getHandleIdentifier().toString(), false);
 
     session.closeOperation(opHandle2);
-    liveSqlOperations = sessionManager.getOperationManager().getLiveSqlOperations();
-    historicSqlOperations = sessionManager.getOperationManager().getHistoricalSQLOperations();
+    liveSqlOperations = sessionManager.getOperationManager().getLiveQueryInfos();
+    historicSqlOperations = sessionManager.getOperationManager().getHistoricalQueryInfos();
     Assert.assertEquals(liveSqlOperations.size(), 0);
     Assert.assertEquals(historicSqlOperations.size(), 2);
     verifyDDL(historicSqlOperations.get(1),"show databases", opHandle1.getHandleIdentifier().toString(), true);
@@ -123,23 +123,23 @@ public class TestQueryDisplay {
     session.close();
   }
 
-  private void verifyDDL(SQLOperationDisplay display, String stmt, String handle, boolean finished) {
+  private void verifyDDL(QueryInfo queryInfo, String stmt, String handle, boolean finished) {
 
-    Assert.assertEquals(display.getUserName(), "testuser");
-    Assert.assertEquals(display.getExecutionEngine(), "mr");
-    Assert.assertEquals(display.getOperationId(), handle);
-    Assert.assertTrue(display.getBeginTime() > 0 && display.getBeginTime() <= System.currentTimeMillis());
+    Assert.assertEquals(queryInfo.getUserName(), "testuser");
+    Assert.assertEquals(queryInfo.getExecutionEngine(), "mr");
+    Assert.assertEquals(queryInfo.getOperationId(), handle);
+    Assert.assertTrue(queryInfo.getBeginTime() > 0 && queryInfo.getBeginTime() <= System.currentTimeMillis());
 
     if (finished) {
-      Assert.assertTrue(display.getEndTime() > 0 && display.getEndTime() >= display.getBeginTime()
-        && display.getEndTime() <= System.currentTimeMillis());
-      Assert.assertTrue(display.getRuntime() > 0);
+      Assert.assertTrue(queryInfo.getEndTime() > 0 && queryInfo.getEndTime() >= queryInfo.getBeginTime()
+        && queryInfo.getEndTime() <= System.currentTimeMillis());
+      Assert.assertTrue(queryInfo.getRuntime() > 0);
     } else {
-      Assert.assertNull(display.getEndTime());
+      Assert.assertNull(queryInfo.getEndTime());
       //For runtime, query may have finished.
     }
 
-    QueryDisplay qDisplay1 = display.getQueryDisplay();
+    QueryDisplay qDisplay1 = queryInfo.getQueryDisplay();
     Assert.assertNotNull(qDisplay1);
     Assert.assertEquals(qDisplay1.getQueryString(), stmt);
     Assert.assertNotNull(qDisplay1.getExplainPlan());
@@ -170,9 +170,9 @@ public class TestQueryDisplay {
    */
   private void verifyDDLHtml(String stmt, String opHandle) throws Exception {
     StringWriter sw = new StringWriter();
-    SQLOperationDisplay sod = sessionManager.getOperationManager().getSQLOperationDisplay(
+    QueryInfo queryInfo = sessionManager.getOperationManager().getQueryInfo(
       opHandle);
-    new QueryProfileTmpl().render(sw, sod);
+    new QueryProfileTmpl().render(sw, queryInfo);
     String html = sw.toString();
 
     Assert.assertTrue(html.contains(stmt));

http://git-wip-us.apache.org/repos/asf/hive/blob/1fecb81f/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
index d32f313..29cce9a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -138,6 +138,7 @@ public class Driver implements CommandProcessor {
   private static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
   static final private LogHelper console = new LogHelper(LOG);
   static final int SHUTDOWN_HOOK_PRIORITY = 0;
+  private final QueryInfo queryInfo;
   private Runnable shutdownRunner = null;
 
   private int maxRows = 100;
@@ -360,18 +361,22 @@ public class Driver implements CommandProcessor {
   }
 
   public Driver(HiveConf conf, String userName) {
-    this(new QueryState(conf), userName);
+    this(new QueryState(conf), userName, null);
   }
 
   public Driver(QueryState queryState, String userName) {
-    this(queryState, userName, new HooksLoader(queryState.getConf()));
+    this(queryState, userName, new HooksLoader(queryState.getConf()), null);
   }
 
   public Driver(HiveConf conf, HooksLoader hooksLoader) {
-    this(new QueryState(conf), null, hooksLoader);
+    this(new QueryState(conf), null, hooksLoader, null);
   }
 
-  private Driver(QueryState queryState, String userName, HooksLoader hooksLoader) {
+  public Driver(QueryState queryState, String userName, QueryInfo queryInfo) {
+     this(queryState, userName, new HooksLoader(queryState.getConf()), queryInfo);
+  }
+
+  public Driver(QueryState queryState, String userName, HooksLoader hooksLoader, QueryInfo queryInfo) {
     this.queryState = queryState;
     this.conf = queryState.getConf();
     isParallelEnabled = (conf != null)
@@ -379,6 +384,7 @@ public class Driver implements CommandProcessor {
     this.userName = userName;
     this.hooksLoader = hooksLoader;
     this.queryLifeTimeHookRunner = new QueryLifeTimeHookRunner(conf, hooksLoader, console);
+    this.queryInfo = queryInfo;
   }
 
   /**
@@ -1736,7 +1742,7 @@ public class Driver implements CommandProcessor {
 
       hookContext = new HookContext(plan, queryState, ctx.getPathToCS(), ss.getUserFromAuthenticator(),
           ss.getUserIpAddress(), InetAddress.getLocalHost().getHostAddress(), operationId,
-          ss.getSessionId(), Thread.currentThread().getName(), ss.isHiveServerQuery(), perfLogger);
+          ss.getSessionId(), Thread.currentThread().getName(), ss.isHiveServerQuery(), perfLogger, queryInfo);
       hookContext.setHookType(HookContext.HookType.PRE_EXEC_HOOK);
 
       for (Hook peh : hooksLoader.getHooks(HiveConf.ConfVars.PREEXECHOOKS, console)) {

http://git-wip-us.apache.org/repos/asf/hive/blob/1fecb81f/ql/src/java/org/apache/hadoop/hive/ql/QueryInfo.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/QueryInfo.java b/ql/src/java/org/apache/hadoop/hive/ql/QueryInfo.java
new file mode 100644
index 0000000..adb72a7
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/QueryInfo.java
@@ -0,0 +1,102 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.hadoop.hive.ql;
+
+/**
+ * The class is synchronized, as WebUI may access information about a running query.
+ */
+public class QueryInfo {
+
+  private final String userName;
+  private final String executionEngine;
+  private final long beginTime;
+  private final String operationId;
+  private Long runtime;  // tracks only running portion of the query.
+
+  private Long endTime;
+  private String state;
+  private QueryDisplay queryDisplay;
+
+  public QueryInfo(String state, String userName, String executionEngine, String operationId) {
+    this.state = state;
+    this.userName = userName;
+    this.executionEngine = executionEngine;
+    this.beginTime = System.currentTimeMillis();
+    this.operationId = operationId;
+  }
+
+  public synchronized long getElapsedTime() {
+    if (isRunning()) {
+      return System.currentTimeMillis() - beginTime;
+    } else {
+      return endTime - beginTime;
+    }
+  }
+
+  public synchronized boolean isRunning() {
+    return endTime == null;
+  }
+
+  public synchronized QueryDisplay getQueryDisplay() {
+    return queryDisplay;
+  }
+
+  public synchronized void setQueryDisplay(QueryDisplay queryDisplay) {
+    this.queryDisplay = queryDisplay;
+  }
+
+  public String getUserName() {
+    return userName;
+  }
+
+  public String getExecutionEngine() {
+    return executionEngine;
+  }
+
+  public synchronized String getState() {
+    return state;
+  }
+
+  public long getBeginTime() {
+    return beginTime;
+  }
+
+  public synchronized Long getEndTime() {
+    return endTime;
+  }
+
+  public synchronized void updateState(String state) {
+    this.state = state;
+  }
+
+  public String getOperationId() {
+    return operationId;
+  }
+
+  public synchronized void setEndTime() {
+    this.endTime = System.currentTimeMillis();
+  }
+
+  public synchronized void setRuntime(long runtime) {
+    this.runtime = runtime;
+  }
+
+  public synchronized Long getRuntime() {
+    return runtime;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1fecb81f/ql/src/java/org/apache/hadoop/hive/ql/hooks/HookContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/HookContext.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/HookContext.java
index 359c238..97ad3c7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/HookContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/HookContext.java
@@ -26,9 +26,11 @@ import java.util.Set;
 
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.QueryInfo;
 import org.apache.hadoop.hive.ql.QueryPlan;
 import org.apache.hadoop.hive.ql.QueryState;
 import org.apache.hadoop.hive.ql.exec.TaskRunner;
+import org.apache.hadoop.hive.ql.history.HiveHistory;
 import org.apache.hadoop.hive.ql.log.PerfLogger;
 import org.apache.hadoop.hive.ql.optimizer.lineage.LineageCtx.Index;
 import org.apache.hadoop.hive.ql.session.SessionState;
@@ -68,11 +70,12 @@ public class HookContext {
   private final String threadId;
   private final boolean isHiveServerQuery;
   private final PerfLogger perfLogger;
+  private final QueryInfo queryInfo;
 
   public HookContext(QueryPlan queryPlan, QueryState queryState,
       Map<String, ContentSummary> inputPathToContentSummary, String userName, String ipAddress,
       String hiveInstanceAddress, String operationId, String sessionId, String threadId,
-      boolean isHiveServerQuery, PerfLogger perfLogger) throws Exception {
+      boolean isHiveServerQuery, PerfLogger perfLogger, QueryInfo queryInfo) throws Exception {
     this.queryPlan = queryPlan;
     this.queryState = queryState;
     this.conf = queryState.getConf();
@@ -95,6 +98,7 @@ public class HookContext {
     this.threadId = threadId;
     this.isHiveServerQuery = isHiveServerQuery;
     this.perfLogger = perfLogger;
+    this.queryInfo = queryInfo;
   }
 
   public QueryPlan getQueryPlan() {
@@ -232,4 +236,8 @@ public class HookContext {
   public PerfLogger getPerfLogger() {
     return perfLogger;
   }
+
+  public QueryInfo getQueryInfo() {
+    return queryInfo;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/1fecb81f/service/src/jamon/org/apache/hive/tmpl/QueryProfileTmpl.jamon
----------------------------------------------------------------------
diff --git a/service/src/jamon/org/apache/hive/tmpl/QueryProfileTmpl.jamon b/service/src/jamon/org/apache/hive/tmpl/QueryProfileTmpl.jamon
index fa69eb2..5e2d68c 100644
--- a/service/src/jamon/org/apache/hive/tmpl/QueryProfileTmpl.jamon
+++ b/service/src/jamon/org/apache/hive/tmpl/QueryProfileTmpl.jamon
@@ -17,12 +17,12 @@ See the License for the specific language governing permissions and
 limitations under the License.
 </%doc>
 <%args>
-SQLOperationDisplay sod;
+QueryInfo queryInfo;
 </%args>
 <%import>
 java.util.*;
 org.apache.hadoop.hive.ql.QueryDisplay;
-org.apache.hive.service.cli.operation.SQLOperationDisplay;
+org.apache.hadoop.hive.ql.QueryInfo;
 </%import>
 <!--[if IE]>
 <!DOCTYPE html>
@@ -65,7 +65,7 @@ org.apache.hive.service.cli.operation.SQLOperationDisplay;
     </div>
 
 
-    <%if sod == null %>
+    <%if queryInfo == null %>
          <div class="jumbotron">
            <p>Query not found.  It may have been deleted, increase <i>hive.server2.webui.max.historic.queries</i>
               to retain more historic query information.</p>
@@ -76,7 +76,7 @@ org.apache.hive.service.cli.operation.SQLOperationDisplay;
     <div class="container">
       <div class="row inner_header">
         <div class="page-header">
-          <h1>Query Information: <% sod.getQueryDisplay() == null ? "Unknown" : sod.getQueryDisplay().getQueryString() %></h1>
+          <h1>Query Information: <% queryInfo.getQueryDisplay() == null ? "Unknown" : queryInfo.getQueryDisplay().getQueryString() %></h1>
         </div>
       </div>
       <div class="row">
@@ -91,16 +91,16 @@ org.apache.hive.service.cli.operation.SQLOperationDisplay;
           </ul>
           <div class="tab-content" style="padding-bottom: 9px; border-bottom: 1px solid #ddd;">
               <div class="tab-pane active" id="tab_baseProfile">
-                  <& baseProfile; sod = sod &>
+                  <& baseProfile; queryInfo = queryInfo &>
               </div>
               <div class="tab-pane" id="tab_stages">
-                  <& stages; sod = sod &>
+                  <& stages; queryInfo = queryInfo &>
               </div>
               <div class="tab-pane" id="tab_queryPlan">
-                  <& queryPlan; sod = sod &>
+                  <& queryPlan; queryInfo = queryInfo &>
               </div>
               <div class="tab-pane" id="tab_perfLogging">
-                  <& perfLogging; sod = sod &>
+                  <& perfLogging; queryInfo = queryInfo &>
               </div>
           </div>
       </div>
@@ -117,57 +117,57 @@ org.apache.hive.service.cli.operation.SQLOperationDisplay;
 
 <%def baseProfile>
 <%args>
-    SQLOperationDisplay sod;
+    QueryInfo queryInfo;
 </%args>
     <table class="table table-striped">
         <tr>
             <td>User Name</td>
-            <td><% sod.getUserName() %></td>
+            <td><% queryInfo.getUserName() %></td>
         </tr>
         <tr>
             <td>Query String</td>
-            <td><% sod.getQueryDisplay() == null ? "Unknown" : sod.getQueryDisplay().getQueryString() %></td>
+            <td><% queryInfo.getQueryDisplay() == null ? "Unknown" : queryInfo.getQueryDisplay().getQueryString() %></td>
         </tr>
         <tr>
             <td>Id</td>
-            <td><% sod.getQueryDisplay() == null ? "Unknown" : sod.getQueryDisplay().getQueryId() %></td>
+            <td><% queryInfo.getQueryDisplay() == null ? "Unknown" : queryInfo.getQueryDisplay().getQueryId() %></td>
         </tr>
         <tr>
             <td>Execution Engine</td>
-            <td><% sod.getExecutionEngine() %>
+            <td><% queryInfo.getExecutionEngine() %>
         </tr>
         <tr>
             <td>State</td>
-            <td><% sod.getState() %></td>
+            <td><% queryInfo.getState() %></td>
         </tr>
         <tr>
             <td>Opened Timestamp</td>
-            <td><% new Date(sod.getBeginTime()) %></td>
+            <td><% new Date(queryInfo.getBeginTime()) %></td>
         </tr>
         <tr>
             <td>Opened (s)</td>
-            <td><% sod.getElapsedTime()/1000 %></td>
+            <td><% queryInfo.getElapsedTime()/1000 %></td>
         </tr>
         <tr>
             <td>Closed Timestamp</td>
-            <td><% sod.getEndTime() == null ? "Open" : new Date(sod.getEndTime()) %></td>
+            <td><% queryInfo.getEndTime() == null ? "Open" : new Date(queryInfo.getEndTime()) %></td>
         </tr>
-        <%if sod.getQueryDisplay() != null && sod.getQueryDisplay().getErrorMessage() != null %>
+        <%if queryInfo.getQueryDisplay() != null && queryInfo.getQueryDisplay().getErrorMessage() != null %>
             <tr>
                 <td>Error</td>
-                <td><% sod.getQueryDisplay().getErrorMessage() %></td>
+                <td><% queryInfo.getQueryDisplay().getErrorMessage() %></td>
             </tr>
         </%if>
         <tr>
             <td>Latency (s)</td>
-            <td><% sod.getRuntime() == null ? "Not finished" : sod.getRuntime()/1000 %></td>
+            <td><% queryInfo.getRuntime() == null ? "Not finished" : queryInfo.getRuntime()/1000 %></td>
         </tr>
     </table>
 </%def>
 
 <%def stages>
 <%args>
-    SQLOperationDisplay sod;
+    QueryInfo queryInfo;
 </%args>
    <table class="table table-striped">
        <tr>
@@ -180,8 +180,8 @@ org.apache.hive.service.cli.operation.SQLOperationDisplay;
            <th>Retry If Fail</th>
         </tr>
 
-       <%if sod.getQueryDisplay() != null && sod.getQueryDisplay().getTaskDisplays() != null %>
-           <%for QueryDisplay.TaskDisplay taskDisplay : sod.getQueryDisplay().getTaskDisplays() %>
+       <%if queryInfo.getQueryDisplay() != null && queryInfo.getQueryDisplay().getTaskDisplays() != null %>
+           <%for QueryDisplay.TaskDisplay taskDisplay : queryInfo.getQueryDisplay().getTaskDisplays() %>
                <tr>
                    <td><% taskDisplay.getTaskId() + ":" + taskDisplay.getTaskType() %></td>
                    <td><% taskDisplay.getStatus() %></td>
@@ -199,13 +199,13 @@ org.apache.hive.service.cli.operation.SQLOperationDisplay;
 
 <%def queryPlan>
 <%args>
-    SQLOperationDisplay sod;
+    QueryInfo queryInfo;
 </%args>
     <div class="panel panel-default">
       <div class="panel-heading">Explain plan</div>
       <div class="panel-body">
         <pre>
-        <% sod.getQueryDisplay() == null ? "Unknown" : sod.getQueryDisplay().getExplainPlan() %>
+        <% queryInfo.getQueryDisplay() == null ? "Unknown" : queryInfo.getQueryDisplay().getExplainPlan() %>
         </pre>
       </div>
     </div>
@@ -214,7 +214,7 @@ org.apache.hive.service.cli.operation.SQLOperationDisplay;
 
 <%def perfLogging>
 <%args>
-    SQLOperationDisplay sod;
+    QueryInfo queryInfo;
 </%args>
     <section>
       <h3>Compile-time metadata operations</h3>
@@ -224,8 +224,8 @@ org.apache.hive.service.cli.operation.SQLOperationDisplay;
              <th>Time (ms)</th>
           </tr>
 
-          <%if sod.getQueryDisplay() != null && sod.getQueryDisplay().getHmsTimings(QueryDisplay.Phase.COMPILATION) != null %>
-             <%for Map.Entry<String, Long> time : sod.getQueryDisplay().getHmsTimings(QueryDisplay.Phase.COMPILATION).entrySet() %>
+          <%if queryInfo.getQueryDisplay() != null && queryInfo.getQueryDisplay().getHmsTimings(QueryDisplay.Phase.COMPILATION) != null %>
+             <%for Map.Entry<String, Long> time : queryInfo.getQueryDisplay().getHmsTimings(QueryDisplay.Phase.COMPILATION).entrySet() %>
                  <tr>
                      <td><% time.getKey() %></td>
                      <td><% time.getValue() %></td>
@@ -243,8 +243,8 @@ org.apache.hive.service.cli.operation.SQLOperationDisplay;
              <th>Time (ms)</th>
           </tr>
 
-          <%if sod.getQueryDisplay() != null && sod.getQueryDisplay().getHmsTimings(QueryDisplay.Phase.EXECUTION) != null %>
-             <%for Map.Entry<String, Long> time : sod.getQueryDisplay().getHmsTimings(QueryDisplay.Phase.EXECUTION).entrySet() %>
+          <%if queryInfo.getQueryDisplay() != null && queryInfo.getQueryDisplay().getHmsTimings(QueryDisplay.Phase.EXECUTION) != null %>
+             <%for Map.Entry<String, Long> time : queryInfo.getQueryDisplay().getHmsTimings(QueryDisplay.Phase.EXECUTION).entrySet() %>
                  <tr>
                      <td><% time.getKey() %></td>
                      <td><% time.getValue() %></td>
@@ -262,8 +262,8 @@ org.apache.hive.service.cli.operation.SQLOperationDisplay;
              <th>Time (ms)</th>
           </tr>
 
-          <%if sod.getQueryDisplay() != null && sod.getQueryDisplay().getPerfLogTimes(QueryDisplay.Phase.COMPILATION) != null %>
-             <%for Map.Entry<String, Long> time : sod.getQueryDisplay().getPerfLogTimes(QueryDisplay.Phase.COMPILATION).entrySet()  %>
+          <%if queryInfo.getQueryDisplay() != null && queryInfo.getQueryDisplay().getPerfLogTimes(QueryDisplay.Phase.COMPILATION) != null %>
+             <%for Map.Entry<String, Long> time : queryInfo.getQueryDisplay().getPerfLogTimes(QueryDisplay.Phase.COMPILATION).entrySet()  %>
                  <tr>
                      <td><% time.getKey() %></td>
                      <td><% time.getValue() %></td>
@@ -281,8 +281,8 @@ org.apache.hive.service.cli.operation.SQLOperationDisplay;
              <th>Time (ms)</th>
           </tr>
 
-          <%if sod.getQueryDisplay() != null && sod.getQueryDisplay().getPerfLogTimes(QueryDisplay.Phase.EXECUTION) != null %>
-             <%for Map.Entry<String, Long> time : sod.getQueryDisplay().getPerfLogTimes(QueryDisplay.Phase.EXECUTION).entrySet()  %>
+          <%if queryInfo.getQueryDisplay() != null && queryInfo.getQueryDisplay().getPerfLogTimes(QueryDisplay.Phase.EXECUTION) != null %>
+             <%for Map.Entry<String, Long> time : queryInfo.getQueryDisplay().getPerfLogTimes(QueryDisplay.Phase.EXECUTION).entrySet()  %>
                  <tr>
                      <td><% time.getKey() %></td>
                      <td><% time.getValue() %></td>

http://git-wip-us.apache.org/repos/asf/hive/blob/1fecb81f/service/src/java/org/apache/hive/service/cli/operation/OperationManager.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/OperationManager.java b/service/src/java/org/apache/hive/service/cli/operation/OperationManager.java
index ac64ab2..46f524d 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/OperationManager.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/OperationManager.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Schema;
+import org.apache.hadoop.hive.ql.QueryInfo;
 import org.apache.hadoop.hive.ql.log.LogDivertAppender;
 import org.apache.hadoop.hive.ql.log.LogDivertAppenderForTest;
 import org.apache.hadoop.hive.ql.session.OperationLog;
@@ -62,8 +63,8 @@ public class OperationManager extends AbstractService {
 
   //Following fields for displaying queries on WebUI
   private Object webuiLock = new Object();
-  private SQLOperationDisplayCache historicSqlOperations;
-  private Map<String, SQLOperationDisplay> liveSqlOperations = new LinkedHashMap<String, SQLOperationDisplay>();
+  private QueryInfoCache historicalQueryInfos;
+  private Map<String, QueryInfo> liveQueryInfos = new LinkedHashMap<>();
 
   public OperationManager() {
     super(OperationManager.class.getSimpleName());
@@ -75,7 +76,7 @@ public class OperationManager extends AbstractService {
     LogDivertAppenderForTest.registerRoutingAppenderIfInTest(hiveConf);
 
     if (hiveConf.isWebUiQueryInfoCacheEnabled()) {
-      historicSqlOperations = new SQLOperationDisplayCache(
+      historicalQueryInfos = new QueryInfoCache(
         hiveConf.getIntVar(ConfVars.HIVE_SERVER2_WEBUI_MAX_HISTORIC_QUERIES));
     }
     super.init(hiveConf);
@@ -187,8 +188,8 @@ public class OperationManager extends AbstractService {
     handleToOperation.put(operation.getHandle(), operation);
     if (operation instanceof SQLOperation) {
       synchronized (webuiLock) {
-        liveSqlOperations.put(operation.getHandle().getHandleIdentifier().toString(),
-          ((SQLOperation) operation).getSQLOperationDisplay());
+        liveQueryInfos.put(operation.getHandle().getHandleIdentifier().toString(),
+          ((SQLOperation) operation).getQueryInfo());
       }
     }
   }
@@ -196,7 +197,7 @@ public class OperationManager extends AbstractService {
   private Operation removeOperation(OperationHandle opHandle) {
     Operation operation = handleToOperation.remove(opHandle);
     if (operation instanceof SQLOperation) {
-      removeSaveSqlOperationDisplay(opHandle);
+      removeSafeQueryInfo(opHandle);
     }
     return operation;
   }
@@ -216,24 +217,24 @@ public class OperationManager extends AbstractService {
 
       handleToOperation.remove(operationHandle, operation);
       if (operation instanceof SQLOperation) {
-        removeSaveSqlOperationDisplay(operationHandle);
+        removeSafeQueryInfo(operationHandle);
       }
       return operation;
     }
     return null;
   }
 
-  private void removeSaveSqlOperationDisplay(OperationHandle operationHandle) {
+  private void removeSafeQueryInfo(OperationHandle operationHandle) {
     synchronized (webuiLock) {
       String opKey = operationHandle.getHandleIdentifier().toString();
       // remove from list of live operations
-      SQLOperationDisplay display = liveSqlOperations.remove(opKey);
+      QueryInfo display = liveQueryInfos.remove(opKey);
       if (display == null) {
         LOG.debug("Unexpected display object value of null for operation {}",
             opKey);
-      } else if (historicSqlOperations != null) {
+      } else if (historicalQueryInfos != null) {
         // add to list of saved historic operations
-        historicSqlOperations.put(opKey, display);
+        historicalQueryInfos.put(opKey, display);
       }
     }
   }
@@ -258,7 +259,7 @@ public class OperationManager extends AbstractService {
       LOG.debug(opHandle + ": Attempting to cancel from state - " + opState);
       operation.cancel(OperationState.CANCELED);
       if (operation instanceof SQLOperation) {
-        removeSaveSqlOperationDisplay(opHandle);
+        removeSafeQueryInfo(opHandle);
       }
     }
   }
@@ -360,11 +361,11 @@ public class OperationManager extends AbstractService {
    * @return displays representing a number of historical SQLOperations, at max number of
    * hive.server2.webui.max.historic.queries. Newest items will be first.
    */
-  public List<SQLOperationDisplay> getHistoricalSQLOperations() {
-    List<SQLOperationDisplay> result = new LinkedList<>();
+  public List<QueryInfo> getHistoricalQueryInfos() {
+    List<QueryInfo> result = new LinkedList<>();
     synchronized (webuiLock) {
-      if (historicSqlOperations != null) {
-        result.addAll(historicSqlOperations.values());
+      if (historicalQueryInfos != null) {
+        result.addAll(historicalQueryInfos.values());
         Collections.reverse(result);
       }
     }
@@ -374,10 +375,10 @@ public class OperationManager extends AbstractService {
   /**
    * @return displays representing live SQLOperations
    */
-  public List<SQLOperationDisplay> getLiveSqlOperations() {
-    List<SQLOperationDisplay> result = new LinkedList<>();
+  public List<QueryInfo> getLiveQueryInfos() {
+    List<QueryInfo> result = new LinkedList<>();
     synchronized (webuiLock) {
-      result.addAll(liveSqlOperations.values());
+      result.addAll(liveQueryInfos.values());
     }
     return result;
   }
@@ -386,17 +387,17 @@ public class OperationManager extends AbstractService {
    * @param handle handle of SQLOperation.
    * @return display representing a particular SQLOperation.
    */
-  public SQLOperationDisplay getSQLOperationDisplay(String handle) {
+  public QueryInfo getQueryInfo(String handle) {
     synchronized (webuiLock) {
-      if (historicSqlOperations == null) {
+      if (historicalQueryInfos == null) {
         return null;
       }
 
-      SQLOperationDisplay result = liveSqlOperations.get(handle);
+      QueryInfo result = liveQueryInfos.get(handle);
       if (result != null) {
         return result;
       }
-      return historicSqlOperations.get(handle);
+      return historicalQueryInfos.get(handle);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/1fecb81f/service/src/java/org/apache/hive/service/cli/operation/QueryInfoCache.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/QueryInfoCache.java b/service/src/java/org/apache/hive/service/cli/operation/QueryInfoCache.java
new file mode 100644
index 0000000..6c9443a
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/cli/operation/QueryInfoCache.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.hive.service.cli.operation;
+
+import org.apache.hadoop.hive.ql.QueryInfo;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * Cache some SQLOperation information for WebUI
+ */
+public class QueryInfoCache extends LinkedHashMap<String, QueryInfo> {
+
+  private final int capacity;
+
+  public QueryInfoCache(int capacity) {
+      super(capacity + 1, 1.1f, false);
+      this.capacity = capacity;
+  }
+
+  @Override
+  protected boolean removeEldestEntry(Map.Entry eldest) {
+    return size() > capacity;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/1fecb81f/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
index d8718b3..0b51591 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.hive.metastore.api.Schema;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.QueryDisplay;
+import org.apache.hadoop.hive.ql.QueryInfo;
 import org.apache.hadoop.hive.ql.QueryState;
 import org.apache.hadoop.hive.ql.exec.ExplainTask;
 import org.apache.hadoop.hive.ql.exec.FetchTask;
@@ -58,7 +59,6 @@ import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.log.PerfLogger;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
-import org.apache.hadoop.hive.ql.session.OperationLog;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.AbstractSerDe;
@@ -68,7 +68,6 @@ import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.StructField;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
-import org.apache.hadoop.hive.serde2.thrift.ThriftJDBCBinarySerDe;
 import org.apache.hadoop.hive.shims.Utils;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -97,8 +96,7 @@ public class SQLOperation extends ExecuteStatementOperation {
   private AbstractSerDe serde = null;
   private boolean fetchStarted = false;
   private volatile MetricsScope currentSQLStateScope;
-  // Display for WebUI.
-  private SQLOperationDisplay sqlOpDisplay;
+  private QueryInfo queryInfo;
   private long queryTimeout;
   private ScheduledExecutorService timeoutExecutor;
   private final boolean runAsync;
@@ -123,11 +121,9 @@ public class SQLOperation extends ExecuteStatementOperation {
     }
 
     setupSessionIO(parentSession.getSessionState());
-    try {
-      sqlOpDisplay = new SQLOperationDisplay(this);
-    } catch (HiveSQLException e) {
-      LOG.warn("Error calcluating SQL Operation Display for webui", e);
-    }
+
+    queryInfo = new QueryInfo(getState().toString(), getParentSession().getUserName(),
+            getExecutionEngine(), getHandle().getHandleIdentifier().toString());
 
     Metrics metrics = MetricsFactory.getInstance();
     if (metrics != null) {
@@ -157,13 +153,13 @@ public class SQLOperation extends ExecuteStatementOperation {
 
   /**
    * Compile the query and extract metadata
-   * @param sqlOperationConf
+   *
    * @throws HiveSQLException
    */
   public void prepare(QueryState queryState) throws HiveSQLException {
     setState(OperationState.RUNNING);
     try {
-      driver = new Driver(queryState, getParentSession().getUserName());
+      driver = new Driver(queryState, getParentSession().getUserName(), queryInfo);
 
       // Start the timer thread for cancelling the query when query timeout is reached
       // queryTimeout == 0 means no timeout
@@ -188,7 +184,7 @@ public class SQLOperation extends ExecuteStatementOperation {
         timeoutExecutor.schedule(timeoutTask, queryTimeout, TimeUnit.SECONDS);
       }
 
-      sqlOpDisplay.setQueryDisplay(driver.getQueryDisplay());
+      queryInfo.setQueryDisplay(driver.getQueryDisplay());
 
       // set the operation handle information in Driver, so that thrift API users
       // can use the operation handle they receive, to lookup query information in
@@ -379,8 +375,9 @@ public class SQLOperation extends ExecuteStatementOperation {
 
   /**
    * Returns the current UGI on the stack
-   * @param opConfig
+   *
    * @return UserGroupInformation
+   *
    * @throws HiveSQLException
    */
   private UserGroupInformation getCurrentUGI() throws HiveSQLException {
@@ -623,8 +620,8 @@ public class SQLOperation extends ExecuteStatementOperation {
   /**
    * Get summary information of this SQLOperation for display in WebUI.
    */
-  public SQLOperationDisplay getSQLOperationDisplay() {
-    return sqlOpDisplay;
+  public QueryInfo getQueryInfo() {
+    return queryInfo;
   }
 
   @Override
@@ -649,17 +646,17 @@ public class SQLOperation extends ExecuteStatementOperation {
 
     if (state == OperationState.FINISHED || state == OperationState.CANCELED || state == OperationState.ERROR) {
       //update runtime
-      sqlOpDisplay.setRuntime(getOperationComplete() - getOperationStart());
+      queryInfo.setRuntime(getOperationComplete() - getOperationStart());
       if (metrics != null && submittedQryScp != null) {
         metrics.endScope(submittedQryScp);
       }
     }
 
     if (state == OperationState.CLOSED) {
-      sqlOpDisplay.closed();
+      queryInfo.setEndTime();
     } else {
       //CLOSED state not interesting, state before (FINISHED, ERROR) is.
-      sqlOpDisplay.updateState(state);
+      queryInfo.updateState(state.toString());
     }
 
     if (state == OperationState.ERROR) {

http://git-wip-us.apache.org/repos/asf/hive/blob/1fecb81f/service/src/java/org/apache/hive/service/cli/operation/SQLOperationDisplay.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/SQLOperationDisplay.java b/service/src/java/org/apache/hive/service/cli/operation/SQLOperationDisplay.java
deleted file mode 100644
index 3cfeccc..0000000
--- a/service/src/java/org/apache/hive/service/cli/operation/SQLOperationDisplay.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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 org.apache.hive.service.cli.operation;
-
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.QueryDisplay;
-import org.apache.hive.service.cli.HiveSQLException;
-import org.apache.hive.service.cli.OperationState;
-
-/**
- * Used to display some info in the HS2 WebUI.
- *
- * The class is synchronized, as WebUI may access information about a running query.
- */
-public class SQLOperationDisplay {
-  public final String userName;
-  public final String executionEngine;
-  public final long beginTime;
-  public final String operationId;
-  public Long runtime;  //tracks only running portion of the query.
-
-  public Long endTime;
-  public OperationState state;
-  public QueryDisplay queryDisplay;
-
-  public SQLOperationDisplay(SQLOperation sqlOperation) throws HiveSQLException {
-    this.state = sqlOperation.getState();
-    this.userName = sqlOperation.getParentSession().getUserName();
-    this.executionEngine = sqlOperation.getExecutionEngine();
-    this.beginTime = System.currentTimeMillis();
-    this.operationId = sqlOperation.getHandle().getHandleIdentifier().toString();
-  }
-
-  public synchronized long getElapsedTime() {
-    if (isRunning()) {
-      return System.currentTimeMillis() - beginTime;
-    } else {
-      return endTime - beginTime;
-    }
-  }
-
-  public synchronized boolean isRunning() {
-    return endTime == null;
-  }
-
-  public synchronized QueryDisplay getQueryDisplay() {
-    return queryDisplay;
-  }
-
-  public synchronized void setQueryDisplay(QueryDisplay queryDisplay) {
-    this.queryDisplay = queryDisplay;
-  }
-
-  public String getUserName() {
-    return userName;
-  }
-
-  public String getExecutionEngine() {
-    return executionEngine;
-  }
-
-  public synchronized OperationState getState() {
-    return state;
-  }
-
-  public long getBeginTime() {
-    return beginTime;
-  }
-
-  public synchronized Long getEndTime() {
-    return endTime;
-  }
-
-  public synchronized void updateState(OperationState state) {
-    this.state = state;
-  }
-
-  public String getOperationId() {
-    return operationId;
-  }
-
-  public synchronized void closed() {
-    this.endTime = System.currentTimeMillis();
-  }
-
-  public synchronized void setRuntime(long runtime) {
-    this.runtime = runtime;
-  }
-
-  public synchronized Long getRuntime() {
-    return runtime;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/1fecb81f/service/src/java/org/apache/hive/service/cli/operation/SQLOperationDisplayCache.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/SQLOperationDisplayCache.java b/service/src/java/org/apache/hive/service/cli/operation/SQLOperationDisplayCache.java
deleted file mode 100644
index 4a33d37..0000000
--- a/service/src/java/org/apache/hive/service/cli/operation/SQLOperationDisplayCache.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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 org.apache.hive.service.cli.operation;
-
-import java.util.LinkedHashMap;
-import java.util.Map;
-
-/**
- * Cache some SQLOperation information for WebUI
- */
-public class SQLOperationDisplayCache extends LinkedHashMap<String, SQLOperationDisplay> {
-
-  private final int capacity;
-
-  public SQLOperationDisplayCache(int capacity) {
-      super(capacity + 1, 1.1f, false);
-      this.capacity = capacity;
-  }
-
-  @Override
-  protected boolean removeEldestEntry(Map.Entry eldest) {
-    return size() > capacity;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/1fecb81f/service/src/java/org/apache/hive/service/servlet/QueryProfileServlet.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/servlet/QueryProfileServlet.java b/service/src/java/org/apache/hive/service/servlet/QueryProfileServlet.java
index 8fa447a..27a3b14 100644
--- a/service/src/java/org/apache/hive/service/servlet/QueryProfileServlet.java
+++ b/service/src/java/org/apache/hive/service/servlet/QueryProfileServlet.java
@@ -17,8 +17,8 @@
  */
 package org.apache.hive.service.servlet;
 
+import org.apache.hadoop.hive.ql.QueryInfo;
 import org.apache.hive.service.cli.operation.OperationManager;
-import org.apache.hive.service.cli.operation.SQLOperationDisplay;
 import org.apache.hive.service.cli.session.SessionManager;
 import org.apache.hive.tmpl.QueryProfileTmpl;
 import org.slf4j.Logger;
@@ -46,12 +46,12 @@ public class QueryProfileServlet extends HttpServlet {
     SessionManager sessionManager =
       (SessionManager)ctx.getAttribute("hive.sm");
     OperationManager opManager = sessionManager.getOperationManager();
-    SQLOperationDisplay sod = opManager.getSQLOperationDisplay(opId);
-    if (sod == null) {
+    QueryInfo queryInfo = opManager.getQueryInfo(opId);
+    if (queryInfo == null) {
       LOG.debug("No display object found for operation {} ", opId);
       return;
     }
 
-    new QueryProfileTmpl().render(response.getWriter(), sod);
+    new QueryProfileTmpl().render(response.getWriter(), queryInfo);
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/1fecb81f/service/src/resources/hive-webapps/hiveserver2/hiveserver2.jsp
----------------------------------------------------------------------
diff --git a/service/src/resources/hive-webapps/hiveserver2/hiveserver2.jsp b/service/src/resources/hive-webapps/hiveserver2/hiveserver2.jsp
index 0e0803b..c0ece6d 100644
--- a/service/src/resources/hive-webapps/hiveserver2/hiveserver2.jsp
+++ b/service/src/resources/hive-webapps/hiveserver2/hiveserver2.jsp
@@ -24,7 +24,7 @@
   import="org.apache.hive.common.util.HiveVersionInfo"
   import="org.apache.hive.service.cli.operation.Operation"
   import="org.apache.hive.service.cli.operation.SQLOperation"
-  import="org.apache.hive.service.cli.operation.SQLOperationDisplay"
+  import="org.apache.hadoop.hive.ql.QueryInfo"
   import="org.apache.hive.service.cli.session.SessionManager"
   import="org.apache.hive.service.cli.session.HiveSession"
   import="javax.servlet.ServletContext"
@@ -141,8 +141,8 @@ for (HiveSession hiveSession: hiveSessions) {
     </tr>
     <%
       int queries = 0;
-      Collection<SQLOperationDisplay> operations = sessionManager.getOperationManager().getLiveSqlOperations();
-      for (SQLOperationDisplay operation : operations) {
+      Collection<QueryInfo> operations = sessionManager.getOperationManager().getLiveQueryInfos();
+      for (QueryInfo operation : operations) {
           queries++;
     %>
     <tr>
@@ -182,8 +182,8 @@ for (HiveSession hiveSession: hiveSessions) {
     </tr>
     <%
       queries = 0;
-      operations = sessionManager.getOperationManager().getHistoricalSQLOperations();
-      for (SQLOperationDisplay operation : operations) {
+      operations = sessionManager.getOperationManager().getHistoricalQueryInfos();
+      for (QueryInfo operation : operations) {
           queries++;
     %>
     <tr>

http://git-wip-us.apache.org/repos/asf/hive/blob/1fecb81f/service/src/test/org/apache/hive/service/cli/operation/TestQueryLifeTimeHooksWithSQLOperation.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/cli/operation/TestQueryLifeTimeHooksWithSQLOperation.java b/service/src/test/org/apache/hive/service/cli/operation/TestQueryLifeTimeHooksWithSQLOperation.java
new file mode 100644
index 0000000..c4f5451
--- /dev/null
+++ b/service/src/test/org/apache/hive/service/cli/operation/TestQueryLifeTimeHooksWithSQLOperation.java
@@ -0,0 +1,115 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 org.apache.hive.service.cli.operation;
+
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.hooks.QueryLifeTimeHookContext;
+import org.apache.hadoop.hive.ql.hooks.QueryLifeTimeHookWithParseHooks;
+import org.apache.hadoop.hive.ql.hooks.TestQueryHooks;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+import org.apache.hive.service.cli.HiveSQLException;
+import org.apache.hive.service.cli.session.HiveSession;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+
+public class TestQueryLifeTimeHooksWithSQLOperation {
+
+  private static final String QUERY = "select 1";
+
+  @Test
+  public void testQueryInfoInHookContext() throws IllegalAccessException, ClassNotFoundException, InstantiationException, HiveSQLException {
+    HiveConf conf = new HiveConf(TestQueryHooks.class);
+    conf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
+    conf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER,
+            "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory");
+    conf.set(HiveConf.ConfVars.HIVE_QUERY_LIFETIME_HOOKS.varname, QueryInfoVerificationHook.class.getName());
+
+    SessionState.start(conf);
+    HiveSession mockHiveSession = mock(HiveSession.class);
+    when(mockHiveSession.getHiveConf()).thenReturn(conf);
+    when(mockHiveSession.getSessionState()).thenReturn(SessionState.get());
+    SQLOperation sqlOperation = new SQLOperation(mockHiveSession, QUERY, ImmutableMap.of(), false, 0);
+    sqlOperation.run();
+  }
+
+  /**
+   * Has to be public so that it can be created by the Driver
+   */
+  public final static class QueryInfoVerificationHook implements QueryLifeTimeHookWithParseHooks {
+
+    @Override
+    public void beforeParse(QueryLifeTimeHookContext ctx) {
+      assertNotNull(ctx);
+      assertEquals(ctx.getCommand().trim(), QUERY);
+    }
+
+    @Override
+    public void afterParse(QueryLifeTimeHookContext ctx, boolean hasError) {
+      assertNotNull(ctx);
+      assertEquals(ctx.getCommand().trim(), QUERY);
+      assertFalse(hasError);
+    }
+
+    @Override
+    public void beforeCompile(QueryLifeTimeHookContext ctx) {
+      assertNotNull(ctx);
+      assertEquals(ctx.getCommand().trim(), QUERY);
+    }
+
+    @Override
+    public void afterCompile(QueryLifeTimeHookContext ctx, boolean hasError) {
+      assertNotNull(ctx);
+      assertEquals(ctx.getCommand().trim(), QUERY);
+      assertFalse(hasError);
+    }
+
+    @Override
+    public void beforeExecution(QueryLifeTimeHookContext ctx) {
+      assertNotNull(ctx);
+      assertEquals(ctx.getCommand().trim(), QUERY);
+      assertNotNull(ctx.getHookContext());
+      assertNotNull(ctx.getHookContext().getQueryInfo());
+      assertNotNull(ctx.getHookContext().getQueryInfo().getQueryDisplay());
+    }
+
+    @Override
+    public void afterExecution(QueryLifeTimeHookContext ctx, boolean hasError) {
+      assertNotNull(ctx);
+      assertEquals(ctx.getCommand().trim(), QUERY);
+      assertFalse(hasError);
+      assertNotNull(ctx.getHookContext());
+      assertNull(ctx.getHookContext().getErrorMessage());
+      assertNull(ctx.getHookContext().getException());
+      assertNotNull(ctx.getHookContext().getQueryInfo());
+      assertNotNull(ctx.getHookContext().getQueryInfo().getQueryDisplay());
+    }
+  }
+}