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

svn commit: r1241729 - in /hive/trunk/ql/src: java/org/apache/hadoop/hive/ql/ java/org/apache/hadoop/hive/ql/metadata/ java/org/apache/hadoop/hive/ql/parse/ test/org/apache/hadoop/hive/ql/hooks/ test/queries/clientpositive/ test/results/clientnegative/...

Author: namit
Date: Wed Feb  8 01:37:59 2012
New Revision: 1241729

URL: http://svn.apache.org/viewvc?rev=1241729&view=rev
Log:
HIVE-2779 Improve Hooks run in Driver
(Kevin Wilfong via namit)


Added:
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/hooks/VerifyHooksRunInOrder.java
    hive/trunk/ql/src/test/queries/clientpositive/hook_order.q
    hive/trunk/ql/src/test/results/clientpositive/hook_order.q.out
Modified:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveSemanticAnalyzerHook.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveSemanticAnalyzerHookContext.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveSemanticAnalyzerHookContextImpl.java
    hive/trunk/ql/src/test/results/clientnegative/bad_exec_hooks.q.out

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java?rev=1241729&r1=1241728&r2=1241729&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java Wed Feb  8 01:37:59 2012
@@ -76,7 +76,6 @@ import org.apache.hadoop.hive.ql.metadat
 import org.apache.hadoop.hive.ql.metadata.DummyPartition;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.metadata.HiveUtils;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.optimizer.ppr.PartitionPruner;
@@ -418,7 +417,9 @@ public class Driver implements CommandPr
       tree = ParseUtils.findRootNonNullToken(tree);
 
       BaseSemanticAnalyzer sem = SemanticAnalyzerFactory.get(conf, tree);
-      List<AbstractSemanticAnalyzerHook> saHooks = getSemanticAnalyzerHooks();
+      List<AbstractSemanticAnalyzerHook> saHooks =
+          getHooks(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK,
+                   AbstractSemanticAnalyzerHook.class);
 
       // Do semantic analysis and plan generation
       if (saHooks != null) {
@@ -428,6 +429,7 @@ public class Driver implements CommandPr
           tree = hook.preAnalyze(hookCtx, tree);
         }
         sem.analyze(tree, ctx);
+        hookCtx.update(sem);
         for (AbstractSemanticAnalyzerHook hook : saHooks) {
           hook.postAnalyze(hookCtx, sem.getRootTasks());
         }
@@ -941,100 +943,57 @@ public class Driver implements CommandPr
     return new CommandProcessorResponse(ret);
   }
 
