You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by su...@apache.org on 2016/08/02 16:21:00 UTC

hive git commit: HIVE-14340: Add a new hook triggers before query compilation and after query execution (Chao Sun, reviewed by Xuefu Zhang)

Repository: hive
Updated Branches:
  refs/heads/master 78a90a62d -> f7c51b6ce


HIVE-14340: Add a new hook triggers before query compilation and after query execution (Chao Sun, reviewed by Xuefu Zhang)


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

Branch: refs/heads/master
Commit: f7c51b6cee5dddf47d58781f1c6087e0ac237cd3
Parents: 78a90a6
Author: Chao Sun <su...@apache.org>
Authored: Tue Aug 2 09:20:40 2016 -0700
Committer: Chao Sun <su...@apache.org>
Committed: Tue Aug 2 09:20:40 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   4 +-
 .../java/org/apache/hadoop/hive/ql/Driver.java  |  67 +++++++++-
 .../hadoop/hive/ql/hooks/QueryLifeTimeHook.java |  57 +++++++++
 .../hive/ql/hooks/QueryLifeTimeHookContext.java |  69 ++++++++++
 .../ql/hooks/QueryLifeTimeHookContextImpl.java  |  57 +++++++++
 .../hadoop/hive/ql/hooks/TestQueryHooks.java    | 127 +++++++++++++++++++
 .../hive/ql/hooks/TestQueryLifeTimeHook.java    |  43 +++++++
 .../clientpositive/query_lifetime_hook.q        |   3 +
 .../clientpositive/query_lifetime_hook.q.out    |  21 +++
 9 files changed, 446 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f7c51b6c/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 598684e..1b64c77 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2113,7 +2113,9 @@ public class HiveConf extends Configuration {
         "hive.exec.temporary.table.storage", "default", new StringSet("memory",
          "ssd", "default"), "Define the storage policy for temporary tables." +
          "Choices between memory, ssd and default"),
-
+    HIVE_QUERY_LIFETIME_HOOKS("hive.query.lifetime.hooks", "",
+        "A comma separated list of hooks which implement QueryLifeTimeHook. These will be triggered" +
+            " before/after query compilation and before/after query execution, in the order specified"),
     HIVE_DRIVER_RUN_HOOKS("hive.exec.driver.run.hooks", "",
         "A comma separated list of hooks which implement HiveDriverRunHook. Will be run at the beginning " +
         "and end of Driver.run, these will be run in the order specified."),

http://git-wip-us.apache.org/repos/asf/hive/blob/f7c51b6c/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 0278673..e1494da 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -63,6 +63,9 @@ import org.apache.hadoop.hive.ql.hooks.HookContext;
 import org.apache.hadoop.hive.ql.hooks.HookUtils;
 import org.apache.hadoop.hive.ql.hooks.PostExecute;
 import org.apache.hadoop.hive.ql.hooks.PreExecute;
+import org.apache.hadoop.hive.ql.hooks.QueryLifeTimeHook;
+import org.apache.hadoop.hive.ql.hooks.QueryLifeTimeHookContext;
+import org.apache.hadoop.hive.ql.hooks.QueryLifeTimeHookContextImpl;
 import org.apache.hadoop.hive.ql.hooks.ReadEntity;
 import org.apache.hadoop.hive.ql.hooks.WriteEntity;
 import org.apache.hadoop.hive.ql.lockmgr.HiveLock;
