You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by dk...@apache.org on 2020/10/21 07:57:13 UTC

[hive] branch master updated: Revert "Make sure transactions get cleaned if they are aborted before addPartitions is called (Denys Kuzmenko, reviewed by Vipin Vishvkarma, Peter Varga, Karen Coppage)"

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

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


The following commit(s) were added to refs/heads/master by this push:
     new da7d42b  Revert "Make sure transactions get cleaned if they are aborted before addPartitions is called (Denys Kuzmenko, reviewed by Vipin Vishvkarma, Peter Varga, Karen Coppage)"
da7d42b is described below

commit da7d42b338ba79a51eb358696d86adff7634828e
Author: Denys Kuzmenko <dk...@cloudera.com>
AuthorDate: Wed Oct 21 09:56:33 2020 +0200

    Revert "Make sure transactions get cleaned if they are aborted before addPartitions is called (Denys Kuzmenko, reviewed by Vipin Vishvkarma, Peter Varga, Karen Coppage)"
    
    This reverts commit b7f39651c5cd74860b764490834ebfbcdf31978b.
---
 .../java/org/apache/hadoop/hive/conf/HiveConf.java |   1 -
 .../hive/ql/txn/compactor/TestCompactor.java       | 322 -----------
 .../apache/hadoop/hive/ql/DriverTxnHandler.java    |   2 +-
 .../org/apache/hadoop/hive/ql/io/AcidUtils.java    |  18 +-
 .../hadoop/hive/ql/lockmgr/DbTxnManager.java       |   4 +-
 .../hadoop/hive/ql/lockmgr/DummyTxnManager.java    |   2 +-
 .../hadoop/hive/ql/lockmgr/HiveTxnManager.java     |   2 +-
 .../org/apache/hadoop/hive/ql/metadata/Hive.java   |   4 +-
 .../hadoop/hive/ql/txn/compactor/Cleaner.java      |   4 +-
 .../hadoop/hive/ql/txn/compactor/Initiator.java    |  11 +-
 .../hadoop/hive/ql/txn/compactor/Worker.java       |  13 +-
 .../apache/hadoop/hive/ql/TestTxnCommands2.java    | 617 +--------------------
 .../hadoop/hive/ql/lockmgr/TestDbTxnManager2.java  |   2 +-
 .../src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp |  92 +--
 .../src/gen/thrift/gen-cpp/ThriftHiveMetastore.h   |  90 +--
 .../ThriftHiveMetastore_server.skeleton.cpp        |   4 +-
 .../hive/metastore/api/ThriftHiveMetastore.java    | 184 +++---
 .../metastore/ThriftHiveMetastoreClient.php        |  22 +-
 .../gen-php/metastore/ThriftHiveMetastoreIf.php    |   2 +-
 ...eMetastore_get_latest_txn_in_conflict_args.php} |   6 +-
 ...etastore_get_latest_txn_in_conflict_result.php} |   6 +-
 .../hive_metastore/ThriftHiveMetastore-remote      |   8 +-
 .../gen-py/hive_metastore/ThriftHiveMetastore.py   |  48 +-
 .../src/gen/thrift/gen-rb/thrift_hive_metastore.rb |  30 +-
 .../hadoop/hive/metastore/HiveMetaStoreClient.java |   4 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java    |   2 +-
 .../src/main/thrift/hive_metastore.thrift          |   2 +-
 .../hadoop/hive/metastore/HiveMetaStore.java       |   4 +-
 .../hadoop/hive/metastore/txn/CompactionInfo.java  |   5 +-
 .../hive/metastore/txn/CompactionTxnHandler.java   | 121 ++--
 .../hadoop/hive/metastore/txn/TxnHandler.java      |  93 ++--
 .../apache/hadoop/hive/metastore/txn/TxnStore.java |   2 +-
 .../metastore/HiveMetaStoreClientPreCatalog.java   |   2 +-
 33 files changed, 429 insertions(+), 1300 deletions(-)

diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index b8eaefe..dc333ca 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2103,7 +2103,6 @@ public class HiveConf extends Configuration {
     HIVETESTCURRENTTIMESTAMP("hive.test.currenttimestamp", null, "current timestamp for test", false),
     HIVETESTMODEROLLBACKTXN("hive.test.rollbacktxn", false, "For testing only.  Will mark every ACID transaction aborted", false),
     HIVETESTMODEFAILCOMPACTION("hive.test.fail.compaction", false, "For testing only.  Will cause CompactorMR to fail.", false),
-    HIVETESTMODEFAILLOADDYNAMICPARTITION("hive.test.fail.load.dynamic.partition", false, "For testing only.  Will cause loadDynamicPartition to fail.", false),
     HIVETESTMODEFAILHEARTBEATER("hive.test.fail.heartbeater", false, "For testing only.  Will cause Heartbeater to fail.", false),
     TESTMODE_BUCKET_CODEC_VERSION("hive.test.bucketcodec.version", 1,
       "For testing only.  Will make ACID subsystem write RecordIdentifier.bucketId in specified\n" +
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 f9bf00b..c7bfbfc 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
@@ -35,7 +35,6 @@ import java.util.Map;
 import java.util.Random;
 import java.util.SortedSet;
 import java.util.TreeSet;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -44,7 +43,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
-import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.Constants;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -132,7 +130,6 @@ public class TestCompactor {
     hiveConf.setVar(HiveConf.ConfVars.METASTOREWAREHOUSE, TEST_WAREHOUSE_DIR);
     hiveConf.setVar(HiveConf.ConfVars.HIVEINPUTFORMAT, HiveInputFormat.class.getName());
     hiveConf.setBoolVar(HiveConf.ConfVars.HIVESTATSAUTOGATHER, false);
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVEOPTIMIZEMETADATAQUERIES, false);
 
     TxnDbUtil.setConfValues(hiveConf);
     TxnDbUtil.cleanDb(hiveConf);
@@ -140,7 +137,6 @@ public class TestCompactor {
 
     conf = hiveConf;
     HiveConf.setBoolVar(conf, ConfVars.HIVE_MM_ALLOW_ORIGINALS, true);
-    HiveConf.setTimeVar(conf, ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD, 0, TimeUnit.MILLISECONDS);
     msClient = new HiveMetaStoreClient(conf);
     driver = DriverFactory.newDriver(hiveConf);
     SessionState.start(new CliSessionState(hiveConf));
@@ -857,324 +853,6 @@ public class TestCompactor {
             Lists.newArrayList(5, 6), 1);
   }
 
-  @Test
-  public void testCleanAbortCompactAfter2ndCommitAbort() throws Exception {
-    String dbName = "default";
-    String tblName = "cws";
-
-    HiveStreamingConnection connection = prepareTableAndConnection(dbName, tblName, 2);
-
-    connection.beginTransaction();
-    connection.write("1,1".getBytes());
-    connection.write("2,2".getBytes());
-    connection.commitTransaction();
-
-    connection.beginTransaction();
-    connection.write("2,3".getBytes());
-    connection.write("3,3".getBytes());
-    connection.abortTransaction();
-
-    assertAndCompactCleanAbort(dbName, tblName, false);
-    connection.close();
-  }
-
-  @Test
-  public void testCleanAbortCompactAfter1stCommitAbort() throws Exception {
-    String dbName = "default";
-    String tblName = "cws";
-
-    HiveStreamingConnection connection = prepareTableAndConnection(dbName, tblName, 2);
-
-    connection.beginTransaction();
-    connection.write("1,1".getBytes());
-    connection.write("2,2".getBytes());
-    connection.abortTransaction();
-
-    connection.beginTransaction();
-    connection.write("2,3".getBytes());
-    connection.write("3,3".getBytes());
-    connection.commitTransaction();
-
-    assertAndCompactCleanAbort(dbName, tblName, false);
-    connection.close();
-  }
-
-  @Test
-  public void testCleanAbortCompactAfterAbortTwoPartitions() throws Exception {
-    String dbName = "default";
-    String tblName = "cws";
-
-    HiveStreamingConnection connection1 = prepareTableTwoPartitionsAndConnection(dbName, tblName, 1);
-    HiveStreamingConnection connection2 = prepareTableTwoPartitionsAndConnection(dbName, tblName, 1);
-
-    // to skip optimization introduced in HIVE-22122
-    executeStatementOnDriver("truncate " + tblName, driver);
-
-    connection1.beginTransaction();
-    connection1.write("1,1".getBytes());
-    connection1.write("2,2".getBytes());
-    connection1.abortTransaction();
-
-    connection2.beginTransaction();
-    connection2.write("1,3".getBytes());
-    connection2.write("2,3".getBytes());
-    connection2.write("3,3".getBytes());
-    connection2.abortTransaction();
-
-    assertAndCompactCleanAbort(dbName, tblName, true);
-
-    connection1.close();
-    connection2.close();
-  }
-
-  @Test
-  public void testCleanAbortCompactAfterAbort() throws Exception {
-    String dbName = "default";
-    String tblName = "cws";
-
-    // Create three folders with two different transactions
-    HiveStreamingConnection connection1 = prepareTableAndConnection(dbName, tblName, 1);
-    HiveStreamingConnection connection2 = prepareTableAndConnection(dbName, tblName, 1);
-
-    // to skip optimization introduced in HIVE-22122
-    executeStatementOnDriver("truncate " + tblName, driver);
-
-    connection1.beginTransaction();
-    connection1.write("1,1".getBytes());
-    connection1.write("2,2".getBytes());
-    connection1.abortTransaction();
-
-    connection2.beginTransaction();
-    connection2.write("1,3".getBytes());
-    connection2.write("2,3".getBytes());
-    connection2.write("3,3".getBytes());
-    connection2.abortTransaction();
-
-    assertAndCompactCleanAbort(dbName, tblName, true);
-
-    connection1.close();
-    connection2.close();
-  }
-
-  private void assertAndCompactCleanAbort(String dbName, String tblName, boolean allAborted) throws Exception {
-    IMetaStoreClient msClient = new HiveMetaStoreClient(conf);
-    TxnStore txnHandler = TxnUtils.getTxnStore(conf);
-    Table table = msClient.getTable(dbName, tblName);
-    FileSystem fs = FileSystem.get(conf);
-    FileStatus[] stat =
-        fs.listStatus(new Path(table.getSd().getLocation()));
-    if (3 != stat.length) {
-      Assert.fail("Expecting three directories corresponding to three partitions, FileStatus[] stat " + Arrays.toString(stat));
-    }
-
-    int count = TxnDbUtil.countQueryAgent(conf, "select count(*) from TXN_COMPONENTS where TC_OPERATION_TYPE='i'");
-    // We should have two rows corresponding to the two aborted transactions
-    Assert.assertEquals(TxnDbUtil.queryToString(conf, "select * from TXN_COMPONENTS"), allAborted ? 2 : 1, count);
-
-    runInitiator(conf);
-    count = TxnDbUtil.countQueryAgent(conf, "select count(*) from COMPACTION_QUEUE");
-    // Only one job is added to the queue per table. This job corresponds to all the entries for a particular table
-    // with rows in TXN_COMPONENTS
-    Assert.assertEquals(TxnDbUtil.queryToString(conf, "select * from COMPACTION_QUEUE"), 1, count);
-    runWorker(conf);
-
-    ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
-    Assert.assertEquals(1, rsp.getCompacts().size());
-    Assert.assertEquals(TxnStore.CLEANING_RESPONSE, rsp.getCompacts().get(0).getState());
-    Assert.assertEquals("cws", rsp.getCompacts().get(0).getTablename());
-    Assert.assertEquals(CompactionType.MINOR, rsp.getCompacts().get(0).getType());
-
-    runCleaner(conf);
-
-    // After the cleaner runs TXN_COMPONENTS and COMPACTION_QUEUE should have zero rows, also the folders should have been deleted.
-    count = TxnDbUtil.countQueryAgent(conf, "select count(*) from TXN_COMPONENTS");
-    Assert.assertEquals(TxnDbUtil.queryToString(conf, "select * from TXN_COMPONENTS"), 0, count);
-
-    count = TxnDbUtil.countQueryAgent(conf, "select count(*) from COMPACTION_QUEUE");
-    Assert.assertEquals(TxnDbUtil.queryToString(conf, "select * from COMPACTION_QUEUE"), 0, count);
-
-    RemoteIterator it =
-        fs.listFiles(new Path(table.getSd().getLocation()), true);
-    if (it.hasNext() && allAborted) {
-      Assert.fail("Expected cleaner to drop aborted delta & base directories, FileStatus[] stat " + Arrays.toString(stat));
-    }
-
-    rsp = txnHandler.showCompact(new ShowCompactRequest());
-    Assert.assertEquals(1, rsp.getCompacts().size());
-    Assert.assertEquals(TxnStore.SUCCEEDED_RESPONSE, rsp.getCompacts().get(0).getState());
-    Assert.assertEquals("cws", rsp.getCompacts().get(0).getTablename());
-    Assert.assertEquals(CompactionType.MINOR, rsp.getCompacts().get(0).getType());
-  }
-
-  @Test
-  public void testCleanAbortCompactSeveralTables() throws Exception {
-    String dbName = "default";
-    String tblName1 = "cws1";
-    String tblName2 = "cws2";
-
-    HiveStreamingConnection connection1 = prepareTableAndConnection(dbName, tblName1, 1);
-    HiveStreamingConnection connection2 = prepareTableAndConnection(dbName, tblName2, 1);
-
-    connection1.beginTransaction();
-    connection1.write("1,1".getBytes());
-    connection1.write("2,2".getBytes());
-    connection1.abortTransaction();
-
-    connection2.beginTransaction();
-    connection2.write("1,1".getBytes());
-    connection2.write("2,2".getBytes());
-    connection2.abortTransaction();
-
-    IMetaStoreClient msClient = new HiveMetaStoreClient(conf);
-    FileSystem fs = FileSystem.get(conf);
-    Table table1 = msClient.getTable(dbName, tblName1);
-    FileStatus[] stat =
-        fs.listStatus(new Path(table1.getSd().getLocation()));
-    if (2 != stat.length) {
-      Assert.fail("Expecting two directories corresponding to two partitions, FileStatus[] stat " + Arrays.toString(stat));
-    }
-    Table table2 = msClient.getTable(dbName, tblName2);
-    stat = fs.listStatus(new Path(table2.getSd().getLocation()));
-    if (2 != stat.length) {
-      Assert.fail("Expecting two directories corresponding to two partitions, FileStatus[] stat " + Arrays.toString(stat));
-    }
-
-    int count = TxnDbUtil.countQueryAgent(conf, "select count(*) from TXN_COMPONENTS where TC_OPERATION_TYPE='i'");
-    // We should have two rows corresponding to the two aborted transactions
-    Assert.assertEquals(TxnDbUtil.queryToString(conf, "select * from TXN_COMPONENTS"), 2, count);
-
-    runInitiator(conf);
-    count = TxnDbUtil.countQueryAgent(conf, "select count(*) from COMPACTION_QUEUE");
-    // Only one job is added to the queue per table. This job corresponds to all the entries for a particular table
-    // with rows in TXN_COMPONENTS
-    Assert.assertEquals(TxnDbUtil.queryToString(conf, "select * from COMPACTION_QUEUE"), 2, count);
-
-    runWorker(conf);
-    runWorker(conf);
-
-    runCleaner(conf);
-    // After the cleaner runs TXN_COMPONENTS and COMPACTION_QUEUE should have zero rows, also the folders should have been deleted.
-    count = TxnDbUtil.countQueryAgent(conf, "select count(*) from TXN_COMPONENTS");
-    Assert.assertEquals(TxnDbUtil.queryToString(conf, "select * from TXN_COMPONENTS"), 0, count);
-
-    count = TxnDbUtil.countQueryAgent(conf, "select count(*) from COMPACTION_QUEUE");
-    Assert.assertEquals(TxnDbUtil.queryToString(conf, "select * from COMPACTION_QUEUE"), 0, count);
-
-    RemoteIterator it =
-        fs.listFiles(new Path(table1.getSd().getLocation()), true);
-    if (it.hasNext()) {
-      Assert.fail("Expected cleaner to drop aborted delta & base directories, FileStatus[] stat " + Arrays.toString(stat));
-    }
-
-    connection1.close();
-    connection2.close();
-  }
-
-  @Test
-  public void testCleanAbortCorrectlyCleaned() throws Exception {
-    // Test that at commit the tables are cleaned properly
-    String dbName = "default";
-    String tblName = "cws";
-    HiveStreamingConnection connection = prepareTableAndConnection(dbName, tblName, 1);
-    connection.beginTransaction();
-    connection.write("1,1".getBytes());
-    connection.write("2,2".getBytes());
-
-    int count = TxnDbUtil.countQueryAgent(conf, "select count(*) from TXN_COMPONENTS where TC_OPERATION_TYPE='i'");
-    // We should have 1 row corresponding to the aborted transaction
-    Assert.assertEquals(TxnDbUtil.queryToString(conf, "select * from TXN_COMPONENTS"), 1, count);
-
-    connection.commitTransaction();
-
-    // After commit the row should have been deleted
-    count = TxnDbUtil.countQueryAgent(conf, "select count(*) from TXN_COMPONENTS where TC_OPERATION_TYPE='i'");
-    Assert.assertEquals(TxnDbUtil.queryToString(conf, "select * from TXN_COMPONENTS"), 0, count);
-  }
-
-  @Test
-  public void testCleanAbortAndMinorCompact() throws Exception {
-    String dbName = "default";
-    String tblName = "cws";
-
-    HiveStreamingConnection connection = prepareTableAndConnection(dbName, tblName, 1);
-
-    connection.beginTransaction();
-    connection.write("1,1".getBytes());
-    connection.write("2,2".getBytes());
-    connection.abortTransaction();
-
-    executeStatementOnDriver("insert into " + tblName + " partition (a) values (1, '1')", driver);
-    executeStatementOnDriver("delete from " + tblName + " where b=1", driver);
-
-    conf.setIntVar(HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_NUM_THRESHOLD, 0);
-    runInitiator(conf);
-
-    int count = TxnDbUtil.countQueryAgent(conf, "select count(*) from COMPACTION_QUEUE");
-    Assert.assertEquals(TxnDbUtil.queryToString(conf, "select * from COMPACTION_QUEUE"), 2, count);
-
-    runWorker(conf);
-    runWorker(conf);
-
-    // Cleaning should happen in threads concurrently for the minor compaction and the clean abort one.
-    runCleaner(conf);
-
-    count = TxnDbUtil.countQueryAgent(conf, "select count(*) from COMPACTION_QUEUE");
-    Assert.assertEquals(TxnDbUtil.queryToString(conf, "select * from COMPACTION_QUEUE"), 0, count);
-
-    count = TxnDbUtil.countQueryAgent(conf, "select count(*) from TXN_COMPONENTS");
-    Assert.assertEquals(TxnDbUtil.queryToString(conf, "select * from TXN_COMPONENTS"), 0, count);
-
-  }
-
-  private HiveStreamingConnection prepareTableAndConnection(String dbName, String tblName, int batchSize) throws Exception {
-    String agentInfo = "UT_" + Thread.currentThread().getName();
-
-    executeStatementOnDriver("drop table if exists " + tblName, driver);
-    executeStatementOnDriver("CREATE TABLE " + tblName + "(b STRING) " +
-        " PARTITIONED BY (a INT)" + //currently ACID requires table to be bucketed
-        " STORED AS ORC  TBLPROPERTIES ('transactional'='true')", driver);
-
-    StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder()
-        .withFieldDelimiter(',')
-        .build();
-
-    // Create three folders with two different transactions
-    return HiveStreamingConnection.newBuilder()
-        .withDatabase(dbName)
-        .withTable(tblName)
-        .withAgentInfo(agentInfo)
-        .withHiveConf(conf)
-        .withRecordWriter(writer)
-        .withStreamingOptimizations(true)
-        // Transaction size has to be one or exception should happen.
-        .withTransactionBatchSize(batchSize)
-        .connect();
-  }
-
-  private HiveStreamingConnection prepareTableTwoPartitionsAndConnection(String dbName, String tblName, int batchSize) throws Exception {
-    String agentInfo = "UT_" + Thread.currentThread().getName();
-
-    executeStatementOnDriver("drop table if exists " + tblName, driver);
-    executeStatementOnDriver("CREATE TABLE " + tblName + "(c STRING) " +
-        " PARTITIONED BY (a INT, b INT)" + //currently ACID requires table to be bucketed
-        " STORED AS ORC  TBLPROPERTIES ('transactional'='true')", driver);
-
-    StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder()
-        .withFieldDelimiter(',')
-        .build();
-
-    // Create three folders with two different transactions
-    return HiveStreamingConnection.newBuilder()
-        .withDatabase(dbName)
-        .withTable(tblName)
-        .withAgentInfo(agentInfo)
-        .withHiveConf(conf)
-        .withRecordWriter(writer)
-        .withStreamingOptimizations(true)
-        // Transaction size has to be one or exception should happen.
-        .withTransactionBatchSize(batchSize)
-        .connect();
-  }
 
   @Test
   public void mmTable() throws Exception {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/DriverTxnHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/DriverTxnHandler.java
index 1336b92..b14d824 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/DriverTxnHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/DriverTxnHandler.java
@@ -403,7 +403,7 @@ class DriverTxnHandler {
     }
 
     // 4) Check if there is conflict
-    long txnId = driverContext.getTxnManager().getLatestTxnIdInConflict();
+    long txnId = driverContext.getTxnManager().getLatestTxnInConflict();
     if (txnId <= 0) {
       return true;
     }
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 4153f01..c6ddf8b 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
@@ -55,14 +55,7 @@ import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.hive.common.AcidConstants;
-import org.apache.hadoop.hive.common.AcidMetaDataFile;
-import org.apache.hadoop.hive.common.ValidTxnList;
-import org.apache.hadoop.hive.common.ValidReadTxnList;
-import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
-import org.apache.hadoop.hive.common.ValidTxnWriteIdList;
-import org.apache.hadoop.hive.common.ValidWriteIdList;
-import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.common.*;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.LockComponentBuilder;
@@ -1374,7 +1367,7 @@ public class AcidUtils {
       original.clear();
       originalDirectories.clear();
     } else {
-      // Okay, we're going to need these originals.
+      // Okay, we're going to need these originals.  
       // Recurse through them and figure out what we really need.
       // If we already have the original list, do nothing
       // If childrenWithId != null, we would have already populated "original"
@@ -1531,7 +1524,7 @@ public class AcidUtils {
   }
 
   /**
-   * DFS dir listing.
+   * DFS dir listing. 
    * Captures a dir and the corresponding list of files it contains,
    * with additional properties about the dir (like isBase etc)
    *
@@ -1818,11 +1811,6 @@ public class AcidUtils {
       bestBase.oldestBase = baseDir;
       bestBase.oldestBaseWriteId = writeId;
     }
-    // Handle aborted IOW base.
-    if (writeIdList.isWriteIdAborted(writeId) && !isCompactedBase(parsedBase, fs, dirSnapshot)) {
-      aborted.add(baseDir);
-      return;
-    }
     if (bestBase.basePath == null) {
       if (isValidBase(parsedBase, writeIdList, fs, dirSnapshot)) {
         bestBase.basePath = baseDir;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
index be7a704..49293db 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
@@ -1006,9 +1006,9 @@ public final class DbTxnManager extends HiveTxnManagerImpl {
   }
 
   @Override
-  public long getLatestTxnIdInConflict() throws LockException {
+  public long getLatestTxnInConflict() throws LockException {
     try {
-      return getMS().getLatestTxnIdInConflict(txnId);
+      return getMS().getLatestTxnInConflict(txnId);
     } catch (TException e) {
       throw new LockException(e);
     }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java
index 0668520..2d1c14e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java
@@ -89,7 +89,7 @@ class DummyTxnManager extends HiveTxnManagerImpl {
   }
 
   @Override
-  public long getLatestTxnIdInConflict() throws LockException {
+  public long getLatestTxnInConflict() throws LockException {
     return 0;
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManager.java
index 5b4fc10..5e1544a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManager.java
@@ -346,5 +346,5 @@ public interface HiveTxnManager {
   LockResponse acquireMaterializationRebuildLock(String dbName, String tableName, long txnId)
       throws LockException;
 
- long getLatestTxnIdInConflict() throws LockException;
+ long getLatestTxnInConflict() throws LockException;
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
index bfd5789..9677457 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
@@ -2952,9 +2952,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
       LOG.debug("Cancelling " + futures.size() + " dynamic loading tasks");
       executor.shutdownNow();
     }
-    if (HiveConf.getBoolVar(conf, ConfVars.HIVE_IN_TEST) && HiveConf.getBoolVar(conf, ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION)) {
-      throw new HiveException(HiveConf.ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION.name() + "=true");
-    }
+
     try {
       if (isTxnTable) {
         List<String> partNames =
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
index ae09088..bde4292 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
@@ -97,9 +97,9 @@ public class Cleaner extends MetaStoreCompactorThread {
           long minOpenTxnId = txnHandler.findMinOpenTxnIdForCleaner();
           LOG.info("Cleaning based on min open txn id: " + minOpenTxnId);
           List<CompletableFuture> cleanerList = new ArrayList<>();
-          for (CompactionInfo compactionInfo : txnHandler.findReadyToClean()) {
+          for(CompactionInfo compactionInfo : txnHandler.findReadyToClean()) {
             cleanerList.add(CompletableFuture.runAsync(CompactorUtil.ThrowingRunnable.unchecked(() ->
-                  clean(compactionInfo, minOpenTxnId)), cleanerExecutor));
+                    clean(compactionInfo, minOpenTxnId)), cleanerExecutor));
           }
           CompletableFuture.allOf(cleanerList.toArray(new CompletableFuture[0])).join();
         } catch (Throwable t) {
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 ee2f4e7..c8f53c5 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
@@ -313,7 +313,7 @@ public class Initiator extends MetaStoreCompactorThread {
 
   private CompactionType determineCompactionType(CompactionInfo ci, ValidWriteIdList writeIds,
                                                  StorageDescriptor sd, Map<String, String> tblproperties)
-      throws IOException {
+      throws IOException, InterruptedException {
 
     boolean noBase = false;
     Path location = new Path(sd.getLocation());
@@ -439,10 +439,11 @@ public class Initiator extends MetaStoreCompactorThread {
     return noAutoCompact != null && noAutoCompact.equalsIgnoreCase("true");
   }
 
-  // Check if it's a dynamic partitioning case. If so, do not initiate compaction for streaming ingest, only for aborts.
-  private static boolean isDynPartIngest(Table t, CompactionInfo ci){
+  // Check to see if this is a table level request on a partitioned table.  If so,
+  // then it's a dynamic partitioning case and we shouldn't check the table itself.
+  private static boolean checkDynPartitioning(Table t, CompactionInfo ci){
     if (t.getPartitionKeys() != null && t.getPartitionKeys().size() > 0 &&
-            ci.partName  == null && !ci.hasOldAbort) {
+            ci.partName  == null) {
       LOG.info("Skipping entry for " + ci.getFullTableName() + " as it is from dynamic" +
               " partitioning");
       return  true;
@@ -481,7 +482,7 @@ public class Initiator extends MetaStoreCompactorThread {
             "=true so we will not compact it.");
         return false;
       }
-      if (isDynPartIngest(t, ci)) {
+      if (checkDynPartitioning(t, ci)) {
         return false;
       }
 
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 d57b28a..572d003 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
@@ -366,7 +366,7 @@ public class Worker extends RemoteCompactorThread implements MetaStoreThread {
    * @return true if cleaning is needed
    */
   public static boolean needsCleaning(AcidUtils.Directory dir, StorageDescriptor sd) {
-    int numObsoleteDirs = dir.getObsolete().size() + dir.getAbortedDirectories().size();
+    int numObsoleteDirs = dir.getObsolete().size();
     boolean needsJustCleaning = numObsoleteDirs > 0;
     if (needsJustCleaning) {
       LOG.info("{} obsolete directories in {} found; marked for cleaning.", numObsoleteDirs,
@@ -508,10 +508,6 @@ public class Worker extends RemoteCompactorThread implements MetaStoreThread {
       jobName.append(ci.getFullPartitionName());
 
       // Don't start compaction or cleaning if not necessary
-      if (isDynPartAbort(t, ci)) {
-        msc.markCompacted(CompactionInfo.compactionInfoToStruct(ci));
-        return false;
-      }
       AcidUtils.Directory dir = AcidUtils.getAcidState(null, new Path(sd.getLocation()), conf,
           tblValidWriteIds, Ref.from(false), true);
       if (!isEnoughToCompact(ci.isMajorCompaction(), dir, sd)) {
@@ -607,9 +603,4 @@ public class Worker extends RemoteCompactorThread implements MetaStoreThread {
       throw new InterruptedException("Compaction execution is interrupted");
     }
   }
-
-  private static boolean isDynPartAbort(Table t, CompactionInfo ci) {
-    return t.getPartitionKeys() != null && t.getPartitionKeys().size() > 0
-        && ci.partName == null;
-  }
-}
\ No newline at end of file
+}
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
index be44951..fcaea79 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
@@ -149,7 +149,6 @@ public class TestTxnCommands2 {
     //TestTxnCommands2WithSplitUpdateAndVectorization has the vectorized version
     //of these tests.
     hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, false);
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVEOPTIMIZEMETADATAQUERIES, false);
 
     TxnDbUtil.setConfValues(hiveConf);
     TxnDbUtil.prepDb(hiveConf);
@@ -2129,601 +2128,24 @@ public class TestTxnCommands2 {
             0, TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from TXN_TO_WRITE_ID"));
   }
 
-  @Test
-  public void testMmTableAbortWithCompaction() throws Exception {
-    // 1. Insert some rows into MM table
-    runStatementOnDriver("insert into " + Table.MMTBL + "(a,b) values(1,2)");
-    // There should be 1 delta directory
-    int [][] resultData1 =  new int[][] {{1,2}};
-    verifyDeltaDirAndResult(1, Table.MMTBL.toString(), "", resultData1);
-    List<String> r1 = runStatementOnDriver("select count(*) from " + Table.MMTBL);
-    Assert.assertEquals("1", r1.get(0));
-
-    // 2. Let a transaction be aborted
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true);
-    runStatementOnDriver("insert into " + Table.MMTBL + "(a,b) values(3,4)");
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, false);
-    // There should be 1 delta and 1 base directory. The base one is the aborted one.
-    verifyDeltaDirAndResult(2, Table.MMTBL.toString(), "", resultData1);
-
-    r1 = runStatementOnDriver("select count(*) from " + Table.MMTBL);
-    Assert.assertEquals("1", r1.get(0));
-
-    // Verify query result
-    int [][] resultData2 = new int[][] {{1,2}, {5,6}};
-
-    runStatementOnDriver("insert into " + Table.MMTBL + "(a,b) values(5,6)");
-    verifyDeltaDirAndResult(3, Table.MMTBL.toString(), "", resultData2);
-    r1 = runStatementOnDriver("select count(*) from " + Table.MMTBL);
-    Assert.assertEquals("2", r1.get(0));
-
-    // 4. Perform a MINOR compaction, expectation is it should remove aborted base dir
-    runStatementOnDriver("alter table "+ Table.MMTBL + " compact 'MINOR'");
-    // The worker should remove the subdir for aborted transaction
-    runWorker(hiveConf);
-    verifyDeltaDirAndResult(3, Table.MMTBL.toString(), "", resultData2);
-    verifyBaseDir(0, Table.MMTBL.toString(), "");
-    // 5. Run Cleaner. Shouldn't impact anything.
-    runCleaner(hiveConf);
-    // 6. Run initiator remove aborted entry from TXNS table
-    runInitiator(hiveConf);
-
-    // Verify query result
-    List<String> rs = runStatementOnDriver("select a,b from " + Table.MMTBL + " order by a");
-    Assert.assertEquals(stringifyValues(resultData2), rs);
-
-    int [][] resultData3 = new int[][] {{1,2}, {5,6}, {7,8}};
-    // 7. add few more rows
-    runStatementOnDriver("insert into " + Table.MMTBL + "(a,b) values(7,8)");
-    // 8. add one more aborted delta
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true);
-    runStatementOnDriver("insert into " + Table.MMTBL + "(a,b) values(9,10)");
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, false);
-
-    // 9. Perform a MAJOR compaction, expectation is it should remove aborted base dir
-    runStatementOnDriver("alter table "+ Table.MMTBL + " compact 'MAJOR'");
-    verifyDeltaDirAndResult(3, Table.MMTBL.toString(), "", resultData3);
-    runWorker(hiveConf);
-    verifyDeltaDirAndResult(2, Table.MMTBL.toString(), "", resultData3);
-    verifyBaseDir(1, Table.MMTBL.toString(), "");
-    runCleaner(hiveConf);
-    verifyDeltaDirAndResult(0, Table.MMTBL.toString(), "", resultData3);
-    verifyBaseDir(1, Table.MMTBL.toString(), "");
-    runInitiator(hiveConf);
-    verifyDeltaDirAndResult(0, Table.MMTBL.toString(), "", resultData3);
-    verifyBaseDir(1, Table.MMTBL.toString(), "");
-
-    // Verify query result
-    rs = runStatementOnDriver("select a,b from " + Table.MMTBL + " order by a");
-    Assert.assertEquals(stringifyValues(resultData3), rs);
-  }
-
-  @Test
-  public void testMmTableAbortWithCompactionNoCleanup() throws Exception {
-    // 1. Insert some rows into MM table
-    runStatementOnDriver("insert into " + Table.MMTBL + "(a,b) values(1,2)");
-    runStatementOnDriver("insert into " + Table.MMTBL + "(a,b) values(5,6)");
-    // There should be 1 delta directory
-    int [][] resultData1 =  new int[][] {{1,2}, {5,6}};
-    verifyDeltaDirAndResult(2, Table.MMTBL.toString(), "", resultData1);
-    List<String> r1 = runStatementOnDriver("select count(*) from " + Table.MMTBL);
-    Assert.assertEquals("2", r1.get(0));
-
-    // 2. Let a transaction be aborted
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true);
-    runStatementOnDriver("insert into " + Table.MMTBL + " values(3,4)");
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, false);
-    // There should be 1 delta and 1 base directory. The base one is the aborted one.
-    verifyDeltaDirAndResult(3, Table.MMTBL.toString(), "", resultData1);
-    r1 = runStatementOnDriver("select count(*) from " + Table.MMTBL);
-    Assert.assertEquals("2", r1.get(0));
-
-    // 3. Perform a MINOR compaction, expectation is it should remove aborted base dir
-    runStatementOnDriver("alter table "+ Table.MMTBL + " compact 'MINOR'");
-    // The worker should remove the subdir for aborted transaction
-    runWorker(hiveConf);
-    verifyDeltaDirAndResult(3, Table.MMTBL.toString(), "", resultData1);
-    verifyBaseDir(0, Table.MMTBL.toString(), "");
-    // Verify query result
-    List<String> rs = runStatementOnDriver("select a,b from " + Table.MMTBL + " order by a");
-    Assert.assertEquals(stringifyValues(resultData1), rs);
-
-    int [][] resultData3 = new int[][] {{1,2}, {5,6}, {7,8}};
-    // 4. add few more rows
-    runStatementOnDriver("insert into " + Table.MMTBL + "(a,b) values(7,8)");
-    // 5. add one more aborted delta
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true);
-    runStatementOnDriver("insert into " + Table.MMTBL + "(a,b) values(9,10)");
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, false);
-    verifyDeltaDirAndResult(5, Table.MMTBL.toString(), "", resultData3);
-
-    // 6. Perform a MAJOR compaction, expectation is it should remove aborted delta dir
-    runStatementOnDriver("alter table "+ Table.MMTBL + " compact 'MAJOR'");
-    runWorker(hiveConf);
-    verifyDeltaDirAndResult(4, Table.MMTBL.toString(), "", resultData3);
-    verifyBaseDir(1, Table.MMTBL.toString(), "");
-
-    // 7. Run one more Major compaction this should not have any affect
-    runStatementOnDriver("alter table "+ Table.MMTBL + " compact 'MAJOR'");
-    runWorker(hiveConf);
-    verifyDeltaDirAndResult(4, Table.MMTBL.toString(), "", resultData3);
-    verifyBaseDir(1, Table.MMTBL.toString(), "");
-
-    runCleaner(hiveConf);
-
-    // Verify query result
-    rs = runStatementOnDriver("select a,b from " + Table.MMTBL + " order by a");
-    Assert.assertEquals(stringifyValues(resultData3), rs);
-  }
-
-  @Test
-  public void testDynPartInsertWithAborts() throws Exception {
-    int[][] resultData = new int[][]{{1, 1}, {2, 2}};
-    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values(1,1,'p1'),(2,2,'p1')");
-    verifyDeltaDirAndResult(1, Table.ACIDTBLPART.toString(), "p=p1", resultData);
-
-    // forcing a txn to abort before addDynamicPartitions
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION, true);
-    runStatementOnDriverWithAbort("insert into " + Table.ACIDTBLPART + " partition(p) values(3,3,'p1'),(4,4,'p1')");
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION, false);
-    verifyDeltaDirAndResult(2, Table.ACIDTBLPART.toString(), "p=p1", resultData);
-
-    int count = TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from TXN_COMPONENTS where TC_OPERATION_TYPE='i'");
-    // We should have 1 row corresponding to the aborted transaction
-    Assert.assertEquals(TxnDbUtil.queryToString(hiveConf, "select * from TXN_COMPONENTS"), 1, count);
-
-    hiveConf.setTimeVar(HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD, 0, TimeUnit.MILLISECONDS);
-    runInitiator(hiveConf);
-
-    count = TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from COMPACTION_QUEUE");
-    // Only one job is added to the queue per table. This job corresponds to all the entries for a particular table
-    // with rows in TXN_COMPONENTS
-    Assert.assertEquals(TxnDbUtil.queryToString(hiveConf, "select * from COMPACTION_QUEUE"), 1, count);
-
-    runWorker(hiveConf);
-    runCleaner(hiveConf);
-
-    verifyDeltaDirAndResult(1, Table.ACIDTBLPART.toString(), "p=p1", resultData);
-  }
-
-  @Test
-  public void testDynPartInsertWithMultiPartitionAborts() throws Exception {
-    int [][] resultData = new int[][] {{1,1}, {2,2}};
-    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values(1,1,'p1'),(2,2,'p1')");
-    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values(1,1,'p2'),(2,2,'p2')");
-    verifyDeltaDirAndResult(1, Table.ACIDTBLPART.toString(), "p=p1", resultData);
-    verifyDeltaDirAndResult(1, Table.ACIDTBLPART.toString(), "p=p2", resultData);
-    List<String> r1 = runStatementOnDriver("select count(*) from " + Table.ACIDTBLPART);
-    Assert.assertEquals("4", r1.get(0));
-
-    // forcing a txn to abort before addDynamicPartitions
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION, true);
-    runStatementOnDriverWithAbort("insert into " + Table.ACIDTBLPART + " partition(p) values(3,3,'p1'),(4,4,'p1')");
-    runStatementOnDriverWithAbort("insert into " + Table.ACIDTBLPART + " partition(p) values(3,3,'p2'),(4,4,'p2')");
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION, false);
-    verifyDeltaDirAndResult(2, Table.ACIDTBLPART.toString(), "p=p1", resultData);
-    verifyDeltaDirAndResult(2, Table.ACIDTBLPART.toString(), "p=p2", resultData);
-    r1 = runStatementOnDriver("select count(*) from " + Table.ACIDTBLPART);
-    Assert.assertEquals("4", r1.get(0));
-
-    int count = TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from TXN_COMPONENTS where TC_OPERATION_TYPE='i'");
-    // We should have 2 rows corresponding to the two aborted transactions
-    Assert.assertEquals(TxnDbUtil.queryToString(hiveConf, "select * from TXN_COMPONENTS"), 2, count);
-
-    hiveConf.setTimeVar(HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD, 0, TimeUnit.MILLISECONDS);
-    runInitiator(hiveConf);
-
-    count = TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from COMPACTION_QUEUE");
-    // Only one job is added to the queue per table. This job corresponds to all the entries for a particular table
-    // with rows in TXN_COMPONENTS
-    Assert.assertEquals(TxnDbUtil.queryToString(hiveConf, "select * from COMPACTION_QUEUE"), 1, count);
-
-    r1 = runStatementOnDriver("select count(*) from " + Table.ACIDTBLPART);
-    Assert.assertEquals("4", r1.get(0));
-
-    runWorker(hiveConf);
-    runCleaner(hiveConf);
-
-    verifyDeltaDirAndResult(1, Table.ACIDTBLPART.toString(), "p=p1", resultData);
-    verifyDeltaDirAndResult(1, Table.ACIDTBLPART.toString(), "p=p2", resultData);
-    r1 = runStatementOnDriver("select count(*) from " + Table.ACIDTBLPART);
-    Assert.assertEquals("4", r1.get(0));
-  }
-
-  @Test
-  public void testDynPartIOWWithAborts() throws Exception {
-    int [][] resultData = new int[][] {{1,1}, {2,2}};
-    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values(1,1,'p1'),(2,2,'p1')");
-    verifyDeltaDirAndResult(1, Table.ACIDTBLPART.toString(), "p=p1", resultData);
-
-    // forcing a txn to abort before addDynamicPartitions
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION, true);
-    runStatementOnDriverWithAbort("insert overwrite table " + Table.ACIDTBLPART + " partition(p) values(3,3,'p1'),(4,4,'p1')");
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION, false);
-    verifyDeltaDirAndResult(1, Table.ACIDTBLPART.toString(), "p=p1", resultData);
-    verifyBaseDir(1, Table.ACIDTBLPART.toString(), "p=p1");
-
-    int count = TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from TXN_COMPONENTS where TC_OPERATION_TYPE='u'");
-    // We should have 1 row corresponding to the aborted transaction
-    Assert.assertEquals(TxnDbUtil.queryToString(hiveConf, "select * from TXN_COMPONENTS"), 1, count);
-
-    hiveConf.setTimeVar(HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD, 0, TimeUnit.MILLISECONDS);
-    runInitiator(hiveConf);
-
-    count = TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from COMPACTION_QUEUE");
-    // Only one job is added to the queue per table. This job corresponds to all the entries for a particular table
-    // with rows in TXN_COMPONENTS
-    Assert.assertEquals(TxnDbUtil.queryToString(hiveConf, "select * from COMPACTION_QUEUE"), 1, count);
-
-    runWorker(hiveConf);
-    runCleaner(hiveConf);
-
-    verifyDeltaDirAndResult(1, Table.ACIDTBLPART.toString(), "p=p1", resultData);
-    verifyBaseDir(0, Table.ACIDTBLPART.toString(), "p=p1");
-  }
-
-  @Test
-  public void testDynPartIOWWithMultiPartitionAborts() throws Exception {
-    int [][] resultData = new int[][] {{1,1}, {2,2}};
-    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values(1,1,'p1'),(2,2,'p1')");
-    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values(1,1,'p2'),(2,2,'p2')");
-    verifyDeltaDirAndResult(1, Table.ACIDTBLPART.toString(), "p=p1", resultData);
-    verifyDeltaDirAndResult(1, Table.ACIDTBLPART.toString(), "p=p2", resultData);
-    List<String> r1 = runStatementOnDriver("select count(*) from " + Table.ACIDTBLPART);
-    Assert.assertEquals("4", r1.get(0));
-
-    // forcing a txn to abort before addDynamicPartitions
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION, true);
-    runStatementOnDriverWithAbort("insert overwrite table " + Table.ACIDTBLPART + " partition(p) values(3,3,'p1'),(4,4,'p1')");
-    runStatementOnDriverWithAbort("insert overwrite table " + Table.ACIDTBLPART + " partition(p) values(3,3,'p2'),(4,4,'p2')");
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION, false);
-    verifyDeltaDirAndResult(1, Table.ACIDTBLPART.toString(), "p=p1", resultData);
-    verifyBaseDir(1, Table.ACIDTBLPART.toString(), "p=p1");
-    verifyDeltaDirAndResult(1, Table.ACIDTBLPART.toString(), "p=p2", resultData);
-    verifyBaseDir(1, Table.ACIDTBLPART.toString(), "p=p2");
-    r1 = runStatementOnDriver("select count(*) from " + Table.ACIDTBLPART);
-    Assert.assertEquals("4", r1.get(0));
-
-    int count = TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from TXN_COMPONENTS where TC_OPERATION_TYPE='u'");
-    // We should have two rows corresponding to the two aborted transactions
-    Assert.assertEquals(TxnDbUtil.queryToString(hiveConf, "select * from TXN_COMPONENTS"), 2, count);
-
-    hiveConf.setTimeVar(HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD, 0, TimeUnit.MILLISECONDS);
-    runInitiator(hiveConf);
-
-    count = TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from COMPACTION_QUEUE");
-    // Only one job is added to the queue per table. This job corresponds to all the entries for a particular table
-    // with rows in TXN_COMPONENTS
-    Assert.assertEquals(TxnDbUtil.queryToString(hiveConf, "select * from COMPACTION_QUEUE"), 1, count);
-
-    r1 = runStatementOnDriver("select count(*) from " + Table.ACIDTBLPART);
-    Assert.assertEquals("4", r1.get(0));
-
-    runWorker(hiveConf);
-    runCleaner(hiveConf);
-
-    verifyDeltaDirAndResult(1, Table.ACIDTBLPART.toString(), "p=p1", resultData);
-    verifyBaseDir(0, Table.ACIDTBLPART.toString(), "p=p1");
-    verifyDeltaDirAndResult(1, Table.ACIDTBLPART.toString(), "p=p2", resultData);
-    verifyBaseDir(0, Table.ACIDTBLPART.toString(), "p=p2");
-    r1 = runStatementOnDriver("select count(*) from " + Table.ACIDTBLPART);
-    Assert.assertEquals("4", r1.get(0));
-  }
-
-  @Test
-  public void testDynPartUpdateWithAborts() throws Exception {
-    int[][] resultData1 = new int[][]{{1, 2}, {3, 4}};
-    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values (1,2,'p1')");
-    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values (3,4,'p1')");
-    verifyDeltaDirAndResult(2, Table.ACIDTBLPART.toString(), "p=p1", resultData1);
-
-    // forcing a txn to abort before addDynamicPartitions
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION, true);
-    runStatementOnDriverWithAbort("update " + Table.ACIDTBLPART + " set b=a+2 where a<5");
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION, false);
-    verifyDeltaDirAndResult(3, Table.ACIDTBLPART.toString(), "p=p1", resultData1);
-    verifyDeleteDeltaDir(1, Table.ACIDTBLPART.toString(), "p=p1");
-
-    int count = TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from TXN_COMPONENTS where TC_OPERATION_TYPE='u'");
-    // We should have 1 row corresponding to the aborted transaction
-    Assert.assertEquals(TxnDbUtil.queryToString(hiveConf, "select * from TXN_COMPONENTS"), 1, count);
-
-    hiveConf.setTimeVar(HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD, 0, TimeUnit.MILLISECONDS);
-    runInitiator(hiveConf);
-
-    count = TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from COMPACTION_QUEUE");
-    // Only one job is added to the queue per table. This job corresponds to all the entries for a particular table
-    // with rows in TXN_COMPONENTS
-    Assert.assertEquals(TxnDbUtil.queryToString(hiveConf, "select * from COMPACTION_QUEUE"), 1, count);
-
-    runWorker(hiveConf);
-    runCleaner(hiveConf);
-
-    verifyDeltaDirAndResult(2, Table.ACIDTBLPART.toString(), "p=p1", resultData1);
-    verifyDeleteDeltaDir(0, Table.ACIDTBLPART.toString(), "p=p1");
-  }
-
-  @Test
-  public void testDynPartMergeWithAborts() throws Exception {
-    int [][] resultData = new int[][] {{1,1}, {2,2}};
-    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values(1,1,'p1'),(2,2,'p1')");
-    verifyDeltaDirAndResult(1, Table.ACIDTBLPART.toString(), "p=p1", resultData);
-    List<String> r1 = runStatementOnDriver("select count(*) from " + Table.ACIDTBLPART);
-    Assert.assertEquals("2", r1.get(0));
-
-    int[][] sourceVals = {{2,15},{4,44},{5,5},{11,11}};
-    runStatementOnDriver("insert into " + TestTxnCommands2.Table.NONACIDORCTBL + " " + TestTxnCommands2.makeValuesClause(sourceVals));
-
-    // forcing a txn to abort before addDynamicPartitions
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION, true);
-    runStatementOnDriverWithAbort("merge into " + Table.ACIDTBLPART + " using " + TestTxnCommands2.Table.NONACIDORCTBL +
-      " as s ON " + Table.ACIDTBLPART + ".a = s.a " +
-      "when matched then update set b = s.b " +
-      "when not matched then insert values(s.a, s.b, 'newpart')");
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILLOADDYNAMICPARTITION, false);
-    verifyDeltaDirAndResult(2, Table.ACIDTBLPART.toString(), "p=p1", resultData);
-    verifyDeleteDeltaDir(1, Table.ACIDTBLPART.toString(), "p=p1");
-    verifyDeltaDirAndResult(1, Table.ACIDTBLPART.toString(), "p=newpart", resultData);
-    r1 = runStatementOnDriver("select count(*) from " + Table.ACIDTBLPART);
-    Assert.assertEquals("2", r1.get(0));
-
-    int count = TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from TXN_COMPONENTS where TC_OPERATION_TYPE='u'");
-    // We should have 1 row corresponding to the aborted transaction
-    Assert.assertEquals(TxnDbUtil.queryToString(hiveConf, "select * from TXN_COMPONENTS"), 1, count);
-
-    hiveConf.setTimeVar(HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD, 0, TimeUnit.MILLISECONDS);
-    runInitiator(hiveConf);
-
-    count = TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from COMPACTION_QUEUE");
-    // Only one job is added to the queue per table. This job corresponds to all the entries for a particular table
-    // with rows in TXN_COMPONENTS
-    Assert.assertEquals(TxnDbUtil.queryToString(hiveConf, "select * from COMPACTION_QUEUE"), 1, count);
-
-    r1 = runStatementOnDriver("select count(*) from " + Table.ACIDTBLPART);
-    Assert.assertEquals("2", r1.get(0));
-
-    runWorker(hiveConf);
-    runCleaner(hiveConf);
-
-    verifyDeltaDirAndResult(1, Table.ACIDTBLPART.toString(), "p=p1", resultData);
-    verifyDeleteDeltaDir(0, Table.ACIDTBLPART.toString(), "p=p1");
-    verifyDeltaDirAndResult(0, Table.ACIDTBLPART.toString(), "p=newpart", resultData);
-    r1 = runStatementOnDriver("select count(*) from " + Table.ACIDTBLPART);
-    Assert.assertEquals("2", r1.get(0));
-  }
-
-  @Test
-  public void testFullACIDAbortWithMinorMajorCompaction() throws Exception {
-    // 1. Insert some rows into acid table
-    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) values(1,2)");
-    // There should be 1 delta directory
-    int [][] resultData1 =  new int[][] {{1,2}};
-    verifyDeltaDirAndResult(1, Table.ACIDTBL.toString(), "", resultData1);
-    List<String> r1 = runStatementOnDriver("select count(*) from " + Table.ACIDTBL);
-    Assert.assertEquals("1", r1.get(0));
-
-    // 2. Let a transaction be aborted
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true);
-    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) values(3,4)");
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, false);
-    // There should be 2 delta directories.
-    verifyDeltaDirAndResult(2, Table.ACIDTBL.toString(), "", resultData1);
-
-    r1 = runStatementOnDriver("select count(*) from " + Table.ACIDTBL);
-    Assert.assertEquals("1", r1.get(0));
-
-    // Verify query result
-    int [][] resultData2 = new int[][] {{1,2}, {5,6}};
-    // 3. insert few more rows in acid table
-    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) values(5,6)");
-    verifyDeltaDirAndResult(3, Table.ACIDTBL.toString(), "", resultData2);
-    r1 = runStatementOnDriver("select count(*) from " + Table.ACIDTBL);
-    Assert.assertEquals("2", r1.get(0));
-
-    // 4. Perform a MINOR compaction
-    runStatementOnDriver("alter table "+ Table.ACIDTBL + " compact 'MINOR'");
-    runWorker(hiveConf);
-    verifyDeltaDirAndResult(4, Table.ACIDTBL.toString(), "", resultData2);
-    verifyBaseDir(0, Table.ACIDTBL.toString(), "");
-    // 5. Run Cleaner, should remove compacted deltas including aborted ones.
-    runCleaner(hiveConf);
-    verifyDeltaDirAndResult(1, Table.ACIDTBL.toString(), "", resultData2);
-
-    // Verify query result
-    List<String> rs = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a");
-    Assert.assertEquals(stringifyValues(resultData2), rs);
-
-    int [][] resultData3 = new int[][] {{1,2}, {5,6}, {7,8}, {9,10}};
-    // 6. add few more rows
-    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) values(7,8)");
-    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) values(9,10)");
-    // 7. add one more aborted delta
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true);
-    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) values(11,12)");
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, false);
-
-    // 8. Perform a MAJOR compaction
-    runStatementOnDriver("alter table "+ Table.ACIDTBL + " compact 'MAJOR'");
-    verifyDeltaDirAndResult(4, Table.ACIDTBL.toString(), "", resultData3);
-    runWorker(hiveConf);
-    verifyDeltaDirAndResult(4, Table.ACIDTBL.toString(), "", resultData3);
-    verifyBaseDir(1, Table.ACIDTBL.toString(), "");
-    // The cleaner should remove compacted deltas including aborted ones.
-    runCleaner(hiveConf);
-    verifyDeltaDirAndResult(0, Table.ACIDTBL.toString(), "", resultData3);
-    verifyBaseDir(1, Table.ACIDTBL.toString(), "");
-
-    // Verify query result
-    rs = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a");
-    Assert.assertEquals(stringifyValues(resultData3), rs);
-  }
-
-  @Test
-  public void testFullACIDAbortWithMajorCompaction() throws Exception {
-    // 1. Insert some rows into acid table
-    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) values(1,2)");
-    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) values(3,4)");
-    // There should be 2 delta directories
-    int [][] resultData1 =  new int[][] {{1,2}, {3,4}};
-    verifyDeltaDirAndResult(2, Table.ACIDTBL.toString(), "", resultData1);
-    List<String> r1 = runStatementOnDriver("select count(*) from " + Table.ACIDTBL);
-    Assert.assertEquals("2", r1.get(0));
-
-    // 2. Let a transaction be aborted
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true);
-    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) values(5,6)");
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, false);
-    // There should be 2 delta and 1 base directory. The base one is the aborted one.
-    verifyDeltaDirAndResult(3, Table.ACIDTBL.toString(), "", resultData1);
-    r1 = runStatementOnDriver("select count(*) from " + Table.ACIDTBL);
-    Assert.assertEquals("2", r1.get(0));
-
-    // 3.Perform a MAJOR compaction
-    runStatementOnDriver("alter table "+ Table.ACIDTBL + " compact 'MAJOR'");
-    runWorker(hiveConf);
-
-    verifyDeltaDirAndResult(3, Table.ACIDTBL.toString(), "", resultData1);
-    verifyBaseDir(1, Table.ACIDTBL.toString(), "");
-    // 4. Run Cleaner, should remove compacted deltas including aborted ones.
-    runCleaner(hiveConf);
-    verifyDeltaDirAndResult(0, Table.ACIDTBL.toString(), "", resultData1);
-    verifyBaseDir(1, Table.ACIDTBL.toString(), "");
-
-    // Verify query result
-    List<String> rs = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a");
-    Assert.assertEquals(stringifyValues(resultData1), rs);
-  }
-
-  @Test
-  public void testFullACIDAbortWithCompactionNoCleanup() throws Exception {
-    // 1. Insert some rows into acid table
-    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) values(1,2)");
-    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) values(3,4)");
-    // There should be 2 delta directories
-    int [][] resultData1 =  new int[][] {{1,2}, {3,4}};
-    verifyDeltaDirAndResult(2, Table.ACIDTBL.toString(), "", resultData1);
-
-    // 2. abort one txns
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true);
-    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) values(5,6)");
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, false);
-    verifyDeltaDirAndResult(3, Table.ACIDTBL.toString(), "", resultData1);
-
-    // 3. Perform a MAJOR compaction.
-    runStatementOnDriver("alter table "+ Table.ACIDTBL + " compact 'MAJOR'");
-    runWorker(hiveConf);
-    verifyDeltaDirAndResult(3, Table.ACIDTBL.toString(), "", resultData1);
-    verifyBaseDir(1, Table.ACIDTBL.toString(), "");
-
-    // Verify query result
-    List<String> rs = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a");
-    Assert.assertEquals(stringifyValues(resultData1), rs);
-  }
-
-  @Test
-  public void testFullACIDAbortWithManyPartitions() throws Exception {
-    // 1. Insert some rows into acid table
-    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p='p1') (a,b) values(1,2)");
-    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p='p2') (a,b) values(1,2)");
-    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p='p3') (a,b) values(1,2)");
-    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p='p1') (a,b) values(3,4)");
-    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p='p2') (a,b) values(3,4)");
-    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p='p3') (a,b) values(3,4)");
-    // There should be 2 delta directories in each partition
-    int [][] resultData1 =  new int[][] {{1,2}, {3,4}};
-    verifyDeltaDirAndResult(2, Table.ACIDTBLPART.toString(), "p=p1", resultData1);
-    verifyDeltaDirAndResult(2, Table.ACIDTBLPART.toString(), "p=p2", resultData1);
-    verifyDeltaDirAndResult(2, Table.ACIDTBLPART.toString(), "p=p3", resultData1);
-
-    // 2. abort two txns in each partition
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true);
-    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p='p1') (a,b) values(5,6)");
-    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p='p2') (a,b) values(5,6)");
-    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p='p3') (a,b) values(5,6)");
-    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p='p1') (a,b) values(5,6)");
-    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p='p2') (a,b) values(5,6)");
-    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p='p3') (a,b) values(5,6)");
-    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, false);
-    verifyDeltaDirAndResult(4, Table.ACIDTBLPART.toString(), "p=p1", resultData1);
-    verifyDeltaDirAndResult(4, Table.ACIDTBLPART.toString(), "p=p2", resultData1);
-    verifyDeltaDirAndResult(4, Table.ACIDTBLPART.toString(), "p=p3", resultData1);
-
-    // We should have total six rows corresponding to the above six aborted transactions
-    Assert.assertEquals(TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from TXN_COMPONENTS"), 6);
-    runCleaner(hiveConf);
-    Assert.assertEquals(TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from TXN_COMPONENTS"), 6);
-
-    // 3.1 Perform a MAJOR compaction on p='p1'.
-    runStatementOnDriver("alter table "+ Table.ACIDTBLPART + " partition(p='p1') compact 'MAJOR'");
-    runWorker(hiveConf);
-    verifyDeltaDirAndResult(4, Table.ACIDTBLPART.toString(), "p=p1", resultData1);
-    verifyDeltaDirAndResult(4, Table.ACIDTBLPART.toString(), "p=p2", resultData1);
-    verifyDeltaDirAndResult(4, Table.ACIDTBLPART.toString(), "p=p3", resultData1);
-    verifyBaseDir(1, Table.ACIDTBLPART.toString(), "p=p1");
-    // The cleaner should remove compacted deltas including aborted ones.
-    runCleaner(hiveConf);
-    Assert.assertEquals(TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from TXN_COMPONENTS"), 4);
-
-    // 3.2 Perform a MAJOR compaction on p='p2'.
-    runStatementOnDriver("alter table "+ Table.ACIDTBLPART + " partition(p='p2') compact 'MAJOR'");
-    runWorker(hiveConf);
-    verifyDeltaDirAndResult(0, Table.ACIDTBLPART.toString(), "p=p1", resultData1);
-    verifyDeltaDirAndResult(4, Table.ACIDTBLPART.toString(), "p=p2", resultData1);
-    verifyDeltaDirAndResult(4, Table.ACIDTBLPART.toString(), "p=p3", resultData1);
-    verifyBaseDir(1, Table.ACIDTBLPART.toString(), "p=p2");
-    // The cleaner should remove compacted deltas including aborted ones.
-    runCleaner(hiveConf);
-    Assert.assertEquals(TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from TXN_COMPONENTS"), 2);
-
-    // 3.3 Perform a MAJOR compaction on p='p3'.
-    runStatementOnDriver("alter table "+ Table.ACIDTBLPART + " partition(p='p3') compact 'MAJOR'");
-    runWorker(hiveConf);
-    verifyDeltaDirAndResult(0, Table.ACIDTBLPART.toString(), "p=p1", resultData1);
-    verifyDeltaDirAndResult(0, Table.ACIDTBLPART.toString(), "p=p2", resultData1);
-    verifyDeltaDirAndResult(4, Table.ACIDTBLPART.toString(), "p=p3", resultData1);
-    verifyBaseDir(1, Table.ACIDTBLPART.toString(), "p=p3");
-    // The cleaner should remove compacted deltas including aborted ones.
-    runCleaner(hiveConf);
-    verifyDeltaDirAndResult(0, Table.ACIDTBLPART.toString(), "p=p3", resultData1);
-    Assert.assertEquals(TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from TXN_COMPONENTS"), 0);
-
-    // 4. Verify query result
-    int [][] resultData2 =  new int[][] {{1,2},{1,2},{1,2},{3,4},{3,4},{3,4}};
-    List<String> rs = runStatementOnDriver("select a,b from " + Table.ACIDTBLPART + " order by a");
-    Assert.assertEquals(stringifyValues(resultData2), rs);
-  }
-
-  private void verifyDeleteDeltaDir(int expectedDeltas, String tblName, String partName) throws Exception {
-    verifyDir(expectedDeltas, tblName, partName, "delete_delta_.*");
-  }
-
-  private void verifyBaseDir(int expectedDeltas, String tblName, String partName) throws Exception {
-    verifyDir(expectedDeltas, tblName, partName, "base_.*");
-  }
-
-  private void verifyDir(int expectedDeltas, String tblName, String partName, String pattern) throws Exception {
-    Path warehouse = new Path(TEST_WAREHOUSE_DIR);
-    tblName = tblName.toLowerCase();
-
-    FileSystem fs = FileSystem.get(warehouse.toUri(), hiveConf);
-    FileStatus[] status = fs.listStatus(new Path(warehouse, tblName + "/" + partName),
-        FileUtils.HIDDEN_FILES_PATH_FILTER);
-
+  private void verifyDirAndResult(int expectedDeltas) throws Exception {
+    FileSystem fs = FileSystem.get(hiveConf);
+    // Verify the content of subdirs
+    FileStatus[] status = fs.listStatus(new Path(TEST_WAREHOUSE_DIR + "/" +
+        (Table.MMTBL).toString().toLowerCase()), FileUtils.HIDDEN_FILES_PATH_FILTER);
     int sawDeltaTimes = 0;
     for (int i = 0; i < status.length; i++) {
-      if (status[i].getPath().getName().matches(pattern)) {
-        sawDeltaTimes++;
-      }
+      Assert.assertTrue(status[i].getPath().getName().matches("delta_.*"));
+      sawDeltaTimes++;
+      FileStatus[] files = fs.listStatus(status[i].getPath(), FileUtils.HIDDEN_FILES_PATH_FILTER);
+      Assert.assertEquals(1, files.length);
+      Assert.assertTrue(files[0].getPath().getName().equals("000000_0"));
     }
     Assert.assertEquals(expectedDeltas, sawDeltaTimes);
-  }
 
-  private void verifyDeltaDirAndResult(int expectedDeltas, String tblName, String partName, int [][] resultData) throws Exception {
-    verifyDir(expectedDeltas, tblName, partName, "delta_.*");
-    if (partName.equals("p=newpart")) return;
-
-    List<String> rs = runStatementOnDriver("select a,b from " + tblName + (partName.isEmpty() ?
-      "" : " where p='" + partName.substring(2) + "'") + " order by a");
+    // Verify query result
+    int [][] resultData = new int[][] {{1,2}, {3,4}};
+    List<String> rs = runStatementOnDriver("select a,b from " + Table.MMTBL);
     Assert.assertEquals(stringifyValues(resultData), rs);
   }
 
@@ -2857,26 +2279,17 @@ public class TestTxnCommands2 {
     return sb.toString();
   }
 
-  private void runStatementOnDriverWithAbort(String stmt) {
-    LOG.info("+runStatementOnDriver(" + stmt + ")");
-    try {
-      d.run(stmt);
-    } catch (CommandProcessorException e) {
-    }
-  }
-
-  private List<String> runStatementOnDriver(String stmt) throws Exception {
+  protected List<String> runStatementOnDriver(String stmt) throws Exception {
     LOG.info("+runStatementOnDriver(" + stmt + ")");
     try {
       d.run(stmt);
     } catch (CommandProcessorException e) {
       throw new RuntimeException(stmt + " failed: " + e);
     }
-    List<String> rs = new ArrayList<>();
+    List<String> rs = new ArrayList<String>();
     d.getResults(rs);
     return rs;
   }
-
   final void assertUniqueID(Table table) throws Exception {
     String partCols = table.getPartitionColumns();
     //check to make sure there are no duplicate ROW__IDs - HIVE-16832
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
index 415900e..339aa12 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
@@ -2566,7 +2566,7 @@ public class TestDbTxnManager2 extends DbTxnManagerEndToEndTestBase{
     Assert.assertEquals(
         "TXN_COMPONENTS mismatch(" + JavaUtils.txnIdToString(txnid1) + "): " +
         TxnDbUtil.queryToString(conf, "select * from \"TXN_COMPONENTS\""),
-        1,
+        0,
         TxnDbUtil.countQueryAgent(conf, "select count(*) from \"TXN_COMPONENTS\" where \"TC_TXNID\"=" + txnid1));
     //now actually write to table to generate some partitions
     driver.run("insert into target partition(p=1,q) values (1,2,2), (3,4,2), (5,6,3), (7,8,2)");
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
index 2af50ce..453d5fd 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@ -41941,11 +41941,11 @@ uint32_t ThriftHiveMetastore_commit_txn_presult::read(::apache::thrift::protocol
 }
 
 
-ThriftHiveMetastore_get_latest_txnid_in_conflict_args::~ThriftHiveMetastore_get_latest_txnid_in_conflict_args() noexcept {
+ThriftHiveMetastore_get_latest_txn_in_conflict_args::~ThriftHiveMetastore_get_latest_txn_in_conflict_args() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_latest_txnid_in_conflict_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_latest_txn_in_conflict_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -41986,10 +41986,10 @@ uint32_t ThriftHiveMetastore_get_latest_txnid_in_conflict_args::read(::apache::t
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_latest_txnid_in_conflict_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_latest_txn_in_conflict_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_latest_txnid_in_conflict_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_latest_txn_in_conflict_args");
 
   xfer += oprot->writeFieldBegin("txnId", ::apache::thrift::protocol::T_I64, 1);
   xfer += oprot->writeI64(this->txnId);
@@ -42001,14 +42001,14 @@ uint32_t ThriftHiveMetastore_get_latest_txnid_in_conflict_args::write(::apache::
 }
 
 
-ThriftHiveMetastore_get_latest_txnid_in_conflict_pargs::~ThriftHiveMetastore_get_latest_txnid_in_conflict_pargs() noexcept {
+ThriftHiveMetastore_get_latest_txn_in_conflict_pargs::~ThriftHiveMetastore_get_latest_txn_in_conflict_pargs() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_latest_txnid_in_conflict_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_latest_txn_in_conflict_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_latest_txnid_in_conflict_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_latest_txn_in_conflict_pargs");
 
   xfer += oprot->writeFieldBegin("txnId", ::apache::thrift::protocol::T_I64, 1);
   xfer += oprot->writeI64((*(this->txnId)));
@@ -42020,11 +42020,11 @@ uint32_t ThriftHiveMetastore_get_latest_txnid_in_conflict_pargs::write(::apache:
 }
 
 
-ThriftHiveMetastore_get_latest_txnid_in_conflict_result::~ThriftHiveMetastore_get_latest_txnid_in_conflict_result() noexcept {
+ThriftHiveMetastore_get_latest_txn_in_conflict_result::~ThriftHiveMetastore_get_latest_txn_in_conflict_result() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_latest_txnid_in_conflict_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_latest_txn_in_conflict_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -42073,11 +42073,11 @@ uint32_t ThriftHiveMetastore_get_latest_txnid_in_conflict_result::read(::apache:
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_latest_txnid_in_conflict_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_latest_txn_in_conflict_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_latest_txnid_in_conflict_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_latest_txn_in_conflict_result");
 
   if (this->__isset.success) {
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_I64, 0);
@@ -42094,11 +42094,11 @@ uint32_t ThriftHiveMetastore_get_latest_txnid_in_conflict_result::write(::apache
 }
 
 
-ThriftHiveMetastore_get_latest_txnid_in_conflict_presult::~ThriftHiveMetastore_get_latest_txnid_in_conflict_presult() noexcept {
+ThriftHiveMetastore_get_latest_txn_in_conflict_presult::~ThriftHiveMetastore_get_latest_txn_in_conflict_presult() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_latest_txnid_in_conflict_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_latest_txn_in_conflict_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -69557,18 +69557,18 @@ void ThriftHiveMetastoreClient::recv_commit_txn()
   return;
 }
 
-int64_t ThriftHiveMetastoreClient::get_latest_txnid_in_conflict(const int64_t txnId)
+int64_t ThriftHiveMetastoreClient::get_latest_txn_in_conflict(const int64_t txnId)
 {
-  send_get_latest_txnid_in_conflict(txnId);
-  return recv_get_latest_txnid_in_conflict();
+  send_get_latest_txn_in_conflict(txnId);
+  return recv_get_latest_txn_in_conflict();
 }
 
-void ThriftHiveMetastoreClient::send_get_latest_txnid_in_conflict(const int64_t txnId)
+void ThriftHiveMetastoreClient::send_get_latest_txn_in_conflict(const int64_t txnId)
 {
   int32_t cseqid = 0;
-  oprot_->writeMessageBegin("get_latest_txnid_in_conflict", ::apache::thrift::protocol::T_CALL, cseqid);
+  oprot_->writeMessageBegin("get_latest_txn_in_conflict", ::apache::thrift::protocol::T_CALL, cseqid);
 
-  ThriftHiveMetastore_get_latest_txnid_in_conflict_pargs args;
+  ThriftHiveMetastore_get_latest_txn_in_conflict_pargs args;
   args.txnId = &txnId;
   args.write(oprot_);
 
@@ -69577,7 +69577,7 @@ void ThriftHiveMetastoreClient::send_get_latest_txnid_in_conflict(const int64_t
   oprot_->getTransport()->flush();
 }
 
-int64_t ThriftHiveMetastoreClient::recv_get_latest_txnid_in_conflict()
+int64_t ThriftHiveMetastoreClient::recv_get_latest_txn_in_conflict()
 {
 
   int32_t rseqid = 0;
@@ -69597,13 +69597,13 @@ int64_t ThriftHiveMetastoreClient::recv_get_latest_txnid_in_conflict()
     iprot_->readMessageEnd();
     iprot_->getTransport()->readEnd();
   }
-  if (fname.compare("get_latest_txnid_in_conflict") != 0) {
+  if (fname.compare("get_latest_txn_in_conflict") != 0) {
     iprot_->skip(::apache::thrift::protocol::T_STRUCT);
     iprot_->readMessageEnd();
     iprot_->getTransport()->readEnd();
   }
   int64_t _return;
-  ThriftHiveMetastore_get_latest_txnid_in_conflict_presult result;
+  ThriftHiveMetastore_get_latest_txn_in_conflict_presult result;
   result.success = &_return;
   result.read(iprot_);
   iprot_->readMessageEnd();
@@ -69615,7 +69615,7 @@ int64_t ThriftHiveMetastoreClient::recv_get_latest_txnid_in_conflict()
   if (result.__isset.o1) {
     throw result.o1;
   }
-  throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_latest_txnid_in_conflict failed: unknown result");
+  throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_latest_txn_in_conflict failed: unknown result");
 }
 
 void ThriftHiveMetastoreClient::repl_tbl_writeid_state(const ReplTblWriteIdStateRequest& rqst)
@@ -84435,41 +84435,41 @@ void ThriftHiveMetastoreProcessor::process_commit_txn(int32_t seqid, ::apache::t
   }
 }
 
-void ThriftHiveMetastoreProcessor::process_get_latest_txnid_in_conflict(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext)
+void ThriftHiveMetastoreProcessor::process_get_latest_txn_in_conflict(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext)
 {
   void* ctx = NULL;
   if (this->eventHandler_.get() != NULL) {
-    ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_latest_txnid_in_conflict", callContext);
+    ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_latest_txn_in_conflict", callContext);
   }
-  ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_latest_txnid_in_conflict");
+  ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_latest_txn_in_conflict");
 
   if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_latest_txnid_in_conflict");
+    this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_latest_txn_in_conflict");
   }
 
-  ThriftHiveMetastore_get_latest_txnid_in_conflict_args args;
+  ThriftHiveMetastore_get_latest_txn_in_conflict_args args;
   args.read(iprot);
   iprot->readMessageEnd();
   uint32_t bytes = iprot->getTransport()->readEnd();
 
   if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_latest_txnid_in_conflict", bytes);
+    this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_latest_txn_in_conflict", bytes);
   }
 
-  ThriftHiveMetastore_get_latest_txnid_in_conflict_result result;
+  ThriftHiveMetastore_get_latest_txn_in_conflict_result result;
   try {
-    result.success = iface_->get_latest_txnid_in_conflict(args.txnId);
+    result.success = iface_->get_latest_txn_in_conflict(args.txnId);
     result.__isset.success = true;
   } catch (MetaException &o1) {
     result.o1 = o1;
     result.__isset.o1 = true;
   } catch (const std::exception& e) {
     if (this->eventHandler_.get() != NULL) {
-      this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_latest_txnid_in_conflict");
+      this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_latest_txn_in_conflict");
     }
 
     ::apache::thrift::TApplicationException x(e.what());
-    oprot->writeMessageBegin("get_latest_txnid_in_conflict", ::apache::thrift::protocol::T_EXCEPTION, seqid);
+    oprot->writeMessageBegin("get_latest_txn_in_conflict", ::apache::thrift::protocol::T_EXCEPTION, seqid);
     x.write(oprot);
     oprot->writeMessageEnd();
     oprot->getTransport()->writeEnd();
@@ -84478,17 +84478,17 @@ void ThriftHiveMetastoreProcessor::process_get_latest_txnid_in_conflict(int32_t
   }
 
   if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_latest_txnid_in_conflict");
+    this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_latest_txn_in_conflict");
   }
 
-  oprot->writeMessageBegin("get_latest_txnid_in_conflict", ::apache::thrift::protocol::T_REPLY, seqid);
+  oprot->writeMessageBegin("get_latest_txn_in_conflict", ::apache::thrift::protocol::T_REPLY, seqid);
   result.write(oprot);
   oprot->writeMessageEnd();
   bytes = oprot->getTransport()->writeEnd();
   oprot->getTransport()->flush();
 
   if (this->eventHandler_.get() != NULL) {
-    this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_latest_txnid_in_conflict", bytes);
+    this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_latest_txn_in_conflict", bytes);
   }
 }
 
@@ -104402,19 +104402,19 @@ void ThriftHiveMetastoreConcurrentClient::recv_commit_txn(const int32_t seqid)
   } // end while(true)
 }
 
-int64_t ThriftHiveMetastoreConcurrentClient::get_latest_txnid_in_conflict(const int64_t txnId)
+int64_t ThriftHiveMetastoreConcurrentClient::get_latest_txn_in_conflict(const int64_t txnId)
 {
-  int32_t seqid = send_get_latest_txnid_in_conflict(txnId);
-  return recv_get_latest_txnid_in_conflict(seqid);
+  int32_t seqid = send_get_latest_txn_in_conflict(txnId);
+  return recv_get_latest_txn_in_conflict(seqid);
 }
 
-int32_t ThriftHiveMetastoreConcurrentClient::send_get_latest_txnid_in_conflict(const int64_t txnId)
+int32_t ThriftHiveMetastoreConcurrentClient::send_get_latest_txn_in_conflict(const int64_t txnId)
 {
   int32_t cseqid = this->sync_->generateSeqId();
   ::apache::thrift::async::TConcurrentSendSentry sentry(this->sync_.get());
-  oprot_->writeMessageBegin("get_latest_txnid_in_conflict", ::apache::thrift::protocol::T_CALL, cseqid);
+  oprot_->writeMessageBegin("get_latest_txn_in_conflict", ::apache::thrift::protocol::T_CALL, cseqid);
 
-  ThriftHiveMetastore_get_latest_txnid_in_conflict_pargs args;
+  ThriftHiveMetastore_get_latest_txn_in_conflict_pargs args;
   args.txnId = &txnId;
   args.write(oprot_);
 
@@ -104426,7 +104426,7 @@ int32_t ThriftHiveMetastoreConcurrentClient::send_get_latest_txnid_in_conflict(c
   return cseqid;
 }
 
-int64_t ThriftHiveMetastoreConcurrentClient::recv_get_latest_txnid_in_conflict(const int32_t seqid)
+int64_t ThriftHiveMetastoreConcurrentClient::recv_get_latest_txn_in_conflict(const int32_t seqid)
 {
 
   int32_t rseqid = 0;
@@ -104455,7 +104455,7 @@ int64_t ThriftHiveMetastoreConcurrentClient::recv_get_latest_txnid_in_conflict(c
         iprot_->readMessageEnd();
         iprot_->getTransport()->readEnd();
       }
-      if (fname.compare("get_latest_txnid_in_conflict") != 0) {
+      if (fname.compare("get_latest_txn_in_conflict") != 0) {
         iprot_->skip(::apache::thrift::protocol::T_STRUCT);
         iprot_->readMessageEnd();
         iprot_->getTransport()->readEnd();
@@ -104465,7 +104465,7 @@ int64_t ThriftHiveMetastoreConcurrentClient::recv_get_latest_txnid_in_conflict(c
         throw TProtocolException(TProtocolException::INVALID_DATA);
       }
       int64_t _return;
-      ThriftHiveMetastore_get_latest_txnid_in_conflict_presult result;
+      ThriftHiveMetastore_get_latest_txn_in_conflict_presult result;
       result.success = &_return;
       result.read(iprot_);
       iprot_->readMessageEnd();
@@ -104480,7 +104480,7 @@ int64_t ThriftHiveMetastoreConcurrentClient::recv_get_latest_txnid_in_conflict(c
         throw result.o1;
       }
       // in a bad state, don't commit
-      throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_latest_txnid_in_conflict failed: unknown result");
+      throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_latest_txn_in_conflict failed: unknown result");
     }
     // seqid != rseqid
     this->sync_->updatePending(fname, mtype, rseqid);
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
index a228f04..690c1f9 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
@@ -190,7 +190,7 @@ class ThriftHiveMetastoreIf : virtual public  ::facebook::fb303::FacebookService
   virtual void abort_txn(const AbortTxnRequest& rqst) = 0;
   virtual void abort_txns(const AbortTxnsRequest& rqst) = 0;
   virtual void commit_txn(const CommitTxnRequest& rqst) = 0;
-  virtual int64_t get_latest_txnid_in_conflict(const int64_t txnId) = 0;
+  virtual int64_t get_latest_txn_in_conflict(const int64_t txnId) = 0;
   virtual void repl_tbl_writeid_state(const ReplTblWriteIdStateRequest& rqst) = 0;
   virtual void get_valid_write_ids(GetValidWriteIdsResponse& _return, const GetValidWriteIdsRequest& rqst) = 0;
   virtual void allocate_table_write_ids(AllocateTableWriteIdsResponse& _return, const AllocateTableWriteIdsRequest& rqst) = 0;
@@ -827,7 +827,7 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p
   void commit_txn(const CommitTxnRequest& /* rqst */) {
     return;
   }
-  int64_t get_latest_txnid_in_conflict(const int64_t /* txnId */) {
+  int64_t get_latest_txn_in_conflict(const int64_t /* txnId */) {
     int64_t _return = 0;
     return _return;
   }
@@ -21918,37 +21918,37 @@ class ThriftHiveMetastore_commit_txn_presult {
 
 };
 
-typedef struct _ThriftHiveMetastore_get_latest_txnid_in_conflict_args__isset {
-  _ThriftHiveMetastore_get_latest_txnid_in_conflict_args__isset() : txnId(false) {}
+typedef struct _ThriftHiveMetastore_get_latest_txn_in_conflict_args__isset {
+  _ThriftHiveMetastore_get_latest_txn_in_conflict_args__isset() : txnId(false) {}
   bool txnId :1;
-} _ThriftHiveMetastore_get_latest_txnid_in_conflict_args__isset;
+} _ThriftHiveMetastore_get_latest_txn_in_conflict_args__isset;
 
-class ThriftHiveMetastore_get_latest_txnid_in_conflict_args {
+class ThriftHiveMetastore_get_latest_txn_in_conflict_args {
  public:
 
-  ThriftHiveMetastore_get_latest_txnid_in_conflict_args(const ThriftHiveMetastore_get_latest_txnid_in_conflict_args&);
-  ThriftHiveMetastore_get_latest_txnid_in_conflict_args& operator=(const ThriftHiveMetastore_get_latest_txnid_in_conflict_args&);
-  ThriftHiveMetastore_get_latest_txnid_in_conflict_args() : txnId(0) {
+  ThriftHiveMetastore_get_latest_txn_in_conflict_args(const ThriftHiveMetastore_get_latest_txn_in_conflict_args&);
+  ThriftHiveMetastore_get_latest_txn_in_conflict_args& operator=(const ThriftHiveMetastore_get_latest_txn_in_conflict_args&);
+  ThriftHiveMetastore_get_latest_txn_in_conflict_args() : txnId(0) {
   }
 
-  virtual ~ThriftHiveMetastore_get_latest_txnid_in_conflict_args() noexcept;
+  virtual ~ThriftHiveMetastore_get_latest_txn_in_conflict_args() noexcept;
   int64_t txnId;
 
-  _ThriftHiveMetastore_get_latest_txnid_in_conflict_args__isset __isset;
+  _ThriftHiveMetastore_get_latest_txn_in_conflict_args__isset __isset;
 
   void __set_txnId(const int64_t val);
 
-  bool operator == (const ThriftHiveMetastore_get_latest_txnid_in_conflict_args & rhs) const
+  bool operator == (const ThriftHiveMetastore_get_latest_txn_in_conflict_args & rhs) const
   {
     if (!(txnId == rhs.txnId))
       return false;
     return true;
   }
-  bool operator != (const ThriftHiveMetastore_get_latest_txnid_in_conflict_args &rhs) const {
+  bool operator != (const ThriftHiveMetastore_get_latest_txn_in_conflict_args &rhs) const {
     return !(*this == rhs);
   }
 
-  bool operator < (const ThriftHiveMetastore_get_latest_txnid_in_conflict_args & ) const;
+  bool operator < (const ThriftHiveMetastore_get_latest_txn_in_conflict_args & ) const;
 
   uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
   uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
@@ -21956,42 +21956,42 @@ class ThriftHiveMetastore_get_latest_txnid_in_conflict_args {
 };
 
 
-class ThriftHiveMetastore_get_latest_txnid_in_conflict_pargs {
+class ThriftHiveMetastore_get_latest_txn_in_conflict_pargs {
  public:
 
 
-  virtual ~ThriftHiveMetastore_get_latest_txnid_in_conflict_pargs() noexcept;
+  virtual ~ThriftHiveMetastore_get_latest_txn_in_conflict_pargs() noexcept;
   const int64_t* txnId;
 
   uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
 
 };
 
-typedef struct _ThriftHiveMetastore_get_latest_txnid_in_conflict_result__isset {
-  _ThriftHiveMetastore_get_latest_txnid_in_conflict_result__isset() : success(false), o1(false) {}
+typedef struct _ThriftHiveMetastore_get_latest_txn_in_conflict_result__isset {
+  _ThriftHiveMetastore_get_latest_txn_in_conflict_result__isset() : success(false), o1(false) {}
   bool success :1;
   bool o1 :1;
-} _ThriftHiveMetastore_get_latest_txnid_in_conflict_result__isset;
+} _ThriftHiveMetastore_get_latest_txn_in_conflict_result__isset;
 
-class ThriftHiveMetastore_get_latest_txnid_in_conflict_result {
+class ThriftHiveMetastore_get_latest_txn_in_conflict_result {
  public:
 
-  ThriftHiveMetastore_get_latest_txnid_in_conflict_result(const ThriftHiveMetastore_get_latest_txnid_in_conflict_result&);
-  ThriftHiveMetastore_get_latest_txnid_in_conflict_result& operator=(const ThriftHiveMetastore_get_latest_txnid_in_conflict_result&);
-  ThriftHiveMetastore_get_latest_txnid_in_conflict_result() : success(0) {
+  ThriftHiveMetastore_get_latest_txn_in_conflict_result(const ThriftHiveMetastore_get_latest_txn_in_conflict_result&);
+  ThriftHiveMetastore_get_latest_txn_in_conflict_result& operator=(const ThriftHiveMetastore_get_latest_txn_in_conflict_result&);
+  ThriftHiveMetastore_get_latest_txn_in_conflict_result() : success(0) {
   }
 
-  virtual ~ThriftHiveMetastore_get_latest_txnid_in_conflict_result() noexcept;
+  virtual ~ThriftHiveMetastore_get_latest_txn_in_conflict_result() noexcept;
   int64_t success;
   MetaException o1;
 
-  _ThriftHiveMetastore_get_latest_txnid_in_conflict_result__isset __isset;
+  _ThriftHiveMetastore_get_latest_txn_in_conflict_result__isset __isset;
 
   void __set_success(const int64_t val);
 
   void __set_o1(const MetaException& val);
 
-  bool operator == (const ThriftHiveMetastore_get_latest_txnid_in_conflict_result & rhs) const
+  bool operator == (const ThriftHiveMetastore_get_latest_txn_in_conflict_result & rhs) const
   {
     if (!(success == rhs.success))
       return false;
@@ -21999,32 +21999,32 @@ class ThriftHiveMetastore_get_latest_txnid_in_conflict_result {
       return false;
     return true;
   }
-  bool operator != (const ThriftHiveMetastore_get_latest_txnid_in_conflict_result &rhs) const {
+  bool operator != (const ThriftHiveMetastore_get_latest_txn_in_conflict_result &rhs) const {
     return !(*this == rhs);
   }
 
-  bool operator < (const ThriftHiveMetastore_get_latest_txnid_in_conflict_result & ) const;
+  bool operator < (const ThriftHiveMetastore_get_latest_txn_in_conflict_result & ) const;
 
   uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
   uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
 
 };
 
-typedef struct _ThriftHiveMetastore_get_latest_txnid_in_conflict_presult__isset {
-  _ThriftHiveMetastore_get_latest_txnid_in_conflict_presult__isset() : success(false), o1(false) {}
+typedef struct _ThriftHiveMetastore_get_latest_txn_in_conflict_presult__isset {
+  _ThriftHiveMetastore_get_latest_txn_in_conflict_presult__isset() : success(false), o1(false) {}
   bool success :1;
   bool o1 :1;
-} _ThriftHiveMetastore_get_latest_txnid_in_conflict_presult__isset;
+} _ThriftHiveMetastore_get_latest_txn_in_conflict_presult__isset;
 
-class ThriftHiveMetastore_get_latest_txnid_in_conflict_presult {
+class ThriftHiveMetastore_get_latest_txn_in_conflict_presult {
  public:
 
 
-  virtual ~ThriftHiveMetastore_get_latest_txnid_in_conflict_presult() noexcept;
+  virtual ~ThriftHiveMetastore_get_latest_txn_in_conflict_presult() noexcept;
   int64_t* success;
   MetaException o1;
 
-  _ThriftHiveMetastore_get_latest_txnid_in_conflict_presult__isset __isset;
+  _ThriftHiveMetastore_get_latest_txn_in_conflict_presult__isset __isset;
 
   uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
 
@@ -31474,9 +31474,9 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public
   void commit_txn(const CommitTxnRequest& rqst);
   void send_commit_txn(const CommitTxnRequest& rqst);
   void recv_commit_txn();
-  int64_t get_latest_txnid_in_conflict(const int64_t txnId);
-  void send_get_latest_txnid_in_conflict(const int64_t txnId);
-  int64_t recv_get_latest_txnid_in_conflict();
+  int64_t get_latest_txn_in_conflict(const int64_t txnId);
+  void send_get_latest_txn_in_conflict(const int64_t txnId);
+  int64_t recv_get_latest_txn_in_conflict();
   void repl_tbl_writeid_state(const ReplTblWriteIdStateRequest& rqst);
   void send_repl_tbl_writeid_state(const ReplTblWriteIdStateRequest& rqst);
   void recv_repl_tbl_writeid_state();
@@ -31891,7 +31891,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
   void process_abort_txn(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_abort_txns(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_commit_txn(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
-  void process_get_latest_txnid_in_conflict(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
+  void process_get_latest_txn_in_conflict(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_repl_tbl_writeid_state(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_valid_write_ids(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_allocate_table_write_ids(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
@@ -32142,7 +32142,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
     processMap_["abort_txn"] = &ThriftHiveMetastoreProcessor::process_abort_txn;
     processMap_["abort_txns"] = &ThriftHiveMetastoreProcessor::process_abort_txns;
     processMap_["commit_txn"] = &ThriftHiveMetastoreProcessor::process_commit_txn;
-    processMap_["get_latest_txnid_in_conflict"] = &ThriftHiveMetastoreProcessor::process_get_latest_txnid_in_conflict;
+    processMap_["get_latest_txn_in_conflict"] = &ThriftHiveMetastoreProcessor::process_get_latest_txn_in_conflict;
     processMap_["repl_tbl_writeid_state"] = &ThriftHiveMetastoreProcessor::process_repl_tbl_writeid_state;
     processMap_["get_valid_write_ids"] = &ThriftHiveMetastoreProcessor::process_get_valid_write_ids;
     processMap_["allocate_table_write_ids"] = &ThriftHiveMetastoreProcessor::process_allocate_table_write_ids;
@@ -33859,13 +33859,13 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi
     ifaces_[i]->commit_txn(rqst);
   }
 
-  int64_t get_latest_txnid_in_conflict(const int64_t txnId) {
+  int64_t get_latest_txn_in_conflict(const int64_t txnId) {
     size_t sz = ifaces_.size();
     size_t i = 0;
     for (; i < (sz - 1); ++i) {
-      ifaces_[i]->get_latest_txnid_in_conflict(txnId);
+      ifaces_[i]->get_latest_txn_in_conflict(txnId);
     }
-    return ifaces_[i]->get_latest_txnid_in_conflict(txnId);
+    return ifaces_[i]->get_latest_txn_in_conflict(txnId);
   }
 
   void repl_tbl_writeid_state(const ReplTblWriteIdStateRequest& rqst) {
@@ -35149,9 +35149,9 @@ class ThriftHiveMetastoreConcurrentClient : virtual public ThriftHiveMetastoreIf
   void commit_txn(const CommitTxnRequest& rqst);
   int32_t send_commit_txn(const CommitTxnRequest& rqst);
   void recv_commit_txn(const int32_t seqid);
-  int64_t get_latest_txnid_in_conflict(const int64_t txnId);
-  int32_t send_get_latest_txnid_in_conflict(const int64_t txnId);
-  int64_t recv_get_latest_txnid_in_conflict(const int32_t seqid);
+  int64_t get_latest_txn_in_conflict(const int64_t txnId);
+  int32_t send_get_latest_txn_in_conflict(const int64_t txnId);
+  int64_t recv_get_latest_txn_in_conflict(const int32_t seqid);
   void repl_tbl_writeid_state(const ReplTblWriteIdStateRequest& rqst);
   int32_t send_repl_tbl_writeid_state(const ReplTblWriteIdStateRequest& rqst);
   void recv_repl_tbl_writeid_state(const int32_t seqid);
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
index cae7df9..ebadac7 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
@@ -855,9 +855,9 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("commit_txn\n");
   }
 
-  int64_t get_latest_txnid_in_conflict(const int64_t txnId) {
+  int64_t get_latest_txn_in_conflict(const int64_t txnId) {
     // Your implementation goes here
-    printf("get_latest_txnid_in_conflict\n");
+    printf("get_latest_txn_in_conflict\n");
   }
 
   void repl_tbl_writeid_state(const ReplTblWriteIdStateRequest& rqst) {
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index f845a3e..e762ab9 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@ -349,7 +349,7 @@ package org.apache.hadoop.hive.metastore.api;
 
     public void commit_txn(CommitTxnRequest rqst) throws NoSuchTxnException, TxnAbortedException, org.apache.thrift.TException;
 
-    public long get_latest_txnid_in_conflict(long txnId) throws MetaException, org.apache.thrift.TException;
+    public long get_latest_txn_in_conflict(long txnId) throws MetaException, org.apache.thrift.TException;
 
     public void repl_tbl_writeid_state(ReplTblWriteIdStateRequest rqst) throws org.apache.thrift.TException;
 
@@ -847,7 +847,7 @@ package org.apache.hadoop.hive.metastore.api;
 
     public void commit_txn(CommitTxnRequest rqst, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void get_latest_txnid_in_conflict(long txnId, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException;
+    public void get_latest_txn_in_conflict(long txnId, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException;
 
     public void repl_tbl_writeid_state(ReplTblWriteIdStateRequest rqst, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
@@ -5925,30 +5925,30 @@ package org.apache.hadoop.hive.metastore.api;
       return;
     }
 
-    public long get_latest_txnid_in_conflict(long txnId) throws MetaException, org.apache.thrift.TException
+    public long get_latest_txn_in_conflict(long txnId) throws MetaException, org.apache.thrift.TException
     {
-      send_get_latest_txnid_in_conflict(txnId);
-      return recv_get_latest_txnid_in_conflict();
+      send_get_latest_txn_in_conflict(txnId);
+      return recv_get_latest_txn_in_conflict();
     }
 
-    public void send_get_latest_txnid_in_conflict(long txnId) throws org.apache.thrift.TException
+    public void send_get_latest_txn_in_conflict(long txnId) throws org.apache.thrift.TException
     {
-      get_latest_txnid_in_conflict_args args = new get_latest_txnid_in_conflict_args();
+      get_latest_txn_in_conflict_args args = new get_latest_txn_in_conflict_args();
       args.setTxnId(txnId);
-      sendBase("get_latest_txnid_in_conflict", args);
+      sendBase("get_latest_txn_in_conflict", args);
     }
 
-    public long recv_get_latest_txnid_in_conflict() throws MetaException, org.apache.thrift.TException
+    public long recv_get_latest_txn_in_conflict() throws MetaException, org.apache.thrift.TException
     {
-      get_latest_txnid_in_conflict_result result = new get_latest_txnid_in_conflict_result();
-      receiveBase(result, "get_latest_txnid_in_conflict");
+      get_latest_txn_in_conflict_result result = new get_latest_txn_in_conflict_result();
+      receiveBase(result, "get_latest_txn_in_conflict");
       if (result.isSetSuccess()) {
         return result.success;
       }
       if (result.o1 != null) {
         throw result.o1;
       }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_latest_txnid_in_conflict failed: unknown result");
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_latest_txn_in_conflict failed: unknown result");
     }
 
     public void repl_tbl_writeid_state(ReplTblWriteIdStateRequest rqst) throws org.apache.thrift.TException
@@ -13910,23 +13910,23 @@ package org.apache.hadoop.hive.metastore.api;
       }
     }
 
-    public void get_latest_txnid_in_conflict(long txnId, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
+    public void get_latest_txn_in_conflict(long txnId, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      get_latest_txnid_in_conflict_call method_call = new get_latest_txnid_in_conflict_call(txnId, resultHandler, this, ___protocolFactory, ___transport);
+      get_latest_txn_in_conflict_call method_call = new get_latest_txn_in_conflict_call(txnId, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_latest_txnid_in_conflict_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Long> {
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_latest_txn_in_conflict_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.Long> {
       private long txnId;
-      public get_latest_txnid_in_conflict_call(long txnId, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public get_latest_txn_in_conflict_call(long txnId, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.txnId = txnId;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_latest_txnid_in_conflict", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        get_latest_txnid_in_conflict_args args = new get_latest_txnid_in_conflict_args();
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_latest_txn_in_conflict", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        get_latest_txn_in_conflict_args args = new get_latest_txn_in_conflict_args();
         args.setTxnId(txnId);
         args.write(prot);
         prot.writeMessageEnd();
@@ -13938,7 +13938,7 @@ package org.apache.hadoop.hive.metastore.api;
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_get_latest_txnid_in_conflict();
+        return (new Client(prot)).recv_get_latest_txn_in_conflict();
       }
     }
 
@@ -16659,7 +16659,7 @@ package org.apache.hadoop.hive.metastore.api;
       processMap.put("abort_txn", new abort_txn());
       processMap.put("abort_txns", new abort_txns());
       processMap.put("commit_txn", new commit_txn());
-      processMap.put("get_latest_txnid_in_conflict", new get_latest_txnid_in_conflict());
+      processMap.put("get_latest_txn_in_conflict", new get_latest_txn_in_conflict());
       processMap.put("repl_tbl_writeid_state", new repl_tbl_writeid_state());
       processMap.put("get_valid_write_ids", new get_valid_write_ids());
       processMap.put("allocate_table_write_ids", new allocate_table_write_ids());
@@ -21930,13 +21930,13 @@ package org.apache.hadoop.hive.metastore.api;
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_latest_txnid_in_conflict<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_latest_txnid_in_conflict_args> {
-      public get_latest_txnid_in_conflict() {
-        super("get_latest_txnid_in_conflict");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_latest_txn_in_conflict<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_latest_txn_in_conflict_args> {
+      public get_latest_txn_in_conflict() {
+        super("get_latest_txn_in_conflict");
       }
 
-      public get_latest_txnid_in_conflict_args getEmptyArgsInstance() {
-        return new get_latest_txnid_in_conflict_args();
+      public get_latest_txn_in_conflict_args getEmptyArgsInstance() {
+        return new get_latest_txn_in_conflict_args();
       }
 
       protected boolean isOneway() {
@@ -21948,10 +21948,10 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public get_latest_txnid_in_conflict_result getResult(I iface, get_latest_txnid_in_conflict_args args) throws org.apache.thrift.TException {
-        get_latest_txnid_in_conflict_result result = new get_latest_txnid_in_conflict_result();
+      public get_latest_txn_in_conflict_result getResult(I iface, get_latest_txn_in_conflict_args args) throws org.apache.thrift.TException {
+        get_latest_txn_in_conflict_result result = new get_latest_txn_in_conflict_result();
         try {
-          result.success = iface.get_latest_txnid_in_conflict(args.txnId);
+          result.success = iface.get_latest_txn_in_conflict(args.txnId);
           result.setSuccessIsSet(true);
         } catch (MetaException o1) {
           result.o1 = o1;
@@ -24468,7 +24468,7 @@ package org.apache.hadoop.hive.metastore.api;
       processMap.put("abort_txn", new abort_txn());
       processMap.put("abort_txns", new abort_txns());
       processMap.put("commit_txn", new commit_txn());
-      processMap.put("get_latest_txnid_in_conflict", new get_latest_txnid_in_conflict());
+      processMap.put("get_latest_txn_in_conflict", new get_latest_txn_in_conflict());
       processMap.put("repl_tbl_writeid_state", new repl_tbl_writeid_state());
       processMap.put("get_valid_write_ids", new get_valid_write_ids());
       processMap.put("allocate_table_write_ids", new allocate_table_write_ids());
@@ -36070,20 +36070,20 @@ package org.apache.hadoop.hive.metastore.api;
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_latest_txnid_in_conflict<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_latest_txnid_in_conflict_args, java.lang.Long> {
-      public get_latest_txnid_in_conflict() {
-        super("get_latest_txnid_in_conflict");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_latest_txn_in_conflict<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_latest_txn_in_conflict_args, java.lang.Long> {
+      public get_latest_txn_in_conflict() {
+        super("get_latest_txn_in_conflict");
       }
 
-      public get_latest_txnid_in_conflict_args getEmptyArgsInstance() {
-        return new get_latest_txnid_in_conflict_args();
+      public get_latest_txn_in_conflict_args getEmptyArgsInstance() {
+        return new get_latest_txn_in_conflict_args();
       }
 
       public org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
         return new org.apache.thrift.async.AsyncMethodCallback<java.lang.Long>() { 
           public void onComplete(java.lang.Long o) {
-            get_latest_txnid_in_conflict_result result = new get_latest_txnid_in_conflict_result();
+            get_latest_txn_in_conflict_result result = new get_latest_txn_in_conflict_result();
             result.success = o;
             result.setSuccessIsSet(true);
             try {
@@ -36099,7 +36099,7 @@ package org.apache.hadoop.hive.metastore.api;
           public void onError(java.lang.Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
-            get_latest_txnid_in_conflict_result result = new get_latest_txnid_in_conflict_result();
+            get_latest_txn_in_conflict_result result = new get_latest_txn_in_conflict_result();
             if (e instanceof MetaException) {
               result.o1 = (MetaException) e;
               result.setO1IsSet(true);
@@ -36131,8 +36131,8 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, get_latest_txnid_in_conflict_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
-        iface.get_latest_txnid_in_conflict(args.txnId,resultHandler);
+      public void start(I iface, get_latest_txn_in_conflict_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.Long> resultHandler) throws org.apache.thrift.TException {
+        iface.get_latest_txn_in_conflict(args.txnId,resultHandler);
       }
     }
 
@@ -215922,13 +215922,13 @@ package org.apache.hadoop.hive.metastore.api;
     }
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_latest_txnid_in_conflict_args implements org.apache.thrift.TBase<get_latest_txnid_in_conflict_args, get_latest_txnid_in_conflict_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_latest_txnid_in_conflict_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_latest_txnid_in_conflict_args");
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_latest_txn_in_conflict_args implements org.apache.thrift.TBase<get_latest_txn_in_conflict_args, get_latest_txn_in_conflict_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_latest_txn_in_conflict_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_latest_txn_in_conflict_args");
 
     private static final org.apache.thrift.protocol.TField TXN_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("txnId", org.apache.thrift.protocol.TType.I64, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new get_latest_txnid_in_conflict_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new get_latest_txnid_in_conflict_argsTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new get_latest_txn_in_conflict_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new get_latest_txn_in_conflict_argsTupleSchemeFactory();
 
     private long txnId; // required
 
@@ -216001,13 +216001,13 @@ package org.apache.hadoop.hive.metastore.api;
       tmpMap.put(_Fields.TXN_ID, new org.apache.thrift.meta_data.FieldMetaData("txnId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_latest_txnid_in_conflict_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_latest_txn_in_conflict_args.class, metaDataMap);
     }
 
-    public get_latest_txnid_in_conflict_args() {
+    public get_latest_txn_in_conflict_args() {
     }
 
-    public get_latest_txnid_in_conflict_args(
+    public get_latest_txn_in_conflict_args(
       long txnId)
     {
       this();
@@ -216018,13 +216018,13 @@ package org.apache.hadoop.hive.metastore.api;
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public get_latest_txnid_in_conflict_args(get_latest_txnid_in_conflict_args other) {
+    public get_latest_txn_in_conflict_args(get_latest_txn_in_conflict_args other) {
       __isset_bitfield = other.__isset_bitfield;
       this.txnId = other.txnId;
     }
 
-    public get_latest_txnid_in_conflict_args deepCopy() {
-      return new get_latest_txnid_in_conflict_args(this);
+    public get_latest_txn_in_conflict_args deepCopy() {
+      return new get_latest_txn_in_conflict_args(this);
     }
 
     @Override
@@ -216095,12 +216095,12 @@ package org.apache.hadoop.hive.metastore.api;
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof get_latest_txnid_in_conflict_args)
-        return this.equals((get_latest_txnid_in_conflict_args)that);
+      if (that instanceof get_latest_txn_in_conflict_args)
+        return this.equals((get_latest_txn_in_conflict_args)that);
       return false;
     }
 
-    public boolean equals(get_latest_txnid_in_conflict_args that) {
+    public boolean equals(get_latest_txn_in_conflict_args that) {
       if (that == null)
         return false;
       if (this == that)
@@ -216128,7 +216128,7 @@ package org.apache.hadoop.hive.metastore.api;
     }
 
     @Override
-    public int compareTo(get_latest_txnid_in_conflict_args other) {
+    public int compareTo(get_latest_txn_in_conflict_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -216163,7 +216163,7 @@ package org.apache.hadoop.hive.metastore.api;
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("get_latest_txnid_in_conflict_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("get_latest_txn_in_conflict_args(");
       boolean first = true;
 
       sb.append("txnId:");
@@ -216196,15 +216196,15 @@ package org.apache.hadoop.hive.metastore.api;
       }
     }
 
-    private static class get_latest_txnid_in_conflict_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public get_latest_txnid_in_conflict_argsStandardScheme getScheme() {
-        return new get_latest_txnid_in_conflict_argsStandardScheme();
+    private static class get_latest_txn_in_conflict_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public get_latest_txn_in_conflict_argsStandardScheme getScheme() {
+        return new get_latest_txn_in_conflict_argsStandardScheme();
       }
     }
 
-    private static class get_latest_txnid_in_conflict_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<get_latest_txnid_in_conflict_args> {
+    private static class get_latest_txn_in_conflict_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<get_latest_txn_in_conflict_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, get_latest_txnid_in_conflict_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_latest_txn_in_conflict_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -216231,7 +216231,7 @@ package org.apache.hadoop.hive.metastore.api;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, get_latest_txnid_in_conflict_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, get_latest_txn_in_conflict_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -216244,16 +216244,16 @@ package org.apache.hadoop.hive.metastore.api;
 
     }
 
-    private static class get_latest_txnid_in_conflict_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public get_latest_txnid_in_conflict_argsTupleScheme getScheme() {
-        return new get_latest_txnid_in_conflict_argsTupleScheme();
+    private static class get_latest_txn_in_conflict_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public get_latest_txn_in_conflict_argsTupleScheme getScheme() {
+        return new get_latest_txn_in_conflict_argsTupleScheme();
       }
     }
 
-    private static class get_latest_txnid_in_conflict_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<get_latest_txnid_in_conflict_args> {
+    private static class get_latest_txn_in_conflict_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<get_latest_txn_in_conflict_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, get_latest_txnid_in_conflict_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_latest_txn_in_conflict_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet optionals = new java.util.BitSet();
         if (struct.isSetTxnId()) {
@@ -216266,7 +216266,7 @@ package org.apache.hadoop.hive.metastore.api;
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, get_latest_txnid_in_conflict_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_latest_txn_in_conflict_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
@@ -216281,14 +216281,14 @@ package org.apache.hadoop.hive.metastore.api;
     }
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_latest_txnid_in_conflict_result implements org.apache.thrift.TBase<get_latest_txnid_in_conflict_result, get_latest_txnid_in_conflict_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_latest_txnid_in_conflict_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_latest_txnid_in_conflict_result");
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_latest_txn_in_conflict_result implements org.apache.thrift.TBase<get_latest_txn_in_conflict_result, get_latest_txn_in_conflict_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_latest_txn_in_conflict_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_latest_txn_in_conflict_result");
 
     private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.I64, (short)0);
     private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new get_latest_txnid_in_conflict_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new get_latest_txnid_in_conflict_resultTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new get_latest_txn_in_conflict_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new get_latest_txn_in_conflict_resultTupleSchemeFactory();
 
     private long success; // required
     private @org.apache.thrift.annotation.Nullable MetaException o1; // required
@@ -216367,13 +216367,13 @@ package org.apache.hadoop.hive.metastore.api;
       tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, MetaException.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_latest_txnid_in_conflict_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_latest_txn_in_conflict_result.class, metaDataMap);
     }
 
-    public get_latest_txnid_in_conflict_result() {
+    public get_latest_txn_in_conflict_result() {
     }
 
-    public get_latest_txnid_in_conflict_result(
+    public get_latest_txn_in_conflict_result(
       long success,
       MetaException o1)
     {
@@ -216386,7 +216386,7 @@ package org.apache.hadoop.hive.metastore.api;
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public get_latest_txnid_in_conflict_result(get_latest_txnid_in_conflict_result other) {
+    public get_latest_txn_in_conflict_result(get_latest_txn_in_conflict_result other) {
       __isset_bitfield = other.__isset_bitfield;
       this.success = other.success;
       if (other.isSetO1()) {
@@ -216394,8 +216394,8 @@ package org.apache.hadoop.hive.metastore.api;
       }
     }
 
-    public get_latest_txnid_in_conflict_result deepCopy() {
-      return new get_latest_txnid_in_conflict_result(this);
+    public get_latest_txn_in_conflict_result deepCopy() {
+      return new get_latest_txn_in_conflict_result(this);
     }
 
     @Override
@@ -216504,12 +216504,12 @@ package org.apache.hadoop.hive.metastore.api;
     public boolean equals(java.lang.Object that) {
       if (that == null)
         return false;
-      if (that instanceof get_latest_txnid_in_conflict_result)
-        return this.equals((get_latest_txnid_in_conflict_result)that);
+      if (that instanceof get_latest_txn_in_conflict_result)
+        return this.equals((get_latest_txn_in_conflict_result)that);
       return false;
     }
 
-    public boolean equals(get_latest_txnid_in_conflict_result that) {
+    public boolean equals(get_latest_txn_in_conflict_result that) {
       if (that == null)
         return false;
       if (this == that)
@@ -216550,7 +216550,7 @@ package org.apache.hadoop.hive.metastore.api;
     }
 
     @Override
-    public int compareTo(get_latest_txnid_in_conflict_result other) {
+    public int compareTo(get_latest_txn_in_conflict_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -216595,7 +216595,7 @@ package org.apache.hadoop.hive.metastore.api;
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("get_latest_txnid_in_conflict_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("get_latest_txn_in_conflict_result(");
       boolean first = true;
 
       sb.append("success:");
@@ -216636,15 +216636,15 @@ package org.apache.hadoop.hive.metastore.api;
       }
     }
 
-    private static class get_latest_txnid_in_conflict_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public get_latest_txnid_in_conflict_resultStandardScheme getScheme() {
-        return new get_latest_txnid_in_conflict_resultStandardScheme();
+    private static class get_latest_txn_in_conflict_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public get_latest_txn_in_conflict_resultStandardScheme getScheme() {
+        return new get_latest_txn_in_conflict_resultStandardScheme();
       }
     }
 
-    private static class get_latest_txnid_in_conflict_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<get_latest_txnid_in_conflict_result> {
+    private static class get_latest_txn_in_conflict_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<get_latest_txn_in_conflict_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, get_latest_txnid_in_conflict_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_latest_txn_in_conflict_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -216680,7 +216680,7 @@ package org.apache.hadoop.hive.metastore.api;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, get_latest_txnid_in_conflict_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, get_latest_txn_in_conflict_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -216700,16 +216700,16 @@ package org.apache.hadoop.hive.metastore.api;
 
     }
 
-    private static class get_latest_txnid_in_conflict_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public get_latest_txnid_in_conflict_resultTupleScheme getScheme() {
-        return new get_latest_txnid_in_conflict_resultTupleScheme();
+    private static class get_latest_txn_in_conflict_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public get_latest_txn_in_conflict_resultTupleScheme getScheme() {
+        return new get_latest_txn_in_conflict_resultTupleScheme();
       }
     }
 
-    private static class get_latest_txnid_in_conflict_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<get_latest_txnid_in_conflict_result> {
+    private static class get_latest_txn_in_conflict_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<get_latest_txn_in_conflict_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, get_latest_txnid_in_conflict_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_latest_txn_in_conflict_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet optionals = new java.util.BitSet();
         if (struct.isSetSuccess()) {
@@ -216728,7 +216728,7 @@ package org.apache.hadoop.hive.metastore.api;
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, get_latest_txnid_in_conflict_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_latest_txn_in_conflict_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastoreClient.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastoreClient.php
index fefb2a1..08a2bc0 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastoreClient.php
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastoreClient.php
@@ -10932,41 +10932,41 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
         return;
     }
 
-    public function get_latest_txnid_in_conflict($txnId)
+    public function get_latest_txn_in_conflict($txnId)
     {
-        $this->send_get_latest_txnid_in_conflict($txnId);
-        return $this->recv_get_latest_txnid_in_conflict();
+        $this->send_get_latest_txn_in_conflict($txnId);
+        return $this->recv_get_latest_txn_in_conflict();
     }
 
-    public function send_get_latest_txnid_in_conflict($txnId)
+    public function send_get_latest_txn_in_conflict($txnId)
     {
-        $args = new \metastore\ThriftHiveMetastore_get_latest_txnid_in_conflict_args();
+        $args = new \metastore\ThriftHiveMetastore_get_latest_txn_in_conflict_args();
         $args->txnId = $txnId;
         $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
         if ($bin_accel) {
             thrift_protocol_write_binary(
                 $this->output_,
-                'get_latest_txnid_in_conflict',
+                'get_latest_txn_in_conflict',
                 TMessageType::CALL,
                 $args,
                 $this->seqid_,
                 $this->output_->isStrictWrite()
             );
         } else {
-            $this->output_->writeMessageBegin('get_latest_txnid_in_conflict', TMessageType::CALL, $this->seqid_);
+            $this->output_->writeMessageBegin('get_latest_txn_in_conflict', TMessageType::CALL, $this->seqid_);
             $args->write($this->output_);
             $this->output_->writeMessageEnd();
             $this->output_->getTransport()->flush();
         }
     }
 
-    public function recv_get_latest_txnid_in_conflict()
+    public function recv_get_latest_txn_in_conflict()
     {
         $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
         if ($bin_accel) {
             $result = thrift_protocol_read_binary(
                 $this->input_,
-                '\metastore\ThriftHiveMetastore_get_latest_txnid_in_conflict_result',
+                '\metastore\ThriftHiveMetastore_get_latest_txn_in_conflict_result',
                 $this->input_->isStrictRead()
             );
         } else {
@@ -10981,7 +10981,7 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
                 $this->input_->readMessageEnd();
                 throw $x;
             }
-            $result = new \metastore\ThriftHiveMetastore_get_latest_txnid_in_conflict_result();
+            $result = new \metastore\ThriftHiveMetastore_get_latest_txn_in_conflict_result();
             $result->read($this->input_);
             $this->input_->readMessageEnd();
         }
@@ -10991,7 +10991,7 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
         if ($result->o1 !== null) {
             throw $result->o1;
         }
-        throw new \Exception("get_latest_txnid_in_conflict failed: unknown result");
+        throw new \Exception("get_latest_txn_in_conflict failed: unknown result");
     }
 
     public function repl_tbl_writeid_state(\metastore\ReplTblWriteIdStateRequest $rqst)
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastoreIf.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastoreIf.php
index 5deb7fb..ba54493 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastoreIf.php
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastoreIf.php
@@ -1322,7 +1322,7 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf
      * @return int
      * @throws \metastore\MetaException
      */
-    public function get_latest_txnid_in_conflict($txnId);
+    public function get_latest_txn_in_conflict($txnId);
     /**
      * @param \metastore\ReplTblWriteIdStateRequest $rqst
      */
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_latest_txnid_in_conflict_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_latest_txn_in_conflict_args.php
similarity index 93%
rename from standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_latest_txnid_in_conflict_args.php
rename to standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_latest_txn_in_conflict_args.php
index 6fc92b4..e3fe2de 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_latest_txnid_in_conflict_args.php
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_latest_txn_in_conflict_args.php
@@ -16,7 +16,7 @@ use Thrift\Protocol\TProtocol;
 use Thrift\Protocol\TBinaryProtocolAccelerated;
 use Thrift\Exception\TApplicationException;
 
-class ThriftHiveMetastore_get_latest_txnid_in_conflict_args
+class ThriftHiveMetastore_get_latest_txn_in_conflict_args
 {
     static public $isValidate = false;
 
@@ -44,7 +44,7 @@ class ThriftHiveMetastore_get_latest_txnid_in_conflict_args
 
     public function getName()
     {
-        return 'ThriftHiveMetastore_get_latest_txnid_in_conflict_args';
+        return 'ThriftHiveMetastore_get_latest_txn_in_conflict_args';
     }
 
 
@@ -81,7 +81,7 @@ class ThriftHiveMetastore_get_latest_txnid_in_conflict_args
     public function write($output)
     {
         $xfer = 0;
-        $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_latest_txnid_in_conflict_args');
+        $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_latest_txn_in_conflict_args');
         if ($this->txnId !== null) {
             $xfer += $output->writeFieldBegin('txnId', TType::I64, 1);
             $xfer += $output->writeI64($this->txnId);
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_latest_txnid_in_conflict_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_latest_txn_in_conflict_result.php
similarity index 94%
rename from standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_latest_txnid_in_conflict_result.php
rename to standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_latest_txn_in_conflict_result.php
index 8577fe2..8c0b3cd 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_latest_txnid_in_conflict_result.php
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_latest_txn_in_conflict_result.php
@@ -16,7 +16,7 @@ use Thrift\Protocol\TProtocol;
 use Thrift\Protocol\TBinaryProtocolAccelerated;
 use Thrift\Exception\TApplicationException;
 
-class ThriftHiveMetastore_get_latest_txnid_in_conflict_result
+class ThriftHiveMetastore_get_latest_txn_in_conflict_result
 {
     static public $isValidate = false;
 
@@ -57,7 +57,7 @@ class ThriftHiveMetastore_get_latest_txnid_in_conflict_result
 
     public function getName()
     {
-        return 'ThriftHiveMetastore_get_latest_txnid_in_conflict_result';
+        return 'ThriftHiveMetastore_get_latest_txn_in_conflict_result';
     }
 
 
@@ -102,7 +102,7 @@ class ThriftHiveMetastore_get_latest_txnid_in_conflict_result
     public function write($output)
     {
         $xfer = 0;
-        $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_latest_txnid_in_conflict_result');
+        $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_latest_txn_in_conflict_result');
         if ($this->success !== null) {
             $xfer += $output->writeFieldBegin('success', TType::I64, 0);
             $xfer += $output->writeI64($this->success);
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
index f7cb638..06b7533 100755
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
@@ -191,7 +191,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
     print('  void abort_txn(AbortTxnRequest rqst)')
     print('  void abort_txns(AbortTxnsRequest rqst)')
     print('  void commit_txn(CommitTxnRequest rqst)')
-    print('  i64 get_latest_txnid_in_conflict(i64 txnId)')
+    print('  i64 get_latest_txn_in_conflict(i64 txnId)')
     print('  void repl_tbl_writeid_state(ReplTblWriteIdStateRequest rqst)')
     print('  GetValidWriteIdsResponse get_valid_write_ids(GetValidWriteIdsRequest rqst)')
     print('  AllocateTableWriteIdsResponse allocate_table_write_ids(AllocateTableWriteIdsRequest rqst)')
@@ -1365,11 +1365,11 @@ elif cmd == 'commit_txn':
         sys.exit(1)
     pp.pprint(client.commit_txn(eval(args[0]),))
 
-elif cmd == 'get_latest_txnid_in_conflict':
+elif cmd == 'get_latest_txn_in_conflict':
     if len(args) != 1:
-        print('get_latest_txnid_in_conflict requires 1 args')
+        print('get_latest_txn_in_conflict requires 1 args')
         sys.exit(1)
-    pp.pprint(client.get_latest_txnid_in_conflict(eval(args[0]),))
+    pp.pprint(client.get_latest_txn_in_conflict(eval(args[0]),))
 
 elif cmd == 'repl_tbl_writeid_state':
     if len(args) != 1:
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
index 087ffbb..2168723 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -1488,7 +1488,7 @@ class Iface(fb303.FacebookService.Iface):
         """
         pass
 
-    def get_latest_txnid_in_conflict(self, txnId):
+    def get_latest_txn_in_conflict(self, txnId):
         """
         Parameters:
          - txnId
@@ -8358,24 +8358,24 @@ class Client(fb303.FacebookService.Client, Iface):
             raise result.o2
         return
 
-    def get_latest_txnid_in_conflict(self, txnId):
+    def get_latest_txn_in_conflict(self, txnId):
         """
         Parameters:
          - txnId
 
         """
-        self.send_get_latest_txnid_in_conflict(txnId)
-        return self.recv_get_latest_txnid_in_conflict()
+        self.send_get_latest_txn_in_conflict(txnId)
+        return self.recv_get_latest_txn_in_conflict()
 
-    def send_get_latest_txnid_in_conflict(self, txnId):
-        self._oprot.writeMessageBegin('get_latest_txnid_in_conflict', TMessageType.CALL, self._seqid)
-        args = get_latest_txnid_in_conflict_args()
+    def send_get_latest_txn_in_conflict(self, txnId):
+        self._oprot.writeMessageBegin('get_latest_txn_in_conflict', TMessageType.CALL, self._seqid)
+        args = get_latest_txn_in_conflict_args()
         args.txnId = txnId
         args.write(self._oprot)
         self._oprot.writeMessageEnd()
         self._oprot.trans.flush()
 
-    def recv_get_latest_txnid_in_conflict(self):
+    def recv_get_latest_txn_in_conflict(self):
         iprot = self._iprot
         (fname, mtype, rseqid) = iprot.readMessageBegin()
         if mtype == TMessageType.EXCEPTION:
@@ -8383,14 +8383,14 @@ class Client(fb303.FacebookService.Client, Iface):
             x.read(iprot)
             iprot.readMessageEnd()
             raise x
-        result = get_latest_txnid_in_conflict_result()
+        result = get_latest_txn_in_conflict_result()
         result.read(iprot)
         iprot.readMessageEnd()
         if result.success is not None:
             return result.success
         if result.o1 is not None:
             raise result.o1
-        raise TApplicationException(TApplicationException.MISSING_RESULT, "get_latest_txnid_in_conflict failed: unknown result")
+        raise TApplicationException(TApplicationException.MISSING_RESULT, "get_latest_txn_in_conflict failed: unknown result")
 
     def repl_tbl_writeid_state(self, rqst):
         """
@@ -11275,7 +11275,7 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
         self._processMap["abort_txn"] = Processor.process_abort_txn
         self._processMap["abort_txns"] = Processor.process_abort_txns
         self._processMap["commit_txn"] = Processor.process_commit_txn
-        self._processMap["get_latest_txnid_in_conflict"] = Processor.process_get_latest_txnid_in_conflict
+        self._processMap["get_latest_txn_in_conflict"] = Processor.process_get_latest_txn_in_conflict
         self._processMap["repl_tbl_writeid_state"] = Processor.process_repl_tbl_writeid_state
         self._processMap["get_valid_write_ids"] = Processor.process_get_valid_write_ids
         self._processMap["allocate_table_write_ids"] = Processor.process_allocate_table_write_ids
@@ -16226,13 +16226,13 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
         oprot.writeMessageEnd()
         oprot.trans.flush()
 
-    def process_get_latest_txnid_in_conflict(self, seqid, iprot, oprot):
-        args = get_latest_txnid_in_conflict_args()
+    def process_get_latest_txn_in_conflict(self, seqid, iprot, oprot):
+        args = get_latest_txn_in_conflict_args()
         args.read(iprot)
         iprot.readMessageEnd()
-        result = get_latest_txnid_in_conflict_result()
+        result = get_latest_txn_in_conflict_result()
         try:
-            result.success = self._handler.get_latest_txnid_in_conflict(args.txnId)
+            result.success = self._handler.get_latest_txn_in_conflict(args.txnId)
             msg_type = TMessageType.REPLY
         except TTransport.TTransportException:
             raise
@@ -16247,7 +16247,7 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
             logging.exception('Unexpected exception in handler')
             msg_type = TMessageType.EXCEPTION
             result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-        oprot.writeMessageBegin("get_latest_txnid_in_conflict", msg_type, seqid)
+        oprot.writeMessageBegin("get_latest_txn_in_conflict", msg_type, seqid)
         result.write(oprot)
         oprot.writeMessageEnd()
         oprot.trans.flush()
@@ -45580,7 +45580,7 @@ commit_txn_result.thrift_spec = (
 )
 
 
-class get_latest_txnid_in_conflict_args(object):
+class get_latest_txn_in_conflict_args(object):
     """
     Attributes:
      - txnId
@@ -45614,7 +45614,7 @@ class get_latest_txnid_in_conflict_args(object):
         if oprot._fast_encode is not None and self.thrift_spec is not None:
             oprot.trans.write(oprot._fast_encode(self, [self.__class__, self.thrift_spec]))
             return
-        oprot.writeStructBegin('get_latest_txnid_in_conflict_args')
+        oprot.writeStructBegin('get_latest_txn_in_conflict_args')
         if self.txnId is not None:
             oprot.writeFieldBegin('txnId', TType.I64, 1)
             oprot.writeI64(self.txnId)
@@ -45635,14 +45635,14 @@ class get_latest_txnid_in_conflict_args(object):
 
     def __ne__(self, other):
         return not (self == other)
-all_structs.append(get_latest_txnid_in_conflict_args)
-get_latest_txnid_in_conflict_args.thrift_spec = (
+all_structs.append(get_latest_txn_in_conflict_args)
+get_latest_txn_in_conflict_args.thrift_spec = (
     None,  # 0
     (1, TType.I64, 'txnId', None, None, ),  # 1
 )
 
 
-class get_latest_txnid_in_conflict_result(object):
+class get_latest_txn_in_conflict_result(object):
     """
     Attributes:
      - success
@@ -45684,7 +45684,7 @@ class get_latest_txnid_in_conflict_result(object):
         if oprot._fast_encode is not None and self.thrift_spec is not None:
             oprot.trans.write(oprot._fast_encode(self, [self.__class__, self.thrift_spec]))
             return
-        oprot.writeStructBegin('get_latest_txnid_in_conflict_result')
+        oprot.writeStructBegin('get_latest_txn_in_conflict_result')
         if self.success is not None:
             oprot.writeFieldBegin('success', TType.I64, 0)
             oprot.writeI64(self.success)
@@ -45709,8 +45709,8 @@ class get_latest_txnid_in_conflict_result(object):
 
     def __ne__(self, other):
         return not (self == other)
-all_structs.append(get_latest_txnid_in_conflict_result)
-get_latest_txnid_in_conflict_result.thrift_spec = (
+all_structs.append(get_latest_txn_in_conflict_result)
+get_latest_txn_in_conflict_result.thrift_spec = (
     (0, TType.I64, 'success', None, None, ),  # 0
     (1, TType.STRUCT, 'o1', [MetaException, None], None, ),  # 1
 )
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/thrift_hive_metastore.rb b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
index 7cd6cd9..abcc25b 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
@@ -2814,20 +2814,20 @@ module ThriftHiveMetastore
       return
     end
 
-    def get_latest_txnid_in_conflict(txnId)
-      send_get_latest_txnid_in_conflict(txnId)
-      return recv_get_latest_txnid_in_conflict()
+    def get_latest_txn_in_conflict(txnId)
+      send_get_latest_txn_in_conflict(txnId)
+      return recv_get_latest_txn_in_conflict()
     end
 
-    def send_get_latest_txnid_in_conflict(txnId)
-      send_message('get_latest_txnid_in_conflict', Get_latest_txnid_in_conflict_args, :txnId => txnId)
+    def send_get_latest_txn_in_conflict(txnId)
+      send_message('get_latest_txn_in_conflict', Get_latest_txn_in_conflict_args, :txnId => txnId)
     end
 
-    def recv_get_latest_txnid_in_conflict()
-      result = receive_message(Get_latest_txnid_in_conflict_result)
+    def recv_get_latest_txn_in_conflict()
+      result = receive_message(Get_latest_txn_in_conflict_result)
       return result.success unless result.success.nil?
       raise result.o1 unless result.o1.nil?
-      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_latest_txnid_in_conflict failed: unknown result')
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_latest_txn_in_conflict failed: unknown result')
     end
 
     def repl_tbl_writeid_state(rqst)
@@ -6270,15 +6270,15 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'commit_txn', seqid)
     end
 
-    def process_get_latest_txnid_in_conflict(seqid, iprot, oprot)
-      args = read_args(iprot, Get_latest_txnid_in_conflict_args)
-      result = Get_latest_txnid_in_conflict_result.new()
+    def process_get_latest_txn_in_conflict(seqid, iprot, oprot)
+      args = read_args(iprot, Get_latest_txn_in_conflict_args)
+      result = Get_latest_txn_in_conflict_result.new()
       begin
-        result.success = @handler.get_latest_txnid_in_conflict(args.txnId)
+        result.success = @handler.get_latest_txn_in_conflict(args.txnId)
       rescue ::MetaException => o1
         result.o1 = o1
       end
-      write_result(result, oprot, 'get_latest_txnid_in_conflict', seqid)
+      write_result(result, oprot, 'get_latest_txn_in_conflict', seqid)
     end
 
     def process_repl_tbl_writeid_state(seqid, iprot, oprot)
@@ -13486,7 +13486,7 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
-  class Get_latest_txnid_in_conflict_args
+  class Get_latest_txn_in_conflict_args
     include ::Thrift::Struct, ::Thrift::Struct_Union
     TXNID = 1
 
@@ -13502,7 +13502,7 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
-  class Get_latest_txnid_in_conflict_result
+  class Get_latest_txn_in_conflict_result
     include ::Thrift::Struct, ::Thrift::Struct_Union
     SUCCESS = 0
     O1 = 1
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index 918d188..f22a1a3 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -3870,8 +3870,8 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   }
 
   @Override
-  public long getLatestTxnIdInConflict(long txnId) throws TException {
-    return client.get_latest_txnid_in_conflict(txnId);
+  public long getLatestTxnInConflict(long txnId) throws TException {
+    return client.get_latest_txn_in_conflict(txnId);
   }
 
   @InterfaceAudience.LimitedPrivate({"HCatalog"})
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index af16458..f0b8319 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -3401,7 +3401,7 @@ public interface IMetaStoreClient {
    */
   void insertTable(Table table, boolean overwrite) throws MetaException;
 
-  long getLatestTxnIdInConflict(long txnId) throws TException;
+  long getLatestTxnInConflict(long txnId) throws TException;
 
   /**
    * A filter provided by the client that determines if a given notification event should be
diff --git a/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift b/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
index eca2316..99731ff 100644
--- a/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
+++ b/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
@@ -2682,7 +2682,7 @@ PartitionsResponse get_partitions_req(1:PartitionsRequest req)
   void abort_txn(1:AbortTxnRequest rqst) throws (1:NoSuchTxnException o1)
   void abort_txns(1:AbortTxnsRequest rqst) throws (1:NoSuchTxnException o1)
   void commit_txn(1:CommitTxnRequest rqst) throws (1:NoSuchTxnException o1, 2:TxnAbortedException o2)
-  i64 get_latest_txnid_in_conflict(1:i64 txnId) throws (1:MetaException o1)
+  i64 get_latest_txn_in_conflict(1:i64 txnId) throws (1:MetaException o1)
   void repl_tbl_writeid_state(1: ReplTblWriteIdStateRequest rqst)
   GetValidWriteIdsResponse get_valid_write_ids(1:GetValidWriteIdsRequest rqst)
       throws (1:NoSuchTxnException o1, 2:MetaException o2)
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index a159db8..4a249fb 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -8405,8 +8405,8 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     }
 
     @Override
-    public long get_latest_txnid_in_conflict(long txnId) throws MetaException {
-      return getTxnHandler().getLatestTxnIdInConflict(txnId);
+    public long get_latest_txn_in_conflict(long txnId) throws MetaException {
+      return getTxnHandler().getLatestTxnInConflict(txnId);
     }
 
     @Override
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/CompactionInfo.java
index 80793c9..828d755 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/CompactionInfo.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hive.metastore.txn;
 import org.apache.hadoop.hive.common.ValidCompactorWriteIdList;
 import org.apache.hadoop.hive.metastore.api.CompactionInfoStruct;
 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;
 
@@ -152,7 +151,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 {
+  static CompactionInfo loadFullFromCompactionQueue(ResultSet rs) throws SQLException {
     CompactionInfo fullCi = new CompactionInfo();
     fullCi.id = rs.getLong(1);
     fullCi.dbname = rs.getString(2);
@@ -171,7 +170,7 @@ public class CompactionInfo implements Comparable<CompactionInfo> {
     fullCi.enqueueTime = rs.getLong(15);
     return fullCi;
   }
-  static void insertIntoCompletedCompactions(PreparedStatement pStmt, CompactionInfo ci, long endTime) throws SQLException, MetaException {
+  static void insertIntoCompletedCompactions(PreparedStatement pStmt, CompactionInfo ci, long endTime) throws SQLException {
     pStmt.setLong(1, ci.id);
     pStmt.setString(2, ci.dbname);
     pStmt.setString(3, ci.tableName);
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 5c6c94c..6d7e9b9 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
@@ -77,7 +77,8 @@ class CompactionTxnHandler extends TxnHandler {
         dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
         stmt = dbConn.createStatement();
         // Check for completed transactions
-        final String s = "SELECT DISTINCT \"TC\".\"CTC_DATABASE\", \"TC\".\"CTC_TABLE\", \"TC\".\"CTC_PARTITION\" " +
+        final String s = "SELECT DISTINCT \"TC\".\"CTC_DATABASE\", \"TC\".\"CTC_TABLE\", \"TC\"" +
+            ".\"CTC_PARTITION\" " +
           "FROM \"COMPLETED_TXN_COMPONENTS\" \"TC\" " + (checkInterval > 0 ?
           "LEFT JOIN ( " +
           "  SELECT \"C1\".* FROM \"COMPLETED_COMPACTIONS\" \"C1\" " +
@@ -106,11 +107,12 @@ class CompactionTxnHandler extends TxnHandler {
         // Check for aborted txns: number of aborted txns past threshold and age of aborted txns
         // past time threshold
         boolean checkAbortedTimeThreshold = abortedTimeThreshold >= 0;
-        String sCheckAborted = "SELECT \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\", " +
-          "MIN(\"TXN_STARTED\"), COUNT(*) FROM \"TXNS\", \"TXN_COMPONENTS\" " +
-          "   WHERE \"TXN_ID\" = \"TC_TXNID\" AND \"TXN_STATE\" = " + TxnStatus.ABORTED + " " +
-          "GROUP BY \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" " +
-              (checkAbortedTimeThreshold ? "" : " HAVING COUNT(*) > " + abortedThreshold);
+        final String sCheckAborted = "SELECT \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\","
+            + "MIN(\"TXN_STARTED\"), COUNT(*)"
+            + "FROM \"TXNS\", \"TXN_COMPONENTS\" "
+            + "WHERE \"TXN_ID\" = \"TC_TXNID\" AND \"TXN_STATE\" = " + TxnStatus.ABORTED + " "
+            + "GROUP BY \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\""
+            + (checkAbortedTimeThreshold ? "" : " HAVING COUNT(*) > " + abortedThreshold);
 
         LOG.debug("Going to execute query <" + sCheckAborted + ">");
         rs = stmt.executeQuery(sCheckAborted);
@@ -132,6 +134,9 @@ class CompactionTxnHandler extends TxnHandler {
             response.add(info);
           }
         }
+
+        LOG.debug("Going to rollback");
+        dbConn.rollback();
       } catch (SQLException e) {
         LOG.error("Unable to connect to transaction database " + e.getMessage());
         checkRetryable(dbConn, e,
@@ -281,19 +286,22 @@ class CompactionTxnHandler extends TxnHandler {
       try {
         dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
         stmt = dbConn.createStatement();
-        String s = "SELECT \"CQ_ID\", \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\", " +
-            "\"CQ_TYPE\", \"CQ_RUN_AS\", \"CQ_HIGHEST_WRITE_ID\" FROM \"COMPACTION_QUEUE\" " +
-            "WHERE \"CQ_STATE\" = '" + READY_FOR_CLEANING + "'";
+        String s = "SELECT \"CQ_ID\", \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\", "
+                + "\"CQ_TYPE\", \"CQ_RUN_AS\", \"CQ_HIGHEST_WRITE_ID\" FROM \"COMPACTION_QUEUE\" WHERE \"CQ_STATE\" = '"
+                + READY_FOR_CLEANING + "'";
         LOG.debug("Going to execute query <" + s + ">");
         rs = stmt.executeQuery(s);
-
         while (rs.next()) {
           CompactionInfo info = new CompactionInfo();
           info.id = rs.getLong(1);
           info.dbname = rs.getString(2);
           info.tableName = rs.getString(3);
           info.partName = rs.getString(4);
-          info.type = dbCompactionType2ThriftType(rs.getString(5).charAt(0));
+          switch (rs.getString(5).charAt(0)) {
+            case MAJOR_TYPE: info.type = CompactionType.MAJOR; break;
+            case MINOR_TYPE: info.type = CompactionType.MINOR; break;
+            default: throw new MetaException("Unexpected compaction type " + rs.getString(5));
+          }
           info.runAs = rs.getString(6);
           info.highestWriteId = rs.getLong(7);
           if (LOG.isDebugEnabled()) {
@@ -301,6 +309,8 @@ class CompactionTxnHandler extends TxnHandler {
           }
           rc.add(info);
         }
+        LOG.debug("Going to rollback");
+        dbConn.rollback();
         return rc;
       } catch (SQLException e) {
         LOG.error("Unable to select next element for cleaning, " + e.getMessage());
@@ -342,14 +352,14 @@ class CompactionTxnHandler extends TxnHandler {
             + "FROM \"COMPACTION_QUEUE\" WHERE \"CQ_ID\" = ?");
         pStmt.setLong(1, info.id);
         rs = pStmt.executeQuery();
-        if (rs.next()) {
+        if(rs.next()) {
           info = CompactionInfo.loadFullFromCompactionQueue(rs);
         }
         else {
           throw new IllegalStateException("No record with CQ_ID=" + info.id + " found in COMPACTION_QUEUE");
         }
         close(rs);
-        String s = "DELETE FROM \"COMPACTION_QUEUE\" WHERE \"CQ_ID\" = ?";
+        String s = "delete from \"COMPACTION_QUEUE\" where \"CQ_ID\" = ?";
         pStmt = dbConn.prepareStatement(s);
         pStmt.setLong(1, info.id);
         LOG.debug("Going to execute update <" + s + ">");
@@ -374,7 +384,8 @@ class CompactionTxnHandler extends TxnHandler {
         // Remove entries from completed_txn_components as well, so we don't start looking there
         // again but only up to the highest write ID include in this compaction job.
         //highestWriteId will be NULL in upgrade scenarios
-        s = "DELETE FROM \"COMPLETED_TXN_COMPONENTS\" WHERE \"CTC_DATABASE\" = ? AND \"CTC_TABLE\" = ?";
+        s = "DELETE FROM \"COMPLETED_TXN_COMPONENTS\" WHERE \"CTC_DATABASE\" = ? AND " +
+            "\"CTC_TABLE\" = ?";
         if (info.partName != null) {
           s += " AND \"CTC_PARTITION\" = ?";
         }
@@ -389,11 +400,11 @@ class CompactionTxnHandler extends TxnHandler {
           pStmt.setString(paramCount++, info.partName);
         }
         if(info.highestWriteId != 0) {
-          pStmt.setLong(paramCount, info.highestWriteId);
+          pStmt.setLong(paramCount++, info.highestWriteId);
         }
         LOG.debug("Going to execute update <" + s + ">");
         if ((updCount = pStmt.executeUpdate()) < 1) {
-          LOG.warn("Expected to remove at least one row from completed_txn_components when " +
+          LOG.error("Expected to remove at least one row from completed_txn_components when " +
             "marking compaction entry as clean!");
         }
         LOG.debug("Removed " + updCount + " records from completed_txn_components");
@@ -403,30 +414,76 @@ class CompactionTxnHandler extends TxnHandler {
          * aborted TXN_COMPONENTS above tc_writeid (and consequently about aborted txns).
          * See {@link ql.txn.compactor.Cleaner.removeFiles()}
          */
-        s = "DELETE FROM \"TXN_COMPONENTS\" WHERE \"TC_TXNID\" IN (" +
-            "   SELECT \"TXN_ID\" FROM \"TXNS\" WHERE \"TXN_STATE\" = " + TxnStatus.ABORTED + ") " +
-            "AND \"TC_DATABASE\" = ? AND \"TC_TABLE\" = ? " +
-            "AND \"TC_PARTITION\" "+ (info.partName != null ? "= ?" : "IS NULL");
-        if (info.highestWriteId != 0) {
-          s += " AND \"TC_WRITEID\" <= ?";
-        }
-        LOG.debug("Going to execute update <" + s + ">");
+        s = "SELECT DISTINCT \"TXN_ID\" FROM \"TXNS\", \"TXN_COMPONENTS\" WHERE \"TXN_ID\" = \"TC_TXNID\" "
+            + "AND \"TXN_STATE\" = " + TxnStatus.ABORTED + " AND \"TC_DATABASE\" = ? AND \"TC_TABLE\" = ?";
+        if (info.highestWriteId != 0) s += " AND \"TC_WRITEID\" <= ?";
+        if (info.partName != null) s += " AND \"TC_PARTITION\" = ?";
+
         pStmt = dbConn.prepareStatement(s);
         paramCount = 1;
-
         pStmt.setString(paramCount++, info.dbname);
         pStmt.setString(paramCount++, info.tableName);
+        if(info.highestWriteId != 0) {
+          pStmt.setLong(paramCount++, info.highestWriteId);
+        }
         if (info.partName != null) {
           pStmt.setString(paramCount++, info.partName);
         }
-        if (info.highestWriteId != 0) {
-          pStmt.setLong(paramCount, info.highestWriteId);
+        LOG.debug("Going to execute update <" + s + ">");
+        rs = pStmt.executeQuery();
+        List<Long> txnids = new ArrayList<>();
+        List<String> questions = new ArrayList<>();
+        while (rs.next()) {
+          long id = rs.getLong(1);
+          txnids.add(id);
+          questions.add("?");
+        }
+        // Remove entries from txn_components, as there may be aborted txn components
+        if (txnids.size() > 0) {
+          List<String> queries = new ArrayList<>();
+
+          // Prepare prefix and suffix
+          StringBuilder prefix = new StringBuilder();
+          StringBuilder suffix = new StringBuilder();
+
+          prefix.append("DELETE FROM \"TXN_COMPONENTS\" WHERE ");
+
+          //because 1 txn may include different partitions/tables even in auto commit mode
+          suffix.append(" AND \"TC_DATABASE\" = ?");
+          suffix.append(" AND \"TC_TABLE\" = ?");
+          if (info.partName != null) {
+            suffix.append(" AND \"TC_PARTITION\" = ?");
+          }
+
+          // Populate the complete query with provided prefix and suffix
+          List<Integer> counts = TxnUtils
+              .buildQueryWithINClauseStrings(conf, queries, prefix, suffix, questions, "\"TC_TXNID\"",
+                  true, false);
+          int totalCount = 0;
+          for (int i = 0; i < queries.size(); i++) {
+            String query = queries.get(i);
+            int insertCount = counts.get(i);
+
+            LOG.debug("Going to execute update <" + query + ">");
+            pStmt = dbConn.prepareStatement(query);
+            for (int j = 0; j < insertCount; j++) {
+              pStmt.setLong(j + 1, txnids.get(totalCount + j));
+            }
+            totalCount += insertCount;
+            paramCount = insertCount + 1;
+            pStmt.setString(paramCount++, info.dbname);
+            pStmt.setString(paramCount++, info.tableName);
+            if (info.partName != null) {
+              pStmt.setString(paramCount++, info.partName);
+            }
+            int rc = pStmt.executeUpdate();
+            LOG.debug("Removed " + rc + " records from txn_components");
+
+            // Don't bother cleaning from the txns table.  A separate call will do that.  We don't
+            // know here which txns still have components from other tables or partitions in the
+            // table, so we don't know which ones we can and cannot clean.
+          }
         }
-        int rc = pStmt.executeUpdate();
-        LOG.debug("Removed " + rc + " records from txn_components");
-        // Don't bother cleaning from the txns table.  A separate call will do that.  We don't
-        // know here which txns still have components from other tables or partitions in the
-        // table, so we don't know which ones we can and cannot clean.
 
         LOG.debug("Going to commit");
         dbConn.commit();
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 0225db2..e541dc7 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
@@ -51,8 +51,6 @@ 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;
 
@@ -252,21 +250,20 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
       "\"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 (?, ?, ?, ?, ?, ?)";
+          "\"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";
+      "WHERE \"TC_TXNID\" = ? 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 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)";
+          "(\"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,?,?,?)";
+      "(\"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\" = ?";
@@ -1268,7 +1265,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
           raiseTxnUnexpectedState(actualTxnStatus, txnid);
         }
 
-        String conflictSQLSuffix = "FROM \"TXN_COMPONENTS\" WHERE \"TC_TXNID\"=" + txnid + " AND \"TC_OPERATION_TYPE\" IN (" +
+        String conflictSQLSuffix = "FROM \"TXN_COMPONENTS\" WHERE \"TC_TXNID\"=" + txnid + " AND \"TC_OPERATION_TYPE\" IN(" +
                 OperationType.UPDATE + "," + OperationType.DELETE + ")";
 
         long tempCommitId = generateTemporaryId();
@@ -1421,7 +1418,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
    * @return max Id for the conflicting transaction, if any, otherwise -1
    * @throws MetaException
    */
-  public long getLatestTxnIdInConflict(long txnid) throws MetaException {
+  public long getLatestTxnInConflict(long txnid) throws MetaException {
     Connection dbConn = null;
     Statement stmt = null;
 
@@ -1435,7 +1432,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         "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\" " +
+        "     AND \"TC_OPERATION_TYPE\" IN (" + OperationType.UPDATE + ", " + OperationType.DELETE + ")) \"CUR\" " +
         "   ON \"COMMITTED\".\"WS_DATABASE\" = \"CUR\".\"TC_DATABASE\" " +
         "     AND \"COMMITTED\".\"WS_TABLE\" = \"CUR\".\"TC_TABLE\" " +
         //For partitioned table we always track writes at partition level (never at table)
@@ -2680,16 +2677,6 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         // 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
@@ -2698,17 +2685,16 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
           if (!shouldUpdateTxnComponent(txnid, rqst, lc)) {
             continue;
           }
+
           String dbName = normalizeCase(lc.getDbname());
           String tblName = normalizeCase(lc.getTablename());
           String partName = normalizeCase(lc.getPartitionname());
-          OperationType opType = OperationType.fromDataOperationType(lc.getOperationType());
 
-          if (isDynPart.test(lc)) {
+          if (lc.isSetIsDynamicPartitionWrite() && lc.isIsDynamicPartitionWrite()) {
             partName = null;
-            if (writeIdCache.containsKey(groupKey.apply(lc))) {
+            if (writeIdCache.containsKey(Pair.of(dbName, tblName))) {
               continue;
             }
-            opType = isDynPartUpdate.contains(groupKey.apply(lc)) ? OperationType.UPDATE : OperationType.INSERT;
           }
           Optional<Long> writeId = getWriteId(writeIdCache, dbName, tblName, txnid, dbConn);
 
@@ -2716,7 +2702,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
           pstmt.setString(2, dbName);
           pstmt.setString(3, tblName);
           pstmt.setString(4, partName);
-          pstmt.setString(5, opType.getSqlConst());
+          pstmt.setString(5, OperationType.fromDataOperationType(lc.getOperationType()).getSqlConst());
           pstmt.setObject(6, writeId.orElse(null));
 
           pstmt.addBatch();
@@ -2776,9 +2762,15 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
     else {
       switch (lc.getOperationType()) {
         case INSERT:
+            /**
+             * we know this is part of DP operation and so we'll get
+             * {@link #addDynamicPartitions(AddDynamicPartitions)} call with the list
+             * of partitions actually changed.
+             */
+            return !lc.isSetIsDynamicPartitionWrite() || !lc.isIsDynamicPartitionWrite();
         case UPDATE:
         case DELETE:
-          return true;
+            return true;
         case SELECT:
           return false;
         case NO_TXN:
@@ -3371,7 +3363,20 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         }
         buf.append(INITIATED_STATE);
         buf.append("', '");
-        buf.append(thriftCompactionType2DbType(rqst.getType()));
+        switch (rqst.getType()) {
+          case MAJOR:
+            buf.append(MAJOR_TYPE);
+            break;
+
+          case MINOR:
+            buf.append(MINOR_TYPE);
+            break;
+
+          default:
+            LOG.debug("Going to rollback");
+            dbConn.rollback();
+            throw new MetaException("Unexpected compaction type " + rqst.getType().toString());
+        }
         buf.append("',");
         buf.append(getEpochFn(dbProduct));
         if (rqst.getProperties() != null) {
@@ -3452,10 +3457,11 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
           e.setTablename(rs.getString(2));
           e.setPartitionname(rs.getString(3));
           e.setState(compactorStateToResponse(rs.getString(4).charAt(0)));
-          try {
-            e.setType(dbCompactionType2ThriftType(rs.getString(5).charAt(0)));
-          } catch (MetaException ex) {
-            //do nothing to handle RU/D if we add another status
+          switch (rs.getString(5).charAt(0)) {
+            case MAJOR_TYPE: e.setType(CompactionType.MAJOR); break;
+            case MINOR_TYPE: e.setType(CompactionType.MINOR); break;
+            default:
+              //do nothing to handle RU/D if we add another status
           }
           e.setWorkerid(rs.getString(6));
           long start = rs.getLong(7);
@@ -3555,8 +3561,6 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         }
         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");
@@ -5144,25 +5148,26 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
   static String quoteChar(char c) {
     return "'" + c + "'";
   }
-
-  static CompactionType dbCompactionType2ThriftType(char dbValue) throws MetaException {
+  static CompactionType dbCompactionType2ThriftType(char dbValue) {
     switch (dbValue) {
       case MAJOR_TYPE:
         return CompactionType.MAJOR;
       case MINOR_TYPE:
         return CompactionType.MINOR;
       default:
-        throw new MetaException("Unexpected compaction type " + dbValue);
+        LOG.warn("Unexpected compaction type " + dbValue);
+        return null;
     }
   }
-  static Character thriftCompactionType2DbType(CompactionType ct) throws MetaException {
+  static Character thriftCompactionType2DbType(CompactionType ct) {
     switch (ct) {
       case MAJOR:
         return MAJOR_TYPE;
       case MINOR:
         return MINOR_TYPE;
       default:
-        throw new MetaException("Unexpected compaction type " + ct);
+        LOG.warn("Unexpected compaction type " + ct);
+        return null;
     }
   }
 
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 6cc38b2..4e300f3 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
@@ -162,7 +162,7 @@ public interface TxnStore extends Configurable {
   long getTxnIdForWriteId(String dbName, String tblName, long writeId)
       throws MetaException;
 
-  long getLatestTxnIdInConflict(long txnid) throws MetaException;
+  long getLatestTxnInConflict(long txnid) throws MetaException;
 
   LockResponse lockMaterializationRebuild(String dbName, String tableName, long txnId)
       throws MetaException;
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
index 1daad06..6ed4684 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
@@ -2635,7 +2635,7 @@ public class HiveMetaStoreClientPreCatalog implements IMetaStoreClient, AutoClos
   }
 
   @Override
-  public long getLatestTxnIdInConflict(long txnId) throws MetaException {
+  public long getLatestTxnInConflict(long txnId) throws MetaException {
     return 0;
   }