Repository: hive
Updated Branches:
  refs/heads/master 169e65592 -> 656a99e7e


HIVE-16363: QueryLifeTimeHooks should catch parse exceptions (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/656a99e7
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/656a99e7
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/656a99e7

Branch: refs/heads/master
Commit: 656a99e7e265ea2aa81816f3e4a4cc064c4c7788
Parents: 169e655
Author: Sahil Takiar <takiar.sa...@gmail.com>
Authored: Thu Apr 20 14:21:53 2017 -0500
Committer: Sergio Pena <sergio.p...@cloudera.com>
Committed: Thu Apr 20 14:21:53 2017 -0500

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   5 +-
 .../java/org/apache/hadoop/hive/ql/Driver.java  | 141 +++++---------
 .../hadoop/hive/ql/QueryLifeTimeHookRunner.java | 186 +++++++++++++++++++
 .../apache/hadoop/hive/ql/hooks/HookUtils.java  |  52 +-----
 .../hadoop/hive/ql/hooks/HooksLoader.java       | 107 +++++++++++
 .../ql/hooks/QueryLifeTimeHookContextImpl.java  |  34 +++-
 .../hooks/QueryLifeTimeHookWithParseHooks.java  |  24 +++
 .../hadoop/hive/ql/hooks/TestQueryHooks.java    | 162 +++++++++-------
 .../results/clientnegative/bad_exec_hooks.q.out |   2 +-
 .../service/cli/session/SessionManager.java     |   4 +-
 10 files changed, 503 insertions(+), 214 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/656a99e7/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 b10b08e..457f7af 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2275,7 +2275,10 @@ public class HiveConf extends Configuration {
          "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"),
+            " before/after query compilation and before/after query execution, 
in the order specified." +
+        "Implementations of QueryLifeTimeHookWithParseHooks can also be 
specified in this list. If they are" +
+        "specified then they will be invoked in the same places as 
QueryLifeTimeHooks and will be invoked during pre " +
+         "and post query parsing"),
     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/656a99e7/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 31cb4e1..6a8cc60 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -37,8 +37,8 @@ import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.ReentrantLock;
 
-import com.google.common.collect.Iterables;
 import org.apache.commons.lang.StringUtils;
+
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.common.metrics.common.Metrics;
@@ -66,12 +66,9 @@ import 
org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext;
 import org.apache.hadoop.hive.ql.hooks.Hook;
 import org.apache.hadoop.hive.ql.hooks.HookContext;
 import org.apache.hadoop.hive.ql.hooks.HookUtils;
-import org.apache.hadoop.hive.ql.hooks.MetricsQueryLifeTimeHook;
+import org.apache.hadoop.hive.ql.hooks.HooksLoader;
 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;
@@ -95,7 +92,7 @@ import 
org.apache.hadoop.hive.ql.parse.HiveSemanticAnalyzerHookContext;
 import org.apache.hadoop.hive.ql.parse.HiveSemanticAnalyzerHookContextImpl;
 import org.apache.hadoop.hive.ql.parse.ImportSemanticAnalyzer;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
-import org.apache.hadoop.hive.ql.parse.ParseDriver;
+import org.apache.hadoop.hive.ql.parse.ParseException;
 import org.apache.hadoop.hive.ql.parse.ParseUtils;
 import org.apache.hadoop.hive.ql.parse.PrunedPartitionList;
 import org.apache.hadoop.hive.ql.parse.SemanticAnalyzer;
@@ -113,8 +110,6 @@ import 
org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObje
 import 
org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivObjectActionType;
 import 
org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType;
 import 
org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzContext;
-import org.apache.hadoop.hive.ql.session.OperationLog;
-import org.apache.hadoop.hive.ql.session.OperationLog.LoggingLevel;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.session.SessionState.LogHelper;
 import org.apache.hadoop.hive.serde2.ByteStream;
@@ -123,7 +118,9 @@ import org.apache.hadoop.mapred.ClusterStatus;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.MRJobConfig;
+
 import org.apache.hive.common.util.ShutdownHookManager;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -131,6 +128,7 @@ import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Sets;
 
