You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2015/10/02 04:37:41 UTC

[06/22] hive git commit: HIVE-11916 TxnHandler.getOpenTxnsInfo() and getOpenTxns() may produce inconsistent result (Eugene Koifman, reviewed by Ashutosh Chauhan)

HIVE-11916 TxnHandler.getOpenTxnsInfo() and getOpenTxns() may produce inconsistent result (Eugene Koifman, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/llap
Commit: a6ab68e8ff889933a6d7d164d8c91ed8d3fa8609
Parents: 0d43e87
Author: Eugene Koifman <ek...@hortonworks.com>
Authored: Wed Sep 30 16:07:20 2015 -0700
Committer: Eugene Koifman <ek...@hortonworks.com>
Committed: Wed Sep 30 16:07:20 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hive/metastore/txn/TxnHandler.java      | 17 ++++++++++-------
 1 file changed, 10 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/a6ab68e8/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
index cc7e2c6..6218a03 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
@@ -47,8 +47,9 @@ import java.util.concurrent.TimeUnit;
  * A handler to answer transaction related calls that come into the metastore
  * server.
  *
- * Note on log messages:  Please include txnid:X and lockid info
- * {@link org.apache.hadoop.hive.common.JavaUtils#lockIdToString(long)} in all messages.
+ * Note on log messages:  Please include txnid:X and lockid info using
+ * {@link org.apache.hadoop.hive.common.JavaUtils#txnIdToString(long)}
+ * and {@link org.apache.hadoop.hive.common.JavaUtils#lockIdToString(long)} in all messages.
  * The txnid:X and lockid:Y matches how Thrift object toString() methods are generated,
  * so keeping the format consistent makes grep'ing the logs much easier.
  */
@@ -166,7 +167,8 @@ public class TxnHandler {
         }
 
         List<TxnInfo> txnInfo = new ArrayList<TxnInfo>();
-        s = "select txn_id, txn_state, txn_user, txn_host from TXNS";
+        //need the WHERE clause below to ensure consistent results with READ_COMMITTED
+        s = "select txn_id, txn_state, txn_user, txn_host from TXNS where txn_id <= " + hwm;
         LOG.debug("Going to execute query<" + s + ">");
         rs = stmt.executeQuery(s);
         while (rs.next()) {
@@ -230,7 +232,8 @@ public class TxnHandler {
         }
 
         Set<Long> openList = new HashSet<Long>();
-        s = "select txn_id from TXNS";
+        //need the WHERE clause below to ensure consistent results with READ_COMMITTED
+        s = "select txn_id from TXNS where txn_id <= " + hwm;
         LOG.debug("Going to execute query<" + s + ">");
         rs = stmt.executeQuery(s);
         while (rs.next()) {
@@ -1459,7 +1462,7 @@ public class TxnHandler {
     LockResponse response = new LockResponse();
     response.setLockid(extLockId);
 
-    LOG.debug("checkLock(): Setting savepoint. extLockId=" + 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, " +
@@ -1685,7 +1688,7 @@ public class TxnHandler {
     if (rc < 1) {
       LOG.debug("Going to rollback");
       dbConn.rollback();
-      throw new NoSuchLockException("No such lock: (" + extLockId + "," +
+      throw new NoSuchLockException("No such lock: (" + JavaUtils.lockIdToString(extLockId) + "," +
         + intLockId + ")");
     }
     // We update the database, but we don't commit because there may be other
@@ -1710,7 +1713,7 @@ public class TxnHandler {
       if (rc < 1) {
         LOG.debug("Going to rollback");
         dbConn.rollback();
-        throw new NoSuchLockException("No such lock: " + extLockId);
+        throw new NoSuchLockException("No such lock: " + JavaUtils.lockIdToString(extLockId));
       }
       LOG.debug("Going to commit");
       dbConn.commit();