You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ve...@apache.org on 2023/12/18 11:59:50 UTC

(hive) branch master updated: HIVE-27481: TxnHandler cleanup (Laszlo Vegh, reviewed by Denys Kuzmenko, Krisztian Kasa, Zoltan Ratkai, Laszlo Bodor)

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

veghlaci05 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 81c89cdb7b0 HIVE-27481: TxnHandler cleanup (Laszlo Vegh, reviewed by Denys Kuzmenko, Krisztian Kasa, Zoltan Ratkai, Laszlo Bodor)
81c89cdb7b0 is described below

commit 81c89cdb7b01617dc1ab847a52805c16c9a5621a
Author: veghlaci05 <ve...@gmail.com>
AuthorDate: Mon Dec 18 12:59:43 2023 +0100

    HIVE-27481: TxnHandler cleanup (Laszlo Vegh, reviewed by Denys Kuzmenko, Krisztian Kasa, Zoltan Ratkai, Laszlo Bodor)
---
 .../hive/compaction/IcebergCompactionService.java  |    2 +-
 .../parse/TestReplicationScenariosAcidTables.java  |    2 +-
 .../hive/ql/txn/compactor/TestCompactor.java       |    5 +-
 .../ql/txn/compactor/TestCrudCompactorOnTez.java   |    2 +-
 .../org/apache/hadoop/hive/ql/io/AcidUtils.java    |    2 +-
 .../hive/ql/txn/compactor/CompactorContext.java    |    2 +-
 .../hive/ql/txn/compactor/CompactorFactory.java    |    4 +-
 .../hive/ql/txn/compactor/CompactorThread.java     |    2 +-
 .../hive/ql/txn/compactor/CompactorUtil.java       |    2 +-
 .../hadoop/hive/ql/txn/compactor/Initiator.java    |    2 +-
 .../hive/ql/txn/compactor/InitiatorBase.java       |    2 +-
 .../hadoop/hive/ql/txn/compactor/MRCompactor.java  |    2 +-
 .../hive/ql/txn/compactor/MergeCompactor.java      |    2 +-
 .../ql/txn/compactor/MetaStoreCompactorThread.java |    2 +-
 .../hive/ql/txn/compactor/QueryCompactor.java      |    2 +-
 .../ql/txn/compactor/RemoteCompactorThread.java    |    2 +-
 .../hive/ql/txn/compactor/RemoteCompactorUtil.java |    2 +-
 .../hadoop/hive/ql/txn/compactor/StatsUpdater.java |    3 +-
 .../hadoop/hive/ql/txn/compactor/Worker.java       |    8 +-
 .../txn/compactor/handler/AbortedTxnCleaner.java   |    2 +-
 .../txn/compactor/handler/CompactionCleaner.java   |    2 +-
 .../hive/ql/txn/compactor/handler/TaskHandler.java |    3 +-
 .../compactor/service/AcidCompactionService.java   |    4 +-
 .../txn/compactor/service/CompactionService.java   |    2 +-
 .../metastore/txn/TestCompactionTxnHandler.java    |    7 +-
 .../hadoop/hive/metastore/txn/TestTxnHandler.java  |  129 +-
 .../apache/hadoop/hive/ql/TestTxnCommands3.java    |    2 +-
 .../hadoop/hive/ql/TxnCommandsBaseForTests.java    |    2 +-
 .../hive/ql/txn/compactor/CompactorTest.java       |    2 +-
 .../hadoop/hive/ql/txn/compactor/TestCleaner.java  |    2 +-
 .../ql/txn/compactor/TestCompactionMetrics.java    |    4 +-
 .../TestMRCompactorJobQueueConfiguration.java      |    2 +-
 .../compactor/handler/TestAbortedTxnCleaner.java   |    2 +-
 .../hadoop/hive/metastore/conf/MetastoreConf.java  |    2 +-
 .../hadoop/hive/metastore/txn/TxnQueries.java      |   88 -
 .../hadoop/hive/metastore/DatabaseProduct.java     |   25 +-
 .../apache/hadoop/hive/metastore/HMSHandler.java   |    1 +
 .../metastore/events/CommitCompactionEvent.java    |    3 +-
 .../hive/metastore/metrics/AcidMetricLogger.java   |   10 +-
 .../hive/metastore/metrics/AcidMetricService.java  |   10 +-
 .../txn/CompactionMetricsDataConverter.java        |    1 +
 .../hive/metastore/txn/CompactionTxnHandler.java   |  139 +-
 .../hive/metastore/txn/DefaultTxnLockManager.java  |  126 +
 ...xnHandler.java => TransactionalRetryProxy.java} |   91 +-
 .../hadoop/hive/metastore/txn/TxnHandler.java      | 6733 ++------------------
 .../hadoop/hive/metastore/txn/TxnLockManager.java  |   85 +
 .../apache/hadoop/hive/metastore/txn/TxnStore.java |  116 +-
 .../hadoop/hive/metastore/txn/TxnStoreMutex.java   |  209 +
 .../apache/hadoop/hive/metastore/txn/TxnUtils.java |  112 +-
 .../txn/{ => entities}/CompactionInfo.java         |    5 +-
 .../txn/{ => entities}/CompactionMetricsData.java  |    2 +-
 .../txn/{ => entities}/CompactionState.java        |   22 +-
 .../hive/metastore/txn/entities/LockInfo.java      |  147 +
 .../metastore/txn/{ => entities}/MetricsInfo.java  |    2 +-
 .../hive/metastore/txn/{ => entities}/OpenTxn.java |    8 +-
 .../metastore/txn/{ => entities}/OpenTxnList.java  |   11 +-
 .../txn/{ => entities}/OperationType.java          |    2 +-
 .../metastore/txn/{ => entities}/TxnStatus.java    |    2 +-
 .../metastore/txn/jdbc/ConditionalCommand.java     |   43 +
 .../metastore/txn/jdbc/InClauseBatchCommand.java   |   75 +
 .../txn/jdbc/MultiDataSourceJdbcResource.java      |  180 +-
 .../metastore/txn/jdbc/NoPoolConnectionPool.java   |  145 +
 .../txn/jdbc/ParameterizedBatchCommand.java        |   61 +
 .../RollbackException.java}                        |   34 +-
 .../metastore/txn/jdbc/TransactionContext.java     |   54 +-
 .../txn/jdbc/TransactionContextManager.java        |   58 +-
 .../metastore/txn/jdbc/TransactionalFunction.java  |    8 +-
 .../commands/AddWriteIdsToMinHistoryCommand.java   |   87 +
 .../commands/AddWriteIdsToTxnToWriteIdCommand.java |   67 +
 .../commands/DeleteInvalidOpenTxnsCommand.java}    |   29 +-
 .../commands/DeleteReplTxnMapEntryCommand.java}    |   35 +-
 .../commands}/InsertCompactionInfoCommand.java     |    6 +-
 .../commands/InsertCompactionRequestCommand.java   |   93 +
 .../InsertCompletedTxnComponentsCommand.java       |   70 +
 .../txn/jdbc/commands/InsertHiveLocksCommand.java  |   97 +
 .../jdbc/commands/InsertTxnComponentsCommand.java  |  185 +
 .../RemoveCompactionMetricsDataCommand.java        |    4 +-
 ...emoveDuplicateCompleteTxnComponentsCommand.java |    2 +-
 .../RemoveTxnsFromMinHistoryLevelCommand.java      |   47 +
 .../RemoveWriteIdsFromMinHistoryCommand.java       |   49 +
 .../jdbc/functions/AbortCompactionFunction.java    |  188 +
 .../txn/jdbc/functions/AbortTxnFunction.java       |  105 +
 .../txn/jdbc/functions/AbortTxnsFunction.java      |  209 +
 .../txn/jdbc/functions/AcquireTxnLockFunction.java |   49 +
 .../functions/AllocateTableWriteIdsFunction.java   |  348 +
 .../txn/jdbc/functions/CheckLockFunction.java      |  342 +
 .../functions}/CleanTxnToWriteIdTableFunction.java |   13 +-
 .../txn/jdbc/functions/CleanupRecordsFunction.java |  186 +
 .../txn/jdbc/functions/CommitTxnFunction.java      |  636 ++
 .../txn/jdbc/functions/CompactFunction.java        |  123 +
 .../txn/jdbc/functions/EnqueueLockFunction.java    |  118 +
 .../txn/jdbc/functions/EnsureValidTxnFunction.java |   70 +
 .../FindPotentialCompactionsFunction.java          |    8 +-
 .../GenerateCompactionQueueIdFunction.java         |   65 +
 ...GetMaterializationInvalidationInfoFunction.java |  180 +
 .../GetValidWriteIdsForTableFunction.java          |  155 +
 .../jdbc/functions/GetValidWriteIdsFunction.java   |  103 +
 .../txn/jdbc/functions/HeartbeatLockFunction.java  |   61 +
 .../txn/jdbc/functions/HeartbeatTxnFunction.java   |   73 +
 .../jdbc/functions/HeartbeatTxnRangeFunction.java  |  104 +
 .../LockMaterializationRebuildFunction.java        |   92 +
 .../functions}/MarkCleanedFunction.java            |   34 +-
 .../functions/MinOpenTxnIdWaterMarkFunction.java   |   67 +
 .../functions}/NextCompactionFunction.java         |    4 +-
 .../txn/jdbc/functions/OnRenameFunction.java       |  159 +
 .../txn/jdbc/functions/OpenTxnsFunction.java       |  233 +
 .../jdbc/functions/PerformTimeoutsFunction.java    |  203 +
 .../functions}/PurgeCompactionHistoryFunction.java |   25 +-
 .../ReleaseMaterializationRebuildLocks.java        |   78 +
 .../functions/ReplTableWriteIdStateFunction.java   |  158 +
 .../TopCompactionMetricsDataPerTypeFunction.java   |   11 +-
 .../jdbc/functions/UpdataDatabasePropFunction.java |   72 +
 .../UpdateCompactionMetricsDataFunction.java       |    6 +-
 .../queries/AbortTxnInfoHandler.java}              |   16 +-
 .../{impl => jdbc/queries}/AbortedTxnHandler.java  |    6 +-
 .../queries}/CheckFailedCompactionsHandler.java    |    4 +-
 .../queries}/CompactionCandidateHandler.java       |    4 +-
 .../queries}/CompactionMetricsDataHandler.java     |    4 +-
 .../txn/jdbc/queries/CountOpenTxnsHandler.java     |   63 +
 .../metastore/txn/jdbc/queries/DbTimeHandler.java  |   51 +
 .../queries}/FindColumnsWithStatsHandler.java      |    4 +-
 .../txn/jdbc/queries/FindTxnStateHandler.java      |   63 +
 .../queries}/GetCompactionInfoHandler.java         |    8 +-
 .../txn/jdbc/queries/GetDatabaseIdHandler.java     |   59 +
 .../txn/jdbc/queries/GetHighWaterMarkHandler.java  |   55 +
 .../GetLatestCommittedCompactionInfoHandler.java   |  100 +
 .../txn/jdbc/queries/GetLocksByLockId.java         |   81 +
 .../GetMaxAllocatedTableWriteIdHandler.java        |   61 +
 .../jdbc/queries/GetOpenTxnTypeAndLockHandler.java |   71 +
 .../txn/jdbc/queries/GetOpenTxnsListHandler.java   |  126 +
 .../queries/GetTxnDbsUpdatedHandler.java}          |   51 +-
 .../txn/jdbc/queries/GetWriteIdsHandler.java       |   81 +
 .../jdbc/queries/LatestTxnIdInConflictHandler.java |   79 +
 .../txn/jdbc/queries/MetricsInfoHandler.java       |   93 +
 .../queries}/MinUncommittedTxnIdHandler.java       |    4 +-
 .../OpenTxnTimeoutLowBoundaryTxnIdHandler.java     |   64 +
 .../queries}/ReadyToCleanAbortHandler.java         |    6 +-
 .../queries}/ReadyToCleanHandler.java              |   15 +-
 .../txn/jdbc/queries/ShowCompactHandler.java       |  185 +
 .../txn/jdbc/queries/ShowLocksHandler.java         |  179 +
 .../queries/TablesWithAbortedTxnsHandler.java}     |   58 +-
 .../queries/TargetTxnIdListHandler.java}           |   57 +-
 .../queries/TxnIdForWriteIdHandler.java}           |   55 +-
 .../{retryhandling => retry}/RetryPropagation.java |    2 +-
 .../txn/{retryhandling => retry}/SqlRetry.java     |    2 +-
 .../SqlRetryCallProperties.java                    |    2 +-
 .../SqlRetryException.java}                        |   30 +-
 .../{retryhandling => retry}/SqlRetryFunction.java |    5 +-
 .../{retryhandling => retry}/SqlRetryHandler.java  |   36 +-
 .../StackThreadLocal.java}                         |   53 +-
 .../hadoop/hive/metastore/DummyCustomRDBMS.java    |    2 +-
 .../metastore/txn/retry/TestSqlRetryHandler.java   |   43 +
 .../org/apache/hive/streaming/TestStreaming.java   |   12 +-
 153 files changed, 8817 insertions(+), 6947 deletions(-)

diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergCompactionService.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergCompactionService.java
index e0ff9782c69..5c985a55e57 100644
--- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergCompactionService.java
+++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergCompactionService.java
@@ -19,7 +19,7 @@
 package org.apache.iceberg.mr.hive.compaction;
 
 import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.ql.txn.compactor.CompactorContext;
 import org.apache.hadoop.hive.ql.txn.compactor.CompactorPipeline;
 import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTables.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTables.java
index 93fd5f0cbc9..63ba2aad188 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTables.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTables.java
@@ -40,7 +40,7 @@ import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.messaging.json.gzip.GzipJSONMessageEncoder;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.apache.hadoop.hive.metastore.InjectableBehaviourObjectStore;
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java
index fd6e111df0a..1083bef80e9 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java
@@ -41,7 +41,7 @@ import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
 import org.apache.hadoop.hive.metastore.api.ShowCompactResponseElement;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil;
@@ -52,9 +52,6 @@ import org.apache.hadoop.hive.ql.io.orc.OrcFile;
 import org.apache.hadoop.hive.ql.io.orc.Reader;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
-import org.apache.hadoop.hive.ql.txn.compactor.Cleaner;
-import org.apache.hadoop.hive.ql.txn.compactor.FSRemover;
-import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache;
 import org.apache.hadoop.hive.ql.txn.compactor.handler.TaskHandler;
 import org.apache.hadoop.hive.ql.txn.compactor.handler.TaskHandlerFactory;
 import org.apache.hive.streaming.HiveStreamingConnection;
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCrudCompactorOnTez.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCrudCompactorOnTez.java
index 295bbdd09f7..1342b25916b 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCrudCompactorOnTez.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCrudCompactorOnTez.java
@@ -50,7 +50,7 @@ import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
 import org.apache.hadoop.hive.metastore.api.ShowCompactResponseElement;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.apache.hadoop.hive.ql.Driver;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
index febe8a2b1e9..4c63a74d853 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
@@ -92,7 +92,7 @@ import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.TxnType;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.txn.CompactionState;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionState;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.ErrorMsg;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorContext.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorContext.java
index dece9ed939c..7d16d64db50 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorContext.java
@@ -22,7 +22,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.ql.io.AcidDirectory;
 
 /**
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorFactory.java
index 2184bcd5072..84ce492ae61 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorFactory.java
@@ -21,7 +21,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.api.CompactionType;
 import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
@@ -70,7 +70,7 @@ public final class CompactorFactory {
    * @param compactionInfo provides insight about the type of compaction, must be not null.
    * @return {@link QueryCompactor} or null.
    */
-  public CompactorPipeline getCompactorPipeline(Table table, HiveConf configuration, CompactionInfo compactionInfo,IMetaStoreClient msc)
+  public CompactorPipeline getCompactorPipeline(Table table, HiveConf configuration, CompactionInfo compactionInfo, IMetaStoreClient msc)
       throws HiveException {
     if (AcidUtils.isFullAcidTable(table.getParameters())) {
       if (!"tez".equalsIgnoreCase(HiveConf.getVar(configuration, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE)) ||
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java
index dda8b219f90..09296293f0a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 
 import org.apache.thrift.TException;
 import org.slf4j.Logger;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorUtil.java
index 6daf665dd36..c8d86c7ea27 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorUtil.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorUtil.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.utils.StringableMap;
 import org.apache.hadoop.hive.ql.io.AcidDirectory;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
index 6396aa8c30d..58cb478bbe8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
@@ -28,7 +28,7 @@ import org.apache.hadoop.hive.metastore.api.*;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
 import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/InitiatorBase.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/InitiatorBase.java
index 369dc69fc30..8f632dbd398 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/InitiatorBase.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/InitiatorBase.java
@@ -39,7 +39,7 @@ import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.metrics.AcidMetricService;
 import org.apache.hadoop.hive.metastore.metrics.Metrics;
 import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.apache.hadoop.hive.ql.io.AcidDirectory;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MRCompactor.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MRCompactor.java
index f9044fa3d29..5bf1c0bffb1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MRCompactor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MRCompactor.java
@@ -47,7 +47,7 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.metastore.utils.StringableMap;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter;
 import org.apache.hadoop.hive.ql.io.AcidInputFormat;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MergeCompactor.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MergeCompactor.java
index 295ebe56417..7e5a3608e46 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MergeCompactor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MergeCompactor.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.hive.common.ValidWriteIdList;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.ql.io.AcidDirectory;
 import org.apache.hadoop.hive.ql.io.AcidOutputFormat;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetaStoreCompactorThread.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetaStoreCompactorThread.java
index bfd59c6fdc7..0878330cc3e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetaStoreCompactorThread.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetaStoreCompactorThread.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.metrics.Metrics;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.apache.hadoop.hive.ql.exec.repl.util.ReplUtils;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/QueryCompactor.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/QueryCompactor.java
index ee0d9c9a7d7..384a386dfb6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/QueryCompactor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/QueryCompactor.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 import org.apache.hadoop.hive.metastore.api.CompactionType;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.metastore.utils.StringableMap;
 import org.apache.hadoop.hive.ql.DriverUtils;
 import org.apache.hadoop.hive.ql.io.AcidDirectory;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/RemoteCompactorThread.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/RemoteCompactorThread.java
index 819e39579c5..f95834ac23d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/RemoteCompactorThread.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/RemoteCompactorThread.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.ql.exec.repl.util.ReplUtils;
 import org.apache.thrift.TException;
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/RemoteCompactorUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/RemoteCompactorUtil.java
index 676a11891a0..c7f20a93605 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/RemoteCompactorUtil.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/RemoteCompactorUtil.java
@@ -23,7 +23,7 @@ import org.apache.hadoop.hive.metastore.api.GetPartitionsByNamesRequest;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/StatsUpdater.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/StatsUpdater.java
index 50d04f8b1b2..ce0998158dc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/StatsUpdater.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/StatsUpdater.java
@@ -21,8 +21,7 @@ import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.Warehouse;
-import org.apache.hadoop.hive.metastore.api.CompactionType;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.ql.DriverUtils;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.stats.StatsUtils;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
index 8f75c06d54b..b39236ef8b9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
@@ -28,6 +28,10 @@ import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.txn.TxnErrorMsg;
+import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus;
+import org.apache.hadoop.hive.ql.io.AcidDirectory;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.log.PerfLogger;
 import org.apache.hadoop.hive.ql.txn.compactor.service.CompactionService;
 import org.apache.hadoop.hive.ql.txn.compactor.service.CompactionExecutorFactory;
@@ -36,7 +40,9 @@ import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
 import java.io.IOException;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java
index acd4519d0fe..2314ce4d2e4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
 import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaner.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaner.java
index c2be268b6f9..721e3ea2266 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaner.java
@@ -38,7 +38,7 @@ import org.apache.hadoop.hive.metastore.api.UnlockRequest;
 import org.apache.hadoop.hive.metastore.metrics.Metrics;
 import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
 import org.apache.hadoop.hive.metastore.metrics.PerfLogger;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java
index d530b7efa34..f4d0a5adc15 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.metrics.AcidMetricService;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.ql.io.AcidDirectory;
@@ -45,6 +45,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/AcidCompactionService.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/AcidCompactionService.java
index 4fb6775eda5..b4143f9dd18 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/AcidCompactionService.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/AcidCompactionService.java
@@ -36,10 +36,10 @@ import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.TxnType;
 import org.apache.hadoop.hive.metastore.metrics.AcidMetricService;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
 import org.apache.hadoop.hive.metastore.txn.TxnErrorMsg;
-import org.apache.hadoop.hive.metastore.txn.TxnStatus;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus;
 import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.io.AcidDirectory;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/CompactionService.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/CompactionService.java
index c896ffc77cf..814ed9cd046 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/CompactionService.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/service/CompactionService.java
@@ -21,7 +21,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.ql.txn.compactor.CompactorFactory;
 import org.apache.thrift.TException;
 import org.slf4j.Logger;
diff --git a/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java b/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java
index 38c44f7722f..d26f3774af7 100644
--- a/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java
+++ b/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hive.metastore.api.ShowCompactRequest;
 import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
 import org.apache.hadoop.hive.metastore.api.ShowCompactResponseElement;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil;
 import org.apache.hive.common.util.HiveVersionInfo;
 import org.junit.After;
@@ -463,7 +464,7 @@ public class TestCompactionTxnHandler {
       txnHandler.markFailed(ci);
       fail("The first call to markFailed() must have failed as this call did "
           + "not throw the expected exception");
-    } catch (MetaException e) {
+    } catch (IllegalStateException e) {
       // This is expected
       assertTrue(e.getMessage().contains("No record with CQ_ID="));
     }
@@ -753,7 +754,7 @@ public class TestCompactionTxnHandler {
     LockResponse res = txnHandler.lock(req);
     assertTrue(res.getState() == LockState.ACQUIRED);
     txnHandler.commitTxn(new CommitTxnRequest(txnid));
-    assertEquals(0, txnHandler.numLocksInLockTable());
+    assertEquals(0, txnHandler.getNumLocks());
 
     Set<CompactionInfo> potentials = txnHandler.findPotentialCompactions(100, -1L);
     assertEquals(2, potentials.size());
@@ -1014,7 +1015,7 @@ public class TestCompactionTxnHandler {
     LockResponse res = txnHandler.lock(req);
     assertSame(res.getState(), LockState.ACQUIRED);
     txnHandler.commitTxn(new CommitTxnRequest(txnId));
-    assertEquals(0, txnHandler.numLocksInLockTable());
+    assertEquals(0, txnHandler.getNumLocks());
 
     Set<CompactionInfo> potentials = txnHandler.findPotentialCompactions(100, -1L);
     assertEquals(1, potentials.size());
diff --git a/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java b/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
index ee810c2117e..4f9f1d8e2fc 100644
--- a/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
+++ b/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
@@ -95,7 +95,6 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.Collectors;
 import java.util.stream.LongStream;
@@ -107,7 +106,6 @@ import static junit.framework.Assert.assertTrue;
 import static junit.framework.Assert.fail;
 import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.TABLE_IS_TRANSACTIONAL;
 import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.TABLE_TRANSACTIONAL_PROPERTIES;
-import static org.apache.hadoop.hive.metastore.utils.LockTypeUtil.getEncoding;
 
 /**
  * Tests for TxnHandler.
@@ -1033,7 +1031,7 @@ public class TestTxnHandler {
     res = txnHandler.checkLock(new CheckLockRequest(lockid));
     assertTrue(res.getState() == LockState.ACQUIRED);
     txnHandler.unlock(new UnlockRequest(lockid));
-    assertEquals(0, txnHandler.numLocksInLockTable());
+    assertEquals(0, txnHandler.getNumLocks());
   }
 
   @Test
@@ -1088,7 +1086,7 @@ public class TestTxnHandler {
     LockResponse res = txnHandler.lock(req);
     assertTrue(res.getState() == LockState.ACQUIRED);
     txnHandler.commitTxn(new CommitTxnRequest(txnid));
-    assertEquals(0, txnHandler.numLocksInLockTable());
+    assertEquals(0, txnHandler.getNumLocks());
   }
 
   @Test
@@ -1104,7 +1102,7 @@ public class TestTxnHandler {
     LockResponse res = txnHandler.lock(req);
     assertTrue(res.getState() == LockState.ACQUIRED);
     txnHandler.abortTxn(new AbortTxnRequest(txnid));
-    assertEquals(0, txnHandler.numLocksInLockTable());
+    assertEquals(0, txnHandler.getNumLocks());
   }
 
   @Test
@@ -1462,113 +1460,6 @@ public class TestTxnHandler {
     for (int i = 0; i < saw.length; i++) assertTrue("Didn't see lock id " + i, saw[i]);
   }
 
-  @Test
-  @Ignore("Wedges Derby")
-  public void deadlockDetected() throws Exception {
-    LOG.debug("Starting deadlock test");
-
-    if (txnHandler instanceof TxnHandler) {
-      final TxnHandler tHndlr = (TxnHandler)txnHandler;
-      Connection conn = tHndlr.getDbConn(Connection.TRANSACTION_SERIALIZABLE);
-      try {
-        Statement stmt = conn.createStatement();
-        long now = tHndlr.getDbTime(conn);
-        stmt.executeUpdate("INSERT INTO \"TXNS\" (\"TXN_ID\", \"TXN_STATE\", \"TXN_STARTED\", \"TXN_LAST_HEARTBEAT\", " +
-                "txn_user, txn_host) values (1, 'o', " + now + ", " + now + ", 'shagy', " +
-                "'scooby.com')");
-        stmt.executeUpdate("INSERT INTO \"HIVE_LOCKS\" (\"HL_LOCK_EXT_ID\", \"HL_LOCK_INT_ID\", \"HL_TXNID\", " +
-                "\"HL_DB\", \"HL_TABLE\", \"HL_PARTITION\", \"HL_LOCK_STATE\", \"HL_LOCK_TYPE\", \"HL_LAST_HEARTBEAT\", " +
-                "\"HL_USER\", \"HL_HOST\") VALUES (1, 1, 1, 'MYDB', 'MYTABLE', 'MYPARTITION', '" +
-                tHndlr.LOCK_WAITING + "', '" + getEncoding(LockType.EXCLUSIVE) + "', " + now + ", 'fred', " +
-                "'scooby.com')");
-        conn.commit();
-      } finally {
-        tHndlr.closeDbConn(conn);
-      }
-
-      final AtomicBoolean sawDeadlock = new AtomicBoolean();
-
-      final Connection conn1 = tHndlr.getDbConn(Connection.TRANSACTION_SERIALIZABLE);
-      final Connection conn2 = tHndlr.getDbConn(Connection.TRANSACTION_SERIALIZABLE);
-      try {
-
-        for (int i = 0; i < 5; i++) {
-          Thread t1 = new Thread() {
-            @Override
-            public void run() {
-              try {
-                try {
-                  updateTxns(conn1);
-                  updateLocks(conn1);
-                  Thread.sleep(1000);
-                  conn1.commit();
-                  LOG.debug("no exception, no deadlock");
-                } catch (SQLException e) {
-                  try {
-                    tHndlr.checkRetryable(e, "thread t1");
-                    LOG.debug("Got an exception, but not a deadlock, SQLState is " +
-                        e.getSQLState() + " class of exception is " + e.getClass().getName() +
-                        " msg is <" + e.getMessage() + ">");
-                  } catch (TxnHandler.RetryException de) {
-                    LOG.debug("Forced a deadlock, SQLState is " + e.getSQLState() + " class of " +
-                        "exception is " + e.getClass().getName() + " msg is <" + e
-                        .getMessage() + ">");
-                    sawDeadlock.set(true);
-                  }
-                }
-                conn1.rollback();
-              } catch (Exception e) {
-                throw new RuntimeException(e);
-              }
-            }
-          };
-
-          Thread t2 = new Thread() {
-            @Override
-            public void run() {
-              try {
-                try {
-                  updateLocks(conn2);
-                  updateTxns(conn2);
-                  Thread.sleep(1000);
-                  conn2.commit();
-                  LOG.debug("no exception, no deadlock");
-                } catch (SQLException e) {
-                  try {
-                    tHndlr.checkRetryable(e, "thread t2");
-                    LOG.debug("Got an exception, but not a deadlock, SQLState is " +
-                        e.getSQLState() + " class of exception is " + e.getClass().getName() +
-                        " msg is <" + e.getMessage() + ">");
-                  } catch (TxnHandler.RetryException de) {
-                    LOG.debug("Forced a deadlock, SQLState is " + e.getSQLState() + " class of " +
-                        "exception is " + e.getClass().getName() + " msg is <" + e
-                        .getMessage() + ">");
-                    sawDeadlock.set(true);
-                  }
-                }
-                conn2.rollback();
-              } catch (Exception e) {
-                throw new RuntimeException(e);
-              }
-            }
-          };
-
-          t1.start();
-          t2.start();
-          t1.join();
-          t2.join();
-          if (sawDeadlock.get()) break;
-        }
-        assertTrue(sawDeadlock.get());
-      } finally {
-        conn1.rollback();
-        tHndlr.closeDbConn(conn1);
-        conn2.rollback();
-        tHndlr.closeDbConn(conn2);
-      }
-    }
-  }
-
   /**
    * This cannnot be run against Derby (thus in UT) but it can run against MySQL.
    * 1. add to metastore/pom.xml
@@ -1662,20 +1553,6 @@ public class TestTxnHandler {
     }
   }
 
-  @Test
-  public void testRetryableRegex() throws Exception {
-    SQLException sqlException = new SQLException("ORA-08177: can't serialize access for this transaction", "72000");
-    // Note that we have 3 regex'es below
-    conf.setVar(HiveConf.ConfVars.HIVE_TXN_RETRYABLE_SQLEX_REGEX, "^Deadlock detected, roll back,.*08177.*,.*08178.*");
-    boolean result = TxnHandler.isRetryable(conf, sqlException);
-    Assert.assertTrue("regex should be retryable", result);
-
-    sqlException = new SQLException("This error message, has comma in it");
-    conf.setVar(HiveConf.ConfVars.HIVE_TXN_RETRYABLE_SQLEX_REGEX, ".*comma.*");
-    result = TxnHandler.isRetryable(conf, sqlException);
-    Assert.assertTrue("regex should be retryable", result);
-  }
-
   private List<Long> replOpenTxnForTest(long startId, int numTxn, String replPolicy)
           throws Exception {
     conf.setIntVar(HiveConf.ConfVars.HIVE_TXN_MAX_OPEN_BATCH, numTxn);
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands3.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands3.java
index 74107802891..c5a2639bdaf 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands3.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands3.java
@@ -28,7 +28,7 @@ import org.apache.hadoop.hive.metastore.api.GetOpenTxnsResponse;
 import org.apache.hadoop.hive.metastore.api.ShowCompactRequest;
 import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java b/ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java
index 03c4d4f23da..3a9b0cb754c 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java
@@ -134,8 +134,8 @@ public abstract class TxnCommandsBaseForTests {
     MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.COMPACTOR_CLEANER_ON, true);
       
     TestTxnDbUtil.setConfValues(hiveConf);
-    txnHandler = TxnUtils.getTxnStore(hiveConf);
     TestTxnDbUtil.prepDb(hiveConf);
+    txnHandler = TxnUtils.getTxnStore(hiveConf);
     File f = new File(getWarehouseDir());
     if (f.exists()) {
       FileUtil.fullyDelete(f);
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java
index ba90d8549d1..38484534b77 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java
@@ -61,7 +61,7 @@ import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.metrics.AcidMetricService;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils;
 import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java
index 7eb573dedac..d435a07c2df 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java
@@ -39,7 +39,7 @@ import org.apache.hadoop.hive.metastore.api.ShowCompactResponseElement;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.TxnType;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCompactionMetrics.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCompactionMetrics.java
index 24c625139ec..b33f8917c74 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCompactionMetrics.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCompactionMetrics.java
@@ -51,7 +51,8 @@ import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.metrics.AcidMetricService;
 import org.apache.hadoop.hive.metastore.metrics.Metrics;
 import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnHandler;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.metastore.txn.ThrowingTxnHandler;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
@@ -665,6 +666,7 @@ public class TestCompactionMetrics  extends CompactorTest {
     Table t = newTable(dbName, tblName, false);
 
     MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.TXN_USE_MIN_HISTORY_LEVEL, false);
+    TxnHandler.ConfVars.setUseMinHistoryLevel(false);
     long start = System.currentTimeMillis();
     burnThroughTransactions(t.getDbName(), t.getTableName(), 24, new HashSet<>(Arrays.asList(22L, 23L, 24L)), null);
     openTxn(TxnType.REPL_CREATED);
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestMRCompactorJobQueueConfiguration.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestMRCompactorJobQueueConfiguration.java
index 8a709b5a168..e775b456715 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestMRCompactorJobQueueConfiguration.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestMRCompactorJobQueueConfiguration.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.hive.metastore.api.CompactionType;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.metastore.utils.StringableMap;
 import org.apache.hadoop.mapred.JobConf;
 import org.junit.jupiter.params.ParameterizedTest;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java
index e0a2a1f5ed3..8f6814d4890 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
 import org.apache.hadoop.hive.metastore.api.ShowCompactResponseElement;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
index a37501a8d11..ffa3d6a06c5 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
@@ -1628,7 +1628,7 @@ public class MetastoreConf {
         "Time before an open transaction operation should persist, otherwise it is considered invalid and rolled back"),
     TXN_USE_MIN_HISTORY_LEVEL("metastore.txn.use.minhistorylevel", "hive.txn.use.minhistorylevel", true,
         "Set this to false, for the TxnHandler and Cleaner to not use MIN_HISTORY_LEVEL table and take advantage of openTxn optimisation.\n"
-            + "If the table is dropped HMS will switch this flag to false."),
+            + "If the table is dropped HMS will switch this flag to false, any other value changes need a restart to take effect."),
     TXN_USE_MIN_HISTORY_WRITE_ID("metastore.txn.use.minhistorywriteid", "hive.txn.use.minhistorywriteid", false,
       "Set this to true, to avoid global minOpenTxn check in Cleaner.\n"
             + "If the table is dropped HMS will switch this flag to false."),
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnQueries.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnQueries.java
deleted file mode 100644
index f3f0e5d939b..00000000000
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnQueries.java
+++ /dev/null
@@ -1,88 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * 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.metastore.txn;
-
-public class TxnQueries {
-    public static final String SHOW_COMPACTION_ORDERBY_CLAUSE =
-            " ORDER BY CASE " +
-                    "   WHEN \"CC_END\" > \"CC_START\" and \"CC_END\" > \"CC_COMMIT_TIME\" " +
-                    "     THEN \"CC_END\" " +
-                    "   WHEN \"CC_START\" > \"CC_COMMIT_TIME\" " +
-                    "     THEN \"CC_START\" " +
-                    "   ELSE \"CC_COMMIT_TIME\" " +
-                    " END desc ," +
-                    " \"CC_ENQUEUE_TIME\" asc";
-
-    public static final String SHOW_COMPACTION_QUERY =
-            "SELECT XX.* FROM ( SELECT " +
-                    "  \"CQ_DATABASE\" AS \"CC_DATABASE\", \"CQ_TABLE\" AS \"CC_TABLE\", \"CQ_PARTITION\" AS \"CC_PARTITION\", " +
-                    "  \"CQ_STATE\" AS \"CC_STATE\", \"CQ_TYPE\" AS \"CC_TYPE\", \"CQ_WORKER_ID\" AS \"CC_WORKER_ID\", " +
-                    "  \"CQ_START\" AS \"CC_START\", -1 \"CC_END\", \"CQ_RUN_AS\" AS \"CC_RUN_AS\", " +
-                    "  \"CQ_HADOOP_JOB_ID\" AS \"CC_HADOOP_JOB_ID\", \"CQ_ID\" AS \"CC_ID\", \"CQ_ERROR_MESSAGE\" AS \"CC_ERROR_MESSAGE\", " +
-                    "  \"CQ_ENQUEUE_TIME\" AS \"CC_ENQUEUE_TIME\", \"CQ_WORKER_VERSION\" AS \"CC_WORKER_VERSION\", " +
-                    "  \"CQ_INITIATOR_ID\" AS \"CC_INITIATOR_ID\", \"CQ_INITIATOR_VERSION\" AS \"CC_INITIATOR_VERSION\", " +
-                    "  \"CQ_CLEANER_START\" AS \"CC_CLEANER_START\", \"CQ_POOL_NAME\" AS \"CC_POOL_NAME\", \"CQ_TXN_ID\" AS \"CC_TXN_ID\", " +
-                    "  \"CQ_NEXT_TXN_ID\" AS \"CC_NEXT_TXN_ID\", \"CQ_COMMIT_TIME\" AS \"CC_COMMIT_TIME\", " +
-                    "  \"CQ_HIGHEST_WRITE_ID\" AS \"CC_HIGHEST_WRITE_ID\" " +
-                    "FROM " +
-                    "  \"COMPACTION_QUEUE\" " +
-                    "UNION ALL " +
-                    "SELECT " +
-                    "  \"CC_DATABASE\" , \"CC_TABLE\", \"CC_PARTITION\", \"CC_STATE\", \"CC_TYPE\", \"CC_WORKER_ID\", " +
-                    "  \"CC_START\", \"CC_END\", \"CC_RUN_AS\", \"CC_HADOOP_JOB_ID\", \"CC_ID\", \"CC_ERROR_MESSAGE\", " +
-                    "  \"CC_ENQUEUE_TIME\", \"CC_WORKER_VERSION\", \"CC_INITIATOR_ID\", \"CC_INITIATOR_VERSION\", " +
-                    "   -1 , \"CC_POOL_NAME\", \"CC_TXN_ID\", \"CC_NEXT_TXN_ID\", \"CC_COMMIT_TIME\", " +
-                    "  \"CC_HIGHEST_WRITE_ID\"" +
-                    "FROM " +
-                    "  \"COMPLETED_COMPACTIONS\" ) XX ";
-
-
-    public static final String SELECT_COMPACTION_QUEUE_BY_COMPID =
-            "SELECT XX.* FROM ( SELECT " +
-                    "   \"CQ_ID\" AS \"CC_ID\", \"CQ_DATABASE\" AS \"CC_DATABASE\", \"CQ_TABLE\" AS \"CC_TABLE\", \"CQ_PARTITION\" AS \"CC_PARTITION\", " +
-                    "   \"CQ_STATE\" AS \"CC_STATE\", \"CQ_TYPE\" AS \"CC_TYPE\", \"CQ_TBLPROPERTIES\" AS \"CC_TBLPROPERTIES\", \"CQ_WORKER_ID\" AS \"CC_WORKER_ID\", " +
-                    "   \"CQ_START\" AS \"CC_START\", \"CQ_RUN_AS\" AS \"CC_RUN_AS\", \"CQ_HIGHEST_WRITE_ID\" AS \"CC_HIGHEST_WRITE_ID\", \"CQ_META_INFO\" AS \"CC_META_INFO\"," +
-                    "   \"CQ_HADOOP_JOB_ID\" AS \"CC_HADOOP_JOB_ID\", \"CQ_ERROR_MESSAGE\" AS \"CC_ERROR_MESSAGE\",  \"CQ_ENQUEUE_TIME\" AS \"CC_ENQUEUE_TIME\"," +
-                    "   \"CQ_WORKER_VERSION\" AS \"CC_WORKER_VERSION\", \"CQ_INITIATOR_ID\" AS \"CC_INITIATOR_ID\", \"CQ_INITIATOR_VERSION\" AS \"CC_INITIATOR_VERSION\", " +
-                    "   \"CQ_RETRY_RETENTION\" AS \"CC_RETRY_RETENTION\", \"CQ_NEXT_TXN_ID\" AS \"CC_NEXT_TXN_ID\", \"CQ_TXN_ID\" AS \"CC_TXN_ID\", " +
-                    "   \"CQ_COMMIT_TIME\" AS \"CC_COMMIT_TIME\", \"CQ_POOL_NAME\" AS \"CC_POOL_NAME\",  " +
-                    "   \"CQ_NUMBER_OF_BUCKETS\" AS \"CC_NUMBER_OF_BUCKETS\", \"CQ_ORDER_BY\" AS \"CC_ORDER_BY\" " +
-                    "   FROM " +
-                    "   \"COMPACTION_QUEUE\" " +
-                    "   UNION ALL " +
-                    "   SELECT " +
-                    "   \"CC_ID\", \"CC_DATABASE\", \"CC_TABLE\", \"CC_PARTITION\", \"CC_STATE\", \"CC_TYPE\", " +
-                    "   \"CC_TBLPROPERTIES\", \"CC_WORKER_ID\", \"CC_START\", \"CC_RUN_AS\", " +
-                    "   \"CC_HIGHEST_WRITE_ID\", \"CC_META_INFO\", \"CC_HADOOP_JOB_ID\", \"CC_ERROR_MESSAGE\", " +
-                    "   \"CC_ENQUEUE_TIME\", \"CC_WORKER_VERSION\", \"CC_INITIATOR_ID\", \"CC_INITIATOR_VERSION\", " +
-                    "    -1 , \"CC_NEXT_TXN_ID\", \"CC_TXN_ID\", \"CC_NEXT_TXN_ID\", \"CC_POOL_NAME\", " +
-                    "   \"CC_NUMBER_OF_BUCKETS\", \"CC_ORDER_BY\" " +
-                    "   FROM   " +
-                    "   \"COMPLETED_COMPACTIONS\") XX ";
-
-
-    public static final String INSERT_INTO_COMPLETED_COMPACTION =
-            "INSERT INTO \"COMPLETED_COMPACTIONS\" " +
-                    "   (\"CC_ID\", \"CC_DATABASE\", \"CC_TABLE\", \"CC_PARTITION\", \"CC_STATE\", \"CC_TYPE\", " +
-                    "   \"CC_TBLPROPERTIES\", \"CC_WORKER_ID\", \"CC_START\", \"CC_END\", \"CC_RUN_AS\", " +
-                    "   \"CC_HIGHEST_WRITE_ID\", \"CC_META_INFO\", \"CC_HADOOP_JOB_ID\", \"CC_ERROR_MESSAGE\", " +
-                    "   \"CC_ENQUEUE_TIME\", \"CC_WORKER_VERSION\", \"CC_INITIATOR_ID\", \"CC_INITIATOR_VERSION\"," +
-                    "   \"CC_NEXT_TXN_ID\", \"CC_TXN_ID\", \"CC_COMMIT_TIME\", \"CC_POOL_NAME\", \"CC_NUMBER_OF_BUCKETS\", " +
-                    "   \"CC_ORDER_BY\") " +
-                    "   VALUES(?,?,?,?,?, ?,?,?,?,?, ?,?,?,?,?,?,?,?,?,?,?,?,?,?,?)";
-}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java
index afee0797420..6e04bf0d6f7 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DatabaseProduct.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -205,10 +206,11 @@ public class DatabaseProduct implements Configurable {
 
   /**
    * Is the given exception a table not found exception
-   * @param e Exception
+   * @param t Exception
    * @return
    */
-  public boolean isTableNotExistsError(SQLException e) {
+  public boolean isTableNotExistsError(Throwable t) {
+    SQLException e = TxnUtils.getSqlException(t);    
     return (isPOSTGRES() && "42P01".equalsIgnoreCase(e.getSQLState()))
         || (isMYSQL() && "42S02".equalsIgnoreCase(e.getSQLState()))
         || (isORACLE() && "42000".equalsIgnoreCase(e.getSQLState()) && e.getMessage().contains("ORA-00942"))
@@ -558,41 +560,42 @@ public class DatabaseProduct implements Configurable {
     }
   }
 
-  public boolean isDuplicateKeyError(SQLException ex) {
+  public boolean isDuplicateKeyError(Throwable t) {
+    SQLException sqlEx = TxnUtils.getSqlException(t); 
     switch (dbType) {
     case DERBY:
     case CUSTOM: // ANSI SQL
-      if("23505".equals(ex.getSQLState())) {
+      if("23505".equals(sqlEx.getSQLState())) {
         return true;
       }
       break;
     case MYSQL:
       //https://dev.mysql.com/doc/refman/5.5/en/error-messages-server.html
-      if((ex.getErrorCode() == 1022 || ex.getErrorCode() == 1062 || ex.getErrorCode() == 1586)
-        && "23000".equals(ex.getSQLState())) {
+      if((sqlEx.getErrorCode() == 1022 || sqlEx.getErrorCode() == 1062 || sqlEx.getErrorCode() == 1586)
+        && "23000".equals(sqlEx.getSQLState())) {
         return true;
       }
       break;
     case SQLSERVER:
       //2627 is unique constaint violation incl PK, 2601 - unique key
-      if ((ex.getErrorCode() == 2627 || ex.getErrorCode() == 2601) && "23000".equals(ex.getSQLState())) {
+      if ((sqlEx.getErrorCode() == 2627 || sqlEx.getErrorCode() == 2601) && "23000".equals(sqlEx.getSQLState())) {
         return true;
       }
       break;
     case ORACLE:
-      if(ex.getErrorCode() == 1 && "23000".equals(ex.getSQLState())) {
+      if(sqlEx.getErrorCode() == 1 && "23000".equals(sqlEx.getSQLState())) {
         return true;
       }
       break;
     case POSTGRES:
       //http://www.postgresql.org/docs/8.1/static/errcodes-appendix.html
-      if("23505".equals(ex.getSQLState())) {
+      if("23505".equals(sqlEx.getSQLState())) {
         return true;
       }
       break;
     default:
-      String msg = ex.getMessage() +
-                " (SQLState=" + ex.getSQLState() + ", ErrorCode=" + ex.getErrorCode() + ")";
+      String msg = sqlEx.getMessage() +
+                " (SQLState=" + sqlEx.getSQLState() + ", ErrorCode=" + sqlEx.getErrorCode() + ")";
       throw new IllegalArgumentException("Unexpected DB type: " + dbType + "; " + msg);
   }
   return false;
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java
index 1399553e40b..a9dbb03d989 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java
@@ -62,6 +62,7 @@ import org.apache.hadoop.hive.metastore.properties.PropertyManager;
 import org.apache.hadoop.hive.metastore.properties.PropertyMap;
 import org.apache.hadoop.hive.metastore.properties.PropertyStore;
 import org.apache.hadoop.hive.metastore.txn.*;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 import org.apache.hadoop.hive.metastore.utils.FileUtils;
 import org.apache.hadoop.hive.metastore.utils.FilterUtils;
 import org.apache.hadoop.hive.metastore.utils.HdfsUtils;
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/CommitCompactionEvent.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/CommitCompactionEvent.java
index f728f9a0b11..265a43d6756 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/CommitCompactionEvent.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/CommitCompactionEvent.java
@@ -22,8 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hive.metastore.IHMSHandler;
 import org.apache.hadoop.hive.metastore.api.CompactionType;
-import org.apache.hadoop.hive.metastore.api.TxnType;
-import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
 
 /**
  * CommitCompactionEvent
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/metrics/AcidMetricLogger.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/metrics/AcidMetricLogger.java
index 35450a4fe60..4fbfced8328 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/metrics/AcidMetricLogger.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/metrics/AcidMetricLogger.java
@@ -23,8 +23,8 @@ import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.ShowCompactRequest;
 import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
-import org.apache.hadoop.hive.metastore.txn.CompactionMetricsData;
-import org.apache.hadoop.hive.metastore.txn.MetricsInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionMetricsData;
+import org.apache.hadoop.hive.metastore.txn.entities.MetricsInfo;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.slf4j.Logger;
@@ -33,9 +33,9 @@ import org.slf4j.LoggerFactory;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
-import static org.apache.hadoop.hive.metastore.txn.CompactionMetricsData.MetricType.NUM_DELTAS;
-import static org.apache.hadoop.hive.metastore.txn.CompactionMetricsData.MetricType.NUM_OBSOLETE_DELTAS;
-import static org.apache.hadoop.hive.metastore.txn.CompactionMetricsData.MetricType.NUM_SMALL_DELTAS;
+import static org.apache.hadoop.hive.metastore.txn.entities.CompactionMetricsData.MetricType.NUM_DELTAS;
+import static org.apache.hadoop.hive.metastore.txn.entities.CompactionMetricsData.MetricType.NUM_OBSOLETE_DELTAS;
+import static org.apache.hadoop.hive.metastore.txn.entities.CompactionMetricsData.MetricType.NUM_SMALL_DELTAS;
 
 /**
  *
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/metrics/AcidMetricService.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/metrics/AcidMetricService.java
index cc29af053f2..d80f84219ee 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/metrics/AcidMetricService.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/metrics/AcidMetricService.java
@@ -32,8 +32,8 @@ import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.ShowCompactRequest;
 import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
-import org.apache.hadoop.hive.metastore.txn.CompactionMetricsData;
-import org.apache.hadoop.hive.metastore.txn.MetricsInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionMetricsData;
+import org.apache.hadoop.hive.metastore.txn.entities.MetricsInfo;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.apache.thrift.TException;
@@ -78,9 +78,9 @@ import static org.apache.hadoop.hive.metastore.metrics.MetricsConstants.OLDEST_O
 import static org.apache.hadoop.hive.metastore.metrics.MetricsConstants.OLDEST_OPEN_REPL_TXN_ID;
 import static org.apache.hadoop.hive.metastore.metrics.MetricsConstants.OLDEST_READY_FOR_CLEANING_AGE;
 import static org.apache.hadoop.hive.metastore.metrics.MetricsConstants.TABLES_WITH_X_ABORTED_TXNS;
-import static org.apache.hadoop.hive.metastore.txn.CompactionMetricsData.MetricType.NUM_DELTAS;
-import static org.apache.hadoop.hive.metastore.txn.CompactionMetricsData.MetricType.NUM_OBSOLETE_DELTAS;
-import static org.apache.hadoop.hive.metastore.txn.CompactionMetricsData.MetricType.NUM_SMALL_DELTAS;
+import static org.apache.hadoop.hive.metastore.txn.entities.CompactionMetricsData.MetricType.NUM_DELTAS;
+import static org.apache.hadoop.hive.metastore.txn.entities.CompactionMetricsData.MetricType.NUM_OBSOLETE_DELTAS;
+import static org.apache.hadoop.hive.metastore.txn.entities.CompactionMetricsData.MetricType.NUM_SMALL_DELTAS;
 
 /**
  * Collect and publish ACID and compaction related metrics.
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionMetricsDataConverter.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionMetricsDataConverter.java
index e579311d368..531ae67fcfa 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionMetricsDataConverter.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionMetricsDataConverter.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.metastore.txn;
 import org.apache.hadoop.hive.metastore.api.CompactionMetricsDataStruct;
 import org.apache.hadoop.hive.metastore.api.CompactionMetricsMetricType;
 import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionMetricsData;
 
 public class CompactionMetricsDataConverter {
 
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
index 6bde27a4759..d3b6091574a 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
@@ -17,45 +17,42 @@
  */
 package org.apache.hadoop.hive.metastore.txn;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.classification.RetrySemantics;
-import org.apache.hadoop.hive.metastore.MetaStoreListenerNotifier;
 import org.apache.hadoop.hive.metastore.api.CompactionType;
 import org.apache.hadoop.hive.metastore.api.FindNextCompactRequest;
 import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.api.TxnType;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
-import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
-import org.apache.hadoop.hive.metastore.datasource.DataSourceProvider;
-import org.apache.hadoop.hive.metastore.events.CommitCompactionEvent;
-import org.apache.hadoop.hive.metastore.messaging.EventMessage;
-import org.apache.hadoop.hive.metastore.txn.impl.CleanTxnToWriteIdTableFunction;
-import org.apache.hadoop.hive.metastore.txn.impl.FindPotentialCompactionsFunction;
-import org.apache.hadoop.hive.metastore.txn.impl.NextCompactionFunction;
-import org.apache.hadoop.hive.metastore.txn.impl.ReadyToCleanAbortHandler;
-import org.apache.hadoop.hive.metastore.txn.impl.CheckFailedCompactionsHandler;
-import org.apache.hadoop.hive.metastore.txn.impl.CompactionMetricsDataHandler;
-import org.apache.hadoop.hive.metastore.txn.impl.FindColumnsWithStatsHandler;
-import org.apache.hadoop.hive.metastore.txn.impl.GetCompactionInfoHandler;
-import org.apache.hadoop.hive.metastore.txn.impl.InsertCompactionInfoCommand;
-import org.apache.hadoop.hive.metastore.txn.impl.MarkCleanedFunction;
-import org.apache.hadoop.hive.metastore.txn.impl.PurgeCompactionHistoryFunction;
-import org.apache.hadoop.hive.metastore.txn.impl.ReadyToCleanHandler;
-import org.apache.hadoop.hive.metastore.txn.impl.RemoveCompactionMetricsDataCommand;
-import org.apache.hadoop.hive.metastore.txn.impl.RemoveDuplicateCompleteTxnComponentsCommand;
-import org.apache.hadoop.hive.metastore.txn.impl.TopCompactionMetricsDataPerTypeFunction;
-import org.apache.hadoop.hive.metastore.txn.impl.UpdateCompactionMetricsDataFunction;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionMetricsData;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionState;
+import org.apache.hadoop.hive.metastore.txn.entities.OperationType;
+import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus;
+import org.apache.hadoop.hive.metastore.txn.jdbc.commands.InsertCompactionInfoCommand;
+import org.apache.hadoop.hive.metastore.txn.jdbc.commands.RemoveCompactionMetricsDataCommand;
+import org.apache.hadoop.hive.metastore.txn.jdbc.commands.RemoveDuplicateCompleteTxnComponentsCommand;
+import org.apache.hadoop.hive.metastore.txn.jdbc.functions.CleanTxnToWriteIdTableFunction;
+import org.apache.hadoop.hive.metastore.txn.jdbc.functions.FindPotentialCompactionsFunction;
+import org.apache.hadoop.hive.metastore.txn.jdbc.functions.GenerateCompactionQueueIdFunction;
+import org.apache.hadoop.hive.metastore.txn.jdbc.functions.MarkCleanedFunction;
+import org.apache.hadoop.hive.metastore.txn.jdbc.functions.MinOpenTxnIdWaterMarkFunction;
+import org.apache.hadoop.hive.metastore.txn.jdbc.functions.NextCompactionFunction;
+import org.apache.hadoop.hive.metastore.txn.jdbc.functions.PurgeCompactionHistoryFunction;
+import org.apache.hadoop.hive.metastore.txn.jdbc.functions.TopCompactionMetricsDataPerTypeFunction;
+import org.apache.hadoop.hive.metastore.txn.jdbc.functions.UpdateCompactionMetricsDataFunction;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.CheckFailedCompactionsHandler;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.CompactionMetricsDataHandler;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.FindColumnsWithStatsHandler;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.GetCompactionInfoHandler;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.OpenTxnTimeoutLowBoundaryTxnIdHandler;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.ReadyToCleanAbortHandler;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.ReadyToCleanHandler;
 import org.apache.hadoop.hive.metastore.txn.jdbc.ParameterizedCommand;
-import org.apache.hadoop.hive.metastore.txn.retryhandling.SqlRetryHandler;
+import org.apache.hadoop.hive.metastore.txn.retry.SqlRetryHandler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.dao.DataAccessException;
-import org.springframework.jdbc.UncategorizedSQLException;
 import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
 
-import java.sql.Connection;
-import java.sql.SQLException;
-import java.sql.Statement;
 import java.sql.Types;
 import java.util.List;
 import java.util.Optional;
@@ -73,26 +70,9 @@ class CompactionTxnHandler extends TxnHandler {
   
   private static final Logger LOG = LoggerFactory.getLogger(CompactionTxnHandler.class.getName());
 
-  private static boolean initialized = false;
-  
   public CompactionTxnHandler() {
   }
 
-  @Override
-  public void setConf(Configuration conf) {
-    super.setConf(conf);
-    synchronized (CompactionTxnHandler.class) {
-      if (!initialized) {
-        int maxPoolSize = MetastoreConf.getIntVar(conf, ConfVars.HIVE_COMPACTOR_CONNECTION_POOLING_MAX_CONNECTIONS);
-        try (DataSourceProvider.DataSourceNameConfigurator configurator =
-                 new DataSourceProvider.DataSourceNameConfigurator(conf, "compactor")) {
-          jdbcResource.registerDataSource(POOL_COMPACTOR, setupJdbcConnectionPool(conf, maxPoolSize));
-          initialized = true;
-        }
-      }
-    }
-  }
-  
   /**
    * This will look through the completed_txn_components table and look for partitions or tables
    * that may be ready for compaction.  Also, look through txns and txn_components tables for
@@ -143,7 +123,7 @@ class CompactionTxnHandler extends TxnHandler {
     if (rqst == null) {
       throw new MetaException("FindNextCompactRequest is null");
     }
-    long poolTimeout = MetastoreConf.getTimeVar(conf, ConfVars.COMPACTOR_WORKER_POOL_TIMEOUT, TimeUnit.MILLISECONDS);
+    long poolTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.COMPACTOR_WORKER_POOL_TIMEOUT, TimeUnit.MILLISECONDS);
     return new NextCompactionFunction(rqst, getDbTime(), poolTimeout).execute(jdbcResource);
   }
 
@@ -172,7 +152,7 @@ class CompactionTxnHandler extends TxnHandler {
   @Override
   @RetrySemantics.ReadOnly
   public List<CompactionInfo> findReadyToClean(long minOpenTxnWaterMark, long retentionTime) throws MetaException {
-    return jdbcResource.execute(new ReadyToCleanHandler(conf, useMinHistoryWriteId, minOpenTxnWaterMark, retentionTime));
+    return jdbcResource.execute(new ReadyToCleanHandler(conf, minOpenTxnWaterMark, retentionTime));
   }
 
   @Override
@@ -227,7 +207,7 @@ class CompactionTxnHandler extends TxnHandler {
   @RetrySemantics.CannotRetry
   public void markCleaned(CompactionInfo info) throws MetaException {
     LOG.debug("Running markCleaned with CompactionInfo: {}", info);
-    new MarkCleanedFunction(info, conf).execute(jdbcResource);
+    new MarkCleanedFunction(info).execute(jdbcResource);
   }
   
   /**
@@ -237,7 +217,7 @@ class CompactionTxnHandler extends TxnHandler {
   @Override
   @RetrySemantics.SafeToRetry
   public void cleanTxnToWriteIdTable() throws MetaException {
-    new CleanTxnToWriteIdTableFunction(useMinHistoryLevel, findMinTxnIdSeenOpen()).execute(jdbcResource);
+    new CleanTxnToWriteIdTableFunction(findMinTxnIdSeenOpen()).execute(jdbcResource);
   }
 
   @Override
@@ -264,7 +244,7 @@ class CompactionTxnHandler extends TxnHandler {
      * 2. never deletes the maximum txnId even if it is before the TXN_OPENTXN_TIMEOUT window
      */
     try {
-      long lowWaterMark = getOpenTxnTimeoutLowBoundaryTxnId(jdbcResource.getConnection());
+      long lowWaterMark = jdbcResource.execute(new OpenTxnTimeoutLowBoundaryTxnIdHandler(openTxnTimeOutMillis));
       jdbcResource.execute(
           "DELETE FROM \"TXNS\" WHERE \"TXN_ID\" NOT IN (SELECT \"TC_TXNID\" FROM \"TXN_COMPONENTS\") " +
               "AND (\"TXN_STATE\" = :abortedState OR \"TXN_STATE\" = :committedState) AND \"TXN_ID\" < :txnId",
@@ -273,7 +253,7 @@ class CompactionTxnHandler extends TxnHandler {
               .addValue("abortedState", TxnStatus.ABORTED.getSqlConst(), Types.CHAR)
               .addValue("committedState", TxnStatus.COMMITTED.getSqlConst(), Types.CHAR),
           null);
-    } catch (SQLException e) {
+    } catch (DataAccessException e) {
       throw new MetaException("Unable to get the txn id: " + SqlRetryHandler.getMessage(e));
     }
   }
@@ -390,7 +370,7 @@ class CompactionTxnHandler extends TxnHandler {
   @Override
   @RetrySemantics.SafeToRetry
   public void purgeCompactionHistory() throws MetaException {
-    new PurgeCompactionHistoryFunction(conf).execute(jdbcResource);
+    new PurgeCompactionHistoryFunction().execute(jdbcResource);
   }
 
   /**
@@ -431,7 +411,7 @@ class CompactionTxnHandler extends TxnHandler {
     if (ciActual.id == 0) {
       //The failure occurred before we even made an entry in COMPACTION_QUEUE
       //generate ID so that we can make an entry in COMPLETED_COMPACTIONS
-      ciActual.id = generateCompactionQueueId();
+      ciActual.id = new GenerateCompactionQueueIdFunction().execute(jdbcResource);
       //this is not strictly accurate, but 'type' cannot be null.
       if (ciActual.type == null) {
         ciActual.type = CompactionType.MINOR;
@@ -480,7 +460,7 @@ class CompactionTxnHandler extends TxnHandler {
        * compactions for any resource.
        */
       try (TxnStore.MutexAPI.LockHandle ignored = getMutexAPI().acquireLock(MUTEX_KEY.CompactionScheduler.name())) {
-        long id = generateCompactionQueueId();
+        long id = new GenerateCompactionQueueIdFunction().execute(jdbcResource);
         int updCnt = jdbcResource.execute(
             "INSERT INTO \"COMPACTION_QUEUE\" (\"CQ_ID\", \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\", " +
                 " \"CQ_TYPE\", \"CQ_STATE\", \"CQ_RETRY_RETENTION\", \"CQ_ERROR_MESSAGE\", \"CQ_COMMIT_TIME\") " +
@@ -527,14 +507,10 @@ class CompactionTxnHandler extends TxnHandler {
   @Override
   @RetrySemantics.Idempotent
   public long findMinOpenTxnIdForCleaner() throws MetaException {
-    if (useMinHistoryWriteId) {
+    if (ConfVars.useMinHistoryWriteId()) {
       return Long.MAX_VALUE;
     }
-    try {
-      return getMinOpenTxnIdWaterMark(jdbcResource.getConnection());      
-    } catch (SQLException e) {
-      throw new UncategorizedSQLException(null, null, e);
-    }
+    return new MinOpenTxnIdWaterMarkFunction(openTxnTimeOutMillis).execute(jdbcResource);      
   }
 
   /**
@@ -546,7 +522,7 @@ class CompactionTxnHandler extends TxnHandler {
   @RetrySemantics.Idempotent
   @Deprecated
   public long findMinTxnIdSeenOpen() {
-    if (!useMinHistoryLevel || useMinHistoryWriteId) {
+    if (!ConfVars.useMinHistoryLevel() || ConfVars.useMinHistoryWriteId()) {
       return Long.MAX_VALUE;
     }
     try {
@@ -554,53 +530,20 @@ class CompactionTxnHandler extends TxnHandler {
           new MapSqlParameterSource(), Long.class);
       return minId == null ? Long.MAX_VALUE : minId;
     } catch (DataAccessException e) {
-      if (e.getCause() instanceof SQLException) {
-        if (dbProduct.isTableNotExistsError((SQLException) e.getCause())) {
-          useMinHistoryLevel = false;
-          return Long.MAX_VALUE;
-        }
+      if (dbProduct.isTableNotExistsError(e)) {
+        ConfVars.setUseMinHistoryLevel(false);
+        return Long.MAX_VALUE;
       }
       LOG.error("Unable to execute findMinTxnIdSeenOpen", e);
       throw e;
     }
   }
 
-  @Override
-  protected void updateWSCommitIdAndCleanUpMetadata(Statement stmt, long txnid, TxnType txnType, 
-      Long commitId, long tempId) throws SQLException, MetaException {
-    super.updateWSCommitIdAndCleanUpMetadata(stmt, txnid, txnType, commitId, tempId);
-    
-    if (txnType == TxnType.SOFT_DELETE || txnType == TxnType.COMPACTION) {
-      stmt.executeUpdate("UPDATE \"COMPACTION_QUEUE\" SET \"CQ_NEXT_TXN_ID\" = " + commitId + ", \"CQ_COMMIT_TIME\" = " +
-          getEpochFn(dbProduct) + " WHERE \"CQ_TXN_ID\" = " + txnid);
-    }
-  }
-
   @Override
   public Optional<CompactionInfo> getCompactionByTxnId(long txnId) throws MetaException {
     return Optional.ofNullable(jdbcResource.execute(new GetCompactionInfoHandler(txnId, true)));
   }
 
-  @Override
-  protected void createCommitNotificationEvent(Connection conn, long txnid, TxnType txnType)
-      throws MetaException, SQLException {
-    super.createCommitNotificationEvent(conn, txnid, txnType);
-    if (transactionalListeners != null) {
-      //Please note that TxnHandler and CompactionTxnHandler are using different DataSources (to have different pools).
-      //This call must use the same transaction and connection as TxnHandler.commitTxn(), therefore we are passing the 
-      //datasource wrapper comming from TxnHandler. Without this, the getCompactionByTxnId(long txnId) call would be
-      //executed using a different connection obtained from CompactionTxnHandler's own datasourceWrapper. 
-      CompactionInfo compactionInfo = getCompactionByTxnId(txnid).orElse(null);
-      if (compactionInfo != null) {
-        MetaStoreListenerNotifier
-            .notifyEventWithDirectSql(transactionalListeners, EventMessage.EventType.COMMIT_COMPACTION,
-                new CommitCompactionEvent(txnid, compactionInfo), conn, sqlGenerator);
-      } else {
-        LOG.warn("No compaction queue record found for Compaction type transaction commit. txnId:" + txnid);
-      }
-    }
-  }
-  
   @Override
   public boolean updateCompactionMetricsData(CompactionMetricsData data) throws MetaException {
     return new UpdateCompactionMetricsDataFunction(data).execute(jdbcResource);
@@ -609,7 +552,7 @@ class CompactionTxnHandler extends TxnHandler {
   @Override
   public List<CompactionMetricsData> getTopCompactionMetricsDataPerType(int limit)      
       throws MetaException {
-    return new TopCompactionMetricsDataPerTypeFunction(limit, sqlGenerator).execute(jdbcResource);
+    return new TopCompactionMetricsDataPerTypeFunction(limit).execute(jdbcResource);
   }
 
   @Override
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/DefaultTxnLockManager.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/DefaultTxnLockManager.java
new file mode 100644
index 00000000000..39b760d016e
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/DefaultTxnLockManager.java
@@ -0,0 +1,126 @@
+/*
+ * 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.metastore.txn;
+
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.api.TxnOpenException;
+import org.apache.hadoop.hive.metastore.api.UnlockRequest;
+import org.apache.hadoop.hive.metastore.txn.entities.LockInfo;
+import org.apache.hadoop.hive.metastore.txn.jdbc.functions.CheckLockFunction;
+import org.apache.hadoop.hive.metastore.txn.jdbc.functions.EnqueueLockFunction;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.GetLocksByLockId;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.ShowLocksHandler;
+import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
+import org.apache.hadoop.hive.metastore.txn.jdbc.RollbackException;
+import org.apache.hadoop.hive.metastore.utils.JavaUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+
+import java.sql.Types;
+import java.util.List;
+
+import static org.apache.hadoop.hive.metastore.txn.entities.LockInfo.LOCK_WAITING;
+
+public class DefaultTxnLockManager implements TxnLockManager {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DefaultTxnLockManager.class);
+
+  private final MultiDataSourceJdbcResource jdbcResource;
+
+  public DefaultTxnLockManager(MultiDataSourceJdbcResource jdbcResource) {
+    this.jdbcResource = jdbcResource;
+  }
+
+  @Override
+  public long enqueueLock(LockRequest rqst) throws NoSuchTxnException, TxnAbortedException, MetaException {
+    return new EnqueueLockFunction(rqst).execute(jdbcResource);
+  }
+
+  @Override
+  public LockResponse checkLock(long extLockId, long txnId, boolean zeroWaitReadEnabled, boolean isExclusiveCTAS) 
+      throws NoSuchTxnException, NoSuchLockException, TxnAbortedException, MetaException {
+    return new CheckLockFunction(extLockId, txnId, zeroWaitReadEnabled, isExclusiveCTAS).execute(jdbcResource);
+  }
+
+  @Override
+  public void unlock(UnlockRequest rqst) throws TxnOpenException, MetaException {
+    long extLockId = rqst.getLockid();
+    /**
+     * This method is logically like commit for read-only auto commit queries.
+     * READ_COMMITTED since this only has 1 delete statement and no new entries with the
+     * same hl_lock_ext_id can be added, i.e. all rows with a given hl_lock_ext_id are
+     * created in a single atomic operation.
+     * Theoretically, this competes with {@link #lock(org.apache.hadoop.hive.metastore.api.LockRequest)}
+     * but hl_lock_ext_id is not known until that method returns.
+     * Also competes with {@link #checkLock(org.apache.hadoop.hive.metastore.api.CheckLockRequest)}
+     * but using SERIALIZABLE doesn't materially change the interaction.
+     * If "delete" stmt misses, additional logic is best effort to produce meaningful error msg.
+     */
+    //hl_txnid <> 0 means it's associated with a transaction
+    int rc = jdbcResource.getJdbcTemplate().update("DELETE FROM \"HIVE_LOCKS\" WHERE \"HL_LOCK_EXT_ID\" = :extLockId " +
+            " AND (\"HL_TXNID\" = 0 OR (\"HL_TXNID\" <> 0 AND \"HL_LOCK_STATE\" = :state))", 
+        new MapSqlParameterSource()
+            .addValue("extLockId", extLockId)
+            .addValue("state", Character.toString(LOCK_WAITING), Types.CHAR));
+    //(hl_txnid <> 0 AND hl_lock_state = '" + LOCK_WAITING + "') is for multi-statement txns where
+    //some query attempted to lock (thus LOCK_WAITING state) but is giving up due to timeout for example
+    if (rc < 1) {
+      LOG.info("Failure to unlock any locks with extLockId={}.", extLockId);
+      List<LockInfo> lockInfos = jdbcResource.execute(new GetLocksByLockId(extLockId, 1, jdbcResource.getSqlGenerator()));
+      if (CollectionUtils.isEmpty(lockInfos)) {
+        //didn't find any lock with extLockId but at ReadCommitted there is a possibility that
+        //it existed when above delete ran but it didn't have the expected state.
+        LOG.info("No lock in {} mode found for unlock({})", LOCK_WAITING,
+            JavaUtils.lockIdToString(rqst.getLockid()));
+        
+        //bail here to make the operation idempotent
+        throw new RollbackException(null);
+      }
+      LockInfo lockInfo = lockInfos.get(0);
+      if (TxnUtils.isValidTxn(lockInfo.getTxnId())) {
+        String msg = "Unlocking locks associated with transaction not permitted.  " + lockInfo;
+        //if a lock is associated with a txn we can only "unlock" it if it's in WAITING state
+        // which really means that the caller wants to give up waiting for the lock
+        LOG.error(msg);
+        throw new TxnOpenException(msg);
+      } else {
+        //we didn't see this lock when running DELETE stmt above but now it showed up
+        //so should "should never happen" happened...
+        String msg = "Found lock in unexpected state " + lockInfo;
+        LOG.error(msg);
+        throw new MetaException(msg);
+      }
+    }
+    LOG.debug("Successfully unlocked at least 1 lock with extLockId={}", extLockId);
+  }
+
+  @Override
+  public ShowLocksResponse showLocks(ShowLocksRequest rqst) throws MetaException {
+    return jdbcResource.execute(new ShowLocksHandler(rqst));
+  }
+  
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/ProxyTxnHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TransactionalRetryProxy.java
similarity index 61%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/ProxyTxnHandler.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TransactionalRetryProxy.java
index 10a6e9b7654..0b85b450f3c 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/ProxyTxnHandler.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TransactionalRetryProxy.java
@@ -17,16 +17,19 @@
  */
 package org.apache.hadoop.hive.metastore.txn;
 
+import org.apache.commons.lang3.ClassUtils;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
+import org.apache.hadoop.hive.metastore.txn.jdbc.RollbackException;
 import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionContext;
-import org.apache.hadoop.hive.metastore.txn.retryhandling.SqlRetryCallProperties;
-import org.apache.hadoop.hive.metastore.txn.retryhandling.SqlRetry;
-import org.apache.hadoop.hive.metastore.txn.retryhandling.SqlRetryFunction;
-import org.apache.hadoop.hive.metastore.txn.retryhandling.SqlRetryHandler;
+import org.apache.hadoop.hive.metastore.txn.retry.SqlRetry;
+import org.apache.hadoop.hive.metastore.txn.retry.SqlRetryCallProperties;
+import org.apache.hadoop.hive.metastore.txn.retry.SqlRetryFunction;
+import org.apache.hadoop.hive.metastore.txn.retry.SqlRetryHandler;
+import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.springframework.dao.DataAccessException;
 import org.springframework.transaction.annotation.Transactional;
 
 import java.lang.reflect.InvocationHandler;
@@ -37,45 +40,44 @@ import java.lang.reflect.UndeclaredThrowableException;
 import java.util.Arrays;
 
 /**
- * Responsible for processing the following annotations: {@link SqlRetry} and {@link Transactional}. The annotated methods
- * will be called accordingly: 
+ * Responsible for proxying an interface, and calling its methods by processing the following annotations: 
+ * {@link SqlRetry} and {@link Transactional}. The annotated methods will be called accordingly: 
  * <ul>
  *   <li>SQL errors in methods annotated with {@link SqlRetry} will be caught and the method will be re-executed</li>
  *   <li>Methods annotated with {@link Transactional} will be executed after creating a transaction, and all operations done
- *   via {@link MultiDataSourceJdbcResource}, {@link org.apache.hadoop.hive.metastore.txn.jdbc.ParameterizedQuery},
- *   {@link org.apache.hadoop.hive.metastore.txn.jdbc.ParameterizedCommand} and 
- *   {@link org.apache.hadoop.hive.metastore.txn.jdbc.QueryHandler} will use the created transaction.</li>
+ *   via {@link MultiDataSourceJdbcResource} will use the created transaction.</li>
  *   <li>In case a method is annotated with both annotations, the transaction will be inside the retry-call. This means 
  *   in case of SQL errors and retries, the transaction will be rolled back and a new one will be created for each retry
  *   attempt.</li>
  * </ul> 
  * Not annotated methods are called directly.
  */
-public class ProxyTxnHandler implements InvocationHandler {
+public class TransactionalRetryProxy<T> implements InvocationHandler {
 
-  private static final Logger LOG = LoggerFactory.getLogger(ProxyTxnHandler.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TransactionalRetryProxy.class);
 
   /**
    * Gets the proxy interface for the given {@link TxnStore}.
    *
-   * @param realStore       The real {@link TxnStore} to proxy.
    * @param sqlRetryHandler Responsible to re-execute the methods in case of failure.
+   * @param interfaceObject The real object to proxy.
    * @return Returns the proxy object capable of retrying the failed calls automatically and transparently.
    */
-  public static TxnStore getProxy(TxnStore realStore, SqlRetryHandler sqlRetryHandler, MultiDataSourceJdbcResource jdbcResourceHandler) {
-    ProxyTxnHandler handler = new ProxyTxnHandler(realStore, sqlRetryHandler, jdbcResourceHandler);
-    return (TxnStore) Proxy.newProxyInstance(
-        ProxyTxnHandler.class.getClassLoader(),
-        new Class[]{ TxnStore.class },
+  public static <T> T getProxy(SqlRetryHandler sqlRetryHandler, MultiDataSourceJdbcResource jdbcResourceHandler, T interfaceObject) {
+    TransactionalRetryProxy<T> handler = new TransactionalRetryProxy<>(interfaceObject, sqlRetryHandler, jdbcResourceHandler);
+    //noinspection unchecked
+    return (T) Proxy.newProxyInstance(
+        TransactionalRetryProxy.class.getClassLoader(),
+        ClassUtils.getAllInterfaces(interfaceObject.getClass()).toArray(new Class[0]),
         handler);
   }
 
+  private final T interfaceObject;
   private final SqlRetryHandler sqlRetryHandler;
-  private final TxnStore realStore;
   private final MultiDataSourceJdbcResource jdbcResource;
 
-  private ProxyTxnHandler(TxnStore realStore, SqlRetryHandler sqlRetryHandler, MultiDataSourceJdbcResource jdbcResource) {
-    this.realStore = realStore;
+  private TransactionalRetryProxy(T interfaceObject, SqlRetryHandler sqlRetryHandler, MultiDataSourceJdbcResource jdbcResource) {
+    this.interfaceObject = interfaceObject;
     this.sqlRetryHandler = sqlRetryHandler;
     this.jdbcResource = jdbcResource;
   }
@@ -95,7 +97,7 @@ public class ProxyTxnHandler implements InvocationHandler {
 
     ThrowingSupplier functionToCall = () -> {
       try {
-        return method.invoke(realStore, args);
+        return method.invoke(interfaceObject, args);
       } catch (InvocationTargetException | UndeclaredThrowableException e) {
         throw e.getCause();
       }
@@ -108,23 +110,36 @@ public class ProxyTxnHandler implements InvocationHandler {
         TransactionContext context = null;
         try {
           jdbcResource.bindDataSource(transactional);
-          context = jdbcResource.getTransactionManager().getTransaction(transactional.propagation().value());
+          context = jdbcResource.getTransactionManager().getNewTransaction(transactional.propagation().value());
           Object result = toCall.execute();
           LOG.debug("Successfull method invocation within transactional context: {}, going to commit.", callerId);
-          jdbcResource.getTransactionManager().commit(context);
+          if (context.isRollbackOnly()) {
+            jdbcResource.getTransactionManager().rollback(context);
+          } else if (!context.isCompleted()) {
+            jdbcResource.getTransactionManager().commit(context);
+          }
           return result;
+        } catch (RollbackException e) {
+          if (context != null && !context.isCompleted()) {
+            jdbcResource.getTransactionManager().rollback(context);
+          }          
+          return e.getResult();
         } catch (Exception e) {
-          if (Arrays.stream(transactional.noRollbackFor()).anyMatch(ex -> ex.isInstance(e)) ||
-              Arrays.stream(transactional.noRollbackForClassName()).anyMatch(exName -> exName.equals(e.getClass().getName()))) {
-            throw e;
-          }
           if (context != null) {
-            if (transactional.rollbackFor().length > 0 || transactional.rollbackForClassName().length > 0) {
+            if (transactional.noRollbackFor().length > 0 || transactional.noRollbackForClassName().length > 0) {
+              if (Arrays.stream(transactional.noRollbackFor()).anyMatch(ex -> ex.isInstance(e)) ||
+                  Arrays.stream(transactional.noRollbackForClassName()).anyMatch(exName -> exName.equals(e.getClass().getName()))) {
+                jdbcResource.getTransactionManager().commit(context);
+              } else {
+                jdbcResource.getTransactionManager().rollback(context);
+              }
+            } else if (transactional.rollbackFor().length > 0 || transactional.rollbackForClassName().length > 0) {
               if (Arrays.stream(transactional.rollbackFor()).anyMatch(ex -> ex.isInstance(e)) ||
                   Arrays.stream(transactional.rollbackForClassName()).anyMatch(exName -> exName.equals(e.getClass().getName()))) {
                 jdbcResource.getTransactionManager().rollback(context);
+              } else {
+                jdbcResource.getTransactionManager().commit(context);                
               }
-              throw e;
             } else {
               jdbcResource.getTransactionManager().rollback(context);
             }
@@ -149,15 +164,23 @@ public class ProxyTxnHandler implements InvocationHandler {
           LOG.debug("Successfull method invocation within retry context: {}", callerId);
           return result;
         } catch (IllegalAccessException | InvocationTargetException | UndeclaredThrowableException e) {
-          if (e.getCause() instanceof MetaException) {
-            throw (MetaException) e.getCause();
+          if (e.getCause() instanceof TException) {
+            throw (TException) e.getCause();
           } else if (e.getCause() instanceof RuntimeException) {
             throw (RuntimeException) e.getCause();
           } else {
             throw new RuntimeException(e);
           }
-        } catch (Throwable e) {
-          throw new RuntimeException(e);
+        } catch (TException | DataAccessException e) {
+          throw e;
+        } catch (Throwable e) {          
+          if (e instanceof RuntimeException) {
+            throw (RuntimeException) e;
+          } else if (e.getCause() instanceof RuntimeException) {
+              throw (RuntimeException)e.getCause();
+          } else {
+            throw new RuntimeException(e);
+          }
         }
       };
       return sqlRetryHandler.executeWithRetry(properties, retryWrapper);
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 8905a01490e..9be5e475b6e 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
@@ -17,76 +17,19 @@
  */
 package org.apache.hadoop.hive.metastore.txn;
 
-import java.io.PrintWriter;
-import java.io.Serializable;
-import java.nio.ByteBuffer;
-import java.sql.Connection;
-import java.sql.Driver;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.SQLFeatureNotSupportedException;
-import java.sql.Statement;
-import java.sql.Timestamp;
-import java.text.MessageFormat;
-import java.time.Instant;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.BitSet;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Optional;
-import java.util.Properties;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Semaphore;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.ReentrantLock;
-import java.util.function.Function;
-import java.util.function.Predicate;
-import java.util.regex.Pattern;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-import javax.sql.DataSource;
-
-import com.google.common.collect.ImmutableList;
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang3.time.StopWatch;
-import org.apache.commons.lang3.ArrayUtils;
-import org.apache.commons.lang3.NotImplementedException;
-import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.common.MaterializationSnapshot;
-import org.apache.hadoop.hive.common.TableName;
-import org.apache.hadoop.hive.common.ValidCompactorWriteIdList;
-import org.apache.hadoop.hive.common.ValidReadTxnList;
-import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
 import org.apache.hadoop.hive.common.ValidTxnList;
-import org.apache.hadoop.hive.common.ValidTxnWriteIdList;
-import org.apache.hadoop.hive.common.ValidWriteIdList;
 import org.apache.hadoop.hive.common.classification.RetrySemantics;
-import org.apache.hadoop.hive.common.repl.ReplConst;
 import org.apache.hadoop.hive.metastore.DatabaseProduct;
-import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.MetaStoreListenerNotifier;
 import org.apache.hadoop.hive.metastore.TransactionalMetaStoreEventListener;
-import org.apache.hadoop.hive.metastore.LockTypeComparator;
 import org.apache.hadoop.hive.metastore.api.AbortCompactResponse;
 import org.apache.hadoop.hive.metastore.api.AbortCompactionRequest;
-import org.apache.hadoop.hive.metastore.api.AbortCompactionResponseElement;
-import org.apache.hadoop.hive.metastore.api.NoSuchCompactionException;
 import org.apache.hadoop.hive.metastore.api.AbortTxnRequest;
 import org.apache.hadoop.hive.metastore.api.AbortTxnsRequest;
 import org.apache.hadoop.hive.metastore.api.AddDynamicPartitions;
@@ -94,14 +37,10 @@ import org.apache.hadoop.hive.metastore.api.AllocateTableWriteIdsRequest;
 import org.apache.hadoop.hive.metastore.api.AllocateTableWriteIdsResponse;
 import org.apache.hadoop.hive.metastore.api.CheckLockRequest;
 import org.apache.hadoop.hive.metastore.api.CommitTxnRequest;
-import org.apache.hadoop.hive.metastore.api.CompactionInfoStruct;
 import org.apache.hadoop.hive.metastore.api.CompactionRequest;
 import org.apache.hadoop.hive.metastore.api.CompactionResponse;
-import org.apache.hadoop.hive.metastore.api.CompactionType;
 import org.apache.hadoop.hive.metastore.api.CreationMetadata;
-import org.apache.hadoop.hive.metastore.api.DataOperationType;
 import org.apache.hadoop.hive.metastore.api.Database;
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.GetLatestCommittedCompactionInfoRequest;
 import org.apache.hadoop.hive.metastore.api.GetLatestCommittedCompactionInfoResponse;
 import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse;
@@ -112,104 +51,101 @@ import org.apache.hadoop.hive.metastore.api.HeartbeatRequest;
 import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeRequest;
 import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeResponse;
 import org.apache.hadoop.hive.metastore.api.HiveObjectType;
-import org.apache.hadoop.hive.metastore.api.LockComponent;
 import org.apache.hadoop.hive.metastore.api.LockRequest;
 import org.apache.hadoop.hive.metastore.api.LockResponse;
-import org.apache.hadoop.hive.metastore.api.LockState;
-import org.apache.hadoop.hive.metastore.api.LockType;
 import org.apache.hadoop.hive.metastore.api.Materialization;
 import org.apache.hadoop.hive.metastore.api.MaxAllocatedTableWriteIdRequest;
 import org.apache.hadoop.hive.metastore.api.MaxAllocatedTableWriteIdResponse;
 import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchCompactionException;
 import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
 import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
 import org.apache.hadoop.hive.metastore.api.OpenTxnRequest;
 import org.apache.hadoop.hive.metastore.api.OpenTxnsResponse;
 import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.ReplLastIdInfo;
 import org.apache.hadoop.hive.metastore.api.ReplTblWriteIdStateRequest;
 import org.apache.hadoop.hive.metastore.api.SeedTableWriteIdsRequest;
 import org.apache.hadoop.hive.metastore.api.SeedTxnIdRequest;
 import org.apache.hadoop.hive.metastore.api.ShowCompactRequest;
 import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
-import org.apache.hadoop.hive.metastore.api.ShowCompactResponseElement;
 import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
 import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
-import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement;
 import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.api.TableValidWriteIds;
 import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
 import org.apache.hadoop.hive.metastore.api.TxnOpenException;
-import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
 import org.apache.hadoop.hive.metastore.api.TxnType;
 import org.apache.hadoop.hive.metastore.api.UnlockRequest;
 import org.apache.hadoop.hive.metastore.api.UpdateTransactionalStatsRequest;
-import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
-import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
 import org.apache.hadoop.hive.metastore.datasource.DataSourceProvider;
 import org.apache.hadoop.hive.metastore.datasource.DataSourceProviderFactory;
 import org.apache.hadoop.hive.metastore.events.AbortTxnEvent;
-import org.apache.hadoop.hive.metastore.events.AllocWriteIdEvent;
-import org.apache.hadoop.hive.metastore.events.CommitTxnEvent;
-import org.apache.hadoop.hive.metastore.events.ListenerEvent;
-import org.apache.hadoop.hive.metastore.events.OpenTxnEvent;
 import org.apache.hadoop.hive.metastore.events.AcidWriteEvent;
+import org.apache.hadoop.hive.metastore.events.ListenerEvent;
 import org.apache.hadoop.hive.metastore.messaging.EventMessage;
 import org.apache.hadoop.hive.metastore.metrics.Metrics;
 import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
 import org.apache.hadoop.hive.metastore.tools.SQLGenerator;
-import org.apache.hadoop.hive.metastore.txn.impl.InsertCompactionInfoCommand;
-import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionContext;
-import org.apache.hadoop.hive.metastore.txn.retryhandling.SqlRetryCallProperties;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionState;
+import org.apache.hadoop.hive.metastore.txn.entities.LockInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.MetricsInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus;
+import org.apache.hadoop.hive.metastore.txn.jdbc.commands.*;
+import org.apache.hadoop.hive.metastore.txn.jdbc.functions.*;
+import org.apache.hadoop.hive.metastore.txn.jdbc.queries.*;
 import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
-import org.apache.hadoop.hive.metastore.txn.retryhandling.SqlRetryFunction;
-import org.apache.hadoop.hive.metastore.txn.retryhandling.SqlRetryHandler;
-import org.apache.hadoop.hive.metastore.utils.FileUtils;
+import org.apache.hadoop.hive.metastore.txn.jdbc.NoPoolConnectionPool;
+import org.apache.hadoop.hive.metastore.txn.jdbc.ParameterizedCommand;
+import org.apache.hadoop.hive.metastore.txn.retry.SqlRetryCallProperties;
+import org.apache.hadoop.hive.metastore.txn.retry.SqlRetryException;
+import org.apache.hadoop.hive.metastore.txn.retry.SqlRetryHandler;
 import org.apache.hadoop.hive.metastore.utils.JavaUtils;
-import org.apache.hadoop.hive.metastore.utils.LockTypeUtil;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
-import org.apache.hadoop.hive.metastore.utils.StringableMap;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Splitter;
-
-import static org.apache.commons.lang3.StringUtils.isNotBlank;
-import static org.apache.commons.lang3.StringUtils.isBlank;
-import static org.apache.commons.lang3.StringUtils.repeat;
-import static org.apache.commons.lang3.StringUtils.EMPTY;
-import static org.apache.commons.lang3.StringUtils.wrap;
-import static org.apache.hadoop.hive.metastore.txn.TxnUtils.getEpochFn;
-import static org.apache.hadoop.hive.metastore.txn.TxnUtils.executeQueriesInBatchNoCount;
-import static org.apache.hadoop.hive.metastore.txn.TxnUtils.executeQueriesInBatch;
-import static org.apache.hadoop.hive.metastore.txn.TxnUtils.getFullTableName;
-import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
-import static org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier;
-import static org.springframework.transaction.TransactionDefinition.PROPAGATION_REQUIRED;
-
-import com.google.common.annotations.VisibleForTesting;
 import org.springframework.dao.DataAccessException;
 import org.springframework.jdbc.UncategorizedSQLException;
 import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
 
+import javax.sql.DataSource;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.sql.Timestamp;
+import java.text.MessageFormat;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.BiPredicate;
+
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
 /**
  * A handler to answer transaction related calls that come into the metastore
  * server.
- *
+ * <p>
  * Note on log messages:  Please include txnid:X and lockid info using
  * {@link JavaUtils#txnIdToString(long)}
  * and {@link 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.
- *
+ * <p>
  * Note on HIVE_LOCKS.hl_last_heartbeat.
  * For locks that are part of transaction, we set this 0 (would rather set it to NULL but
  * Currently the DB schema has this NOT NULL) and only update/read heartbeat from corresponding
  * transaction in TXNS.
- *
+ * <p>
  * In general there can be multiple metastores where this logic can execute, thus the DB is
  * used to ensure proper mutexing of operations.
  * Select ... For Update (or equivalent: either MsSql with(updlock) or actual Update stmt) is
@@ -221,22 +157,21 @@ import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
  *  This allows almost all operations to run at READ_COMMITTED and minimizes DB deadlocks.
  * 3. checkLock() - this is mutexted entirely since we must ensure that while we check if some lock
  *  can be granted, no other (strictly speaking "earlier") lock can change state.
- *
- * The exception to his is Derby which doesn't support proper S4U.  Derby is always running embedded
+ * <p>
+ * The exception to this is Derby which doesn't support proper S4U.  Derby is always running embedded
  * (this is the only supported configuration for Derby)
  * in the same JVM as HiveMetaStoreHandler thus we use JVM wide lock to properly sequnce the operations.
- *
- * {@link #derbyLock}
+ * <p>
 
  * If we ever decide to run remote Derby server, according to
  * https://db.apache.org/derby/docs/10.0/manuals/develop/develop78.html all transactions will be
  * seriazlied, so that would also work though has not been tested.
- *
+ * <p>
  * General design note:
  * It's imperative that any operation on a txn (e.g. commit), ensure (atomically) that this txn is
  * still valid and active.  In the code this is usually achieved at the same time the txn record
  * is locked for some operation.
- *
+ * <p>
  * Note on retry logic:
  * Metastore has retry logic in both {@link org.apache.hadoop.hive.metastore.RetryingMetaStoreClient}
  * and {@link org.apache.hadoop.hive.metastore.RetryingHMSHandler}.  The retry logic there is very
@@ -248,136 +183,82 @@ import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
  * the metstore call stack should have logic not to retry.  There are {@link RetrySemantics}
  * annotations to document the behavior.
  */
+@SuppressWarnings("SqlSourceToSinkFlow")
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
-  
-  private static final String TXN_TMP_STATE = "_";
-  private static final String DEFAULT_POOL_NAME = "default";
+public abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
 
-  // Lock states
-  static final protected char LOCK_ACQUIRED = 'a';
-  static final protected char LOCK_WAITING = 'w';
+  
+  public final static class ConfVars {
+    
+    private ConfVars() {}
 
-  private static final int ALLOWED_REPEATED_DEADLOCKS = 10;
-  private static final Logger LOG = LoggerFactory.getLogger(TxnHandler.class.getName());
+    // Whether to use min_history_level table or not.
+    // At startup we read it from the config, but set it to false if min_history_level does nto exists.
+    private boolean useMinHistoryLevel;
+    private boolean useMinHistoryWriteId;
 
-  private static DataSource connPool;
-  private static DataSource connPoolMutex;
+    public boolean useMinHistoryLevel() {
+      return useMinHistoryLevel;
+    }
 
-  private static final String MANUAL_RETRY = "ManualRetry";
+    public void setUseMinHistoryLevel(boolean useMinHistoryLevel) {
+      this.useMinHistoryLevel = useMinHistoryLevel;
+    }
 
-  // Query definitions
-  private static final String HIVE_LOCKS_INSERT_QRY = "INSERT INTO \"HIVE_LOCKS\" ( " +
-      "\"HL_LOCK_EXT_ID\", \"HL_LOCK_INT_ID\", \"HL_TXNID\", \"HL_DB\", \"HL_TABLE\", \"HL_PARTITION\", " +
-      "\"HL_LOCK_STATE\", \"HL_LOCK_TYPE\", \"HL_LAST_HEARTBEAT\", \"HL_USER\", \"HL_HOST\", \"HL_AGENT_INFO\") " +
-      "VALUES (?, ?, ?, ?, ?, ?, ?, ?, %s, ?, ?, ?)";
-  private static final String TXN_COMPONENTS_INSERT_QUERY = "INSERT INTO \"TXN_COMPONENTS\" (" +
-      "\"TC_TXNID\", \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\", \"TC_OPERATION_TYPE\", \"TC_WRITEID\")" +
-      " VALUES (?, ?, ?, ?, ?, ?)";
-  private static final String TXN_COMPONENTS_DP_DELETE_QUERY = "DELETE FROM \"TXN_COMPONENTS\" " +
-      "WHERE \"TC_TXNID\" = ? AND \"TC_DATABASE\" = ? AND \"TC_TABLE\" = ? AND \"TC_PARTITION\" IS NULL";
-  private static final String INCREMENT_NEXT_LOCK_ID_QUERY = "UPDATE \"NEXT_LOCK_ID\" SET \"NL_NEXT\" = %s";
-  private static final String UPDATE_HIVE_LOCKS_EXT_ID_QUERY = "UPDATE \"HIVE_LOCKS\" SET \"HL_LOCK_EXT_ID\" = %s " +
-      "WHERE \"HL_LOCK_EXT_ID\" = %s";
-  private static final String SELECT_WRITE_ID_QUERY = "SELECT \"T2W_WRITEID\" FROM \"TXN_TO_WRITE_ID\" WHERE" +
-      " \"T2W_DATABASE\" = ? AND \"T2W_TABLE\" = ? AND \"T2W_TXNID\" = ?";
-  private static final String COMPL_TXN_COMPONENTS_INSERT_QUERY = "INSERT INTO \"COMPLETED_TXN_COMPONENTS\" " +
-      "(\"CTC_TXNID\"," + " \"CTC_DATABASE\", \"CTC_TABLE\", \"CTC_PARTITION\", \"CTC_WRITEID\", \"CTC_UPDATE_DELETE\")" +
-      " VALUES (%s, ?, ?, ?, ?, %s)";
-  private static final String TXNS_INSERT_QRY = "INSERT INTO \"TXNS\" " +
-      "(\"TXN_STATE\", \"TXN_STARTED\", \"TXN_LAST_HEARTBEAT\", \"TXN_USER\", \"TXN_HOST\", \"TXN_TYPE\") " +
-      "VALUES(?,%s,%s,?,?,?)";
-  private static final String SELECT_LOCKS_FOR_LOCK_ID_QUERY = "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_LOCK_EXT_ID\" = ?";
-  private static final String SELECT_TIMED_OUT_LOCKS_QUERY = "SELECT DISTINCT \"HL_LOCK_EXT_ID\" FROM \"HIVE_LOCKS\" " +
-      "WHERE \"HL_LAST_HEARTBEAT\" < %s - :timeout AND \"HL_TXNID\" = 0";
-  private static final String TXN_TO_WRITE_ID_INSERT_QUERY = "INSERT INTO \"TXN_TO_WRITE_ID\" (\"T2W_TXNID\", " +
-      "\"T2W_DATABASE\", \"T2W_TABLE\", \"T2W_WRITEID\") VALUES (?, ?, ?, ?)";
-  private static final String MIN_HISTORY_WRITE_ID_INSERT_QUERY = "INSERT INTO \"MIN_HISTORY_WRITE_ID\" (\"MH_TXNID\", " +
-      "\"MH_DATABASE\", \"MH_TABLE\", \"MH_WRITEID\") VALUES (?, ?, ?, ?)";
-  private static final String SELECT_NWI_NEXT_FROM_NEXT_WRITE_ID =
-      "SELECT \"NWI_NEXT\" FROM \"NEXT_WRITE_ID\" WHERE \"NWI_DATABASE\" = ? AND \"NWI_TABLE\" = ?";
-  private static final String SELECT_METRICS_INFO_QUERY =
-      "SELECT * FROM (SELECT COUNT(*) FROM \"TXN_TO_WRITE_ID\") \"TTWID\" CROSS JOIN (" +
-      "SELECT COUNT(*) FROM \"COMPLETED_TXN_COMPONENTS\") \"CTC\" CROSS JOIN (" +
-      "SELECT COUNT(*), MIN(\"TXN_ID\"), ({0} - MIN(\"TXN_STARTED\"))/1000 FROM \"TXNS\" WHERE \"TXN_STATE\"='" +
-        TxnStatus.OPEN + "' AND \"TXN_TYPE\" = "+ TxnType.REPL_CREATED.getValue() +") \"TR\" CROSS JOIN (" +
-      "SELECT COUNT(*), MIN(\"TXN_ID\"), ({0} - MIN(\"TXN_STARTED\"))/1000 FROM \"TXNS\" WHERE \"TXN_STATE\"='" +
-        TxnStatus.OPEN + "' AND \"TXN_TYPE\" != "+ TxnType.REPL_CREATED.getValue() +") \"T\" CROSS JOIN (" +
-      "SELECT COUNT(*), MIN(\"TXN_ID\"), ({0} - MIN(\"TXN_STARTED\"))/1000 FROM \"TXNS\" WHERE \"TXN_STATE\"='" +
-        TxnStatus.ABORTED + "') \"A\" CROSS JOIN (" +
-      "SELECT COUNT(*), ({0} - MIN(\"HL_ACQUIRED_AT\"))/1000 FROM \"HIVE_LOCKS\") \"HL\" CROSS JOIN (" +
-      "SELECT ({0} - MIN(\"CQ_COMMIT_TIME\"))/1000 from \"COMPACTION_QUEUE\" WHERE " +
-          "\"CQ_STATE\"=''" + READY_FOR_CLEANING + "'') OLDEST_CLEAN";
-  private static final String SELECT_TABLES_WITH_X_ABORTED_TXNS =
-      "SELECT \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" FROM \"TXN_COMPONENTS\" " +
-          "INNER JOIN \"TXNS\" ON \"TC_TXNID\" = \"TXN_ID\" WHERE \"TXN_STATE\" = " + TxnStatus.ABORTED +
-      " GROUP BY \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" HAVING COUNT(\"TXN_ID\") > ?";
-  
-  private static final String  EXCL_CTAS_ERR_MSG = 
-      "Failed to initiate a concurrent CTAS operation with the same table name, lockInfo : %s";
-  private static final String ZERO_WAIT_READ_ERR_MSG = "Unable to acquire read lock due to an existing exclusive lock {%s}";
+    public boolean useMinHistoryWriteId() {
+      return useMinHistoryWriteId;
+    }
 
+    public void setUseMinHistoryWriteId(boolean useMinHistoryWriteId) {
+      this.useMinHistoryWriteId = useMinHistoryWriteId;
+    }
 
-  protected List<TransactionalMetaStoreEventListener> transactionalListeners;
+    public void init(BiPredicate<String, Boolean> tableCheck, Configuration conf){
+      useMinHistoryWriteId = tableCheck.test("MIN_HISTORY_WRITE_ID",
+          MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.TXN_USE_MIN_HISTORY_WRITE_ID));
+      useMinHistoryLevel = tableCheck.test("MIN_HISTORY_LEVEL",
+          MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.TXN_USE_MIN_HISTORY_LEVEL));
+      
+    }
+  }
+  
+  private static final Logger LOG = LoggerFactory.getLogger(TxnHandler.class.getName());
+  public static final TxnHandler.ConfVars ConfVars = new ConfVars();
 
   // Maximum number of open transactions that's allowed
   private static volatile int maxOpenTxns = 0;
   // Whether number of open transactions reaches the threshold
   private static volatile boolean tooManyOpenTxns = false;
+  // Current number of open txns
+  private static AtomicInteger numOpenTxns;
+
+  private static volatile boolean initialized = false;
+  private static DataSource connPool;
+  private static DataSource connPoolMutex;
+  protected static DataSource connPoolCompactor;
+
+  protected static DatabaseProduct dbProduct;
+  protected static SQLGenerator sqlGenerator;
+  protected static long openTxnTimeOutMillis;
 
   /**
    * Number of consecutive deadlocks we have seen
    */
-  private int deadlockCnt;
-  private long deadlockRetryInterval;
   protected Configuration conf;
-  protected static DatabaseProduct dbProduct;
-  protected static SQLGenerator sqlGenerator;
-  private static long openTxnTimeOutMillis;
 
+  protected List<TransactionalMetaStoreEventListener> transactionalListeners;
   // (End user) Transaction timeout, in milliseconds.
   private long timeout;
   private long replicationTxnTimeout;
 
-  private int maxBatchSize;
-  private String identifierQuoteString; // quotes to use for quoting tables, where necessary
-  private long retryInterval;
-  private int retryLimit;
-  private int retryNum;
-  // Current number of open txns
-  private AtomicInteger numOpenTxns;
-  // Whether to use min_history_level table or not.
-  // At startup we read it from the config, but set it to false if min_history_level does nto exists.
-  static boolean useMinHistoryLevel;
-  static boolean useMinHistoryWriteId;
-
-  private static SqlRetryHandler sqlRetryHandler;
-  protected static MultiDataSourceJdbcResource jdbcResource;
+  private MutexAPI mutexAPI;
+  private TxnLockManager txnLockManager;
+  private SqlRetryHandler sqlRetryHandler;
+  protected MultiDataSourceJdbcResource jdbcResource;
 
-  /**
-   * Derby specific concurrency control
-   */
-  private static final ReentrantLock derbyLock = new ReentrantLock(true);
-  /**
-   * must be static since even in UT there may be > 1 instance of TxnHandler
-   * (e.g. via Compactor services)
-   */
-  private final static ConcurrentHashMap<String, Semaphore> derbyKey2Lock = new ConcurrentHashMap<>();
   private static final String hostname = JavaUtils.hostname();
 
-  // Private methods should never catch SQLException and then throw MetaException.  The public
-  // methods depend on SQLException coming back so they can detect and handle deadlocks.  Private
-  // methods should only throw MetaException when they explicitly know there's a logic error and
-  // they want to throw past the public methods.
-  //
-  // All public methods that write to the database have to check for deadlocks when a SQLException
-  // comes back and handle it if they see one.  This has to be done with the connection pooling
-  // in mind.  To do this they should call checkRetryable() AFTER rolling back the db transaction,
-  // and then they should catch RetryException and call themselves recursively. See commitTxn for an example.
-
   public TxnHandler() {
   }
 
@@ -385,123 +266,86 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
    * This is logically part of c'tor and must be called prior to any other method.
    * Not physically part of c'tor due to use of reflection
    */
-  public void setConf(Configuration conf){
+  @Override
+  public void setConf(Configuration conf) {
     this.conf = conf;
 
-    int maxPoolSize = MetastoreConf.getIntVar(conf, ConfVars.CONNECTION_POOLING_MAX_CONNECTIONS);
-    synchronized (TxnHandler.class) {
-      try (DataSourceProvider.DataSourceNameConfigurator configurator =
-               new DataSourceProvider.DataSourceNameConfigurator(conf, POOL_TX)) {
-        if (connPool == null) {
-          connPool = setupJdbcConnectionPool(conf, maxPoolSize);
-        }
-        if (connPoolMutex == null) {
-          configurator.resetName(POOL_MUTEX);
-          connPoolMutex = setupJdbcConnectionPool(conf, maxPoolSize);
-        }
-      }
-      if (dbProduct == null) {
-        try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED)) {
-          determineDatabaseProduct(dbConn);
-        } catch (SQLException e) {
-          LOG.error("Unable to determine database product", e);
-          throw new RuntimeException(e);
-        }
-      }
+    if (!initialized) {
+      synchronized (TxnHandler.class) {
+        if (!initialized) {
+          try (DataSourceProvider.DataSourceNameConfigurator configurator =
+                   new DataSourceProvider.DataSourceNameConfigurator(conf, POOL_TX)) {
+            int maxPoolSize = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.CONNECTION_POOLING_MAX_CONNECTIONS);
+            if (connPool == null) {
+              connPool = setupJdbcConnectionPool(conf, maxPoolSize);
+            }
+            if (connPoolMutex == null) {
+              configurator.resetName(POOL_MUTEX);
+              connPoolMutex = setupJdbcConnectionPool(conf, maxPoolSize);
+            }
+            if (connPoolCompactor == null) {
+              configurator.resetName(POOL_COMPACTOR);
+              connPoolCompactor = setupJdbcConnectionPool(conf,
+                  MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.HIVE_COMPACTOR_CONNECTION_POOLING_MAX_CONNECTIONS));
+            }
+          }
+          if (dbProduct == null) {
+            try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED, connPool)) {
+              determineDatabaseProduct(dbConn);
+            } catch (SQLException e) {
+              LOG.error("Unable to determine database product", e);
+              throw new RuntimeException(e);
+            }
+          }
+          if (sqlGenerator == null) {
+            sqlGenerator = new SQLGenerator(dbProduct, conf);
+          }
+          
+          initJdbcResource();
 
-      if (sqlGenerator == null) {
-        sqlGenerator = new SQLGenerator(dbProduct, conf);
+          try {
+            TxnHandler.ConfVars.init(this::checkIfTableIsUsable, conf);
+          } catch (Exception e) {
+            String msg = "Error during TxnHandler initialization, " + e.getMessage();
+            LOG.error(msg);
+            throw e;
+          }          
+          initialized = true;
+        }
       }
-      
-      if (jdbcResource == null) {
-        jdbcResource = new MultiDataSourceJdbcResource(dbProduct);
-        jdbcResource.registerDataSource(POOL_TX, connPool);
-        jdbcResource.registerDataSource(POOL_MUTEX, connPoolMutex);
-      }      
     }
 
-    numOpenTxns = Metrics.getOrCreateGauge(MetricsConstants.NUM_OPEN_TXNS);
+    initJdbcResource();
 
-    timeout = MetastoreConf.getTimeVar(conf, ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
-    replicationTxnTimeout = MetastoreConf.getTimeVar(conf, ConfVars.REPL_TXN_TIMEOUT, TimeUnit.MILLISECONDS);
-    retryInterval = MetastoreConf.getTimeVar(conf, ConfVars.HMS_HANDLER_INTERVAL,
-        TimeUnit.MILLISECONDS);
-    retryLimit = MetastoreConf.getIntVar(conf, ConfVars.HMS_HANDLER_ATTEMPTS);
-    deadlockRetryInterval = retryInterval / 10;
-    maxOpenTxns = MetastoreConf.getIntVar(conf, ConfVars.MAX_OPEN_TXNS);
-    maxBatchSize = MetastoreConf.getIntVar(conf, ConfVars.JDBC_MAX_BATCH_SIZE);
+    mutexAPI = new TxnStoreMutex(sqlGenerator, jdbcResource);
 
-    openTxnTimeOutMillis = MetastoreConf.getTimeVar(conf, ConfVars.TXN_OPENTXN_TIMEOUT, TimeUnit.MILLISECONDS);
-    
-    try {
-      useMinHistoryWriteId = checkIfTableIsUsable("MIN_HISTORY_WRITE_ID", 
-        MetastoreConf.getBoolVar(conf, ConfVars.TXN_USE_MIN_HISTORY_WRITE_ID));
-      
-      // override the config if table does not exists anymore
-      // this helps to roll out his feature when multiple HMS is accessing the same backend DB
-      useMinHistoryLevel = checkIfTableIsUsable("MIN_HISTORY_LEVEL",
-        MetastoreConf.getBoolVar(conf, ConfVars.TXN_USE_MIN_HISTORY_LEVEL));
-    } catch (MetaException e) {
-      String msg = "Error during TxnHandler startup, " + e.getMessage();
-      LOG.error(msg);
-      throw new RuntimeException(e);
-    }
+    numOpenTxns = Metrics.getOrCreateGauge(MetricsConstants.NUM_OPEN_TXNS);
+
+    timeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    replicationTxnTimeout = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.REPL_TXN_TIMEOUT, TimeUnit.MILLISECONDS);
+    maxOpenTxns = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.MAX_OPEN_TXNS);
+    openTxnTimeOutMillis = MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.TXN_OPENTXN_TIMEOUT, TimeUnit.MILLISECONDS);
 
     try {
       transactionalListeners = MetaStoreServerUtils.getMetaStoreListeners(
-              TransactionalMetaStoreEventListener.class,
-                      conf, MetastoreConf.getVar(conf, ConfVars.TRANSACTIONAL_EVENT_LISTENERS));
-    } catch(MetaException e) {
+          TransactionalMetaStoreEventListener.class,
+          conf, MetastoreConf.getVar(conf, MetastoreConf.ConfVars.TRANSACTIONAL_EVENT_LISTENERS));
+    } catch (MetaException e) {
       String msg = "Unable to get transaction listeners, " + e.getMessage();
       LOG.error(msg);
       throw new RuntimeException(e);
     }
 
-    sqlRetryHandler = new SqlRetryHandler(conf, jdbcResource.getDatabaseProduct());    
-  }
-
-  /**
-   * Check if provided table is usable
-   * @return
-   * @throws MetaException
-   */
-  private boolean checkIfTableIsUsable(String tableName, boolean configValue) throws MetaException {
-    if (!configValue) {
-      // don't check it if disabled
-      return false;
-    }
-    Connection dbConn = null;
-    boolean tableExists = true;
-    try {
-      dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-      try (Statement stmt = dbConn.createStatement()) {
-        // Dummy query to see if table exists
-        try (ResultSet rs = stmt.executeQuery("SELECT 1 FROM \"" + tableName + "\"")) {
-          rs.next();
-        }
-      }
-    } catch (SQLException e) {
-      LOG.debug("Catching sql exception in " + tableName + " check", e);
-      if (dbProduct.isTableNotExistsError(e)) {
-        tableExists = false;
-      } else {
-        throw new MetaException(
-          "Unable to select from transaction database: " + getMessage(e) + StringUtils.stringifyException(e));
-      }
-    } finally {
-      closeDbConn(dbConn);
-    }
-    return tableExists;
+    sqlRetryHandler = new SqlRetryHandler(conf, jdbcResource.getDatabaseProduct());
+    txnLockManager = TransactionalRetryProxy.getProxy(sqlRetryHandler, jdbcResource, new DefaultTxnLockManager(jdbcResource));
   }
-  
+ 
   @Override
-  @RetrySemantics.ReadOnly
   public SqlRetryHandler getRetryHandler() {
     return sqlRetryHandler;
   }
 
   @Override
-  @RetrySemantics.ReadOnly
   public MultiDataSourceJdbcResource getJdbcResourceHolder() {
     return jdbcResource;
   }
@@ -512,114 +356,21 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
   }
 
   @Override
-  @RetrySemantics.ReadOnly
   public GetOpenTxnsInfoResponse getOpenTxnsInfo() throws MetaException {
-    return getOpenTxnsList(true).toOpenTxnsInfoResponse();
+    return jdbcResource.execute(new GetOpenTxnsListHandler(true, openTxnTimeOutMillis))
+        .toOpenTxnsInfoResponse();
   }
 
   @Override
-  @RetrySemantics.ReadOnly
   public GetOpenTxnsResponse getOpenTxns() throws MetaException {
-    return getOpenTxnsList(false).toOpenTxnsResponse(Arrays.asList(TxnType.READ_ONLY));
-  }
-
-  private GetOpenTxnsResponse getOpenTxns(Connection dbConn) throws MetaException {
-    return getOpenTxnsList(false, dbConn).toOpenTxnsResponse(Arrays.asList(TxnType.READ_ONLY));
+    return jdbcResource.execute(new GetOpenTxnsListHandler(false, openTxnTimeOutMillis))
+        .toOpenTxnsResponse(Collections.singletonList(TxnType.READ_ONLY));
   }
 
   @Override
-  @RetrySemantics.ReadOnly
   public GetOpenTxnsResponse getOpenTxns(List<TxnType> excludeTxnTypes) throws MetaException {
-    return getOpenTxnsList(false).toOpenTxnsResponse(excludeTxnTypes);
-  }
-
-  private OpenTxnList getOpenTxnsList(boolean infoFileds) throws MetaException {
-    Connection dbConn = null;
-    try {
-      dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-      return getOpenTxnsList(infoFileds, dbConn);
-    } catch (SQLException e) {
-      throw new MetaException(
-          "Unable to get a connection: " + getMessage(e) + StringUtils.stringifyException(e));
-    } finally {
-      closeDbConn(dbConn);
-    }
-  }
-
-  private OpenTxnList getOpenTxnsList(boolean infoFields, Connection dbConn) throws MetaException {
-    try {
-      // We need to figure out the HighWaterMark and the list of open transactions.
-      Statement stmt = null;
-      ResultSet rs = null;
-      try {
-        /*
-         * This method need guarantees from
-         * {@link #openTxns(OpenTxnRequest)} and  {@link #commitTxn(CommitTxnRequest)}.
-         * It will look at the TXNS table and find each transaction between the max(txn_id) as HighWaterMark
-         * and the max(txn_id) before the TXN_OPENTXN_TIMEOUT period as LowWaterMark.
-         * Every transaction that is not found between these will be considered as open, since it may appear later.
-         * openTxns must ensure, that no new transaction will be opened with txn_id below LWM and
-         * commitTxn must ensure, that no committed transaction will be removed before the time period expires.
-         */
-        stmt = dbConn.createStatement();
-        List<OpenTxn> txnInfos = new ArrayList<>();
-        String txnsQuery = String.format(infoFields ? OpenTxn.OPEN_TXNS_INFO_QUERY : OpenTxn.OPEN_TXNS_QUERY,
-            getEpochFn(dbProduct));
-        LOG.debug("Going to execute query<{}>", txnsQuery);
-        rs = stmt.executeQuery(txnsQuery);
-        /*
-         * We can use the maximum txn_id from the TXNS table as high water mark, since the commitTxn and the Initiator
-         * guarantees, that the transaction with the highest txn_id will never be removed from the TXNS table.
-         * If there is a pending openTxns, that is already acquired it's sequenceId but not yet committed the insert
-         * into the TXNS table, will have either a lower txn_id than HWM and will be listed in the openTxn list,
-         * or will have a higher txn_id and don't effect this getOpenTxns() call.
-         */
-        long hwm = 0;
-        long openTxnLowBoundary = 0;
-
-        while (rs.next()) {
-          long txnId = rs.getLong(1);
-          long age = rs.getLong(4);
-          hwm = txnId;
-          if (age < getOpenTxnTimeOutMillis()) {
-            // We will consider every gap as an open transaction from the previous txnId
-            openTxnLowBoundary++;
-            while (txnId > openTxnLowBoundary) {
-              // Add an empty open transaction for every missing value
-              txnInfos.add(new OpenTxn(openTxnLowBoundary, TxnStatus.OPEN, TxnType.DEFAULT));
-              LOG.debug("Open transaction added for missing value in TXNS {}",
-                  JavaUtils.txnIdToString(openTxnLowBoundary));
-              openTxnLowBoundary++;
-            }
-          } else {
-            openTxnLowBoundary = txnId;
-          }
-          TxnStatus state = TxnStatus.fromString(rs.getString(2));
-          if (state == TxnStatus.COMMITTED) {
-            // This is only here, to avoid adding this txnId as possible gap
-            continue;
-          }
-          OpenTxn txnInfo = new OpenTxn(txnId, state, TxnType.findByValue(rs.getInt(3)));
-          if (infoFields) {
-            txnInfo.setUser(rs.getString(5));
-            txnInfo.setHost(rs.getString(6));
-            txnInfo.setStartedTime(rs.getLong(7));
-            txnInfo.setLastHeartBeatTime(rs.getLong(8));
-          }
-          txnInfos.add(txnInfo);
-        }
-        LOG.debug("Got OpenTxnList with hwm: {} and openTxnList size {}.", hwm, txnInfos.size());
-        return new OpenTxnList(hwm, txnInfos);
-      } catch (SQLException e) {
-        checkRetryable(e, "getOpenTxnsList");
-        throw new MetaException(
-            "Unable to select from transaction database: " + getMessage(e) + StringUtils.stringifyException(e));
-      } finally {
-        close(rs, stmt, null);
-      }
-    } catch (RetryException e) {
-      return getOpenTxnsList(infoFields, dbConn);
-    }
+    return jdbcResource.execute(new GetOpenTxnsListHandler(false, openTxnTimeOutMillis))
+        .toOpenTxnsResponse(excludeTxnTypes);
   }
 
   /**
@@ -627,7 +378,6 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
    * Worst case, it will leave an open txn which will timeout.
    */
   @Override
-  @RetrySemantics.Idempotent
   public OpenTxnsResponse openTxns(OpenTxnRequest rqst) throws MetaException {
     if (!tooManyOpenTxns && numOpenTxns.get() >= maxOpenTxns) {
       tooManyOpenTxns = true;
@@ -648,260 +398,75 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
       throw new MetaException("Invalid input for number of txns: " + numTxns);
     }
 
-    try {
-      Connection dbConn = null;
-      Statement stmt = null;
-      try {
-        /*
-         * To make {@link #getOpenTxns()}/{@link #getOpenTxnsInfo()} work correctly, this operation must ensure
-         * that looking at the TXNS table every open transaction could be identified below a given High Water Mark.
-         * One way to do it, would be to serialize the openTxns call with a S4U lock, but that would cause
-         * performance degradation with high transaction load.
-         * To enable parallel openTxn calls, we define a time period (TXN_OPENTXN_TIMEOUT) and consider every
-         * transaction missing from the TXNS table in that period open, and prevent opening transaction outside
-         * the period.
-         * Example: At t[0] there is one open transaction in the TXNS table, T[1].
-         * T[2] acquires the next sequence at t[1] but only commits into the TXNS table at t[10].
-         * T[3] acquires its sequence at t[2], and commits into the TXNS table at t[3].
-         * Then T[3] calculates it’s snapshot at t[4] and puts T[1] and also T[2] in the snapshot’s
-         * open transaction list. T[1] because it is presented as open in TXNS,
-         * T[2] because it is a missing sequence.
-         *
-         * In the current design, there can be several metastore instances running in a given Warehouse.
-         * This makes ideas like reserving a range of IDs to save trips to DB impossible.  For example,
-         * a client may go to MS1 and start a transaction with ID 500 to update a particular row.
-         * Now the same client will start another transaction, except it ends up on MS2 and may get
-         * transaction ID 400 and update the same row.  Now the merge that happens to materialize the snapshot
-         * on read will thing the version of the row from transaction ID 500 is the latest one.
-         *
-         * Longer term we can consider running Active-Passive MS (at least wrt to ACID operations).  This
-         * set could support a write-through cache for added performance.
-         */
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-        stmt = dbConn.createStatement();
-        /*
-         * The openTxn and commitTxn must be mutexed, when committing a not read only transaction.
-         * This is achieved by requesting a shared table lock here, and an exclusive one at commit.
-         * Since table locks are working in Derby, we don't need the lockInternal call here.
-         * Example: Suppose we have two transactions with update like x = x+1.
-         * We have T[3,3] that was using a value from a snapshot with T[2,2]. If we allow committing T[3,3]
-         * and opening T[4] parallel it is possible, that T[4] will be using the value from a snapshot with T[2,2],
-         * and we will have a lost update problem
-         */
-        acquireTxnLock(stmt, true);
-        // Measure the time from acquiring the sequence value, till committing in the TXNS table
-        StopWatch generateTransactionWatch = new StopWatch();
-        generateTransactionWatch.start();
-
-        List<Long> txnIds = openTxns(dbConn, rqst);
+    /*
+     * To make {@link #getOpenTxns()}/{@link #getOpenTxnsInfo()} work correctly, this operation must ensure
+     * that looking at the TXNS table every open transaction could be identified below a given High Water Mark.
+     * One way to do it, would be to serialize the openTxns call with a S4U lock, but that would cause
+     * performance degradation with high transaction load.
+     * To enable parallel openTxn calls, we define a time period (TXN_OPENTXN_TIMEOUT) and consider every
+     * transaction missing from the TXNS table in that period open, and prevent opening transaction outside
+     * the period.
+     * Example: At t[0] there is one open transaction in the TXNS table, T[1].
+     * T[2] acquires the next sequence at t[1] but only commits into the TXNS table at t[10].
+     * T[3] acquires its sequence at t[2], and commits into the TXNS table at t[3].
+     * Then T[3] calculates it’s snapshot at t[4] and puts T[1] and also T[2] in the snapshot’s
+     * open transaction list. T[1] because it is presented as open in TXNS,
+     * T[2] because it is a missing sequence.
+     *
+     * In the current design, there can be several metastore instances running in a given Warehouse.
+     * This makes ideas like reserving a range of IDs to save trips to DB impossible.  For example,
+     * a client may go to MS1 and start a transaction with ID 500 to update a particular row.
+     * Now the same client will start another transaction, except it ends up on MS2 and may get
+     * transaction ID 400 and update the same row.  Now the merge that happens to materialize the snapshot
+     * on read will thing the version of the row from transaction ID 500 is the latest one.
+     *
+     * Longer term we can consider running Active-Passive MS (at least wrt to ACID operations).  This
+     * set could support a write-through cache for added performance.
+     */
+    /*
+     * The openTxn and commitTxn must be mutexed, when committing a not read only transaction.
+     * This is achieved by requesting a shared table lock here, and an exclusive one at commit.
+     * Since table locks are working in Derby, we don't need the lockInternal call here.
+     * Example: Suppose we have two transactions with update like x = x+1.
+     * We have T[3,3] that was using a value from a snapshot with T[2,2]. If we allow committing T[3,3]
+     * and opening T[4] parallel it is possible, that T[4] will be using the value from a snapshot with T[2,2],
+     * and we will have a lost update problem
+     */
+    acquireTxnLock(true);
+    // Measure the time from acquiring the sequence value, till committing in the TXNS table
+    StopWatch generateTransactionWatch = new StopWatch();
+    generateTransactionWatch.start();
 
-        LOG.debug("Going to commit");
-        dbConn.commit();
-        generateTransactionWatch.stop();
-        long elapsedMillis = generateTransactionWatch.getTime(TimeUnit.MILLISECONDS);
-        TxnType txnType = rqst.isSetTxn_type() ? rqst.getTxn_type() : TxnType.DEFAULT;
-        if (txnType != TxnType.READ_ONLY && elapsedMillis >= openTxnTimeOutMillis) {
-          /*
-           * The commit was too slow, we can not allow this to continue (except if it is read only,
-           * since that can not cause dirty reads).
-           * When calculating the snapshot for a given transaction, we look back for possible open transactions
-           * (that are not yet committed in the TXNS table), for TXN_OPENTXN_TIMEOUT period.
-           * We can not allow a write transaction, that was slower than TXN_OPENTXN_TIMEOUT to continue,
-           * because there can be other transactions running, that didn't considered this transactionId open,
-           * this could cause dirty reads.
-           */
-          LOG.error("OpenTxnTimeOut exceeded commit duration {}, deleting transactionIds: {}", elapsedMillis, txnIds);
-          deleteInvalidOpenTransactions(dbConn, txnIds);
-          dbConn.commit();
-          /*
-           * We do not throw RetryException directly, to not circumvent the max retry limit
-           */
-          throw new SQLException("OpenTxnTimeOut exceeded", MANUAL_RETRY);
-        }
-        return new OpenTxnsResponse(txnIds);
-      } catch (SQLException e) {
-        LOG.debug("Going to rollback: ", e);
-        rollbackDBConn(dbConn);
-        checkRetryable(e, "openTxns(" + rqst + ")");
-        throw new MetaException("Unable to select from transaction database " + StringUtils.stringifyException(e));
-      } finally {
-        close(null, stmt, dbConn);
-      }
-    } catch (RetryException e) {
-      return openTxns(rqst);
-    }
-  }
+    List<Long> txnIds = new OpenTxnsFunction(rqst, openTxnTimeOutMillis, transactionalListeners).execute(jdbcResource);
 
-  private List<Long> openTxns(Connection dbConn, OpenTxnRequest rqst)
-          throws SQLException, MetaException {
-    int numTxns = rqst.getNum_txns();
-    // Make sure the user has not requested an insane amount of txns.
-    int maxTxns = MetastoreConf.getIntVar(conf, ConfVars.TXN_MAX_OPEN_BATCH);
-    if (numTxns > maxTxns) {
-      numTxns = maxTxns;
-    }
-    List<PreparedStatement> insertPreparedStmts = null;
+    LOG.debug("Going to commit");
+    jdbcResource.getTransactionManager().getActiveTransaction().createSavepoint();
+    generateTransactionWatch.stop();
+    long elapsedMillis = generateTransactionWatch.getTime(TimeUnit.MILLISECONDS);
     TxnType txnType = rqst.isSetTxn_type() ? rqst.getTxn_type() : TxnType.DEFAULT;
-    boolean isReplayedReplTxn = txnType == TxnType.REPL_CREATED;
-    boolean isHiveReplTxn = rqst.isSetReplPolicy() && txnType == TxnType.DEFAULT;
-    try {
-      if (isReplayedReplTxn) {
-        assert rqst.isSetReplPolicy();
-        List<Long> targetTxnIdList = getTargetTxnIdList(rqst.getReplPolicy(), rqst.getReplSrcTxnIds(), dbConn);
-
-        if (!targetTxnIdList.isEmpty()) {
-          if (targetTxnIdList.size() != rqst.getReplSrcTxnIds().size()) {
-            LOG.warn("target txn id number {} is not matching with source txn id number {}",
-                targetTxnIdList, rqst.getReplSrcTxnIds());
-          }
-          LOG.info("Target transactions {} are present for repl policy : {} and Source transaction id : {}",
-              targetTxnIdList.toString(), rqst.getReplPolicy(), rqst.getReplSrcTxnIds().toString());
-          return targetTxnIdList;
-        }
-      }
-
-      long minOpenTxnId = 0;
-      if (useMinHistoryLevel) {
-        minOpenTxnId = getMinOpenTxnIdWaterMark(dbConn);
-      }
-
-      List<Long> txnIds = new ArrayList<>(numTxns);
+    if (txnType != TxnType.READ_ONLY && elapsedMillis >= openTxnTimeOutMillis) {
       /*
-       * The getGeneratedKeys are not supported in every dbms, after executing a multi line insert.
-       * But it is supported in every used dbms for single line insert, even if the metadata says otherwise.
-       * If the getGeneratedKeys are not supported first we insert a random batchId in the TXN_META_INFO field,
-       * then the keys are selected beck with that batchid.
+       * The commit was too slow, we can not allow this to continue (except if it is read only,
+       * since that can not cause dirty reads).
+       * When calculating the snapshot for a given transaction, we look back for possible open transactions
+       * (that are not yet committed in the TXNS table), for TXN_OPENTXN_TIMEOUT period.
+       * We can not allow a write transaction, that was slower than TXN_OPENTXN_TIMEOUT to continue,
+       * because there can be other transactions running, that didn't considered this transactionId open,
+       * this could cause dirty reads.
        */
-      boolean genKeySupport = dbProduct.supportsGetGeneratedKeys();
-      genKeySupport = genKeySupport || (numTxns == 1);
-
-      String insertQuery = String.format(TXNS_INSERT_QRY, getEpochFn(dbProduct), getEpochFn(dbProduct));
-      LOG.debug("Going to execute insert <{}>", insertQuery);
-      try (PreparedStatement ps = dbConn.prepareStatement(insertQuery, new String[] {"TXN_ID"})) {
-        String state = genKeySupport ? TxnStatus.OPEN.getSqlConst() : TXN_TMP_STATE;
-        if (numTxns == 1) {
-          ps.setString(1, state);
-          ps.setString(2, rqst.getUser());
-          ps.setString(3, rqst.getHostname());
-          ps.setInt(4, txnType.getValue());
-          txnIds.addAll(executeTxnInsertBatchAndExtractGeneratedKeys(dbConn, genKeySupport, ps, false));
-        } else {
-          for (int i = 0; i < numTxns; ++i) {
-            ps.setString(1, state);
-            ps.setString(2, rqst.getUser());
-            ps.setString(3, rqst.getHostname());
-            ps.setInt(4, txnType.getValue());
-            ps.addBatch();
-
-            if ((i + 1) % maxBatchSize == 0) {
-              txnIds.addAll(executeTxnInsertBatchAndExtractGeneratedKeys(dbConn, genKeySupport, ps, true));
-            }
-          }
-          if (numTxns % maxBatchSize != 0) {
-            txnIds.addAll(executeTxnInsertBatchAndExtractGeneratedKeys(dbConn, genKeySupport, ps, true));
-          }
-        }
-      }
-
-      assert txnIds.size() == numTxns;
-
-      addTxnToMinHistoryLevel(dbConn, txnIds, minOpenTxnId);
-
-      if (isReplayedReplTxn) {
-        List<String> rowsRepl = new ArrayList<>(numTxns);
-        List<String> params = Collections.singletonList(rqst.getReplPolicy());
-        List<List<String>> paramsList = new ArrayList<>(numTxns);
-        for (int i = 0; i < numTxns; i++) {
-          rowsRepl.add("?," + rqst.getReplSrcTxnIds().get(i) + "," + txnIds.get(i));
-          paramsList.add(params);
-        }
-
-        insertPreparedStmts = sqlGenerator.createInsertValuesPreparedStmt(dbConn,
-                "\"REPL_TXN_MAP\" (\"RTM_REPL_POLICY\", \"RTM_SRC_TXN_ID\", \"RTM_TARGET_TXN_ID\")", rowsRepl,
-                paramsList);
-        for (PreparedStatement pst : insertPreparedStmts) {
-          pst.execute();
-        }
-      }
+      LOG.error("OpenTxnTimeOut exceeded commit duration {}, deleting transactionIds: {}", elapsedMillis, txnIds);
 
-      if (transactionalListeners != null && !isHiveReplTxn) {
-        MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners,
-                EventMessage.EventType.OPEN_TXN, new OpenTxnEvent(txnIds, txnType), dbConn, sqlGenerator);
-      }
-      return txnIds;
-    } finally {
-      if (insertPreparedStmts != null) {
-        for (PreparedStatement pst : insertPreparedStmts) {
-          pst.close();
-        }
-      }
-    }
-  }
-
-  private List<Long> executeTxnInsertBatchAndExtractGeneratedKeys(Connection dbConn, boolean genKeySupport,
-      PreparedStatement ps, boolean batch) throws SQLException {
-    List<Long> txnIds = new ArrayList<>();
-    if (batch) {
-      ps.executeBatch();
-    } else {
-      // For slight performance advantage we do not use the executeBatch, when we only have one row
-      ps.execute();
-    }
-    if (genKeySupport) {
-      try (ResultSet generatedKeys = ps.getGeneratedKeys()) {
-        while (generatedKeys.next()) {
-          txnIds.add(generatedKeys.getLong(1));
-        }
-      }
-    } else {
-      try (PreparedStatement pstmt =
-          dbConn.prepareStatement("SELECT \"TXN_ID\" FROM \"TXNS\" WHERE \"TXN_STATE\" = ?")) {
-        pstmt.setString(1, TXN_TMP_STATE);
-        try (ResultSet rs = pstmt.executeQuery()) {
-          while (rs.next()) {
-            txnIds.add(rs.getLong(1));
-          }
-        }
-      }
-      try (PreparedStatement pstmt = dbConn
-          .prepareStatement("UPDATE \"TXNS\" SET \"TXN_STATE\" = ? WHERE \"TXN_STATE\" = ?")) {
-        pstmt.setString(1, TxnStatus.OPEN.getSqlConst());
-        pstmt.setString(2, TXN_TMP_STATE);
-        pstmt.executeUpdate();
+      if (!txnIds.isEmpty()) {
+        deleteInvalidOpenTransactions(txnIds);
       }
-    }
-    return txnIds;
-  }
 
-  private void deleteInvalidOpenTransactions(Connection dbConn, List<Long> txnIds) throws MetaException {
-    if (txnIds.size() == 0) {
-      return;
+      /*
+       * We cannot throw SQLException directly, as it is not in the throws clause
+       */
+      throw new SqlRetryException("OpenTxnTimeOut exceeded");
     }
-    try {
-      Statement stmt = null;
-      try {
-        stmt = dbConn.createStatement();
 
-        List<String> queries = new ArrayList<>();
-        StringBuilder prefix = new StringBuilder();
-        StringBuilder suffix = new StringBuilder();
-        prefix.append("DELETE FROM \"TXNS\" WHERE ");
-        TxnUtils.buildQueryWithINClause(conf, queries, prefix, suffix, txnIds, "\"TXN_ID\"", false, false);
-        executeQueriesInBatchNoCount(dbProduct, stmt, queries, maxBatchSize);
-        LOG.info("Removed transactions: ({}) from TXNS", txnIds);
-
-        removeTxnsFromMinHistoryLevel(dbConn, txnIds);
-      } catch (SQLException e) {
-        LOG.debug("Going to rollback: ", e);
-        rollbackDBConn(dbConn);
-        checkRetryable(e, "deleteInvalidOpenTransactions(" + txnIds + ")");
-        throw new MetaException("Unable to select from transaction database " + StringUtils.stringifyException(e));
-      } finally {
-        closeStmt(stmt);
-      }
-    } catch (RetryException ex) {
-      deleteInvalidOpenTransactions(dbConn, txnIds);
-    }
+    return new OpenTxnsResponse(txnIds);
   }
 
   @Override
@@ -914,518 +479,89 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
     TxnHandler.openTxnTimeOutMillis = openTxnTimeOutMillis;
   }
 
-  protected long getOpenTxnTimeoutLowBoundaryTxnId(Connection dbConn) throws MetaException, SQLException {
-    long maxTxnId;
-    String s =
-        "SELECT MAX(\"TXN_ID\") FROM \"TXNS\" WHERE \"TXN_STARTED\" < (" + getEpochFn(dbProduct) + " - "
-            + openTxnTimeOutMillis + ")";
-    try (Statement stmt = dbConn.createStatement()) {
-      LOG.debug("Going to execute query <{}>", s);
-      try (ResultSet maxTxnIdRs = stmt.executeQuery(s)) {
-        maxTxnIdRs.next();
-        maxTxnId = maxTxnIdRs.getLong(1);
-        if (maxTxnIdRs.wasNull()) {
-          /*
-           * TXNS always contains at least one transaction,
-           * the row where txnid = (select max(txnid) where txn_started < epoch - TXN_OPENTXN_TIMEOUT) is never deleted
-           */
-          throw new MetaException("Transaction tables not properly " + "initialized, null record found in MAX(TXN_ID)");
-        }
-      }
+  @Override
+  public long getTargetTxnId(String replPolicy, long sourceTxnId) throws MetaException {
+    List<Long> targetTxnIds =jdbcResource.execute(new TargetTxnIdListHandler(replPolicy, Collections.singletonList(sourceTxnId)));
+    if (targetTxnIds.isEmpty()) {
+      LOG.info("Txn {} not present for repl policy {}", sourceTxnId, replPolicy);
+      return -1;
     }
-    return maxTxnId;
+    assert (targetTxnIds.size() == 1);
+    return targetTxnIds.get(0);
   }
 
-  private long getHighWaterMark(Statement stmt) throws SQLException, MetaException {
-    String s = "SELECT MAX(\"TXN_ID\") FROM \"TXNS\"";
-    LOG.debug("Going to execute query <{}>", s);
-    long maxOpenTxnId;
-    try (ResultSet maxOpenTxnIdRs = stmt.executeQuery(s)) {
-      maxOpenTxnIdRs.next();
-      maxOpenTxnId = maxOpenTxnIdRs.getLong(1);
-      if (maxOpenTxnIdRs.wasNull()) {
-        /*
-         * TXNS always contains at least one transaction,
-         * the row where txnid = (select max(txnid) where txn_started < epoch - TXN_OPENTXN_TIMEOUT) is never deleted
-         */
-        throw new MetaException("Transaction tables not properly " + "initialized, null record found in MAX(TXN_ID)");
+  @Override
+  public void abortTxn(AbortTxnRequest rqst) throws NoSuchTxnException, MetaException, TxnAbortedException {
+    TxnType txnType = new AbortTxnFunction(rqst).execute(jdbcResource); 
+    if (txnType != null) {
+      if (transactionalListeners != null && (!rqst.isSetReplPolicy() || !TxnType.DEFAULT.equals(rqst.getTxn_type()))) {
+        List<String> dbsUpdated = getTxnDbsUpdated(rqst.getTxnid());
+        MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners, EventMessage.EventType.ABORT_TXN,
+            new AbortTxnEvent(rqst.getTxnid(), txnType, null, dbsUpdated), jdbcResource.getConnection(), sqlGenerator);
       }
     }
-    return maxOpenTxnId;
   }
 
-  private List<Long> getTargetTxnIdList(String replPolicy, List<Long> sourceTxnIdList, Connection dbConn)
-          throws SQLException {
-    PreparedStatement pst = null;
-    ResultSet rs = null;
+  @Override
+  public void abortTxns(AbortTxnsRequest rqst) throws MetaException {
+    List<Long> txnIds = rqst.getTxn_ids();
+    TxnErrorMsg txnErrorMsg = TxnErrorMsg.NONE;
+    if (rqst.isSetErrorCode()) {
+      txnErrorMsg = TxnErrorMsg.getTxnErrorMsg(rqst.getErrorCode());
+    }
+
+    List<String> queries = new ArrayList<>();
+    StringBuilder prefix =
+        new StringBuilder("SELECT \"TXN_ID\", \"TXN_TYPE\" from \"TXNS\" where \"TXN_STATE\" = ")
+            .append(TxnStatus.OPEN)
+            .append(" and \"TXN_TYPE\" != ").append(TxnType.READ_ONLY.getValue()).append(" and ");
+
+    TxnUtils.buildQueryWithINClause(conf, queries, prefix, new StringBuilder(),
+        txnIds, "\"TXN_ID\"", false, false);
+
+    Connection dbConn = jdbcResource.getConnection();
     try {
-      List<String> inQueries = new ArrayList<>();
-      StringBuilder prefix = new StringBuilder();
-      StringBuilder suffix = new StringBuilder();
-      List<Long> targetTxnIdList = new ArrayList<>();
-      prefix.append("SELECT \"RTM_TARGET_TXN_ID\" FROM \"REPL_TXN_MAP\" WHERE ");
-      suffix.append(" AND \"RTM_REPL_POLICY\" = ?");
-      TxnUtils.buildQueryWithINClause(conf, inQueries, prefix, suffix, sourceTxnIdList,
-              "\"RTM_SRC_TXN_ID\"", false, false);
-      List<String> params = Arrays.asList(replPolicy);
-      for (String query : inQueries) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Going to execute select <" + query.replace("?", "{}") + ">", quoteString(replPolicy));
+      Map<Long, TxnType> nonReadOnlyTxns = new HashMap<>();
+      for (String query : queries) {
+        LOG.debug("Going to execute query <{}>", query);
+        try (Statement stmt = dbConn.createStatement(); ResultSet rs = stmt.executeQuery(sqlGenerator.addForUpdateClause(query))) {
+          while (rs.next()) {
+            TxnType txnType = TxnType.findByValue(rs.getInt(2));
+            nonReadOnlyTxns.put(rs.getLong(1), txnType);
+          }
         }
-        pst = sqlGenerator.prepareStmtWithParameters(dbConn, query, params);
-        rs = pst.executeQuery();
-        while (rs.next()) {
-          targetTxnIdList.add(rs.getLong(1));
+      }
+      int numAborted = new AbortTxnsFunction(txnIds, false, false, false, txnErrorMsg).execute(jdbcResource); 
+      if (numAborted != txnIds.size()) {
+        LOG.warn(
+            "Abort Transactions command only aborted {} out of {} transactions. It's possible that the other"
+                + " {} transactions have been aborted or committed, or the transaction ids are invalid.",
+            numAborted, txnIds.size(), (txnIds.size() - numAborted));
+      }
+
+      if (transactionalListeners != null) {
+        for (Long txnId : txnIds) {
+          List<String> dbsUpdated = getTxnDbsUpdated(txnId);
+          MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners,
+              EventMessage.EventType.ABORT_TXN, new AbortTxnEvent(txnId,
+                  nonReadOnlyTxns.getOrDefault(txnId, TxnType.READ_ONLY), null, dbsUpdated), dbConn, sqlGenerator);
         }
-        closeStmt(pst);
       }
-      LOG.debug("targetTxnid for srcTxnId " + sourceTxnIdList.toString() + " is " + targetTxnIdList.toString());
-      return targetTxnIdList;
-    } finally {
-      closeStmt(pst);
-      close(rs);
-    }
-  }
-
-  @Override
-  @RetrySemantics.Idempotent
-  public long getTargetTxnId(String replPolicy, long sourceTxnId) throws MetaException {
-    try {
-      Connection dbConn = null;
-      try {
-        lockInternal();
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-        List<Long> targetTxnIds = getTargetTxnIdList(replPolicy, Collections.singletonList(sourceTxnId), dbConn);
-        if (targetTxnIds.isEmpty()) {
-          LOG.info("Txn {} not present for repl policy {}", sourceTxnId, replPolicy);
-          return -1;
-        }
-        assert (targetTxnIds.size() == 1);
-        return targetTxnIds.get(0);
-      } catch (SQLException e) {
-        checkRetryable(e, "getTargetTxnId(" + replPolicy + sourceTxnId + ")");
-        throw new MetaException("Unable to get target transaction id "
-                + StringUtils.stringifyException(e));
-      } finally {
-        closeDbConn(dbConn);
-        unlockInternal();
-      }
-    } catch (RetryException e) {
-      return getTargetTxnId(replPolicy, sourceTxnId);
-    }
-  }
-
-  private Set<String> getDbNamesForReplayedTxns(Connection dbConn, List<Long> targetTxnIds) throws SQLException {
-    Set<String> dbNames = new HashSet<>();
-    if (targetTxnIds.isEmpty()) {
-      return dbNames;
-    }
-    PreparedStatement pst = null;
-    ResultSet rs = null;
-    try {
-      List<String> inQueries = new ArrayList<>();
-      StringBuilder prefix = new StringBuilder();
-      prefix.append("SELECT \"RTM_REPL_POLICY\" FROM \"REPL_TXN_MAP\" WHERE ");
-      TxnUtils.buildQueryWithINClause(conf, inQueries, prefix, new StringBuilder(), targetTxnIds,
-              "\"RTM_TARGET_TXN_ID\"", false, false);
-      for (String query : inQueries) {
-        LOG.debug("Going to execute select <{}>", query);
-        pst = sqlGenerator.prepareStmtWithParameters(dbConn, query, null);
-        rs = pst.executeQuery();
-        while (rs.next()) {
-          dbNames.add(MetaStoreUtils.getDbNameFromReplPolicy(rs.getString(1)));
-        }
-      }
-      return dbNames;
-    } finally {
-      closeStmt(pst);
-      close(rs);
-    }
-  }
-
-  private void deleteReplTxnMapEntry(Connection dbConn, long sourceTxnId, String replPolicy) throws SQLException {
-    String s = "DELETE FROM \"REPL_TXN_MAP\" WHERE \"RTM_SRC_TXN_ID\" = " + sourceTxnId + " AND \"RTM_REPL_POLICY\" = ?";
-    try (PreparedStatement pst = sqlGenerator.prepareStmtWithParameters(dbConn, s, Arrays.asList(replPolicy))) {
-      LOG.info("Going to execute  <" + s.replace("?", "{}") + ">", quoteString(replPolicy));
-      pst.executeUpdate();
-    }
-  }
-
-  @Override
-  @RetrySemantics.Idempotent
-  public void abortTxn(AbortTxnRequest rqst) throws NoSuchTxnException, MetaException, TxnAbortedException {
-    long txnid = rqst.getTxnid();
-    TxnErrorMsg txnErrorMsg = TxnErrorMsg.NONE;
-    long sourceTxnId = -1;
-    boolean isReplayedReplTxn = TxnType.REPL_CREATED.equals(rqst.getTxn_type());
-    boolean isHiveReplTxn = rqst.isSetReplPolicy() && TxnType.DEFAULT.equals(rqst.getTxn_type());
-    try {
-      Connection dbConn = null;
-      Statement stmt = null;
-      try {
-        lockInternal();
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-        stmt = dbConn.createStatement();
-
-        if (isReplayedReplTxn) {
-          assert (rqst.isSetReplPolicy());
-          sourceTxnId = rqst.getTxnid();
-          List<Long> targetTxnIds = getTargetTxnIdList(rqst.getReplPolicy(),
-                  Collections.singletonList(sourceTxnId), dbConn);
-          if (targetTxnIds.isEmpty()) {
-            // Idempotent case where txn was already closed or abort txn event received without
-            // corresponding open txn event.
-            LOG.info("Target txn id is missing for source txn id : {} and repl policy {}", sourceTxnId,
-                rqst.getReplPolicy());
-            return;
-          }
-          assert targetTxnIds.size() == 1;
-          txnid = targetTxnIds.get(0);
-        }
-
-        TxnType txnType = getOpenTxnTypeAndLock(stmt, txnid);
-        if (txnType == null) {
-          TxnStatus status = findTxnState(txnid, stmt);
-          if (status == TxnStatus.ABORTED) {
-            if (isReplayedReplTxn) {
-              // in case of replication, idempotent is taken care by getTargetTxnId
-              LOG.warn("Invalid state ABORTED for transactions started using replication replay task");
-              deleteReplTxnMapEntry(dbConn, sourceTxnId, rqst.getReplPolicy());
-            }
-            LOG.info("abortTxn({}) requested by it is already {}", JavaUtils.txnIdToString(txnid), TxnStatus.ABORTED);
-            return;
-          }
-          raiseTxnUnexpectedState(status, txnid);
-        }
-
-        if (isReplayedReplTxn) {
-          txnErrorMsg = TxnErrorMsg.ABORT_REPLAYED_REPL_TXN;
-        } else if (isHiveReplTxn) {
-          txnErrorMsg = TxnErrorMsg.ABORT_DEFAULT_REPL_TXN;
-        } else if (rqst.isSetErrorCode()) {
-          txnErrorMsg = TxnErrorMsg.getTxnErrorMsg(rqst.getErrorCode());
-        }
-
-        abortTxns(dbConn, Collections.singletonList(txnid), true, isReplayedReplTxn, txnErrorMsg);
-
-        if (isReplayedReplTxn) {
-          deleteReplTxnMapEntry(dbConn, sourceTxnId, rqst.getReplPolicy());
-        }
-
-        if (transactionalListeners != null && !isHiveReplTxn) {
-          List<String> dbsUpdated = getTxnDbsUpdated(txnid, dbConn);
-          MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners,
-                  EventMessage.EventType.ABORT_TXN,
-                  new AbortTxnEvent(txnid, txnType, null, dbsUpdated), dbConn, sqlGenerator);
-        }
-
-        LOG.debug("Going to commit");
-        dbConn.commit();
-      } catch (SQLException e) {
-        LOG.debug("Going to rollback: ", e);
-        rollbackDBConn(dbConn);
-        checkRetryable(e, "abortTxn(" + rqst + ")");
-        throw new MetaException("Unable to update transaction database "
-          + StringUtils.stringifyException(e));
-      } finally {
-        close(null, stmt, dbConn);
-        unlockInternal();
-      }
-    } catch (RetryException e) {
-      abortTxn(rqst);
-    }
-  }
-
-  @Override
-  @RetrySemantics.Idempotent
-  public void abortTxns(AbortTxnsRequest rqst) throws MetaException {
-    List<Long> txnIds = rqst.getTxn_ids();
-    TxnErrorMsg txnErrorMsg = TxnErrorMsg.NONE;
-    if (rqst.isSetErrorCode()) {
-      txnErrorMsg = TxnErrorMsg.getTxnErrorMsg(rqst.getErrorCode());
-    }
-    try {
-      Connection dbConn = null;
-      Statement stmt = null;
-      try {
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-        stmt = dbConn.createStatement();
-
-        List<String> queries = new ArrayList<>();
-        StringBuilder prefix =
-            new StringBuilder("SELECT \"TXN_ID\", \"TXN_TYPE\" from \"TXNS\" where \"TXN_STATE\" = ")
-                .append(TxnStatus.OPEN)
-                .append(" and \"TXN_TYPE\" != ").append(TxnType.READ_ONLY.getValue()).append(" and ");
-
-        TxnUtils.buildQueryWithINClause(conf, queries, prefix, new StringBuilder(),
-            txnIds, "\"TXN_ID\"", false, false);
-
-        Map<Long, TxnType> nonReadOnlyTxns = new HashMap<>();
-        for (String query : queries) {
-          LOG.debug("Going to execute query <{}>", query);
-          try (ResultSet rs = stmt.executeQuery(sqlGenerator.addForUpdateClause(query))) {
-            while (rs.next()) {
-              TxnType txnType = TxnType.findByValue(rs.getInt(2));
-              nonReadOnlyTxns.put(rs.getLong(1), txnType);
-            }
-          }
-        }
-        int numAborted = abortTxns(dbConn, txnIds, false, false, txnErrorMsg);
-        if (numAborted != txnIds.size()) {
-          LOG.warn(
-              "Abort Transactions command only aborted {} out of {} transactions. It's possible that the other"
-                  + " {} transactions have been aborted or committed, or the transaction ids are invalid.",
-              numAborted, txnIds.size(), (txnIds.size() - numAborted));
-        }
-
-        if (transactionalListeners != null){
-          for (Long txnId : txnIds) {
-            List<String> dbsUpdated = getTxnDbsUpdated(txnId, dbConn);
-            MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners,
-                    EventMessage.EventType.ABORT_TXN, new AbortTxnEvent(txnId,
-                nonReadOnlyTxns.getOrDefault(txnId, TxnType.READ_ONLY), null, dbsUpdated), dbConn, sqlGenerator);
-          }
-        }
-        LOG.debug("Going to commit");
-        dbConn.commit();
-      } catch (SQLException e) {
-        LOG.debug("Going to rollback: ", e);
-        rollbackDBConn(dbConn);
-        checkRetryable(e, "abortTxns(" + rqst + ")");
-        throw new MetaException("Unable to update transaction database "
-            + StringUtils.stringifyException(e));
-      } finally {
-        closeStmt(stmt);
-        closeDbConn(dbConn);
-      }
-    } catch (RetryException e) {
-      abortTxns(rqst);
-    }
-  }
-
-  private long getDatabaseId(Connection dbConn, String database, String catalog) throws SQLException, MetaException {
-    ResultSet rs = null;
-    PreparedStatement pst = null;
-    try {
-      String query = "select \"DB_ID\" from \"DBS\" where \"NAME\" = ?  and \"CTLG_NAME\" = ?";
-      pst = sqlGenerator.prepareStmtWithParameters(dbConn, query, Arrays.asList(database, catalog));
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Going to execute query <" + query.replace("?", "{}") + ">",
-            quoteString(database), quoteString(catalog));
-      }
-      rs = pst.executeQuery();
-      if (!rs.next()) {
-        throw new MetaException("DB with name " + database + " does not exist in catalog " + catalog);
-      }
-      return rs.getLong(1);
-    } finally {
-      close(rs);
-      closeStmt(pst);
-    }
-  }
-
-  private void updateDatabaseProp(Connection dbConn, String database,
-                                  long dbId, String prop, String propValue) throws SQLException {
-    ResultSet rs = null;
-    PreparedStatement pst = null;
-    try {
-      String query = "SELECT \"PARAM_VALUE\" FROM \"DATABASE_PARAMS\" WHERE \"PARAM_KEY\" = " +
-              "'" + prop + "' AND \"DB_ID\" = " + dbId;
-      pst = sqlGenerator.prepareStmtWithParameters(dbConn, query, null);
-      rs = pst.executeQuery();
-      query = null;
-      if (!rs.next()) {
-        query = "INSERT INTO \"DATABASE_PARAMS\" VALUES ( " + dbId + " , '" + prop + "' , ? )";
-      } else if (!rs.getString(1).equals(propValue)) {
-        query = "UPDATE \"DATABASE_PARAMS\" SET \"PARAM_VALUE\" = ? WHERE \"DB_ID\" = " + dbId +
-                " AND \"PARAM_KEY\" = '" + prop + "'";
-      }
-      closeStmt(pst);
-      if (query == null) {
-        LOG.info("Database property: {} with value: {} already updated for db: {}", prop, propValue, database);
-        return;
-      }
-      pst = sqlGenerator.prepareStmtWithParameters(dbConn, query, Arrays.asList(propValue));
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Updating " + prop + " for db: " + database + " <" + query.replace("?", "{}") + ">", propValue);
-      }
-      if (pst.executeUpdate() != 1) {
-        //only one row insert or update should happen
-        throw new RuntimeException("DATABASE_PARAMS is corrupted for database: " + database);
-      }
-    } finally {
-      close(rs);
-      closeStmt(pst);
-    }
-  }
-
-  private void markDbAsReplIncompatible(Connection dbConn, String database) throws SQLException, MetaException {
-    Statement stmt = null;
-    try {
-      stmt = dbConn.createStatement();
-      String catalog = MetaStoreUtils.getDefaultCatalog(conf);
-      String s = sqlGenerator.getDbProduct().getPrepareTxnStmt();
-      if (s != null) {
-        stmt.execute(s);
-      }
-      long dbId = getDatabaseId(dbConn, database, catalog);
-      updateDatabaseProp(dbConn, database, dbId, ReplConst.REPL_INCOMPATIBLE, ReplConst.TRUE);
-    } finally {
-      closeStmt(stmt);
-    }
-  }
-
-  private void updateReplId(Connection dbConn, ReplLastIdInfo replLastIdInfo) throws SQLException, MetaException {
-    PreparedStatement pst = null;
-    PreparedStatement pstInt = null;
-    ResultSet rs = null;
-    ResultSet prs = null;
-    Statement stmt = null;
-    String query;
-    List<String> params;
-    String lastReplId = Long.toString(replLastIdInfo.getLastReplId());
-    String catalog = replLastIdInfo.isSetCatalog() ? normalizeIdentifier(replLastIdInfo.getCatalog()) :
-            MetaStoreUtils.getDefaultCatalog(conf);
-    String db = normalizeIdentifier(replLastIdInfo.getDatabase());
-    String table = replLastIdInfo.isSetTable() ? normalizeIdentifier(replLastIdInfo.getTable()) : null;
-    List<String> partList = replLastIdInfo.isSetPartitionList() ? replLastIdInfo.getPartitionList() : null;
-
-    try {
-      stmt = dbConn.createStatement();
-
-      String s = sqlGenerator.getDbProduct().getPrepareTxnStmt();
-      if (s != null) {
-        stmt.execute(s);
-      }
-
-      long dbId = getDatabaseId(dbConn, db, catalog);
-
-      // not used select for update as it will be updated by single thread only from repl load
-      updateDatabaseProp(dbConn, db, dbId, ReplConst.REPL_TARGET_TABLE_PROPERTY, lastReplId);
-
-      if (table == null) {
-        // if only database last repl id to be updated.
-        return;
-      }
-
-      query = "SELECT \"TBL_ID\" FROM \"TBLS\" WHERE \"TBL_NAME\" = ? AND \"DB_ID\" = " + dbId;
-      params = Arrays.asList(table);
-      pst = sqlGenerator.prepareStmtWithParameters(dbConn, query, params);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Going to execute query <" + query.replace("?", "{}") + ">", quoteString(table));
-      }
-
-      rs = pst.executeQuery();
-      if (!rs.next()) {
-        throw new MetaException("Table with name " + table + " does not exist in db " + catalog + "." + db);
-      }
-      long tblId = rs.getLong(1);
-      rs.close();
-      pst.close();
-
-      // select for update is not required as only one task will update this during repl load.
-      rs = stmt.executeQuery("SELECT \"PARAM_VALUE\" FROM \"TABLE_PARAMS\" WHERE \"PARAM_KEY\" = " +
-              "'repl.last.id' AND \"TBL_ID\" = " + tblId);
-      if (!rs.next()) {
-        query = "INSERT INTO \"TABLE_PARAMS\" VALUES ( " + tblId + " , 'repl.last.id' , ? )";
-      } else {
-        query = "UPDATE \"TABLE_PARAMS\" SET \"PARAM_VALUE\" = ? WHERE \"TBL_ID\" = " + tblId +
-                " AND \"PARAM_KEY\" = 'repl.last.id'";
-      }
-      rs.close();
-
-      params = Arrays.asList(lastReplId);
-      pst = sqlGenerator.prepareStmtWithParameters(dbConn, query, params);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Updating repl id for table <" + query.replace("?", "{}") + ">", lastReplId);
-      }
-      if (pst.executeUpdate() != 1) {
-        //only one row insert or update should happen
-        throw new RuntimeException("TABLE_PARAMS is corrupted for table " + table);
-      }
-      pst.close();
-
-      if (partList == null || partList.isEmpty()) {
-        return;
-      }
-
-      List<String> questions = new ArrayList<>();
-      for(int i = 0; i < partList.size(); ++i) {
-        questions.add("?");
-      }
-
-      List<String> queries = new ArrayList<>();
-      StringBuilder prefix = new StringBuilder();
-      StringBuilder suffix = new StringBuilder();
-      prefix.append("SELECT \"PART_ID\" FROM \"PARTITIONS\" WHERE \"TBL_ID\" = " + tblId + " and ");
-
-      // Populate the complete query with provided prefix and suffix
-      List<Integer> counts = TxnUtils.buildQueryWithINClauseStrings(conf, queries, prefix, suffix,
-              questions, "\"PART_NAME\"", true, false);
-      int totalCount = 0;
-      assert queries.size() == counts.size();
-      params = Arrays.asList(lastReplId);
-      for (int i = 0; i < queries.size(); i++) {
-        query = queries.get(i);
-        int partCount = counts.get(i);
-
-        LOG.debug("Going to execute query {} with partitions {}", query,
-            partList.subList(totalCount, (totalCount + partCount)));
-        pst = dbConn.prepareStatement(query);
-        for (int j = 0; j < partCount; j++) {
-          pst.setString(j + 1, partList.get(totalCount + j));
-        }
-        totalCount += partCount;
-        prs = pst.executeQuery();
-        while (prs.next()) {
-          long partId = prs.getLong(1);
-          rs = stmt.executeQuery("SELECT \"PARAM_VALUE\" FROM \"PARTITION_PARAMS\" WHERE \"PARAM_KEY\" " +
-                  " = 'repl.last.id' AND \"PART_ID\" = " + partId);
-          if (!rs.next()) {
-            query = "INSERT INTO \"PARTITION_PARAMS\" VALUES ( " + partId + " , 'repl.last.id' , ? )";
-          } else {
-            query = "UPDATE \"PARTITION_PARAMS\" SET \"PARAM_VALUE\" = ? " +
-                    " WHERE \"PART_ID\" = " + partId + " AND \"PARAM_KEY\" = 'repl.last.id'";
-          }
-          rs.close();
-
-          pstInt = sqlGenerator.prepareStmtWithParameters(dbConn, query, params);
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Updating repl id for part <" + query.replace("?", "{}") + ">", lastReplId);
-          }
-          if (pstInt.executeUpdate() != 1) {
-            //only one row insert or update should happen
-            throw new RuntimeException("PARTITION_PARAMS is corrupted for partition " + partId);
-          }
-          partCount--;
-          pstInt.close();
-        }
-        if (partCount != 0) {
-          throw new MetaException(partCount + " Number of partition among " + partList + " does not exist in table " +
-                  catalog + "." + db + "." + table);
-        }
-        prs.close();
-        pst.close();
-      }
-    } finally {
-      closeStmt(stmt);
-      close(rs);
-      close(prs);
-      closeStmt(pst);
-      closeStmt(pstInt);
+    } catch (SQLException e) {
+      throw new UncategorizedSQLException(null, null, e);
     }
   }
 
   /**
    * Concurrency/isolation notes:
-   * This is mutexed with {@link #openTxns(OpenTxnRequest)} and other {@link #commitTxn(CommitTxnRequest)}
+   * This is mutexed with {@link #openTxns(OpenTxnRequest)} and other commitTxn(CommitTxnRequest)
    * operations using select4update on NEXT_TXN_ID. Also, mutexes on TXNS table for specific txnid:X
    * see more notes below.
    * In order to prevent lost updates, we need to determine if any 2 transactions overlap.  Each txn
    * is viewed as an interval [M,N]. M is the txnid and N is taken from the same NEXT_TXN_ID sequence
    * so that we can compare commit time of txn T with start time of txn S.  This sequence can be thought of
-   * as a logical time counter. If S.commitTime < T.startTime, T and S do NOT overlap.
-   *
+   * as a logical time counter. If S.commitTime &lt; T.startTime, T and S do NOT overlap.
+   * <p>
    * Motivating example:
    * Suppose we have multi-statement transactions T and S both of which are attempting x = x + 1
    * In order to prevent lost update problem, then the non-overlapping txns must lock in the snapshot
@@ -1442,248 +578,8 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
    * 'x' would be updated to the same value by both, i.e. lost update.
    */
   @Override
-  @RetrySemantics.Idempotent("No-op if already committed")
   public void commitTxn(CommitTxnRequest rqst) throws NoSuchTxnException, TxnAbortedException, MetaException {
-    char isUpdateDelete = 'N';
-    long txnid = rqst.getTxnid();
-    long sourceTxnId = -1;
-
-    boolean isReplayedReplTxn = TxnType.REPL_CREATED.equals(rqst.getTxn_type());
-    boolean isHiveReplTxn = rqst.isSetReplPolicy() && TxnType.DEFAULT.equals(rqst.getTxn_type());
-    //start a new transaction
-    jdbcResource.bindDataSource(POOL_TX);
-    try (TransactionContext context = jdbcResource.getTransactionManager().getTransaction(PROPAGATION_REQUIRED)) {
-      Connection dbConn = null;
-      Statement stmt = null;
-      Long commitId = null;
-      try {
-        lockInternal();
-        //make sure we are using the connection bound to the transaction, so obtain it via DataSourceUtils.getConnection() 
-        dbConn = jdbcResource.getConnection();
-        stmt = dbConn.createStatement();
-
-        if (rqst.isSetReplLastIdInfo()) {
-          updateReplId(dbConn, rqst.getReplLastIdInfo());
-        }
-
-        if (isReplayedReplTxn) {
-          assert (rqst.isSetReplPolicy());
-          sourceTxnId = rqst.getTxnid();
-          List<Long> targetTxnIds = getTargetTxnIdList(rqst.getReplPolicy(),
-                  Collections.singletonList(sourceTxnId), dbConn);
-          if (targetTxnIds.isEmpty()) {
-            // Idempotent case where txn was already closed or commit txn event received without
-            // corresponding open txn event.
-            LOG.info("Target txn id is missing for source txn id : {} and repl policy {}", sourceTxnId,
-                rqst.getReplPolicy());
-            return;
-          }
-          assert targetTxnIds.size() == 1;
-          txnid = targetTxnIds.get(0);
-        }
-
-        /**
-         * Runs at READ_COMMITTED with S4U on TXNS row for "txnid".  S4U ensures that no other
-         * operation can change this txn (such acquiring locks). While lock() and commitTxn()
-         * should not normally run concurrently (for same txn) but could due to bugs in the client
-         * which could then corrupt internal transaction manager state.  Also competes with abortTxn().
-         */
-        TxnType txnType = getOpenTxnTypeAndLock(stmt, txnid);
-        if (txnType == null) {
-          //if here, txn was not found (in expected state)
-          TxnStatus actualTxnStatus = findTxnState(txnid, stmt);
-          if (actualTxnStatus == TxnStatus.COMMITTED) {
-            if (isReplayedReplTxn) {
-              // in case of replication, idempotent is taken care by getTargetTxnId
-              LOG.warn("Invalid state COMMITTED for transactions started using replication replay task");
-            }
-            /**
-             * This makes the operation idempotent
-             * (assume that this is most likely due to retry logic)
-             */
-            LOG.info("Nth commitTxn({}) msg", JavaUtils.txnIdToString(txnid));
-            return;
-          }
-          raiseTxnUnexpectedState(actualTxnStatus, txnid);
-        }
-
-        String conflictSQLSuffix = "FROM \"TXN_COMPONENTS\" WHERE \"TC_TXNID\"=" + txnid + " AND \"TC_OPERATION_TYPE\" IN (" +
-                OperationType.UPDATE + "," + OperationType.DELETE + ")";
-        long tempCommitId = generateTemporaryId();
-
-        if (txnType == TxnType.SOFT_DELETE || txnType == TxnType.COMPACTION) {
-          acquireTxnLock(stmt, false);
-          commitId = getHighWaterMark(stmt);
-
-        } else if (txnType != TxnType.READ_ONLY && !isReplayedReplTxn) {
-          String writeSetInsertSql = "INSERT INTO \"WRITE_SET\" (\"WS_DATABASE\", \"WS_TABLE\", \"WS_PARTITION\"," +
-            "   \"WS_TXNID\", \"WS_COMMIT_ID\", \"WS_OPERATION_TYPE\")" +
-            " SELECT DISTINCT \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\", \"TC_TXNID\", " + tempCommitId + ", \"TC_OPERATION_TYPE\" ";
-
-          if (isUpdateOrDelete(stmt, conflictSQLSuffix)) {
-            isUpdateDelete = 'Y';
-            //if here it means currently committing txn performed update/delete and we should check WW conflict
-            /**
-             * "select distinct" is used below because
-             * 1. once we get to multi-statement txns, we only care to record that something was updated once
-             * 2. if {@link #addDynamicPartitions(AddDynamicPartitions)} is retried by caller it may create
-             *  duplicate entries in TXN_COMPONENTS
-             * but we want to add a PK on WRITE_SET which won't have unique rows w/o this distinct
-             * even if it includes all of its columns
-             *
-             * First insert into write_set using a temporary commitID, which will be updated in a separate call,
-             * see: {@link #updateWSCommitIdAndCleanUpMetadata(Statement, long, TxnType, Long, long)}}.
-             * This should decrease the scope of the S4U lock on the next_txn_id table.
-             */
-            Object undoWriteSetForCurrentTxn = context.getTransactionStatus().createSavepoint();
-            stmt.executeUpdate(writeSetInsertSql + (useMinHistoryLevel ? conflictSQLSuffix :
-              "FROM \"TXN_COMPONENTS\" WHERE \"TC_TXNID\"=" + txnid + " AND \"TC_OPERATION_TYPE\" <> " + OperationType.COMPACT));
-
-            /**
-             * This S4U will mutex with other commitTxn() and openTxns().
-             * -1 below makes txn intervals look like [3,3] [4,4] if all txns are serial
-             * Note: it's possible to have several txns have the same commit id.  Suppose 3 txns start
-             * at the same time and no new txns start until all 3 commit.
-             * We could've incremented the sequence for commitId as well but it doesn't add anything functionally.
-             */
-            acquireTxnLock(stmt, false);
-            commitId = getHighWaterMark(stmt);
-
-            if (!rqst.isExclWriteEnabled()) {
-              /**
-               * see if there are any overlapping txns that wrote the same element, i.e. have a conflict
-               * Since entire commit operation is mutexed wrt other start/commit ops,
-               * committed.ws_commit_id <= current.ws_commit_id for all txns
-               * thus if committed.ws_commit_id < current.ws_txnid, transactions do NOT overlap
-               * For example, [17,20] is committed, [6,80] is being committed right now - these overlap
-               * [17,20] committed and [21,21] committing now - these do not overlap.
-               * [17,18] committed and [18,19] committing now - these overlap  (here 18 started while 17 was still running)
-               */
-              try (ResultSet rs = checkForWriteConflict(stmt, txnid)) {
-                if (rs.next()) {
-                  //found a conflict, so let's abort the txn
-                  String committedTxn = "[" + JavaUtils.txnIdToString(rs.getLong(1)) + "," + rs.getLong(2) + "]";
-                  StringBuilder resource = new StringBuilder(rs.getString(3)).append("/").append(rs.getString(4));
-                  String partitionName = rs.getString(5);
-                  if (partitionName != null) {
-                    resource.append('/').append(partitionName);
-                  }
-                  String msg = "Aborting [" + JavaUtils.txnIdToString(txnid) + "," + commitId + "]" + " due to a write conflict on " + resource +
-                          " committed by " + committedTxn + " " + rs.getString(7) + "/" + rs.getString(8);
-                  //remove WRITE_SET info for current txn since it's about to abort
-                  context.getTransactionStatus().rollbackToSavepoint(undoWriteSetForCurrentTxn);
-                  LOG.info(msg);
-                  //todo: should make abortTxns() write something into TXNS.TXN_META_INFO about this
-                  if (abortTxns(dbConn, Collections.singletonList(txnid), false, isReplayedReplTxn,
-                          TxnErrorMsg.ABORT_WRITE_CONFLICT) != 1) {
-                    throw new IllegalStateException(msg + " FAILED!");
-                  }
-                  jdbcResource.getTransactionManager().commit(context);
-                  throw new TxnAbortedException(msg);
-                }
-              }
-            }
-          } else if (!useMinHistoryLevel) {
-            stmt.executeUpdate(writeSetInsertSql + "FROM \"TXN_COMPONENTS\" WHERE \"TC_TXNID\"=" + txnid +
-              " AND \"TC_OPERATION_TYPE\" <> " + OperationType.COMPACT);
-            commitId = getHighWaterMark(stmt);
-          }
-        } else {
-          /*
-           * current txn didn't update/delete anything (may have inserted), so just proceed with commit
-           *
-           * We only care about commit id for write txns, so for RO (when supported) txns we don't
-           * have to mutex on NEXT_TXN_ID.
-           * Consider: if RO txn is after a W txn, then RO's openTxns() will be mutexed with W's
-           * commitTxn() because both do S4U on NEXT_TXN_ID and thus RO will see result of W txn.
-           * If RO < W, then there is no reads-from relationship.
-           * In replication flow we don't expect any write write conflict as it should have been handled at source.
-           */
-          assert true;
-        }
-
-        
-        if (txnType != TxnType.READ_ONLY && !isReplayedReplTxn && !MetaStoreServerUtils.isCompactionTxn(txnType)) {
-          moveTxnComponentsToCompleted(stmt, txnid, isUpdateDelete);
-        } else if (isReplayedReplTxn) {
-          if (rqst.isSetWriteEventInfos()) {
-            String sql = String.format(COMPL_TXN_COMPONENTS_INSERT_QUERY, txnid, quoteChar(isUpdateDelete));
-            try (PreparedStatement pstmt = dbConn.prepareStatement(sql)) {
-              int insertCounter = 0;
-              for (WriteEventInfo writeEventInfo : rqst.getWriteEventInfos()) {
-                pstmt.setString(1, writeEventInfo.getDatabase());
-                pstmt.setString(2, writeEventInfo.getTable());
-                pstmt.setString(3, writeEventInfo.getPartition());
-                pstmt.setLong(4, writeEventInfo.getWriteId());
-
-                pstmt.addBatch();
-                insertCounter++;
-                if (insertCounter % maxBatchSize == 0) {
-                  LOG.debug("Executing a batch of <{}> queries. Batch size: {}", sql, maxBatchSize);
-                  pstmt.executeBatch();
-                }
-              }
-              if (insertCounter % maxBatchSize != 0) {
-                LOG.debug("Executing a batch of <{}> queries. Batch size: {}", sql, insertCounter % maxBatchSize);
-                pstmt.executeBatch();
-              }
-            }
-          }
-          deleteReplTxnMapEntry(dbConn, sourceTxnId, rqst.getReplPolicy());
-        }
-        updateWSCommitIdAndCleanUpMetadata(stmt, txnid, txnType, commitId, tempCommitId);
-        removeTxnsFromMinHistoryLevel(dbConn, ImmutableList.of(txnid));
-        removeWriteIdsFromMinHistory(dbConn, ImmutableList.of(txnid));
-        if (rqst.isSetKeyValue()) {
-          updateKeyValueAssociatedWithTxn(rqst, stmt);
-        }
-
-        if (!isHiveReplTxn) {
-          createCommitNotificationEvent(dbConn, txnid , txnType);
-        }
-
-        LOG.debug("Going to commit");
-        jdbcResource.getTransactionManager().commit(context);
-
-        if (MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON)) {
-          Metrics.getOrCreateCounter(MetricsConstants.TOTAL_NUM_COMMITTED_TXNS).inc();
-        }
-      } catch (SQLException e) {
-        LOG.debug("Going to rollback: ", e);
-        jdbcResource.getTransactionManager().rollback(context);
-        checkRetryable(e, "commitTxn(" + rqst + ")");
-        throw new MetaException("Unable to update transaction database "
-          + StringUtils.stringifyException(e));
-      } finally {
-        closeStmt(stmt);
-        unlockInternal();
-      }
-    } catch (RetryException e) {
-      commitTxn(rqst);
-    } finally {
-      jdbcResource.unbindDataSource();
-    }
-  }
-
-  /**
-   * Create Notifiaction Events on txn commit
-   * @param txnid committed txn
-   * @param txnType transaction type
-   * @throws MetaException ex
-   */
-  protected void createCommitNotificationEvent(Connection conn, long txnid, TxnType txnType)
-      throws MetaException, SQLException {
-    if (transactionalListeners != null) {
-      MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners,
-          EventMessage.EventType.COMMIT_TXN, new CommitTxnEvent(txnid, txnType), conn, sqlGenerator);
-    }
-  }
-
-  private boolean isUpdateOrDelete(Statement stmt, String conflictSQLSuffix) throws SQLException, MetaException {
-    try (ResultSet rs = stmt.executeQuery(sqlGenerator.addLimitClause(1,
-            "\"TC_OPERATION_TYPE\" " + conflictSQLSuffix))) {
-      return rs.next();
-    }
+    new CommitTxnFunction(rqst, transactionalListeners).execute(jdbcResource);
   }
 
   /**
@@ -1694,4085 +590,364 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
    * @return max Id for the conflicting transaction, if any, otherwise -1
    * @throws MetaException
    */
-  @RetrySemantics.ReadOnly
+  @Override
   public long getLatestTxnIdInConflict(long txnid) throws MetaException {
-    try {
-      try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-           Statement stmt = dbConn.createStatement()) {
-
-        String writeConflictQuery = "SELECT MAX(\"COMMITTED\".\"WS_TXNID\")" +
-          " FROM \"WRITE_SET\" \"COMMITTED\"" +
-          " INNER JOIN (" +
-          "   SELECT DISTINCT \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\", \"TC_TXNID\"" +
-          "   FROM \"TXN_COMPONENTS\"" +
-          "   WHERE \"TC_TXNID\" = " + txnid +
-          "     AND \"TC_OPERATION_TYPE\" IN (" + OperationType.UPDATE + "," + OperationType.DELETE + ")" +
-          " ) \"CUR\"" +
-          " ON \"COMMITTED\".\"WS_DATABASE\" = \"CUR\".\"TC_DATABASE\"" +
-          "   AND \"COMMITTED\".\"WS_TABLE\" = \"CUR\".\"TC_TABLE\"" +
-          (useMinHistoryLevel ? "" :
-          "   AND \"COMMITTED\".\"WS_OPERATION_TYPE\" != " + OperationType.INSERT) +
-          // For partitioned table we always track writes at partition level (never at table)
-          // and for non partitioned - always at table level, thus the same table should never
-          // have entries with partition key and w/o
-          "   AND (\"COMMITTED\".\"WS_PARTITION\" = \"CUR\".\"TC_PARTITION\" OR" +
-          "     \"CUR\".\"TC_PARTITION\" IS NULL) " +
-          // txns overlap
-          " WHERE \"CUR\".\"TC_TXNID\" <= \"COMMITTED\".\"WS_COMMIT_ID\"";
-
-        LOG.debug("Going to execute query: <{}>", writeConflictQuery);
-        try (ResultSet rs = stmt.executeQuery(writeConflictQuery)) {
-          return rs.next() ? rs.getLong(1) : -1;
-        }
-      } catch (SQLException e) {
-        checkRetryable(e, "getLatestTxnIdInConflict");
-        throw new MetaException(StringUtils.stringifyException(e));
-      }
-    } catch (RetryException e) {
-      return getLatestTxnIdInConflict(txnid);
-    }
-  }
-
-  /**
-   * Returns the databases updated by txnId.
-   * Queries TXN_TO_WRITE_ID using txnId.
-   *
-   * @param txnId
-   * @throws MetaException
-   */
-    private List<String> getTxnDbsUpdated(long txnId, Connection dbConn) throws MetaException {
-    try {
-      try (Statement stmt = dbConn.createStatement()) {
-
-        String query = "SELECT DISTINCT \"T2W_DATABASE\" " +
-                " FROM \"TXN_TO_WRITE_ID\" \"COMMITTED\"" +
-                "   WHERE \"T2W_TXNID\" = " + txnId;
-
-        LOG.debug("Going to execute query: <{}>", query);
-        try (ResultSet rs = stmt.executeQuery(query)) {
-          List<String> dbsUpdated = new ArrayList<String>();
-          while (rs.next()) {
-            dbsUpdated.add(rs.getString(1));
-          }
-          return dbsUpdated;
-        }
-      } catch (SQLException e) {
-        checkRetryable(e, "getTxnDbsUpdated");
-        throw new MetaException(StringUtils.stringifyException(e));
-      }
-    } catch (RetryException e) {
-      return getTxnDbsUpdated(txnId, dbConn);
-    }
-  }
-
-
-  private ResultSet checkForWriteConflict(Statement stmt, long txnid) throws SQLException, MetaException {
-    String writeConflictQuery = sqlGenerator.addLimitClause(1, "\"COMMITTED\".\"WS_TXNID\", \"COMMITTED\".\"WS_COMMIT_ID\", " +
-            "\"COMMITTED\".\"WS_DATABASE\", \"COMMITTED\".\"WS_TABLE\", \"COMMITTED\".\"WS_PARTITION\", " +
-            "\"CUR\".\"WS_COMMIT_ID\" \"CUR_WS_COMMIT_ID\", \"CUR\".\"WS_OPERATION_TYPE\" \"CUR_OP\", " +
-            "\"COMMITTED\".\"WS_OPERATION_TYPE\" \"COMMITTED_OP\" FROM \"WRITE_SET\" \"COMMITTED\" INNER JOIN \"WRITE_SET\" \"CUR\" " +
-            "ON \"COMMITTED\".\"WS_DATABASE\"=\"CUR\".\"WS_DATABASE\" AND \"COMMITTED\".\"WS_TABLE\"=\"CUR\".\"WS_TABLE\" " +
-            //For partitioned table we always track writes at partition level (never at table)
-            //and for non partitioned - always at table level, thus the same table should never
-            //have entries with partition key and w/o
-            "AND (\"COMMITTED\".\"WS_PARTITION\"=\"CUR\".\"WS_PARTITION\" OR (\"COMMITTED\".\"WS_PARTITION\" IS NULL AND \"CUR\".\"WS_PARTITION\" IS NULL)) " +
-            "WHERE \"CUR\".\"WS_TXNID\" <= \"COMMITTED\".\"WS_COMMIT_ID\"" + //txns overlap; could replace ws_txnid
-            // with txnid, though any decent DB should infer this
-            " AND \"CUR\".\"WS_TXNID\"=" + txnid + //make sure RHS of join only has rows we just inserted as
-            // part of this commitTxn() op
-            " AND \"COMMITTED\".\"WS_TXNID\" <> " + txnid + //and LHS only has committed txns
-            //U+U and U+D and D+D is a conflict and we don't currently track I in WRITE_SET at all
-            //it may seem like D+D should not be in conflict but consider 2 multi-stmt txns
-            //where each does "delete X + insert X, where X is a row with the same PK.  This is
-            //equivalent to an update of X but won't be in conflict unless D+D is in conflict.
-            //The same happens when Hive splits U=I+D early so it looks like 2 branches of a
-            //multi-insert stmt (an Insert and a Delete branch).  It also 'feels'
-            // un-serializable to allow concurrent deletes
-            " and (\"COMMITTED\".\"WS_OPERATION_TYPE\" IN(" + OperationType.UPDATE +
-            ", " + OperationType.DELETE +
-            ") AND \"CUR\".\"WS_OPERATION_TYPE\" IN(" + OperationType.UPDATE+ ", "
-            + OperationType.DELETE + "))");
-    LOG.debug("Going to execute query: <{}>", writeConflictQuery);
-    return stmt.executeQuery(writeConflictQuery);
-  }
-
-  private void moveTxnComponentsToCompleted(Statement stmt, long txnid, char isUpdateDelete) throws SQLException {
-    // Move the record from txn_components into completed_txn_components so that the compactor
-    // knows where to look to compact.
-    String s = "INSERT INTO \"COMPLETED_TXN_COMPONENTS\" (\"CTC_TXNID\", \"CTC_DATABASE\", " +
-            "\"CTC_TABLE\", \"CTC_PARTITION\", \"CTC_WRITEID\", \"CTC_UPDATE_DELETE\") SELECT \"TC_TXNID\"," +
-        " \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\", \"TC_WRITEID\", '" + isUpdateDelete +
-        "' FROM \"TXN_COMPONENTS\" WHERE \"TC_TXNID\" = " + txnid +
-        //we only track compactor activity in TXN_COMPONENTS to handle the case where the
-        //compactor txn aborts - so don't bother copying it to COMPLETED_TXN_COMPONENTS
-        " AND \"TC_OPERATION_TYPE\" <> " + OperationType.COMPACT;
-    LOG.debug("Going to execute insert <{}>", s);
-
-    if ((stmt.executeUpdate(s)) < 1) {
-      //this can be reasonable for an empty txn START/COMMIT or read-only txn
-      //also an IUD with DP that didn't match any rows.
-      LOG.info("Expected to move at least one record from txn_components to "
-          + "completed_txn_components when committing txn! {}", JavaUtils.txnIdToString(txnid));
-    }
-  }
-
-  /**
-   * See overridden method in CompactionTxnHandler also.
-   */
-  protected void updateWSCommitIdAndCleanUpMetadata(Statement stmt, long txnid, TxnType txnType,
-      Long commitId, long tempId) throws SQLException, MetaException {
-    List<String> queryBatch = new ArrayList<>(5);
-    // update write_set with real commitId
-    if (commitId != null) {
-      queryBatch.add("UPDATE \"WRITE_SET\" SET \"WS_COMMIT_ID\" = " + commitId +
-              " WHERE \"WS_COMMIT_ID\" = " + tempId + " AND \"WS_TXNID\" = " + txnid);
-    }
-    // clean up txn related metadata
-    if (txnType != TxnType.READ_ONLY) {
-      queryBatch.add("DELETE FROM \"TXN_COMPONENTS\" WHERE \"TC_TXNID\" = " + txnid);
-    }
-    queryBatch.add("DELETE FROM \"HIVE_LOCKS\" WHERE \"HL_TXNID\" = " + txnid);
-    // DO NOT remove the transaction from the TXN table, the cleaner will remove it when appropriate
-    queryBatch.add("UPDATE \"TXNS\" SET \"TXN_STATE\" = " + TxnStatus.COMMITTED + " WHERE \"TXN_ID\" = " + txnid);
-    if (txnType == TxnType.MATER_VIEW_REBUILD) {
-      queryBatch.add("DELETE FROM \"MATERIALIZATION_REBUILD_LOCKS\" WHERE \"MRL_TXN_ID\" = " + txnid);
-    }
-    // execute all in one batch
-    executeQueriesInBatchNoCount(dbProduct, stmt, queryBatch, maxBatchSize);
-  }
-
-  private void updateKeyValueAssociatedWithTxn(CommitTxnRequest rqst, Statement stmt) throws SQLException {
-    if (!rqst.getKeyValue().getKey().startsWith(TxnStore.TXN_KEY_START)) {
-      String errorMsg = "Error updating key/value in the sql backend with"
-          + " txnId=" + rqst.getTxnid() + ","
-          + " tableId=" + rqst.getKeyValue().getTableId() + ","
-          + " key=" + rqst.getKeyValue().getKey() + ","
-          + " value=" + rqst.getKeyValue().getValue() + "."
-          + " key should start with " + TXN_KEY_START + ".";
-      LOG.warn(errorMsg);
-      throw new IllegalArgumentException(errorMsg);
-    }
-    String s = "UPDATE \"TABLE_PARAMS\" SET"
-        + " \"PARAM_VALUE\" = " + quoteString(rqst.getKeyValue().getValue())
-        + " WHERE \"TBL_ID\" = " + rqst.getKeyValue().getTableId()
-        + " AND \"PARAM_KEY\" = " + quoteString(rqst.getKeyValue().getKey());
-    LOG.debug("Going to execute update <{}>", s);
-    int affectedRows = stmt.executeUpdate(s);
-    if (affectedRows != 1) {
-      String errorMsg = "Error updating key/value in the sql backend with"
-          + " txnId=" + rqst.getTxnid() + ","
-          + " tableId=" + rqst.getKeyValue().getTableId() + ","
-          + " key=" + rqst.getKeyValue().getKey() + ","
-          + " value=" + rqst.getKeyValue().getValue() + "."
-          + " Only one row should have been affected but "
-          + affectedRows + " rows where affected.";
-      LOG.warn(errorMsg);
-      throw new IllegalStateException(errorMsg);
-    }
+    return jdbcResource.execute(new LatestTxnIdInConflictHandler(txnid));
   }
 
   /**
-   * Replicate Table Write Ids state to mark aborted write ids and writeid high water mark.
+   * Replicate Table Write Ids state to mark aborted write ids and writeid high watermark.
    * @param rqst info on table/partitions and writeid snapshot to replicate.
    * @throws MetaException
    */
-  @Override
-  @RetrySemantics.Idempotent("No-op if already replicated the writeid state")
-  public void replTableWriteIdState(ReplTblWriteIdStateRequest rqst) throws MetaException {
-    String dbName = rqst.getDbName().toLowerCase();
-    String tblName = rqst.getTableName().toLowerCase();
-    ValidWriteIdList validWriteIdList = new ValidReaderWriteIdList(rqst.getValidWriteIdlist());
-
-    // Get the abortedWriteIds which are already sorted in ascending order.
-    List<Long> abortedWriteIds = getAbortedWriteIds(validWriteIdList);
-    int numAbortedWrites = abortedWriteIds.size();
-    try {
-      Connection dbConn = null;
-      Statement stmt = null;
-      PreparedStatement pStmt = null;
-      List<PreparedStatement> insertPreparedStmts = null;
-      ResultSet rs = null;
-      List<String> params = Arrays.asList(dbName, tblName);
-      try {
-        lockInternal();
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-        stmt = dbConn.createStatement();
-
-        // Check if this txn state is already replicated for this given table. If yes, then it is
-        // idempotent case and just return.
-        String sql = "SELECT \"NWI_NEXT\" FROM \"NEXT_WRITE_ID\" WHERE \"NWI_DATABASE\" = ? AND \"NWI_TABLE\" = ?";
-        pStmt = sqlGenerator.prepareStmtWithParameters(dbConn, sql, params);
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Going to execute query <" + sql.replace("?", "{}") + ">",
-              quoteString(dbName), quoteString(tblName));
-        }
-        rs = pStmt.executeQuery();
-        if (rs.next()) {
-          LOG.info("Idempotent flow: WriteId state <{}> is already applied for the table: {}.{}", validWriteIdList,
-              dbName, tblName);
-          rollbackDBConn(dbConn);
-          return;
-        }
-
-        if (numAbortedWrites > 0) {
-          // Allocate/Map one txn per aborted writeId and abort the txn to mark writeid as aborted.
-          // We don't use the txnLock, all of these transactions will be aborted in this one rdbm transaction
-          // So they will not effect the commitTxn in any way
-          List<Long> txnIds = openTxns(dbConn,
-                  new OpenTxnRequest(numAbortedWrites, rqst.getUser(), rqst.getHostName()));
-          assert(numAbortedWrites == txnIds.size());
-
-          // Map each aborted write id with each allocated txn.
-          List<String> rows = new ArrayList<>();
-          List<List<String>> paramsList = new ArrayList<>();
-          int i = 0;
-          for (long txn : txnIds) {
-            long writeId = abortedWriteIds.get(i++);
-            rows.add(txn + ", ?, ?, " + writeId);
-            paramsList.add(params);
-            LOG.info("Allocated writeID: {} for txnId: {}", writeId, txn);
-          }
-
-          // Insert entries to TXN_TO_WRITE_ID for aborted write ids
-          insertPreparedStmts = sqlGenerator.createInsertValuesPreparedStmt(dbConn,
-                  "\"TXN_TO_WRITE_ID\" (\"T2W_TXNID\", \"T2W_DATABASE\", \"T2W_TABLE\", \"T2W_WRITEID\")", rows,
-                  paramsList);
-          for (PreparedStatement pst : insertPreparedStmts) {
-            pst.execute();
-          }
-
-          // Abort all the allocated txns so that the mapped write ids are referred as aborted ones.
-          int numAborts = abortTxns(dbConn, txnIds, false, false, TxnErrorMsg.ABORT_REPL_WRITEID_TXN);
-          assert(numAborts == numAbortedWrites);
-        }
-
-        // There are some txns in the list which has no write id allocated and hence go ahead and do it.
-        // Get the next write id for the given table and update it with new next write id.
-        // It is expected NEXT_WRITE_ID doesn't have entry for this table and hence directly insert it.
-        long nextWriteId = validWriteIdList.getHighWatermark() + 1;
-
-        // First allocation of write id (hwm+1) should add the table to the next_write_id meta table.
-        sql = "INSERT INTO \"NEXT_WRITE_ID\" (\"NWI_DATABASE\", \"NWI_TABLE\", \"NWI_NEXT\") VALUES (?, ?, "
-                + Long.toString(nextWriteId) + ")";
-        closeStmt(pStmt);
-        pStmt = sqlGenerator.prepareStmtWithParameters(dbConn, sql, params);
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Going to execute insert <" + sql.replace("?", "{}") + ">",
-              quoteString(dbName), quoteString(tblName));
-        }
-        pStmt.execute();
-
-        LOG.info("WriteId state <{}> is applied for the table: {}.{}", validWriteIdList, dbName, tblName);
-        LOG.debug("Going to commit");
-        dbConn.commit();
-      } catch (SQLException e) {
-        LOG.debug("Going to rollback: ", e);
-        rollbackDBConn(dbConn);
-        checkRetryable(e, "replTableWriteIdState(" + rqst + ")", true);
-        throw new MetaException("Unable to update transaction database "
-                + StringUtils.stringifyException(e));
-      } finally {
-        if (insertPreparedStmts != null) {
-          for (PreparedStatement pst : insertPreparedStmts) {
-            closeStmt(pst);
-          }
-        }
-        closeStmt(pStmt);
-        close(rs, stmt, dbConn);
-        unlockInternal();
-      }
-    } catch (RetryException e) {
-      replTableWriteIdState(rqst);
-    }
-
-    // Schedule Major compaction on all the partitions/table to clean aborted data
-    if (numAbortedWrites > 0) {
-      CompactionRequest compactRqst = new CompactionRequest(rqst.getDbName(), rqst.getTableName(),
-              CompactionType.MAJOR);
-      if (rqst.isSetPartNames()) {
-        for (String partName : rqst.getPartNames()) {
-          compactRqst.setPartitionname(partName);
-          compact(compactRqst);
-        }
-      } else {
-        compact(compactRqst);
-      }
-    }
-  }
-
-  private List<Long> getAbortedWriteIds(ValidWriteIdList validWriteIdList) {
-    return Arrays.stream(validWriteIdList.getInvalidWriteIds())
-        .filter(validWriteIdList::isWriteIdAborted)
-        .boxed()
-        .collect(Collectors.toList());
-  }
-
-  private ValidTxnList getValidTxnList(Connection dbConn, String fullTableName, Long writeId) throws MetaException,
-          SQLException {
-    PreparedStatement pst = null;
-    ResultSet rs = null;
-    try {
-      String[] names = TxnUtils.getDbTableName(fullTableName);
-      assert names.length == 2;
-      List<String> params = Arrays.asList(names[0], names[1]);
-      String s =
-          "SELECT \"T2W_TXNID\" FROM \"TXN_TO_WRITE_ID\" WHERE \"T2W_DATABASE\" = ? AND "
-              + "\"T2W_TABLE\" = ? AND \"T2W_WRITEID\" = "+ writeId;
-      pst = sqlGenerator.prepareStmtWithParameters(dbConn, s, params);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Going to execute query <" + s.replace("?", "{}") + ">", quoteString(names[0]),
-            quoteString(names[1]));
-      }
-      rs = pst.executeQuery();
-      if (rs.next()) {
-        long txnId = rs.getLong(1);
-        return TxnCommonUtils.createValidReadTxnList(getOpenTxns(dbConn), txnId);
-      }
-      throw new MetaException("invalid write id " + writeId + " for table " + fullTableName);
-    } finally {
-      close(rs, pst, null);
-    }
-  }
-
-  @Override
-  @RetrySemantics.ReadOnly
-  public GetValidWriteIdsResponse getValidWriteIds(GetValidWriteIdsRequest rqst) throws MetaException {
-    try {
-      Connection dbConn = null;
-      ValidTxnList validTxnList;
-
-      try {
-        /**
-         * This runs at READ_COMMITTED for exactly the same reason as {@link #getOpenTxnsInfo()}
-         */
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-
-        // We should prepare the valid write ids list based on validTxnList of current txn.
-        // If no txn exists in the caller, then they would pass null for validTxnList and so it is
-        // required to get the current state of txns to make validTxnList
-        if (rqst.isSetValidTxnList()) {
-          assert rqst.isSetWriteId() == false;
-          validTxnList = new ValidReadTxnList(rqst.getValidTxnList());
-        } else if (rqst.isSetWriteId()) {
-          validTxnList = getValidTxnList(dbConn, rqst.getFullTableNames().get(0), rqst.getWriteId());
-        } else {
-          // Passing 0 for currentTxn means, this validTxnList is not wrt to any txn
-          validTxnList = TxnCommonUtils.createValidReadTxnList(getOpenTxns(dbConn), 0);
-        }
-
-        // Get the valid write id list for all the tables read by the current txn
-        List<TableValidWriteIds> tblValidWriteIdsList = new ArrayList<>();
-        for (String fullTableName : rqst.getFullTableNames()) {
-          tblValidWriteIdsList.add(getValidWriteIdsForTable(dbConn, fullTableName, validTxnList));
-        }
-
-        GetValidWriteIdsResponse owr = new GetValidWriteIdsResponse(tblValidWriteIdsList);
-        return owr;
-      } catch (SQLException e) {
-        checkRetryable(e, "getValidWriteIds");
-        throw new MetaException("Unable to select from transaction database, "
-                + StringUtils.stringifyException(e));
-      } finally {
-        closeDbConn(dbConn);
-      }
-    } catch (RetryException e) {
-      return getValidWriteIds(rqst);
-    }
-  }
-  
-  // Method to get the Valid write ids list for the given table
-  // Input fullTableName is expected to be of format <db_name>.<table_name>
-  private TableValidWriteIds getValidWriteIdsForTable(Connection dbConn, String fullTableName,
-                                               ValidTxnList validTxnList) throws SQLException {
-    PreparedStatement pst = null;
-    ResultSet rs = null;
-    String[] names = TxnUtils.getDbTableName(fullTableName);
-    assert(names.length == 2);
-    List<String> params = Arrays.asList(names[0], names[1]);
-    try {
-      // Need to initialize to 0 to make sure if nobody modified this table, then current txn
-      // shouldn't read any data.
-      // If there is a conversion from non-acid to acid table, then by default 0 would be assigned as
-      // writeId for data from non-acid table and so writeIdHwm=0 would ensure those data are readable by any txns.
-      long writeIdHwm = 0;
-      List<Long> invalidWriteIdList = new ArrayList<>();
-      long minOpenWriteId = Long.MAX_VALUE;
-      BitSet abortedBits = new BitSet();
-      long txnHwm = validTxnList.getHighWatermark();
-
-      // Find the writeId high water mark based upon txnId high water mark. If found, then, need to
-      // traverse through all write Ids less than writeId HWM to make exceptions list.
-      // The writeHWM = min(NEXT_WRITE_ID.nwi_next-1, max(TXN_TO_WRITE_ID.t2w_writeid under txnHwm))
-      String s = "SELECT MAX(\"T2W_WRITEID\") FROM \"TXN_TO_WRITE_ID\" WHERE \"T2W_TXNID\" <= " + txnHwm
-              + " AND \"T2W_DATABASE\" = ? AND \"T2W_TABLE\" = ?";
-      pst = sqlGenerator.prepareStmtWithParameters(dbConn, s, params);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Going to execute query<" + s.replace("?", "{}") + ">",
-            quoteString(names[0]), quoteString(names[1]));
-      }
-      rs = pst.executeQuery();
-      if (rs.next()) {
-        writeIdHwm = rs.getLong(1);
-      }
-
-      // If no writeIds allocated by txns under txnHwm, then find writeHwm from NEXT_WRITE_ID.
-      if (writeIdHwm <= 0) {
-        // Need to subtract 1 as nwi_next would be the next write id to be allocated but we need highest
-        // allocated write id.
-        s = "SELECT \"NWI_NEXT\"-1 FROM \"NEXT_WRITE_ID\" WHERE \"NWI_DATABASE\" = ? AND \"NWI_TABLE\" = ?";
-        closeStmt(pst);
-        pst = sqlGenerator.prepareStmtWithParameters(dbConn, s, params);
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Going to execute query<" + s.replace("?", "{}") + ">",
-              quoteString(names[0]), quoteString(names[1]));
-        }
-        rs = pst.executeQuery();
-        if (rs.next()) {
-          writeIdHwm = rs.getLong(1);
-        }
-      }
-      boolean foundValidUncompactedWrite = false;
-      // As writeIdHwm is known, query all writeIds under the writeId HWM.
-      // If any writeId under HWM is allocated by txn > txnId HWM or belongs to open/aborted txns,
-      // then will be added to invalid list. The results should be sorted in ascending order based
-      // on write id. The sorting is needed as exceptions list in ValidWriteIdList would be looked-up
-      // using binary search.
-      s = "SELECT \"T2W_TXNID\", \"T2W_WRITEID\" FROM \"TXN_TO_WRITE_ID\" WHERE \"T2W_WRITEID\" <= " + Long.toString(writeIdHwm)
-              + " AND \"T2W_DATABASE\" = ? AND \"T2W_TABLE\" = ? ORDER BY \"T2W_WRITEID\" ASC";
-      closeStmt(pst);
-      pst = sqlGenerator.prepareStmtWithParameters(dbConn, s, params);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Going to execute query<" + s.replace("?", "{}") + ">",
-            quoteString(names[0]), quoteString(names[1]));
-      }
-      rs = pst.executeQuery();
-      while (rs.next()) {
-        long txnId = rs.getLong(1);
-        long writeId = rs.getLong(2);
-        if (validTxnList.isTxnValid(txnId)) {
-          // Skip if the transaction under evaluation is already committed.
-          foundValidUncompactedWrite = true;
-          continue;
-        }
-        // The current txn is either in open or aborted state.
-        // Mark the write ids state as per the txn state.
-        invalidWriteIdList.add(writeId);
-        if (validTxnList.isTxnAborted(txnId)) {
-          abortedBits.set(invalidWriteIdList.size() - 1);
-        } else {
-          minOpenWriteId = Math.min(minOpenWriteId, writeId);
-        }
-      }
-      // If we have compacted writes and some invalid writes on the table,
-      // return the lowest invalid write as a writeIdHwm and set it as invalid.
-      if (!foundValidUncompactedWrite) {
-        long writeId = invalidWriteIdList.isEmpty() ? -1 : invalidWriteIdList.get(0);
-        invalidWriteIdList = new ArrayList<>();
-        abortedBits = new BitSet();
-
-        if (writeId != -1) {
-          invalidWriteIdList.add(writeId);
-          writeIdHwm = writeId;
-          if (writeId != minOpenWriteId) {
-            abortedBits.set(0);
-          }
-        }
-      }
-      ByteBuffer byteBuffer = ByteBuffer.wrap(abortedBits.toByteArray());
-      TableValidWriteIds owi = new TableValidWriteIds(fullTableName, writeIdHwm, invalidWriteIdList, byteBuffer);
-      if (minOpenWriteId < Long.MAX_VALUE) {
-        owi.setMinOpenWriteId(minOpenWriteId);
-      }
-      return owi;
-    } finally {
-      closeStmt(pst);
-      close(rs);
-    }
-  }
-
-  @Override
-  @RetrySemantics.Idempotent
-  public AllocateTableWriteIdsResponse allocateTableWriteIds(AllocateTableWriteIdsRequest rqst)
-          throws MetaException {
-    List<Long> txnIds;
-    String dbName = rqst.getDbName().toLowerCase();
-    String tblName = rqst.getTableName().toLowerCase();
-    boolean shouldReallocate = rqst.isReallocate();
-    try {
-      Connection dbConn = null;
-      PreparedStatement pStmt = null;
-      ResultSet rs = null;
-      List<TxnToWriteId> txnToWriteIds = new ArrayList<>();
-      List<TxnToWriteId> srcTxnToWriteIds = null;
-      try {
-        lockInternal();
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-
-        if (rqst.isSetReplPolicy()) {
-          srcTxnToWriteIds = rqst.getSrcTxnToWriteIdList();
-          List<Long> srcTxnIds = new ArrayList<>();
-          assert (rqst.isSetSrcTxnToWriteIdList());
-          assert (!rqst.isSetTxnIds());
-          assert (!srcTxnToWriteIds.isEmpty());
-
-          for (TxnToWriteId txnToWriteId : srcTxnToWriteIds) {
-            srcTxnIds.add(txnToWriteId.getTxnId());
-          }
-          txnIds = getTargetTxnIdList(rqst.getReplPolicy(), srcTxnIds, dbConn);
-          if (srcTxnIds.size() != txnIds.size()) {
-            // Idempotent case where txn was already closed but gets allocate write id event.
-            // So, just ignore it and return empty list.
-            LOG.info("Idempotent case: Target txn id is missing for source txn id : {} and repl policy {}", srcTxnIds,
-                rqst.getReplPolicy());
-            return new AllocateTableWriteIdsResponse(txnToWriteIds);
-          }
-        } else {
-          assert (!rqst.isSetSrcTxnToWriteIdList());
-          assert (rqst.isSetTxnIds());
-          txnIds = rqst.getTxnIds();
-        }
-
-        //Easiest check since we can't differentiate do we handle singleton list or list with multiple txn ids.
-        if (txnIds.size() > 1) {
-          Collections.sort(txnIds); //easier to read logs and for assumption done in replication flow
-        }
-
-        // Check if all the input txns are in valid state.
-        // Write IDs should be allocated only for open and not read-only transactions.
-        try (Statement stmt = dbConn.createStatement()) {
-          if (!isTxnsOpenAndNotReadOnly(txnIds, stmt)) {
-            String errorMsg = "Write ID allocation on " + TableName.getDbTable(dbName, tblName)
-                    + " failed for input txns: "
-                    + getAbortedAndReadOnlyTxns(txnIds, stmt)
-                    + getCommittedTxns(txnIds, stmt);
-            LOG.error(errorMsg);
-
-            throw new IllegalStateException("Write ID allocation failed on " + TableName.getDbTable(dbName, tblName)
-                    + " as not all input txns in open state or read-only");
-          }
-        }
-
-        List<String> queries = new ArrayList<>();
-        StringBuilder prefix = new StringBuilder();
-        StringBuilder suffix = new StringBuilder();
-        long writeId;
-        int allocatedTxnsCount = 0;
-        List<String> params = Arrays.asList(dbName, tblName);
-        if (shouldReallocate) {
-          // during query recompilation after lock acquistion, it is important to realloc new writeIds
-          // to ensure writeIds are committed in increasing order.
-          prefix.append("DELETE FROM \"TXN_TO_WRITE_ID\" WHERE")
-                .append(" \"T2W_DATABASE\" = ? AND \"T2W_TABLE\" = ? AND ");
-          TxnUtils.buildQueryWithINClause(conf, queries, prefix, suffix,
-              txnIds, "\"T2W_TXNID\"", false, false);
-          for (String query : queries) {
-            pStmt = sqlGenerator.prepareStmtWithParameters(dbConn, query, params);
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("Going to execute delete <" + query.replace("?", "{}") + ">",
-                  quoteString(dbName), quoteString(tblName));
-            }
-            int numRowsDeleted = pStmt.executeUpdate();
-            LOG.info("Removed {} prior writeIds during reallocation", numRowsDeleted);
-            closeStmt(pStmt);
-          }
-        } else {
-          // Traverse the TXN_TO_WRITE_ID to see if any of the input txns already have allocated a
-          // write id for the same db.table. If yes, then need to reuse it else have to allocate new one
-          // The write id would have been already allocated in case of multi-statement txns where
-          // first write on a table will allocate write id and rest of the writes should re-use it.
-          prefix.append("SELECT \"T2W_TXNID\", \"T2W_WRITEID\" FROM \"TXN_TO_WRITE_ID\" WHERE")
-                .append(" \"T2W_DATABASE\" = ? AND \"T2W_TABLE\" = ? AND ");
-          TxnUtils.buildQueryWithINClause(conf, queries, prefix, suffix,
-              txnIds, "\"T2W_TXNID\"", false, false);
-          for (String query : queries) {
-            pStmt = sqlGenerator.prepareStmtWithParameters(dbConn, query, params);
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("Going to execute query <" + query.replace("?", "{}") + ">",
-                  quoteString(dbName), quoteString(tblName));
-            }
-            rs = pStmt.executeQuery();
-            while (rs.next()) {
-              // If table write ID is already allocated for the given transaction, then just use it
-              long txnId = rs.getLong(1);
-              writeId = rs.getLong(2);
-              txnToWriteIds.add(new TxnToWriteId(txnId, writeId));
-              allocatedTxnsCount++;
-              LOG.info("Reused already allocated writeID: {} for txnId: {}", writeId, txnId);
-            }
-            closeStmt(pStmt);
-          }
-        }
-
-        // Batch allocation should always happen atomically. Either write ids for all txns is allocated or none.
-        long numOfWriteIds = txnIds.size();
-        assert ((allocatedTxnsCount == 0) || (numOfWriteIds == allocatedTxnsCount));
-        if (allocatedTxnsCount == numOfWriteIds) {
-          // If all the txns in the list have pre-allocated write ids for the given table, then just return.
-          // This is for idempotent case.
-          return new AllocateTableWriteIdsResponse(txnToWriteIds);
-        }
-
-        long srcWriteId = 0;
-        if (rqst.isSetReplPolicy()) {
-          // In replication flow, we always need to allocate write ID equal to that of source.
-          assert (srcTxnToWriteIds != null);
-          srcWriteId = srcTxnToWriteIds.get(0).getWriteId();
-        }
-
-
-        // There are some txns in the list which does not have write id allocated and hence go ahead and do it.
-        // Get the next write id for the given table and update it with new next write id.
-        // This is select for update query which takes a lock if the table entry is already there in NEXT_WRITE_ID
-        String s = sqlGenerator.addForUpdateClause(
-            "SELECT \"NWI_NEXT\" FROM \"NEXT_WRITE_ID\" WHERE \"NWI_DATABASE\" = ? AND \"NWI_TABLE\" = ?");
-        closeStmt(pStmt);
-        pStmt = sqlGenerator.prepareStmtWithParameters(dbConn, s, params);
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Going to execute query <" + s.replace("?", "{}") + ">",
-              quoteString(dbName), quoteString(tblName));
-        }
-        rs = pStmt.executeQuery();
-        if (!rs.next()) {
-          // First allocation of write id should add the table to the next_write_id meta table
-          // The initial value for write id should be 1 and hence we add 1 with number of write ids allocated here
-          // For repl flow, we need to force set the incoming write id.
-          writeId = (srcWriteId > 0) ? srcWriteId : 1;
-          s = "INSERT INTO \"NEXT_WRITE_ID\" (\"NWI_DATABASE\", \"NWI_TABLE\", \"NWI_NEXT\") VALUES (?, ?, "
-                  + (writeId + numOfWriteIds) + ")";
-          closeStmt(pStmt);
-          pStmt = sqlGenerator.prepareStmtWithParameters(dbConn, s, params);
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Going to execute insert <" + s.replace("?", "{}") + ">",
-                quoteString(dbName), quoteString(tblName));
-          }
-          pStmt.execute();
-        } else {
-          long nextWriteId = rs.getLong(1);
-          writeId = (srcWriteId > 0) ? srcWriteId : nextWriteId;
-
-          // Update the NEXT_WRITE_ID for the given table after incrementing by number of write ids allocated
-          s = "UPDATE \"NEXT_WRITE_ID\" SET \"NWI_NEXT\" = " + (writeId + numOfWriteIds)
-                  + " WHERE \"NWI_DATABASE\" = ? AND \"NWI_TABLE\" = ?";
-          closeStmt(pStmt);
-          pStmt = sqlGenerator.prepareStmtWithParameters(dbConn, s, params);
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Going to execute update <" + s.replace("?", "{}") + ">",
-                quoteString(dbName), quoteString(tblName));
-          }
-          pStmt.executeUpdate();
-
-          // For repl flow, if the source write id is mismatching with target next write id, then current
-          // metadata in TXN_TO_WRITE_ID is stale for this table and hence need to clean-up TXN_TO_WRITE_ID.
-          // This is possible in case of first incremental repl after bootstrap where concurrent write
-          // and drop table was performed at source during bootstrap dump.
-          if ((srcWriteId > 0) && (srcWriteId != nextWriteId)) {
-            s = "DELETE FROM \"TXN_TO_WRITE_ID\" WHERE \"T2W_DATABASE\" = ? AND \"T2W_TABLE\" = ?";
-            closeStmt(pStmt);
-            pStmt = sqlGenerator.prepareStmtWithParameters(dbConn, s, params);
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("Going to execute delete <" + s.replace("?", "{}") + ">",
-                  quoteString(dbName), quoteString(tblName));
-            }
-            pStmt.executeUpdate();
-          }
-        }
-
-        // Map the newly allocated write ids against the list of txns which doesn't have pre-allocated write ids
-        try (PreparedStatement pstmt = dbConn.prepareStatement(TXN_TO_WRITE_ID_INSERT_QUERY)) {
-          for (long txnId : txnIds) {
-            pstmt.setLong(1, txnId);
-            pstmt.setString(2, dbName);
-            pstmt.setString(3, tblName);
-            pstmt.setLong(4, writeId);
-            pstmt.addBatch();
-
-            txnToWriteIds.add(new TxnToWriteId(txnId, writeId));
-            LOG.info("Allocated writeId: {} for txnId: {}", writeId, txnId);
-            writeId++;
-            if (txnToWriteIds.size() % maxBatchSize == 0) {
-              LOG.debug("Executing a batch of <{}> queries. Batch size: {}", TXN_TO_WRITE_ID_INSERT_QUERY,
-                  maxBatchSize);
-              pstmt.executeBatch();
-            }
-          }
-          if (txnToWriteIds.size() % maxBatchSize != 0) {
-            LOG.debug("Executing a batch of <{}> queries. Batch size: {}", TXN_TO_WRITE_ID_INSERT_QUERY,
-                txnToWriteIds.size() % maxBatchSize);
-            pstmt.executeBatch();
-          }
-        }
-
-        if (transactionalListeners != null) {
-          MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners,
-                  EventMessage.EventType.ALLOC_WRITE_ID,
-                  new AllocWriteIdEvent(txnToWriteIds, dbName, tblName),
-                  dbConn, sqlGenerator);
-        }
-
-        LOG.info("Allocated write ids for dbName={}, tblName={} (txnIds: {})", dbName, tblName, rqst.getTxnIds());
-        dbConn.commit();
-        return new AllocateTableWriteIdsResponse(txnToWriteIds);
-      } catch (SQLException e) {
-        LOG.error("Exception during write ids allocation for request={}. Will retry if possible.", rqst, e);
-        rollbackDBConn(dbConn);
-        checkRetryable(e, "allocateTableWriteIds(" + rqst + ")", true);
-        throw new MetaException("Unable to update transaction database "
-                + StringUtils.stringifyException(e));
-      } finally {
-        close(rs, pStmt, dbConn);
-        unlockInternal();
-      }
-    } catch (RetryException e) {
-      return allocateTableWriteIds(rqst);
-    }
-  }
-
-  @Override
-  public MaxAllocatedTableWriteIdResponse getMaxAllocatedTableWrited(MaxAllocatedTableWriteIdRequest rqst) throws MetaException {
-    String dbName = rqst.getDbName();
-    String tableName = rqst.getTableName();
-    try {
-      Connection dbConn = null;
-      PreparedStatement pStmt = null;
-      ResultSet rs = null;
-      try {
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-        pStmt = sqlGenerator.prepareStmtWithParameters(dbConn, SELECT_NWI_NEXT_FROM_NEXT_WRITE_ID,
-            Arrays.asList(dbName, tableName));
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Going to execute query <" + SELECT_NWI_NEXT_FROM_NEXT_WRITE_ID.replace("?", "{}") + ">",
-              quoteString(dbName), quoteString(tableName));
-        }
-        rs = pStmt.executeQuery();
-        // If there is no record, we never allocated anything
-        long maxWriteId = 0l;
-        if (rs.next()) {
-          // The row contains the nextId not the previously allocated
-          maxWriteId = rs.getLong(1) - 1;
-        }
-        return new MaxAllocatedTableWriteIdResponse(maxWriteId);
-      } catch (SQLException e) {
-        LOG.error(
-            "Exception during reading the max allocated writeId for dbName={}, tableName={}. Will retry if possible.",
-            dbName, tableName, e);
-        checkRetryable(e, "getMaxAllocatedTableWrited(" + rqst + ")");
-        throw new MetaException("Unable to update transaction database " + StringUtils.stringifyException(e));
-      } finally {
-        close(rs, pStmt, dbConn);
-      }
-    } catch (RetryException e) {
-      return getMaxAllocatedTableWrited(rqst);
-    }
-  }
-
-  @Override
-  public void seedWriteId(SeedTableWriteIdsRequest rqst)
-      throws MetaException {
-    try {
-      Connection dbConn = null;
-      PreparedStatement pst = null;
-      try {
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-
-        //since this is on conversion from non-acid to acid, NEXT_WRITE_ID should not have an entry
-        //for this table.  It also has a unique index in case 'should not' is violated
-
-        // First allocation of write id should add the table to the next_write_id meta table
-        // The initial value for write id should be 1 and hence we add 1 with number of write ids
-        // allocated here
-        String s = "INSERT INTO \"NEXT_WRITE_ID\" (\"NWI_DATABASE\", \"NWI_TABLE\", \"NWI_NEXT\") VALUES (?, ?, "
-                + Long.toString(rqst.getSeedWriteId() + 1) + ")";
-        pst = sqlGenerator.prepareStmtWithParameters(dbConn, s, Arrays.asList(rqst.getDbName(), rqst.getTableName()));
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Going to execute insert <" + s.replace("?", "{}") + ">",
-              quoteString(rqst.getDbName()), quoteString(rqst.getTableName()));
-        }
-        pst.execute();
-        LOG.debug("Going to commit");
-        dbConn.commit();
-      } catch (SQLException e) {
-        rollbackDBConn(dbConn);
-        checkRetryable(e, "seedWriteId(" + rqst + ")");
-        throw new MetaException("Unable to update transaction database " + StringUtils.stringifyException(e));
-      } finally {
-        close(null, pst, dbConn);
-      }
-    } catch (RetryException e) {
-      seedWriteId(rqst);
-    }
-  }
-
-  @Override
-  public void seedTxnId(SeedTxnIdRequest rqst) throws MetaException {
-    try {
-      Connection dbConn = null;
-      Statement stmt = null;
-      try {
-        lockInternal();
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-        stmt = dbConn.createStatement();
-        /*
-         * Locking the txnLock an exclusive way, we do not want to set the txnId backward accidentally
-         * if there are concurrent open transactions
-         */
-        acquireTxnLock(stmt, false);
-        long highWaterMark = getHighWaterMark(stmt);
-        if (highWaterMark >= rqst.getSeedTxnId()) {
-          throw new MetaException(MessageFormat
-              .format("Invalid txnId seed {}, the highWaterMark is {}", rqst.getSeedTxnId(), highWaterMark));
-        }
-        TxnUtils.seedTxnSequence(dbConn, conf, stmt, rqst.getSeedTxnId());
-        dbConn.commit();
-
-      } catch (SQLException e) {
-        rollbackDBConn(dbConn);
-        checkRetryable(e, "seedTxnId(" + rqst + ")");
-        throw new MetaException("Unable to update transaction database " + StringUtils.stringifyException(e));
-      } finally {
-        close(null, stmt, dbConn);
-        unlockInternal();
-      }
-    } catch (RetryException e) {
-      seedTxnId(rqst);
-    }
-  }
-
-  @Override
-  @RetrySemantics.Idempotent
-  public void addWriteNotificationLog(ListenerEvent acidWriteEvent) throws MetaException {
-      Connection dbConn = jdbcResource.getConnection();
-      MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners,
-          acidWriteEvent instanceof AcidWriteEvent ? EventMessage.EventType.ACID_WRITE
-              : EventMessage.EventType.BATCH_ACID_WRITE,
-          acidWriteEvent, dbConn, sqlGenerator);
-  }
-
-  @Override
-  @RetrySemantics.SafeToRetry
-  public void performWriteSetGC() throws MetaException {
-    Connection dbConn = null;
-    Statement stmt = null;
-    try {
-      dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-      stmt = dbConn.createStatement();
-      long commitHighWaterMark = getMinOpenTxnIdWaterMark(dbConn);
-      int delCnt = stmt.executeUpdate("DELETE FROM \"WRITE_SET\" WHERE \"WS_COMMIT_ID\" < " + commitHighWaterMark);
-      LOG.info("Deleted {} obsolete rows from WRITE_SET", delCnt);
-      dbConn.commit();
-    } catch (SQLException ex) {
-      LOG.warn("WriteSet GC failed due to " + getMessage(ex), ex);
-    } finally {
-      close(null, stmt, dbConn);
-    }
-  }
-
-  protected long getMinOpenTxnIdWaterMark(Connection dbConn) throws SQLException, MetaException {
-    /**
-     * We try to find the highest transactionId below everything was committed or aborted.
-     * For that we look for the lowest open transaction in the TXNS and the TxnMinTimeout boundary,
-     * because it is guaranteed there won't be open transactions below that.
-     */
-    long minOpenTxn;
-    try (Statement stmt = dbConn.createStatement()) {
-      try (ResultSet rs = stmt
-          .executeQuery("SELECT MIN(\"TXN_ID\") FROM \"TXNS\" WHERE \"TXN_STATE\"=" + TxnStatus.OPEN)) {
-        if (!rs.next()) {
-          throw new IllegalStateException("Scalar query returned no rows?!?!!");
-        }
-        minOpenTxn = rs.getLong(1);
-        if (rs.wasNull()) {
-          minOpenTxn = Long.MAX_VALUE;
-        }
-      }
-    } catch (SQLException e) {
-      throw new UncategorizedSQLException(null, null, e);
-    }
-    long lowWaterMark = getOpenTxnTimeoutLowBoundaryTxnId(dbConn);
-    LOG.debug("MinOpenTxnIdWaterMark calculated with minOpenTxn {}, lowWaterMark {}", minOpenTxn, lowWaterMark);
-    return Long.min(minOpenTxn, lowWaterMark + 1);
-  }
-
-  @Override
-  public void updateTransactionStatistics(UpdateTransactionalStatsRequest req) throws MetaException {
-    String queryText = "UPDATE \"MV_TABLES_USED\" " +
-            "SET \"INSERTED_COUNT\"=\"INSERTED_COUNT\"+?" +
-            ",\"UPDATED_COUNT\"=\"UPDATED_COUNT\"+?" +
-            ",\"DELETED_COUNT\"=\"DELETED_COUNT\"+?" +
-            " WHERE \"TBL_ID\"=?";
-    try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED)) {
-      try (PreparedStatement pstmt = dbConn.prepareStatement(queryText)) {
-        pstmt.setLong(1, req.getInsertCount());
-        pstmt.setLong(2, req.getUpdatedCount());
-        pstmt.setLong(3, req.getDeletedCount());
-        pstmt.setLong(4, req.getTableId());
-        LOG.debug("Going to execute query <{}>", queryText);
-        int res = pstmt.executeUpdate();
-        dbConn.commit();
-        LOG.debug("Updated {} records tblId={}", res, req.getTableId());
-      }
-    } catch (SQLException ex) {
-      LOG.warn("Unable to update transactional statistics tblId=" + req.getTableId(), ex);
-      throw new MetaException("Unable to update transactional statistics" + " " + StringUtils.stringifyException(ex));
-    }
-  }
-
-  /**
-   * Get invalidation info for the materialization. Materialization information
-   * contains information about whether there was update/delete/compaction operations on the source
-   * tables used by the materialization since it was created.
-   */
-  @Override
-  @RetrySemantics.ReadOnly
-  public Materialization getMaterializationInvalidationInfo(
-          CreationMetadata creationMetadata, String validTxnListStr) throws MetaException {
-    if (creationMetadata.getTablesUsed().isEmpty()) {
-      // Bail out
-      LOG.warn("Materialization creation metadata does not contain any table");
-      return null;
-    }
-
-    // We are composing a query that returns a single row if an update happened after
-    // the materialization was created. Otherwise, query returns 0 rows.
-
-    // Parse validReaderWriteIdList from creation metadata
-    MaterializationSnapshot mvSnapshot = MaterializationSnapshot.fromJson(creationMetadata.getValidTxnList());
-    if (mvSnapshot.getTableSnapshots() != null && !mvSnapshot.getTableSnapshots().isEmpty()) {
-      // Incremental rebuild of MVs on Iceberg sources is not supported.
-      return null;
-    }
-    final ValidTxnWriteIdList validReaderWriteIdList = new ValidTxnWriteIdList(mvSnapshot.getValidTxnList());
-
-    // Parse validTxnList
-    final ValidReadTxnList currentValidTxnList = new ValidReadTxnList(validTxnListStr);
-    // Get the valid write id list for the tables in current state
-    final List<TableValidWriteIds> currentTblValidWriteIdsList = new ArrayList<>();
-    Connection dbConn = null;
-    for (String fullTableName : creationMetadata.getTablesUsed()) {
-      try {
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-        currentTblValidWriteIdsList.add(getValidWriteIdsForTable(dbConn, fullTableName, currentValidTxnList));
-      } catch (SQLException ex) {
-        String errorMsg = "Unable to query Valid writeIds of table " + fullTableName;
-        LOG.warn(errorMsg, ex);
-        throw new MetaException(errorMsg + " " + StringUtils.stringifyException(ex));
-      } finally {
-        closeDbConn(dbConn);
-      }
-    }
-    final ValidTxnWriteIdList currentValidReaderWriteIdList = TxnCommonUtils.createValidTxnWriteIdList(
-            currentValidTxnList.getHighWatermark(), currentTblValidWriteIdsList);
-
-    List<String> params = new ArrayList<>();
-    StringBuilder queryUpdateDelete = new StringBuilder();
-    StringBuilder queryCompletedCompactions = new StringBuilder();
-    StringBuilder queryCompactionQueue = new StringBuilder();
-    // compose a query that select transactions containing an update...
-    queryUpdateDelete.append("SELECT \"CTC_UPDATE_DELETE\" FROM \"COMPLETED_TXN_COMPONENTS\" WHERE \"CTC_UPDATE_DELETE\" ='Y' AND (");
-    queryCompletedCompactions.append("SELECT 1 FROM \"COMPLETED_COMPACTIONS\" WHERE (");
-    queryCompactionQueue.append("SELECT 1 FROM \"COMPACTION_QUEUE\" WHERE (");
-    int i = 0;
-    for (String fullyQualifiedName : creationMetadata.getTablesUsed()) {
-      ValidWriteIdList tblValidWriteIdList =
-              validReaderWriteIdList.getTableValidWriteIdList(fullyQualifiedName);
-      if (tblValidWriteIdList == null) {
-        LOG.warn("ValidWriteIdList for table {} not present in creation metadata, this should not happen", fullyQualifiedName);
-        return null;
-      }
-
-      // First, we check whether the low watermark has moved for any of the tables.
-      // If it has, we return true, since it is not incrementally refreshable, e.g.,
-      // one of the commits that are not available may be an update/delete.
-      ValidWriteIdList currentTblValidWriteIdList =
-              currentValidReaderWriteIdList.getTableValidWriteIdList(fullyQualifiedName);
-      if (currentTblValidWriteIdList == null) {
-        LOG.warn("Current ValidWriteIdList for table {} not present in creation metadata, this should not happen", fullyQualifiedName);
-        return null;
-      }
-      if (!Objects.equals(currentTblValidWriteIdList.getMinOpenWriteId(), tblValidWriteIdList.getMinOpenWriteId())) {
-        LOG.debug("Minimum open write id do not match for table {}", fullyQualifiedName);
-        return null;
-      }
-
-      // ...for each of the tables that are part of the materialized view,
-      // where the transaction had to be committed after the materialization was created...
-      if (i != 0) {
-        queryUpdateDelete.append("OR");
-        queryCompletedCompactions.append("OR");
-        queryCompactionQueue.append("OR");
-      }
-      String[] names = TxnUtils.getDbTableName(fullyQualifiedName);
-      assert (names.length == 2);
-      queryUpdateDelete.append(" (\"CTC_DATABASE\"=? AND \"CTC_TABLE\"=?");
-      queryCompletedCompactions.append(" (\"CC_DATABASE\"=? AND \"CC_TABLE\"=?");
-      queryCompactionQueue.append(" (\"CQ_DATABASE\"=? AND \"CQ_TABLE\"=?");
-      params.add(names[0]);
-      params.add(names[1]);
-      queryUpdateDelete.append(" AND (\"CTC_WRITEID\" > " + tblValidWriteIdList.getHighWatermark());
-      queryCompletedCompactions.append(" AND (\"CC_HIGHEST_WRITE_ID\" > " + tblValidWriteIdList.getHighWatermark());
-      queryUpdateDelete.append(tblValidWriteIdList.getInvalidWriteIds().length == 0 ? ") " :
-              " OR \"CTC_WRITEID\" IN(" + StringUtils.join(",",
-                      Arrays.asList(ArrayUtils.toObject(tblValidWriteIdList.getInvalidWriteIds()))) + ") ) ");
-      queryCompletedCompactions.append(tblValidWriteIdList.getInvalidWriteIds().length == 0 ? ") " :
-              " OR \"CC_HIGHEST_WRITE_ID\" IN(" + StringUtils.join(",",
-                      Arrays.asList(ArrayUtils.toObject(tblValidWriteIdList.getInvalidWriteIds()))) + ") ) ");
-      queryUpdateDelete.append(") ");
-      queryCompletedCompactions.append(") ");
-      queryCompactionQueue.append(") ");
-      i++;
-    }
-    // ... and where the transaction has already been committed as per snapshot taken
-    // when we are running current query
-    queryUpdateDelete.append(") AND \"CTC_TXNID\" <= " + currentValidTxnList.getHighWatermark());
-    queryUpdateDelete.append(currentValidTxnList.getInvalidTransactions().length == 0 ? " " :
-            " AND \"CTC_TXNID\" NOT IN(" + StringUtils.join(",",
-                    Arrays.asList(ArrayUtils.toObject(currentValidTxnList.getInvalidTransactions()))) + ") ");
-    queryCompletedCompactions.append(")");
-    queryCompactionQueue.append(") ");
-
-    boolean hasUpdateDelete = executeBoolean(queryUpdateDelete.toString(), params,
-            "Unable to retrieve materialization invalidation information: completed transaction components.");
-
-    // Execute query
-    queryCompletedCompactions.append(" UNION ");
-    queryCompletedCompactions.append(queryCompactionQueue.toString());
-    List<String> paramsTwice = new ArrayList<>(params);
-    paramsTwice.addAll(params);
-    boolean hasCompaction = executeBoolean(queryCompletedCompactions.toString(), paramsTwice,
-            "Unable to retrieve materialization invalidation information: compactions");
-
-    return new Materialization(hasUpdateDelete, hasCompaction);
-  }
-
-  private boolean executeBoolean(String queryText, List<String> params, String errorMessage) throws MetaException {
-    Connection dbConn = null;
-    PreparedStatement pst = null;
-    ResultSet rs = null;
-    try {
-      dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-      LOG.debug("Going to execute query <{}>", queryText);
-      pst = sqlGenerator.prepareStmtWithParameters(dbConn, queryText, params);
-      pst.setMaxRows(1);
-      rs = pst.executeQuery();
-
-      return rs.next();
-    } catch (SQLException ex) {
-      LOG.warn(errorMessage, ex);
-      throw new MetaException(errorMessage + " " + StringUtils.stringifyException(ex));
-    } finally {
-      close(rs, pst, dbConn);
-    }
-  }
-
-  @Override
-  public LockResponse lockMaterializationRebuild(String dbName, String tableName, long txnId)
-      throws MetaException {
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Acquiring lock for materialization rebuild with {} for {}",
-          JavaUtils.txnIdToString(txnId), TableName.getDbTable(dbName, tableName));
-    }
-
-    TxnStore.MutexAPI.LockHandle handle = null;
-    Connection dbConn = null;
-    PreparedStatement pst = null;
-    ResultSet rs = null;
-    try {
-      lockInternal();
-      /**
-       * MUTEX_KEY.MaterializationRebuild lock ensures that there is only 1 entry in
-       * Initiated/Working state for any resource. This ensures we do not run concurrent
-       * rebuild operations on any materialization.
-       */
-      handle = getMutexAPI().acquireLock(MUTEX_KEY.MaterializationRebuild.name());
-      dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-
-      List<String> params = Arrays.asList(dbName, tableName);
-      String selectQ = "SELECT \"MRL_TXN_ID\" FROM \"MATERIALIZATION_REBUILD_LOCKS\" WHERE" +
-          " \"MRL_DB_NAME\" = ? AND \"MRL_TBL_NAME\" = ?";
-      pst = sqlGenerator.prepareStmtWithParameters(dbConn, selectQ, params);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Going to execute query <" + selectQ.replace("?", "{}") + ">",
-            quoteString(dbName), quoteString(tableName));
-      }
-      rs = pst.executeQuery();
-      if(rs.next()) {
-        LOG.info("Ignoring request to rebuild {}/{} since it is already being rebuilt", dbName, tableName);
-        return new LockResponse(txnId, LockState.NOT_ACQUIRED);
-      }
-      String insertQ = "INSERT INTO \"MATERIALIZATION_REBUILD_LOCKS\" " +
-          "(\"MRL_TXN_ID\", \"MRL_DB_NAME\", \"MRL_TBL_NAME\", \"MRL_LAST_HEARTBEAT\") VALUES (" + txnId +
-          ", ?, ?, " + Instant.now().toEpochMilli() + ")";
-      closeStmt(pst);
-      pst = sqlGenerator.prepareStmtWithParameters(dbConn, insertQ, params);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Going to execute update <" + insertQ.replace("?", "{}") + ">",
-            quoteString(dbName), quoteString(tableName));
-      }
-      pst.executeUpdate();
-      LOG.debug("Going to commit");
-      dbConn.commit();
-      return new LockResponse(txnId, LockState.ACQUIRED);
-    } catch (SQLException ex) {
-      LOG.warn("lockMaterializationRebuild failed due to " + getMessage(ex), ex);
-      throw new MetaException("Unable to retrieve materialization invalidation information due to " +
-          StringUtils.stringifyException(ex));
-    } finally {
-      close(rs, pst, dbConn);
-      if(handle != null) {
-        handle.releaseLocks();
-      }
-      unlockInternal();
-    }
-  }
-
-  @Override
-  public boolean heartbeatLockMaterializationRebuild(String dbName, String tableName, long txnId)
-      throws MetaException {
-    try {
-      Connection dbConn = null;
-      PreparedStatement pst = null;
-      try {
-        lockInternal();
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-        String s = "UPDATE \"MATERIALIZATION_REBUILD_LOCKS\"" +
-            " SET \"MRL_LAST_HEARTBEAT\" = " + Instant.now().toEpochMilli() +
-            " WHERE \"MRL_TXN_ID\" = " + txnId +
-            " AND \"MRL_DB_NAME\" = ?" +
-            " AND \"MRL_TBL_NAME\" = ?";
-        pst = sqlGenerator.prepareStmtWithParameters(dbConn, s, Arrays.asList(dbName, tableName));
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Going to execute update <" + s.replace("?", "{}") + ">",
-              quoteString(dbName), quoteString(tableName));
-        }
-        int rc = pst.executeUpdate();
-        if (rc < 1) {
-          LOG.debug("Going to rollback");
-          dbConn.rollback();
-          LOG.info("No lock found for rebuild of {} when trying to heartbeat", TableName.getDbTable(dbName, tableName));
-          // It could not be renewed, return that information
-          return false;
-        }
-        LOG.debug("Going to commit");
-        dbConn.commit();
-        // It could be renewed, return that information
-        return true;
-      } catch (SQLException e) {
-        LOG.debug("Going to rollback: ", e);
-        rollbackDBConn(dbConn);
-        checkRetryable(e,
-            "heartbeatLockMaterializationRebuild(" + TableName.getDbTable(dbName, tableName) + ", " + txnId + ")");
-        throw new MetaException("Unable to heartbeat rebuild lock due to " +
-            StringUtils.stringifyException(e));
-      } finally {
-        close(null, pst, dbConn);
-        unlockInternal();
-      }
-    } catch (RetryException e) {
-      return heartbeatLockMaterializationRebuild(dbName, tableName ,txnId);
-    }
-  }
-
-  @Override
-  public long cleanupMaterializationRebuildLocks(ValidTxnList validTxnList, long timeout) throws MetaException {
-    try {
-      // Aux values
-      long cnt = 0L;
-      List<Long> txnIds = new ArrayList<>();
-      long timeoutTime = Instant.now().toEpochMilli() - timeout;
-
-      Connection dbConn = null;
-      Statement stmt = null;
-      ResultSet rs = null;
-      try {
-        lockInternal();
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-        stmt = dbConn.createStatement();
-
-        String selectQ = "SELECT \"MRL_TXN_ID\", \"MRL_LAST_HEARTBEAT\" FROM \"MATERIALIZATION_REBUILD_LOCKS\"";
-        LOG.debug("Going to execute query <{}>", selectQ);
-        rs = stmt.executeQuery(selectQ);
-        while(rs.next()) {
-          long lastHeartbeat = rs.getLong(2);
-          if (lastHeartbeat < timeoutTime) {
-            // The heartbeat has timeout, double check whether we can remove it
-            long txnId = rs.getLong(1);
-            if (validTxnList.isTxnValid(txnId) || validTxnList.isTxnAborted(txnId)) {
-              // Txn was committed (but notification was not received) or it was aborted.
-              // Either case, we can clean it up
-              txnIds.add(txnId);
-            }
-          }
-        }
-        if (!txnIds.isEmpty()) {
-          String deleteQ = "DELETE FROM \"MATERIALIZATION_REBUILD_LOCKS\" WHERE" +
-              " \"MRL_TXN_ID\" IN(" + StringUtils.join(",", txnIds) + ") ";
-          LOG.debug("Going to execute update <{}>", deleteQ);
-          cnt = stmt.executeUpdate(deleteQ);
-        }
-        LOG.debug("Going to commit");
-        dbConn.commit();
-        return cnt;
-      } catch (SQLException e) {
-        LOG.debug("Going to rollback: ", e);
-        rollbackDBConn(dbConn);
-        checkRetryable(e, "cleanupMaterializationRebuildLocks");
-        throw new MetaException("Unable to clean rebuild locks due to " +
-            StringUtils.stringifyException(e));
-      } finally {
-        close(rs, stmt, dbConn);
-        unlockInternal();
-      }
-    } catch (RetryException e) {
-      return cleanupMaterializationRebuildLocks(validTxnList, timeout);
-    }
-  }
-
-  /**
-   * As much as possible (i.e. in absence of retries) we want both operations to be done on the same
-   * connection (but separate transactions).
-   *
-   * Retry-by-caller note: If the call to lock is from a transaction, then in the worst case
-   * there will be a duplicate set of locks but both sets will belong to the same txn so they
-   * will not conflict with each other.  For locks w/o txn context (i.e. read-only query), this
-   * may lead to deadlock (at least a long wait).  (e.g. 1st call creates locks in {@code LOCK_WAITING}
-   * mode and response gets lost.  Then {@link org.apache.hadoop.hive.metastore.RetryingMetaStoreClient}
-   * retries, and enqueues another set of locks in LOCK_WAITING.  The 2nd LockResponse is delivered
-   * to the DbLockManager, which will keep dong {@link #checkLock(CheckLockRequest)} until the 1st
-   * set of locks times out.
-   */
-  @RetrySemantics.CannotRetry
-  public LockResponse lock(LockRequest rqst) throws NoSuchTxnException, TxnAbortedException, MetaException {
-    ConnectionLockIdPair connAndLockId = enqueueLockWithRetry(rqst);
-    try {
-      return checkLockWithRetry(connAndLockId.dbConn, connAndLockId.extLockId, rqst.getTxnid(),
-          rqst.isZeroWaitReadEnabled(), rqst.isExclusiveCTAS());
-    }
-    catch(NoSuchLockException e) {
-      // This should never happen, as we just added the lock id
-      throw new MetaException("Couldn't find a lock we just created! " + e.getMessage());
-    }
-  }
-  private static final class ConnectionLockIdPair {
-    private final Connection dbConn;
-    private final long extLockId;
-    private ConnectionLockIdPair(Connection dbConn, long extLockId) {
-      this.dbConn = dbConn;
-      this.extLockId = extLockId;
-    }
-  }
-
-  /**
-   * Note that by definition select for update is divorced from update, i.e. you executeQuery() to read
-   * and then executeUpdate().  One other alternative would be to actually update the row in TXNS but
-   * to the same value as before thus forcing db to acquire write lock for duration of the transaction.
-   *
-   * SELECT ... FOR UPDATE locks the row until the transaction commits or rolls back.
-   * Second connection using `SELECT ... FOR UPDATE` will suspend until the lock is released.
-   * @return the txnType wrapped in an {@link Optional}
-   * @throws SQLException
-   * @throws MetaException
-   */
-  private TxnType getOpenTxnTypeAndLock(Statement stmt, long txnId) throws SQLException, MetaException {
-    String query = "SELECT \"TXN_TYPE\" FROM \"TXNS\" WHERE \"TXN_ID\" = " + txnId
-        + " AND \"TXN_STATE\" = " + TxnStatus.OPEN;
-    try (ResultSet rs = stmt.executeQuery(sqlGenerator.addForUpdateClause(query))) {
-      return rs.next() ? TxnType.findByValue(rs.getInt(1)) : null;
-    }
-  }
-
-  /**
-   * This enters locks into the queue in {@link #LOCK_WAITING} mode.
-   *
-   * Isolation Level Notes:
-   * 1. We use S4U (withe read_committed) to generate the next (ext) lock id.  This serializes
-   * any 2 {@code enqueueLockWithRetry()} calls.
-   * 2. We use S4U on the relevant TXNS row to block any concurrent abort/commit/etc operations
-   * @see #checkLockWithRetry(Connection, long, long, boolean, boolean)
-   */
-  private ConnectionLockIdPair enqueueLockWithRetry(LockRequest rqst)
-      throws NoSuchTxnException, TxnAbortedException, MetaException {
-    boolean success = false;
-    Connection dbConn = null;
-    try {
-      Statement stmt = null;
-      try {
-        lockInternal();
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-        long txnid = rqst.getTxnid();
-        stmt = dbConn.createStatement();
-        if (isValidTxn(txnid)) {
-          //this also ensures that txn is still there in expected state
-          TxnType txnType = getOpenTxnTypeAndLock(stmt, txnid);
-          if (txnType == null) {
-            ensureValidTxn(dbConn, txnid, stmt);
-            shouldNeverHappen(txnid);
-          }
-        }
-        /* Insert txn components and hive locks (with a temp extLockId) first, before getting the next lock ID in a select-for-update.
-           This should minimize the scope of the S4U and decrease the table lock duration. */
-        insertTxnComponents(txnid, rqst, dbConn);
-        long tempExtLockId = insertHiveLocksWithTemporaryExtLockId(txnid, dbConn, rqst);
-
-        /** Get the next lock id.
-         * This has to be atomic with adding entries to HIVE_LOCK entries (1st add in W state) to prevent a race.
-         * Suppose ID gen is a separate txn and 2 concurrent lock() methods are running.  1st one generates nl_next=7,
-         * 2nd nl_next=8.  Then 8 goes first to insert into HIVE_LOCKS and acquires the locks.  Then 7 unblocks,
-         * and add it's W locks but it won't see locks from 8 since to be 'fair' {@link #checkLock(java.sql.Connection, long)}
-         * doesn't block on locks acquired later than one it's checking*/
-        long extLockId = getNextLockIdForUpdate(dbConn, stmt);
-        incrementLockIdAndUpdateHiveLocks(stmt, extLockId, tempExtLockId);
-
-        dbConn.commit();
-        success = true;
-        return new ConnectionLockIdPair(dbConn, extLockId);
-      } catch (SQLException e) {
-        LOG.error("enqueueLock failed for request: {}. Exception msg: {}", rqst, getMessage(e));
-        rollbackDBConn(dbConn);
-        checkRetryable(e, "enqueueLockWithRetry(" + rqst + ")");
-        throw new MetaException("Unable to update transaction database " +
-          StringUtils.stringifyException(e));
-      } finally {
-        closeStmt(stmt);
-        if (!success) {
-          /* This needs to return a "live" connection to be used by operation that follows it.
-          Thus it only closes Connection on failure/retry. */
-          closeDbConn(dbConn);
-        }
-        unlockInternal();
-      }
-    }
-    catch(RetryException e) {
-      LOG.debug("Going to retry enqueueLock for request: {}, after catching RetryException with message: {}",
-              rqst, e.getMessage());
-      return enqueueLockWithRetry(rqst);
-    }
-  }
-
-  private long getNextLockIdForUpdate(Connection dbConn, Statement stmt) throws SQLException, MetaException {
-    String s = sqlGenerator.addForUpdateClause("SELECT \"NL_NEXT\" FROM \"NEXT_LOCK_ID\"");
-    LOG.debug("Going to execute query <{}>", s);
-    try (ResultSet rs = stmt.executeQuery(s)) {
-      if (!rs.next()) {
-        LOG.error("Failure to get next lock ID for update! SELECT query returned empty ResultSet.");
-        dbConn.rollback();
-        throw new MetaException("Transaction tables not properly " +
-                "initialized, no record found in next_lock_id");
-      }
-      return rs.getLong(1);
-    }
-  }
-
-  private void incrementLockIdAndUpdateHiveLocks(Statement stmt, long extLockId, long tempId) throws SQLException {
-    String incrCmd = String.format(INCREMENT_NEXT_LOCK_ID_QUERY, (extLockId + 1));
-    // update hive locks entries with the real EXT_LOCK_ID (replace temp ID)
-    String updateLocksCmd = String.format(UPDATE_HIVE_LOCKS_EXT_ID_QUERY, extLockId, tempId);
-    LOG.debug("Going to execute updates in batch: <{}>, and <{}>", incrCmd, updateLocksCmd);
-    stmt.addBatch(incrCmd);
-    stmt.addBatch(updateLocksCmd);
-    stmt.executeBatch();
-  }
-
-  private void insertTxnComponents(long txnid, LockRequest rqst, Connection dbConn) throws SQLException {
-    if (txnid > 0) {
-      Map<Pair<String, String>, Optional<Long>> writeIdCache = new HashMap<>();
-      try (PreparedStatement pstmt = dbConn.prepareStatement(TXN_COMPONENTS_INSERT_QUERY)) {
-        // For each component in this lock request,
-        // add an entry to the txn_components table
-        int insertCounter = 0;
-
-        Predicate<LockComponent> isDynPart = lc -> lc.isSetIsDynamicPartitionWrite() && lc.isIsDynamicPartitionWrite();
-        Function<LockComponent, Pair<String, String>> groupKey = lc ->
-            Pair.of(normalizeCase(lc.getDbname()), normalizeCase(lc.getTablename()));
-
-        Set<Pair<String, String>> isDynPartUpdate = rqst.getComponent().stream().filter(isDynPart)
-          .filter(lc -> lc.getOperationType() == DataOperationType.UPDATE || lc.getOperationType() == DataOperationType.DELETE)
-          .map(groupKey)
-        .collect(Collectors.toSet());
-
-        for (LockComponent lc : rqst.getComponent()) {
-          if (lc.isSetIsTransactional() && !lc.isIsTransactional()) {
-            //we don't prevent using non-acid resources in a txn but we do lock them
-            continue;
-          }
-          if (!shouldUpdateTxnComponent(txnid, rqst, lc)) {
-            continue;
-          }
-          String dbName = normalizeCase(lc.getDbname());
-          String tblName = normalizeCase(lc.getTablename());
-          String partName = normalizePartitionCase(lc.getPartitionname());
-          OperationType opType = OperationType.fromDataOperationType(lc.getOperationType());
-
-          if (isDynPart.test(lc)) {
-            partName = null;
-            if (writeIdCache.containsKey(groupKey.apply(lc))) {
-              continue;
-            }
-            opType = isDynPartUpdate.contains(groupKey.apply(lc)) ? OperationType.UPDATE : OperationType.INSERT;
-          }
-          Optional<Long> writeId = getWriteId(writeIdCache, dbName, tblName, txnid, dbConn);
-
-          pstmt.setLong(1, txnid);
-          pstmt.setString(2, dbName);
-          pstmt.setString(3, tblName);
-          pstmt.setString(4, partName);
-          pstmt.setString(5, opType.getSqlConst());
-          pstmt.setObject(6, writeId.orElse(null));
-
-          pstmt.addBatch();
-          insertCounter++;
-          if (insertCounter % maxBatchSize == 0) {
-            LOG.debug("Executing a batch of <{}> queries. Batch size: {}", TXN_COMPONENTS_INSERT_QUERY, maxBatchSize);
-            pstmt.executeBatch();
-          }
-        }
-        if (insertCounter % maxBatchSize != 0) {
-          LOG.debug("Executing a batch of <{}> queries. Batch size: {}", TXN_COMPONENTS_INSERT_QUERY,
-              insertCounter % maxBatchSize);
-          pstmt.executeBatch();
-        }
-      }
-    }
-  }
-
-  private Optional<Long> getWriteId(Map<Pair<String, String>, Optional<Long>> writeIdCache, String dbName, String tblName, long txnid, Connection dbConn) throws SQLException {
-    /* we can cache writeIDs based on dbName and tblName because txnid is invariant and
-    partitionName is not part of the writeID select query */
-    Pair<String, String> dbAndTable = Pair.of(dbName, tblName);
-    if (writeIdCache.containsKey(dbAndTable)) {
-      return writeIdCache.get(dbAndTable);
-    } else {
-      Optional<Long> writeId = getWriteIdFromDb(txnid, dbConn, dbName, tblName);
-      writeIdCache.put(dbAndTable, writeId);
-      return writeId;
-    }
-  }
-
-  private Optional<Long> getWriteIdFromDb(long txnid, Connection dbConn, String dbName, String tblName) throws SQLException {
-    if (tblName != null) {
-      // It is assumed the caller have already allocated write id for adding/updating data to
-      // the acid tables. However, DDL operatons won't allocate write id and hence this query
-      // may return empty result sets.
-      // Get the write id allocated by this txn for the given table writes
-      try (PreparedStatement pstmt = dbConn.prepareStatement(SELECT_WRITE_ID_QUERY)) {
-        pstmt.setString(1, dbName);
-        pstmt.setString(2, tblName);
-        pstmt.setLong(3, txnid);
-        LOG.debug("Going to execute query <{}>", SELECT_WRITE_ID_QUERY);
-        try (ResultSet rs = pstmt.executeQuery()) {
-          if (rs.next()) {
-            return Optional.of(rs.getLong(1));
-          }
-        }
-      }
-    }
-    return Optional.empty();
-  }
-
-  private boolean shouldUpdateTxnComponent(long txnid, LockRequest rqst, LockComponent lc) {
-    if(!lc.isSetOperationType()) {
-      //request came from old version of the client
-      return true; //this matches old behavior
-    }
-    else {
-      switch (lc.getOperationType()) {
-        case INSERT:
-        case UPDATE:
-        case DELETE:
-          return true;
-        case SELECT:
-          return false;
-        case NO_TXN:
-              /*this constant is a bit of a misnomer since we now always have a txn context.  It
-               just means the operation is such that we don't care what tables/partitions it
-               affected as it doesn't trigger a compaction or conflict detection.  A better name
-               would be NON_TRANSACTIONAL.*/
-          return false;
-        default:
-          //since we have an open transaction, only 4 values above are expected
-          throw new IllegalStateException("Unexpected DataOperationType: " + lc.getOperationType()
-                  + " agentInfo=" + rqst.getAgentInfo() + " " + JavaUtils.txnIdToString(txnid));
-      }
-    }
-  }
-
-  private long insertHiveLocksWithTemporaryExtLockId(long txnid, Connection dbConn, LockRequest rqst) throws MetaException, SQLException {
-
-    String lastHB = isValidTxn(txnid) ? "0" : getEpochFn(dbProduct);
-    String insertLocksQuery = String.format(HIVE_LOCKS_INSERT_QRY, lastHB);
-    long intLockId = 0;
-    long tempExtLockId = generateTemporaryId();
-
-    try (PreparedStatement pstmt = dbConn.prepareStatement(insertLocksQuery)) {
-      for (LockComponent lc : rqst.getComponent()) {
-        intLockId++;
-        String lockType = LockTypeUtil.getEncodingAsStr(lc.getType());
-
-        pstmt.setLong(1, tempExtLockId);
-        pstmt.setLong(2, intLockId);
-        pstmt.setLong(3, txnid);
-        pstmt.setString(4, normalizeCase(lc.getDbname()));
-        pstmt.setString(5, normalizeCase(lc.getTablename()));
-        pstmt.setString(6, normalizePartitionCase(lc.getPartitionname()));
-        pstmt.setString(7, Character.toString(LOCK_WAITING));
-        pstmt.setString(8, lockType);
-        pstmt.setString(9, rqst.getUser());
-        pstmt.setString(10, rqst.getHostname());
-        pstmt.setString(11, rqst.getAgentInfo());
-
-        pstmt.addBatch();
-        if (intLockId % maxBatchSize == 0) {
-          LOG.debug("Executing a batch of <{}> queries. Batch size: {}", insertLocksQuery, maxBatchSize);
-          pstmt.executeBatch();
-        }
-      }
-      if (intLockId % maxBatchSize != 0) {
-        LOG.debug("Executing a batch of <{}> queries. Batch size: {}", insertLocksQuery, intLockId % maxBatchSize);
-        pstmt.executeBatch();
-      }
-    }
-    return tempExtLockId;
-  }
-
-  private long generateTemporaryId() {
-    return -1 * ThreadLocalRandom.current().nextLong();
-  }
-
-  private static String normalizeCase(String s) {
-    return s == null ? null : s.toLowerCase();
-  }
-
-  private static String normalizePartitionCase(String s) {
-    if (s == null) {
-      return null;
-    }
-    Map<String, String> map = Splitter.on(Path.SEPARATOR).withKeyValueSeparator('=').split(s);
-    return FileUtils.makePartName(new ArrayList<>(map.keySet()), new ArrayList<>(map.values()));
-  }
-
-  private LockResponse checkLockWithRetry(Connection dbConn, long extLockId, long txnId, boolean zeroWaitReadEnabled, 
-          boolean isExclusiveCTAS)
-      throws NoSuchLockException, TxnAbortedException, MetaException {
-    try {
-      try {
-        lockInternal();
-        if(dbConn.isClosed()) {
-          //should only get here if retrying this op
-          dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-        }
-        return checkLock(dbConn, extLockId, txnId, zeroWaitReadEnabled, isExclusiveCTAS);
-      } catch (SQLException e) {
-        LOG.error("checkLock failed for extLockId={}/txnId={}. Exception msg: {}", extLockId, txnId, getMessage(e));
-        rollbackDBConn(dbConn);
-        checkRetryable(e, "checkLockWithRetry(" + extLockId + "," + txnId + ")");
-        throw new MetaException("Unable to update transaction database " +
-          StringUtils.stringifyException(e));
-      } finally {
-        unlockInternal();
-        closeDbConn(dbConn);
-      }
-    }
-    catch(RetryException e) {
-      LOG.debug("Going to retry checkLock for extLockId={}/txnId={} after catching RetryException with message: {}",
-              extLockId, txnId, e.getMessage());
-      return checkLockWithRetry(dbConn, extLockId, txnId, zeroWaitReadEnabled, isExclusiveCTAS);
-    }
-  }
-  /**
-   * Why doesn't this get a txnid as parameter?  The caller should either know the txnid or know there isn't one.
-   * Either way getTxnIdFromLockId() will not be needed.  This would be a Thrift change.
-   *
-   * Also, when lock acquisition returns WAITING, it's retried every 15 seconds (best case, see DbLockManager.backoff(),
-   * in practice more often)
-   * which means this is heartbeating way more often than hive.txn.timeout and creating extra load on DB.
-   *
-   * The clients that operate in blocking mode, can't heartbeat a lock until the lock is acquired.
-   * We should make CheckLockRequest include timestamp or last request to skip unnecessary heartbeats. Thrift change.
-   *
-   * {@link #checkLock(java.sql.Connection, long, long, boolean, boolean)}  must run at SERIALIZABLE
-   * (make sure some lock we are checking against doesn't move from W to A in another txn)
-   * but this method can heartbeat in separate txn at READ_COMMITTED.
-   *
-   * Retry-by-caller note:
-   * Retryable because {@link #checkLock(Connection, long, long, boolean, boolean)} is
-   */
-  @Override
-  @RetrySemantics.SafeToRetry
-  public LockResponse checkLock(CheckLockRequest rqst)
-    throws NoSuchTxnException, NoSuchLockException, TxnAbortedException, MetaException {
-    try {
-      Connection dbConn = null;
-      long extLockId = rqst.getLockid();
-      try {
-        lockInternal();
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-        // Heartbeat on the lockid first, to assure that our lock is still valid.
-        // Then look up the lock info (hopefully in the cache).  If these locks
-        // are associated with a transaction then heartbeat on that as well.
-        LockInfo lockInfo = getLockFromLockId(dbConn, extLockId)
-                .orElseThrow(() -> new NoSuchLockException("No such lock " + JavaUtils.lockIdToString(extLockId)));
-        if (lockInfo.txnId > 0) {
-          heartbeatTxn(dbConn, lockInfo.txnId);
-        }
-        else {
-          heartbeatLock(dbConn, extLockId);
-        }
-        //todo: strictly speaking there is a bug here.  heartbeat*() commits but both heartbeat and
-        //checkLock() are in the same retry block, so if checkLock() throws, heartbeat is also retired
-        //extra heartbeat is logically harmless, but ...
-        return checkLock(dbConn, extLockId, lockInfo.txnId, false, false);
-      } catch (SQLException e) {
-        LOG.error("checkLock failed for request={}. Exception msg: {}", rqst, getMessage(e));
-        rollbackDBConn(dbConn);
-        checkRetryable(e, "checkLock(" + rqst + " )");
-        throw new MetaException("Unable to update transaction database " +
-          JavaUtils.lockIdToString(extLockId) + " " + StringUtils.stringifyException(e));
-      } finally {
-        closeDbConn(dbConn);
-        unlockInternal();
-      }
-    } catch (RetryException e) {
-      LOG.debug("Going to retry checkLock for request={} after catching RetryException with message: {}",
-              rqst, e.getMessage());
-      return checkLock(rqst);
-    }
-
-  }
-
-  /**
-   * This would have been made simpler if all locks were associated with a txn.  Then only txn needs to
-   * be heartbeated, committed, etc.  no need for client to track individual locks.
-   * When removing locks not associated with txn this potentially conflicts with
-   * heartbeat/performTimeout which are update/delete of HIVE_LOCKS thus will be locked as needed by db.
-   * since this only removes from HIVE_LOCKS at worst some lock acquire is delayed
-   */
-  @RetrySemantics.Idempotent
-  public void unlock(UnlockRequest rqst) throws TxnOpenException, MetaException {
-    try {
-      Connection dbConn = null;
-      Statement stmt = null;
-      long extLockId = rqst.getLockid();
-      try {
-        /**
-         * This method is logically like commit for read-only auto commit queries.
-         * READ_COMMITTED since this only has 1 delete statement and no new entries with the
-         * same hl_lock_ext_id can be added, i.e. all rows with a given hl_lock_ext_id are
-         * created in a single atomic operation.
-         * Theoretically, this competes with {@link #lock(org.apache.hadoop.hive.metastore.api.LockRequest)}
-         * but hl_lock_ext_id is not known until that method returns.
-         * Also competes with {@link #checkLock(org.apache.hadoop.hive.metastore.api.CheckLockRequest)}
-         * but using SERIALIZABLE doesn't materially change the interaction.
-         * If "delete" stmt misses, additional logic is best effort to produce meaningful error msg.
-         */
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-        stmt = dbConn.createStatement();
-        //hl_txnid <> 0 means it's associated with a transaction
-        String s = "DELETE FROM \"HIVE_LOCKS\" WHERE \"HL_LOCK_EXT_ID\" = " + extLockId + " AND (\"HL_TXNID\" = 0 OR" +
-          " (\"HL_TXNID\" <> 0 AND \"HL_LOCK_STATE\" = '" + LOCK_WAITING + "'))";
-        //(hl_txnid <> 0 AND hl_lock_state = '" + LOCK_WAITING + "') is for multi-statement txns where
-        //some query attempted to lock (thus LOCK_WAITING state) but is giving up due to timeout for example
-        LOG.debug("Going to execute update <{}>", s);
-        int rc = stmt.executeUpdate(s);
-        if (rc < 1) {
-          LOG.info("Failure to unlock any locks with extLockId={}.", extLockId);
-          dbConn.rollback();
-          Optional<LockInfo> optLockInfo = getLockFromLockId(dbConn, extLockId);
-          if (!optLockInfo.isPresent()) {
-            //didn't find any lock with extLockId but at ReadCommitted there is a possibility that
-            //it existed when above delete ran but it didn't have the expected state.
-            LOG.info("No lock in {} mode found for unlock({})", LOCK_WAITING,
-                JavaUtils.lockIdToString(rqst.getLockid()));
-            //bail here to make the operation idempotent
-            return;
-          }
-          LockInfo lockInfo = optLockInfo.get();
-          if (isValidTxn(lockInfo.txnId)) {
-            String msg = "Unlocking locks associated with transaction not permitted.  " + lockInfo;
-            //if a lock is associated with a txn we can only "unlock" if if it's in WAITING state
-            // which really means that the caller wants to give up waiting for the lock
-            LOG.error(msg);
-            throw new TxnOpenException(msg);
-          } else {
-            //we didn't see this lock when running DELETE stmt above but now it showed up
-            //so should "should never happen" happened...
-            String msg = "Found lock in unexpected state " + lockInfo;
-            LOG.error(msg);
-            throw new MetaException(msg);
-          }
-        }
-        LOG.debug("Successfully unlocked at least 1 lock with extLockId={}", extLockId);
-        dbConn.commit();
-      } catch (SQLException e) {
-        LOG.error("Unlock failed for request={}. Exception msg: {}", rqst, getMessage(e));
-        rollbackDBConn(dbConn);
-        checkRetryable(e, "unlock(" + rqst + ")");
-        throw new MetaException("Unable to update transaction database " +
-          JavaUtils.lockIdToString(extLockId) + " " + StringUtils.stringifyException(e));
-      } finally {
-        closeStmt(stmt);
-        closeDbConn(dbConn);
-      }
-    } catch (RetryException e) {
-      unlock(rqst);
-    }
-  }
-
-  /**
-   * used to sort entries in {@link org.apache.hadoop.hive.metastore.api.ShowLocksResponse}
-   */
-  private static class LockInfoExt extends LockInfo {
-    private final ShowLocksResponseElement e;
-    LockInfoExt(ShowLocksResponseElement e) {
-      super(e);
-      this.e = e;
-    }
-  }
-  @RetrySemantics.ReadOnly
-  public ShowLocksResponse showLocks(ShowLocksRequest rqst) throws MetaException {
-    try {
-      Connection dbConn = null;
-      ShowLocksResponse rsp = new ShowLocksResponse();
-      List<ShowLocksResponseElement> elems = new ArrayList<>();
-      List<LockInfoExt> sortedList = new ArrayList<>();
-      PreparedStatement pst = null;
-      try {
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-
-        String s = "SELECT \"HL_LOCK_EXT_ID\", \"HL_TXNID\", \"HL_DB\", \"HL_TABLE\", \"HL_PARTITION\", \"HL_LOCK_STATE\", " +
-          "\"HL_LOCK_TYPE\", \"HL_LAST_HEARTBEAT\", \"HL_ACQUIRED_AT\", \"HL_USER\", \"HL_HOST\", \"HL_LOCK_INT_ID\"," +
-          "\"HL_BLOCKEDBY_EXT_ID\", \"HL_BLOCKEDBY_INT_ID\", \"HL_AGENT_INFO\" FROM \"HIVE_LOCKS\"";
-
-        // Some filters may have been specified in the SHOW LOCKS statement. Add them to the query.
-        String dbName = rqst.getDbname();
-        String tableName = rqst.getTablename();
-        String partName = rqst.getPartname();
-        List<String> params = new ArrayList<>();
-
-        StringBuilder filter = new StringBuilder();
-        if (dbName != null && !dbName.isEmpty()) {
-          filter.append("\"HL_DB\"=?");
-          params.add(dbName);
-        }
-        if (tableName != null && !tableName.isEmpty()) {
-          if (filter.length() > 0) {
-            filter.append(" and ");
-          }
-          filter.append("\"HL_TABLE\"=?");
-          params.add(tableName);
-        }
-        if (partName != null && !partName.isEmpty()) {
-          if (filter.length() > 0) {
-            filter.append(" and ");
-          }
-          filter.append("\"HL_PARTITION\"=?");
-          params.add(partName);
-        }
-        if (rqst.isSetTxnid()) {
-          if (filter.length() > 0) {
-            filter.append(" and ");
-          }
-          filter.append("\"HL_TXNID\"=" + rqst.getTxnid());
-        }
-        String whereClause = filter.toString();
-
-        if (!whereClause.isEmpty()) {
-          s = s + " where " + whereClause;
-        }
-
-        pst = sqlGenerator.prepareStmtWithParameters(dbConn, s, params);
-        LOG.debug("Going to execute query <{}>", s);
-        ResultSet rs = pst.executeQuery();
-        while (rs.next()) {
-          ShowLocksResponseElement e = new ShowLocksResponseElement();
-          e.setLockid(rs.getLong(1));
-          long txnid = rs.getLong(2);
-          if (!rs.wasNull()) e.setTxnid(txnid);
-          e.setDbname(rs.getString(3));
-          e.setTablename(rs.getString(4));
-          String partition = rs.getString(5);
-          if (partition != null) e.setPartname(partition);
-          switch (rs.getString(6).charAt(0)) {
-            case LOCK_ACQUIRED: e.setState(LockState.ACQUIRED); break;
-            case LOCK_WAITING: e.setState(LockState.WAITING); break;
-            default: throw new MetaException("Unknown lock state " + rs.getString(6).charAt(0));
-          }
-
-          char lockChar = rs.getString(7).charAt(0);
-          LockType lockType = LockTypeUtil.getLockTypeFromEncoding(lockChar)
-                  .orElseThrow(() -> new MetaException("Unknown lock type: " + lockChar));
-          e.setType(lockType);
-
-          e.setLastheartbeat(rs.getLong(8));
-          long acquiredAt = rs.getLong(9);
-          if (!rs.wasNull()) e.setAcquiredat(acquiredAt);
-          e.setUser(rs.getString(10));
-          e.setHostname(rs.getString(11));
-          e.setLockIdInternal(rs.getLong(12));
-          long id = rs.getLong(13);
-          if(!rs.wasNull()) {
-            e.setBlockedByExtId(id);
-          }
-          id = rs.getLong(14);
-          if(!rs.wasNull()) {
-            e.setBlockedByIntId(id);
-          }
-          e.setAgentInfo(rs.getString(15));
-          sortedList.add(new LockInfoExt(e));
-        }
-      } catch (SQLException e) {
-        checkRetryable(e, "showLocks(" + rqst + ")");
-        throw new MetaException("Unable to select from transaction database " +
-          StringUtils.stringifyException(e));
-      } finally {
-        closeStmt(pst);
-        closeDbConn(dbConn);
-      }
-      //this ensures that "SHOW LOCKS" prints the locks in the same order as they are examined
-      //by checkLock() - makes diagnostics easier.
-      Collections.sort(sortedList, new LockInfoComparator());
-      for(LockInfoExt lockInfoExt : sortedList) {
-        elems.add(lockInfoExt.e);
-      }
-      rsp.setLocks(elems);
-      return rsp;
-    } catch (RetryException e) {
-      return showLocks(rqst);
-    }
-  }
-
-  /**
-   * {@code ids} should only have txnid or lockid but not both, ideally.
-   * Currently DBTxnManager.heartbeat() enforces this.
-   */
-  @Override
-  @RetrySemantics.SafeToRetry
-  public void heartbeat(HeartbeatRequest ids)
-    throws NoSuchTxnException,  NoSuchLockException, TxnAbortedException, MetaException {
-    try {
-      Connection dbConn = null;
-      try {
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-        heartbeatLock(dbConn, ids.getLockid());
-        heartbeatTxn(dbConn, ids.getTxnid());
-      } catch (SQLException e) {
-        LOG.debug("Going to rollback: ", e);
-        rollbackDBConn(dbConn);
-        checkRetryable(e, "heartbeat(" + ids + ")");
-        throw new MetaException("Unable to select from transaction database " +
-          StringUtils.stringifyException(e));
-      } finally {
-        closeDbConn(dbConn);
-      }
-    } catch (RetryException e) {
-      heartbeat(ids);
-    }
-  }
-  @Override
-  @RetrySemantics.SafeToRetry
-  public HeartbeatTxnRangeResponse heartbeatTxnRange(HeartbeatTxnRangeRequest rqst)
-    throws MetaException {
-    try {
-      Connection dbConn = null;
-      Statement stmt = null;
-      HeartbeatTxnRangeResponse rsp = new HeartbeatTxnRangeResponse();
-      Set<Long> nosuch = new HashSet<>();
-      Set<Long> aborted = new HashSet<>();
-      rsp.setNosuch(nosuch);
-      rsp.setAborted(aborted);
-      try {
-        /**
-         * READ_COMMITTED is sufficient since {@link #heartbeatTxn(java.sql.Connection, long)}
-         * only has 1 update statement in it and
-         * we only update existing txns, i.e. nothing can add additional txns that this operation
-         * would care about (which would have required SERIALIZABLE)
-         */
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-        /*do fast path first (in 1 statement) if doesn't work, rollback and do the long version*/
-        stmt = dbConn.createStatement();
-        List<String> queries = new ArrayList<>();
-        int numTxnsToHeartbeat = (int) (rqst.getMax() - rqst.getMin() + 1);
-        List<Long> txnIds = new ArrayList<>(numTxnsToHeartbeat);
-        for (long txn = rqst.getMin(); txn <= rqst.getMax(); txn++) {
-          txnIds.add(txn);
-        }
-        TxnUtils.buildQueryWithINClause(conf, queries,
-          new StringBuilder("UPDATE \"TXNS\" SET \"TXN_LAST_HEARTBEAT\" = " + getEpochFn(dbProduct) +
-            " WHERE \"TXN_STATE\" = " + TxnStatus.OPEN + " AND "),
-          new StringBuilder(""), txnIds, "\"TXN_ID\"", true, false);
-        int updateCnt = 0;
-        for (String query : queries) {
-          LOG.debug("Going to execute update <{}>", query);
-          updateCnt += stmt.executeUpdate(query);
-        }
-        if (updateCnt == numTxnsToHeartbeat) {
-          //fast pass worked, i.e. all txns we were asked to heartbeat were Open as expected
-          dbConn.commit();
-          return rsp;
-        }
-        //if here, do the slow path so that we can return info txns which were not in expected state
-        dbConn.rollback();
-        for (long txn = rqst.getMin(); txn <= rqst.getMax(); txn++) {
-          try {
-            heartbeatTxn(dbConn, txn);
-          } catch (NoSuchTxnException e) {
-            nosuch.add(txn);
-          } catch (TxnAbortedException e) {
-            aborted.add(txn);
-          }
-        }
-        return rsp;
-      } catch (SQLException e) {
-        LOG.debug("Going to rollback: ", e);
-        rollbackDBConn(dbConn);
-        checkRetryable(e, "heartbeatTxnRange(" + rqst + ")");
-        throw new MetaException("Unable to select from transaction database " +
-          StringUtils.stringifyException(e));
-      } finally {
-        close(null, stmt, dbConn);
-      }
-    } catch (RetryException e) {
-      return heartbeatTxnRange(rqst);
-    }
-  }
-
-  @Deprecated
-  long generateCompactionQueueId(Statement stmt) throws SQLException, MetaException {
-    // Get the id for the next entry in the queue
-    String s = sqlGenerator.addForUpdateClause("SELECT \"NCQ_NEXT\" FROM \"NEXT_COMPACTION_QUEUE_ID\"");
-    LOG.debug("going to execute query <{}>", s);
-    try (ResultSet rs = stmt.executeQuery(s)) {
-      if (!rs.next()) {
-        throw new IllegalStateException("Transaction tables not properly initiated, "
-            + "no record found in next_compaction_queue_id");
-      }
-      long id = rs.getLong(1);
-      s = "UPDATE \"NEXT_COMPACTION_QUEUE_ID\" SET \"NCQ_NEXT\" = " + (id + 1) + " WHERE \"NCQ_NEXT\" = " + id;
-      LOG.debug("Going to execute update <{}>", s);
-      if (stmt.executeUpdate(s) != 1) {
-        //TODO: Eliminate this id generation by implementing: https://issues.apache.org/jira/browse/HIVE-27121
-        LOG.info("The returned compaction ID ({}) already taken, obtaining new", id);
-        return generateCompactionQueueId(stmt);
-      }
-      return id;
-    }
-  }
-
-  long generateCompactionQueueId() throws MetaException {
-    // Get the id for the next entry in the queue
-    String sql = sqlGenerator.addForUpdateClause("SELECT \"NCQ_NEXT\" FROM \"NEXT_COMPACTION_QUEUE_ID\"");
-    LOG.debug("going to execute SQL <{}>", sql);
-    
-    Long allocatedId = jdbcResource.getJdbcTemplate().query(sql, rs -> {
-      if (!rs.next()) {
-        throw new IllegalStateException("Transaction tables not properly initiated, "
-            + "no record found in next_compaction_queue_id");
-      }
-      long id = rs.getLong(1);
-      
-      int count = jdbcResource.getJdbcTemplate().update("UPDATE \"NEXT_COMPACTION_QUEUE_ID\" SET \"NCQ_NEXT\" = :newId WHERE \"NCQ_NEXT\" = :id",
-          new MapSqlParameterSource()
-              .addValue("id", id)
-              .addValue("newId", id + 1));
-      
-      if (count != 1) {
-        //TODO: Eliminate this id generation by implementing: https://issues.apache.org/jira/browse/HIVE-27121
-        LOG.info("The returned compaction ID ({}) already taken, obtaining new", id);
-        return null;
-      }
-      return id;
-    });
-    if (allocatedId == null) {
-      return generateCompactionQueueId();
-    } else {
-      return allocatedId;
-    }
-  }
-
-
-  @Override
-  @RetrySemantics.ReadOnly
-  public long getTxnIdForWriteId(
-      String dbName, String tblName, long writeId) throws MetaException {
-    try {
-      Connection dbConn = null;
-      PreparedStatement pst = null;
-      try {
-        /**
-         * This runs at READ_COMMITTED for exactly the same reason as {@link #getOpenTxnsInfo()}
-         */
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-
-        String query = "SELECT \"T2W_TXNID\" FROM \"TXN_TO_WRITE_ID\" WHERE"
-            + " \"T2W_DATABASE\" = ? AND \"T2W_TABLE\" = ? AND \"T2W_WRITEID\" = " + writeId;
-        pst = sqlGenerator.prepareStmtWithParameters(dbConn, query, Arrays.asList(dbName, tblName));
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Going to execute query <" + query.replace("?", "{}") + ">",
-              quoteString(dbName), quoteString(tblName));
-        }
-        ResultSet rs  = pst.executeQuery();
-        long txnId = -1;
-        if (rs.next()) {
-          txnId = rs.getLong(1);
-        }
-        return txnId;
-      } catch (SQLException e) {
-        checkRetryable(e, "getTxnIdForWriteId");
-        throw new MetaException("Unable to select from transaction database, "
-                + StringUtils.stringifyException(e));
-      } finally {
-        close(null, pst, dbConn);
-      }
-    } catch (RetryException e) {
-      return getTxnIdForWriteId(dbName, tblName, writeId);
-    }
-  }
-
-  @Override
-  @RetrySemantics.Idempotent
-  public CompactionResponse compact(CompactionRequest rqst) throws MetaException {
-    // Put a compaction request in the queue.
-    try {
-      TxnStore.MutexAPI.LockHandle handle = null;
-      try {
-        lockInternal();
-        /**
-         * MUTEX_KEY.CompactionScheduler lock ensures that there is only 1 entry in
-         * Initiated/Working state for any resource.  This ensures that we don't run concurrent
-         * compactions for any resource.
-         */
-        handle = getMutexAPI().acquireLock(MUTEX_KEY.CompactionScheduler.name());
-
-        try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED)) {
-          try (Statement stmt = dbConn.createStatement()) {
-
-            long id = generateCompactionQueueId(stmt);
-
-            GetValidWriteIdsRequest request = new GetValidWriteIdsRequest(
-                Collections.singletonList(getFullTableName(rqst.getDbname(), rqst.getTablename())));
-            final ValidCompactorWriteIdList tblValidWriteIds =
-                TxnUtils.createValidCompactWriteIdList(getValidWriteIds(request).getTblValidWriteIds().get(0));
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("ValidCompactWriteIdList: {}", tblValidWriteIds.writeToString());
-            }
-
-            StringBuilder sb = new StringBuilder("SELECT \"CQ_ID\", \"CQ_STATE\" FROM \"COMPACTION_QUEUE\" WHERE").
-                append(" (\"CQ_STATE\" IN(").
-                append(quoteChar(INITIATED_STATE)).append(",").append(quoteChar(WORKING_STATE)).
-                append(") OR (\"CQ_STATE\" = ").append(quoteChar(READY_FOR_CLEANING)).
-                append(" AND \"CQ_HIGHEST_WRITE_ID\" = ?))").
-                append(" AND \"CQ_DATABASE\"=?").
-                append(" AND \"CQ_TABLE\"=?").append(" AND ");
-            if(rqst.getPartitionname() == null) {
-              sb.append("\"CQ_PARTITION\" is null");
-            } else {
-              sb.append("\"CQ_PARTITION\"=?");
-            }
-
-            try (PreparedStatement pst = dbConn.prepareStatement(sqlGenerator.addEscapeCharacters(sb.toString()))) {
-              pst.setLong(1, tblValidWriteIds.getHighWatermark());
-              pst.setString(2, rqst.getDbname());
-              pst.setString(3, rqst.getTablename());
-              if (rqst.getPartitionname() != null) {
-                pst.setString(4, rqst.getPartitionname());
-              }
-              LOG.debug("Going to execute query <{}>", sb);
-              try (ResultSet rs = pst.executeQuery()) {
-                if(rs.next()) {
-                  long enqueuedId = rs.getLong(1);
-                  String state = CompactionState.fromSqlConst(rs.getString(2)).toString();
-                  LOG.info("Ignoring request to compact {}/{}/{} since it is already {} with id={}", rqst.getDbname(),
-                      rqst.getTablename(), rqst.getPartitionname(), quoteString(state), enqueuedId);
-                  CompactionResponse resp = new CompactionResponse(-1, REFUSED_RESPONSE, false);
-                  resp.setErrormessage("Compaction is already scheduled with state=" + quoteString(state) +
-                      " and id=" + enqueuedId);
-                  return resp;
-                }
-              }
-            }
-            List<String> params = new ArrayList<>();
-            StringBuilder buf = new StringBuilder("INSERT INTO \"COMPACTION_QUEUE\" (\"CQ_ID\", \"CQ_DATABASE\", " +
-                "\"CQ_TABLE\", ");
-            String partName = rqst.getPartitionname();
-            if (partName != null) buf.append("\"CQ_PARTITION\", ");
-            buf.append("\"CQ_STATE\", \"CQ_TYPE\", \"CQ_ENQUEUE_TIME\", \"CQ_POOL_NAME\"");
-            if (rqst.isSetNumberOfBuckets()) {
-              buf.append(", \"CQ_NUMBER_OF_BUCKETS\"");
-            }
-            if (rqst.isSetOrderByClause()) {
-              buf.append(", \"CQ_ORDER_BY\"");
-            }
-            if (rqst.getProperties() != null) {
-              buf.append(", \"CQ_TBLPROPERTIES\"");
-            }
-            if (rqst.getRunas() != null) {
-              buf.append(", \"CQ_RUN_AS\"");
-            }
-            if (rqst.getInitiatorId() != null) {
-              buf.append(", \"CQ_INITIATOR_ID\"");
-            }
-            if (rqst.getInitiatorVersion() != null) {
-              buf.append(", \"CQ_INITIATOR_VERSION\"");
-            }
-            buf.append(") values (");
-            buf.append(id);
-            buf.append(", ?");
-            buf.append(", ?");
-            buf.append(", ");
-            params.add(rqst.getDbname());
-            params.add(rqst.getTablename());
-            if (partName != null) {
-              buf.append("?, '");
-              params.add(partName);
-            } else {
-              buf.append("'");
-            }
-            buf.append(INITIATED_STATE);
-            buf.append("', '");
-            buf.append(TxnUtils.thriftCompactionType2DbType(rqst.getType()));
-            buf.append("',");
-            buf.append(getEpochFn(dbProduct));
-            buf.append(", ?");
-            params.add(rqst.getPoolName());
-            if (rqst.isSetNumberOfBuckets()) {
-              buf.append(", ").append(rqst.getNumberOfBuckets());
-            }
-            if (rqst.isSetOrderByClause()) {
-              buf.append(", ?");
-              params.add(rqst.getOrderByClause());
-            }
-            if (rqst.getProperties() != null) {
-              buf.append(", ?");
-              params.add(new StringableMap(rqst.getProperties()).toString());
-            }
-            if (rqst.getRunas() != null) {
-              buf.append(", ?");
-              params.add(rqst.getRunas());
-            }
-            if (rqst.getInitiatorId() != null) {
-              buf.append(", ?");
-              params.add(rqst.getInitiatorId());
-            }
-            if (rqst.getInitiatorVersion() != null) {
-              buf.append(", ?");
-              params.add(rqst.getInitiatorVersion());
-            }
-            buf.append(")");
-            String s = buf.toString();
-
-            try (PreparedStatement pst = sqlGenerator.prepareStmtWithParameters(dbConn, s, params)) {
-              LOG.debug("Going to execute update <{}>", s);
-              pst.executeUpdate();
-            }
-            LOG.debug("Going to commit");
-            dbConn.commit();
-            return new CompactionResponse(id, INITIATED_RESPONSE, true);
-          } catch (SQLException e) {
-            LOG.debug("Going to rollback: ", e);
-            dbConn.rollback();
-            throw e;
-          }
-        }
-      } catch (SQLException e) {
-        checkRetryable(e, "COMPACT(" + rqst + ")");
-        throw new MetaException("Unable to put the compaction request into the queue: " +
-          StringUtils.stringifyException(e));
-      } finally {
-        if (handle != null) {
-          handle.releaseLocks();
-        }
-        unlockInternal();
-      }
-    } catch (RetryException e) {
-      return compact(rqst);
-    }
-  }
-
-  @Override
-  @RetrySemantics.SafeToRetry
-  public boolean submitForCleanup(CompactionRequest rqst, long highestWriteId, long txnId) throws MetaException {
-    // Put a compaction request in the queue.
-    try {
-      Connection dbConn = null;
-      try {
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-        lockInternal();
-
-        List<String> params = new ArrayList<String>() {{
-          add(rqst.getDbname());
-          add(rqst.getTablename());
-        }};
-        long cqId;
-        try (Statement stmt = dbConn.createStatement()) {
-          cqId = generateCompactionQueueId(stmt);
-        }
-        StringBuilder buf = new StringBuilder(
-            "INSERT INTO \"COMPACTION_QUEUE\" (\"CQ_ID\", \"CQ_HIGHEST_WRITE_ID\", \"CQ_TXN_ID\", \"CQ_ENQUEUE_TIME\", \"CQ_DATABASE\", \"CQ_TABLE\", ");
-        String partName = rqst.getPartitionname();
-        if (partName != null) {
-          buf.append("\"CQ_PARTITION\", ");
-          params.add(partName);
-        }
-        buf.append("\"CQ_STATE\", \"CQ_TYPE\"");
-        params.add(String.valueOf(READY_FOR_CLEANING));
-        params.add(TxnUtils.thriftCompactionType2DbType(rqst.getType()).toString());
-
-        if (rqst.getProperties() != null) {
-          buf.append(", \"CQ_TBLPROPERTIES\"");
-          params.add(new StringableMap(rqst.getProperties()).toString());
-        }
-        if (rqst.getRunas() != null) {
-          buf.append(", \"CQ_RUN_AS\"");
-          params.add(rqst.getRunas());
-        }
-        buf.append(") values (")
-          .append(
-            Stream.of(cqId, highestWriteId, txnId, getEpochFn(dbProduct))
-              .map(Object::toString)
-              .collect(Collectors.joining(", ")))
-          .append(repeat(", ?", params.size()))
-          .append(")");
-
-        String s = buf.toString();
-        try (PreparedStatement pst = sqlGenerator.prepareStmtWithParameters(dbConn, s, params)) {
-          LOG.debug("Going to execute update <{}>", s);
-          pst.executeUpdate();
-        }
-        LOG.debug("Going to commit");
-        dbConn.commit();
-        return true;
-      } catch (SQLException e) {
-        LOG.debug("Going to rollback: ", e);
-        rollbackDBConn(dbConn);
-        checkRetryable(e, "submitForCleanup(" + rqst + ")");
-        throw new MetaException("Failed to submit cleanup request: " +
-          StringUtils.stringifyException(e));
-      } finally {
-        closeDbConn(dbConn);
-        unlockInternal();
-      }
-    } catch (RetryException e) {
-      return submitForCleanup(rqst, highestWriteId, txnId);
-    }
-  }
-  
-  @RetrySemantics.ReadOnly
-  @SuppressWarnings("squid:S2095")
-  public ShowCompactResponse showCompact(ShowCompactRequest rqst) throws MetaException {
-    try {
-      ShowCompactResponse response = new ShowCompactResponse(new ArrayList<>());
-      String query = TxnQueries.SHOW_COMPACTION_QUERY +
-        getShowCompactFilterClause(rqst) +
-        getShowCompactSortingOrderClause(rqst);
-      List<String> params = getShowCompactParamList(rqst);
-
-      try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-        PreparedStatement stmt = sqlGenerator.prepareStmtWithParameters(dbConn, query, params)) {
-        if (rqst.isSetId()) {
-          stmt.setLong(1, rqst.getId());
-        }
-        int rowLimit = (int) rqst.getLimit();
-        if (rowLimit > 0) {
-          stmt.setMaxRows(rowLimit);
-        }
-        LOG.debug("Going to execute query <" + query + ">");
-        try (ResultSet rs = stmt.executeQuery()) {
-          while (rs.next()) {
-            ShowCompactResponseElement e = new ShowCompactResponseElement();
-            e.setDbname(rs.getString(1));
-            e.setTablename(rs.getString(2));
-            e.setPartitionname(rs.getString(3));
-            e.setState(CompactionState.fromSqlConst(rs.getString(4)).toString());
-            try {
-              e.setType(TxnUtils.dbCompactionType2ThriftType(rs.getString(5).charAt(0)));
-            } catch (SQLException ex) {
-              //do nothing to handle RU/D if we add another status
-            }
-            e.setWorkerid(rs.getString(6));
-            long start = rs.getLong(7);
-            if (!rs.wasNull()) {
-              e.setStart(start);
-            }
-            long endTime = rs.getLong(8);
-            if (endTime != -1) {
-              e.setEndTime(endTime);
-            }
-            e.setRunAs(rs.getString(9));
-            e.setHadoopJobId(rs.getString(10));
-            e.setId(rs.getLong(11));
-            e.setErrorMessage(rs.getString(12));
-            long enqueueTime = rs.getLong(13);
-            if (!rs.wasNull()) {
-              e.setEnqueueTime(enqueueTime);
-            }
-            e.setWorkerVersion(rs.getString(14));
-            e.setInitiatorId(rs.getString(15));
-            e.setInitiatorVersion(rs.getString(16));
-            long cleanerStart = rs.getLong(17);
-            if (!rs.wasNull() && (cleanerStart != -1)) {
-              e.setCleanerStart(cleanerStart);
-            }
-            String poolName = rs.getString(18);
-            if (isBlank(poolName)) {
-              e.setPoolName(DEFAULT_POOL_NAME);
-            } else {
-              e.setPoolName(poolName);
-            }
-            e.setTxnId(rs.getLong(19));
-            e.setNextTxnId(rs.getLong(20));
-            e.setCommitTime(rs.getLong(21));
-            e.setHightestTxnId(rs.getLong(22));
-            response.addToCompacts(e);
-          }
-        }
-      } catch (SQLException e) {
-        checkRetryable(e, "showCompact(" + rqst + ")");
-        throw new MetaException("Unable to select from transaction database " +
-            StringUtils.stringifyException(e));
-      }
-      return response;
-    } catch (RetryException e) {
-      return showCompact(rqst);
-    }
-  }
-
-  private String getShowCompactSortingOrderClause(ShowCompactRequest request) {
-    String sortingOrder = request.getOrder();
-    return isNotBlank(sortingOrder) ? "  ORDER BY  " + sortingOrder : TxnQueries.SHOW_COMPACTION_ORDERBY_CLAUSE;
-  }
-
-  private List<String> getShowCompactParamList(ShowCompactRequest request) throws MetaException {
-    if (request.getId() > 0) {
-      return Collections.emptyList();
-    }
-    String poolName = request.getPoolName();
-    String dbName = request.getDbName();
-    String tableName = request.getTbName();
-    String partName = request.getPartName();
-    CompactionType type = request.getType();
-    String state = request.getState();
-  
-    List<String> params = new ArrayList<>();
-    if (isNotBlank(dbName)) {
-      params.add(dbName);
-    }
-    if (isNotBlank(tableName)) {
-      params.add(tableName);
-    }
-    if (isNotBlank(partName)) {
-      params.add(partName);
-    }
-    if (isNotBlank(state)) {
-      params.add(state);
-    }
-    if (type != null) {
-      params.add(TxnUtils.thriftCompactionType2DbType(type).toString());
-    }
-    if (isNotBlank(poolName)) {
-      params.add(poolName);
-    }
-    return params;
-  }
-
-  private String getShowCompactFilterClause(ShowCompactRequest request) {
-    List<String> params = new ArrayList<>();
-    
-    if (request.getId() > 0) {
-      params.add("\"CC_ID\"=?");
-    } else {
-      if (isNotBlank(request.getDbName())) {
-        params.add("\"CC_DATABASE\"=?");
-      }
-      if (isNotBlank(request.getTbName())) {
-        params.add("\"CC_TABLE\"=?");
-      }
-      if (isNotBlank(request.getPartName())) {
-        params.add("\"CC_PARTITION\"=?");
-      }
-      if (isNotBlank(request.getState())) {
-        params.add("\"CC_STATE\"=?");
-      }
-      if (request.getType() != null) {
-        params.add("\"CC_TYPE\"=?");
-      }
-      if (isNotBlank(request.getPoolName())) {
-        params.add("\"CC_POOL_NAME\"=?");
-      }
-    }
-    return !params.isEmpty() ? 
-      " WHERE " + StringUtils.join(" AND ", params) : EMPTY;
-  }
-
-  /**
-   * We assume this is only called by metadata cache server to know if there are new base/delta files should be read.
-   * The query filters compactions by state and only returns SUCCEEDED or READY_FOR_CLEANING compactions because
-   * only these two states means there are new files ready to be read.
-   */
-  @RetrySemantics.ReadOnly
-  public GetLatestCommittedCompactionInfoResponse getLatestCommittedCompactionInfo(
-      GetLatestCommittedCompactionInfoRequest rqst) throws MetaException {
-    GetLatestCommittedCompactionInfoResponse response = new GetLatestCommittedCompactionInfoResponse(new ArrayList<>());
-    Connection dbConn = null;
-    PreparedStatement pst = null;
-    ResultSet rs = null;
-    try {
-      try {
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-
-        List<String> params = new ArrayList<>();
-        // This query combines the result sets of SUCCEEDED compactions and READY_FOR_CLEANING compactions
-        // We also sort the result by CC_ID in descending order so that we can keep only the latest record
-        // according to the order in result set
-        StringBuilder sb = new StringBuilder()
-            .append("SELECT * FROM (")
-            .append("   SELECT")
-            .append("   \"CC_ID\", \"CC_DATABASE\", \"CC_TABLE\", \"CC_PARTITION\", \"CC_TYPE\"")
-            .append("   FROM \"COMPLETED_COMPACTIONS\"")
-            .append("     WHERE \"CC_STATE\" = " + quoteChar(SUCCEEDED_STATE))
-            .append("   UNION ALL")
-            .append("   SELECT")
-            .append("   \"CQ_ID\" AS \"CC_ID\", \"CQ_DATABASE\" AS \"CC_DATABASE\"")
-            .append("   ,\"CQ_TABLE\" AS \"CC_TABLE\", \"CQ_PARTITION\" AS \"CC_PARTITION\"")
-            .append("   ,\"CQ_TYPE\" AS \"CC_TYPE\"")
-            .append("   FROM \"COMPACTION_QUEUE\"")
-            .append("     WHERE \"CQ_STATE\" = " + quoteChar(READY_FOR_CLEANING))
-            .append(") AS compactions ")
-            .append(" WHERE \"CC_DATABASE\" = ? AND \"CC_TABLE\" = ?");
-        params.add(rqst.getDbname());
-        params.add(rqst.getTablename());
-        if (rqst.getPartitionnamesSize() > 0) {
-          sb.append(" AND \"CC_PARTITION\" IN (");
-          sb.append(String.join(",",
-              Collections.nCopies(rqst.getPartitionnamesSize(), "?")));
-          sb.append(")");
-          params.addAll(rqst.getPartitionnames());
-        }
-        if (rqst.isSetLastCompactionId()) {
-          sb.append(" AND \"CC_ID\" > ?");
-        }
-        sb.append(" ORDER BY \"CC_ID\" DESC");
-
-        pst = sqlGenerator.prepareStmtWithParameters(dbConn, sb.toString(), params);
-        if (rqst.isSetLastCompactionId()) {
-          pst.setLong(params.size() + 1, rqst.getLastCompactionId());
-        }
-        LOG.debug("Going to execute query <{}>", sb);
-        rs = pst.executeQuery();
-        Set<String> partitionSet = new HashSet<>();
-        while (rs.next()) {
-          CompactionInfoStruct lci = new CompactionInfoStruct();
-          lci.setId(rs.getLong(1));
-          lci.setDbname(rs.getString(2));
-          lci.setTablename(rs.getString(3));
-          String partition = rs.getString(4);
-          if (!rs.wasNull()) {
-            lci.setPartitionname(partition);
-          }
-          lci.setType(TxnUtils.dbCompactionType2ThriftType(rs.getString(5).charAt(0)));
-          // Only put the latest record of each partition into response
-          if (!partitionSet.contains(partition)) {
-            response.addToCompactions(lci);
-            partitionSet.add(partition);
-          }
-        }
-      } catch (SQLException e) {
-        LOG.error("Unable to execute query", e);
-        checkRetryable(e, "getLatestCommittedCompactionInfo");
-      } finally {
-        close(rs, pst, dbConn);
-      }
-      return response;
-    } catch (RetryException e) {
-      return getLatestCommittedCompactionInfo(rqst);
-    }
-  }
-
-  public MetricsInfo getMetricsInfo() throws MetaException {
-    Connection dbConn = null;
-    try {
-      MetricsInfo metrics = new MetricsInfo();
-      String s = MessageFormat.format(SELECT_METRICS_INFO_QUERY, getEpochFn(dbProduct));
-      try {
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-        try (Statement stmt = dbConn.createStatement()){
-          ResultSet rs = stmt.executeQuery(s);
-          if (rs.next()) {
-            metrics.setTxnToWriteIdCount(rs.getInt(1));
-            metrics.setCompletedTxnsCount(rs.getInt(2));
-            metrics.setOpenReplTxnsCount(rs.getInt(3));
-            metrics.setOldestOpenReplTxnId(rs.getInt(4));
-            metrics.setOldestOpenReplTxnAge(rs.getInt(5));
-            metrics.setOpenNonReplTxnsCount(rs.getInt(6));
-            metrics.setOldestOpenNonReplTxnId(rs.getInt(7));
-            metrics.setOldestOpenNonReplTxnAge(rs.getInt(8));
-            metrics.setAbortedTxnsCount(rs.getInt(9));
-            metrics.setOldestAbortedTxnId(rs.getInt(10));
-            metrics.setOldestAbortedTxnAge(rs.getInt(11));
-            metrics.setLocksCount(rs.getInt(12));
-            metrics.setOldestLockAge(rs.getInt(13));
-            metrics.setOldestReadyForCleaningAge(rs.getInt(14));
-          }
-        }
-        try (PreparedStatement pstmt = dbConn.prepareStatement(SELECT_TABLES_WITH_X_ABORTED_TXNS)) {
-          Set<String> resourceNames = new TreeSet<>();
-          pstmt.setInt(1, MetastoreConf.getIntVar(conf, ConfVars.METASTORE_ACIDMETRICS_TABLES_WITH_ABORTED_TXNS_THRESHOLD));
-          ResultSet rs = pstmt.executeQuery();
-          while (rs.next()) {
-            String resourceName = rs.getString(1) + "." + rs.getString(2);
-            String partName = rs.getString(3);
-            resourceName = partName != null ? resourceName + "#" + partName : resourceName;
-            resourceNames.add(resourceName);
-          }
-          metrics.setTablesWithXAbortedTxnsCount(resourceNames.size());
-          metrics.setTablesWithXAbortedTxns(resourceNames);
-        }
-        return metrics;
-      } catch (SQLException e) {
-        LOG.error("Unable to getMetricsInfo", e);
-        checkRetryable(e, "getMetricsInfo");
-        throw new MetaException("Unable to execute getMetricsInfo() " + StringUtils.stringifyException(e));
-      } finally {
-        closeDbConn(dbConn);
-      }
-    } catch (RetryException e) {
-      return getMetricsInfo();
-    }
-  }
-
-
-  private static void shouldNeverHappen(long txnid) {
-    throw new RuntimeException("This should never happen: " + JavaUtils.txnIdToString(txnid));
-  }
-  private static void shouldNeverHappen(long txnid, long extLockId, long intLockId) {
-    throw new RuntimeException("This should never happen: " + JavaUtils.txnIdToString(txnid) + " "
-      + JavaUtils.lockIdToString(extLockId) + " " + intLockId);
-  }
-
-  /**
-   * Retry-by-caller note:
-   * This may be retried after dbConn.commit.  At worst, it will create duplicate entries in
-   * TXN_COMPONENTS which won't affect anything.  See more comments in {@link #commitTxn(CommitTxnRequest)}
-   */
-  @Override
-  @RetrySemantics.SafeToRetry
-  public void addDynamicPartitions(AddDynamicPartitions rqst)
-      throws NoSuchTxnException,  TxnAbortedException, MetaException {
-    Connection dbConn = null;
-    Statement stmt = null;
-    try {
-      try {
-        lockInternal();
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-        stmt = dbConn.createStatement();
-        TxnType txnType = getOpenTxnTypeAndLock(stmt, rqst.getTxnid());
-        if (txnType == null) {
-          //ensures txn is still there and in expected state
-          ensureValidTxn(dbConn, rqst.getTxnid(), stmt);
-          shouldNeverHappen(rqst.getTxnid());
-        }
-        //for RU this may be null so we should default it to 'u' which is most restrictive
-        OperationType ot = OperationType.UPDATE;
-        if(rqst.isSetOperationType()) {
-          ot = OperationType.fromDataOperationType(rqst.getOperationType());
-        }
-
-        Long writeId = rqst.getWriteid();
-        try (PreparedStatement pstmt = dbConn.prepareStatement(TXN_COMPONENTS_INSERT_QUERY)) {
-          int insertCounter = 0;
-          for (String partName : rqst.getPartitionnames()) {
-            pstmt.setLong(1, rqst.getTxnid());
-            pstmt.setString(2, normalizeCase(rqst.getDbname()));
-            pstmt.setString(3, normalizeCase(rqst.getTablename()));
-            pstmt.setString(4, partName);
-            pstmt.setString(5, ot.getSqlConst());
-            pstmt.setObject(6, writeId);
-
-            pstmt.addBatch();
-            insertCounter++;
-            if (insertCounter % maxBatchSize == 0) {
-              LOG.debug("Executing a batch of <{}> queries. Batch size: {}", TXN_COMPONENTS_INSERT_QUERY, maxBatchSize);
-              pstmt.executeBatch();
-            }
-          }
-          if (insertCounter % maxBatchSize != 0) {
-            LOG.debug("Executing a batch of <{}> queries. Batch size: {}", TXN_COMPONENTS_INSERT_QUERY,
-                insertCounter % maxBatchSize);
-            pstmt.executeBatch();
-          }
-        }
-        try (PreparedStatement pstmt = dbConn.prepareStatement(TXN_COMPONENTS_DP_DELETE_QUERY)) {
-          pstmt.setLong(1, rqst.getTxnid());
-          pstmt.setString(2, normalizeCase(rqst.getDbname()));
-          pstmt.setString(3, normalizeCase(rqst.getTablename()));
-          pstmt.execute();
-        }
-        LOG.debug("Going to commit");
-        dbConn.commit();
-      } catch (SQLException e) {
-        LOG.debug("Going to rollback: ", e);
-        rollbackDBConn(dbConn);
-        checkRetryable(e, "addDynamicPartitions(" + rqst + ")");
-        throw new MetaException("Unable to insert into from transaction database " +
-          StringUtils.stringifyException(e));
-      } finally {
-        close(null, stmt, dbConn);
-        unlockInternal();
-      }
-    } catch (RetryException e) {
-      addDynamicPartitions(rqst);
-    }
-  }
-
-  /**
-   * Clean up corresponding records in metastore tables when corresponding object is dropped,
-   * specifically: TXN_COMPONENTS, COMPLETED_TXN_COMPONENTS, COMPACTION_QUEUE, COMPLETED_COMPACTIONS
-   * Retry-by-caller note: this is only idempotent assuming it's only called by dropTable/Db/etc
-   * operations.
-   *
-   * HIVE_LOCKS and WS_SET are cleaned up by {@link AcidHouseKeeperService}, if turned on
-   */
-  @Override
-  @RetrySemantics.Idempotent
-  public void cleanupRecords(HiveObjectType type, Database db, Table table,
-        Iterator<Partition> partitionIterator, boolean keepTxnToWriteIdMetaData) throws MetaException {
-    cleanupRecords(type, db, table, partitionIterator, keepTxnToWriteIdMetaData, 0);
-  }
-
-  @Override
-  @RetrySemantics.Idempotent
-  public void cleanupRecords(HiveObjectType type, Database db, Table table,
-        Iterator<Partition> partitionIterator, long txnId) throws MetaException {
-    cleanupRecords(type, db , table, partitionIterator, false, txnId);
-  }
-  
-  private void cleanupRecords(HiveObjectType type, Database db, Table table,
-      Iterator<Partition> partitionIterator, boolean keepTxnToWriteIdMetaData, long txnId) throws MetaException {
-
-    // cleanup should be done only for objects belonging to default catalog
-    final String defaultCatalog = getDefaultCatalog(conf);
-
-    try {
-      Connection dbConn = null;
-      Statement stmt = null;
-
-      try {
-        String dbName;
-        String tblName;
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-        stmt = dbConn.createStatement();
-        List<String> queries = new ArrayList<>();
-        StringBuilder buff = new StringBuilder();
-
-        switch (type) {
-          case DATABASE: {
-            dbName = db.getName();
-            if(!defaultCatalog.equals(db.getCatalogName())) {
-              LOG.debug("Skipping cleanup because db: " + dbName + " belongs to catalog "
-                  + "other than default catalog: " + db.getCatalogName());
-              return;
-            }
-
-            buff.append("DELETE FROM \"TXN_COMPONENTS\" WHERE \"TC_DATABASE\"='");
-            buff.append(dbName);
-            buff.append("'");
-            queries.add(buff.toString());
-
-            buff.setLength(0);
-            buff.append("DELETE FROM \"COMPLETED_TXN_COMPONENTS\" WHERE \"CTC_DATABASE\"='");
-            buff.append(dbName);
-            buff.append("'");
-            queries.add(buff.toString());
-            
-            buff.setLength(0);
-            buff.append("DELETE FROM \"COMPACTION_QUEUE\" WHERE \"CQ_DATABASE\"='");
-            buff.append(dbName);
-            buff.append("' AND \"CQ_TXN_ID\"!=").append(txnId);
-            queries.add(buff.toString());
-
-            buff.setLength(0);
-            buff.append("DELETE FROM \"COMPLETED_COMPACTIONS\" WHERE \"CC_DATABASE\"='");
-            buff.append(dbName);
-            buff.append("'");
-            queries.add(buff.toString());
-
-            buff.setLength(0);
-            buff.append("DELETE FROM \"TXN_TO_WRITE_ID\" WHERE \"T2W_DATABASE\"='");
-            buff.append(dbName.toLowerCase());
-            buff.append("'");
-            queries.add(buff.toString());
-
-            buff.setLength(0);
-            buff.append("DELETE FROM \"NEXT_WRITE_ID\" WHERE \"NWI_DATABASE\"='");
-            buff.append(dbName.toLowerCase());
-            buff.append("'");
-            queries.add(buff.toString());
-
-            buff.setLength(0);
-            buff.append("DELETE FROM \"COMPACTION_METRICS_CACHE\" WHERE \"CMC_DATABASE\"='");
-            buff.append(dbName);
-            buff.append("'");
-            queries.add(buff.toString());
-
-            break;
-          }
-          case TABLE: {
-            dbName = table.getDbName();
-            tblName = table.getTableName();
-            if(!defaultCatalog.equals(table.getCatName())) {
-              LOG.debug("Skipping cleanup because table: {} belongs to catalog other than default catalog: {}", tblName,
-                  table.getCatName());
-              return;
-            }
-
-            buff.append("DELETE FROM \"TXN_COMPONENTS\" WHERE \"TC_DATABASE\"='");
-            buff.append(dbName);
-            buff.append("' AND \"TC_TABLE\"='");
-            buff.append(tblName);
-            buff.append("'");
-            queries.add(buff.toString());
-
-            buff.setLength(0);
-            buff.append("DELETE FROM \"COMPLETED_TXN_COMPONENTS\" WHERE \"CTC_DATABASE\"='");
-            buff.append(dbName);
-            buff.append("' AND \"CTC_TABLE\"='");
-            buff.append(tblName);
-            buff.append("'");
-            queries.add(buff.toString());
-
-            buff.setLength(0);
-            buff.append("DELETE FROM \"COMPACTION_QUEUE\" WHERE \"CQ_DATABASE\"='");
-            buff.append(dbName);
-            buff.append("' AND \"CQ_TABLE\"='");
-            buff.append(tblName);
-            buff.append("'");
-            queries.add(buff.toString());
-
-            buff.setLength(0);
-            buff.append("DELETE FROM \"COMPLETED_COMPACTIONS\" WHERE \"CC_DATABASE\"='");
-            buff.append(dbName);
-            buff.append("' AND \"CC_TABLE\"='");
-            buff.append(tblName);
-            buff.append("'");
-            queries.add(buff.toString());
-
-            if (!keepTxnToWriteIdMetaData) {
-              buff.setLength(0);
-              buff.append("DELETE FROM \"TXN_TO_WRITE_ID\" WHERE \"T2W_DATABASE\"='");
-              buff.append(dbName.toLowerCase());
-              buff.append("' AND \"T2W_TABLE\"='");
-              buff.append(tblName.toLowerCase());
-              buff.append("'");
-              queries.add(buff.toString());
-
-              buff.setLength(0);
-              buff.append("DELETE FROM \"NEXT_WRITE_ID\" WHERE \"NWI_DATABASE\"='");
-              buff.append(dbName.toLowerCase());
-              buff.append("' AND \"NWI_TABLE\"='");
-              buff.append(tblName.toLowerCase());
-              buff.append("'");
-              queries.add(buff.toString());
-            }
-            buff.setLength(0);
-            buff.append("DELETE FROM \"COMPACTION_METRICS_CACHE\" WHERE \"CMC_DATABASE\"='");
-            buff.append(dbName);
-            buff.append("' AND \"CMC_TABLE\"='");
-            buff.append(tblName);
-            buff.append("'");
-            queries.add(buff.toString());
-
-            break;
-          }
-          case PARTITION: {
-            dbName = table.getDbName();
-            tblName = table.getTableName();
-            if(!defaultCatalog.equals(table.getCatName())) {
-              LOG.debug("Skipping cleanup because partitions belong to catalog other than default catalog: {}",
-                  table.getCatName());
-              return;
-            }
-
-            List<FieldSchema> partCols = table.getPartitionKeys();  // partition columns
-            List<String> partVals;                                  // partition values
-            String partName;
-
-            while (partitionIterator.hasNext()) {
-              Partition p = partitionIterator.next();
-              partVals = p.getValues();
-              partName = Warehouse.makePartName(partCols, partVals);
-
-              buff.setLength(0);
-              buff.append("DELETE FROM \"TXN_COMPONENTS\" WHERE \"TC_DATABASE\"='");
-              buff.append(dbName);
-              buff.append("' AND \"TC_TABLE\"='");
-              buff.append(tblName);
-              buff.append("' AND \"TC_PARTITION\"='");
-              buff.append(partName);
-              buff.append("'");
-              queries.add(buff.toString());
-
-              buff.setLength(0);
-              buff.append("DELETE FROM \"COMPLETED_TXN_COMPONENTS\" WHERE \"CTC_DATABASE\"='");
-              buff.append(dbName);
-              buff.append("' AND \"CTC_TABLE\"='");
-              buff.append(tblName);
-              buff.append("' AND \"CTC_PARTITION\"='");
-              buff.append(partName);
-              buff.append("'");
-              queries.add(buff.toString());
-
-              buff.setLength(0);
-              buff.append("DELETE FROM \"COMPACTION_QUEUE\" WHERE \"CQ_DATABASE\"='");
-              buff.append(dbName);
-              buff.append("' AND \"CQ_TABLE\"='");
-              buff.append(tblName);
-              buff.append("' AND \"CQ_PARTITION\"='");
-              buff.append(partName);
-              buff.append("'");
-              queries.add(buff.toString());
-
-              buff.setLength(0);
-              buff.append("DELETE FROM \"COMPLETED_COMPACTIONS\" WHERE \"CC_DATABASE\"='");
-              buff.append(dbName);
-              buff.append("' AND \"CC_TABLE\"='");
-              buff.append(tblName);
-              buff.append("' AND \"CC_PARTITION\"='");
-              buff.append(partName);
-              buff.append("'");
-              queries.add(buff.toString());
-
-              buff.setLength(0);
-              buff.append("DELETE FROM \"COMPACTION_METRICS_CACHE\" WHERE \"CMC_DATABASE\"='");
-              buff.append(dbName);
-              buff.append("' AND \"CMC_TABLE\"='");
-              buff.append(tblName);
-              buff.append("' AND \"CMC_PARTITION\"='");
-              buff.append(partName);
-              buff.append("'");
-              queries.add(buff.toString());
-            }
-
-            break;
-          }
-          default: {
-            throw new MetaException("Invalid object type for cleanup: " + type);
-          }
-        }
-
-        for (String query : queries) {
-          LOG.debug("Going to execute update <{}>", query);
-          stmt.executeUpdate(query);
-        }
-
-        LOG.debug("Going to commit");
-        dbConn.commit();
-      } catch (SQLException e) {
-        LOG.debug("Going to rollback: ", e);
-        rollbackDBConn(dbConn);
-        checkRetryable(e, "cleanupRecords");
-        if (e.getMessage().contains("does not exist")) {
-          LOG.warn("Cannot perform cleanup since metastore table does not exist");
-        } else {
-          throw new MetaException("Unable to clean up " + StringUtils.stringifyException(e));
-        }
-      } finally {
-        closeStmt(stmt);
-        closeDbConn(dbConn);
-      }
-    } catch (RetryException e) {
-      cleanupRecords(type, db, table, partitionIterator, keepTxnToWriteIdMetaData);
-    }
-  }
-  /**
-   * Catalog hasn't been added to transactional tables yet, so it's passed in but not used.
-   */
-  @Override
-  public void onRename(String oldCatName, String oldDbName, String oldTabName, String oldPartName,
-      String newCatName, String newDbName, String newTabName, String newPartName)
-      throws MetaException {
-    String callSig = "onRename(" +
-        oldCatName + "," + oldDbName + "," + oldTabName + "," + oldPartName + "," +
-        newCatName + "," + newDbName + "," + newTabName + "," + newPartName + ")";
-
-    if(newPartName != null) {
-      assert oldPartName != null && oldTabName != null && oldDbName != null && oldCatName != null :
-      callSig;
-    }
-    if(newTabName != null) {
-      assert oldTabName != null && oldDbName != null && oldCatName != null : callSig;
-    }
-    if(newDbName != null) {
-      assert oldDbName != null && oldCatName != null : callSig;
-    }
-
-    try {
-      Connection dbConn = null;
-      Statement stmt = null;
-      try {
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-        stmt = dbConn.createStatement();
-        List<String> queries = new ArrayList<>();
-
-        String update = "UPDATE \"TXN_COMPONENTS\" SET ";
-        String where = " WHERE ";
-        if(oldPartName != null) {
-          update += "\"TC_PARTITION\" = " + quoteString(newPartName) + ", ";
-          where += "\"TC_PARTITION\" = " + quoteString(oldPartName) + " AND ";
-        }
-        if(oldTabName != null) {
-          update += "\"TC_TABLE\" = " + quoteString(normalizeCase(newTabName)) + ", ";
-          where += "\"TC_TABLE\" = " + quoteString(normalizeCase(oldTabName)) + " AND ";
-        }
-        if(oldDbName != null) {
-          update += "\"TC_DATABASE\" = " + quoteString(normalizeCase(newDbName));
-          where += "\"TC_DATABASE\" = " + quoteString(normalizeCase(oldDbName));
-        }
-        queries.add(update + where);
-
-        update = "UPDATE \"COMPLETED_TXN_COMPONENTS\" SET ";
-        where = " WHERE ";
-        if(oldPartName != null) {
-          update += "\"CTC_PARTITION\" = " + quoteString(newPartName) + ", ";
-          where += "\"CTC_PARTITION\" = " + quoteString(oldPartName) + " AND ";
-        }
-        if(oldTabName != null) {
-          update += "\"CTC_TABLE\" = " + quoteString(normalizeCase(newTabName)) + ", ";
-          where += "\"CTC_TABLE\" = " + quoteString(normalizeCase(oldTabName)) + " AND ";
-        }
-        if(oldDbName != null) {
-          update += "\"CTC_DATABASE\" = " + quoteString(normalizeCase(newDbName));
-          where += "\"CTC_DATABASE\" = " + quoteString(normalizeCase(oldDbName));
-        }
-        queries.add(update + where);
-
-        update = "UPDATE \"HIVE_LOCKS\" SET ";
-        where = " WHERE ";
-        if(oldPartName != null) {
-          update += "\"HL_PARTITION\" = " + quoteString(newPartName) + ", ";
-          where += "\"HL_PARTITION\" = " + quoteString(oldPartName) + " AND ";
-        }
-        if(oldTabName != null) {
-          update += "\"HL_TABLE\" = " + quoteString(normalizeCase(newTabName)) + ", ";
-          where += "\"HL_TABLE\" = " + quoteString(normalizeCase(oldTabName)) + " AND ";
-        }
-        if(oldDbName != null) {
-          update += "\"HL_DB\" = " + quoteString(normalizeCase(newDbName));
-          where += "\"HL_DB\" = " + quoteString(normalizeCase(oldDbName));
-        }
-        queries.add(update + where);
-
-        update = "UPDATE \"COMPACTION_QUEUE\" SET ";
-        where = " WHERE ";
-        if(oldPartName != null) {
-          update += "\"CQ_PARTITION\" = " + quoteString(newPartName) + ", ";
-          where += "\"CQ_PARTITION\" = " + quoteString(oldPartName) + " AND ";
-        }
-        if(oldTabName != null) {
-          update += "\"CQ_TABLE\" = " + quoteString(normalizeCase(newTabName)) + ", ";
-          where += "\"CQ_TABLE\" = " + quoteString(normalizeCase(oldTabName)) + " AND ";
-        }
-        if(oldDbName != null) {
-          update += "\"CQ_DATABASE\" = " + quoteString(normalizeCase(newDbName));
-          where += "\"CQ_DATABASE\" = " + quoteString(normalizeCase(oldDbName));
-        }
-        queries.add(update + where);
-
-        update = "UPDATE \"COMPLETED_COMPACTIONS\" SET ";
-        where = " WHERE ";
-        if(oldPartName != null) {
-          update += "\"CC_PARTITION\" = " + quoteString(newPartName) + ", ";
-          where += "\"CC_PARTITION\" = " + quoteString(oldPartName) + " AND ";
-        }
-        if(oldTabName != null) {
-          update += "\"CC_TABLE\" = " + quoteString(normalizeCase(newTabName)) + ", ";
-          where += "\"CC_TABLE\" = " + quoteString(normalizeCase(oldTabName)) + " AND ";
-        }
-        if(oldDbName != null) {
-          update += "\"CC_DATABASE\" = " + quoteString(normalizeCase(newDbName));
-          where += "\"CC_DATABASE\" = " + quoteString(normalizeCase(oldDbName));
-        }
-        queries.add(update + where);
-
-        update = "UPDATE \"WRITE_SET\" SET ";
-        where = " WHERE ";
-        if(oldPartName != null) {
-          update += "\"WS_PARTITION\" = " + quoteString(newPartName) + ", ";
-          where += "\"WS_PARTITION\" = " + quoteString(oldPartName) + " AND ";
-        }
-        if(oldTabName != null) {
-          update += "\"WS_TABLE\" = " + quoteString(normalizeCase(newTabName)) + ", ";
-          where += "\"WS_TABLE\" = " + quoteString(normalizeCase(oldTabName)) + " AND ";
-        }
-        if(oldDbName != null) {
-          update += "\"WS_DATABASE\" = " + quoteString(normalizeCase(newDbName));
-          where += "\"WS_DATABASE\" = " + quoteString(normalizeCase(oldDbName));
-        }
-        queries.add(update + where);
-
-        update = "UPDATE \"TXN_TO_WRITE_ID\" SET ";
-        where = " WHERE ";
-        if(oldTabName != null) {
-          update += "\"T2W_TABLE\" = " + quoteString(normalizeCase(newTabName)) + ", ";
-          where += "\"T2W_TABLE\" = " + quoteString(normalizeCase(oldTabName)) + " AND ";
-        }
-        if(oldDbName != null) {
-          update += "\"T2W_DATABASE\" = " + quoteString(normalizeCase(newDbName));
-          where += "\"T2W_DATABASE\" = " + quoteString(normalizeCase(oldDbName));
-        }
-        queries.add(update + where);
-
-        update = "UPDATE \"NEXT_WRITE_ID\" SET ";
-        where = " WHERE ";
-        if(oldTabName != null) {
-          update += "\"NWI_TABLE\" = " + quoteString(normalizeCase(newTabName)) + ", ";
-          where += "\"NWI_TABLE\" = " + quoteString(normalizeCase(oldTabName)) + " AND ";
-        }
-        if(oldDbName != null) {
-          update += "\"NWI_DATABASE\" = " + quoteString(normalizeCase(newDbName));
-          where += "\"NWI_DATABASE\" = " + quoteString(normalizeCase(oldDbName));
-        }
-        queries.add(update + where);
-
-        update = "UPDATE \"COMPACTION_METRICS_CACHE\" SET";
-        where = " WHERE ";
-        if (oldPartName != null) {
-          update += "\"CMC_PARTITION\" = " + quoteString(normalizeCase(newPartName)) + ", ";
-          where += "\"CMC_PARTITION\" = " + quoteString(normalizeCase(oldPartName)) + " AND ";
-        }
-        if (oldTabName != null) {
-          update += "\"CMC_TABLE\" = " + quoteString(normalizeCase(newTabName)) + ", ";
-          where += "\"CMC_TABLE\" = " + quoteString(normalizeCase(oldTabName)) + " AND ";
-        }
-        if (oldDbName != null) {
-          update += "\"CMC_DATABASE\" = " + quoteString(normalizeCase(newDbName));
-          where += "\"CMC_DATABASE\" = " + quoteString(normalizeCase(oldDbName));
-        }
-        queries.add(update + where);
-
-        for (String query : queries) {
-          LOG.debug("Going to execute update <{}>", query);
-          stmt.executeUpdate(query);
-        }
-
-        LOG.debug("Going to commit: {}", callSig);
-        dbConn.commit();
-      } catch (SQLException e) {
-        LOG.debug("Going to rollback: {}", callSig);
-        rollbackDBConn(dbConn);
-        checkRetryable(e, callSig);
-        if (e.getMessage().contains("does not exist")) {
-          LOG.warn("Cannot perform {} since metastore table does not exist", callSig);
-        } else {
-          throw new MetaException("Unable to " + callSig + ":" + StringUtils.stringifyException(e));
-        }
-      } finally {
-        closeStmt(stmt);
-        closeDbConn(dbConn);
-      }
-    } catch (RetryException e) {
-      onRename(oldCatName, oldDbName, oldTabName, oldPartName,
-          newCatName, newDbName, newTabName, newPartName);
-    }
-  }
-  /**
-   * For testing only, do not use.
-   */
-  @VisibleForTesting
-  public int numLocksInLockTable() throws SQLException, MetaException {
-    Connection dbConn = null;
-    Statement stmt = null;
-    ResultSet rs = null;
-    try {
-      dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-      stmt = dbConn.createStatement();
-      String s = "SELECT COUNT(*) FROM \"HIVE_LOCKS\"";
-      LOG.debug("Going to execute query <{}>", s);
-      rs = stmt.executeQuery(s);
-      rs.next();
-      int rc = rs.getInt(1);
-      // Necessary to clean up the transaction in the db.
-      dbConn.rollback();
-      return rc;
-    } finally {
-      close(rs, stmt, dbConn);
-    }
-  }
-
-  /**
-   * For testing only, do not use.
-   */
-  public long setTimeout(long milliseconds) {
-    long previous_timeout = timeout;
-    timeout = milliseconds;
-    return previous_timeout;
-  }
-
-  protected class RetryException extends Exception {
-
-  }
-
-  Connection getDbConn(int isolationLevel) throws SQLException {
-    return getDbConn(isolationLevel, connPool);
-  }
-
-  protected Connection getDbConn(int isolationLevel, DataSource connPool) throws SQLException {
-    Connection dbConn = null;
-    try {
-      dbConn = connPool.getConnection();
-      dbConn.setAutoCommit(false);
-      dbConn.setTransactionIsolation(isolationLevel);
-      return dbConn;
-    } catch (SQLException e) {
-      closeDbConn(dbConn);
-      throw e;
-    }
-  }
-
-  static void rollbackDBConn(Connection dbConn) {
-    try {
-      if (dbConn != null && !dbConn.isClosed()) dbConn.rollback();
-    } catch (SQLException e) {
-      LOG.warn("Failed to rollback db connection " + getMessage(e));
-    }
-  }
-  protected static void closeDbConn(Connection dbConn) {
-    try {
-      if (dbConn != null && !dbConn.isClosed()) {
-        dbConn.close();
-      }
-    } catch (SQLException e) {
-      LOG.warn("Failed to close db connection " + getMessage(e));
-    }
-  }
-
-  /**
-   * Close statement instance.
-   * @param stmt statement instance.
-   */
-  protected static void closeStmt(Statement stmt) {
-    try {
-      if (stmt != null && !stmt.isClosed()) stmt.close();
-    } catch (SQLException e) {
-      LOG.warn("Failed to close statement " + getMessage(e));
-    }
-  }
-
-  /**
-   * Close the ResultSet.
-   * @param rs may be {@code null}
-   */
-  static void close(ResultSet rs) {
-    try {
-      if (rs != null && !rs.isClosed()) {
-        rs.close();
-      }
-    }
-    catch(SQLException ex) {
-      LOG.warn("Failed to close statement " + getMessage(ex));
-    }
-  }
-
-  /**
-   * Close all 3 JDBC artifacts in order: {@code rs stmt dbConn}
-   */
-  static void close(ResultSet rs, Statement stmt, Connection dbConn) {
-    close(rs);
-    closeStmt(stmt);
-    closeDbConn(dbConn);
-  }
-
-  private boolean waitForRetry(String caller, String errMsg) {
-    if (retryNum++ < retryLimit) {
-      LOG.warn("Retryable error detected in {}. Will wait {} ms and retry up to {} times. Error: {}", caller,
-          retryInterval, (retryLimit - retryNum + 1), errMsg);
-      try {
-        Thread.sleep(retryInterval);
-      } catch (InterruptedException ex) {
-        //
-      }
-      return true;
-    } else {
-      LOG.error("Fatal error in {}. Retry limit ({}) reached. Last error: {}", caller, retryLimit, errMsg);
-    }
-    return false;
-  }
-
-  /**
-   * See {@link #checkRetryable(SQLException, String, boolean)}.
-   */
-  void checkRetryable(SQLException e, String caller) throws RetryException {
-    checkRetryable(e, caller, false);
-  }
-
-  /**
-   * Determine if an exception was such that it makes sense to retry.  Unfortunately there is no standard way to do
-   * this, so we have to inspect the error messages and catch the telltale signs for each
-   * different database.  This method will throw {@code RetryException}
-   * if the error is retry-able.
-   * @param e exception that was thrown.
-   * @param caller name of the method calling this (and other info useful to log)
-   * @param retryOnDuplicateKey whether to retry on unique key constraint violation
-   * @throws org.apache.hadoop.hive.metastore.txn.TxnHandler.RetryException when the operation should be retried
-   */
-  void checkRetryable(SQLException e, String caller, boolean retryOnDuplicateKey)
-      throws RetryException {
-
-    // If you change this function, remove the @Ignore from TestTxnHandler.deadlockIsDetected()
-    // to test these changes.
-    // MySQL and MSSQL use 40001 as the state code for rollback.  Postgres uses 40001 and 40P01.
-    // Oracle seems to return different SQLStates and messages each time,
-    // so I've tried to capture the different error messages (there appear to be fewer different
-    // error messages than SQL states).
-    // Derby and newer MySQL driver use the new SQLTransactionRollbackException
-    boolean sendRetrySignal = false;
-    try {
-      if(dbProduct == null) {
-        throw new IllegalStateException("DB Type not determined yet.");
-      }
-      if (dbProduct.isDeadlock(e)) {
-        if (deadlockCnt++ < ALLOWED_REPEATED_DEADLOCKS) {
-          long waitInterval = deadlockRetryInterval * deadlockCnt;
-          LOG.warn("Deadlock detected in {}. Will wait {} ms try again up to {} times.", caller, waitInterval,
-              (ALLOWED_REPEATED_DEADLOCKS - deadlockCnt + 1));
-          // Pause for a just a bit for retrying to avoid immediately jumping back into the deadlock.
-          try {
-            Thread.sleep(waitInterval);
-          } catch (InterruptedException ie) {
-            // NOP
-          }
-          sendRetrySignal = true;
-        } else {
-          LOG.error("Too many repeated deadlocks in {}, giving up.", caller);
-        }
-      } else if (isRetryable(conf, e)) {
-        //in MSSQL this means Communication Link Failure
-        sendRetrySignal = waitForRetry(caller, e.getMessage());
-      } else if (retryOnDuplicateKey && isDuplicateKeyError(e)) {
-        sendRetrySignal = waitForRetry(caller, e.getMessage());
-      }
-      else {
-        //make sure we know we saw an error that we don't recognize
-        LOG.info("Non-retryable error in {} : {}", caller, getMessage(e));
-      }
-    }
-    finally {
-      /*if this method ends with anything except a retry signal, the caller should fail the operation
-      and propagate the error up to the its caller (Metastore client); thus must reset retry counters*/
-      if(!sendRetrySignal) {
-        deadlockCnt = 0;
-        retryNum = 0;
-      }
-    }
-    if(sendRetrySignal) {
-      throw new RetryException();
-    }
-  }
-
-  /**
-   * Determine the current time, using the RDBMS as a source of truth
-   * @param conn database connection
-   * @return current time in milliseconds
-   * @throws org.apache.hadoop.hive.metastore.api.MetaException if the time cannot be determined
-   */
-  protected long getDbTime(Connection conn) throws MetaException {
-    Statement stmt = null;
-    try {
-      stmt = conn.createStatement();
-      String s = dbProduct.getDBTime();
-
-      LOG.debug("Going to execute query <{}>", s);
-      ResultSet rs = stmt.executeQuery(s);
-      if (!rs.next()) throw new MetaException("No results from date query");
-      return rs.getTimestamp(1).getTime();
-    } catch (SQLException e) {
-      String msg = "Unable to determine current time: " + e.getMessage();
-      LOG.error(msg);
-      throw new MetaException(msg);
-    } finally {
-      closeStmt(stmt);
-    }
-  }
-
-  /**
-   * Determine the current time, using the RDBMS as a source of truth
-   * @return current time in milliseconds
-   * @throws org.apache.hadoop.hive.metastore.api.MetaException if the time cannot be determined
-   */
-  protected Timestamp getDbTime() throws MetaException {
-    return jdbcResource.getJdbcTemplate().queryForObject(
-        dbProduct.getDBTime(), 
-        new MapSqlParameterSource(),
-        (ResultSet rs, int rowNum) -> rs.getTimestamp(1));
-  }
-  
-
-  protected String isWithinCheckInterval(String expr, long interval) throws MetaException {
-    return dbProduct.isWithinCheckInterval(expr, interval);
-  }
-
-  private void determineDatabaseProduct(Connection conn) {
-    try {
-      String s = conn.getMetaData().getDatabaseProductName();
-      dbProduct = DatabaseProduct.determineDatabaseProduct(s, conf);
-      if (dbProduct.isUNDEFINED()) {
-        String msg = "Unrecognized database product name <" + s + ">";
-        LOG.error(msg);
-        throw new IllegalStateException(msg);
-      }
-    } catch (SQLException e) {
-      String msg = "Unable to get database product name";
-      LOG.error(msg, e);
-      throw new IllegalStateException(msg, e);
-    }
-  }
-
-  private static class LockInfo {
-    private final long extLockId;
-    private final long intLockId;
-    //0 means there is no transaction, i.e. it a select statement which is not part of
-    //explicit transaction or a IUD statement that is not writing to ACID table
-    private final long txnId;
-    private final String db;
-    private final String table;
-    private final String partition;
-    private final LockState state;
-    private final LockType type;
-
-    // Assumes the result set is set to a valid row
-    LockInfo(ResultSet rs) throws SQLException, MetaException {
-      extLockId = rs.getLong("HL_LOCK_EXT_ID"); // can't be null
-      intLockId = rs.getLong("HL_LOCK_INT_ID"); // can't be null
-      db = rs.getString("HL_DB"); // can't be null
-      String t = rs.getString("HL_TABLE");
-      table = (rs.wasNull() ? null : t);
-      String p = rs.getString("HL_PARTITION");
-      partition = (rs.wasNull() ? null : p);
-      switch (rs.getString("HL_LOCK_STATE").charAt(0)) {
-        case LOCK_WAITING: state = LockState.WAITING; break;
-        case LOCK_ACQUIRED: state = LockState.ACQUIRED; break;
-        default:
-          throw new MetaException("Unknown lock state " + rs.getString("HL_LOCK_STATE").charAt(0));
-      }
-      char lockChar = rs.getString("HL_LOCK_TYPE").charAt(0);
-      type = LockTypeUtil.getLockTypeFromEncoding(lockChar)
-              .orElseThrow(() -> new MetaException("Unknown lock type: " + lockChar));
-      txnId = rs.getLong("HL_TXNID"); //returns 0 if value is NULL
-    }
-
-    LockInfo(ShowLocksResponseElement e) {
-      extLockId = e.getLockid();
-      intLockId = e.getLockIdInternal();
-      txnId = e.getTxnid();
-      db = e.getDbname();
-      table = e.getTablename();
-      partition = e.getPartname();
-      state = e.getState();
-      type = e.getType();
-    }
-
-    public boolean equals(Object other) {
-      if (!(other instanceof LockInfo)) return false;
-      LockInfo o = (LockInfo)other;
-      // Lock ids are unique across the system.
-      return extLockId == o.extLockId && intLockId == o.intLockId;
-    }
+  @Override
+  public void replTableWriteIdState(ReplTblWriteIdStateRequest rqst) throws MetaException {
+    new ReplTableWriteIdStateFunction(rqst, mutexAPI, transactionalListeners).execute(jdbcResource);
+  }
 
-    @Override
-    public String toString() {
-      return JavaUtils.lockIdToString(extLockId) + " intLockId:" +
-        intLockId + " " + JavaUtils.txnIdToString(txnId)
-        + " db:" + db + " table:" + table + " partition:" +
-        partition + " state:" + (state == null ? "null" : state.toString())
-        + " type:" + (type == null ? "null" : type.toString());
-    }
-    private boolean isDbLock() {
-      return db != null && table == null && partition == null;
-    }
-    private boolean isTableLock() {
-      return db != null && table != null && partition == null;
-    }
-    private boolean isPartitionLock() {
-      return !(isDbLock() || isTableLock());
-    }
+  @Override
+  public GetValidWriteIdsResponse getValidWriteIds(GetValidWriteIdsRequest rqst) throws MetaException {
+    return new GetValidWriteIdsFunction(rqst, openTxnTimeOutMillis).execute(jdbcResource);
+  }
+  
+  @Override
+  public AllocateTableWriteIdsResponse allocateTableWriteIds(AllocateTableWriteIdsRequest rqst) throws MetaException {
+    return new AllocateTableWriteIdsFunction(rqst, transactionalListeners).execute(jdbcResource);
   }
 
-  private static class LockInfoComparator implements Comparator<LockInfo>, Serializable {
-    private LockTypeComparator lockTypeComparator = new LockTypeComparator();
+  @Override
+  public MaxAllocatedTableWriteIdResponse getMaxAllocatedTableWrited(MaxAllocatedTableWriteIdRequest rqst) throws MetaException {
+    return jdbcResource.execute(new GetMaxAllocatedTableWriteIdHandler(rqst));
+  }
 
-    public boolean equals(Object other) {
-      return this == other;
-    }
+  @Override
+  public void seedWriteId(SeedTableWriteIdsRequest rqst) throws MetaException {
+    //since this is on conversion from non-acid to acid, NEXT_WRITE_ID should not have an entry
+    //for this table.  It also has a unique index in case 'should not' is violated
 
-    public int compare(LockInfo info1, LockInfo info2) {
-      // We sort by state (acquired vs waiting) and then by LockType, then by id
-      if (info1.state == LockState.ACQUIRED &&
-        info2.state != LockState .ACQUIRED) {
-        return -1;
-      }
-      if (info1.state != LockState.ACQUIRED &&
-        info2.state == LockState .ACQUIRED) {
-        return 1;
-      }
+    // First allocation of write id should add the table to the next_write_id meta table
+    // The initial value for write id should be 1 and hence we add 1 with number of write ids
+    // allocated here
+    jdbcResource.getJdbcTemplate().update(
+        "INSERT INTO \"NEXT_WRITE_ID\" (\"NWI_DATABASE\", \"NWI_TABLE\", \"NWI_NEXT\") VALUES (:db, :table, :writeId)",
+        new MapSqlParameterSource()
+            .addValue("db", rqst.getDbName())
+            .addValue("table", rqst.getTableName())
+            .addValue("writeId", rqst.getSeedWriteId() + 1));
+  }
 
-      int sortByType = lockTypeComparator.compare(info1.type, info2.type);
-      if(sortByType != 0) {
-        return sortByType;
-      }
-      if (info1.extLockId < info2.extLockId) {
-        return -1;
-      } else if (info1.extLockId > info2.extLockId) {
-        return 1;
-      } else {
-        if (info1.intLockId < info2.intLockId) {
-          return -1;
-        } else if (info1.intLockId > info2.intLockId) {
-          return 1;
-        } else {
-          return 0;
-        }
-      }
+  @Override
+  public void seedTxnId(SeedTxnIdRequest rqst) throws MetaException {
+    /*
+     * Locking the txnLock an exclusive way, we do not want to set the txnId backward accidentally
+     * if there are concurrent open transactions
+     */
+    acquireTxnLock(false);
+    long highWaterMark = jdbcResource.execute(new GetHighWaterMarkHandler());
+    if (highWaterMark >= rqst.getSeedTxnId()) {
+      throw new MetaException(MessageFormat
+          .format("Invalid txnId seed {}, the highWaterMark is {}", rqst.getSeedTxnId(), highWaterMark));
     }
+    jdbcResource.getJdbcTemplate().getJdbcTemplate()
+        .execute((Statement stmt) -> stmt.execute(dbProduct.getTxnSeedFn(rqst.getSeedTxnId())));
   }
 
-  private enum LockAction {ACQUIRE, WAIT, KEEP_LOOKING}
+  @Override
+  public void addWriteNotificationLog(ListenerEvent acidWriteEvent) throws MetaException {
+      MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners,
+          acidWriteEvent instanceof AcidWriteEvent ? EventMessage.EventType.ACID_WRITE
+              : EventMessage.EventType.BATCH_ACID_WRITE,
+          acidWriteEvent, jdbcResource.getConnection(), sqlGenerator);
+  }
 
-  // A jump table to figure out whether to wait, acquire,
-  // or keep looking .  Since
-  // java doesn't have function pointers (grumble grumble) we store a
-  // character that we'll use to determine which function to call.
-  // The table maps the lock type of the lock we are looking to acquire to
-  // the lock type of the lock we are checking to the lock state of the lock
-  // we are checking to the desired action.
-  private static Map<LockType, Map<LockType, Map<LockState, LockAction>>> jumpTable;
+  @Override
+  public void performWriteSetGC() throws MetaException {
+    long commitHighWaterMark = new MinOpenTxnIdWaterMarkFunction(openTxnTimeOutMillis).execute(jdbcResource);
+    jdbcResource.getJdbcTemplate().update(
+        "DELETE FROM \"WRITE_SET\" WHERE \"WS_COMMIT_ID\" < :hwm",
+        new MapSqlParameterSource()
+            .addValue("hwm", commitHighWaterMark));
+  }
 
-  private int abortTxns(Connection dbConn, List<Long> txnids,
-                        boolean skipCount, boolean isReplReplayed, TxnErrorMsg txnErrorMsg) throws SQLException, MetaException {
-    return abortTxns(dbConn, txnids, false, skipCount, isReplReplayed, txnErrorMsg);
+  @Override
+  public void updateTransactionStatistics(UpdateTransactionalStatsRequest req) throws MetaException {
+    jdbcResource.execute(
+        "UPDATE \"MV_TABLES_USED\" " +
+        "SET \"INSERTED_COUNT\"=\"INSERTED_COUNT\"+ :insertCount" +
+        ",\"UPDATED_COUNT\"=\"UPDATED_COUNT\"+ :updateCount" +
+        ",\"DELETED_COUNT\"=\"DELETED_COUNT\"+ :deleteCount" +
+        " WHERE \"TBL_ID\"= :tableId",
+        new MapSqlParameterSource()
+            .addValue("insertCount", req.getInsertCount())
+            .addValue("updateCount", req.getUpdatedCount())
+            .addValue("deleteCount", req.getDeletedCount())
+            .addValue("tableId", req.getTableId()), null);
   }
+
   /**
-   * TODO: expose this as an operation to client.  Useful for streaming API to abort all remaining
-   * transactions in a batch on IOExceptions.
-   * Caller must rollback the transaction if not all transactions were aborted since this will not
-   * attempt to delete associated locks in this case.
-   *
-   * @param dbConn An active connection
-   * @param txnids list of transactions to abort
-   * @param checkHeartbeat value used by {@link #performTimeOuts()} to ensure this doesn't Abort txn which were
-   *                      heartbeated after #performTimeOuts() select and this operation.
-   * @param skipCount If true, the method always returns 0, otherwise returns the number of actually aborted txns
-   * @return 0 if skipCount is true, the number of aborted transactions otherwise
-   * @throws SQLException
+   * Get invalidation info for the materialization. Materialization information
+   * contains information about whether there was update/delete/compaction operations on the source
+   * tables used by the materialization since it was created.
    */
-  private int abortTxns(Connection dbConn, List<Long> txnids, boolean checkHeartbeat,
-                        boolean skipCount, boolean isReplReplayed, TxnErrorMsg txnErrorMsg)
-      throws SQLException, MetaException {
-    Statement stmt = null;
-    if (txnids.isEmpty()) {
-      return 0;
-    }
-    Collections.sort(txnids);
-    LOG.debug("Aborting {} transaction(s) {} due to {}", txnids.size(), txnids, txnErrorMsg);
-    removeTxnsFromMinHistoryLevel(dbConn, txnids);
-    removeWriteIdsFromMinHistory(dbConn, txnids);
-    try {
-      stmt = dbConn.createStatement();
-      //This is an update statement, thus at any Isolation level will take Write locks so will block
-      //all other ops using S4U on TXNS row.
-      List<String> queries = new ArrayList<>();
-      StringBuilder prefix = new StringBuilder();
-      StringBuilder suffix = new StringBuilder();
-
-      // add update txns queries to query list
-      prefix.append("UPDATE \"TXNS\" SET \"TXN_STATE\" = ").append(TxnStatus.ABORTED)
-              .append(" , \"TXN_META_INFO\" = ").append(txnErrorMsg.toSqlString())
-              .append(" WHERE \"TXN_STATE\" = ").append(TxnStatus.OPEN).append(" AND ");
-      if (checkHeartbeat) {
-        suffix.append(" AND \"TXN_LAST_HEARTBEAT\" < ")
-                .append(getEpochFn(dbProduct)).append("-").append(timeout);
-      }
-      TxnUtils.buildQueryWithINClause(conf, queries, prefix, suffix, txnids, "\"TXN_ID\"", true, false);
-      int numUpdateQueries = queries.size();
-
-      // add delete hive locks queries to query list
-      prefix.setLength(0);
-      suffix.setLength(0);
-      prefix.append("DELETE FROM \"HIVE_LOCKS\" WHERE ");
-      TxnUtils.buildQueryWithINClause(conf, queries, prefix, suffix, txnids, "\"HL_TXNID\"", false, false);
-
-      //If this abort is for REPL_CREATED TXN initiated outside the replication flow, then clean the corresponding entry
-      //from REPL_TXN_MAP and mark that database as replication incompatible.
-      if (!isReplReplayed) {
-        for (String database : getDbNamesForReplayedTxns(dbConn, txnids)) {
-          markDbAsReplIncompatible(dbConn, database);
-        }
-        // Delete mapping from REPL_TXN_MAP if it exists.
-        prefix.setLength(0);
-        suffix.setLength(0);
-        prefix.append("DELETE FROM \"REPL_TXN_MAP\" WHERE ");
-        TxnUtils.buildQueryWithINClause(conf, queries, prefix, suffix, txnids, "\"RTM_TARGET_TXN_ID\"", false, false);
-      }
-
-      // execute all queries in the list in one batch
-      int numAborted = 0;
-      if (skipCount) {
-        executeQueriesInBatchNoCount(dbProduct, stmt, queries, maxBatchSize);
-      } else {
-        List<Integer> affectedRowsByQuery = executeQueriesInBatch(stmt, queries, maxBatchSize);
-        numAborted = getUpdateCount(numUpdateQueries, affectedRowsByQuery);
-      }
+  @Override
+  public Materialization getMaterializationInvalidationInfo(
+          CreationMetadata creationMetadata, String validTxnListStr) throws MetaException {
+    return new GetMaterializationInvalidationInfoFunction(creationMetadata, validTxnListStr).execute(jdbcResource);
+  }
 
-      if (MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON)) {
-        Metrics.getOrCreateCounter(MetricsConstants.TOTAL_NUM_ABORTED_TXNS).inc(txnids.size());
-      }
-      LOG.warn("Aborted {} transaction(s) {} due to {}", txnids.size(), txnids, txnErrorMsg);
-      return numAborted;
-    } finally {
-      closeStmt(stmt);
-    }
+  @Override
+  public LockResponse lockMaterializationRebuild(String dbName, String tableName, long txnId) throws MetaException {
+    return new LockMaterializationRebuildFunction(dbName, tableName, txnId, mutexAPI).execute(jdbcResource);
   }
 
-  private int getUpdateCount(int numUpdateQueries, List<Integer> affectedRowsByQuery) {
-    return affectedRowsByQuery.stream()
-            .limit(numUpdateQueries)
-            .mapToInt(Integer::intValue)
-            .sum();
+  @Override
+  public boolean heartbeatLockMaterializationRebuild(String dbName, String tableName, long txnId) throws MetaException {
+    int result = jdbcResource.execute(
+        "UPDATE \"MATERIALIZATION_REBUILD_LOCKS\"" +
+            " SET \"MRL_LAST_HEARTBEAT\" = " + Instant.now().toEpochMilli() +
+            " WHERE \"MRL_TXN_ID\" = " + txnId +
+            " AND \"MRL_DB_NAME\" = ?" +
+            " AND \"MRL_TBL_NAME\" = ?",
+        new MapSqlParameterSource()
+            .addValue("now", Instant.now().toEpochMilli())
+            .addValue("txnId", txnId)
+            .addValue("dbName", dbName)
+            .addValue("tableNane", tableName),
+        ParameterizedCommand.AT_LEAST_ONE_ROW);
+    return result >= 1;
   }
 
-  private static boolean isValidTxn(long txnId) {
-    return txnId != 0;
+  @Override
+  public long cleanupMaterializationRebuildLocks(ValidTxnList validTxnList, long timeout) throws MetaException {
+    return new ReleaseMaterializationRebuildLocks(validTxnList, timeout).execute(jdbcResource);
   }
+
   /**
-   * Lock acquisition is meant to be fair, so every lock can only block on some lock with smaller
-   * hl_lock_ext_id by only checking earlier locks.
-   *
-   * For any given SQL statement all locks required by it are grouped under single extLockId and are
-   * granted all at once or all locks wait.
-   *
-   * This is expected to run at READ_COMMITTED.
-   *
-   * If there is a concurrent commitTxn/rollbackTxn, those can only remove rows from HIVE_LOCKS.
-   * If they happen to be for the same txnid, there will be a WW conflict (in MS DB), if different txnid,
-   * checkLock() will in the worst case keep locks in Waiting state a little longer.
+   * As much as possible (i.e. in absence of retries) we want both operations to be done on the same
+   * connection (but separate transactions).
+   * Retry-by-caller note: If the call to lock is from a transaction, then in the worst case
+   * there will be a duplicate set of locks but both sets will belong to the same txn so they
+   * will not conflict with each other.  For locks w/o txn context (i.e. read-only query), this
+   * may lead to deadlock (at least a long wait).  (e.g. 1st call creates locks in {@code LOCK_WAITING}
+   * mode and response gets lost.  Then {@link org.apache.hadoop.hive.metastore.RetryingMetaStoreClient}
+   * retries, and enqueues another set of locks in LOCK_WAITING.  The 2nd LockResponse is delivered
+   * to the DbLockManager, which will keep dong {@link #checkLock(CheckLockRequest)} until the 1st
+   * set of locks times out.
    */
-  @RetrySemantics.SafeToRetry("See @SafeToRetry")
-  private LockResponse checkLock(Connection dbConn, long extLockId, long txnId, boolean zeroWaitReadEnabled, 
-          boolean isExclusiveCTAS)
-      throws NoSuchLockException, TxnAbortedException, MetaException, SQLException {
-    Statement stmt = null;
-    ResultSet rs = null;
-    LockResponse response = new LockResponse();
-    /**
-     * todo: Longer term we should pass this from client somehow - this would be an optimization;  once
-     * that is in place make sure to build and test "writeSet" below using OperationType not LockType
-     * With Static Partitions we assume that the query modifies exactly the partitions it locked.  (not entirely
-     * realistic since Update/Delete may have some predicate that filters out all records out of
-     * some partition(s), but plausible).  For DP, we acquire locks very wide (all known partitions),
-     * but for most queries only a fraction will actually be updated.  #addDynamicPartitions() tells
-     * us exactly which ones were written to.  Thus using this trick to kill a query early for
-     * DP queries may be too restrictive.
-     */
-    boolean isPartOfDynamicPartitionInsert = true;
+  @Override
+  public LockResponse lock(LockRequest rqst) throws NoSuchTxnException, TxnAbortedException, MetaException {
+    long lockId = txnLockManager.enqueueLock(rqst);
     try {
-      List<LockInfo> locksBeingChecked = getLocksFromLockId(dbConn, extLockId); //being acquired now
-      response.setLockid(extLockId);
-
-      //This is the set of entities that the statement represented by extLockId wants to update
-      List<LockInfo> writeSet = new ArrayList<>();
-
-      for (LockInfo info : locksBeingChecked) {
-        if(!isPartOfDynamicPartitionInsert && info.type == LockType.SHARED_WRITE) {
-          writeSet.add(info);
-        }
-      }
-      if(!writeSet.isEmpty()) {
-        if(writeSet.get(0).txnId == 0) {
-          //Write operation always start a txn
-          throw new IllegalStateException("Found Write lock for " + JavaUtils.lockIdToString(extLockId) + " but no txnid");
-        }
-        stmt = dbConn.createStatement();
-        StringBuilder sb = new StringBuilder(" \"WS_DATABASE\", \"WS_TABLE\", \"WS_PARTITION\", " +
-          "\"WS_TXNID\", \"WS_COMMIT_ID\" " +
-          "FROM \"WRITE_SET\" WHERE WS_COMMIT_ID >= " + writeSet.get(0).txnId + " AND (");//see commitTxn() for more info on this inequality
-        for(LockInfo info : writeSet) {
-          sb.append("(\"WS_DATABASE\" = ").append(quoteString(info.db)).append(" AND \"WS_TABLE\" = ")
-            .append(quoteString(info.table)).append(" AND \"WS_PARTITION\" ")
-            .append(info.partition == null ? "IS NULL" : "= " + quoteString(info.partition)).append(") OR ");
-        }
-        sb.setLength(sb.length() - 4);//nuke trailing " or "
-        sb.append(")");
-        //1 row is sufficient to know we have to kill the query
-        rs = stmt.executeQuery(sqlGenerator.addLimitClause(1, sb.toString()));
-        if(rs.next()) {
-          /**
-           * if here, it means we found an already committed txn which overlaps with the current one and
-           * it updated the same resource the current txn wants to update.  By First-committer-wins
-           * rule, current txn will not be allowed to commit so  may as well kill it now;  This is just an
-           * optimization to prevent wasting cluster resources to run a query which is known to be DOA.
-           * {@link #commitTxn(CommitTxnRequest)} has the primary responsibility to ensure this.
-           * checkLock() runs at READ_COMMITTED so you could have another (Hive) txn running commitTxn()
-           * in parallel and thus writing to WRITE_SET.  commitTxn() logic is properly mutexed to ensure
-           * that we don't "miss" any WW conflicts. We could've mutexed the checkLock() and commitTxn()
-           * as well but this reduces concurrency for very little gain.
-           * Note that update/delete (which runs as dynamic partition insert) acquires a lock on the table,
-           * but WRITE_SET has entries for actual partitions updated.  Thus this optimization will "miss"
-           * the WW conflict but it will be caught in commitTxn() where actual partitions written are known.
-           * This is OK since we want 2 concurrent updates that update different sets of partitions to both commit.
-           */
-          String resourceName = rs.getString(1) + '/' + rs.getString(2);
-          String partName = rs.getString(3);
-          if(partName != null) {
-            resourceName += '/' + partName;
-          }
-
-          String msg = "Aborting " + JavaUtils.txnIdToString(writeSet.get(0).txnId) +
-            " since a concurrent committed transaction [" + JavaUtils.txnIdToString(rs.getLong(4)) + "," + rs.getLong(5) +
-            "] has already updated resource '" + resourceName + "'";
-          LOG.info(msg);
-          if (abortTxns(dbConn, Collections.singletonList(writeSet.get(0).txnId), false, false,
-                  TxnErrorMsg.ABORT_CONCURRENT) != 1) {
-            throw new IllegalStateException(msg + " FAILED!");
-          }
-          dbConn.commit();
-          throw new TxnAbortedException(msg);
-        }
-        close(rs, stmt, null);
-      }
-
-      String queryStr =
-        " \"EX\".*, \"REQ\".\"HL_LOCK_INT_ID\" \"LOCK_INT_ID\", \"REQ\".\"HL_LOCK_TYPE\" \"LOCK_TYPE\" FROM (" +
-            " SELECT \"HL_LOCK_EXT_ID\", \"HL_LOCK_INT_ID\", \"HL_TXNID\", \"HL_DB\", \"HL_TABLE\", \"HL_PARTITION\"," +
-                " \"HL_LOCK_STATE\", \"HL_LOCK_TYPE\" FROM \"HIVE_LOCKS\"" +
-            " WHERE \"HL_LOCK_EXT_ID\" < " + extLockId + ") \"EX\"" +
-        " INNER JOIN (" +
-            " SELECT \"HL_LOCK_INT_ID\", \"HL_TXNID\", \"HL_DB\", \"HL_TABLE\", \"HL_PARTITION\"," +
-                " \"HL_LOCK_TYPE\" FROM \"HIVE_LOCKS\"" +
-            " WHERE \"HL_LOCK_EXT_ID\" = " + extLockId + ") \"REQ\"" +
-        " ON \"EX\".\"HL_DB\" = \"REQ\".\"HL_DB\"" +
-            " AND (\"EX\".\"HL_TABLE\" IS NULL OR \"REQ\".\"HL_TABLE\" IS NULL" +
-                " OR \"EX\".\"HL_TABLE\" = \"REQ\".\"HL_TABLE\"" +
-                " AND (\"EX\".\"HL_PARTITION\" IS NULL OR \"REQ\".\"HL_PARTITION\" IS NULL" +
-                " OR \"EX\".\"HL_PARTITION\" = \"REQ\".\"HL_PARTITION\"))" +
-        /*different locks from same txn should not conflict with each other,
-          txnId=0 means it's a select or IUD which does not write to ACID table*/
-        " WHERE (\"REQ\".\"HL_TXNID\" = 0 OR \"EX\".\"HL_TXNID\" != \"REQ\".\"HL_TXNID\")" +
-            " AND ";
-
-      /*EXCLUSIVE lock on partition should prevent SHARED_READ on the table, however there is no reason
-        for an EXCLUSIVE on a table to prevent SHARED_READ on a database. Similarly, EXCLUSIVE on a partition
-        should not conflict with SHARED_READ on a database.
-        SHARED_READ is usually acquired on a database to make sure it's not dropped, while some operation
-        is performed on that db (e.g. show tables, created table, etc).
-        EXCLUSIVE on an object may mean it's being dropped or overwritten.*/
-      String[] whereStr = {
-        // shared-read
-        " \"REQ\".\"HL_LOCK_TYPE\"=" + LockTypeUtil.sharedRead() + " AND \"EX\".\"HL_LOCK_TYPE\"=" +
-          LockTypeUtil.exclusive() + " AND NOT (\"EX\".\"HL_TABLE\" IS NOT NULL AND \"REQ\".\"HL_TABLE\" IS NULL)",
-        // exclusive
-        " \"REQ\".\"HL_LOCK_TYPE\"=" + LockTypeUtil.exclusive() +
-        " AND NOT (\"EX\".\"HL_TABLE\" IS NULL AND \"EX\".\"HL_LOCK_TYPE\"=" +
-          LockTypeUtil.sharedRead() + " AND \"REQ\".\"HL_TABLE\" IS NOT NULL)",
-        // shared-write
-        " \"REQ\".\"HL_LOCK_TYPE\"=" + LockTypeUtil.sharedWrite() + " AND \"EX\".\"HL_LOCK_TYPE\" IN (" +
-          LockTypeUtil.exclWrite() + "," + LockTypeUtil.exclusive() + ")",
-        // excl-write
-        " \"REQ\".\"HL_LOCK_TYPE\"=" + LockTypeUtil.exclWrite() + " AND \"EX\".\"HL_LOCK_TYPE\"!=" +
-          LockTypeUtil.sharedRead()
-      };
-
-      List<String> subQuery = new ArrayList<>();
-      for (String subCond : whereStr) {
-        subQuery.add("(" + sqlGenerator.addLimitClause(1, queryStr + subCond) + ")");
-      }
-      String query = String.join(" UNION ALL ", subQuery);
-
-      stmt = dbConn.createStatement();
-      LOG.debug("Going to execute query <{}>", query);
-      rs = stmt.executeQuery(query);
-
-      if (rs.next()) {
-        // We acquire all locks for a given query atomically; if 1 blocks, all remain in Waiting state.
-        LockInfo blockedBy = new LockInfo(rs);
-        long intLockId = rs.getLong("LOCK_INT_ID");
-        char lockChar = rs.getString("LOCK_TYPE").charAt(0);
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Failure to acquire lock({} intLockId:{} {}), blocked by ({})", JavaUtils.lockIdToString(extLockId),
-              intLockId, JavaUtils.txnIdToString(txnId), blockedBy);
-        }
-
-        LockType lockType = LockTypeUtil.getLockTypeFromEncoding(lockChar)
-            .orElseThrow(() -> new MetaException("Unknown lock type: " + lockChar));
-        
-        if ((zeroWaitReadEnabled && LockType.SHARED_READ == lockType || isExclusiveCTAS) && isValidTxn(txnId)) {
-          String cleanupQuery = "DELETE FROM \"HIVE_LOCKS\" WHERE \"HL_LOCK_EXT_ID\" = " + extLockId;
-          LOG.debug("Going to execute query: <{}>", cleanupQuery);
-          stmt.executeUpdate(cleanupQuery);
-          dbConn.commit();
-
-          response.setErrorMessage(String.format(
-              isExclusiveCTAS ? EXCL_CTAS_ERR_MSG : ZERO_WAIT_READ_ERR_MSG, blockedBy));
-          response.setState(LockState.NOT_ACQUIRED);
-          return response;
-        }
-        String updateBlockedByQuery = "UPDATE \"HIVE_LOCKS\"" +
-            " SET \"HL_BLOCKEDBY_EXT_ID\" = " + blockedBy.extLockId +
-            ", \"HL_BLOCKEDBY_INT_ID\" = " + blockedBy.intLockId +
-            " WHERE \"HL_LOCK_EXT_ID\" = " + extLockId + " AND \"HL_LOCK_INT_ID\" = " + intLockId;
-
-        LOG.debug("Going to execute query: <{}>", updateBlockedByQuery);
-        int updCnt = stmt.executeUpdate(updateBlockedByQuery);
-
-        if (updCnt != 1) {
-          LOG.error("Failure to update lock (extLockId={}, intLockId={}) with the blocking lock's IDs " +
-              "(extLockId={}, intLockId={})", extLockId, intLockId, blockedBy.extLockId, blockedBy.intLockId);
-          shouldNeverHappen(txnId, extLockId, intLockId);
-        }
-        dbConn.commit();
-
-        response.setState(LockState.WAITING);
-        return response;
-      }
-      // If here, there were no locks that would block any item from 'locksBeingChecked' - acquire them all
-      acquire(dbConn, stmt, locksBeingChecked);
-
-      // We acquired all the locks, so commit and return acquired.
-      LOG.debug("Successfully acquired locks: {}", locksBeingChecked);
-      dbConn.commit();
-      response.setState(LockState.ACQUIRED);
-    } finally {
-      close(rs, stmt, null);
+      return txnLockManager.checkLock(lockId, rqst.getTxnid(), rqst.isZeroWaitReadEnabled(), rqst.isExclusiveCTAS());
+    } catch (NoSuchLockException e) {
+      // This should never happen, as we just added the lock id
+      throw new MetaException("Couldn't find a lock we just created! " + e.getMessage());      
     }
-    return response;
   }
-
-  private void acquire(Connection dbConn, Statement stmt, List<LockInfo> locksBeingChecked)
-    throws SQLException, NoSuchLockException, MetaException {
-    if (locksBeingChecked == null || locksBeingChecked.isEmpty()) {
-      return;
-    }
-    long txnId = locksBeingChecked.get(0).txnId;
-    long extLockId = locksBeingChecked.get(0).extLockId;
-    String s = "UPDATE \"HIVE_LOCKS\" SET \"HL_LOCK_STATE\" = '" + LOCK_ACQUIRED + "', " +
-      //if lock is part of txn, heartbeat info is in txn record
-      "\"HL_LAST_HEARTBEAT\" = " + (isValidTxn(txnId) ? 0 : getEpochFn(dbProduct)) +
-      ",\"HL_ACQUIRED_AT\" = " + getEpochFn(dbProduct) +
-      ",\"HL_BLOCKEDBY_EXT_ID\"=NULL,\"HL_BLOCKEDBY_INT_ID\"=NULL" +
-      " WHERE \"HL_LOCK_EXT_ID\" = " +  extLockId;
-    LOG.debug("Going to execute update <{}>", s);
-    int rc = stmt.executeUpdate(s);
-    if (rc < locksBeingChecked.size()) {
-      LOG.error("Failure to acquire all locks (acquired: {}, total needed: {}).", rc, locksBeingChecked.size());
-      dbConn.rollback();
-      /*select all locks for this ext ID and see which ones are missing*/
-      String errorMsgTemplate = "No such lock(s): (%s: %s) %s";
-      Set<String> notFoundIds = locksBeingChecked.stream()
-              .map(lockInfo -> Long.toString(lockInfo.intLockId))
-              .collect(Collectors.toSet());
-      String getIntIdsQuery = "SELECT \"HL_LOCK_INT_ID\" FROM \"HIVE_LOCKS\" WHERE \"HL_LOCK_EXT_ID\" = " + extLockId;
-      LOG.debug("Going to execute query: <{}>", getIntIdsQuery);
-      try (ResultSet rs = stmt.executeQuery(getIntIdsQuery)) {
-        while (rs.next()) {
-          notFoundIds.remove(rs.getString(1));
-        }
-      }
-      String errorMsg = String.format(errorMsgTemplate,
-              JavaUtils.lockIdToString(extLockId), String.join(", ", notFoundIds), JavaUtils.txnIdToString(txnId));
-      throw new NoSuchLockException(errorMsg);
+  
+  /**
+   * Why doesn't this get a txnid as parameter?  The caller should either know the txnid or know there isn't one.
+   * Either way getTxnIdFromLockId() will not be needed.  This would be a Thrift change.
+   * <p>
+   * Also, when lock acquisition returns WAITING, it's retried every 15 seconds (best case, see DbLockManager.backoff(),
+   * in practice more often)
+   * which means this is heartbeating way more often than hive.txn.timeout and creating extra load on DB.
+   * <p>
+   * The clients that operate in blocking mode, can't heartbeat a lock until the lock is acquired.
+   * We should make CheckLockRequest include timestamp or last request to skip unnecessary heartbeats. Thrift change.
+   * <p>
+   * {@link #checkLock(CheckLockRequest)}  must run at SERIALIZABLE
+   * (make sure some lock we are checking against doesn't move from W to A in another txn)
+   * but this method can heartbeat in separate txn at READ_COMMITTED.
+   * <p>
+   * Retry-by-caller note:
+   * Retryable because {@link #checkLock(CheckLockRequest)} is
+   */
+  @Override
+  public LockResponse checkLock(CheckLockRequest rqst)
+      throws NoSuchTxnException, NoSuchLockException, TxnAbortedException, MetaException {
+    long extLockId = rqst.getLockid();
+    // Heartbeat on the lockid first, to assure that our lock is still valid.
+    // Then look up the lock info (hopefully in the cache).  If these locks
+    // are associated with a transaction then heartbeat on that as well.
+    List<LockInfo> lockInfos = jdbcResource.execute(new GetLocksByLockId(extLockId, 1, sqlGenerator));
+    if (CollectionUtils.isEmpty(lockInfos)) {
+      throw new NoSuchLockException("No such lock " + JavaUtils.lockIdToString(extLockId));
+    }
+    LockInfo lockInfo = lockInfos.get(0);
+    if (lockInfo.getTxnId() > 0) {
+      new HeartbeatTxnFunction(lockInfo.getTxnId()).execute(jdbcResource);
+    } else {
+      new HeartbeatLockFunction(rqst.getLockid()).execute(jdbcResource);
     }
+    return txnLockManager.checkLock(extLockId, lockInfo.getTxnId(), false, false);
   }
 
   /**
-   * 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.
+   * This would have been made simpler if all locks were associated with a txn.  Then only txn needs to
+   * be heartbeated, committed, etc.  no need for client to track individual locks.
+   * When removing locks not associated with txn this potentially conflicts with
+   * heartbeat/performTimeout which are update/delete of HIVE_LOCKS thus will be locked as needed by db.
+   * since this only removes from HIVE_LOCKS at worst some lock acquire is delayed
    */
-  private void heartbeatLock(Connection dbConn, long extLockId)
-    throws NoSuchLockException, SQLException, MetaException {
-    // If the lock id is 0, then there are no locks in this heartbeat
-    if (extLockId == 0) {
-      return;
-    }
-    try (Statement stmt = dbConn.createStatement()) {
-      String updateHeartbeatQuery = "UPDATE \"HIVE_LOCKS\" SET \"HL_LAST_HEARTBEAT\" = " +
-          getEpochFn(dbProduct) + " WHERE \"HL_LOCK_EXT_ID\" = " + extLockId;
-      LOG.debug("Going to execute update <{}>", updateHeartbeatQuery);
-      int rc = stmt.executeUpdate(updateHeartbeatQuery);
-      if (rc < 1) {
-        LOG.error("Failure to update last heartbeat for extLockId={}.", extLockId);
-        dbConn.rollback();
-        throw new NoSuchLockException("No such lock: " + JavaUtils.lockIdToString(extLockId));
-      }
-      LOG.debug("Successfully heartbeated for extLockId={}", extLockId);
-      dbConn.commit();
-    }
+  @Override
+  public void unlock(UnlockRequest rqst) throws TxnOpenException, MetaException {
+    txnLockManager.unlock(rqst);
   }
 
-  // Heartbeats on the txn table.  This commits, so do not enter it with any state
-  private void heartbeatTxn(Connection dbConn, long txnid)
-    throws NoSuchTxnException, TxnAbortedException, SQLException, MetaException {
-    // If the txnid is 0, then there are no transactions in this heartbeat
-    if (txnid == 0) {
-      return;
-    }
-    try (Statement stmt = dbConn.createStatement()) {
-      String s = "UPDATE \"TXNS\" SET \"TXN_LAST_HEARTBEAT\" = " + getEpochFn(dbProduct) +
-          " WHERE \"TXN_ID\" = " + txnid + " AND \"TXN_STATE\" = " + TxnStatus.OPEN;
-      LOG.debug("Going to execute update <{}>", s);
-      int rc = stmt.executeUpdate(s);
-      if (rc < 1) {
-        ensureValidTxn(dbConn, txnid, stmt); // This should now throw some useful exception.
-        LOG.error("Can neither heartbeat txn (txnId={}) nor confirm it as invalid.", txnid);
-        dbConn.rollback();
-        throw new NoSuchTxnException("No such txn: " + txnid);
-      }
-      LOG.debug("Successfully heartbeated for txnId={}", txnid);
-      dbConn.commit();
-    }
+  @Override
+  public ShowLocksResponse showLocks(ShowLocksRequest rqst) throws MetaException {
+    return txnLockManager.showLocks(rqst);
   }
 
   /**
-   * Returns the state of the transaction if it's able to determine it. Some cases where it cannot:
-   * 1. txnid was Aborted/Committed and then GC'd (compacted)
-   * 2. txnid was committed but it didn't modify anything (nothing in COMPLETED_TXN_COMPONENTS)
+   * {@code ids} should only have txnid or lockid but not both, ideally.
+   * Currently DBTxnManager.heartbeat() enforces this.
    */
-  private TxnStatus findTxnState(long txnid, Statement stmt) throws SQLException, MetaException {
-    String s = "SELECT \"TXN_STATE\" FROM \"TXNS\" WHERE \"TXN_ID\" = " + txnid;
-    LOG.debug("Going to execute query <{}>", s);
-    try (ResultSet rs = stmt.executeQuery(s)) {
-      if (!rs.next()) {
-        s =
-            sqlGenerator.addLimitClause(1, "1 FROM \"COMPLETED_TXN_COMPONENTS\" WHERE \"CTC_TXNID\" = "
-                + txnid);
-        LOG.debug("Going to execute query <{}>", s);
-        try (ResultSet rs2 = stmt.executeQuery(s)) {
-          if (rs2.next()) {
-            return TxnStatus.COMMITTED;
-          }
-        }
-        // could also check WRITE_SET but that seems overkill
-        return TxnStatus.UNKNOWN;
-      }
-      return TxnStatus.fromString(rs.getString(1));
-    }
+  @Override
+  public void heartbeat(HeartbeatRequest ids)
+      throws NoSuchTxnException,  NoSuchLockException, TxnAbortedException, MetaException {
+    new HeartbeatTxnFunction(ids.getTxnid()).execute(jdbcResource);
+    new HeartbeatLockFunction(ids.getLockid()).execute(jdbcResource);
+  }
+  
+  @Override
+  public HeartbeatTxnRangeResponse heartbeatTxnRange(HeartbeatTxnRangeRequest rqst) throws MetaException {
+    return new HeartbeatTxnRangeFunction(rqst).execute(jdbcResource);
   }
 
-  /**
-   * Checks if all the txns in the list are in open state and not read-only.
-   * @param txnIds list of txns to be evaluated for open state/read-only status
-   * @param stmt db statement
-   * @return If all the txns in open state and not read-only, then return true else false
-   */
-  private boolean isTxnsOpenAndNotReadOnly(List<Long> txnIds, Statement stmt) throws SQLException {
-    List<String> queries = new ArrayList<>();
-    StringBuilder prefix = new StringBuilder();
+  @Override
+  public long getTxnIdForWriteId(String dbName, String tblName, long writeId) throws MetaException {
+    return jdbcResource.execute(new TxnIdForWriteIdHandler(writeId, dbName, tblName));
+  }
 
-    // Get the count of txns from the given list that are in open state and not read-only.
-    // If the returned count is same as the input number of txns, then all txns are in open state and not read-only.
-    prefix.append("SELECT COUNT(*) FROM \"TXNS\" WHERE \"TXN_STATE\" = " + TxnStatus.OPEN
-        + " AND \"TXN_TYPE\" != " + TxnType.READ_ONLY.getValue() + " AND ");
+  @Override
+  public CompactionResponse compact(CompactionRequest rqst) throws MetaException {
+    return new CompactFunction(rqst, openTxnTimeOutMillis, getMutexAPI()).execute(jdbcResource);
+  }
 
-    TxnUtils.buildQueryWithINClause(conf, queries, prefix, new StringBuilder(),
-        txnIds, "\"TXN_ID\"", false, false);
+  @Override
+  public boolean submitForCleanup(CompactionRequest rqst, long highestWriteId, long txnId) throws MetaException {
+    // Put a compaction request in the queue.
+    long id = new GenerateCompactionQueueIdFunction().execute(jdbcResource);
+    jdbcResource.execute(new InsertCompactionRequestCommand(id, CompactionState.READY_FOR_CLEANING, rqst).withTxnDetails(highestWriteId, txnId));
+    return true;
+  }
 
-    long count = 0;
-    for (String query : queries) {
-      LOG.debug("Going to execute query <{}>", query);
-      try (ResultSet rs = stmt.executeQuery(query)) {
-        if (rs.next()) {
-          count += rs.getLong(1);
-        }
-      }
-    }
-    return count == txnIds.size();
+  @Override
+  public ShowCompactResponse showCompact(ShowCompactRequest rqst) throws MetaException {
+    return jdbcResource.execute(new ShowCompactHandler(rqst, sqlGenerator));
   }
 
   /**
-   * Get txns from the list that are either aborted or read-only.
-   * @param txnIds list of txns to be evaluated for aborted state/read-only status
-   * @param stmt db statement
+   * We assume this is only called by metadata cache server to know if there are new base/delta files should be read.
+   * The query filters compactions by state and only returns SUCCEEDED or READY_FOR_CLEANING compactions because
+   * only these two states means there are new files ready to be read.
    */
-  private String getAbortedAndReadOnlyTxns(List<Long> txnIds, Statement stmt) throws SQLException {
-    List<String> queries = new ArrayList<>();
-    StringBuilder prefix = new StringBuilder();
-
-    // Check if any of the txns in the list are either aborted or read-only.
-    prefix.append("SELECT \"TXN_ID\", \"TXN_STATE\", \"TXN_TYPE\" FROM \"TXNS\" WHERE ");
-    TxnUtils.buildQueryWithINClause(conf, queries, prefix, new StringBuilder(),
-        txnIds, "\"TXN_ID\"", false, false);
-    StringBuilder txnInfo = new StringBuilder();
-
-    for (String query : queries) {
-      LOG.debug("Going to execute query <{}>", query);
-      try (ResultSet rs = stmt.executeQuery(query)) {
-        while (rs.next()) {
-          long txnId = rs.getLong(1);
-          TxnStatus txnState = TxnStatus.fromString(rs.getString(2));
-          TxnType txnType = TxnType.findByValue(rs.getInt(3));
+  @Override
+  public GetLatestCommittedCompactionInfoResponse getLatestCommittedCompactionInfo(
+      GetLatestCommittedCompactionInfoRequest rqst) throws MetaException {
+    return jdbcResource.execute(new GetLatestCommittedCompactionInfoHandler(rqst));
+  }
 
-          if (txnState != TxnStatus.OPEN) {
-            txnInfo.append("{").append(txnId).append(",").append(txnState).append("}");
-          } else if (txnType == TxnType.READ_ONLY) {
-            txnInfo.append("{").append(txnId).append(",read-only}");
-          }
-        }
-      }
-    }
-    return txnInfo.toString();
+  @Override
+  public MetricsInfo getMetricsInfo() throws MetaException {
+    int threshold = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_TABLES_WITH_ABORTED_TXNS_THRESHOLD);
+    MetricsInfo metrics = jdbcResource.execute(MetricsInfoHandler.INSTANCE);
+    Set<String> resourceNames = jdbcResource.execute(new TablesWithAbortedTxnsHandler(threshold));
+    metrics.setTablesWithXAbortedTxnsCount(resourceNames.size());
+    metrics.setTablesWithXAbortedTxns(resourceNames);    
+    return metrics;
   }
 
   /**
-   * Get txns from the list that are committed.
-   * @param txnIds list of txns to be evaluated for committed state
-   * @param stmt db statement
+   * Retry-by-caller note:
+   * This may be retried after dbConn.commit.  At worst, it will create duplicate entries in
+   * TXN_COMPONENTS which won't affect anything.  See more comments in {@link #commitTxn(CommitTxnRequest)}
    */
-  private String getCommittedTxns(List<Long> txnIds, Statement stmt) throws SQLException {
-    List<String> queries = new ArrayList<>();
-    StringBuilder prefix = new StringBuilder();
-
-    // Check if any of the txns in the list are committed.
-    prefix.append("SELECT \"CTC_TXNID\" FROM \"COMPLETED_TXN_COMPONENTS\" WHERE ");
-    TxnUtils.buildQueryWithINClause(conf, queries, prefix, new StringBuilder(),
-        txnIds, "\"CTC_TXNID\"", false, false);
-    StringBuilder txnInfo = new StringBuilder();
-
-    for (String query : queries) {
-      LOG.debug("Going to execute query <{}>", query);
-      try (ResultSet rs = stmt.executeQuery(query)) {
-        while (rs.next()) {
-          long txnId = rs.getLong(1);
-          txnInfo.append("{").append(txnId).append(",c}");
-        }
-      }
-    }
-    return txnInfo.toString();
+  @Override
+  public void addDynamicPartitions(AddDynamicPartitions rqst) throws NoSuchTxnException,  TxnAbortedException, MetaException {
+    TxnType txnType = jdbcResource.execute(new GetOpenTxnTypeAndLockHandler(sqlGenerator, rqst.getTxnid()));
+    if (txnType == null) {
+      //ensures txn is still there and in expected state
+      new EnsureValidTxnFunction(rqst.getTxnid()).execute(jdbcResource);
+      shouldNeverHappen(rqst.getTxnid());
+    }
+    jdbcResource.execute(new InsertTxnComponentsCommand(rqst));
+    jdbcResource.getJdbcTemplate().update("DELETE FROM \"TXN_COMPONENTS\" " +
+            "WHERE \"TC_TXNID\" = :txnId AND \"TC_DATABASE\" = :dbName AND \"TC_TABLE\" = :tableName AND \"TC_PARTITION\" IS NULL",
+        new MapSqlParameterSource()
+            .addValue("txnId", rqst.getTxnid())
+            .addValue("dbName", org.apache.commons.lang3.StringUtils.lowerCase(rqst.getDbname()))
+            .addValue("tableName", org.apache.commons.lang3.StringUtils.lowerCase(rqst.getTablename())));
   }
 
   /**
-   * Used to raise an informative error when the caller expected a txn in a particular TxnStatus
-   * but found it in some other status
+   * Clean up corresponding records in metastore tables when corresponding object is dropped,
+   * specifically: TXN_COMPONENTS, COMPLETED_TXN_COMPONENTS, COMPACTION_QUEUE, COMPLETED_COMPACTIONS
+   * Retry-by-caller note: this is only idempotent assuming it's only called by dropTable/Db/etc
+   * operations.
+   * <p>
+   * HIVE_LOCKS and WS_SET are cleaned up by {@link AcidHouseKeeperService}, if turned on
    */
-  private static void raiseTxnUnexpectedState(TxnStatus actualStatus, long txnid)
-    throws NoSuchTxnException, TxnAbortedException {
-    switch (actualStatus) {
-      case ABORTED:
-        throw new TxnAbortedException("Transaction " + JavaUtils.txnIdToString(txnid) + " already aborted");
-      case COMMITTED:
-        throw new NoSuchTxnException("Transaction " + JavaUtils.txnIdToString(txnid) + " is already committed.");
-      case UNKNOWN:
-        throw new NoSuchTxnException("No such transaction " + JavaUtils.txnIdToString(txnid));
-      case OPEN:
-        throw new NoSuchTxnException(JavaUtils.txnIdToString(txnid) + " is " + TxnStatus.OPEN);
-      default:
-        throw new IllegalArgumentException("Unknown TxnStatus " + actualStatus);
-    }
+  @Override
+  public void cleanupRecords(HiveObjectType type, Database db, Table table,
+        Iterator<Partition> partitionIterator, boolean keepTxnToWriteIdMetaData) throws MetaException {
+    new CleanupRecordsFunction(type, db, table, partitionIterator, getDefaultCatalog(conf), keepTxnToWriteIdMetaData, null)
+        .execute(jdbcResource);
+  }
+
+  @Override
+  public void cleanupRecords(HiveObjectType type, Database db, Table table,
+        Iterator<Partition> partitionIterator, long txnId) throws MetaException {
+    new CleanupRecordsFunction(type, db, table, partitionIterator, getDefaultCatalog(conf), false, txnId)
+        .execute(jdbcResource);
   }
+  
   /**
-   * Returns the state of the transaction with {@code txnid} or throws if {@code raiseError} is true.
+   * Catalog hasn't been added to transactional tables yet, so it's passed in but not used.
    */
-  private static void ensureValidTxn(Connection dbConn, long txnid, Statement stmt)
-      throws SQLException, NoSuchTxnException, TxnAbortedException {
-    // We need to check whether this transaction is valid and open
-    String s = "SELECT \"TXN_STATE\" FROM \"TXNS\" WHERE \"TXN_ID\" = " + txnid;
-    LOG.debug("Going to execute query <{}>", s);
-    try (ResultSet rs = stmt.executeQuery(s)) {
-      if (!rs.next()) {
-        // todo: add LIMIT 1 instead of count - should be more efficient
-        s = "SELECT COUNT(*) FROM \"COMPLETED_TXN_COMPONENTS\" WHERE \"CTC_TXNID\" = " + txnid;
-        try (ResultSet rs2 = stmt.executeQuery(s)) {
-          // todo: strictly speaking you can commit an empty txn, thus 2nd conjunct is wrong but
-          // only
-          // possible for for multi-stmt txns
-          boolean alreadyCommitted = rs2.next() && rs2.getInt(1) > 0;
-          LOG.debug("Going to rollback");
-          rollbackDBConn(dbConn);
-          if (alreadyCommitted) {
-            // makes the message more informative - helps to find bugs in client code
-            throw new NoSuchTxnException("Transaction " + JavaUtils.txnIdToString(txnid)
-                + " is already committed.");
-          }
-          throw new NoSuchTxnException("No such transaction " + JavaUtils.txnIdToString(txnid));
-        }
-      }
-      if (TxnStatus.fromString(rs.getString(1)) == TxnStatus.ABORTED) {
-        LOG.debug("Going to rollback");
-        rollbackDBConn(dbConn);
-        throw new TxnAbortedException("Transaction " + JavaUtils.txnIdToString(txnid)
-            + " already aborted");// todo: add time of abort, which is not currently tracked.
-                                  // Requires schema change
-      }
-    }
+  @Override
+  public void onRename(String oldCatName, String oldDbName, String oldTabName, String oldPartName,
+      String newCatName, String newDbName, String newTabName, String newPartName)
+      throws MetaException {
+    new OnRenameFunction(oldCatName, oldDbName, oldTabName, oldPartName, newCatName, newDbName, newTabName, newPartName).execute(jdbcResource);
   }
-
-  private Optional<LockInfo> getLockFromLockId(Connection dbConn, long extLockId) throws MetaException, SQLException {
-    try (PreparedStatement pstmt = dbConn.prepareStatement(SELECT_LOCKS_FOR_LOCK_ID_QUERY)) {
-      pstmt.setLong(1, extLockId);
-      LOG.debug("Going to execute query <{}> for extLockId={}", SELECT_LOCKS_FOR_LOCK_ID_QUERY, extLockId);
-      try (ResultSet rs = pstmt.executeQuery()) {
-        if (!rs.next()) {
-          return Optional.empty();
-        }
-        LockInfo info = new LockInfo(rs);
-        LOG.debug("getTxnIdFromLockId({}) Return {}", extLockId, JavaUtils.txnIdToString(info.txnId));
-        return Optional.of(info);
-      }
-    }
+  
+  /**
+   * TODO: remove in future, for testing only, do not use.
+   */
+  @VisibleForTesting
+  @Override
+  public int getNumLocks() {
+    return Objects.requireNonNull(
+        jdbcResource.getJdbcTemplate().queryForObject("SELECT COUNT(*) FROM \"HIVE_LOCKS\"", new MapSqlParameterSource(), Integer.TYPE),
+        "This never should be null, it's just to suppress warnings");
   }
 
-  // NEVER call this function without first calling heartbeat(long, long)
-  private List<LockInfo> getLocksFromLockId(Connection dbConn, long extLockId) throws MetaException, SQLException {
-    try (PreparedStatement pstmt = dbConn.prepareStatement(SELECT_LOCKS_FOR_LOCK_ID_QUERY)) {
-      List<LockInfo> locks = new ArrayList<>();
-      pstmt.setLong(1, extLockId);
-      LOG.debug("Going to execute query <{}> for extLockId={}", SELECT_LOCKS_FOR_LOCK_ID_QUERY, extLockId);
-      try (ResultSet rs = pstmt.executeQuery()) {
-        while (rs.next()) {
-          locks.add(new LockInfo(rs));
-        }
-      }
-      if (locks.isEmpty()) {
-        throw new MetaException("This should never happen!  We already " +
-          "checked the lock(" + JavaUtils.lockIdToString(extLockId) + ") existed but now we can't find it!");
-      }
-      LOG.debug("Found {} locks for extLockId={}. Locks: {}", locks.size(), extLockId, locks);
-      return locks;
-    }
+  /**
+   * TODO: remove in future, for testing only, do not use.
+   */
+  @VisibleForTesting
+  @Override
+  public long setTimeout(long milliseconds) {
+    long previous_timeout = timeout;
+    timeout = milliseconds;
+    return previous_timeout;
   }
 
-  // Clean time out locks from the database not associated with a transactions, i.e. locks
-  // for read-only autoCommit=true statements.  This does a commit,
-  // and thus should be done before any calls to heartbeat that will leave
-  // open transactions.
-  private void timeOutLocks() {
-    //doing a SELECT first is less efficient but makes it easier to debug things
-    //when txnid is <> 0, the lock is associated with a txn and is handled by performTimeOuts()
-    //want to avoid expiring locks for a txn w/o expiring the txn itself
+  protected Connection getDbConn(int isolationLevel, DataSource connPool) throws SQLException {
+    Connection dbConn = null;
     try {
-      Set<Long> timedOutLockIds = new TreeSet<>(
-          jdbcResource.getJdbcTemplate().query(String.format(SELECT_TIMED_OUT_LOCKS_QUERY, getEpochFn(dbProduct)),
-              new MapSqlParameterSource().addValue("timeout", timeout),
-              (rs, rowNum) -> rs.getLong(1)));
-      if (timedOutLockIds.isEmpty()) {
-        LOG.debug("Did not find any timed-out locks, therefore retuning.");
-        return;
+      dbConn = connPool.getConnection();
+      dbConn.setAutoCommit(false);
+      dbConn.setTransactionIsolation(isolationLevel);
+      return dbConn;
+    } catch (SQLException e) {
+      if (dbConn != null) {
+        dbConn.close();
       }
-
-      List<String> queries = new ArrayList<>();
-      StringBuilder prefix = new StringBuilder();
-      StringBuilder suffix = new StringBuilder();
-
-      //include same hl_last_heartbeat condition in case someone heartbeated since the select
-      prefix.append("DELETE FROM \"HIVE_LOCKS\" WHERE \"HL_LAST_HEARTBEAT\" < ");
-      prefix.append(getEpochFn(dbProduct)).append("-").append(timeout);
-      prefix.append(" AND \"HL_TXNID\" = 0 AND ");
-
-      TxnUtils.buildQueryWithINClause(conf, queries, prefix, suffix, timedOutLockIds,
-              "\"HL_LOCK_EXT_ID\"", true, false);
-      
-        int deletedLocks = 0;
-        for (String query : queries) {
-          LOG.debug("Going to execute update: <{}>", query);
-          deletedLocks += jdbcResource.getJdbcTemplate().update(query, new MapSqlParameterSource());
-        }
-        if (deletedLocks > 0) {
-          LOG.info("Deleted {} locks due to timed-out. Lock ids: {}", deletedLocks, timedOutLockIds);
-        }
-    } catch (Exception ex) {
-      LOG.error("Failed to purge timed-out locks: " + ex.getMessage(), ex);
+      throw e;
     }
   }
 
@@ -5783,274 +958,22 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
    * Will also delete locks which are not associated with a transaction and have timed out
    * Tries to keep transactions (against metastore db) small to reduce lock contention.
    */
-  @RetrySemantics.Idempotent
+  @Override
   public void performTimeOuts() {
-    jdbcResource.bindDataSource(POOL_TX);
-    try (TransactionContext context = jdbcResource.getTransactionManager().getTransaction(PROPAGATION_REQUIRED)) {
-      //We currently commit after selecting the TXNS to abort.  So whether SERIALIZABLE
-      //READ_COMMITTED, the effect is the same.  We could use FOR UPDATE on Select from TXNS
-      //and do the whole performTimeOuts() in a single huge transaction, but the only benefit
-      //would be to make sure someone cannot heartbeat one of these txns at the same time.
-      //The attempt to heartbeat would block and fail immediately after it's unblocked.
-      //With current (RC + multiple txns) implementation it is possible for someone to send
-      //heartbeat at the very end of the expiry interval, and just after the Select from TXNS
-      //is made, in which case heartbeat will succeed but txn will still be Aborted.
-      //Solving this corner case is not worth the perf penalty.  The client should heartbeat in a
-      //timely way.
-      timeOutLocks();
-      while (true) {
-        String s = " \"TXN_ID\", \"TXN_TYPE\" FROM \"TXNS\" WHERE \"TXN_STATE\" = " + TxnStatus.OPEN +
-            " AND (" +
-            "\"TXN_TYPE\" != " + TxnType.REPL_CREATED.getValue() +
-            " AND \"TXN_LAST_HEARTBEAT\" <  " + getEpochFn(dbProduct) + "-" + timeout +
-            " OR " +
-            " \"TXN_TYPE\" = " + TxnType.REPL_CREATED.getValue() +
-            " AND \"TXN_LAST_HEARTBEAT\" <  " + getEpochFn(dbProduct) + "-" + replicationTxnTimeout +
-            ")";
-        //safety valve for extreme cases
-        s = sqlGenerator.addLimitClause(10 * TIMED_OUT_TXN_ABORT_BATCH_SIZE, s);
-
-        LOG.debug("Going to execute query <{}>", s);
-        List<Map<Long, TxnType>> timedOutTxns = jdbcResource.getJdbcTemplate().query(s, rs -> {
-          List<Map<Long, TxnType>> txnbatch = new ArrayList<>();
-          Map<Long, TxnType> currentBatch = new HashMap<>(TIMED_OUT_TXN_ABORT_BATCH_SIZE);
-          while (rs.next()) {
-            currentBatch.put(rs.getLong(1),TxnType.findByValue(rs.getInt(2)));
-            if (currentBatch.size() == TIMED_OUT_TXN_ABORT_BATCH_SIZE) {
-              txnbatch.add(currentBatch);
-              currentBatch = new HashMap<>(TIMED_OUT_TXN_ABORT_BATCH_SIZE);
-            }
-          }
-          if (currentBatch.size() > 0) {
-            txnbatch.add(currentBatch);
-          }
-          return txnbatch;
-        });
-        //noinspection DataFlowIssue
-        if (timedOutTxns.size() == 0) {
-          jdbcResource.getTransactionManager().commit(context);
-          return;
-        }
-
-        Object savePoint = context.getTransactionStatus().createSavepoint();
-
-        int numTxnsAborted = 0;
-        for (Map<Long, TxnType> batchToAbort : timedOutTxns) {
-          context.getTransactionStatus().releaseSavepoint(savePoint);
-          savePoint = context.getTransactionStatus().createSavepoint();
-          if (abortTxns(jdbcResource.getConnection(), new ArrayList<>(batchToAbort.keySet()), true, false, false, TxnErrorMsg.ABORT_TIMEOUT) == batchToAbort.size()) {
-            numTxnsAborted += batchToAbort.size();
-            //todo: add TXNS.COMMENT filed and set it to 'aborted by system due to timeout'
-            LOG.info("Aborted the following transactions due to timeout: {}", batchToAbort);
-            if (transactionalListeners != null) {
-              for (Map.Entry<Long, TxnType> txnEntry : batchToAbort.entrySet()) {
-                List<String> dbsUpdated = getTxnDbsUpdated(txnEntry.getKey(), jdbcResource.getConnection());
-                MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners,
-                    EventMessage.EventType.ABORT_TXN,
-                    new AbortTxnEvent(txnEntry.getKey(), txnEntry.getValue(), null, dbsUpdated),
-                    jdbcResource.getConnection(), sqlGenerator);
-              }
-              LOG.debug("Added Notifications for the transactions that are aborted due to timeout: {}", batchToAbort);
-            }
-          } else {
-            //could not abort all txns in this batch - this may happen because in parallel with this
-            //operation there was activity on one of the txns in this batch (commit/abort/heartbeat)
-            //This is not likely but may happen if client experiences long pause between heartbeats or
-            //unusually long/extreme pauses between heartbeat() calls and other logic in checkLock(),
-            //lock(), etc.
-            context.getTransactionStatus().rollbackToSavepoint(savePoint);
-          }
-        }
-        LOG.info("Aborted {} transaction(s) due to timeout", numTxnsAborted);
-        if (MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON)) {
-          Metrics.getOrCreateCounter(MetricsConstants.TOTAL_NUM_TIMED_OUT_TXNS).inc(numTxnsAborted);
-        }
-      }
-    } catch (MetaException | SQLException e) {
-      LOG.warn("Aborting timed out transactions failed due to " + e.getMessage(), e);
-    } finally {
-      jdbcResource.unbindDataSource();
-    }
+    new PerformTimeoutsFunction(timeout, replicationTxnTimeout, transactionalListeners).execute(jdbcResource);
   }
 
   @Override
-  @RetrySemantics.ReadOnly
   public void countOpenTxns() throws MetaException {
-    Connection dbConn = null;
-    Statement stmt = null;
-    ResultSet rs = null;
-    try {
-      try {
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-        stmt = dbConn.createStatement();
-        String s = "SELECT COUNT(*) FROM \"TXNS\" WHERE \"TXN_STATE\" = " + TxnStatus.OPEN;
-        LOG.debug("Going to execute query <{}>", s);
-        rs = stmt.executeQuery(s);
-        if (!rs.next()) {
-          LOG.error("Transaction database not properly configured, can't find txn_state from TXNS.");
-        } else {
-          Long numOpen = rs.getLong(1);
-          if (numOpen > Integer.MAX_VALUE) {
-            LOG.error("Open transaction count above {}, can't count that high!", Integer.MAX_VALUE);
-          } else {
-            numOpenTxns.set(numOpen.intValue());
-          }
-        }
-      } catch (SQLException e) {
-        LOG.info("Failed to update number of open transactions");
-        checkRetryable(e, "countOpenTxns()");
-      } finally {
-        close(rs, stmt, dbConn);
-      }
-    } catch (RetryException e) {
-      countOpenTxns();
-    }
-  }
-
-  /**
-   * Add min history level entry for each generated txn record
-   * @param dbConn Connection
-   * @param txnIds new transaction ids
-   * @deprecated Remove this method when min_history_level table is dropped
-   * @throws SQLException ex
-   */
-  @Deprecated
-  private void addTxnToMinHistoryLevel(Connection dbConn, List<Long> txnIds, long minOpenTxnId) throws SQLException {
-    if (!useMinHistoryLevel) {
-      return;
-    }
-    // Need to register minimum open txnid for current transactions into MIN_HISTORY table.
-    try (Statement stmt = dbConn.createStatement()) {
-      List<String> rows = txnIds.stream().map(txnId -> txnId + ", " + minOpenTxnId).collect(Collectors.toList());
-
-      // Insert transaction entries into MIN_HISTORY_LEVEL.
-      List<String> inserts =
-          sqlGenerator.createInsertValuesStmt("\"MIN_HISTORY_LEVEL\" (\"MHL_TXNID\", \"MHL_MIN_OPEN_TXNID\")", rows);
-      for (String insert : inserts) {
-        LOG.debug("Going to execute insert <{}>", insert);
-        stmt.execute(insert);
-      }
-      LOG.info("Added entries to MIN_HISTORY_LEVEL for current txns: ({}) with min_open_txn: {}", txnIds, minOpenTxnId);
-    } catch (SQLException e) {
-      if (dbProduct.isTableNotExistsError(e)) {
-        // If the table does not exists anymore, we disable the flag and start to work the new way
-        // This enables to switch to the new functionality without a restart
-        useMinHistoryLevel = false;
-      } else {
-        throw e;
-      }
+    int openTxns = jdbcResource.execute(new CountOpenTxnsHandler());
+    if (openTxns > -1) {
+      numOpenTxns.set(openTxns);
     }
   }
 
   @Override
-  @RetrySemantics.SafeToRetry
   public void addWriteIdsToMinHistory(long txnid, Map<String, Long> minOpenWriteIds) throws MetaException {
-    if (!useMinHistoryWriteId) {
-      return;
-    }
-    // Need to register minimum open writeId for current transactions into MIN_HISTORY_WRITE_ID table.
-    try {
-      Connection dbConn = null;
-      try {
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-        try (PreparedStatement pstmt = dbConn.prepareStatement(MIN_HISTORY_WRITE_ID_INSERT_QUERY)) {
-          int writeId = 0;
-
-          for (Map.Entry<String, Long> validWriteId : minOpenWriteIds.entrySet()) {
-            String[] names = TxnUtils.getDbTableName(validWriteId.getKey());
-
-            pstmt.setLong(1, txnid);
-            pstmt.setString(2, names[0]);
-            pstmt.setString(3, names[1]);
-            pstmt.setLong(4, validWriteId.getValue());
-
-            pstmt.addBatch();
-            writeId++;
-            if (writeId % maxBatchSize == 0) {
-              LOG.debug("Executing a batch of <" + TXN_TO_WRITE_ID_INSERT_QUERY + "> queries. " +
-                "Batch size: " + maxBatchSize);
-              pstmt.executeBatch();
-            }
-          }
-          if (writeId % maxBatchSize != 0) {
-            LOG.debug("Executing a batch of <" + TXN_TO_WRITE_ID_INSERT_QUERY + "> queries. " +
-              "Batch size: " + writeId % maxBatchSize);
-            pstmt.executeBatch();
-          }
-        }
-        dbConn.commit();
-        LOG.info("Added entries to MIN_HISTORY_WRITE_ID for current txn: {} with min_open_write_ids: ({})", txnid, minOpenWriteIds);
-      } catch (SQLException e) {
-        if (dbProduct.isTableNotExistsError(e)) {
-          // If the table does not exists anymore, we disable the flag and start to work the new way
-          // This enables to switch to the new functionality without a restart
-          useMinHistoryWriteId = false;
-        } else {
-          LOG.error("Caught exception while storing minOpenWriteIds: ", e);
-          rollbackDBConn(dbConn);
-          checkRetryable(e, "addWriteIdsToMinHistory");
-          throw new MetaException(e.getMessage());
-        }
-      } finally {
-        closeDbConn(dbConn);
-      }
-    } catch (RetryException e) {
-      addWriteIdsToMinHistory(txnid, minOpenWriteIds);
-    }
-  }
-
-  /**
-   * Remove txns from min_history_level table
-   * @param dbConn connection
-   * @param txnids transactions
-   * @deprecated Remove this method when min_history_level table is dropped
-   */
-  @Deprecated
-  private void removeTxnsFromMinHistoryLevel(Connection dbConn, List<Long> txnids) throws SQLException {
-    if (!useMinHistoryLevel) {
-      return;
-    }
-    List<String> queries = new ArrayList<>();
-    StringBuilder prefix = new StringBuilder("DELETE FROM \"MIN_HISTORY_LEVEL\" WHERE ");
-    try (Statement stmt = dbConn.createStatement()) {
-      TxnUtils.buildQueryWithINClause(conf, queries, prefix, new StringBuilder(), txnids, "\"MHL_TXNID\"", false, false);
-      executeQueriesInBatchNoCount(dbProduct, stmt, queries, maxBatchSize);
-      LOG.info("Removed transactions: ({}) from MIN_HISTORY_LEVEL", txnids);
-    } catch (SQLException e) {
-      if (dbProduct.isTableNotExistsError(e)) {
-        // If the table does not exists anymore, we disable the flag and start to work the new way
-        // This enables to switch to the new functionality without a restart
-        useMinHistoryLevel = false;
-      } else {
-        throw e;
-      }
-    }
-  }
-
-  /**
-   * Remove minOpenWriteIds from min_history_write_id tables
-   * @param dbConn connection
-   * @param txnids transactions
-   */
-  private void removeWriteIdsFromMinHistory(Connection dbConn, List<Long> txnids) throws SQLException {
-    if (!useMinHistoryWriteId) {
-      return;
-    }
-    List<String> queries = new ArrayList<>();
-    StringBuilder prefix = new StringBuilder("DELETE FROM \"MIN_HISTORY_WRITE_ID\" WHERE ");
-    try (Statement stmt = dbConn.createStatement()) {
-      TxnUtils.buildQueryWithINClause(conf, queries, prefix, new StringBuilder(), txnids, "\"MH_TXNID\"", false, false);
-      executeQueriesInBatchNoCount(dbProduct, stmt, queries, maxBatchSize);
-      LOG.info("Removed transactions: ({}) from MIN_HISTORY_WRITE_ID", txnids);
-    } catch (SQLException e) {
-      if (dbProduct.isTableNotExistsError(e)) {
-        // If the table does not exists anymore, we disable the flag and start to work the new way
-        // This enables to switch to the new functionality without a restart
-        useMinHistoryWriteId = false;
-      } else {
-        throw e;
-      }
-    }
+    jdbcResource.execute(new AddWriteIdsToMinHistoryCommand(txnid, minOpenWriteIds));
   }
 
   protected synchronized static DataSource setupJdbcConnectionPool(Configuration conf, int maxPoolSize) {
@@ -6063,450 +986,154 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         throw new RuntimeException(e);
       }
     } else {
-      String connectionPooler = MetastoreConf.getVar(conf, ConfVars.CONNECTION_POOLING_TYPE).toLowerCase();
+      String connectionPooler = MetastoreConf.getVar(conf, MetastoreConf.ConfVars.CONNECTION_POOLING_TYPE).toLowerCase();
       if ("none".equals(connectionPooler)) {
         LOG.info("Choosing not to pool JDBC connections");
-        return new NoPoolConnectionPool(conf);
+        return new NoPoolConnectionPool(conf, dbProduct);
       } else {
         throw new RuntimeException("Unknown JDBC connection pooling " + connectionPooler);
       }
     }
   }
 
-  /**
-   * Returns true if {@code ex} should be retried
-   */
-  static boolean isRetryable(Configuration conf, Exception ex) {
-    if(ex instanceof SQLException) {
-      SQLException sqlException = (SQLException)ex;
-      if (MANUAL_RETRY.equalsIgnoreCase(sqlException.getSQLState())) {
-        // Manual retry exception was thrown
-        return true;
-      }
-      if ("08S01".equalsIgnoreCase(sqlException.getSQLState())) {
-        //in MSSQL this means Communication Link Failure
-        return true;
-      }
-      if ("ORA-08176".equalsIgnoreCase(sqlException.getSQLState()) ||
-        sqlException.getMessage().contains("consistent read failure; rollback data not available")) {
-        return true;
-      }
-
-      String regex = MetastoreConf.getVar(conf, ConfVars.TXN_RETRYABLE_SQLEX_REGEX);
-      if (regex != null && !regex.isEmpty()) {
-        String[] patterns = regex.split(",(?=\\S)");
-        String message = getMessage((SQLException)ex);
-        for (String p : patterns) {
-          if (Pattern.matches(p, message)) {
-            return true;
-          }
-        }
-      }
-      //see also https://issues.apache.org/jira/browse/HIVE-9938
-    }
-    return false;
-  }
-
-  private boolean isDuplicateKeyError(SQLException ex) {
-    return dbProduct.isDuplicateKeyError(ex);
+  @Override
+  public MutexAPI getMutexAPI() {
+    return mutexAPI;
   }
 
-  private static String getMessage(SQLException ex) {
-    return ex.getMessage() + " (SQLState=" + ex.getSQLState() + ", ErrorCode=" + ex.getErrorCode() + ")";
-  }
-  static String quoteString(String input) {
-    return "'" + input + "'";
-  }
-  static String quoteChar(char c) {
-    return "'" + c + "'";
+  @Override
+  public LockHandle acquireLock(String key) throws MetaException {
+    return mutexAPI.acquireLock(key);
   }
 
-  /**
-   * {@link #lockInternal()} and {@link #unlockInternal()} are used to serialize those operations that require
-   * Select ... For Update to sequence operations properly.  In practice that means when running
-   * with Derby database.  See more notes at class level.
-   */
-  protected void lockInternal() {
-    if(dbProduct.isDERBY()) {
-      derbyLock.lock();
-    }
-  }
-  protected void unlockInternal() {
-    if(dbProduct.isDERBY()) {
-      derbyLock.unlock();
-    }
-  }
   @Override
-  @RetrySemantics.Idempotent
-  public MutexAPI getMutexAPI() {
-    return this;
+  public void acquireLock(String key, LockHandle handle) throws MetaException {
+    mutexAPI.acquireLock(key, handle);
   }
 
   @Override
-  public LockHandle acquireLock(String key) throws MetaException {
-    /**
-     * The implementation here is a bit kludgey but done so that code exercised by unit tests
-     * (which run against Derby which has no support for select for update) is as similar to
-     * production code as possible.
-     * In particular, with Derby we always run in a single process with a single metastore and
-     * the absence of For Update is handled via a Semaphore.  The later would strictly speaking
-     * make the SQL statements below unnecessary (for Derby), but then they would not be tested.
-     */
-    Connection dbConn = null;
-    Statement stmt = null;
-    ResultSet rs = null;
-    boolean needToCloseConn = true;
-    try {
-      try {
-        String sqlStmt = sqlGenerator.addForUpdateClause("SELECT \"MT_COMMENT\", \"MT_KEY2\" FROM \"AUX_TABLE\" WHERE \"MT_KEY1\"=" + quoteString(key));
-        lockInternal();
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED, connPoolMutex);
-        stmt = dbConn.createStatement();
-        LOG.debug("About to execute SQL: {}", sqlStmt);
-        rs = stmt.executeQuery(sqlStmt);
-        if (!rs.next()) {
-          close(rs);
-          try {
-            stmt.executeUpdate("INSERT INTO \"AUX_TABLE\" (\"MT_KEY1\", \"MT_KEY2\") VALUES(" + quoteString(key) + ", 0)");
-            dbConn.commit();
-          } catch (SQLException ex) {
-            if (!isDuplicateKeyError(ex)) {
-              throw new RuntimeException("Unable to lock " + quoteString(key) + " due to: " + getMessage(ex), ex);
-            }
-            //if here, it means a concrurrent acquireLock() inserted the 'key'
-
-            //rollback is done for the benefit of Postgres which throws (SQLState=25P02, ErrorCode=0) if
-            //you attempt any stmt in a txn which had an error.
-            dbConn.rollback();
-          }
-          rs = stmt.executeQuery(sqlStmt);
-          if (!rs.next()) {
-            throw new IllegalStateException("Unable to lock " + quoteString(key) + ".  Expected row in AUX_TABLE is missing.");
-          }
-        }
-        Semaphore derbySemaphore = null;
-        if(dbProduct.isDERBY()) {
-          derbyKey2Lock.putIfAbsent(key, new Semaphore(1));
-          derbySemaphore =  derbyKey2Lock.get(key);
-          derbySemaphore.acquire();
-        }
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("{} locked by {}", quoteString(key), quoteString(TxnHandler.hostname));
-        }
-        needToCloseConn = false;  //The connection is good, we need not close it
-        //OK, so now we have a lock
-        return new LockHandleImpl(dbConn, stmt, rs, key, derbySemaphore);
-      } catch (SQLException ex) {
-        checkRetryable(ex, "acquireLock(" + key + ")");
-        throw new MetaException("Unable to lock " + quoteString(key) + " due to: " + getMessage(ex) + "; " + StringUtils.stringifyException(ex));
-      }
-      catch(InterruptedException ex) {
-        throw new MetaException("Unable to lock " + quoteString(key) + " due to: " + ex.getMessage() + StringUtils.stringifyException(ex));
-      }
-      finally {
-        if (needToCloseConn) {
-          rollbackDBConn(dbConn);
-          close(rs, stmt, dbConn);
-        }
-        unlockInternal();
-      }
-    }
-    catch(RetryException ex) {
-      return acquireLock(key);
+  public AbortCompactResponse abortCompactions(AbortCompactionRequest reqst) throws MetaException, NoSuchCompactionException {
+    if (reqst.getCompactionIds().isEmpty()) {
+      LOG.info("Compaction ids are missing in request. No compactions to abort");
+      throw new NoSuchCompactionException("Compaction ids missing in request. No compactions to abort");
     }
+    return new AbortCompactionFunction(reqst, sqlRetryHandler).execute(jdbcResource);    
   }
 
-  @Override
-  public void acquireLock(String key, LockHandle handle) {
-    //the idea is that this will use LockHandle.dbConn
-    throw new NotImplementedException("acquireLock(String, LockHandle) is not implemented");
+  private static void shouldNeverHappen(long txnid) {
+    throw new RuntimeException("This should never happen: " + JavaUtils.txnIdToString(txnid));
+  }  
+  
+  private void deleteInvalidOpenTransactions(List<Long> txnIds) throws MetaException {
+    try {
+      sqlRetryHandler.executeWithRetry(new SqlRetryCallProperties().withCallerId("deleteInvalidOpenTransactions"),
+          () -> {
+            jdbcResource.execute(new DeleteInvalidOpenTxnsCommand(txnIds));
+            LOG.info("Removed transactions: ({}) from TXNS", txnIds);
+            jdbcResource.execute(new RemoveTxnsFromMinHistoryLevelCommand(txnIds));
+            return null;
+          });
+    } catch (TException e) {
+      throw new MetaException(e.getMessage());
+    }    
   }
 
   /**
    * Acquire the global txn lock, used to mutex the openTxn and commitTxn.
    * @param shared either SHARED_READ or EXCLUSIVE
-   * @throws SQLException
    */
-  private void acquireTxnLock(Statement stmt, boolean shared) throws SQLException, MetaException {
+  private void acquireTxnLock(boolean shared) throws MetaException {
     String sqlStmt = sqlGenerator.createTxnLockStatement(shared);
-    stmt.execute(sqlStmt);
-    LOG.debug("TXN lock locked by {} in mode {}", quoteString(TxnHandler.hostname), shared);
+    jdbcResource.getJdbcTemplate().getJdbcTemplate().execute((Statement stmt) -> {
+      stmt.execute(sqlStmt);
+      return null;
+    });
+    LOG.debug("TXN lock locked by '{}' in mode {}", TxnHandler.hostname, shared);
   }
 
-  private static final class LockHandleImpl implements LockHandle {
-    private final Connection dbConn;
-    private final Statement stmt;
-    private final ResultSet rs;
-    private final Semaphore derbySemaphore;
-    private final String key;
-    private final Long lastUpdateTime;
-
-    LockHandleImpl(Connection conn, Statement stmt, ResultSet rs, String key, Semaphore derbySemaphore) {
-      this.dbConn = conn;
-      this.stmt = stmt;
-      this.rs = rs;
-      this.derbySemaphore = derbySemaphore;
-      if(derbySemaphore != null) {
-        //oterwise it may later release permit acquired by someone else
-        assert derbySemaphore.availablePermits() == 0 : "Expected locked Semaphore";
-      }
-      this.key = key;
-      Long lastUpdateTime;
-      try {
-        lastUpdateTime = rs.getLong("MT_KEY2");
-      } catch (SQLException e) {
-        LOG.warn("Couldn't resolve MT_KEY2 for MT_KEY1=" + quoteString(this.key), e);
-        lastUpdateTime = -1L;
-      }
-      this.lastUpdateTime = lastUpdateTime;
-    }
-
-    @Override
-    public void releaseLocks() {
-      rollbackDBConn(dbConn);
-      TxnHandler.close(rs, stmt, dbConn);
-      if(derbySemaphore != null) {
-        derbySemaphore.release();
-      }
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("{} unlocked by {}", quoteString(key), quoteString(TxnHandler.hostname));
-      }
-    }
-
-    @Override
-    public Long getLastUpdateTime() {
-      return lastUpdateTime;
-    }
+  /**
+   * Determine the current time, using the RDBMS as a source of truth
+   * @return current time in milliseconds
+   * @throws org.apache.hadoop.hive.metastore.api.MetaException if the time cannot be determined
+   */
+  protected Timestamp getDbTime() throws MetaException {
+    return jdbcResource.getJdbcTemplate().queryForObject(
+        dbProduct.getDBTime(),
+        new MapSqlParameterSource(),
+        (ResultSet rs, int rowNum) -> rs.getTimestamp(1));
+  }
 
-    @Override
-    public void releaseLocks(Long timestamp) {
-      try {
-        stmt.executeUpdate("UPDATE \"AUX_TABLE\" SET \"MT_KEY2\" = "+ timestamp + " WHERE \"MT_KEY1\"=" + quoteString(key));
-        dbConn.commit();
-      } catch (SQLException ex) {
-        LOG.warn("Unable to update MT_KEY2 value for MT_KEY1=" + key, ex);
-        rollbackDBConn(dbConn);
-      }
-      TxnHandler.close(rs, stmt, dbConn);
-      if(derbySemaphore != null) {
-        derbySemaphore.release();
-      }
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("{} unlocked by {}", quoteString(key), quoteString(TxnHandler.hostname));
+  private void determineDatabaseProduct(Connection conn) {
+    try {
+      String s = conn.getMetaData().getDatabaseProductName();
+      dbProduct = DatabaseProduct.determineDatabaseProduct(s, conf);
+      if (dbProduct.isUNDEFINED()) {
+        String msg = "Unrecognized database product name <" + s + ">";
+        LOG.error(msg);
+        throw new IllegalStateException(msg);
       }
+    } catch (SQLException e) {
+      String msg = "Unable to get database product name";
+      LOG.error(msg, e);
+      throw new IllegalStateException(msg, e);
     }
-
-    @Override
-    public void close() {
-      releaseLocks();
-    }
-
   }
-
-
-  private static class NoPoolConnectionPool implements DataSource {
-    // Note that this depends on the fact that no-one in this class calls anything but
-    // getConnection.  If you want to use any of the Logger or wrap calls you'll have to
-    // implement them.
-    private final Configuration conf;
-    private Driver driver;
-    private String connString;
-    private String user;
-    private String passwd;
-
-    public NoPoolConnectionPool(Configuration conf) {
-      this.conf = conf;
+  
+  private void initJdbcResource() {
+    if (jdbcResource == null) {
+      jdbcResource = new MultiDataSourceJdbcResource(dbProduct, conf, sqlGenerator);
+      jdbcResource.registerDataSource(POOL_TX, connPool);
+      jdbcResource.registerDataSource(POOL_MUTEX, connPoolMutex);
+      jdbcResource.registerDataSource(POOL_COMPACTOR, connPoolCompactor);
     }
+  }
 
-    @Override
-    public Connection getConnection() throws SQLException {
-      if (user == null) {
-        user = DataSourceProvider.getMetastoreJdbcUser(conf);
-        passwd = DataSourceProvider.getMetastoreJdbcPasswd(conf);
-      }
-      return getConnection(user, passwd);
+  /**
+   * Check if provided table is usable
+   * @return
+   */
+  private boolean checkIfTableIsUsable(String tableName, boolean configValue) {
+    if (!configValue) {
+      // don't check it if disabled
+      return false;
     }
-
-    @Override
-    public Connection getConnection(String username, String password) throws SQLException {
-      // Find the JDBC driver
-      if (driver == null) {
-        String driverName = MetastoreConf.getVar(conf, ConfVars.CONNECTION_DRIVER);
-        if (driverName == null || driverName.equals("")) {
-          String msg = "JDBC driver for transaction db not set in configuration " +
-              "file, need to set " + ConfVars.CONNECTION_DRIVER.getVarname();
-          LOG.error(msg);
-          throw new RuntimeException(msg);
-        }
-        try {
-          LOG.info("Going to load JDBC driver {}", driverName);
-          driver = (Driver) Class.forName(driverName).newInstance();
-        } catch (InstantiationException e) {
-          throw new RuntimeException("Unable to instantiate driver " + driverName + ", " +
-              e.getMessage(), e);
-        } catch (IllegalAccessException e) {
+    jdbcResource.bindDataSource(POOL_TX);
+    try {
+      jdbcResource.getJdbcTemplate().query("SELECT 1 FROM \"" + tableName + "\"",
+          new MapSqlParameterSource(), ResultSet::next);
+    } catch (DataAccessException e) {
+      LOG.debug("Catching sql exception in " + tableName + " check", e);
+      if (e.getCause() instanceof SQLException) {
+        if (dbProduct.isTableNotExistsError(e)) {
+          return false;
+        } else {
           throw new RuntimeException(
-              "Unable to access driver " + driverName + ", " + e.getMessage(),
-              e);
-        } catch (ClassNotFoundException e) {
-          throw new RuntimeException("Unable to find driver " + driverName + ", " + e.getMessage(),
-              e);
-        }
-        connString = MetastoreConf.getVar(conf, ConfVars.CONNECT_URL_KEY);
-      }
-
-      try {
-        LOG.info("Connecting to transaction db with connection string {}", connString);
-        Properties connectionProps = new Properties();
-        connectionProps.setProperty("user", username);
-        connectionProps.setProperty("password", password);
-        Connection conn = driver.connect(connString, connectionProps);
-        String prepareStmt = dbProduct != null ? dbProduct.getPrepareTxnStmt() : null;
-        if (prepareStmt != null) {
-          try (Statement stmt = conn.createStatement()) {
-            stmt.execute(prepareStmt);
-          }
+              "Unable to select from transaction database: " + SqlRetryHandler.getMessage(e) + StringUtils.stringifyException(e));
         }
-        conn.setAutoCommit(false);
-        return conn;
-      } catch (SQLException e) {
-        throw new RuntimeException("Unable to connect to transaction manager using " + connString
-            + ", " + e.getMessage(), e);
       }
+    } finally {
+      jdbcResource.unbindDataSource();
     }
-
-    @Override
-    public PrintWriter getLogWriter() throws SQLException {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void setLogWriter(PrintWriter out) throws SQLException {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void setLoginTimeout(int seconds) throws SQLException {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public int getLoginTimeout() throws SQLException {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public java.util.logging.Logger getParentLogger() throws SQLFeatureNotSupportedException {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public <T> T unwrap(Class<T> iface) throws SQLException {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public boolean isWrapperFor(Class<?> iface) throws SQLException {
-      throw new UnsupportedOperationException();
-    }
-  }
-
-  @Override
-  @RetrySemantics.SafeToRetry
-  public AbortCompactResponse abortCompactions(AbortCompactionRequest reqst) throws MetaException, NoSuchCompactionException {
-    Map<Long, AbortCompactionResponseElement> abortCompactionResponseElements = new HashMap<>();
-    AbortCompactResponse response = new AbortCompactResponse(new HashMap<>());
-    response.setAbortedcompacts(abortCompactionResponseElements);
-
-    List<Long> compactionIdsToAbort = reqst.getCompactionIds();
-    if (compactionIdsToAbort.isEmpty()) {
-      LOG.info("Compaction ids are missing in request. No compactions to abort");
-      throw new NoSuchCompactionException("Compaction ids missing in request. No compactions to abort");
-    }
-    reqst.getCompactionIds().forEach(x -> abortCompactionResponseElements.put(x, getAbortCompactionResponseElement(x,"Error","No Such Compaction Id Available")));
-
-    List<CompactionInfo> eligibleCompactionsToAbort = findEligibleCompactionsToAbort(abortCompactionResponseElements,
-            compactionIdsToAbort);
-    for (CompactionInfo compactionInfo : eligibleCompactionsToAbort) {
-      abortCompactionResponseElements.put(compactionInfo.id, abortCompaction(compactionInfo));
-    }
-    return response;
-  }
-
-  private AbortCompactionResponseElement getAbortCompactionResponseElement(long compactionId, String status, String message) {
-    AbortCompactionResponseElement resEle = new AbortCompactionResponseElement(compactionId);
-    resEle.setMessage(message);
-    resEle.setStatus(status);
-    return resEle;
-  }
-
-  @RetrySemantics.SafeToRetry
-  private AbortCompactionResponseElement abortCompaction(CompactionInfo compactionInfo) throws MetaException {
-    SqlRetryFunction<AbortCompactionResponseElement> function = () -> {
-      jdbcResource.bindDataSource(POOL_TX);
-      try (TransactionContext context = jdbcResource.getTransactionManager().getTransaction(PROPAGATION_REQUIRED)) {
-        compactionInfo.state = TxnStore.ABORTED_STATE;
-        compactionInfo.errorMessage = "Compaction Aborted by Abort Comapction request.";
-        int updCount;
-        try {
-          updCount = jdbcResource.execute(new InsertCompactionInfoCommand(compactionInfo, getDbTime().getTime()));
-        } catch (Exception e) {
-          LOG.error("Unable to update compaction record: {}.", compactionInfo);
-          return getAbortCompactionResponseElement(compactionInfo.id, "Error",
-              "Error while aborting compaction:Unable to update compaction record in COMPLETED_COMPACTIONS");
-        }
-        LOG.debug("Inserted {} entries into COMPLETED_COMPACTIONS", updCount);
-        try {
-          updCount = jdbcResource.getJdbcTemplate().update("DELETE FROM \"COMPACTION_QUEUE\" WHERE \"CQ_ID\" = :id",
-              new MapSqlParameterSource().addValue("id", compactionInfo.id));
-          if (updCount != 1) {
-            LOG.error("Unable to update compaction record: {}. updCnt={}", compactionInfo, updCount);
-            return getAbortCompactionResponseElement(compactionInfo.id, "Error",
-                "Error while aborting compaction: Unable to update compaction record in COMPACTION_QUEUE");
-          } else {
-            jdbcResource.getTransactionManager().commit(context);
-            return getAbortCompactionResponseElement(compactionInfo.id, "Success",
-                "Successfully aborted compaction");
-          }
-        } catch (DataAccessException e) {
-          return getAbortCompactionResponseElement(compactionInfo.id, "Error",
-              "Error while aborting compaction:" + e.getMessage());
-        }
-      } finally {
-        jdbcResource.unbindDataSource();
-      }
-    };
-    return sqlRetryHandler.executeWithRetry(
-        new SqlRetryCallProperties().withCallerId("abortCompaction(" + compactionInfo + ")"), function);
+    return true;
   }
-  
-  private List<CompactionInfo> findEligibleCompactionsToAbort(Map<Long,
-          AbortCompactionResponseElement> abortCompactionResponseElements, List<Long> requestedCompId) throws MetaException {
 
-    List<CompactionInfo> compactionInfoList = new ArrayList<>();
-    String queryText = TxnQueries.SELECT_COMPACTION_QUEUE_BY_COMPID + "  WHERE \"CC_ID\" IN (?) " ;
-    String sqlIN = requestedCompId.stream()
-            .map(x -> String.valueOf(x))
-            .collect(Collectors.joining(",", "(", ")"));
-    queryText = queryText.replace("(?)", sqlIN);
-    try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-         Statement pStmt = dbConn.createStatement()) {
-      try (ResultSet rs = pStmt.executeQuery(queryText)) {
-        while (rs.next()) {
-          char compState = rs.getString(5).charAt(0);
-          long compID = rs.getLong(1);
-          if (CompactionState.INITIATED.equals(CompactionState.fromSqlConst(compState))) {
-            compactionInfoList.add(CompactionInfo.loadFullFromCompactionQueue(rs));
-          } else {
-            abortCompactionResponseElements.put(compID, getAbortCompactionResponseElement(compID,"Error",
-                    "Error while aborting compaction as compaction is in state-" + CompactionState.fromSqlConst(compState)));
-          }
-        }
-      }
-    } catch (SQLException e) {
-      throw new MetaException("Unable to select from transaction database-" + StringUtils.stringifyException(e));
+  /**
+   * Returns the databases updated by txnId.
+   * Queries TXN_TO_WRITE_ID using txnId.
+   *
+   * @param txnId
+   * @throws MetaException
+   */
+  private List<String> getTxnDbsUpdated(long txnId) throws MetaException {
+    try {
+      return sqlRetryHandler.executeWithRetry(
+          new SqlRetryCallProperties().withCallerId("GetTxnDbsUpdatedHandler"),
+          () -> jdbcResource.execute(new GetTxnDbsUpdatedHandler(txnId)));
+    } catch (MetaException e) {
+      throw e;
+    } catch (TException e) {
+      throw new MetaException(e.getMessage());
     }
-    return compactionInfoList;
   }
 
 }
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnLockManager.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnLockManager.java
new file mode 100644
index 00000000000..e5f31f40212
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnLockManager.java
@@ -0,0 +1,85 @@
+/*
+ * 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.metastore.txn;
+
+import org.apache.hadoop.hive.common.classification.RetrySemantics;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.api.TxnOpenException;
+import org.apache.hadoop.hive.metastore.api.UnlockRequest;
+import org.apache.hadoop.hive.metastore.txn.retry.SqlRetry;
+import org.springframework.transaction.annotation.Transactional;
+
+import static org.apache.hadoop.hive.metastore.txn.TxnStore.POOL_TX;
+
+public interface TxnLockManager {
+
+  @SqlRetry(lockInternally = true)
+  @Transactional(POOL_TX)
+  long enqueueLock(LockRequest rqst) throws NoSuchTxnException, TxnAbortedException, MetaException;
+
+  /**
+   * Check whether a lock has been obtained.  This is used after {@link #enqueueLock(LockRequest)} returned a wait
+   * state.
+   * @param extLockId 
+   * @param txnId Transaction id
+   * @param zeroWaitReadEnabled
+   * @param isExclusiveCTAS
+   * @return info on the state of the lock
+   * @throws NoSuchTxnException
+   * @throws NoSuchLockException
+   * @throws TxnAbortedException
+   * @throws MetaException
+   */
+  @SqlRetry(lockInternally = true)
+  @Transactional(value = POOL_TX, noRollbackFor = {TxnAbortedException.class})
+  LockResponse checkLock(long extLockId, long txnId, boolean zeroWaitReadEnabled, boolean isExclusiveCTAS)
+      throws NoSuchTxnException, NoSuchLockException, TxnAbortedException, MetaException;
+
+  /**
+   * Unlock a lock.  It is not legal to call this if the caller is part of a txn.  In that case
+   * the txn should be committed or aborted instead.  (Note someday this will change since
+   * multi-statement transactions will allow unlocking in the transaction.)
+   * @param rqst lock to unlock
+   * @throws TxnOpenException
+   * @throws MetaException
+   */
+  @SqlRetry
+  @Transactional(POOL_TX)
+  @RetrySemantics.Idempotent
+  void unlock(UnlockRequest rqst)
+      throws TxnOpenException, MetaException;
+
+  /**
+   * Get information on current locks.
+   * @param rqst lock information to retrieve
+   * @return lock information.
+   * @throws MetaException
+   */
+  @SqlRetry
+  @Transactional(POOL_TX)
+  @RetrySemantics.ReadOnly
+  ShowLocksResponse showLocks(ShowLocksRequest rqst) throws MetaException;
+  
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
index bb1a5858191..f31308ba397 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
@@ -18,17 +18,15 @@
 package org.apache.hadoop.hive.metastore.txn;
 
 import com.google.common.annotations.VisibleForTesting;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.common.classification.RetrySemantics;
-import org.apache.hadoop.hive.metastore.api.AbortTxnRequest;
-import org.apache.hadoop.hive.metastore.api.NoSuchCompactionException;
-import org.apache.hadoop.hive.metastore.api.AbortTxnsRequest;
 import org.apache.hadoop.hive.metastore.api.AbortCompactResponse;
 import org.apache.hadoop.hive.metastore.api.AbortCompactionRequest;
+import org.apache.hadoop.hive.metastore.api.AbortTxnRequest;
+import org.apache.hadoop.hive.metastore.api.AbortTxnsRequest;
 import org.apache.hadoop.hive.metastore.api.AddDynamicPartitions;
 import org.apache.hadoop.hive.metastore.api.AllocateTableWriteIdsRequest;
 import org.apache.hadoop.hive.metastore.api.AllocateTableWriteIdsResponse;
@@ -55,6 +53,7 @@ import org.apache.hadoop.hive.metastore.api.Materialization;
 import org.apache.hadoop.hive.metastore.api.MaxAllocatedTableWriteIdRequest;
 import org.apache.hadoop.hive.metastore.api.MaxAllocatedTableWriteIdResponse;
 import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchCompactionException;
 import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
 import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
 import org.apache.hadoop.hive.metastore.api.OpenTxnRequest;
@@ -74,17 +73,21 @@ import org.apache.hadoop.hive.metastore.api.TxnType;
 import org.apache.hadoop.hive.metastore.api.UnlockRequest;
 import org.apache.hadoop.hive.metastore.api.UpdateTransactionalStatsRequest;
 import org.apache.hadoop.hive.metastore.events.ListenerEvent;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.entities.CompactionMetricsData;
+import org.apache.hadoop.hive.metastore.txn.entities.MetricsInfo;
 import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
-import org.apache.hadoop.hive.metastore.txn.retryhandling.SqlRetry;
-import org.apache.hadoop.hive.metastore.txn.retryhandling.SqlRetryHandler;
+import org.apache.hadoop.hive.metastore.txn.retry.SqlRetry;
+import org.apache.hadoop.hive.metastore.txn.retry.SqlRetryException;
+import org.apache.hadoop.hive.metastore.txn.retry.SqlRetryHandler;
 import org.springframework.transaction.annotation.Transactional;
 
 import java.sql.SQLException;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
-import java.util.Map;
 
 /**
  * A handler to answer transaction related calls that come into the metastore
@@ -157,23 +160,29 @@ public interface TxnStore extends Configurable {
    * @return information about open transactions
    * @throws MetaException
    */
+  @SqlRetry
+  @Transactional(POOL_TX)
   @RetrySemantics.ReadOnly
   GetOpenTxnsInfoResponse getOpenTxnsInfo() throws MetaException;
 
   /**
    * Get list of valid transactions.  This gives just the list of transactions that are open.
-   * @return list of open transactions, as well as a high water mark.
+   * @return list of open transactions, as well as a high watermark.
    * @throws MetaException
    */
+  @SqlRetry
+  @Transactional(POOL_TX)
   @RetrySemantics.ReadOnly
   GetOpenTxnsResponse getOpenTxns() throws MetaException;
 
   /**
    * Get list of valid transactions.  This gives just the list of transactions that are open.
    * @param excludeTxnTypes : excludes this type of txns while getting the open txns
-   * @return list of open transactions, as well as a high water mark.
+   * @return list of open transactions, as well as a high watermark.
    * @throws MetaException
    */
+  @SqlRetry
+  @Transactional(POOL_TX)
   @RetrySemantics.ReadOnly
   GetOpenTxnsResponse getOpenTxns(List<TxnType> excludeTxnTypes) throws MetaException;
 
@@ -181,6 +190,8 @@ public interface TxnStore extends Configurable {
    * Get the count for open transactions.
    * @throws MetaException
    */
+  @SqlRetry
+  @Transactional(POOL_TX)
   @RetrySemantics.ReadOnly
   void countOpenTxns() throws MetaException;
 
@@ -190,9 +201,13 @@ public interface TxnStore extends Configurable {
    * @return information on opened transactions
    * @throws MetaException
    */
+  @SqlRetry
+  @Transactional(value = POOL_TX, noRollbackFor = SqlRetryException.class)
   @RetrySemantics.Idempotent
   OpenTxnsResponse openTxns(OpenTxnRequest rqst) throws MetaException;
 
+  @SqlRetry(lockInternally = true)
+  @Transactional(POOL_TX)
   @RetrySemantics.Idempotent
   long getTargetTxnId(String replPolicy, long sourceTxnId) throws MetaException;
 
@@ -202,6 +217,8 @@ public interface TxnStore extends Configurable {
    * @throws NoSuchTxnException
    * @throws MetaException
    */
+  @SqlRetry(lockInternally = true)
+  @Transactional(POOL_TX)
   @RetrySemantics.Idempotent
   void abortTxn(AbortTxnRequest rqst) throws NoSuchTxnException, MetaException, TxnAbortedException;
 
@@ -211,6 +228,8 @@ public interface TxnStore extends Configurable {
    * @throws NoSuchTxnException
    * @throws MetaException
    */
+  @SqlRetry(lockInternally = true)
+  @Transactional(POOL_TX)
   @RetrySemantics.Idempotent
   void abortTxns(AbortTxnsRequest rqst) throws NoSuchTxnException, MetaException;
 
@@ -221,7 +240,9 @@ public interface TxnStore extends Configurable {
    * @throws TxnAbortedException
    * @throws MetaException
    */
-  @RetrySemantics.Idempotent
+  @SqlRetry(lockInternally = true)
+  @Transactional(value = POOL_TX, noRollbackFor = TxnAbortedException.class)
+  @RetrySemantics.Idempotent("No-op if already committed")
   void commitTxn(CommitTxnRequest rqst)
     throws NoSuchTxnException, TxnAbortedException,  MetaException;
 
@@ -230,9 +251,12 @@ public interface TxnStore extends Configurable {
    * @param rqst info on table/partitions and writeid snapshot to replicate.
    * @throws MetaException in case of failure
    */
-  @RetrySemantics.Idempotent
+  @SqlRetry(lockInternally = true)
+  @Transactional(POOL_TX)
+  @RetrySemantics.Idempotent("No-op if already replicated the writeid state")
   void replTableWriteIdState(ReplTblWriteIdStateRequest rqst) throws MetaException;
 
+  @Transactional(POOL_TX)
   void updateTransactionStatistics(UpdateTransactionalStatsRequest req) throws MetaException;
 
   /**
@@ -243,6 +267,7 @@ public interface TxnStore extends Configurable {
    * @param validTxnList valid transaction list for snapshot taken for current query
    * @throws MetaException
    */
+  @Transactional(POOL_TX)
   @RetrySemantics.Idempotent
   Materialization getMaterializationInvalidationInfo(
           final CreationMetadata cm, final String validTxnList)
@@ -252,27 +277,40 @@ public interface TxnStore extends Configurable {
   long getTxnIdForWriteId(String dbName, String tblName, long writeId)
       throws MetaException;
 
+  @SqlRetry
+  @Transactional(POOL_TX)
+  @RetrySemantics.ReadOnly
   long getLatestTxnIdInConflict(long txnid) throws MetaException;
 
+  @SqlRetry(lockInternally = true)
+  @Transactional(POOL_TX)
   LockResponse lockMaterializationRebuild(String dbName, String tableName, long txnId)
       throws MetaException;
 
+  @SqlRetry(lockInternally = true)
+  @Transactional(POOL_TX)
   boolean heartbeatLockMaterializationRebuild(String dbName, String tableName, long txnId)
       throws MetaException;
 
+  @SqlRetry(lockInternally = true)
+  @Transactional(POOL_TX)
   long cleanupMaterializationRebuildLocks(ValidTxnList validTxnList, long timeout)
       throws MetaException;
 
-    /**
-     * Gets the list of valid write ids for the given table wrt to current txn
-     * @param rqst info on transaction and list of table names associated with given transaction
-     * @throws NoSuchTxnException
-     * @throws MetaException
-     */
+  /**
+   * Gets the list of valid write ids for the given table wrt to current txn
+   * @param rqst info on transaction and list of table names associated with given transaction
+   * @throws NoSuchTxnException
+   * @throws MetaException
+   */
+  @SqlRetry  
+  @Transactional(POOL_TX)
   @RetrySemantics.ReadOnly
   GetValidWriteIdsResponse getValidWriteIds(GetValidWriteIdsRequest rqst)
           throws NoSuchTxnException,  MetaException;
 
+  @SqlRetry
+  @Transactional(POOL_TX)
   @RetrySemantics.SafeToRetry
   void addWriteIdsToMinHistory(long txnId, Map<String, Long> minOpenWriteIds) throws MetaException;
 
@@ -283,6 +321,8 @@ public interface TxnStore extends Configurable {
    * @throws TxnAbortedException
    * @throws MetaException
    */
+  @SqlRetry(lockInternally = true, retryOnDuplicateKey = true)
+  @Transactional(POOL_TX)
   AllocateTableWriteIdsResponse allocateTableWriteIds(AllocateTableWriteIdsRequest rqst)
     throws NoSuchTxnException, TxnAbortedException, MetaException;
 
@@ -291,6 +331,8 @@ public interface TxnStore extends Configurable {
    * @param rqst table for which the maximum writeId is requested
    * @return the maximum allocated writeId
    */
+  @SqlRetry
+  @Transactional(POOL_TX)
   MaxAllocatedTableWriteIdResponse getMaxAllocatedTableWrited(MaxAllocatedTableWriteIdRequest rqst)
       throws MetaException;
 
@@ -298,6 +340,8 @@ public interface TxnStore extends Configurable {
    * Called on conversion of existing table to full acid.  Sets initial write ID to a high
    * enough value so that we can assign unique ROW__IDs to data in existing files.
    */
+  @SqlRetry
+  @Transactional(POOL_TX)
   void seedWriteId(SeedTableWriteIdsRequest rqst) throws MetaException;
 
   /**
@@ -305,6 +349,8 @@ public interface TxnStore extends Configurable {
    * If the actual txnId is greater it will throw an exception.
    * @param rqst
    */
+  @SqlRetry(lockInternally = true)
+  @Transactional(POOL_TX)
   void seedTxnId(SeedTxnIdRequest rqst) throws MetaException;
 
   /**
@@ -330,6 +376,7 @@ public interface TxnStore extends Configurable {
    * @throws TxnAbortedException
    * @throws MetaException
    */
+  @Transactional(POOL_TX)
   @RetrySemantics.SafeToRetry
   LockResponse checkLock(CheckLockRequest rqst)
     throws NoSuchTxnException, NoSuchLockException, TxnAbortedException, MetaException;
@@ -343,6 +390,7 @@ public interface TxnStore extends Configurable {
    * @throws TxnOpenException
    * @throws MetaException
    */
+  @Transactional(POOL_TX)
   @RetrySemantics.Idempotent
   void unlock(UnlockRequest rqst)
     throws NoSuchLockException, TxnOpenException, MetaException;
@@ -353,6 +401,7 @@ public interface TxnStore extends Configurable {
    * @return lock information.
    * @throws MetaException
    */
+  @Transactional(POOL_TX)
   @RetrySemantics.ReadOnly
   ShowLocksResponse showLocks(ShowLocksRequest rqst) throws MetaException;
 
@@ -364,6 +413,8 @@ public interface TxnStore extends Configurable {
    * @throws TxnAbortedException
    * @throws MetaException
    */
+  @SqlRetry
+  @Transactional(POOL_TX)
   @RetrySemantics.SafeToRetry
   void heartbeat(HeartbeatRequest ids)
     throws NoSuchTxnException,  NoSuchLockException, TxnAbortedException, MetaException;
@@ -374,6 +425,8 @@ public interface TxnStore extends Configurable {
    * @return info on txns that were heartbeated
    * @throws MetaException
    */
+  @SqlRetry
+  @Transactional(POOL_TX)
   @RetrySemantics.SafeToRetry
   HeartbeatTxnRangeResponse heartbeatTxnRange(HeartbeatTxnRangeRequest rqst)
     throws MetaException;
@@ -385,9 +438,13 @@ public interface TxnStore extends Configurable {
    * @return id of the compaction that has been started or existing id if this resource is already scheduled
    * @throws MetaException
    */
+  @SqlRetry
+  @Transactional(POOL_TX)
   @RetrySemantics.Idempotent
   CompactionResponse compact(CompactionRequest rqst) throws MetaException;
 
+  @SqlRetry
+  @Transactional(POOL_TX)
   @RetrySemantics.SafeToRetry
   boolean submitForCleanup(CompactionRequest rqst, long highestWriteId, long txnId) throws MetaException;
 
@@ -397,6 +454,8 @@ public interface TxnStore extends Configurable {
    * @return compaction information
    * @throws MetaException
    */
+  @SqlRetry
+  @Transactional(POOL_TX)
   @RetrySemantics.ReadOnly
   ShowCompactResponse showCompact(ShowCompactRequest rqst) throws MetaException;
 
@@ -408,6 +467,7 @@ public interface TxnStore extends Configurable {
    * @throws NoSuchCompactionException
    * @throws MetaException
    */
+  @Transactional(POOL_TX)
   @RetrySemantics.Idempotent
   AbortCompactResponse abortCompactions(AbortCompactionRequest rqst) throws NoSuchCompactionException, MetaException;
 
@@ -422,6 +482,8 @@ public interface TxnStore extends Configurable {
    * partition specified by the request.
    * @throws MetaException
    */
+  @SqlRetry
+  @Transactional(POOL_TX)
   @RetrySemantics.ReadOnly
   GetLatestCommittedCompactionInfoResponse getLatestCommittedCompactionInfo(
       GetLatestCommittedCompactionInfoRequest rqst) throws MetaException;
@@ -433,6 +495,8 @@ public interface TxnStore extends Configurable {
    * @throws TxnAbortedException
    * @throws MetaException
    */
+  @SqlRetry(lockInternally = true)
+  @Transactional(POOL_TX)
   @RetrySemantics.SafeToRetry
   void addDynamicPartitions(AddDynamicPartitions rqst)
       throws NoSuchTxnException,  TxnAbortedException, MetaException;
@@ -445,20 +509,28 @@ public interface TxnStore extends Configurable {
    * @param partitionIterator partition iterator
    * @throws MetaException
    */
+  @SqlRetry
+  @Transactional(POOL_TX)
   @RetrySemantics.Idempotent
   default void cleanupRecords(HiveObjectType type, Database db, Table table, 
       Iterator<Partition> partitionIterator) throws MetaException {
     cleanupRecords(type, db, table, partitionIterator, false);
   }
 
+  @SqlRetry
+  @Transactional(POOL_TX)
   @RetrySemantics.Idempotent
   void cleanupRecords(HiveObjectType type, Database db, Table table, 
       Iterator<Partition> partitionIterator, boolean keepTxnToWriteIdMetaData) throws MetaException;
 
+  @SqlRetry
+  @Transactional(POOL_TX)
   @RetrySemantics.Idempotent
   void cleanupRecords(HiveObjectType type, Database db, Table table,
       Iterator<Partition> partitionIterator, long txnId) throws MetaException;
 
+  @SqlRetry
+  @Transactional(POOL_TX)
   @RetrySemantics.Idempotent
   void onRename(String oldCatName, String oldDbName, String oldTabName, String oldPartName,
       String newCatName, String newDbName, String newTabName, String newPartName)
@@ -467,6 +539,7 @@ public interface TxnStore extends Configurable {
   /**
    * Timeout transactions and/or locks.  This should only be called by the compactor.
    */
+  @Transactional(POOL_TX)
   @RetrySemantics.Idempotent
   void performTimeOuts();
 
@@ -712,6 +785,7 @@ public interface TxnStore extends Configurable {
    * WriteSet tracking is used to ensure proper transaction isolation.  This method deletes the
    * transaction metadata once it becomes unnecessary.
    */
+  @Transactional(POOL_TX)
   @RetrySemantics.SafeToRetry
   void performWriteSetGC() throws MetaException;
 
@@ -729,7 +803,8 @@ public interface TxnStore extends Configurable {
   boolean checkFailedCompactions(CompactionInfo ci) throws MetaException;
 
   @VisibleForTesting
-  int numLocksInLockTable() throws SQLException, MetaException;
+  @Transactional(POOL_TX)
+  int getNumLocks() throws SQLException, MetaException;
 
   @VisibleForTesting
   long setTimeout(long milliseconds);
@@ -755,6 +830,7 @@ public interface TxnStore extends Configurable {
      * The {@code key} is name of the lock. Will acquire an exclusive lock or block.  It returns
      * a handle which must be used to release the lock.  Each invocation returns a new handle.
      */
+    @SqlRetry(lockInternally = true)
     LockHandle acquireLock(String key) throws MetaException;
 
     /**
@@ -838,6 +914,8 @@ public interface TxnStore extends Configurable {
    * Returns ACID metadata related metrics info.
    * @return metrics info object
    */
+  @SqlRetry
+  @Transactional(POOL_TX)
   @RetrySemantics.ReadOnly
   MetricsInfo getMetricsInfo() throws MetaException;
 
@@ -855,7 +933,7 @@ public interface TxnStore extends Configurable {
   @SqlRetry
   @Transactional(POOL_COMPACTOR)
   CompactionMetricsData getCompactionMetricsData(String dbName, String tblName, String partitionName,
-      CompactionMetricsData.MetricType type) throws MetaException;
+                                                 CompactionMetricsData.MetricType type) throws MetaException;
 
   /**
    * Remove records from the compaction metrics cache matching the filter criteria passed in as parameters
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStoreMutex.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStoreMutex.java
new file mode 100644
index 00000000000..1013493a791
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStoreMutex.java
@@ -0,0 +1,209 @@
+/*
+ * 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.metastore.txn;
+
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.tools.SQLGenerator;
+import org.apache.hadoop.hive.metastore.txn.jdbc.MultiDataSourceJdbcResource;
+import org.apache.hadoop.hive.metastore.txn.jdbc.TransactionContext;
+import org.apache.hadoop.hive.metastore.txn.retry.SqlRetryHandler;
+import org.apache.hadoop.hive.metastore.utils.JavaUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.dao.DataAccessException;
+import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
+
+import java.sql.SQLException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Semaphore;
+
+import static org.apache.hadoop.hive.metastore.txn.TxnStore.POOL_MUTEX;
+import static org.springframework.transaction.TransactionDefinition.PROPAGATION_REQUIRED;
+
+public class TxnStoreMutex implements TxnStore.MutexAPI {
+
+  private static final Logger LOG = LoggerFactory.getLogger(TxnStoreMutex.class);
+  /**
+   * must be static since even in UT there may be > 1 instance of TxnHandler
+   * (e.g. via Compactor services)
+   */
+  private final static ConcurrentHashMap<String, Semaphore> derbyKey2Lock = new ConcurrentHashMap<>();
+  
+
+  private final SQLGenerator sqlGenerator;
+  private final MultiDataSourceJdbcResource jdbcResource;
+
+  public TxnStoreMutex(SQLGenerator sqlGenerator, MultiDataSourceJdbcResource jdbcResource) {
+    this.sqlGenerator = sqlGenerator;
+    this.jdbcResource = jdbcResource;
+  }
+
+  @Override
+  public LockHandle acquireLock(String key) throws MetaException {
+    /**
+     * The implementation here is a bit kludgey but done so that code exercised by unit tests
+     * (which run against Derby which has no support for select for update) is as similar to
+     * production code as possible.
+     * In particular, with Derby we always run in a single process with a single metastore and
+     * the absence of For Update is handled via a Semaphore.  The later would strictly speaking
+     * make the SQL statements below unnecessary (for Derby), but then they would not be tested.
+     */
+    TransactionContext context = null;
+    try {
+      jdbcResource.bindDataSource(POOL_MUTEX);
+      context = jdbcResource.getTransactionManager().getNewTransaction(PROPAGATION_REQUIRED);
+      
+      MapSqlParameterSource paramSource = new MapSqlParameterSource().addValue("key", key);
+      String sqlStmt = sqlGenerator.addForUpdateClause("SELECT \"MT_COMMENT\", \"MT_KEY2\" FROM \"AUX_TABLE\" WHERE \"MT_KEY1\" = :key");
+
+      LOG.debug("About to execute SQL: {}", sqlStmt);
+
+      Long lastUpdateTime = jdbcResource.getJdbcTemplate().query(sqlStmt, paramSource, rs -> rs.next() ? rs.getLong("MT_KEY2") : null);
+      if (lastUpdateTime == null) {
+        try {
+          jdbcResource.getJdbcTemplate().update("INSERT INTO \"AUX_TABLE\" (\"MT_KEY1\", \"MT_KEY2\") VALUES(:key, 0)", paramSource);
+          context.createSavepoint();
+        } catch (DataAccessException e) {
+          if (!jdbcResource.getDatabaseProduct().isDuplicateKeyError(e)) {
+            throw new RuntimeException("Unable to lock " + key + " due to: " + SqlRetryHandler.getMessage(e), e);
+          }
+          //if here, it means a concrurrent acquireLock() inserted the 'key'
+
+          //rollback is done for the benefit of Postgres which throws (SQLState=25P02, ErrorCode=0) if
+          //you attempt any stmt in a txn which had an error.
+          try {
+            jdbcResource.getConnection().rollback();
+          } catch (SQLException ex) {
+            throw new MetaException("Unable to lock " + key + " due to: " + SqlRetryHandler.getMessage(ex) + "; " + StringUtils.stringifyException(ex));
+          }
+        }
+        lastUpdateTime = jdbcResource.getJdbcTemplate().query(sqlStmt, paramSource, rs -> rs.next() ? rs.getLong("MT_KEY2") : null);        
+        if (lastUpdateTime ==null) {
+          throw new IllegalStateException("Unable to lock " + key + ".  Expected row in AUX_TABLE is missing.");
+        }
+      }
+      Semaphore derbySemaphore = null;
+      if (jdbcResource.getDatabaseProduct().isDERBY()) {
+        derbyKey2Lock.putIfAbsent(key, new Semaphore(1));
+        derbySemaphore = derbyKey2Lock.get(key);
+        derbySemaphore.acquire();
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("{} locked by {}", key, JavaUtils.hostname());
+      }
+      //OK, so now we have a lock
+      return new LockHandleImpl(jdbcResource, context, key, lastUpdateTime, derbySemaphore);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      if (context != null) {
+        jdbcResource.getTransactionManager().rollback(context);
+      }
+      throw new MetaException("Unable to lock " + key + " due to: " + ex.getMessage() + StringUtils.stringifyException(ex));
+    } catch (Throwable e) {
+      if (context != null) {
+        jdbcResource.getTransactionManager().rollback(context);
+      }
+      throw e;
+    } finally {
+      jdbcResource.unbindDataSource();
+    }
+  }
+
+  @Override
+  public void acquireLock(String key, LockHandle handle) throws MetaException {
+    //the idea is that this will use LockHandle.dbConn
+    throw new NotImplementedException("acquireLock(String, LockHandle) is not implemented");
+  }
+
+  public static final class LockHandleImpl implements LockHandle {
+  
+    private static final Logger LOG = LoggerFactory.getLogger(LockHandleImpl.class);
+    private static final String HOSTNAME = JavaUtils.hostname();
+  
+    private final MultiDataSourceJdbcResource jdbcResource;
+    private final TransactionContext context;
+    private final Semaphore derbySemaphore;
+    private final String key;
+    private final Long lastUpdateTime;
+  
+    public LockHandleImpl(MultiDataSourceJdbcResource jdbcResource, TransactionContext context,  String key, 
+                          Long lastUpdateTime, Semaphore derbySemaphore) {
+      assert derbySemaphore == null || derbySemaphore.availablePermits() == 0 : "Expected locked Semaphore";
+      
+      this.jdbcResource = jdbcResource;
+      this.context = context;
+      this.derbySemaphore = derbySemaphore;
+      this.key = key;
+      this.lastUpdateTime = lastUpdateTime == null ? -1L : lastUpdateTime;
+    }
+  
+    @Override
+    public void releaseLocks() {
+      try {
+        jdbcResource.bindDataSource(POOL_MUTEX);
+        jdbcResource.getTransactionManager().rollback(context);
+        if (derbySemaphore != null) {
+          derbySemaphore.release();
+        }
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("{} unlocked by {}", key, HOSTNAME);
+        }
+      } finally {
+        jdbcResource.unbindDataSource();
+      }
+    }
+  
+    @Override
+    public Long getLastUpdateTime() {
+      return lastUpdateTime;
+    }
+  
+    @Override
+    public void releaseLocks(Long timestamp) {
+      try {
+        jdbcResource.bindDataSource(POOL_MUTEX);
+        try {
+          jdbcResource.getJdbcTemplate().update("UPDATE \"AUX_TABLE\" SET \"MT_KEY2\" = :time WHERE \"MT_KEY1\"= :key",
+              new MapSqlParameterSource()
+                  .addValue("time", timestamp)
+                  .addValue("key", key));
+          jdbcResource.getTransactionManager().commit(context);
+        } catch (DataAccessException ex) {
+          LOG.warn("Unable to update MT_KEY2 value for MT_KEY1=" + key, ex);
+        }
+        
+        if (derbySemaphore != null) {
+          derbySemaphore.release();
+        }
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("{} unlocked by {}", key, HOSTNAME);
+        }
+      } finally {
+        jdbcResource.unbindDataSource();
+      }
+    }
+  
+    @Override
+    public void close() {
+      releaseLocks();
+    }
+  
+  }
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java
index 35a8ab71cc8..e7e97b5f23d 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hive.metastore.txn;
 
+import com.google.common.base.Splitter;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -28,18 +29,20 @@ import org.apache.hadoop.hive.metastore.DatabaseProduct;
 import org.apache.hadoop.hive.metastore.api.CompactionType;
 import org.apache.hadoop.hive.metastore.api.GetOpenTxnsResponse;
 import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.TableValidWriteIds;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+import org.apache.hadoop.hive.metastore.txn.entities.TxnStatus;
+import org.apache.hadoop.hive.metastore.utils.FileUtils;
 import org.apache.hadoop.hive.metastore.utils.JavaUtils;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.springframework.jdbc.core.namedparam.MapSqlParameterSource;
-import org.springframework.jdbc.core.namedparam.NamedParameterJdbcTemplate;
 
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
@@ -50,10 +53,9 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.BitSet;
 import java.util.Collection;
-import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
-import java.util.regex.Pattern;
+import java.util.concurrent.ThreadLocalRandom;
 import java.util.stream.Collectors;
 
 import static org.apache.hadoop.hive.common.AcidConstants.SOFT_DELETE_TABLE;
@@ -147,7 +149,7 @@ public class TxnUtils {
     try {
       TxnStore handler = JavaUtils.getClass(className, TxnStore.class).newInstance();
       handler.setConf(conf);
-      handler = ProxyTxnHandler.getProxy(handler, handler.getRetryHandler(), handler.getJdbcResourceHolder());
+      handler = TransactionalRetryProxy.getProxy(handler.getRetryHandler(), handler.getJdbcResourceHolder(), handler);
       return handler;
     } catch (Exception e) {
       LOG.error("Unable to instantiate raw store directly in fastpath mode", e);
@@ -410,50 +412,6 @@ public class TxnUtils {
     return ret;
   }
 
-  /**
-   * Executes the statement with an IN clause. If the number of elements or the length of the constructed statement would be
-   * too big, the IN clause will be split into multiple smaller ranges, and the statement will be executed multiple times.
-   * @param conf Hive configuration used to get the query and IN clause length limits.
-   * @param jdbcTemplate The {@link NamedParameterJdbcTemplate} instance to used for statement execution.
-   * @param query The query with the IN clause
-   * @param params A {@link MapSqlParameterSource} instance with the parameters of the query
-   * @param inClauseParamName The name of the parameter representing the content of the IN clause 
-   * @param elements A {@link List} containing the elements to put in the IN clause
-   * @param comparator A {@link Comparator} instance used to find the longest element in the list. Used to
-   *                   estimate the length of the query.
-   * @return Returns the total number of affected rows.
-   * @param <T> Type of the elements in the list.
-   */
-  public static <T> int executeStatementWithInClause(Configuration conf, NamedParameterJdbcTemplate jdbcTemplate, 
-                                                     String query, MapSqlParameterSource params, String inClauseParamName, 
-                                                     List<T> elements, Comparator<T> comparator) {
-    if (elements.size() == 0) {
-      throw new IllegalArgumentException("The elements list cannot be empty! An empty IN clause is invalid!");
-    }
-    if (!Pattern.compile("IN\\s*\\(\\s*:" + inClauseParamName + "\\s*\\)", Pattern.CASE_INSENSITIVE).matcher(query).find()) {
-      throw new IllegalArgumentException("The query must contain the IN(:" + inClauseParamName + ") clause!");      
-    }
-
-    int maxQueryLength = MetastoreConf.getIntVar(conf, ConfVars.DIRECT_SQL_MAX_QUERY_LENGTH) * 1024;
-    int batchSize = MetastoreConf.getIntVar(conf, ConfVars.DIRECT_SQL_MAX_ELEMENTS_IN_CLAUSE);
-    // The length of a single element is the string length of the longest element + 2 characters (comma, space) 
-    int elementLength = elements.stream().max(comparator).get().toString().length() + 2;
-    // estimated base query size: query size + the length of all parameters.
-    int baseQuerySize = query.length() + params.getValues().values().stream().mapToInt(s -> s.toString().length()).sum();
-    int maxElementsByLength = (maxQueryLength - baseQuerySize) / elementLength;
-
-    int inClauseMaxSize = Math.min(batchSize, maxElementsByLength);
-
-    int fromIndex = 0, totalCount = 0;
-    while (fromIndex < elements.size()) {
-      int endIndex = Math.min(elements.size(), fromIndex + inClauseMaxSize);
-      params.addValue(inClauseParamName, elements.subList(fromIndex, endIndex));
-      totalCount += jdbcTemplate.update(query, params);
-      fromIndex = endIndex;
-    }
-    return totalCount;
-  }
-
   /**
    * Compute and return the size of a query statement with the given parameters as input variables.
    *
@@ -666,4 +624,60 @@ public class TxnUtils {
   public static String nvl(String input) {
     return input != null ? " = ? " : " IS NULL ";
   }
+
+  public static String normalizePartitionCase(String s) {
+    if (s == null) {
+      return null;
+    }
+    Map<String, String> map = Splitter.on(Path.SEPARATOR).withKeyValueSeparator('=').split(s);
+    return FileUtils.makePartName(new ArrayList<>(map.keySet()), new ArrayList<>(map.values()));
+  }
+
+  @SuppressWarnings("squid:S2245")
+  public static long generateTemporaryId() {
+    return -1 * ThreadLocalRandom.current().nextLong();
+  }
+
+  public static boolean isValidTxn(long txnId) {
+    return txnId != 0;
+  }
+
+  /**
+   * Used to raise an informative error when the caller expected a txn in a particular TxnStatus
+   * but found it in some other status
+   */
+  public static void raiseTxnUnexpectedState(TxnStatus actualStatus, long txnid)
+      throws NoSuchTxnException, TxnAbortedException {
+    switch (actualStatus) {
+      case ABORTED:
+        throw new TxnAbortedException("Transaction " + JavaUtils.txnIdToString(txnid) + " already aborted");
+      case COMMITTED:
+        throw new NoSuchTxnException("Transaction " + JavaUtils.txnIdToString(txnid) + " is already committed.");
+      case UNKNOWN:
+        throw new NoSuchTxnException("No such transaction " + JavaUtils.txnIdToString(txnid));
+      case OPEN:
+        throw new NoSuchTxnException(JavaUtils.txnIdToString(txnid) + " is " + TxnStatus.OPEN);
+      default:
+        throw new IllegalArgumentException("Unknown TxnStatus " + actualStatus);
+    }
+  }
+
+  /**
+   * Checks is the passed exception, or any of the root (cause) exceptions are an instance of {@link SQLException}.
+   * Returns with the found {@link SQLException} or throws an {@link IllegalArgumentException} if no {@link SQLException}
+   * found in the chain.
+   * @param ex The exception to check
+   * @return Returns with the {@link SQLException} found in the exception chain. 
+   * @throws IllegalArgumentException Thrown if there is no {@link SQLException} in the exception chain
+   */
+  public static SQLException getSqlException(Throwable ex) throws IllegalArgumentException {
+    while (ex != null && !(ex instanceof SQLException) ) {
+      ex = ex.getCause();
+    }
+    if (ex == null) {
+      throw new IllegalArgumentException("No SQLException found in the exception chain!");
+    }
+    return (SQLException)ex;
+  }
+
 }
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionInfo.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/CompactionInfo.java
similarity index 98%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionInfo.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/CompactionInfo.java
index 40b32482cf9..ca8dff5b8f2 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionInfo.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/CompactionInfo.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn;
+package org.apache.hadoop.hive.metastore.txn.entities;
 
 import org.apache.commons.lang3.builder.ToStringBuilder;
 import org.apache.hadoop.hive.common.ValidCompactorWriteIdList;
@@ -24,6 +24,7 @@ import org.apache.hadoop.hive.metastore.api.CompactionType;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.OptionalCompactionInfoStruct;
 import org.apache.hadoop.hive.metastore.api.TableValidWriteIds;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.apache.hadoop.hive.metastore.utils.StringableMap;
 
 import java.sql.PreparedStatement;
@@ -208,7 +209,7 @@ public class CompactionInfo implements Comparable<CompactionInfo> {
    * @param rs ResultSet after call to rs.next()
    * @throws SQLException
    */
-  static CompactionInfo loadFullFromCompactionQueue(ResultSet rs) throws SQLException, MetaException {
+  public static CompactionInfo loadFullFromCompactionQueue(ResultSet rs) throws SQLException {
     CompactionInfo fullCi = new CompactionInfo();
     fullCi.id = rs.getLong(1);
     fullCi.dbname = rs.getString(2);
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionMetricsData.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/CompactionMetricsData.java
similarity index 98%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionMetricsData.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/CompactionMetricsData.java
index b56009563b4..33dd362307d 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionMetricsData.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/CompactionMetricsData.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn;
+package org.apache.hadoop.hive.metastore.txn.entities;
 
 public class CompactionMetricsData {
 
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionState.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/CompactionState.java
similarity index 81%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionState.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/CompactionState.java
index b0b2d0cd66e..fdafd3fe259 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionState.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/CompactionState.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn;
+package org.apache.hadoop.hive.metastore.txn.entities;
 
 import java.util.Arrays;
 import java.util.Map;
@@ -28,22 +28,24 @@ import static java.util.stream.Collectors.toMap;
  * These are the valid values for Compaction states.
  */
 public enum CompactionState {
-  INITIATED('i'),
-  WORKING('w'),
-  READY_FOR_CLEANING('r'),
-  FAILED('f'),
-  SUCCEEDED('s'),
-  DID_NOT_INITIATE('a'),
-  REFUSED('c'),
-  ABORTED('x');
+  INITIATED('i', "initiated"),
+  WORKING('w', "working"),
+  READY_FOR_CLEANING('r', "ready for cleaning"),
+  FAILED('f', "failed"),
+  SUCCEEDED('s', "succeeded"),
+  DID_NOT_INITIATE('a', "did not initiate"),
+  REFUSED('c', "refused"),
+  ABORTED('x', "aborted");
 
+  private final String message;
   private final char sqlConst;
 
   private static final Map<String, CompactionState> LOOKUP =
       Arrays.stream(CompactionState.values()).collect(toMap(CompactionState::getSqlConst, identity()));
 
-  CompactionState(char sqlConst) {
+  CompactionState(char sqlConst, String message) {
     this.sqlConst = sqlConst;
+    this.message = message;
   }
 
   @Override
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/LockInfo.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/LockInfo.java
new file mode 100644
index 00000000000..a7550b6fed7
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/LockInfo.java
@@ -0,0 +1,147 @@
+/*
+ * 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.metastore.txn.entities;
+
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement;
+import org.apache.hadoop.hive.metastore.utils.JavaUtils;
+import org.apache.hadoop.hive.metastore.utils.LockTypeUtil;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+
+public class LockInfo {
+
+  public static final char LOCK_ACQUIRED = 'a';
+  public static final char LOCK_WAITING = 'w';
+  
+
+  private final long extLockId;
+  private final long intLockId;
+  //0 means there is no transaction, i.e. it a select statement which is not part of
+  //explicit transaction or a IUD statement that is not writing to ACID table
+  private final long txnId;
+  private final String db;
+  private final String table;
+  private final String partition;
+  private final LockState state;
+  private final LockType type;
+
+  // Assumes the result set is set to a valid row
+  public LockInfo(ResultSet rs) throws SQLException, MetaException {
+    extLockId = rs.getLong("HL_LOCK_EXT_ID"); // can't be null
+    intLockId = rs.getLong("HL_LOCK_INT_ID"); // can't be null
+    db = rs.getString("HL_DB"); // can't be null
+    String t = rs.getString("HL_TABLE");
+    table = (rs.wasNull() ? null : t);
+    String p = rs.getString("HL_PARTITION");
+    partition = (rs.wasNull() ? null : p);
+    switch (rs.getString("HL_LOCK_STATE").charAt(0)) {
+      case LOCK_WAITING: state = LockState.WAITING; break;
+      case LOCK_ACQUIRED: state = LockState.ACQUIRED; break;
+      default:
+        throw new MetaException("Unknown lock state " + rs.getString("HL_LOCK_STATE").charAt(0));
+    }
+    char lockChar = rs.getString("HL_LOCK_TYPE").charAt(0);
+    type = LockTypeUtil.getLockTypeFromEncoding(lockChar)
+        .orElseThrow(() -> new MetaException("Unknown lock type: " + lockChar));
+    txnId = rs.getLong("HL_TXNID"); //returns 0 if value is NULL
+  }
+
+  public LockInfo(ShowLocksResponseElement e) {
+    extLockId = e.getLockid();
+    intLockId = e.getLockIdInternal();
+    txnId = e.getTxnid();
+    db = e.getDbname();
+    table = e.getTablename();
+    partition = e.getPartname();
+    state = e.getState();
+    type = e.getType();
+  }
+
+  public long getExtLockId() {
+    return extLockId;
+  }
+
+  public long getIntLockId() {
+    return intLockId;
+  }
+
+  public long getTxnId() {
+    return txnId;
+  }
+
+  public String getDb() {
+    return db;
+  }
+
+  public String getTable() {
+    return table;
+  }
+
+  public String getPartition() {
+    return partition;
+  }
+
+  public LockState getState() {
+    return state;
+  }
+
+  public LockType getType() {
+    return type;
+  }
+
+  public boolean equals(Object other) {
+    if (!(other instanceof LockInfo)) return false;
+    LockInfo o = (LockInfo)other;
+    // Lock ids are unique across the system.
+    return extLockId == o.extLockId && intLockId == o.intLockId;
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder()
+        .append(intLockId)
+        .append(extLockId)
+        .append(txnId)
+        .append(db)
+        .build();
+  }
+
+  @Override
+  public String toString() {
+    return JavaUtils.lockIdToString(extLockId) + " intLockId:" +
+        intLockId + " " + JavaUtils.txnIdToString(txnId)
+        + " db:" + db + " table:" + table + " partition:" +
+        partition + " state:" + (state == null ? "null" : state.toString())
+        + " type:" + (type == null ? "null" : type.toString());
+  }
+  private boolean isDbLock() {
+    return db != null && table == null && partition == null;
+  }
+  private boolean isTableLock() {
+    return db != null && table != null && partition == null;
+  }
+  private boolean isPartitionLock() {
+    return !(isDbLock() || isTableLock());
+  }
+  
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/MetricsInfo.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/MetricsInfo.java
similarity index 98%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/MetricsInfo.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/MetricsInfo.java
index 8fc7d8fed0c..b76a5d1070c 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/MetricsInfo.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/MetricsInfo.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hive.metastore.txn;
+package org.apache.hadoop.hive.metastore.txn.entities;
 
 import java.util.Set;
 
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/OpenTxn.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/OpenTxn.java
similarity index 84%
rename from standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/OpenTxn.java
rename to standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/entities/OpenTxn.java
index 8ef5fa167f3..60044977b2f 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/OpenTxn.java
... 9168 lines suppressed ...