-  private List<AbstractSemanticAnalyzerHook> getSemanticAnalyzerHooks() throws Exception {
-    ArrayList<AbstractSemanticAnalyzerHook> saHooks = new ArrayList<AbstractSemanticAnalyzerHook>();
-    String pestr = conf.getVar(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK);
-    if(pestr == null) {
-      return saHooks;
-    }
-    pestr = pestr.trim();
-    if (pestr.equals("")) {
-      return saHooks;
-    }
-
-    String[] peClasses = pestr.split(",");
-
-    for (String peClass : peClasses) {
-      try {
-        AbstractSemanticAnalyzerHook hook = HiveUtils.getSemanticAnalyzerHook(conf, peClass);
-        saHooks.add(hook);
-      } catch (HiveException e) {
-        console.printError("Pre Exec Hook Class not found:" + e.getMessage());
-        throw e;
-      }
-    }
-
-    return saHooks;
-  }
-
-
-  private List<Hook> getPreExecHooks() throws Exception {
-    List<Hook> pehooks = new ArrayList<Hook>();
-    String pestr = conf.getVar(HiveConf.ConfVars.PREEXECHOOKS);
-    pestr = pestr.trim();
-    if (pestr.equals("")) {
-      return pehooks;
-    }
-
-    String[] peClasses = pestr.split(",");
-
-    for (String peClass : peClasses) {
-      try {
-        pehooks.add((Hook) Class.forName(peClass.trim(), true, JavaUtils.getClassLoader())
-            .newInstance());
-      } catch (ClassNotFoundException e) {
-        console.printError("Pre Exec Hook Class not found:" + e.getMessage());
-        throw e;
-      }
-    }
-
-    return pehooks;
+  /**
+   * Returns a set of hooks specified in a configuration variable.
+   *
+   * See getHooks(HiveConf.ConfVars hookConfVar, Class<T> clazz)
+   * @param hookConfVar
+   * @return
+   * @throws Exception
+   */
+  private List<Hook> getHooks(HiveConf.ConfVars hookConfVar) throws Exception {
+    return getHooks(hookConfVar, Hook.class);
   }
 
-  private List<Hook> getPostExecHooks() throws Exception {
-    List<Hook> pehooks = new ArrayList<Hook>();
-    String pestr = conf.getVar(HiveConf.ConfVars.POSTEXECHOOKS);
-    pestr = pestr.trim();
-    if (pestr.equals("")) {
-      return pehooks;
-    }
-
-    String[] peClasses = pestr.split(",");
+  /**
+   * 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 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(HiveConf.ConfVars hookConfVar, Class<T> clazz)
+      throws Exception {
 
-    for (String peClass : peClasses) {
-      try {
-        pehooks.add((Hook) Class.forName(peClass.trim(), true, JavaUtils.getClassLoader())
-            .newInstance());
-      } catch (ClassNotFoundException e) {
-        console.printError("Post Exec Hook Class not found:" + e.getMessage());
-        throw e;
-      }
+    List<T> hooks = new ArrayList<T>();
+    String csHooks = conf.getVar(hookConfVar);
+    if (csHooks == null) {
+      return hooks;
     }
 
-    return pehooks;
-  }
-
-  private List<Hook> getOnFailureHooks() throws Exception {
-    List<Hook> ofhooks = new ArrayList<Hook>();
-    String ofstr = conf.getVar(HiveConf.ConfVars.ONFAILUREHOOKS);
-    ofstr = ofstr.trim();
-    if (ofstr.equals("")) {
-      return ofhooks;
+    csHooks = csHooks.trim();
+    if (csHooks.equals("")) {
+      return hooks;
     }
 
-    String[] ofClasses = ofstr.split(",");
+    String[] hookClasses = csHooks.split(",");
 
-    for (String ofClass : ofClasses) {
+    for (String hookClass : hookClasses) {
       try {
-        ofhooks.add((Hook) Class.forName(ofClass.trim(), true, JavaUtils.getClassLoader())
-            .newInstance());
+        T hook =
+            (T) Class.forName(hookClass.trim(), true, JavaUtils.getClassLoader()).newInstance();
+        hooks.add(hook);
       } catch (ClassNotFoundException e) {
-        console.printError("On Failure Hook Class not found:" + e.getMessage());
+        console.printError(hookConfVar.varname + " Class not found:" + e.getMessage());
         throw e;
       }
     }
 
-    return ofhooks;
+    return hooks;
   }
 
   public int execute() throws CommandNeedRetryException {
@@ -1066,7 +1025,7 @@ public class Driver implements CommandPr
       HookContext hookContext = new HookContext(plan, conf, ctx.getPathToCS());
       hookContext.setHookType(HookContext.HookType.PRE_EXEC_HOOK);
 
-      for (Hook peh : getPreExecHooks()) {
+      for (Hook peh : getHooks(HiveConf.ConfVars.PREEXECHOOKS)) {
         if (peh instanceof ExecuteWithHookContext) {
           perfLogger.PerfLogBegin(LOG, PerfLogger.PRE_HOOK + peh.getClass().getName());
 
@@ -1158,7 +1117,7 @@ public class Driver implements CommandPr
           } else {
             hookContext.setHookType(HookContext.HookType.ON_FAILURE_HOOK);
             // Get all the failure execution hooks and execute them.
-            for (Hook ofh : getOnFailureHooks()) {
+            for (Hook ofh : getHooks(HiveConf.ConfVars.ONFAILUREHOOKS)) {
               perfLogger.PerfLogBegin(LOG, PerfLogger.FAILURE_HOOK + ofh.getClass().getName());
 
               ((ExecuteWithHookContext) ofh).run(hookContext);
@@ -1216,7 +1175,7 @@ public class Driver implements CommandPr
 
       hookContext.setHookType(HookContext.HookType.POST_EXEC_HOOK);
       // Get all the post execution hooks and execute them.
-      for (Hook peh : getPostExecHooks()) {
+      for (Hook peh : getHooks(HiveConf.ConfVars.POSTEXECHOOKS)) {
         if (peh instanceof ExecuteWithHookContext) {
           perfLogger.PerfLogBegin(LOG, PerfLogger.POST_HOOK + peh.getClass().getName());
 

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java?rev=1241729&r1=1241728&r2=1241729&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java Wed Feb  8 01:37:59 2012
@@ -23,18 +23,17 @@ import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.index.HiveIndexHandler;
-import org.apache.hadoop.hive.ql.parse.AbstractSemanticAnalyzerHook;
 import org.apache.hadoop.hive.ql.security.HadoopDefaultAuthenticator;
 import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider;
 import org.apache.hadoop.hive.ql.security.authorization.DefaultHiveAuthorizationProvider;
 import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
 
 /**
  * General collection of helper functions.
- * 
+ *
  */
 public final class HiveUtils {
 
@@ -136,7 +135,7 @@ public final class HiveUtils {
 
   public static HiveStorageHandler getStorageHandler(
     Configuration conf, String className) throws HiveException {
-    
+
     if (className == null) {
       return null;
     }
@@ -175,7 +174,7 @@ public final class HiveUtils {
           + e.getMessage(), e);
     }
   }
-  
+
   @SuppressWarnings("unchecked")
   public static HiveAuthorizationProvider getAuthorizeProviderManager(
       Configuration conf, HiveAuthenticationProvider authenticator) throws HiveException {
@@ -227,34 +226,18 @@ public final class HiveUtils {
     return ret;
   }
 
-  public static AbstractSemanticAnalyzerHook getSemanticAnalyzerHook(
-      HiveConf conf, String hookName) throws HiveException{
-    try {
-      Class<? extends AbstractSemanticAnalyzerHook> hookClass =
-        (Class<? extends AbstractSemanticAnalyzerHook>)
-        Class.forName(hookName, true, JavaUtils.getClassLoader());
-      return (AbstractSemanticAnalyzerHook) ReflectionUtils.newInstance(
-          hookClass, conf);
-    } catch (ClassNotFoundException e) {
-      throw new HiveException("Error in loading semantic analyzer hook: "+
-          hookName +e.getMessage(),e);
-    }
-
-  }
-
-
-    /**
-     * Convert FieldSchemas to columnNames with backticks around them.
-     */
-    public static String getUnparsedColumnNamesFromFieldSchema(
-        List<FieldSchema> fieldSchemas) {
-      StringBuilder sb = new StringBuilder();
-      for (int i = 0; i < fieldSchemas.size(); i++) {
-        if (i > 0) {
-          sb.append(",");
-        }
-        sb.append(HiveUtils.unparseIdentifier(fieldSchemas.get(i).getName()));
+  /**
+   * Convert FieldSchemas to columnNames with backticks around them.
+   */
+  public static String getUnparsedColumnNamesFromFieldSchema(
+      List<FieldSchema> fieldSchemas) {
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < fieldSchemas.size(); i++) {
+      if (i > 0) {
+        sb.append(",");
       }
-      return sb.toString();
+      sb.append(HiveUtils.unparseIdentifier(fieldSchemas.get(i).getName()));
     }
+    return sb.toString();
+  }
 }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveSemanticAnalyzerHook.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveSemanticAnalyzerHook.java?rev=1241729&r1=1241728&r2=1241729&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveSemanticAnalyzerHook.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveSemanticAnalyzerHook.java Wed Feb  8 01:37:59 2012
@@ -22,6 +22,7 @@ import java.io.Serializable;
 import java.util.List;
 
 import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.hooks.Hook;
 
 /**
  * HiveSemanticAnalyzerHook allows Hive to be extended with custom
@@ -35,7 +36,7 @@ import org.apache.hadoop.hive.ql.exec.Ta
  * Note that the lifetime of an instantiated hook object is scoped to
  * the analysis of a single statement; hook instances are never reused.
  */
-public interface HiveSemanticAnalyzerHook {
+public interface HiveSemanticAnalyzerHook extends Hook {
   /**
    * Invoked before Hive performs its own semantic analysis on
    * a statement.  The implementation may inspect the statement AST and

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveSemanticAnalyzerHookContext.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveSemanticAnalyzerHookContext.java?rev=1241729&r1=1241728&r2=1241729&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveSemanticAnalyzerHookContext.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveSemanticAnalyzerHookContext.java Wed Feb  8 01:37:59 2012
@@ -18,7 +18,11 @@
 
 package org.apache.hadoop.hive.ql.parse;
 
+import java.util.Set;
+
 import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.hooks.WriteEntity;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 
@@ -32,4 +36,21 @@ public interface HiveSemanticAnalyzerHoo
    * purposes such as getting configuration information or making metastore calls
    */
   public Hive getHive() throws HiveException;
+
+
+  /**
+   * This should be called after the semantic analyzer completes.
+   * @param sem
+   */
+  public void update(BaseSemanticAnalyzer sem);
+
+
+  /**
+   * The following methods will only be available to hooks executing postAnalyze.  If called in a
+   * preAnalyze method, they should return null.
+   * @return
+   */
+  public Set<ReadEntity> getInputs();
+
+  public Set<WriteEntity> getOutputs();
 }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveSemanticAnalyzerHookContextImpl.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveSemanticAnalyzerHookContextImpl.java?rev=1241729&r1=1241728&r2=1241729&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveSemanticAnalyzerHookContextImpl.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveSemanticAnalyzerHookContextImpl.java Wed Feb  8 01:37:59 2012
@@ -18,14 +18,20 @@
 
 package org.apache.hadoop.hive.ql.parse;
 
+import java.util.Set;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.hooks.WriteEntity;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 public class HiveSemanticAnalyzerHookContextImpl implements HiveSemanticAnalyzerHookContext {
 
   Configuration conf;
+  Set<ReadEntity> inputs = null;
+  Set<WriteEntity> outputs = null;
 
   @Override
   public Hive getHive() throws HiveException {
@@ -43,4 +49,19 @@ public class HiveSemanticAnalyzerHookCon
     this.conf = conf;
   }
 
+  @Override
+  public void update(BaseSemanticAnalyzer sem) {
+    this.inputs = sem.getInputs();
+    this.outputs = sem.getOutputs();
+  }
+
+  @Override
+  public Set<ReadEntity> getInputs() {
+    return inputs;
+  }
+
+  @Override
+  public Set<WriteEntity> getOutputs() {
+    return outputs;
+  }
 }

Added: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/hooks/VerifyHooksRunInOrder.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/hooks/VerifyHooksRunInOrder.java?rev=1241729&view=auto
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/hooks/VerifyHooksRunInOrder.java (added)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/hooks/VerifyHooksRunInOrder.java Wed Feb  8 01:37:59 2012
@@ -0,0 +1,158 @@
+/**
+ * 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 java.io.Serializable;
+import java.util.List;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.hooks.HookContext.HookType;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.AbstractSemanticAnalyzerHook;
+import org.apache.hadoop.hive.ql.parse.HiveSemanticAnalyzerHookContext;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.ql.session.SessionState.LogHelper;
+
+/**
+ * VerifyHooksRunInOrder.
+ *
+ * Has to subclasses RunFirst and RunSecond which can be run as either pre or post hooks.
+ * Verifies that RunFirst is executed before RunSecond as the same type of hook.  I.e. if they are
+ * run as both Pre and Post hooks, RunSecond checks that RunFirst was run as a Pre or Post hook
+ * respectively.
+ *
+ * When running this, be sure to specify RunFirst before RunSecond in the configuration variable.
+ */
+public class VerifyHooksRunInOrder {
+
+  private static boolean preHookRunFirstRan = false;
+  private static boolean postHookRunFirstRan = false;
+  private static boolean staticAnalysisPreHookFirstRan = false;
+  private static boolean staticAnalysisPostHookFirstRan = false;
+
+  public static class RunFirst implements ExecuteWithHookContext {
+    public void run(HookContext hookContext) {
+      LogHelper console = SessionState.getConsole();
+
+      if (console == null) {
+        return;
+      }
+
+      // This is simply to verify that the hooks were in fact run
+      console.printError("Running RunFirst for " + hookContext.getHookType());
+
+      if (hookContext.getHookType() == HookType.PRE_EXEC_HOOK) {
+        preHookRunFirstRan = true;
+      } else {
+        postHookRunFirstRan = true;
+      }
+    }
+  }
+
+  public static class RunSecond implements ExecuteWithHookContext {
+    public void run(HookContext hookContext) throws Exception {
+      LogHelper console = SessionState.getConsole();
+
+      if (console == null) {
+        return;
+      }
+
+      // This is simply to verify that the hooks were in fact run
+      console.printError("Running RunSecond for " + hookContext.getHookType());
+
+      if (hookContext.getHookType() == HookType.PRE_EXEC_HOOK) {
+        Assert.assertTrue("Pre hooks did not run in the order specified.", preHookRunFirstRan);
+      } else {
+        Assert.assertTrue("Post hooks did not run in the order specified.", postHookRunFirstRan);
+      }
+    }
+  }
+
+  public static class RunFirstSemanticAnalysisHook extends AbstractSemanticAnalyzerHook {
+    @Override
+    public ASTNode preAnalyze(HiveSemanticAnalyzerHookContext context,ASTNode ast)
+        throws SemanticException {
+      LogHelper console = SessionState.getConsole();
+
+      if (console == null) {
+        return ast;
+      }
+
+      // This is simply to verify that the hooks were in fact run
+      console.printError("Running RunFirst for Pre Analysis Hook");
+
+      staticAnalysisPreHookFirstRan = true;
+
+      return ast;
+    }
+
+    @Override
+    public void postAnalyze(HiveSemanticAnalyzerHookContext context,
+        List<Task<? extends Serializable>> rootTasks) throws SemanticException {
+      LogHelper console = SessionState.getConsole();
+
+      if (console == null) {
+        return;
+      }
+
+      // This is simply to verify that the hooks were in fact run
+      console.printError("Running RunFirst for Post Analysis Hook");
+
+      staticAnalysisPostHookFirstRan = true;
+    }
+  }
+
+  public static class RunSecondSemanticAnalysisHook extends AbstractSemanticAnalyzerHook {
+    @Override
+    public ASTNode preAnalyze(HiveSemanticAnalyzerHookContext context,ASTNode ast)
+        throws SemanticException {
+      LogHelper console = SessionState.getConsole();
+
+      if (console == null) {
+        return ast;
+      }
+
+      // This is simply to verify that the hooks were in fact run
+      console.printError("Running RunSecond for Pre Analysis Hook");
+
+      Assert.assertTrue("Pre Analysis Hooks did not run in the order specified.",
+                        staticAnalysisPreHookFirstRan);
+
+      return ast;
+    }
+
+    @Override
+    public void postAnalyze(HiveSemanticAnalyzerHookContext context,
+        List<Task<? extends Serializable>> rootTasks) throws SemanticException {
+      LogHelper console = SessionState.getConsole();
+
+      if (console == null) {
+        return;
+      }
+
+      // This is simply to verify that the hooks were in fact run
+      console.printError("Running RunSecond for Post Analysis Hook");
+
+      Assert.assertTrue("Post Analysis Hooks did not run in the order specified.",
+                        staticAnalysisPostHookFirstRan);
+    }
+  }
+}

Added: hive/trunk/ql/src/test/queries/clientpositive/hook_order.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/hook_order.q?rev=1241729&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/hook_order.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/hook_order.q Wed Feb  8 01:37:59 2012
@@ -0,0 +1,9 @@
+SET hive.exec.pre.hooks=org.apache.hadoop.hive.ql.hooks.VerifyHooksRunInOrder$RunFirst,org.apache.hadoop.hive.ql.hooks.VerifyHooksRunInOrder$RunSecond;
+SET hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.VerifyHooksRunInOrder$RunFirst,org.apache.hadoop.hive.ql.hooks.VerifyHooksRunInOrder$RunSecond;
+SET hive.semantic.analyzer.hook=org.apache.hadoop.hive.ql.hooks.VerifyHooksRunInOrder$RunFirstSemanticAnalysisHook,org.apache.hadoop.hive.ql.hooks.VerifyHooksRunInOrder$RunSecondSemanticAnalysisHook;
+
+SELECT count(*) FROM src;
+
+SET hive.exec.pre.hooks=;
+SET hive.exec.post.hooks=;
+SET hive.semantic.analyzer.hook=;

Modified: hive/trunk/ql/src/test/results/clientnegative/bad_exec_hooks.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/bad_exec_hooks.q.out?rev=1241729&r1=1241728&r2=1241729&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/bad_exec_hooks.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/bad_exec_hooks.q.out Wed Feb  8 01:37:59 2012
@@ -1,4 +1,4 @@
-Pre Exec Hook 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 ####

Added: hive/trunk/ql/src/test/results/clientpositive/hook_order.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/hook_order.q.out?rev=1241729&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/hook_order.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/hook_order.q.out Wed Feb  8 01:37:59 2012
@@ -0,0 +1,9 @@
+Running RunFirst for Pre Analysis Hook
+Running RunSecond for Pre Analysis Hook
+Running RunFirst for Post Analysis Hook
+Running RunSecond for Post Analysis Hook
+Running RunFirst for PRE_EXEC_HOOK
+Running RunSecond for PRE_EXEC_HOOK
+Running RunFirst for POST_EXEC_HOOK
+Running RunSecond for POST_EXEC_HOOK
+500