You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2017/11/15 01:30:21 UTC

hive git commit: HIVE-18057 : remove PostExecute / PreExecute hook support (Zoltan Haindrich via Ashutosh Chauhan)

Repository: hive
Updated Branches:
  refs/heads/master d2958495a -> 3bb46de8c


HIVE-18057 : remove PostExecute / PreExecute hook support (Zoltan Haindrich via Ashutosh Chauhan)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/master
Commit: 3bb46de8cb20a2116c439fa9dbd9c4e56c041706
Parents: d295849
Author: Zoltan Haindrich <ki...@rxd.hu>
Authored: Tue Nov 14 17:29:03 2017 -0800
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Tue Nov 14 17:29:03 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hive/ql/Driver.java  | 33 +++---------
 .../hadoop/hive/ql/hooks/PostExecute.java       | 55 --------------------
 .../apache/hadoop/hive/ql/hooks/PreExecute.java | 53 -------------------
 .../ql/hooks/UpdateInputAccessTimeHook.java     | 15 +++---
 4 files changed, 14 insertions(+), 142 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/3bb46de8/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 017373c..af9f193 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -69,8 +69,6 @@ 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.HooksLoader;
-import org.apache.hadoop.hive.ql.hooks.PostExecute;
-import org.apache.hadoop.hive.ql.hooks.PreExecute;
 import org.apache.hadoop.hive.ql.hooks.ReadEntity;
 import org.apache.hadoop.hive.ql.hooks.WriteEntity;
 import org.apache.hadoop.hive.ql.lockmgr.HiveLock;
@@ -116,7 +114,6 @@ import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.session.SessionState.LogHelper;
 import org.apache.hadoop.hive.ql.wm.TriggerContext;
 import org.apache.hadoop.hive.serde2.ByteStream;
-import org.apache.hadoop.hive.shims.Utils;
 import org.apache.hadoop.mapred.ClusterStatus;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
@@ -1822,20 +1819,12 @@ public class Driver implements CommandProcessor {
       hookContext.setHookType(HookContext.HookType.PRE_EXEC_HOOK);
 
       for (Hook peh : hooksLoader.getHooks(HiveConf.ConfVars.PREEXECHOOKS, console)) {
-        if (peh instanceof ExecuteWithHookContext) {
-          perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.PRE_HOOK + peh.getClass().getName());
+        perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.PRE_HOOK + peh.getClass().getName());
 
-          ((ExecuteWithHookContext) peh).run(hookContext);
+        ((ExecuteWithHookContext) peh).run(hookContext);
 
-          perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.PRE_HOOK + peh.getClass().getName());
-        } else if (peh instanceof PreExecute) {
-          perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.PRE_HOOK + peh.getClass().getName());
+        perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.PRE_HOOK + peh.getClass().getName());
 
-          ((PreExecute) peh).run(SessionState.get(), plan.getInputs(), plan.getOutputs(),
-              Utils.getUGI());
-
-          perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.PRE_HOOK + peh.getClass().getName());
-        }
       }
 
       // Trigger query hooks before query execution.
@@ -2024,21 +2013,11 @@ public class Driver implements CommandProcessor {
       hookContext.setHookType(HookContext.HookType.POST_EXEC_HOOK);
       // Get all the post execution hooks and execute them.
       for (Hook peh : hooksLoader.getHooks(HiveConf.ConfVars.POSTEXECHOOKS, console)) {
-        if (peh instanceof ExecuteWithHookContext) {
-          perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.POST_HOOK + peh.getClass().getName());
-
-          ((ExecuteWithHookContext) peh).run(hookContext);
+        perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.POST_HOOK + peh.getClass().getName());
 
-          perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.POST_HOOK + peh.getClass().getName());
-        } else if (peh instanceof PostExecute) {
-          perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.POST_HOOK + peh.getClass().getName());
+        ((ExecuteWithHookContext) peh).run(hookContext);
 