+
 public class Driver implements CommandProcessor {
 
   static final private String CLASS_NAME = Driver.class.getName();
@@ -178,7 +176,8 @@ public class Driver implements CommandProcessor {
   private QueryState queryState;
 
   // Query hooks that execute before compilation and after execution
-  private List<QueryLifeTimeHook> queryHooks;
+  private QueryLifeTimeHookRunner queryLifeTimeHookRunner;
+  private final HooksLoader hooksLoader;
 
   public enum DriverState {
     INITIALIZED,
@@ -348,11 +347,21 @@ public class Driver implements CommandProcessor {
   }
 
   public Driver(QueryState queryState, String userName) {
+    this(queryState, userName, new HooksLoader(queryState.getConf()));
+  }
+
+  public Driver(HiveConf conf, HooksLoader hooksLoader) {
+    this(new QueryState(conf), null, hooksLoader);
+  }
+
+  private Driver(QueryState queryState, String userName, HooksLoader 
hooksLoader) {
     this.queryState = queryState;
     this.conf = queryState.getConf();
     isParallelEnabled = (conf != null)
         && HiveConf.getBoolVar(conf, 
ConfVars.HIVE_SERVER2_PARALLEL_COMPILATION);
     this.userName = userName;
+    this.hooksLoader = hooksLoader;
+    this.queryLifeTimeHookRunner = new QueryLifeTimeHookRunner(conf, 
hooksLoader, console);
   }
 
   /**
@@ -432,6 +441,8 @@ public class Driver implements CommandProcessor {
 
     // Whether any error occurred during query compilation. Used for query 
lifetime hook.
     boolean compileError = false;
+    boolean parseError = false;
+
     try {
 
       // Initialize the transaction manager.  This must be done before analyze 
is called.
@@ -465,26 +476,27 @@ public class Driver implements CommandProcessor {
       ctx.setHDFSCleanup(true);
 
       perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.PARSE);
-      ASTNode tree = ParseUtils.parse(command, ctx);
-      perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.PARSE);
 
       // Trigger query hook before compilation
-      queryHooks = loadQueryHooks();
-      if (queryHooks != null && !queryHooks.isEmpty()) {
-        QueryLifeTimeHookContext qhc = new QueryLifeTimeHookContextImpl();
-        qhc.setHiveConf(conf);
-        qhc.setCommand(command);
+      queryLifeTimeHookRunner.runBeforeParseHook(command);
 
-        for (QueryLifeTimeHook hook : queryHooks) {
-          hook.beforeCompile(qhc);
-        }
+      ASTNode tree;
+      try {
+        tree = ParseUtils.parse(command, ctx);
+      } catch (ParseException e) {
+        parseError = true;
+        throw e;
+      } finally {
+        queryLifeTimeHookRunner.runAfterParseHook(command, parseError);
       }
+      perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.PARSE);
+
+      queryLifeTimeHookRunner.runBeforeCompileHook(command);
 
       perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.ANALYZE);
       BaseSemanticAnalyzer sem = SemanticAnalyzerFactory.get(queryState, tree);
       List<HiveSemanticAnalyzerHook> saHooks =
-          getHooks(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK,
-              HiveSemanticAnalyzerHook.class);
+          hooksLoader.getHooks(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK, 
console);
 
       // Flush the metastore cache.  This assures that we don't pick up 
objects from a previous
       // query running in this same thread.  This has to be done after we get 
our semantic
@@ -601,17 +613,12 @@ public class Driver implements CommandProcessor {
     } finally {
       // Trigger post compilation hook. Note that if the compilation fails 
here then
       // before/after execution hook will never be executed.
-      try {
-        if (queryHooks != null && !queryHooks.isEmpty()) {
-          QueryLifeTimeHookContext qhc = new QueryLifeTimeHookContextImpl();
-          qhc.setHiveConf(conf);
-          qhc.setCommand(command);
-          for (QueryLifeTimeHook hook : queryHooks) {
-            hook.afterCompile(qhc, compileError);
-          }
+      if (!parseError) {
+        try {
+          queryLifeTimeHookRunner.runAfterCompilationHook(command, 
compileError);
+        } catch (Exception e) {
+          LOG.warn("Failed when invoking query after-compilation hook.", e);
         }
-      } catch (Exception e) {
-        LOG.warn("Failed when invoking query after-compilation hook.", e);
       }
 
       double duration = perfLogger.PerfLogEnd(CLASS_NAME, 
PerfLogger.COMPILE)/1000.00;
@@ -674,19 +681,6 @@ public class Driver implements CommandProcessor {
     }
   }
 
-  private List<QueryLifeTimeHook> loadQueryHooks() throws Exception {
-    List<QueryLifeTimeHook> hooks = new ArrayList<>();
-
-    if (conf.getBoolVar(ConfVars.HIVE_SERVER2_METRICS_ENABLED)) {
-      hooks.add(new MetricsQueryLifeTimeHook());
-    }
-    List<QueryLifeTimeHook> propertyDefinedHoooks = 
getHooks(ConfVars.HIVE_QUERY_LIFETIME_HOOKS, QueryLifeTimeHook.class);
-    if (propertyDefinedHoooks != null) {
-      Iterables.addAll(hooks, propertyDefinedHoooks);
-    }
-    return hooks;
-  }
-
   private ImmutableMap<String, Long> dumpMetaCallTimingWithoutEx(String phase) 
{
     try {
       return Hive.get().dumpAndClearMetaCallTiming(phase);
@@ -1439,8 +1433,7 @@ public class Driver implements CommandProcessor {
       // Get all the driver run hooks and pre-execute them.
       List<HiveDriverRunHook> driverRunHooks;
       try {
-        driverRunHooks = getHooks(HiveConf.ConfVars.HIVE_DRIVER_RUN_HOOKS,
-            HiveDriverRunHook.class);
+        driverRunHooks = 
hooksLoader.getHooks(HiveConf.ConfVars.HIVE_DRIVER_RUN_HOOKS, console);
         for (HiveDriverRunHook driverRunHook : driverRunHooks) {
             driverRunHook.preDriverRun(hookContext);
         }
@@ -1665,34 +1658,6 @@ public class Driver implements CommandProcessor {
     return new CommandProcessorResponse(ret, errorMessage, SQLState, 
downstreamError);
   }
 
-  /**
-   * Returns a set of hooks specified in a configuration variable.
-   * See getHooks(HiveConf.ConfVars hookConfVar, Class<T> clazz)
-   */
-  private List<Hook> getHooks(HiveConf.ConfVars hookConfVar) throws Exception {
-    return getHooks(hookConfVar, Hook.class);
-  }
-
-  /**
-   * Returns the hooks specified in a configuration variable.
-   *
-   * @param hookConfVar The configuration variable specifying a comma 
separated list of the hook
-   *                    class names.
-   * @param clazz       The super type of the hooks.
-   * @return            A list of the hooks cast as the type specified in 
clazz, in the order
-   *                    they are listed in the value of hookConfVar
-   * @throws Exception
-   */
-  private <T extends Hook> List<T> getHooks(ConfVars hookConfVar,
-      Class<T> clazz) throws Exception {
-    try {
-      return HookUtils.getHooks(conf, hookConfVar, clazz);
-    } catch (ClassNotFoundException e) {
-      console.printError(hookConfVar.varname + " Class not found:" + 
e.getMessage());
-      throw e;
-    }
-  }
-
   public int execute() throws CommandNeedRetryException {
     return execute(false);
   }
@@ -1757,7 +1722,7 @@ public class Driver implements CommandProcessor {
           ss.getSessionId(), Thread.currentThread().getName(), 
ss.isHiveServerQuery(), perfLogger);
       hookContext.setHookType(HookContext.HookType.PRE_EXEC_HOOK);
 
-      for (Hook peh : getHooks(HiveConf.ConfVars.PREEXECHOOKS)) {
+      for (Hook peh : hooksLoader.getHooks(HiveConf.ConfVars.PREEXECHOOKS, 
console)) {
         if (peh instanceof ExecuteWithHookContext) {
           perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.PRE_HOOK + 
peh.getClass().getName());
 
@@ -1775,16 +1740,7 @@ public class Driver implements CommandProcessor {
       }
 
       // Trigger query hooks before query execution.
-      if (queryHooks != null && !queryHooks.isEmpty()) {
-        QueryLifeTimeHookContext qhc = new QueryLifeTimeHookContextImpl();
-        qhc.setHiveConf(conf);
-        qhc.setCommand(queryStr);
-        qhc.setHookContext(hookContext);
-
-        for (QueryLifeTimeHook hook : queryHooks) {
-          hook.beforeExecution(qhc);
-        }
-      }
+      queryLifeTimeHookRunner.runBeforeExecutionHook(queryStr, hookContext);
 
       setQueryDisplays(plan.getRootTasks());
       int mrJobs = Utilities.getMRTasks(plan.getRootTasks()).size();
@@ -1945,7 +1901,7 @@ public class Driver implements CommandProcessor {
 
       hookContext.setHookType(HookContext.HookType.POST_EXEC_HOOK);
       // Get all the post execution hooks and execute them.
-      for (Hook peh : getHooks(HiveConf.ConfVars.POSTEXECHOOKS)) {
+      for (Hook peh : hooksLoader.getHooks(HiveConf.ConfVars.POSTEXECHOOKS, 
console)) {
         if (peh instanceof ExecuteWithHookContext) {
           perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.POST_HOOK + 
peh.getClass().getName());
 
@@ -2000,16 +1956,7 @@ public class Driver implements CommandProcessor {
     } finally {
       // Trigger query hooks after query completes its execution.
       try {
-        if (queryHooks != null && !queryHooks.isEmpty()) {
-          QueryLifeTimeHookContext qhc = new QueryLifeTimeHookContextImpl();
-          qhc.setHiveConf(conf);
-          qhc.setCommand(queryStr);
-          qhc.setHookContext(hookContext);
-
-          for (QueryLifeTimeHook hook : queryHooks) {
-            hook.afterExecution(qhc, executionError);
-          }
-        }
+        queryLifeTimeHookRunner.runAfterExecutionHook(queryStr, hookContext, 
executionError);
       } catch (Exception e) {
         LOG.warn("Failed when invoking query after execution hook", e);
       }
@@ -2124,7 +2071,7 @@ public class Driver implements CommandProcessor {
     hookContext.setErrorMessage(errorMessage);
     hookContext.setException(exception);
     // Get all the failure execution hooks and execute them.
-    for (Hook ofh : getHooks(HiveConf.ConfVars.ONFAILUREHOOKS)) {
+    for (Hook ofh : hooksLoader.getHooks(HiveConf.ConfVars.ONFAILUREHOOKS, 
console)) {
       perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.FAILURE_HOOK + 
ofh.getClass().getName());
 
       ((ExecuteWithHookContext) ofh).run(hookContext);

http://git-wip-us.apache.org/repos/asf/hive/blob/656a99e7/ql/src/java/org/apache/hadoop/hive/ql/QueryLifeTimeHookRunner.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/QueryLifeTimeHookRunner.java 
b/ql/src/java/org/apache/hadoop/hive/ql/QueryLifeTimeHookRunner.java
new file mode 100644
index 0000000..85e038c
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/QueryLifeTimeHookRunner.java
@@ -0,0 +1,186 @@
+/**
+ * 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;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.google.common.collect.Iterables;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.hooks.HookContext;
+import org.apache.hadoop.hive.ql.hooks.HooksLoader;
+import org.apache.hadoop.hive.ql.hooks.MetricsQueryLifeTimeHook;
+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.QueryLifeTimeHookWithParseHooks;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+
+/**
+ * A runner class for {@link QueryLifeTimeHook}s and {@link 
QueryLifeTimeHookWithParseHooks}. The class has run methods
+ * for each phase of a {@link QueryLifeTimeHook} and {@link 
QueryLifeTimeHookWithParseHooks}. Each run method checks if
+ * a list of hooks has be specified, and if so invokes the appropriate 
callback method of each hook. Each method
+ * constructs a {@link QueryLifeTimeHookContext} object and pass it to the 
callback functions.
+ */
+class QueryLifeTimeHookRunner {
+
+  private final HiveConf conf;
+  private final List<QueryLifeTimeHook> queryHooks;
+
+  /**
+   * Constructs a {@link QueryLifeTimeHookRunner} that loads all hooks to be 
run via a {@link HooksLoader}.
+   *
+   * @param conf the {@link HiveConf} to use when creating {@link 
QueryLifeTimeHookContext} objects
+   * @param hooksLoader the {@link HooksLoader} to use when loading all hooks 
to be run
+   * @param console the {@link SessionState.LogHelper} to use when running 
{@link HooksLoader#getHooks(HiveConf.ConfVars)}
+   */
+  QueryLifeTimeHookRunner(HiveConf conf, HooksLoader hooksLoader, 
SessionState.LogHelper console) {
+    this.conf = conf;
+    this.queryHooks = new ArrayList<>();
+
+    if (conf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_METRICS_ENABLED)) {
+      queryHooks.add(new MetricsQueryLifeTimeHook());
+    }
+    List<QueryLifeTimeHook> propertyDefinedHoooks;
+    try {
+      propertyDefinedHoooks = hooksLoader.getHooks(
+              HiveConf.ConfVars.HIVE_QUERY_LIFETIME_HOOKS, console);
+    } catch (IllegalAccessException | InstantiationException | 
ClassNotFoundException e) {
+      throw new IllegalArgumentException(e);
+    }
+    if (propertyDefinedHoooks != null) {
+      Iterables.addAll(queryHooks, propertyDefinedHoooks);
+    }
+  }
+
+  /**
+   * If {@link QueryLifeTimeHookWithParseHooks} have been loaded via the 
{@link HooksLoader} then invoke the
+   * {@link 
QueryLifeTimeHookWithParseHooks#beforeParse(QueryLifeTimeHookContext)} method 
for each
+   * {@link QueryLifeTimeHookWithParseHooks}.
+   *
+   * @param command the Hive command that is being run
+   */
+  void runBeforeParseHook(String command) {
+    if (containsHooks()) {
+      QueryLifeTimeHookContext qhc = new 
QueryLifeTimeHookContextImpl.Builder().withHiveConf(conf).withCommand(
+              command).build();
+
+      for (QueryLifeTimeHook hook : queryHooks) {
+        if (hook instanceof QueryLifeTimeHookWithParseHooks) {
+          ((QueryLifeTimeHookWithParseHooks) hook).beforeParse(qhc);
+        }
+      }
+    }
+  }
+
+  /**
+   * If {@link QueryLifeTimeHookWithParseHooks} have been loaded via the 
{@link HooksLoader} then invoke the
+   * {@link 
QueryLifeTimeHookWithParseHooks#afterParse(QueryLifeTimeHookContext, boolean)} 
method for each
+   * {@link QueryLifeTimeHookWithParseHooks}.
+   *
+   * @param command the Hive command that is being run
+   * @param parseError true if there was an error while parsing the command, 
false otherwise
+   */
+  void runAfterParseHook(String command, boolean parseError) {
+    if (containsHooks()) {
+      QueryLifeTimeHookContext qhc = new 
QueryLifeTimeHookContextImpl.Builder().withHiveConf(conf).withCommand(
+              command).build();
+
+      for (QueryLifeTimeHook hook : queryHooks) {
+        if (hook instanceof QueryLifeTimeHookWithParseHooks) {
+          ((QueryLifeTimeHookWithParseHooks) hook).afterParse(qhc, parseError);
+        }
+      }
+    }
+  }
+
+  /**
+   * Invoke the {@link 
QueryLifeTimeHook#beforeCompile(QueryLifeTimeHookContext)} method for each 
{@link QueryLifeTimeHook}
+   *
+   * @param command the Hive command that is being run
+   */
+  void runBeforeCompileHook(String command) {
+    if (containsHooks()) {
+      QueryLifeTimeHookContext qhc = new 
QueryLifeTimeHookContextImpl.Builder().withHiveConf(conf).withCommand(
+              command).build();
+
+      for (QueryLifeTimeHook hook : queryHooks) {
+        hook.beforeCompile(qhc);
+      }
+    }
+  }
+
+   /**
+   * Invoke the {@link 
QueryLifeTimeHook#afterCompile(QueryLifeTimeHookContext, boolean)} method for 
each {@link QueryLifeTimeHook}
+   *
+   * @param command the Hive command that is being run
+   * @param compileError true if there was an error while compiling the 
command, false otherwise
+   */
+  void runAfterCompilationHook(String command, boolean compileError) {
+    if (containsHooks()) {
+      QueryLifeTimeHookContext qhc = new 
QueryLifeTimeHookContextImpl.Builder().withHiveConf(conf).withCommand(
+              command).build();
+
+      for (QueryLifeTimeHook hook : queryHooks) {
+        hook.afterCompile(qhc, compileError);
+      }
+    }
+  }
+
+  /**
+   * Invoke the {@link 
QueryLifeTimeHook#beforeExecution(QueryLifeTimeHookContext)} method for each 
{@link QueryLifeTimeHook}
+   *
+   * @param command the Hive command that is being run
+   * @param hookContext the {@link HookContext} of the command being run
+   */
+  void runBeforeExecutionHook(String command, HookContext hookContext) {
+    if (containsHooks()) {
+      QueryLifeTimeHookContext qhc = new 
QueryLifeTimeHookContextImpl.Builder().withHiveConf(conf).withCommand(
+              command).withHookContext(hookContext).build();
+
+      for (QueryLifeTimeHook hook : queryHooks) {
+        hook.beforeExecution(qhc);
+      }
+    }
+  }
+
+  /**
+   * Invoke the {@link 
QueryLifeTimeHook#afterExecution(QueryLifeTimeHookContext, boolean)} method for 
each {@link QueryLifeTimeHook}
+   *
+   * @param command the Hive command that is being run
+   * @param hookContext the {@link HookContext} of the command being run
+   * @param executionError true if there was an error while executing the 
command, false otherwise
+   */
+  void runAfterExecutionHook(String command, HookContext hookContext, boolean 
executionError) {
+    if (containsHooks()) {
+      QueryLifeTimeHookContext qhc = new 
QueryLifeTimeHookContextImpl.Builder().withHiveConf(conf).withCommand(
+              command).withHookContext(hookContext).build();
+
+      for (QueryLifeTimeHook hook : queryHooks) {
+        hook.afterExecution(qhc, executionError);
+      }
+    }
+  }
+
+  private boolean containsHooks() {
+    return queryHooks != null && !queryHooks.isEmpty();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/656a99e7/ql/src/java/org/apache/hadoop/hive/ql/hooks/HookUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/HookUtils.java 
b/ql/src/java/org/apache/hadoop/hive/ql/hooks/HookUtils.java
index 2f0bd88..4380fe3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/HookUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/HookUtils.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <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.
@@ -18,66 +18,26 @@
 
 package org.apache.hadoop.hive.ql.hooks;
 
-import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.ql.exec.Utilities;
 
-public class HookUtils {
-  /**
-   * Returns the hooks specified in a configuration variable.  The hooks are 
returned
-   * in a list in the order they were specified in the configuration variable.
-   *
-   * @param conf        Configuration object
-   * @param hookConfVar The configuration variable specifying a comma 
separated list
-   *                    of the hook class names.
-   * @param clazz       The super type of the hooks.
-   * @return            A list of the hooks cast as the type specified in 
clazz,
-   *                    in the order they are listed in the value of 
hookConfVar
-   * @throws ClassNotFoundException
-   * @throws IllegalAccessException
-   * @throws InstantiationException
-   */
-  public static <T extends Hook> List<T> getHooks(HiveConf conf,
-      ConfVars hookConfVar, Class<T> clazz)
-      throws InstantiationException, IllegalAccessException, 
ClassNotFoundException  {
-    String csHooks = conf.getVar(hookConfVar);
-    List<T> hooks = new ArrayList<T>();
-    if (csHooks == null) {
-      return hooks;
-    }
 
-    csHooks = csHooks.trim();
-    if (csHooks.equals("")) {
-      return hooks;
-    }
-
-    String[] hookClasses = csHooks.split(",");
-    for (String hookClass : hookClasses) {
-        T hook = (T) Class.forName(hookClass.trim(), true,
-                Utilities.getSessionSpecifiedClassLoader()).newInstance();
-        hooks.add(hook);
-    }
-
-    return hooks;
-  }
+public class HookUtils {
 
   public static String redactLogString(HiveConf conf, String logString)
-      throws InstantiationException, IllegalAccessException, 
ClassNotFoundException {
+          throws InstantiationException, IllegalAccessException, 
ClassNotFoundException {
 
     String redactedString = logString;
 
     if (conf != null && logString != null) {
-      List<Redactor> queryRedactors = getHooks(conf, 
ConfVars.QUERYREDACTORHOOKS, Redactor.class);
+      List<Redactor> queryRedactors = new 
HooksLoader(conf).getHooks(ConfVars.QUERYREDACTORHOOKS);
       for (Redactor redactor : queryRedactors) {
         redactor.setConf(conf);
         redactedString = redactor.redactQuery(redactedString);
       }
     }
-
     return redactedString;
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/656a99e7/ql/src/java/org/apache/hadoop/hive/ql/hooks/HooksLoader.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/HooksLoader.java 
b/ql/src/java/org/apache/hadoop/hive/ql/hooks/HooksLoader.java
new file mode 100644
index 0000000..0008726
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/HooksLoader.java
@@ -0,0 +1,107 @@
+/**
+ * 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 java.util.List;
+
+import com.google.common.collect.ImmutableList;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+
+/**
+ * A loader class for {@link Hook}s. The class provides a way to create and 
instantiate {@link Hook} objects. The
+ * methodology for how hooks are loaded is left up to the individual methods.
+ */
+public class HooksLoader {
+
+  private final HiveConf conf;
+
+  /**
+   * Creates a new {@link HooksLoader} that uses the specified {@link 
HiveConf} to load the {@link Hook}s.
+   *
+   * @param conf the {@link HiveConf} to use when loading the {@link Hook}s
+   */
+  public HooksLoader(HiveConf conf) {
+    this.conf = conf;
+  }
+
+  /**
+   * Delegates to {@link #getHooks(HiveConf.ConfVars)} and prints the to the 
specified {@link SessionState.LogHelper} if
+   * a {@link ClassNotFoundException} is thrown.
+   *
+   * @param hookConfVar the configuration variable specifying a comma 
separated list of the hook class names
+   * @param console the {@link SessionState.LogHelper} to print to if a {@link 
ClassNotFoundException} is thrown by the
+   *                {@link #getHooks(HiveConf.ConfVars)} method
+   *
+   * @return a list of the hooks objects, in the order they are listed in the 
value of hookConfVar
+   *
+   * @throws ClassNotFoundException if the specified class names could not be 
found
+   * @throws IllegalAccessException if the specified class names could not be 
accessed
+   * @throws InstantiationException if the specified class names could not be 
instantiated
+   */
+  public <T extends Hook> List<T> getHooks(HiveConf.ConfVars hookConfVar, 
SessionState.LogHelper console)
+          throws IllegalAccessException, InstantiationException, 
ClassNotFoundException {
+    try {
+      return getHooks(hookConfVar);
+    } catch (ClassNotFoundException e) {
+      console.printError(hookConfVar.varname + " Class not found: " + 
e.getMessage());
+      throw e;
+    }
+  }
+
+  /**
+   * Returns the hooks specified in a configuration variable. The hooks are 
returned in a list in the order they were
+   * specified in the configuration variable. The value of the specified conf 
variable should be a comma separated list
+   * of class names where each class implements the {@link Hook} interface. 
The method uses reflection to an instance
+   * of each class and then returns them in a {@link List}.
+   *
+   * @param hookConfVar The configuration variable specifying a comma 
separated list of the hook class names
+   *
+   * @return a list of the hooks objects, in the order they are listed in the 
value of hookConfVar
+   *
+   * @throws ClassNotFoundException if the specified class names could not be 
found
+   * @throws IllegalAccessException if the specified class names could not be 
accessed
+   * @throws InstantiationException if the specified class names could not be 
instantiated
+   */
+  public <T extends Hook> List<T> getHooks(HiveConf.ConfVars hookConfVar)
+          throws InstantiationException, IllegalAccessException, 
ClassNotFoundException {
+    String csHooks = conf.getVar(hookConfVar);
+    ImmutableList.Builder<T> hooks = ImmutableList.builder();
+    if (csHooks == null) {
+      return ImmutableList.of();
+    }
+
+    csHooks = csHooks.trim();
+    if (csHooks.isEmpty()) {
+      return ImmutableList.of();
+    }
+
+    String[] hookClasses = csHooks.split(",");
+    for (String hookClass : hookClasses) {
+      T hook = (T) Class.forName(hookClass.trim(), true,
+              Utilities.getSessionSpecifiedClassLoader()).newInstance();
+      hooks.add(hook);
+    }
+
+    return hooks.build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/656a99e7/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
index 5340848..1845121 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/hooks/QueryLifeTimeHookContextImpl.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/hooks/QueryLifeTimeHookContextImpl.java
@@ -20,10 +20,12 @@ 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;
+  private HookContext hc;
 
   @Override
   public HiveConf getHiveConf() {
@@ -54,4 +56,34 @@ public class QueryLifeTimeHookContextImpl implements 
QueryLifeTimeHookContext {
   public void setHookContext(HookContext hc) {
     this.hc = hc;
   }
+
+  public static class Builder {
+
+    private HiveConf conf;
+    private String command;
+    private HookContext hc;
+
+    public Builder withHiveConf(HiveConf conf) {
+      this.conf = conf;
+      return this;
+    }
+
+    public Builder withCommand(String command) {
+      this.command = command;
+      return this;
+    }
+
+    public Builder withHookContext(HookContext hc) {
+      this.hc = hc;
+      return this;
+    }
+
+    public QueryLifeTimeHookContextImpl build() {
+      QueryLifeTimeHookContextImpl queryLifeTimeHookContext = new 
QueryLifeTimeHookContextImpl();
+      queryLifeTimeHookContext.setHiveConf(this.conf);
+      queryLifeTimeHookContext.setCommand(this.command);
+      queryLifeTimeHookContext.setHookContext(this.hc);
+      return queryLifeTimeHookContext;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/656a99e7/ql/src/java/org/apache/hadoop/hive/ql/hooks/QueryLifeTimeHookWithParseHooks.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/hooks/QueryLifeTimeHookWithParseHooks.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/hooks/QueryLifeTimeHookWithParseHooks.java
new file mode 100644
index 0000000..1bde191
--- /dev/null
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/hooks/QueryLifeTimeHookWithParseHooks.java
@@ -0,0 +1,24 @@
+package org.apache.hadoop.hive.ql.hooks;
+
+
+/**
+ * Extension of {@link QueryLifeTimeHook} that has hooks for pre and post 
parsing of a query.
+ */
+public interface QueryLifeTimeHookWithParseHooks extends QueryLifeTimeHook {
+
+  /**
+   * Invoked before a query enters the parse phase.
+   *
+   * @param ctx the context for the hook
+   */
+  void beforeParse(QueryLifeTimeHookContext ctx);
+
+  /**
+   * Invoked after a query parsing. Note: if 'hasError' is true,
+   * the query won't enter the following compilation phase.
+   *
+   * @param ctx the context for the hook
+   * @param hasError whether any error occurred during compilation.
+   */
+  void afterParse(QueryLifeTimeHookContext ctx, boolean hasError);
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/656a99e7/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
index aa5d429..88a392a 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestQueryHooks.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestQueryHooks.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <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.
@@ -18,110 +18,140 @@
 
 package org.apache.hadoop.hive.ql.hooks;
 
+import com.google.common.collect.Lists;
+
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 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 org.mockito.ArgumentMatcher;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.Matchers.argThat;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
 
 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");
+            
"org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory");
   }
 
-  @Before
-  public void setUpBefore() {
-    ctxs = new QueryLifeTimeHookContext[4];
+  @Test
+  public void testAllQueryLifeTimeWithParseHooks() throws 
IllegalAccessException, ClassNotFoundException, InstantiationException, 
CommandNeedRetryException {
+    String query = "select 1";
+    ArgumentMatcher<QueryLifeTimeHookContext> argMatcher = new 
QueryLifeTimeHookContextMatcher(query);
+    QueryLifeTimeHookWithParseHooks mockHook = 
mock(QueryLifeTimeHookWithParseHooks.class);
+    int ret = createDriver(mockHook).run(query).getResponseCode();
+    assertEquals("Expected query to succeed", 0, ret);
+
+    verify(mockHook).beforeParse(argThat(argMatcher));
+    verify(mockHook).afterParse(argThat(argMatcher), eq(false));
+    verify(mockHook).beforeCompile(argThat(argMatcher));
+    verify(mockHook).afterCompile(argThat(argMatcher), eq(false));
+    verify(mockHook).beforeExecution(argThat(argMatcher));
+    verify(mockHook).afterExecution(argThat(argMatcher), eq(false));
   }
 
   @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());
+  public void testQueryLifeTimeWithParseHooksWithParseError() throws 
IllegalAccessException, ClassNotFoundException, InstantiationException, 
CommandNeedRetryException {
+    String query = "invalidquery";
+    ArgumentMatcher<QueryLifeTimeHookContext> argMatcher = new 
QueryLifeTimeHookContextMatcher(query);
+    QueryLifeTimeHookWithParseHooks mockHook = 
mock(QueryLifeTimeHookWithParseHooks.class);
+    int ret = createDriver(mockHook).run(query).getResponseCode();
+    assertNotEquals("Expected parsing to fail", 0, ret);
+
+    verify(mockHook).beforeParse(argThat(argMatcher));
+    verify(mockHook).afterParse(argThat(argMatcher), eq(true));
+    verify(mockHook, never()).beforeCompile(any());
+    verify(mockHook, never()).afterCompile(any(), anyBoolean());
+    verify(mockHook, never()).beforeExecution(any());
+    verify(mockHook, never()).afterExecution(any(), anyBoolean());
   }
 
   @Test
-  public void testCompileFailure() {
-    Driver driver = createDriver();
-    int ret = driver.compile("SELECT * FROM foo");
+  public void testQueryLifeTimeWithParseHooksWithCompileError() throws 
IllegalAccessException, ClassNotFoundException, InstantiationException, 
CommandNeedRetryException {
+    String query = "select * from foo";
+    ArgumentMatcher<QueryLifeTimeHookContext> argMatcher = new 
QueryLifeTimeHookContextMatcher(query);
+    QueryLifeTimeHookWithParseHooks mockHook = 
mock(QueryLifeTimeHookWithParseHooks.class);
+    int ret = createDriver(mockHook).run(query).getResponseCode();
     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());
+
+    verify(mockHook).beforeParse(argThat(argMatcher));
+    verify(mockHook).afterParse(argThat(argMatcher), eq(false));
+    verify(mockHook).beforeCompile(argThat(argMatcher));
+    verify(mockHook).afterCompile(argThat(argMatcher), eq(true));
+    verify(mockHook, never()).beforeExecution(any());
+    verify(mockHook, never()).afterExecution(any(), anyBoolean());
   }
 
   @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());
+  public void testAllQueryLifeTimeHooks() throws IllegalAccessException, 
ClassNotFoundException, InstantiationException, CommandNeedRetryException {
+    String query = "select 1";
+    ArgumentMatcher<QueryLifeTimeHookContext> argMatcher = new 
QueryLifeTimeHookContextMatcher(query);
+    QueryLifeTimeHook mockHook = mock(QueryLifeTimeHook.class);
+    int ret = createDriver(mockHook).run(query).getResponseCode();
+    assertEquals("Expected query to succeed", 0, ret);
+
+    verify(mockHook).beforeCompile(argThat(argMatcher));
+    verify(mockHook).afterCompile(argThat(argMatcher), eq(false));
+    verify(mockHook).beforeExecution(argThat(argMatcher));
+    verify(mockHook).afterExecution(argThat(argMatcher), eq(false));
+  }
+
+  @Test
+  public void testQueryLifeTimeWithCompileError() throws 
IllegalAccessException, ClassNotFoundException, InstantiationException, 
CommandNeedRetryException {
+    String query = "select * from foo";
+    ArgumentMatcher<QueryLifeTimeHookContext> argMatcher = new 
QueryLifeTimeHookContextMatcher(query);
+    QueryLifeTimeHook mockHook = mock(QueryLifeTimeHook.class);
+    int ret = createDriver(mockHook).run(query).getResponseCode();
+    assertNotEquals("Expected compilation to fail", 0, ret);
+
+    verify(mockHook).beforeCompile(argThat(argMatcher));
+    verify(mockHook).afterCompile(argThat(argMatcher), eq(true));
+    verify(mockHook, never()).beforeExecution(any());
+    verify(mockHook, never()).afterExecution(any(), anyBoolean());
   }
 
-  private static Driver createDriver() {
+  private Driver createDriver(QueryLifeTimeHook mockHook) throws 
IllegalAccessException, ClassNotFoundException, InstantiationException {
+    HooksLoader mockLoader = mock(HooksLoader.class);
+    when(mockLoader.getHooks(eq(HiveConf.ConfVars.HIVE_QUERY_LIFETIME_HOOKS), 
any())).thenReturn(
+            Lists.newArrayList(mockHook));
+
     SessionState.start(conf);
-    Driver driver = new Driver(conf);
+    Driver driver = new Driver(conf, mockLoader);
     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;
-    }
+  private static final class QueryLifeTimeHookContextMatcher extends 
ArgumentMatcher<QueryLifeTimeHookContext> {
 
-    @Override
-    public void afterCompile(QueryLifeTimeHookContext ctx, boolean hasError) {
-      ctxs[1] = ctx;
-    }
+    private final String command;
 
-    @Override
-    public void beforeExecution(QueryLifeTimeHookContext ctx) {
-      ctxs[2] = ctx;
+    private QueryLifeTimeHookContextMatcher(String command) {
+      this.command = command;
     }
 
     @Override
-    public void afterExecution(QueryLifeTimeHookContext ctx, boolean hasError) 
{
-      ctxs[3] = ctx;
+    public boolean matches(Object o) {
+      return ((QueryLifeTimeHookContext) o).getCommand().equals(this.command);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/656a99e7/ql/src/test/results/clientnegative/bad_exec_hooks.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/bad_exec_hooks.q.out 
b/ql/src/test/results/clientnegative/bad_exec_hooks.q.out
index 9d81fc5..6f2ee5c 100644
--- a/ql/src/test/results/clientnegative/bad_exec_hooks.q.out
+++ b/ql/src/test/results/clientnegative/bad_exec_hooks.q.out
@@ -1,4 +1,4 @@
-hive.exec.pre.hooks Class not found:"org.this.is.a.bad.class"
+hive.exec.pre.hooks Class not found: "org.this.is.a.bad.class"
 FAILED: Hive Internal Error: 
java.lang.ClassNotFoundException("org.this.is.a.bad.class")
 java.lang.ClassNotFoundException: "org.this.is.a.bad.class"
 #### A masked pattern was here ####

http://git-wip-us.apache.org/repos/asf/hive/blob/656a99e7/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
----------------------------------------------------------------------
diff --git 
a/service/src/java/org/apache/hive/service/cli/session/SessionManager.java 
b/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
index 26c8812..5082689 100644
--- a/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
+++ b/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
@@ -44,6 +44,7 @@ import 
org.apache.hadoop.hive.common.metrics.common.MetricsVariable;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.hooks.HookUtils;
+import org.apache.hadoop.hive.ql.hooks.HooksLoader;
 import org.apache.hive.service.CompositeService;
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.SessionHandle;
@@ -549,8 +550,7 @@ public class SessionManager extends CompositeService {
 
   // execute session hooks
   private void executeSessionHooks(HiveSession session) throws Exception {
-    List<HiveSessionHook> sessionHooks = HookUtils.getHooks(hiveConf,
-        HiveConf.ConfVars.HIVE_SERVER2_SESSION_HOOK, HiveSessionHook.class);
+    List<HiveSessionHook> sessionHooks = new 
HooksLoader(hiveConf).getHooks(HiveConf.ConfVars.HIVE_SERVER2_SESSION_HOOK);
     for (HiveSessionHook sessionHook : sessionHooks) {
       sessionHook.run(new HiveSessionHookContextImpl(session));
     }

Reply via email to