You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by an...@apache.org on 2017/11/22 06:03:22 UTC

hive git commit: HIVE-18090: acid heartbeat fails when metastore is connected via hadoop credential (Anishek Agarwal, reviewed by Eugene Koifman)

Repository: hive
Updated Branches:
  refs/heads/master 179e32f0b -> 67499f41a


HIVE-18090: acid heartbeat fails when metastore is connected via hadoop credential (Anishek Agarwal, reviewed by Eugene Koifman)


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

Branch: refs/heads/master
Commit: 67499f41a02c2f739a3a005a102c0ceda8d2eeba
Parents: 179e32f
Author: Anishek Agarwal <an...@gmail.com>
Authored: Wed Nov 22 11:33:14 2017 +0530
Committer: Anishek Agarwal <an...@gmail.com>
Committed: Wed Nov 22 11:33:14 2017 +0530

----------------------------------------------------------------------
 .../hadoop/hive/ql/lockmgr/DbTxnManager.java    | 68 +++++++++++++-------
 1 file changed, 43 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/67499f41/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
index 82804b2..6fb0c43 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
@@ -1,19 +1,19 @@
-/**
- * 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.
+/*
+  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.lockmgr;
 
@@ -26,6 +26,7 @@ import org.apache.hadoop.hive.ql.plan.LockDatabaseDesc;
 import org.apache.hadoop.hive.ql.plan.LockTableDesc;
 import org.apache.hadoop.hive.ql.plan.UnlockDatabaseDesc;
 import org.apache.hadoop.hive.ql.plan.UnlockTableDesc;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hive.common.util.ShutdownHookManager;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -46,6 +47,8 @@ import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.thrift.TException;
 
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.Executors;
@@ -634,7 +637,14 @@ public final class DbTxnManager extends HiveTxnManagerImpl {
   private Heartbeater startHeartbeat(long initialDelay) throws LockException {
     long heartbeatInterval = getHeartbeatInterval(conf);
     assert heartbeatInterval > 0;
-    Heartbeater heartbeater = new Heartbeater(this, conf, queryId);
+    UserGroupInformation currentUser;
+    try {
+      currentUser = UserGroupInformation.getCurrentUser();
+    } catch (IOException e) {
+      throw new LockException("error while getting current user,", e);
+    }
+
+    Heartbeater heartbeater = new Heartbeater(this, conf, queryId, currentUser);
     // For negative testing purpose..
     if(conf.getBoolVar(HiveConf.ConfVars.HIVE_IN_TEST) && conf.getBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILHEARTBEATER)) {
       initialDelay = 0;
@@ -847,6 +857,7 @@ public final class DbTxnManager extends HiveTxnManagerImpl {
   public static class Heartbeater implements Runnable {
     private HiveTxnManager txnMgr;
     private HiveConf conf;
+    private UserGroupInformation currentUser;
     LockException lockException;
     private final String queryId;
 
@@ -856,10 +867,13 @@ public final class DbTxnManager extends HiveTxnManagerImpl {
     /**
      *
      * @param txnMgr transaction manager for this operation
+     * @param currentUser
      */
-    Heartbeater(HiveTxnManager txnMgr, HiveConf conf, String queryId) {
+    Heartbeater(HiveTxnManager txnMgr, HiveConf conf, String queryId,
+        UserGroupInformation currentUser) {
       this.txnMgr = txnMgr;
       this.conf = conf;
+      this.currentUser = currentUser;
       lockException = null;
       this.queryId = queryId;
     }
@@ -874,16 +888,20 @@ public final class DbTxnManager extends HiveTxnManagerImpl {
         if(conf.getBoolVar(HiveConf.ConfVars.HIVE_IN_TEST) && conf.getBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILHEARTBEATER)) {
           throw new LockException(HiveConf.ConfVars.HIVETESTMODEFAILHEARTBEATER.name() + "=true");
         }
-        LOG.debug("Heartbeating...");
-        txnMgr.heartbeat();
+        LOG.debug("Heartbeating...for currentUser: " + currentUser);
+        currentUser.doAs((PrivilegedExceptionAction<Object>) () -> {
+          txnMgr.heartbeat();
+          return null;
+        });
       } catch (LockException e) {
-        LOG.error("Failed trying to heartbeat queryId=" + queryId + ": " + e.getMessage());
+        LOG.error("Failed trying to heartbeat queryId=" + queryId + ", currentUser: "
+            + currentUser + ": " + e.getMessage());
         lockException = e;
       } catch (Throwable t) {
-        LOG.error("Failed trying to heartbeat queryId=" + queryId + ": " + t.getMessage(), t);
-        lockException =
-            new LockException("Failed trying to heartbeat queryId=" + queryId + ": "
-                + t.getMessage(), t);
+        String errorMsg = "Failed trying to heartbeat queryId=" + queryId + ", currentUser: "
+            + currentUser + ": " + t.getMessage();
+        LOG.error(errorMsg, t);
+        lockException = new LockException(errorMsg, t);
       }
     }
   }