-          ((PostExecute) peh).run(SessionState.get(), plan.getInputs(), plan.getOutputs(),
-              (SessionState.get() != null ? SessionState.get().getLineageState().getLineageInfo()
-                  : null), Utils.getUGI());
-
-          perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.POST_HOOK + peh.getClass().getName());
-        }
+        perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.POST_HOOK + peh.getClass().getName());
       }
 
       if (SessionState.get() != null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/3bb46de8/ql/src/java/org/apache/hadoop/hive/ql/hooks/PostExecute.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/PostExecute.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/PostExecute.java
deleted file mode 100644
index 466450e..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/PostExecute.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.ql.hooks;
-
-import java.util.Set;
-
-import org.apache.hadoop.hive.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.hadoop.security.UserGroupInformation;
-
-/**
- * The post execute hook interface. A list of such hooks can be configured to be
- * called after compilation and before execution.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public interface PostExecute extends Hook {
-
-  /**
-   * The run command that is called just before the execution of the query.
-   *
-   * @param sess
-   *          The session state.
-   * @param inputs
-   *          The set of input tables and partitions.
-   * @param outputs
-   *          The set of output tables, partitions, local and hdfs directories.
-   * @param lInfo
-   *           The column level lineage information.
-   * @param ugi
-   *          The user group security information.
-   */
-  @Deprecated
-  void run(SessionState sess, Set<ReadEntity> inputs,
-      Set<WriteEntity> outputs, LineageInfo lInfo,
-      UserGroupInformation ugi) throws Exception;
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/3bb46de8/ql/src/java/org/apache/hadoop/hive/ql/hooks/PreExecute.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/PreExecute.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/PreExecute.java
deleted file mode 100644
index bc16334..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/PreExecute.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.ql.hooks;
-
-import java.util.Set;
-
-import org.apache.hadoop.hive.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.hadoop.security.UserGroupInformation;
-
-/**
- * The pre execute hook interface. A list of such hooks can be configured to be
- * called after compilation and before execution.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public interface PreExecute extends Hook {
-
-  /**
-   * The run command that is called just before the execution of the query.
-   *
-   * @param sess
-   *          The session state.
-   * @param inputs
-   *          The set of input tables and partitions.
-   * @param outputs
-   *          The set of output tables, partitions, local and hdfs directories.
-   * @param ugi
-   *          The user group security information.
-   */
-  @Deprecated
-  public void run(SessionState sess, Set<ReadEntity> inputs,
-      Set<WriteEntity> outputs, UserGroupInformation ugi)
-    throws Exception;
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/3bb46de8/ql/src/java/org/apache/hadoop/hive/ql/hooks/UpdateInputAccessTimeHook.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/UpdateInputAccessTimeHook.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/UpdateInputAccessTimeHook.java
index dd1c1e1..edd9bfc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/UpdateInputAccessTimeHook.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/UpdateInputAccessTimeHook.java
@@ -19,8 +19,7 @@ package org.apache.hadoop.hive.ql.hooks;
 
 import java.util.Set;
 
-import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Partition;
@@ -34,14 +33,16 @@ public class UpdateInputAccessTimeHook {
 
   private static final String LAST_ACCESS_TIME = "lastAccessTime";
 
-  public static class PreExec implements PreExecute {
-    public void run(SessionState sess, Set<ReadEntity> inputs,
-                    Set<WriteEntity> outputs, UserGroupInformation ugi)
-      throws Exception {
+  public static class PreExec implements ExecuteWithHookContext {
+
+    @Override
+    public void run(HookContext hookContext) throws Exception {
+      HiveConf conf = hookContext.getConf();
+      Set<ReadEntity> inputs = hookContext.getQueryPlan().getInputs();
 
       Hive db;
       try {
-        db = Hive.get(sess.getConf());
+        db = Hive.get(conf);
       } catch (HiveException e) {
         // ignore
         db = null;