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

[hive] branch master updated: HIVE-22906: Redundant checkLock Mutex blocks concurrent Lock requests (Denys Kuzmenko, reviewed by Peter Vary)

This is an automated email from the ASF dual-hosted git repository.

dkuzmenko pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new 326f311  HIVE-22906: Redundant checkLock Mutex blocks concurrent Lock requests (Denys Kuzmenko, reviewed by Peter Vary)
326f311 is described below

commit 326f31194c46d5c30bc1551a01ca1f4aad1231c2
Author: Denys Kuzmenko <dk...@apache.org>
AuthorDate: Wed Mar 11 12:35:07 2020 +0100

    HIVE-22906: Redundant checkLock Mutex blocks concurrent Lock requests (Denys Kuzmenko, reviewed by Peter Vary)
---
 .../hadoop/hive/metastore/txn/TxnHandler.java      | 29 ++--------------------
 1 file changed, 2 insertions(+), 27 deletions(-)

diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
index f53aebe..2995afa 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
@@ -4311,8 +4311,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
    */
   @RetrySemantics.SafeToRetry("See @SafeToRetry")
   private LockResponse checkLock(Connection dbConn, long extLockId)
-    throws NoSuchLockException, NoSuchTxnException, TxnAbortedException, MetaException, SQLException {
-    TxnStore.MutexAPI.LockHandle handle =  null;
+    throws NoSuchLockException, TxnAbortedException, MetaException, SQLException {
     Statement stmt = null;
     ResultSet rs = null;
     LockResponse response = new LockResponse();
@@ -4328,16 +4327,10 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
      */
     boolean isPartOfDynamicPartitionInsert = true;
     try {
-      /**
-       * checkLock() must be mutex'd against any other checkLock to make sure 2 conflicting locks
-       * are not granted by parallel checkLock() calls.
-       */
-      handle = getMutexAPI().acquireLock(MUTEX_KEY.CheckLock.name());
       List<LockInfo> locksBeingChecked = getLockInfoFromLockId(dbConn, extLockId);//being acquired now
       response.setLockid(extLockId);
-
       LOG.debug("checkLock(): Setting savepoint. extLockId=" + JavaUtils.lockIdToString(extLockId));
-      Savepoint save = dbConn.setSavepoint();
+
       StringBuilder query = new StringBuilder("SELECT \"HL_LOCK_EXT_ID\", " +
         "\"HL_LOCK_INT_ID\", \"HL_DB\", \"HL_TABLE\", \"HL_PARTITION\", \"HL_LOCK_STATE\", " +
         "\"HL_LOCK_TYPE\", \"HL_TXNID\" FROM \"HIVE_LOCKS\" WHERE \"HL_DB\" IN (");
@@ -4526,11 +4519,6 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
           switch (lockAction) {
             case WAIT:
               if(!ignoreConflict(info, locks[i])) {
-                /*we acquire all locks for a given query atomically; if 1 blocks, all go into (remain) in
-                * Waiting state.  wait() will undo any 'acquire()' which may have happened as part of
-                * this (metastore db) transaction and then we record which lock blocked the lock
-                * we were testing ('info').*/
-                wait(dbConn, save);
                 String sqlText = "UPDATE \"HIVE_LOCKS\"" +
                   " SET \"HL_BLOCKEDBY_EXT_ID\"=" + locks[i].extLockId +
                   ", \"HL_BLOCKEDBY_INT_ID\"=" + locks[i].intLockId +
@@ -4565,9 +4553,6 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
       response.setState(LockState.ACQUIRED);
     } finally {
       close(rs, stmt, null);
-      if(handle != null) {
-        handle.releaseLocks();
-      }
     }
     return response;
   }
@@ -4645,16 +4630,6 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
       (desiredLock.txnId == 0 &&  desiredLock.extLockId == existingLock.extLockId);
   }
 
-  private void wait(Connection dbConn, Savepoint save) throws SQLException {
-    // Need to rollback because we did a select that acquired locks but we didn't
-    // actually update anything.  Also, we may have locked some locks as
-    // acquired that we now want to not acquire.  It's ok to rollback because
-    // once we see one wait, we're done, we won't look for more.
-    // Only rollback to savepoint because we want to commit our heartbeat
-    // changes.
-    LOG.debug("Going to rollback to savepoint");
-    dbConn.rollback(save);
-  }
   /**
    * Heartbeats on the lock table.  This commits, so do not enter it with any state.
    * Should not be called on a lock that belongs to transaction.