@@ -169,6 +172,9 @@ public class Driver implements CommandProcessor {
   // Query specific info
   private QueryState queryState;
 
+  // Query hooks that execute before compilation and after execution
+  List<QueryLifeTimeHook> queryHooks;
+
   private boolean checkConcurrency() {
     boolean supportConcurrency = conf.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY);
     if (!supportConcurrency) {
@@ -372,6 +378,9 @@ public class Driver implements CommandProcessor {
 
     SessionState.get().setupQueryCurrentTimestamp();
 
+    // Whether any error occurred during query compilation. Used for query lifetime hook.
+    boolean compileError = false;
+
     try {
       // Initialize the transaction manager.  This must be done before analyze is called.
       final HiveTxnManager txnManager = SessionState.get().initTxnMgr(conf);
@@ -403,6 +412,17 @@ public class Driver implements CommandProcessor {
       tree = ParseUtils.findRootNonNullToken(tree);
       perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.PARSE);
 
+      // Trigger query hook before compilation
+      queryHooks = getHooks(ConfVars.HIVE_QUERY_LIFETIME_HOOKS, QueryLifeTimeHook.class);
+      if (queryHooks != null && !queryHooks.isEmpty()) {
+        QueryLifeTimeHookContext qhc = new QueryLifeTimeHookContextImpl();
+        qhc.setHiveConf(conf);
+        qhc.setCommand(command);
+
+        for (QueryLifeTimeHook hook : queryHooks) {
+          hook.beforeCompile(qhc);
+        }
+      }
 
       perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.ANALYZE);
       BaseSemanticAnalyzer sem = SemanticAnalyzerFactory.get(queryState, tree);
@@ -491,8 +511,11 @@ public class Driver implements CommandProcessor {
           }
         }
       }
+
       return 0;
     } catch (Exception e) {
+      compileError = true;
+
       ErrorMsg error = ErrorMsg.getErrorMsg(e.getMessage());
       errorMessage = "FAILED: " + e.getClass().getSimpleName();
       if (error != ErrorMsg.GENERIC_ERROR) {
@@ -517,6 +540,19 @@ public class Driver implements CommandProcessor {
       return error.getErrorCode();//todo: this is bad if returned as cmd shell exit
       // since it exceeds valid range of shell return values
     } finally {
+
+      // Trigger post compilation hook. Note that if the compilation fails here then
+      // before/after execution hook will never be executed.
+      if (queryHooks != null && !queryHooks.isEmpty()) {
+        QueryLifeTimeHookContext qhc = new QueryLifeTimeHookContextImpl();
+        qhc.setHiveConf(conf);
+        qhc.setCommand(command);
+
+        for (QueryLifeTimeHook hook : queryHooks) {
+          hook.afterCompile(qhc, compileError);
+        }
+      }
+
       double duration = perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.COMPILE)/1000.00;
       ImmutableMap<String, Long> compileHMSTimings = dumpMetaCallTimingWithoutEx("compilation");
       queryDisplay.setHmsTimings(QueryDisplay.Phase.COMPILATION, compileHMSTimings);
@@ -1513,6 +1549,9 @@ public class Driver implements CommandProcessor {
 
     HookContext hookContext = null;
 
+    // Whether there's any error occurred during query execution. Used for query lifetime hook.
+    boolean executionError = false;
+
     try {
       LOG.info("Executing command(queryId=" + queryId + "): " + queryStr);
       // compile and execute can get called from different threads in case of HS2
@@ -1549,6 +1588,19 @@ public class Driver implements CommandProcessor {
           perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.PRE_HOOK + peh.getClass().getName());
         }
       }
+
+      // Trigger query hooks before query execution.
+      if (queryHooks != null && !queryHooks.isEmpty()) {
+        QueryLifeTimeHookContext qhc = new QueryLifeTimeHookContextImpl();
+        qhc.setHiveConf(conf);
+        qhc.setCommand(ctx.getCmd());
+        qhc.setHookContext(hookContext);
+
+        for (QueryLifeTimeHook hook : queryHooks) {
+          hook.beforeExecution(qhc);
+        }
+      }
+
       setQueryDisplays(plan.getRootTasks());
       int mrJobs = Utilities.getMRTasks(plan.getRootTasks()).size();
       int jobs = mrJobs
@@ -1715,15 +1767,16 @@ public class Driver implements CommandProcessor {
         }
       }
 
-
       if (SessionState.get() != null) {
         SessionState.get().getHiveHistory().setQueryProperty(queryId, Keys.QUERY_RET_CODE,
             String.valueOf(0));
         SessionState.get().getHiveHistory().printRowCount(queryId);
       }
     } catch (CommandNeedRetryException e) {
+      executionError = true;
       throw e;
     } catch (Exception e) {
+      executionError = true;
       ctx.restoreOriginalTracker();
       if (SessionState.get() != null) {
         SessionState.get().getHiveHistory().setQueryProperty(queryId, Keys.QUERY_RET_CODE,
@@ -1744,6 +1797,18 @@ public class Driver implements CommandProcessor {
           + org.apache.hadoop.util.StringUtils.stringifyException(e));
       return (12);
     } finally {
+      // Trigger query hooks after query completes its execution.
+      if (queryHooks != null && !queryHooks.isEmpty()) {
+        QueryLifeTimeHookContext qhc = new QueryLifeTimeHookContextImpl();
+        qhc.setHiveConf(conf);
+        qhc.setCommand(ctx.getCmd());
+        qhc.setHookContext(hookContext);
+
+        for (QueryLifeTimeHook hook : queryHooks) {
+          hook.afterExecution(qhc, executionError);
+        }
+      }
+
       if (SessionState.get() != null) {
         SessionState.get().getHiveHistory().endQuery(queryId);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/f7c51b6c/ql/src/java/org/apache/hadoop/hive/ql/hooks/QueryLifeTimeHook.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/QueryLifeTimeHook.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/QueryLifeTimeHook.java
new file mode 100644
index 0000000..be57452
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/QueryLifeTimeHook.java
@@ -0,0 +1,57 @@
+/**
+ * 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.hadoop.hive.ql.hooks;
+
+/**
+ * A type of hook which triggers before query compilation and after query execution.
+ */
+public interface QueryLifeTimeHook extends Hook {
+
+  /**
+   * Invoked before a query enters the compilation phase.
+   *
+   * @param ctx the context for the hook
+   */
+  void beforeCompile(QueryLifeTimeHookContext ctx);
+
+  /**
+   * Invoked after a query compilation. Note: if 'hasError' is true,
+   * the query won't enter the following execution phase.
+   *
+   * @param ctx the context for the hook
+   * @param hasError whether any error occurred during compilation.
+   */
+  void afterCompile(QueryLifeTimeHookContext ctx, boolean hasError);
+
+  /**
+   * Invoked before a query enters the execution phase.
+   *
+   * @param ctx the context for the hook
+   */
+  void beforeExecution(QueryLifeTimeHookContext ctx);
+
+  /**
+   * Invoked after a query finishes its execution.
+   *
+   * @param ctx the context for the hook
+   * @param hasError whether any error occurred during query execution.
+   */
+  void afterExecution(QueryLifeTimeHookContext ctx, boolean hasError);
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f7c51b6c/ql/src/java/org/apache/hadoop/hive/ql/hooks/QueryLifeTimeHookContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/QueryLifeTimeHookContext.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/QueryLifeTimeHookContext.java
new file mode 100644
index 0000000..c599173
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/QueryLifeTimeHookContext.java
@@ -0,0 +1,69 @@
+/**
+ * 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.hadoop.hive.ql.hooks;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+
+/**
+ * Hook context for {@link QueryLifeTimeHook}.
+ */
+public interface QueryLifeTimeHookContext {
+  /**
+   * Get the current Hive configuration
+   *
+   * @return the Hive configuration being used
+   */
+  HiveConf getHiveConf();
+
+  /**
+   * Set Hive configuration
+   */
+  void setHiveConf(HiveConf conf);
+
+  /**
+   * Get the current command.
+   *
+   * @return the current query command
+   */
+  String getCommand();
+
+  /**
+   * Set the current command
+   *
+   * @param command the query command to set
+   */
+  void setCommand(String command);
+
+
+  /**
+   * Get the hook context for query execution.
+   * Note: this result value is null during query compilation phase.
+   *
+   * @return a {@link HookContext} instance containing information such as query
+   * plan, list of tasks, etc.
+   */
+  HookContext getHookContext();
+
+  /**
+   * Set the hook context
+   *
+   * @param hc a {@link HookContext} containing information for the current query.
+   */
+  void setHookContext(HookContext hc);
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f7c51b6c/ql/src/java/org/apache/hadoop/hive/ql/hooks/QueryLifeTimeHookContextImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/QueryLifeTimeHookContextImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/QueryLifeTimeHookContextImpl.java
new file mode 100644
index 0000000..5340848
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/QueryLifeTimeHookContextImpl.java
@@ -0,0 +1,57 @@
+/**
+ * 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.hadoop.hive.ql.hooks;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+
+public class QueryLifeTimeHookContextImpl implements QueryLifeTimeHookContext {
+  private HiveConf conf;
+  private String command;
+  private HookContext hc = null;
+
+  @Override
+  public HiveConf getHiveConf() {
+    return conf;
+  }
+
+  @Override
+  public void setHiveConf(HiveConf conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public String getCommand() {
+    return command;
+  }
+
+  @Override
+  public void setCommand(String command) {
+    this.command = command;
+  }
+
+  @Override
+  public HookContext getHookContext() {
+    return hc;
+  }
+
+  @Override
+  public void setHookContext(HookContext hc) {
+    this.hc = hc;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f7c51b6c/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestQueryHooks.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestQueryHooks.java b/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestQueryHooks.java
new file mode 100644
index 0000000..aa5d429
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestQueryHooks.java
@@ -0,0 +1,127 @@
+/**
+ * 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.hooks;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+public class TestQueryHooks {
+  private static HiveConf conf;
+  private static QueryLifeTimeHookContext[] ctxs;
+
+  @BeforeClass
+  public static void setUpBeforeClass() {
+    conf = new HiveConf(TestQueryHooks.class);
+    conf.setVar(HiveConf.ConfVars.HIVE_QUERY_LIFETIME_HOOKS, TestLifeTimeHook.class.getName());
+    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");
+  }
+
+  @Before
+  public void setUpBefore() {
+    ctxs = new QueryLifeTimeHookContext[4];
+  }
+
+  @Test
+  public void testCompile() throws Exception {
+    Driver driver = createDriver();
+    int ret = driver.compile("SELECT 1");
+    assertEquals("Expected compilation to succeed", 0, ret);
+    assertNotNull(ctxs[0]);
+    assertNotNull(ctxs[1]);
+    assertNull(ctxs[2]);
+    assertNull(ctxs[3]);
+    assertEquals("SELECT 1", ctxs[0].getCommand());
+    assertEquals("SELECT 1", ctxs[1].getCommand());
+  }
+
+  @Test
+  public void testCompileFailure() {
+    Driver driver = createDriver();
+    int ret = driver.compile("SELECT * FROM foo");
+    assertNotEquals("Expected compilation to fail", 0, ret);
+    assertNotNull(ctxs[0]);
+    assertNotNull(ctxs[1]);
+    assertNull(ctxs[2]);
+    assertNull(ctxs[3]);
+    assertEquals("SELECT * FROM foo", ctxs[0].getCommand());
+    assertEquals("SELECT * FROM foo", ctxs[1].getCommand());
+  }
+
+  @Test
+  public void testAll() throws Exception {
+    Driver driver = createDriver();
+    int ret = driver.run("SELECT 1").getResponseCode();
+    assertEquals("Expected query to run", 0, ret);
+    assertNotNull(ctxs[0]);
+    assertNotNull(ctxs[1]);
+    assertNotNull(ctxs[2]);
+    assertNotNull(ctxs[3]);
+    for (int i = 0; i < ctxs.length; i++) {
+      assertEquals("SELECT 1", ctxs[i].getCommand());
+    }
+    assertNotNull(ctxs[2].getHookContext());
+    assertNotNull(ctxs[3].getHookContext());
+  }
+
+  private static Driver createDriver() {
+    SessionState.start(conf);
+    Driver driver = new Driver(conf);
+    driver.init();
+    return driver;
+  }
+
+  /**
+   * Testing hook which just saves the context
+   */
+  private static class TestLifeTimeHook implements QueryLifeTimeHook {
+    public TestLifeTimeHook() {
+    }
+
+    @Override
+    public void beforeCompile(QueryLifeTimeHookContext ctx) {
+      ctxs[0] = ctx;
+    }
+
+    @Override
+    public void afterCompile(QueryLifeTimeHookContext ctx, boolean hasError) {
+      ctxs[1] = ctx;
+    }
+
+    @Override
+    public void beforeExecution(QueryLifeTimeHookContext ctx) {
+      ctxs[2] = ctx;
+    }
+
+    @Override
+    public void afterExecution(QueryLifeTimeHookContext ctx, boolean hasError) {
+      ctxs[3] = ctx;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f7c51b6c/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestQueryLifeTimeHook.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestQueryLifeTimeHook.java b/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestQueryLifeTimeHook.java
new file mode 100644
index 0000000..1f6239c
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestQueryLifeTimeHook.java
@@ -0,0 +1,43 @@
+/**
+ * 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.hadoop.hive.ql.hooks;
+
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+public class TestQueryLifeTimeHook implements QueryLifeTimeHook {
+  @Override
+  public void beforeCompile(QueryLifeTimeHookContext ctx) {
+    SessionState.getConsole().getOutStream().println("beforeCompile: " + ctx.getCommand());
+  }
+
+  @Override
+  public void afterCompile(QueryLifeTimeHookContext ctx, boolean hasError) {
+    SessionState.getConsole().getOutStream().println("afterCompile: " + ctx.getCommand());
+  }
+
+  @Override
+  public void beforeExecution(QueryLifeTimeHookContext ctx) {
+    SessionState.getConsole().getOutStream().println("beforeExecution: " + ctx.getCommand());
+  }
+
+  @Override
+  public void afterExecution(QueryLifeTimeHookContext ctx, boolean hasError) {
+    SessionState.getConsole().getOutStream().println("afterExecution: " + ctx.getCommand());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f7c51b6c/ql/src/test/queries/clientpositive/query_lifetime_hook.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/query_lifetime_hook.q b/ql/src/test/queries/clientpositive/query_lifetime_hook.q
new file mode 100644
index 0000000..8fda055
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/query_lifetime_hook.q
@@ -0,0 +1,3 @@
+SET hive.query.lifetime.hooks=org.apache.hadoop.hive.ql.hooks.TestQueryLifeTimeHook;
+
+SELECT * FROM src LIMIT 1;

http://git-wip-us.apache.org/repos/asf/hive/blob/f7c51b6c/ql/src/test/results/clientpositive/query_lifetime_hook.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/query_lifetime_hook.q.out b/ql/src/test/results/clientpositive/query_lifetime_hook.q.out
new file mode 100644
index 0000000..4a8e7ee
--- /dev/null
+++ b/ql/src/test/results/clientpositive/query_lifetime_hook.q.out
@@ -0,0 +1,21 @@
+beforeCompile: 
+
+SELECT * FROM src LIMIT 1
+afterCompile: 
+
+SELECT * FROM src LIMIT 1
+PREHOOK: query: SELECT * FROM src LIMIT 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+beforeExecution: 
+
+SELECT * FROM src LIMIT 1
+POSTHOOK: query: SELECT * FROM src LIMIT 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+afterExecution: 
+
+SELECT * FROM src LIMIT 1
+238	val_238