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 2023/03/02 11:10:35 UTC

[hive] branch master updated: HIVE-26704: Cleaner shouldn't be blocked by global min open txnId (Denys Kuzmenko, reviewed by Kirti Ruge, Laszlo Vegh, Sourabh Badhya)

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 c63db9dcc55 HIVE-26704: Cleaner shouldn't be blocked by global min open txnId (Denys Kuzmenko, reviewed by Kirti Ruge, Laszlo Vegh, Sourabh Badhya)
c63db9dcc55 is described below

commit c63db9dcc5570ab44e72d80b6496c48abad00e8d
Author: Denys Kuzmenko <de...@gmail.com>
AuthorDate: Thu Mar 2 13:10:22 2023 +0200

    HIVE-26704: Cleaner shouldn't be blocked by global min open txnId (Denys Kuzmenko, reviewed by Kirti Ruge, Laszlo Vegh, Sourabh Badhya)
    
    Closes #3576
---
 .../apache/hadoop/hive/ql/DriverTxnHandler.java    |    5 +-
 .../hadoop/hive/ql/lockmgr/DbTxnManager.java       |   76 +-
 .../hadoop/hive/ql/lockmgr/DummyTxnManager.java    |    5 +
 .../hadoop/hive/ql/lockmgr/HiveTxnManager.java     |    9 +
 .../hadoop/hive/ql/txn/compactor/Cleaner.java      |   47 +-
 .../hive/ql/txn/compactor/CompactorTest.java       |   17 +
 .../hadoop/hive/ql/txn/compactor/TestCleaner.java  |   26 +-
 .../hadoop/hive/ql/txn/compactor/TestCleaner2.java |    2 +-
 ....java => TestCleanerWithMinHistoryWriteId.java} |   12 +-
 .../src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp |  636 ++-
 .../src/gen/thrift/gen-cpp/ThriftHiveMetastore.h   |  133 +
 .../ThriftHiveMetastore_server.skeleton.cpp        |    5 +
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp    |   22 +
 .../src/gen/thrift/gen-cpp/hive_metastore_types.h  |   13 +-
 .../hadoop/hive/metastore/api/LockRequest.java     |  108 +-
 .../hive/metastore/api/ThriftHiveMetastore.java    | 4133 ++++++++++++--------
 .../gen/thrift/gen-php/metastore/LockRequest.php   |   24 +
 .../metastore/ThriftHiveMetastoreClient.php        |   60 +
 .../gen-php/metastore/ThriftHiveMetastoreIf.php    |    6 +
 ...etastore_add_write_ids_to_min_history_args.php} |  106 +-
 ...astore_add_write_ids_to_min_history_result.php} |   54 +-
 ...iveMetastore_find_columns_with_stats_result.php |   18 +-
 ...ThriftHiveMetastore_get_all_packages_result.php |   18 +-
 ...eMetastore_get_all_stored_procedures_result.php |   18 +-
 ...veMetastore_get_all_write_event_info_result.php |   20 +-
 ...hriftHiveMetastore_get_runtime_stats_result.php |   20 +-
 ...iveMetastore_get_schema_all_versions_result.php |   20 +-
 .../hive_metastore/ThriftHiveMetastore-remote      |    7 +
 .../gen-py/hive_metastore/ThriftHiveMetastore.py   |  294 +-
 .../src/gen/thrift/gen-py/hive_metastore/ttypes.py |   14 +-
 .../src/gen/thrift/gen-rb/hive_metastore_types.rb  |    4 +-
 .../src/gen/thrift/gen-rb/thrift_hive_metastore.rb |   60 +
 .../hadoop/hive/metastore/HiveMetaStoreClient.java |    5 +
 .../hadoop/hive/metastore/IMetaStoreClient.java    |   12 +
 .../hadoop/hive/metastore/conf/MetastoreConf.java  |    5 +-
 .../src/main/thrift/hive_metastore.thrift          |    4 +-
 .../apache/hadoop/hive/metastore/HMSHandler.java   |    5 +
 .../hadoop/hive/metastore/txn/CompactionInfo.java  |    1 +
 .../hive/metastore/txn/CompactionTxnHandler.java   |  102 +-
 .../hadoop/hive/metastore/txn/TxnHandler.java      |  121 +-
 .../apache/hadoop/hive/metastore/txn/TxnStore.java |    4 +
 .../apache/hadoop/hive/metastore/txn/TxnUtils.java |   16 +-
 .../src/main/sql/derby/hive-schema-4.0.0.derby.sql |    7 +
 .../derby/upgrade-4.0.0-alpha-2-to-4.0.0.derby.sql |    8 +
 .../src/main/sql/mssql/hive-schema-4.0.0.mssql.sql |    7 +
 .../mssql/upgrade-4.0.0-alpha-2-to-4.0.0.mssql.sql |    8 +
 .../src/main/sql/mysql/hive-schema-4.0.0.mysql.sql |    8 +
 .../mysql/upgrade-4.0.0-alpha-2-to-4.0.0.mysql.sql |    9 +
 .../main/sql/oracle/hive-schema-4.0.0.oracle.sql   |    7 +
 .../upgrade-4.0.0-alpha-2-to-4.0.0.oracle.sql      |    8 +
 .../sql/postgres/hive-schema-4.0.0.postgres.sql    |    7 +
 .../upgrade-4.0.0-alpha-2-to-4.0.0.postgres.sql    |    8 +
 .../metastore/HiveMetaStoreClientPreCatalog.java   |    5 +
 .../hadoop/hive/metastore/utils/TestTxnDbUtil.java |    1 +
 .../upgrade-3.1.3000-to-4.0.0.postgres.sql         |    8 +
 55 files changed, 4382 insertions(+), 1976 deletions(-)

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 9fdd4ef37ec..93b153886c9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/DriverTxnHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/DriverTxnHandler.java
@@ -345,13 +345,14 @@ class DriverTxnHandler {
   ValidTxnWriteIdList recordValidWriteIds() throws LockException {
     String txnString = driverContext.getConf().get(ValidTxnList.VALID_TXNS_KEY);
     if (Strings.isNullOrEmpty(txnString)) {
-      throw new IllegalStateException("calling recordValidWritsIdss() without initializing ValidTxnList " +
+      throw new IllegalStateException("calling recordValidWriteIds() without initializing ValidTxnList " +
           JavaUtils.txnIdToString(driverContext.getTxnManager().getCurrentTxnId()));
     }
 
     ValidTxnWriteIdList txnWriteIds = getTxnWriteIds(txnString);
     setValidWriteIds(txnWriteIds);
-
+    driverContext.getTxnManager().addWriteIdsToMinHistory(driverContext.getPlan(), txnWriteIds);
+    
     LOG.debug("Encoding valid txn write ids info {} txnid: {}", txnWriteIds.toString(),
         driverContext.getTxnManager().getCurrentTxnId());
     return txnWriteIds;
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 8fc77b09f0c..b60570b1ec7 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
@@ -24,6 +24,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.common.ValidTxnWriteIdList;
+import org.apache.hadoop.hive.common.ValidWriteIdList;
 import org.apache.hadoop.hive.conf.Constants;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
@@ -31,6 +32,7 @@ import org.apache.hadoop.hive.metastore.LockComponentBuilder;
 import org.apache.hadoop.hive.metastore.LockRequestBuilder;
 import org.apache.hadoop.hive.metastore.api.AbortTxnRequest;
 import org.apache.hadoop.hive.metastore.api.LockComponent;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
 import org.apache.hadoop.hive.metastore.api.LockResponse;
 import org.apache.hadoop.hive.metastore.api.LockState;
 import org.apache.hadoop.hive.metastore.api.MetaException;
@@ -79,6 +81,8 @@ import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import java.util.stream.Collectors;
 
 /**
  * An implementation of HiveTxnManager that stores the transactions in the metastore database.
@@ -395,6 +399,32 @@ public final class DbTxnManager extends HiveTxnManagerImpl {
     return false;
   }
 
+  @Override
+  public void addWriteIdsToMinHistory(QueryPlan plan, ValidTxnWriteIdList txnWriteIds) {
+    if (plan.getInputs().isEmpty()) {
+      return;
+    }
+    Map<String, Long> writeIds = plan.getInputs().stream()
+      .filter(input -> !input.isDummy() && AcidUtils.isTransactionalTable(input.getTable()))
+      .map(input -> input.getTable().getFullyQualifiedName())
+      .distinct()
+      .collect(Collectors.toMap(Function.identity(), table -> getMinOpenWriteId(txnWriteIds, table)));
+
+    if (!writeIds.isEmpty()) {
+      try {
+        getMS().addWriteIdsToMinHistory(txnId, writeIds);
+      } catch (TException | LockException e) {
+        throw new RuntimeException(ErrorMsg.METASTORE_COMMUNICATION_FAILED.getMsg(), e);
+      }
+    }
+  }
+
+  private Long getMinOpenWriteId(ValidTxnWriteIdList txnWriteIds, String table) {
+    ValidWriteIdList tableValidWriteIdList = txnWriteIds.getTableValidWriteIdList(table);
+    Long minOpenWriteId = tableValidWriteIdList.getMinOpenWriteId();
+    return minOpenWriteId != null ? minOpenWriteId : tableValidWriteIdList.getHighWatermark() + 1;
+  }
+
   /**
    * Normally client should call {@link #acquireLocks(org.apache.hadoop.hive.ql.QueryPlan, org.apache.hadoop.hive.ql.Context, String)}
    * @param isBlocking if false, the method will return immediately; thus the locks may be in LockState.WAITING
@@ -407,32 +437,22 @@ public final class DbTxnManager extends HiveTxnManagerImpl {
     getLockManager();
     verifyState(plan);
     queryId = plan.getQueryId();
-    switch (plan.getOperation()) {
-      case SET_AUTOCOMMIT:
-        /**This is here for documentation purposes.  This TM doesn't support this - only has one
-        * mode of operation documented at {@link DbTxnManager#isExplicitTransaction}*/
-        return  null;
+    
+    if (plan.getOperation() == HiveOperation.SET_AUTOCOMMIT) {
+      /**This is here for documentation purposes.  This TM doesn't support this - only has one
+       * mode of operation documented at {@link DbTxnManager#isExplicitTransaction}*/
+      return null;
     }
-
-    LockRequestBuilder rqstBuilder = new LockRequestBuilder(queryId);
-    //link queryId to txnId
     LOG.info("Setting lock request transaction to " + JavaUtils.txnIdToString(txnId) + " for queryId=" + queryId);
-    rqstBuilder.setTransactionId(txnId)
-        .setUser(username);
-
-    rqstBuilder.setZeroWaitReadEnabled(!conf.getBoolVar(HiveConf.ConfVars.TXN_OVERWRITE_X_LOCK) ||
-        !conf.getBoolVar(HiveConf.ConfVars.TXN_WRITE_X_LOCK));
 
-    // Make sure we need locks.  It's possible there's nothing to lock in
-    // this operation.
-    if(plan.getInputs().isEmpty() && plan.getOutputs().isEmpty()) {
+    // Make sure we need locks. It's possible there's nothing to lock in this operation.
+    if (plan.getInputs().isEmpty() && plan.getOutputs().isEmpty()) {
       LOG.debug("No locks needed for queryId=" + queryId);
       return null;
     }
-
-    List<LockComponent> lockComponents = AcidUtils.makeLockComponents(plan.getOutputs(), plan.getInputs(),
+    List<LockComponent> lockComponents = AcidUtils.makeLockComponents(
+        plan.getOutputs(), plan.getInputs(),
         ctx.getOperation(), conf);
-    rqstBuilder.setExclusiveCTAS(AcidUtils.isExclusiveCTAS(plan.getOutputs(), conf));
     lockComponents.addAll(getGlobalLocks(ctx.getConf()));
 
     //It's possible there's nothing to lock even if we have w/r entities.
@@ -440,10 +460,20 @@ public final class DbTxnManager extends HiveTxnManagerImpl {
       LOG.debug("No locks needed for queryId=" + queryId);
       return null;
     }
-    rqstBuilder.addLockComponents(lockComponents);
 
-    List<HiveLock> locks = new ArrayList<HiveLock>(1);
-    LockState lockState = lockMgr.lock(rqstBuilder.build(), queryId, isBlocking, locks);
+    LockRequest lockRqst = new LockRequestBuilder(queryId)
+      .setTransactionId(txnId) //link queryId to txnId
+      .setUser(username)
+      .setExclusiveCTAS(AcidUtils.isExclusiveCTAS(plan.getOutputs(), conf))
+      .setZeroWaitReadEnabled(
+          !conf.getBoolVar(HiveConf.ConfVars.TXN_OVERWRITE_X_LOCK)
+            || !conf.getBoolVar(HiveConf.ConfVars.TXN_WRITE_X_LOCK))
+      .addLockComponents(lockComponents)
+      .build();
+
+    List<HiveLock> locks = new ArrayList<>(1);
+    LockState lockState = lockMgr.lock(lockRqst, queryId, isBlocking, locks);
+    
     ctx.setHiveLocks(locks);
     return lockState;
   }
@@ -470,7 +500,7 @@ public final class DbTxnManager extends HiveTxnManagerImpl {
     }
     return globalLocks;
   }
-
+  
   /**
    * @param delay time to delay for first heartbeat
    */
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 00ee1a1fecb..fa18a30af3c 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
@@ -447,4 +447,9 @@ class DummyTxnManager extends HiveTxnManagerImpl {
   public String getQueryid() {
     return null;
   }
+
+  @Override
+  public void addWriteIdsToMinHistory(QueryPlan plan, ValidTxnWriteIdList txnWriteIds) {
+    
+  }
 }
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 b114414e9c7..8fc6dc2257c 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
@@ -365,6 +365,10 @@ public interface HiveTxnManager {
   LockResponse acquireMaterializationRebuildLock(String dbName, String tableName, long txnId)
       throws LockException;
 
+ /**
+  * Checks if there is a conflicting transaction
+  * @return latest txnId in conflict
+  */ 
  long getLatestTxnIdInConflict() throws LockException;
 
  /**
@@ -372,4 +376,9 @@ public interface HiveTxnManager {
   * @return
   */
  String getQueryid();
+
+ /**
+  * Persists txnWriteId hwm list into a backend DB to identify obsolete directories eligible for cleanup
+  */
+ void addWriteIdsToMinHistory(QueryPlan plan, ValidTxnWriteIdList txnWriteIds);
 }
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 bd7af20e7a1..509343041ac 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
@@ -140,21 +140,15 @@ public class Cleaner extends MetaStoreCompactorThread {
                     HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_DURATION_UPDATE_INTERVAL, TimeUnit.MILLISECONDS),
                     new CleanerCycleUpdater(MetricsConstants.COMPACTION_CLEANER_CYCLE_DURATION, startedAt));
           }
-
           long minOpenTxnId = txnHandler.findMinOpenTxnIdForCleaner();
-
           checkInterrupt();
 
           List<CompactionInfo> readyToClean = txnHandler.findReadyToClean(minOpenTxnId, retentionTime);
-
           checkInterrupt();
 
           if (!readyToClean.isEmpty()) {
-            long minTxnIdSeenOpen = txnHandler.findMinTxnIdSeenOpen();
-            final long cleanerWaterMark =
-                minTxnIdSeenOpen < 0 ? minOpenTxnId : Math.min(minOpenTxnId, minTxnIdSeenOpen);
-
-            LOG.info("Cleaning based on min open txn id: " + cleanerWaterMark);
+            long minTxnIdSeenOpen = Math.min(minOpenTxnId, txnHandler.findMinTxnIdSeenOpen());
+            
             List<CompletableFuture<Void>> cleanerList = new ArrayList<>();
             // For checking which compaction can be cleaned we can use the minOpenTxnId
             // However findReadyToClean will return all records that were compacted with old version of HMS
@@ -162,19 +156,21 @@ public class Cleaner extends MetaStoreCompactorThread {
             // to the clean method, to avoid cleaning up deltas needed for running queries
             // when min_history_level is finally dropped, than every HMS will commit compaction the new way
             // and minTxnIdSeenOpen can be removed and minOpenTxnId can be used instead.
-            for (CompactionInfo compactionInfo : readyToClean) {
-
+            for (CompactionInfo ci : readyToClean) {
               //Check for interruption before scheduling each compactionInfo and return if necessary
               checkInterrupt();
-
+              long cleanerWaterMark = (ci.minOpenWriteId > 0) ? ci.nextTxnId + 1 : minTxnIdSeenOpen;
+              
               CompletableFuture<Void> asyncJob =
                   CompletableFuture.runAsync(
-                          ThrowingRunnable.unchecked(() -> clean(compactionInfo, cleanerWaterMark, metricsEnabled)),
-                          cleanerExecutor)
-                      .exceptionally(t -> {
-                        LOG.error("Error clearing {}", compactionInfo.getFullPartitionName(), t);
-                        return null;
-                      });
+                      ThrowingRunnable.unchecked(() -> {
+                        LOG.info("Cleaning based on min open txn id: " + cleanerWaterMark);
+                        clean(ci, cleanerWaterMark, metricsEnabled);
+                      }), cleanerExecutor)
+                  .exceptionally(t -> {
+                    LOG.error("Error clearing {}", ci.getFullPartitionName(), t);
+                    return null;
+                  });
               cleanerList.add(asyncJob);
             }
 
@@ -212,7 +208,7 @@ public class Cleaner extends MetaStoreCompactorThread {
     }
   }
 
-  private void clean(CompactionInfo ci, long minOpenTxnGLB, boolean metricsEnabled) throws MetaException {
+  private void clean(CompactionInfo ci, long minOpenTxn, boolean metricsEnabled) throws MetaException {
     LOG.info("Starting cleaning for " + ci);
     PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
     String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_" +
@@ -266,7 +262,7 @@ public class Cleaner extends MetaStoreCompactorThread {
             ? resolveStorageDescriptor(t, p).getLocation()
             : location;
         boolean dropPartition = ci.partName != null && p == null;
-        cleanUpTask = () -> removeFiles(path, minOpenTxnGLB, ci, dropPartition);
+        cleanUpTask = () -> removeFiles(path, minOpenTxn, ci, dropPartition);
       } else {
         cleanUpTask = () -> removeFiles(location, ci);
       }
@@ -361,7 +357,7 @@ public class Cleaner extends MetaStoreCompactorThread {
     return " id=" + ci.id;
   }
 
-  private boolean removeFiles(String location, long minOpenTxnGLB, CompactionInfo ci, boolean dropPartition)
+  private boolean removeFiles(String location, long minOpenTxn, CompactionInfo ci, boolean dropPartition)
       throws Exception {
 
     if (dropPartition) {
@@ -390,11 +386,11 @@ public class Cleaner extends MetaStoreCompactorThread {
     }
 
     ValidTxnList validTxnList =
-      TxnUtils.createValidTxnListForCleaner(txnHandler.getOpenTxns(), minOpenTxnGLB);
+      TxnUtils.createValidTxnListForCleaner(txnHandler.getOpenTxns(), minOpenTxn);
     //save it so that getAcidState() sees it
     conf.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString());
     /**
-     * {@code validTxnList} is capped by minOpenTxnGLB so if
+     * {@code validTxnList} is capped by minOpenTxn so if
      * {@link AcidUtils#getAcidState(Path, Configuration, ValidWriteIdList)} sees a base/delta
      * produced by a compactor, that means every reader that could be active right now see it
      * as well.  That means if this base/delta shadows some earlier base/delta, the it will be
@@ -455,7 +451,9 @@ public class Cleaner extends MetaStoreCompactorThread {
           txnHandler);
     }
     // Make sure there are no leftovers below the compacted watermark
-    conf.set(ValidTxnList.VALID_TXNS_KEY, new ValidReadTxnList().toString());
+    if (ci.minOpenWriteId < 0) {
+      conf.set(ValidTxnList.VALID_TXNS_KEY, new ValidReadTxnList().toString());
+    }
     dir = AcidUtils.getAcidState(fs, path, conf, new ValidReaderWriteIdList(
         ci.getFullTableName(), new long[0], new BitSet(), ci.highestWriteId, Long.MAX_VALUE),
       Ref.from(false), false, dirSnapshots);
@@ -490,7 +488,8 @@ public class Cleaner extends MetaStoreCompactorThread {
     return obsoleteDirs;
   }
 
-  private boolean removeFiles(String location, CompactionInfo ci) throws IOException, MetaException {
+  private boolean removeFiles(String location, CompactionInfo ci) 
+      throws IOException, MetaException {
     String strIfPurge = ci.getProperty("ifPurge");
     boolean ifPurge = strIfPurge != null || Boolean.parseBoolean(ci.getProperty("ifPurge"));
     
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java
index 4abe5d066ed..a556c7fdf3b 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
 import org.apache.hadoop.hive.metastore.api.TxnType;
+import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.metrics.AcidMetricService;
@@ -133,6 +134,7 @@ public abstract class CompactorTest {
     MetastoreConf.setTimeVar(conf, MetastoreConf.ConfVars.TXN_OPENTXN_TIMEOUT, 2, TimeUnit.SECONDS);
     MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.COMPACTOR_INITIATOR_ON, true);
     MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.COMPACTOR_CLEANER_ON, true);
+    MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.TXN_USE_MIN_HISTORY_WRITE_ID, useMinHistoryWriteId());
     TestTxnDbUtil.setConfValues(conf);
     TestTxnDbUtil.cleanDb(conf);
     TestTxnDbUtil.prepDb(conf);
@@ -324,6 +326,14 @@ public abstract class CompactorTest {
     AllocateTableWriteIdsRequest awiRqst = new AllocateTableWriteIdsRequest(dbName, tblName);
     awiRqst.setTxnIds(rsp.getTxn_ids());
     AllocateTableWriteIdsResponse awiResp = txnHandler.allocateTableWriteIds(awiRqst);
+
+    long minOpenWriteId = Long.MAX_VALUE;
+    if (open != null && useMinHistoryWriteId()) {
+      long minOpenTxnId = open.stream().mapToLong(v -> v).min().orElse(-1);
+      minOpenWriteId = awiResp.getTxnToWriteIds().stream()
+        .filter(v -> v.getTxnId() == minOpenTxnId).map(TxnToWriteId::getWriteId)
+        .findFirst().orElse(minOpenWriteId);
+    }
     int i = 0;
     for (long tid : rsp.getTxn_ids()) {
       assert (awiResp.getTxnToWriteIds().get(i).getTxnId() == tid);
@@ -336,6 +346,9 @@ public abstract class CompactorTest {
         txnHandler.abortTxn(new AbortTxnRequest(tid));
       } else if (open == null || !open.contains(tid)) {
         txnHandler.commitTxn(new CommitTxnRequest(tid));
+      } else if (open.contains(tid) && useMinHistoryWriteId()){
+        txnHandler.addWriteIdsToMinHistory(tid, 
+          Collections.singletonMap(dbName + "." + tblName, minOpenWriteId));
       }
     }
   }
@@ -651,6 +664,10 @@ public abstract class CompactorTest {
    * are used since new (1.3) code has to be able to read old files.
    */
   abstract boolean useHive130DeltaDirName();
+  
+  protected boolean useMinHistoryWriteId() {
+    return false;
+  }
 
   String makeDeltaDirName(long minTxnId, long maxTxnId) {
     if(minTxnId != maxTxnId) {
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java
index d30169f0587..8812c0ca44f 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java
@@ -21,6 +21,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.ValidCompactorWriteIdList;
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.AllocateTableWriteIdsRequest;
+import org.apache.hadoop.hive.metastore.api.AllocateTableWriteIdsResponse;
 import org.apache.hadoop.hive.metastore.api.CommitTxnRequest;
 import org.apache.hadoop.hive.metastore.api.CompactionRequest;
 import org.apache.hadoop.hive.metastore.api.CompactionResponse;
@@ -92,8 +94,8 @@ public class TestCleaner extends CompactorTest {
   }
 
   public void testRetryAfterFailedCleanup(boolean delayEnabled) throws Exception {
-    conf.setBoolVar(HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED, delayEnabled);
-    conf.setTimeVar(HIVE_COMPACTOR_CLEANER_RETENTION_TIME, 2, TimeUnit.SECONDS);
+    HiveConf.setBoolVar(conf, HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED, delayEnabled);
+    HiveConf.setTimeVar(conf, HIVE_COMPACTOR_CLEANER_RETENTION_TIME, 2, TimeUnit.SECONDS);
     MetastoreConf.setLongVar(conf, MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS, 3);
     MetastoreConf.setTimeVar(conf, MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, 100, TimeUnit.MILLISECONDS);
     String errorMessage = "No cleanup here!";
@@ -300,6 +302,9 @@ public class TestCleaner extends CompactorTest {
     txnHandler.markCompacted(ci);
     // Open a query during compaction
     long longQuery = openTxn();
+    if (useMinHistoryWriteId()) {
+      allocateTableWriteId("default", "camtc", longQuery);
+    }
     txnHandler.commitTxn(new CommitTxnRequest(compactTxn));
 
     startCleaner();
@@ -739,6 +744,8 @@ public class TestCleaner extends CompactorTest {
     rqst.setPartitionname(partName);
     long compactTxn = compactInTxn(rqst);
     addDeltaFile(t, p, 21, 22, 2);
+    
+    txnHandler.addWriteIdsToMinHistory(1, Collections.singletonMap("default.trfcp", 23L));
     startCleaner();
 
     // make sure cleaner didn't remove anything, and cleaning is still queued
@@ -1059,7 +1066,7 @@ public class TestCleaner extends CompactorTest {
 
     List<Path> paths = getDirectories(conf, t, null);
     Assert.assertEquals(1, paths.size());
-    Assert.assertEquals("delta_0000020_0000020", paths.get(0).getName());
+    Assert.assertEquals(makeDeltaDirName(20,20), paths.get(0).getName());
   }
 
   @Test
@@ -1078,8 +1085,10 @@ public class TestCleaner extends CompactorTest {
     burnThroughTransactions(dbName, tblName, 22);
 
     // block cleaner with an open txn
-    openTxn();
-    
+    long txnId = openTxn();
+    if (useMinHistoryWriteId()) {
+      allocateTableWriteId(dbName, tblName, txnId);
+    }
     CompactionRequest rqst = new CompactionRequest(dbName, tblName, CompactionType.MINOR);
     rqst.setPartitionname(partName);
     long ctxnid = compactInTxn(rqst);
@@ -1126,5 +1135,12 @@ public class TestCleaner extends CompactorTest {
     Mockito.verify(cleaner, times(1)).resolveTable(Mockito.any());
   }
 
+  private void allocateTableWriteId(String dbName, String tblName, long txnId) throws Exception {
+    AllocateTableWriteIdsRequest awiRqst = new AllocateTableWriteIdsRequest(dbName, tblName);
+    awiRqst.setTxnIds(Collections.singletonList(txnId));
+    AllocateTableWriteIdsResponse awiResp = txnHandler.allocateTableWriteIds(awiRqst);
 
+    txnHandler.addWriteIdsToMinHistory(txnId, Collections.singletonMap(dbName + "." + tblName,
+      awiResp.getTxnToWriteIds().get(0).getWriteId()));
+  }
 }
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner2.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner2.java
index e2aeb9cdbeb..fa663e35b6c 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner2.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner2.java
@@ -23,6 +23,6 @@ package org.apache.hadoop.hive.ql.txn.compactor;
 public class TestCleaner2 extends TestCleaner {
   @Override
   boolean useHive130DeltaDirName() {
-    return false;
+    return true;
   }
 }
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner2.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleanerWithMinHistoryWriteId.java
similarity index 78%
copy from ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner2.java
copy to ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleanerWithMinHistoryWriteId.java
index e2aeb9cdbeb..279f953d9be 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner2.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleanerWithMinHistoryWriteId.java
@@ -7,7 +7,7 @@
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
  *
- *    http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -15,14 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.hadoop.hive.ql.txn.compactor;
 
-/**
- * Same as TestCleaner but tests delta file names in Hive 1.3.0 format 
- */
-public class TestCleaner2 extends TestCleaner {
+public class TestCleanerWithMinHistoryWriteId extends TestCleaner {
   @Override
-  boolean useHive130DeltaDirName() {
-    return false;
+  protected boolean useMinHistoryWriteId() {
+    return true;
   }
 }
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 e4c3444e92d..16ad566a14b 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
@@ -44553,6 +44553,242 @@ uint32_t ThriftHiveMetastore_get_valid_write_ids_presult::read(::apache::thrift:
 }
 
 
+ThriftHiveMetastore_add_write_ids_to_min_history_args::~ThriftHiveMetastore_add_write_ids_to_min_history_args() noexcept {
+}
+
+
+uint32_t ThriftHiveMetastore_add_write_ids_to_min_history_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_I64) {
+          xfer += iprot->readI64(this->txnId);
+          this->__isset.txnId = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_MAP) {
+          {
+            this->writeIds.clear();
+            uint32_t _size2458;
+            ::apache::thrift::protocol::TType _ktype2459;
+            ::apache::thrift::protocol::TType _vtype2460;
+            xfer += iprot->readMapBegin(_ktype2459, _vtype2460, _size2458);
+            uint32_t _i2462;
+            for (_i2462 = 0; _i2462 < _size2458; ++_i2462)
+            {
+              std::string _key2463;
+              xfer += iprot->readString(_key2463);
+              int64_t& _val2464 = this->writeIds[_key2463];
+              xfer += iprot->readI64(_val2464);
+            }
+            xfer += iprot->readMapEnd();
+          }
+          this->__isset.writeIds = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t ThriftHiveMetastore_add_write_ids_to_min_history_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_write_ids_to_min_history_args");
+
+  xfer += oprot->writeFieldBegin("txnId", ::apache::thrift::protocol::T_I64, 1);
+  xfer += oprot->writeI64(this->txnId);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("writeIds", ::apache::thrift::protocol::T_MAP, 2);
+  {
+    xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->writeIds.size()));
+    std::map<std::string, int64_t> ::const_iterator _iter2465;
+    for (_iter2465 = this->writeIds.begin(); _iter2465 != this->writeIds.end(); ++_iter2465)
+    {
+      xfer += oprot->writeString(_iter2465->first);
+      xfer += oprot->writeI64(_iter2465->second);
+    }
+    xfer += oprot->writeMapEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_add_write_ids_to_min_history_pargs::~ThriftHiveMetastore_add_write_ids_to_min_history_pargs() noexcept {
+}
+
+
+uint32_t ThriftHiveMetastore_add_write_ids_to_min_history_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_write_ids_to_min_history_pargs");
+
+  xfer += oprot->writeFieldBegin("txnId", ::apache::thrift::protocol::T_I64, 1);
+  xfer += oprot->writeI64((*(this->txnId)));
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("writeIds", ::apache::thrift::protocol::T_MAP, 2);
+  {
+    xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_I64, static_cast<uint32_t>((*(this->writeIds)).size()));
+    std::map<std::string, int64_t> ::const_iterator _iter2466;
+    for (_iter2466 = (*(this->writeIds)).begin(); _iter2466 != (*(this->writeIds)).end(); ++_iter2466)
+    {
+      xfer += oprot->writeString(_iter2466->first);
+      xfer += oprot->writeI64(_iter2466->second);
+    }
+    xfer += oprot->writeMapEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_add_write_ids_to_min_history_result::~ThriftHiveMetastore_add_write_ids_to_min_history_result() noexcept {
+}
+
+
+uint32_t ThriftHiveMetastore_add_write_ids_to_min_history_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o2.read(iprot);
+          this->__isset.o2 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t ThriftHiveMetastore_add_write_ids_to_min_history_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+
+  uint32_t xfer = 0;
+
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_write_ids_to_min_history_result");
+
+  if (this->__isset.o2) {
+    xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 1);
+    xfer += this->o2.write(oprot);
+    xfer += oprot->writeFieldEnd();
+  }
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_add_write_ids_to_min_history_presult::~ThriftHiveMetastore_add_write_ids_to_min_history_presult() noexcept {
+}
+
+
+uint32_t ThriftHiveMetastore_add_write_ids_to_min_history_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o2.read(iprot);
+          this->__isset.o2 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+
 ThriftHiveMetastore_allocate_table_write_ids_args::~ThriftHiveMetastore_allocate_table_write_ids_args() noexcept {
 }
 
@@ -48333,14 +48569,14 @@ uint32_t ThriftHiveMetastore_find_columns_with_stats_result::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2458;
-            ::apache::thrift::protocol::TType _etype2461;
-            xfer += iprot->readListBegin(_etype2461, _size2458);
-            this->success.resize(_size2458);
-            uint32_t _i2462;
-            for (_i2462 = 0; _i2462 < _size2458; ++_i2462)
+            uint32_t _size2467;
+            ::apache::thrift::protocol::TType _etype2470;
+            xfer += iprot->readListBegin(_etype2470, _size2467);
+            this->success.resize(_size2467);
+            uint32_t _i2471;
+            for (_i2471 = 0; _i2471 < _size2467; ++_i2471)
             {
-              xfer += iprot->readString(this->success[_i2462]);
+              xfer += iprot->readString(this->success[_i2471]);
             }
             xfer += iprot->readListEnd();
           }
@@ -48371,10 +48607,10 @@ uint32_t ThriftHiveMetastore_find_columns_with_stats_result::write(::apache::thr
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter2463;
-      for (_iter2463 = this->success.begin(); _iter2463 != this->success.end(); ++_iter2463)
+      std::vector<std::string> ::const_iterator _iter2472;
+      for (_iter2472 = this->success.begin(); _iter2472 != this->success.end(); ++_iter2472)
       {
-        xfer += oprot->writeString((*_iter2463));
+        xfer += oprot->writeString((*_iter2472));
       }
       xfer += oprot->writeListEnd();
     }
@@ -48415,14 +48651,14 @@ uint32_t ThriftHiveMetastore_find_columns_with_stats_presult::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2464;
-            ::apache::thrift::protocol::TType _etype2467;
-            xfer += iprot->readListBegin(_etype2467, _size2464);
-            (*(this->success)).resize(_size2464);
-            uint32_t _i2468;
-            for (_i2468 = 0; _i2468 < _size2464; ++_i2468)
+            uint32_t _size2473;
+            ::apache::thrift::protocol::TType _etype2476;
+            xfer += iprot->readListBegin(_etype2476, _size2473);
+            (*(this->success)).resize(_size2473);
+            uint32_t _i2477;
+            for (_i2477 = 0; _i2477 < _size2473; ++_i2477)
             {
-              xfer += iprot->readString((*(this->success))[_i2468]);
+              xfer += iprot->readString((*(this->success))[_i2477]);
             }
             xfer += iprot->readListEnd();
           }
@@ -58345,14 +58581,14 @@ uint32_t ThriftHiveMetastore_get_schema_all_versions_result::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2469;
-            ::apache::thrift::protocol::TType _etype2472;
-            xfer += iprot->readListBegin(_etype2472, _size2469);
-            this->success.resize(_size2469);
-            uint32_t _i2473;
-            for (_i2473 = 0; _i2473 < _size2469; ++_i2473)
+            uint32_t _size2478;
+            ::apache::thrift::protocol::TType _etype2481;
+            xfer += iprot->readListBegin(_etype2481, _size2478);
+            this->success.resize(_size2478);
+            uint32_t _i2482;
+            for (_i2482 = 0; _i2482 < _size2478; ++_i2482)
             {
-              xfer += this->success[_i2473].read(iprot);
+              xfer += this->success[_i2482].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -58399,10 +58635,10 @@ uint32_t ThriftHiveMetastore_get_schema_all_versions_result::write(::apache::thr
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<SchemaVersion> ::const_iterator _iter2474;
-      for (_iter2474 = this->success.begin(); _iter2474 != this->success.end(); ++_iter2474)
+      std::vector<SchemaVersion> ::const_iterator _iter2483;
+      for (_iter2483 = this->success.begin(); _iter2483 != this->success.end(); ++_iter2483)
       {
-        xfer += (*_iter2474).write(oprot);
+        xfer += (*_iter2483).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -58451,14 +58687,14 @@ uint32_t ThriftHiveMetastore_get_schema_all_versions_presult::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2475;
-            ::apache::thrift::protocol::TType _etype2478;
-            xfer += iprot->readListBegin(_etype2478, _size2475);
-            (*(this->success)).resize(_size2475);
-            uint32_t _i2479;
-            for (_i2479 = 0; _i2479 < _size2475; ++_i2479)
+            uint32_t _size2484;
+            ::apache::thrift::protocol::TType _etype2487;
+            xfer += iprot->readListBegin(_etype2487, _size2484);
+            (*(this->success)).resize(_size2484);
+            uint32_t _i2488;
+            for (_i2488 = 0; _i2488 < _size2484; ++_i2488)
             {
-              xfer += (*(this->success))[_i2479].read(iprot);
+              xfer += (*(this->success))[_i2488].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -60511,14 +60747,14 @@ uint32_t ThriftHiveMetastore_get_runtime_stats_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2480;
-            ::apache::thrift::protocol::TType _etype2483;
-            xfer += iprot->readListBegin(_etype2483, _size2480);
-            this->success.resize(_size2480);
-            uint32_t _i2484;
-            for (_i2484 = 0; _i2484 < _size2480; ++_i2484)
+            uint32_t _size2489;
+            ::apache::thrift::protocol::TType _etype2492;
+            xfer += iprot->readListBegin(_etype2492, _size2489);
+            this->success.resize(_size2489);
+            uint32_t _i2493;
+            for (_i2493 = 0; _i2493 < _size2489; ++_i2493)
             {
-              xfer += this->success[_i2484].read(iprot);
+              xfer += this->success[_i2493].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -60557,10 +60793,10 @@ uint32_t ThriftHiveMetastore_get_runtime_stats_result::write(::apache::thrift::p
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<RuntimeStat> ::const_iterator _iter2485;
-      for (_iter2485 = this->success.begin(); _iter2485 != this->success.end(); ++_iter2485)
+      std::vector<RuntimeStat> ::const_iterator _iter2494;
+      for (_iter2494 = this->success.begin(); _iter2494 != this->success.end(); ++_iter2494)
       {
-        xfer += (*_iter2485).write(oprot);
+        xfer += (*_iter2494).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -60605,14 +60841,14 @@ uint32_t ThriftHiveMetastore_get_runtime_stats_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2486;
-            ::apache::thrift::protocol::TType _etype2489;
-            xfer += iprot->readListBegin(_etype2489, _size2486);
-            (*(this->success)).resize(_size2486);
-            uint32_t _i2490;
-            for (_i2490 = 0; _i2490 < _size2486; ++_i2490)
+            uint32_t _size2495;
+            ::apache::thrift::protocol::TType _etype2498;
+            xfer += iprot->readListBegin(_etype2498, _size2495);
+            (*(this->success)).resize(_size2495);
+            uint32_t _i2499;
+            for (_i2499 = 0; _i2499 < _size2495; ++_i2499)
             {
-              xfer += (*(this->success))[_i2490].read(iprot);
+              xfer += (*(this->success))[_i2499].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -63047,14 +63283,14 @@ uint32_t ThriftHiveMetastore_get_all_stored_procedures_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2491;
-            ::apache::thrift::protocol::TType _etype2494;
-            xfer += iprot->readListBegin(_etype2494, _size2491);
-            this->success.resize(_size2491);
-            uint32_t _i2495;
-            for (_i2495 = 0; _i2495 < _size2491; ++_i2495)
+            uint32_t _size2500;
+            ::apache::thrift::protocol::TType _etype2503;
+            xfer += iprot->readListBegin(_etype2503, _size2500);
+            this->success.resize(_size2500);
+            uint32_t _i2504;
+            for (_i2504 = 0; _i2504 < _size2500; ++_i2504)
             {
-              xfer += iprot->readString(this->success[_i2495]);
+              xfer += iprot->readString(this->success[_i2504]);
             }
             xfer += iprot->readListEnd();
           }
@@ -63093,10 +63329,10 @@ uint32_t ThriftHiveMetastore_get_all_stored_procedures_result::write(::apache::t
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter2496;
-      for (_iter2496 = this->success.begin(); _iter2496 != this->success.end(); ++_iter2496)
+      std::vector<std::string> ::const_iterator _iter2505;
+      for (_iter2505 = this->success.begin(); _iter2505 != this->success.end(); ++_iter2505)
       {
-        xfer += oprot->writeString((*_iter2496));
+        xfer += oprot->writeString((*_iter2505));
       }
       xfer += oprot->writeListEnd();
     }
@@ -63141,14 +63377,14 @@ uint32_t ThriftHiveMetastore_get_all_stored_procedures_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2497;
-            ::apache::thrift::protocol::TType _etype2500;
-            xfer += iprot->readListBegin(_etype2500, _size2497);
-            (*(this->success)).resize(_size2497);
-            uint32_t _i2501;
-            for (_i2501 = 0; _i2501 < _size2497; ++_i2501)
+            uint32_t _size2506;
+            ::apache::thrift::protocol::TType _etype2509;
+            xfer += iprot->readListBegin(_etype2509, _size2506);
+            (*(this->success)).resize(_size2506);
+            uint32_t _i2510;
+            for (_i2510 = 0; _i2510 < _size2506; ++_i2510)
             {
-              xfer += iprot->readString((*(this->success))[_i2501]);
+              xfer += iprot->readString((*(this->success))[_i2510]);
             }
             xfer += iprot->readListEnd();
           }
@@ -63700,14 +63936,14 @@ uint32_t ThriftHiveMetastore_get_all_packages_result::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2502;
-            ::apache::thrift::protocol::TType _etype2505;
-            xfer += iprot->readListBegin(_etype2505, _size2502);
-            this->success.resize(_size2502);
-            uint32_t _i2506;
-            for (_i2506 = 0; _i2506 < _size2502; ++_i2506)
+            uint32_t _size2511;
+            ::apache::thrift::protocol::TType _etype2514;
+            xfer += iprot->readListBegin(_etype2514, _size2511);
+            this->success.resize(_size2511);
+            uint32_t _i2515;
+            for (_i2515 = 0; _i2515 < _size2511; ++_i2515)
             {
-              xfer += iprot->readString(this->success[_i2506]);
+              xfer += iprot->readString(this->success[_i2515]);
             }
             xfer += iprot->readListEnd();
           }
@@ -63746,10 +63982,10 @@ uint32_t ThriftHiveMetastore_get_all_packages_result::write(::apache::thrift::pr
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter2507;
-      for (_iter2507 = this->success.begin(); _iter2507 != this->success.end(); ++_iter2507)
+      std::vector<std::string> ::const_iterator _iter2516;
+      for (_iter2516 = this->success.begin(); _iter2516 != this->success.end(); ++_iter2516)
       {
-        xfer += oprot->writeString((*_iter2507));
+        xfer += oprot->writeString((*_iter2516));
       }
       xfer += oprot->writeListEnd();
     }
@@ -63794,14 +64030,14 @@ uint32_t ThriftHiveMetastore_get_all_packages_presult::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2508;
-            ::apache::thrift::protocol::TType _etype2511;
-            xfer += iprot->readListBegin(_etype2511, _size2508);
-            (*(this->success)).resize(_size2508);
-            uint32_t _i2512;
-            for (_i2512 = 0; _i2512 < _size2508; ++_i2512)
+            uint32_t _size2517;
+            ::apache::thrift::protocol::TType _etype2520;
+            xfer += iprot->readListBegin(_etype2520, _size2517);
+            (*(this->success)).resize(_size2517);
+            uint32_t _i2521;
+            for (_i2521 = 0; _i2521 < _size2517; ++_i2521)
             {
-              xfer += iprot->readString((*(this->success))[_i2512]);
+              xfer += iprot->readString((*(this->success))[_i2521]);
             }
             xfer += iprot->readListEnd();
           }
@@ -64126,14 +64362,14 @@ uint32_t ThriftHiveMetastore_get_all_write_event_info_result::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2513;
-            ::apache::thrift::protocol::TType _etype2516;
-            xfer += iprot->readListBegin(_etype2516, _size2513);
-            this->success.resize(_size2513);
-            uint32_t _i2517;
-            for (_i2517 = 0; _i2517 < _size2513; ++_i2517)
+            uint32_t _size2522;
+            ::apache::thrift::protocol::TType _etype2525;
+            xfer += iprot->readListBegin(_etype2525, _size2522);
+            this->success.resize(_size2522);
+            uint32_t _i2526;
+            for (_i2526 = 0; _i2526 < _size2522; ++_i2526)
             {
-              xfer += this->success[_i2517].read(iprot);
+              xfer += this->success[_i2526].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -64172,10 +64408,10 @@ uint32_t ThriftHiveMetastore_get_all_write_event_info_result::write(::apache::th
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<WriteEventInfo> ::const_iterator _iter2518;
-      for (_iter2518 = this->success.begin(); _iter2518 != this->success.end(); ++_iter2518)
+      std::vector<WriteEventInfo> ::const_iterator _iter2527;
+      for (_iter2527 = this->success.begin(); _iter2527 != this->success.end(); ++_iter2527)
       {
-        xfer += (*_iter2518).write(oprot);
+        xfer += (*_iter2527).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -64220,14 +64456,14 @@ uint32_t ThriftHiveMetastore_get_all_write_event_info_presult::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2519;
-            ::apache::thrift::protocol::TType _etype2522;
-            xfer += iprot->readListBegin(_etype2522, _size2519);
-            (*(this->success)).resize(_size2519);
-            uint32_t _i2523;
-            for (_i2523 = 0; _i2523 < _size2519; ++_i2523)
+            uint32_t _size2528;
+            ::apache::thrift::protocol::TType _etype2531;
+            xfer += iprot->readListBegin(_etype2531, _size2528);
+            (*(this->success)).resize(_size2528);
+            uint32_t _i2532;
+            for (_i2532 = 0; _i2532 < _size2528; ++_i2532)
             {
-              xfer += (*(this->success))[_i2523].read(iprot);
+              xfer += (*(this->success))[_i2532].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -75653,6 +75889,63 @@ void ThriftHiveMetastoreClient::recv_get_valid_write_ids(GetValidWriteIdsRespons
   throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_valid_write_ids failed: unknown result");
 }
 
+void ThriftHiveMetastoreClient::add_write_ids_to_min_history(const int64_t txnId, const std::map<std::string, int64_t> & writeIds)
+{
+  send_add_write_ids_to_min_history(txnId, writeIds);
+  recv_add_write_ids_to_min_history();
+}
+
+void ThriftHiveMetastoreClient::send_add_write_ids_to_min_history(const int64_t txnId, const std::map<std::string, int64_t> & writeIds)
+{
+  int32_t cseqid = 0;
+  oprot_->writeMessageBegin("add_write_ids_to_min_history", ::apache::thrift::protocol::T_CALL, cseqid);
+
+  ThriftHiveMetastore_add_write_ids_to_min_history_pargs args;
+  args.txnId = &txnId;
+  args.writeIds = &writeIds;
+  args.write(oprot_);
+
+  oprot_->writeMessageEnd();
+  oprot_->getTransport()->writeEnd();
+  oprot_->getTransport()->flush();
+}
+
+void ThriftHiveMetastoreClient::recv_add_write_ids_to_min_history()
+{
+
+  int32_t rseqid = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TMessageType mtype;
+
+  iprot_->readMessageBegin(fname, mtype, rseqid);
+  if (mtype == ::apache::thrift::protocol::T_EXCEPTION) {
+    ::apache::thrift::TApplicationException x;
+    x.read(iprot_);
+    iprot_->readMessageEnd();
+    iprot_->getTransport()->readEnd();
+    throw x;
+  }
+  if (mtype != ::apache::thrift::protocol::T_REPLY) {
+    iprot_->skip(::apache::thrift::protocol::T_STRUCT);
+    iprot_->readMessageEnd();
+    iprot_->getTransport()->readEnd();
+  }
+  if (fname.compare("add_write_ids_to_min_history") != 0) {
+    iprot_->skip(::apache::thrift::protocol::T_STRUCT);
+    iprot_->readMessageEnd();
+    iprot_->getTransport()->readEnd();
+  }
+  ThriftHiveMetastore_add_write_ids_to_min_history_presult result;
+  result.read(iprot_);
+  iprot_->readMessageEnd();
+  iprot_->getTransport()->readEnd();
+
+  if (result.__isset.o2) {
+    throw result.o2;
+  }
+  return;
+}
+
 void ThriftHiveMetastoreClient::allocate_table_write_ids(AllocateTableWriteIdsResponse& _return, const AllocateTableWriteIdsRequest& rqst)
 {
   send_allocate_table_write_ids(rqst);
@@ -92012,6 +92305,62 @@ void ThriftHiveMetastoreProcessor::process_get_valid_write_ids(int32_t seqid, ::
   }
 }
 
+void ThriftHiveMetastoreProcessor::process_add_write_ids_to_min_history(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext)
+{
+  void* ctx = nullptr;
+  if (this->eventHandler_.get() != nullptr) {
+    ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_write_ids_to_min_history", callContext);
+  }
+  ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_write_ids_to_min_history");
+
+  if (this->eventHandler_.get() != nullptr) {
+    this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_write_ids_to_min_history");
+  }
+
+  ThriftHiveMetastore_add_write_ids_to_min_history_args args;
+  args.read(iprot);
+  iprot->readMessageEnd();
+  uint32_t bytes = iprot->getTransport()->readEnd();
+
+  if (this->eventHandler_.get() != nullptr) {
+    this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_write_ids_to_min_history", bytes);
+  }
+
+  ThriftHiveMetastore_add_write_ids_to_min_history_result result;
+  try {
+    iface_->add_write_ids_to_min_history(args.txnId, args.writeIds);
+  } catch (MetaException &o2) {
+    result.o2 = std::move(o2);
+    result.__isset.o2 = true;
+  } catch (const std::exception& e) {
+    if (this->eventHandler_.get() != nullptr) {
+      this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_write_ids_to_min_history");
+    }
+
+    ::apache::thrift::TApplicationException x(e.what());
+    oprot->writeMessageBegin("add_write_ids_to_min_history", ::apache::thrift::protocol::T_EXCEPTION, seqid);
+    x.write(oprot);
+    oprot->writeMessageEnd();
+    oprot->getTransport()->writeEnd();
+    oprot->getTransport()->flush();
+    return;
+  }
+
+  if (this->eventHandler_.get() != nullptr) {
+    this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_write_ids_to_min_history");
+  }
+
+  oprot->writeMessageBegin("add_write_ids_to_min_history", ::apache::thrift::protocol::T_REPLY, seqid);
+  result.write(oprot);
+  oprot->writeMessageEnd();
+  bytes = oprot->getTransport()->writeEnd();
+  oprot->getTransport()->flush();
+
+  if (this->eventHandler_.get() != nullptr) {
+    this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_write_ids_to_min_history", bytes);
+  }
+}
+
 void ThriftHiveMetastoreProcessor::process_allocate_table_write_ids(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext)
 {
   void* ctx = nullptr;
@@ -113780,6 +114129,89 @@ void ThriftHiveMetastoreConcurrentClient::recv_get_valid_write_ids(GetValidWrite
   } // end while(true)
 }
 
+void ThriftHiveMetastoreConcurrentClient::add_write_ids_to_min_history(const int64_t txnId, const std::map<std::string, int64_t> & writeIds)
+{
+  int32_t seqid = send_add_write_ids_to_min_history(txnId, writeIds);
+  recv_add_write_ids_to_min_history(seqid);
+}
+
+int32_t ThriftHiveMetastoreConcurrentClient::send_add_write_ids_to_min_history(const int64_t txnId, const std::map<std::string, int64_t> & writeIds)
+{
+  int32_t cseqid = this->sync_->generateSeqId();
+  ::apache::thrift::async::TConcurrentSendSentry sentry(this->sync_.get());
+  oprot_->writeMessageBegin("add_write_ids_to_min_history", ::apache::thrift::protocol::T_CALL, cseqid);
+
+  ThriftHiveMetastore_add_write_ids_to_min_history_pargs args;
+  args.txnId = &txnId;
+  args.writeIds = &writeIds;
+  args.write(oprot_);
+
+  oprot_->writeMessageEnd();
+  oprot_->getTransport()->writeEnd();
+  oprot_->getTransport()->flush();
+
+  sentry.commit();
+  return cseqid;
+}
+
+void ThriftHiveMetastoreConcurrentClient::recv_add_write_ids_to_min_history(const int32_t seqid)
+{
+
+  int32_t rseqid = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TMessageType mtype;
+
+  // the read mutex gets dropped and reacquired as part of waitForWork()
+  // The destructor of this sentry wakes up other clients
+  ::apache::thrift::async::TConcurrentRecvSentry sentry(this->sync_.get(), seqid);
+
+  while(true) {
+    if(!this->sync_->getPending(fname, mtype, rseqid)) {
+      iprot_->readMessageBegin(fname, mtype, rseqid);
+    }
+    if(seqid == rseqid) {
+      if (mtype == ::apache::thrift::protocol::T_EXCEPTION) {
+        ::apache::thrift::TApplicationException x;
+        x.read(iprot_);
+        iprot_->readMessageEnd();
+        iprot_->getTransport()->readEnd();
+        sentry.commit();
+        throw x;
+      }
+      if (mtype != ::apache::thrift::protocol::T_REPLY) {
+        iprot_->skip(::apache::thrift::protocol::T_STRUCT);
+        iprot_->readMessageEnd();
+        iprot_->getTransport()->readEnd();
+      }
+      if (fname.compare("add_write_ids_to_min_history") != 0) {
+        iprot_->skip(::apache::thrift::protocol::T_STRUCT);
+        iprot_->readMessageEnd();
+        iprot_->getTransport()->readEnd();
+
+        // in a bad state, don't commit
+        using ::apache::thrift::protocol::TProtocolException;
+        throw TProtocolException(TProtocolException::INVALID_DATA);
+      }
+      ThriftHiveMetastore_add_write_ids_to_min_history_presult result;
+      result.read(iprot_);
+      iprot_->readMessageEnd();
+      iprot_->getTransport()->readEnd();
+
+      if (result.__isset.o2) {
+        sentry.commit();
+        throw result.o2;
+      }
+      sentry.commit();
+      return;
+    }
+    // seqid != rseqid
+    this->sync_->updatePending(fname, mtype, rseqid);
+
+    // this will temporarily unlock the readMutex, and let other clients get work done
+    this->sync_->waitForWork(seqid);
+  } // end while(true)
+}
+
 void ThriftHiveMetastoreConcurrentClient::allocate_table_write_ids(AllocateTableWriteIdsResponse& _return, const AllocateTableWriteIdsRequest& rqst)
 {
   int32_t seqid = send_allocate_table_write_ids(rqst);
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 504b9fd1d9a..9a0132ac839 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
@@ -205,6 +205,7 @@ class ThriftHiveMetastoreIf : virtual public  ::facebook::fb303::FacebookService
   virtual int64_t get_latest_txnid_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 add_write_ids_to_min_history(const int64_t txnId, const std::map<std::string, int64_t> & writeIds) = 0;
   virtual void allocate_table_write_ids(AllocateTableWriteIdsResponse& _return, const AllocateTableWriteIdsRequest& rqst) = 0;
   virtual void get_max_allocated_table_write_id(MaxAllocatedTableWriteIdResponse& _return, const MaxAllocatedTableWriteIdRequest& rqst) = 0;
   virtual void seed_write_id(const SeedTableWriteIdsRequest& rqst) = 0;
@@ -892,6 +893,9 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p
   void get_valid_write_ids(GetValidWriteIdsResponse& /* _return */, const GetValidWriteIdsRequest& /* rqst */) override {
     return;
   }
+  void add_write_ids_to_min_history(const int64_t /* txnId */, const std::map<std::string, int64_t> & /* writeIds */) override {
+    return;
+  }
   void allocate_table_write_ids(AllocateTableWriteIdsResponse& /* _return */, const AllocateTableWriteIdsRequest& /* rqst */) override {
     return;
   }
@@ -23695,6 +23699,118 @@ class ThriftHiveMetastore_get_valid_write_ids_presult {
 
 };
 
+typedef struct _ThriftHiveMetastore_add_write_ids_to_min_history_args__isset {
+  _ThriftHiveMetastore_add_write_ids_to_min_history_args__isset() : txnId(false), writeIds(false) {}
+  bool txnId :1;
+  bool writeIds :1;
+} _ThriftHiveMetastore_add_write_ids_to_min_history_args__isset;
+
+class ThriftHiveMetastore_add_write_ids_to_min_history_args {
+ public:
+
+  ThriftHiveMetastore_add_write_ids_to_min_history_args(const ThriftHiveMetastore_add_write_ids_to_min_history_args&);
+  ThriftHiveMetastore_add_write_ids_to_min_history_args& operator=(const ThriftHiveMetastore_add_write_ids_to_min_history_args&);
+  ThriftHiveMetastore_add_write_ids_to_min_history_args() noexcept
+                                                        : txnId(0) {
+  }
+
+  virtual ~ThriftHiveMetastore_add_write_ids_to_min_history_args() noexcept;
+  int64_t txnId;
+  std::map<std::string, int64_t>  writeIds;
+
+  _ThriftHiveMetastore_add_write_ids_to_min_history_args__isset __isset;
+
+  void __set_txnId(const int64_t val);
+
+  void __set_writeIds(const std::map<std::string, int64_t> & val);
+
+  bool operator == (const ThriftHiveMetastore_add_write_ids_to_min_history_args & rhs) const
+  {
+    if (!(txnId == rhs.txnId))
+      return false;
+    if (!(writeIds == rhs.writeIds))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_add_write_ids_to_min_history_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_add_write_ids_to_min_history_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_add_write_ids_to_min_history_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_add_write_ids_to_min_history_pargs() noexcept;
+  const int64_t* txnId;
+  const std::map<std::string, int64_t> * writeIds;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_add_write_ids_to_min_history_result__isset {
+  _ThriftHiveMetastore_add_write_ids_to_min_history_result__isset() : o2(false) {}
+  bool o2 :1;
+} _ThriftHiveMetastore_add_write_ids_to_min_history_result__isset;
+
+class ThriftHiveMetastore_add_write_ids_to_min_history_result {
+ public:
+
+  ThriftHiveMetastore_add_write_ids_to_min_history_result(const ThriftHiveMetastore_add_write_ids_to_min_history_result&);
+  ThriftHiveMetastore_add_write_ids_to_min_history_result& operator=(const ThriftHiveMetastore_add_write_ids_to_min_history_result&);
+  ThriftHiveMetastore_add_write_ids_to_min_history_result() noexcept {
+  }
+
+  virtual ~ThriftHiveMetastore_add_write_ids_to_min_history_result() noexcept;
+  MetaException o2;
+
+  _ThriftHiveMetastore_add_write_ids_to_min_history_result__isset __isset;
+
+  void __set_o2(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_add_write_ids_to_min_history_result & rhs) const
+  {
+    if (!(o2 == rhs.o2))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_add_write_ids_to_min_history_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_add_write_ids_to_min_history_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_add_write_ids_to_min_history_presult__isset {
+  _ThriftHiveMetastore_add_write_ids_to_min_history_presult__isset() : o2(false) {}
+  bool o2 :1;
+} _ThriftHiveMetastore_add_write_ids_to_min_history_presult__isset;
+
+class ThriftHiveMetastore_add_write_ids_to_min_history_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_add_write_ids_to_min_history_presult() noexcept;
+  MetaException o2;
+
+  _ThriftHiveMetastore_add_write_ids_to_min_history_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
 typedef struct _ThriftHiveMetastore_allocate_table_write_ids_args__isset {
   _ThriftHiveMetastore_allocate_table_write_ids_args__isset() : rqst(false) {}
   bool rqst :1;
@@ -34751,6 +34867,9 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public
   void get_valid_write_ids(GetValidWriteIdsResponse& _return, const GetValidWriteIdsRequest& rqst) override;
   void send_get_valid_write_ids(const GetValidWriteIdsRequest& rqst);
   void recv_get_valid_write_ids(GetValidWriteIdsResponse& _return);
+  void add_write_ids_to_min_history(const int64_t txnId, const std::map<std::string, int64_t> & writeIds) override;
+  void send_add_write_ids_to_min_history(const int64_t txnId, const std::map<std::string, int64_t> & writeIds);
+  void recv_add_write_ids_to_min_history();
   void allocate_table_write_ids(AllocateTableWriteIdsResponse& _return, const AllocateTableWriteIdsRequest& rqst) override;
   void send_allocate_table_write_ids(const AllocateTableWriteIdsRequest& rqst);
   void recv_allocate_table_write_ids(AllocateTableWriteIdsResponse& _return);
@@ -35219,6 +35338,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
   void process_get_latest_txnid_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_add_write_ids_to_min_history(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);
   void process_get_max_allocated_table_write_id(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_seed_write_id(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
@@ -35495,6 +35615,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
     processMap_["get_latest_txnid_in_conflict"] = &ThriftHiveMetastoreProcessor::process_get_latest_txnid_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_["add_write_ids_to_min_history"] = &ThriftHiveMetastoreProcessor::process_add_write_ids_to_min_history;
     processMap_["allocate_table_write_ids"] = &ThriftHiveMetastoreProcessor::process_allocate_table_write_ids;
     processMap_["get_max_allocated_table_write_id"] = &ThriftHiveMetastoreProcessor::process_get_max_allocated_table_write_id;
     processMap_["seed_write_id"] = &ThriftHiveMetastoreProcessor::process_seed_write_id;
@@ -37338,6 +37459,15 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi
     return;
   }
 
+  void add_write_ids_to_min_history(const int64_t txnId, const std::map<std::string, int64_t> & writeIds) override {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->add_write_ids_to_min_history(txnId, writeIds);
+    }
+    ifaces_[i]->add_write_ids_to_min_history(txnId, writeIds);
+  }
+
   void allocate_table_write_ids(AllocateTableWriteIdsResponse& _return, const AllocateTableWriteIdsRequest& rqst) override {
     size_t sz = ifaces_.size();
     size_t i = 0;
@@ -38788,6 +38918,9 @@ class ThriftHiveMetastoreConcurrentClient : virtual public ThriftHiveMetastoreIf
   void get_valid_write_ids(GetValidWriteIdsResponse& _return, const GetValidWriteIdsRequest& rqst) override;
   int32_t send_get_valid_write_ids(const GetValidWriteIdsRequest& rqst);
   void recv_get_valid_write_ids(GetValidWriteIdsResponse& _return, const int32_t seqid);
+  void add_write_ids_to_min_history(const int64_t txnId, const std::map<std::string, int64_t> & writeIds) override;
+  int32_t send_add_write_ids_to_min_history(const int64_t txnId, const std::map<std::string, int64_t> & writeIds);
+  void recv_add_write_ids_to_min_history(const int32_t seqid);
   void allocate_table_write_ids(AllocateTableWriteIdsResponse& _return, const AllocateTableWriteIdsRequest& rqst) override;
   int32_t send_allocate_table_write_ids(const AllocateTableWriteIdsRequest& rqst);
   void recv_allocate_table_write_ids(AllocateTableWriteIdsResponse& _return, 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 3970cc07dc1..6487a902bd3 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
@@ -915,6 +915,11 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("get_valid_write_ids\n");
   }
 
+  void add_write_ids_to_min_history(const int64_t txnId, const std::map<std::string, int64_t> & writeIds) {
+    // Your implementation goes here
+    printf("add_write_ids_to_min_history\n");
+  }
+
   void allocate_table_write_ids(AllocateTableWriteIdsResponse& _return, const AllocateTableWriteIdsRequest& rqst) {
     // Your implementation goes here
     printf("allocate_table_write_ids\n");
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
index 7a0b1f6180b..00343d9054a 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -25069,6 +25069,11 @@ void LockRequest::__set_exclusiveCTAS(const bool val) {
   this->exclusiveCTAS = val;
 __isset.exclusiveCTAS = true;
 }
+
+void LockRequest::__set_locklessReadsEnabled(const bool val) {
+  this->locklessReadsEnabled = val;
+__isset.locklessReadsEnabled = true;
+}
 std::ostream& operator<<(std::ostream& out, const LockRequest& obj)
 {
   obj.printTo(out);
@@ -25168,6 +25173,14 @@ uint32_t LockRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
           xfer += iprot->skip(ftype);
         }
         break;
+      case 8:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->locklessReadsEnabled);
+          this->__isset.locklessReadsEnabled = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -25231,6 +25244,11 @@ uint32_t LockRequest::write(::apache::thrift::protocol::TProtocol* oprot) const
     xfer += oprot->writeBool(this->exclusiveCTAS);
     xfer += oprot->writeFieldEnd();
   }
+  if (this->__isset.locklessReadsEnabled) {
+    xfer += oprot->writeFieldBegin("locklessReadsEnabled", ::apache::thrift::protocol::T_BOOL, 8);
+    xfer += oprot->writeBool(this->locklessReadsEnabled);
+    xfer += oprot->writeFieldEnd();
+  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
@@ -25245,6 +25263,7 @@ void swap(LockRequest &a, LockRequest &b) {
   swap(a.agentInfo, b.agentInfo);
   swap(a.zeroWaitReadEnabled, b.zeroWaitReadEnabled);
   swap(a.exclusiveCTAS, b.exclusiveCTAS);
+  swap(a.locklessReadsEnabled, b.locklessReadsEnabled);
   swap(a.__isset, b.__isset);
 }
 
@@ -25256,6 +25275,7 @@ LockRequest::LockRequest(const LockRequest& other926) {
   agentInfo = other926.agentInfo;
   zeroWaitReadEnabled = other926.zeroWaitReadEnabled;
   exclusiveCTAS = other926.exclusiveCTAS;
+  locklessReadsEnabled = other926.locklessReadsEnabled;
   __isset = other926.__isset;
 }
 LockRequest& LockRequest::operator=(const LockRequest& other927) {
@@ -25266,6 +25286,7 @@ LockRequest& LockRequest::operator=(const LockRequest& other927) {
   agentInfo = other927.agentInfo;
   zeroWaitReadEnabled = other927.zeroWaitReadEnabled;
   exclusiveCTAS = other927.exclusiveCTAS;
+  locklessReadsEnabled = other927.locklessReadsEnabled;
   __isset = other927.__isset;
   return *this;
 }
@@ -25279,6 +25300,7 @@ void LockRequest::printTo(std::ostream& out) const {
   out << ", " << "agentInfo="; (__isset.agentInfo ? (out << to_string(agentInfo)) : (out << "<null>"));
   out << ", " << "zeroWaitReadEnabled="; (__isset.zeroWaitReadEnabled ? (out << to_string(zeroWaitReadEnabled)) : (out << "<null>"));
   out << ", " << "exclusiveCTAS="; (__isset.exclusiveCTAS ? (out << to_string(exclusiveCTAS)) : (out << "<null>"));
+  out << ", " << "locklessReadsEnabled="; (__isset.locklessReadsEnabled ? (out << to_string(locklessReadsEnabled)) : (out << "<null>"));
   out << ")";
 }
 
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h
index eace9e27f19..ab8c1c03322 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -9965,11 +9965,12 @@ void swap(LockComponent &a, LockComponent &b);
 std::ostream& operator<<(std::ostream& out, const LockComponent& obj);
 
 typedef struct _LockRequest__isset {
-  _LockRequest__isset() : txnid(false), agentInfo(true), zeroWaitReadEnabled(true), exclusiveCTAS(true) {}
+  _LockRequest__isset() : txnid(false), agentInfo(true), zeroWaitReadEnabled(true), exclusiveCTAS(true), locklessReadsEnabled(true) {}
   bool txnid :1;
   bool agentInfo :1;
   bool zeroWaitReadEnabled :1;
   bool exclusiveCTAS :1;
+  bool locklessReadsEnabled :1;
 } _LockRequest__isset;
 
 class LockRequest : public virtual ::apache::thrift::TBase {
@@ -9982,7 +9983,8 @@ class LockRequest : public virtual ::apache::thrift::TBase {
                   hostname(),
                   agentInfo("Unknown"),
                   zeroWaitReadEnabled(false),
-                  exclusiveCTAS(false) {
+                  exclusiveCTAS(false),
+                  locklessReadsEnabled(false) {
   }
 
   virtual ~LockRequest() noexcept;
@@ -9993,6 +9995,7 @@ class LockRequest : public virtual ::apache::thrift::TBase {
   std::string agentInfo;
   bool zeroWaitReadEnabled;
   bool exclusiveCTAS;
+  bool locklessReadsEnabled;
 
   _LockRequest__isset __isset;
 
@@ -10010,6 +10013,8 @@ class LockRequest : public virtual ::apache::thrift::TBase {
 
   void __set_exclusiveCTAS(const bool val);
 
+  void __set_locklessReadsEnabled(const bool val);
+
   bool operator == (const LockRequest & rhs) const
   {
     if (!(component == rhs.component))
@@ -10034,6 +10039,10 @@ class LockRequest : public virtual ::apache::thrift::TBase {
       return false;
     else if (__isset.exclusiveCTAS && !(exclusiveCTAS == rhs.exclusiveCTAS))
       return false;
+    if (__isset.locklessReadsEnabled != rhs.__isset.locklessReadsEnabled)
+      return false;
+    else if (__isset.locklessReadsEnabled && !(locklessReadsEnabled == rhs.locklessReadsEnabled))
+      return false;
     return true;
   }
   bool operator != (const LockRequest &rhs) const {
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
index cd8644cf29c..6994e0bae2e 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
@@ -18,6 +18,7 @@ package org.apache.hadoop.hive.metastore.api;
   private static final org.apache.thrift.protocol.TField AGENT_INFO_FIELD_DESC = new org.apache.thrift.protocol.TField("agentInfo", org.apache.thrift.protocol.TType.STRING, (short)5);
   private static final org.apache.thrift.protocol.TField ZERO_WAIT_READ_ENABLED_FIELD_DESC = new org.apache.thrift.protocol.TField("zeroWaitReadEnabled", org.apache.thrift.protocol.TType.BOOL, (short)6);
   private static final org.apache.thrift.protocol.TField EXCLUSIVE_CTAS_FIELD_DESC = new org.apache.thrift.protocol.TField("exclusiveCTAS", org.apache.thrift.protocol.TType.BOOL, (short)7);
+  private static final org.apache.thrift.protocol.TField LOCKLESS_READS_ENABLED_FIELD_DESC = new org.apache.thrift.protocol.TField("locklessReadsEnabled", org.apache.thrift.protocol.TType.BOOL, (short)8);
 
   private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new LockRequestStandardSchemeFactory();
   private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new LockRequestTupleSchemeFactory();
@@ -29,6 +30,7 @@ package org.apache.hadoop.hive.metastore.api;
   private @org.apache.thrift.annotation.Nullable java.lang.String agentInfo; // optional
   private boolean zeroWaitReadEnabled; // optional
   private boolean exclusiveCTAS; // optional
+  private boolean locklessReadsEnabled; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -38,7 +40,8 @@ package org.apache.hadoop.hive.metastore.api;
     HOSTNAME((short)4, "hostname"),
     AGENT_INFO((short)5, "agentInfo"),
     ZERO_WAIT_READ_ENABLED((short)6, "zeroWaitReadEnabled"),
-    EXCLUSIVE_CTAS((short)7, "exclusiveCTAS");
+    EXCLUSIVE_CTAS((short)7, "exclusiveCTAS"),
+    LOCKLESS_READS_ENABLED((short)8, "locklessReadsEnabled");
 
     private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
 
@@ -68,6 +71,8 @@ package org.apache.hadoop.hive.metastore.api;
           return ZERO_WAIT_READ_ENABLED;
         case 7: // EXCLUSIVE_CTAS
           return EXCLUSIVE_CTAS;
+        case 8: // LOCKLESS_READS_ENABLED
+          return LOCKLESS_READS_ENABLED;
         default:
           return null;
       }
@@ -112,8 +117,9 @@ package org.apache.hadoop.hive.metastore.api;
   private static final int __TXNID_ISSET_ID = 0;
   private static final int __ZEROWAITREADENABLED_ISSET_ID = 1;
   private static final int __EXCLUSIVECTAS_ISSET_ID = 2;
+  private static final int __LOCKLESSREADSENABLED_ISSET_ID = 3;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.TXNID,_Fields.AGENT_INFO,_Fields.ZERO_WAIT_READ_ENABLED,_Fields.EXCLUSIVE_CTAS};
+  private static final _Fields optionals[] = {_Fields.TXNID,_Fields.AGENT_INFO,_Fields.ZERO_WAIT_READ_ENABLED,_Fields.EXCLUSIVE_CTAS,_Fields.LOCKLESS_READS_ENABLED};
   public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -132,6 +138,8 @@ package org.apache.hadoop.hive.metastore.api;
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
     tmpMap.put(_Fields.EXCLUSIVE_CTAS, new org.apache.thrift.meta_data.FieldMetaData("exclusiveCTAS", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.LOCKLESS_READS_ENABLED, new org.apache.thrift.meta_data.FieldMetaData("locklessReadsEnabled", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
     metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(LockRequest.class, metaDataMap);
   }
@@ -143,6 +151,8 @@ package org.apache.hadoop.hive.metastore.api;
 
     this.exclusiveCTAS = false;
 
+    this.locklessReadsEnabled = false;
+
   }
 
   public LockRequest(
@@ -180,6 +190,7 @@ package org.apache.hadoop.hive.metastore.api;
     }
     this.zeroWaitReadEnabled = other.zeroWaitReadEnabled;
     this.exclusiveCTAS = other.exclusiveCTAS;
+    this.locklessReadsEnabled = other.locklessReadsEnabled;
   }
 
   public LockRequest deepCopy() {
@@ -199,6 +210,8 @@ package org.apache.hadoop.hive.metastore.api;
 
     this.exclusiveCTAS = false;
 
+    this.locklessReadsEnabled = false;
+
   }
 
   public int getComponentSize() {
@@ -379,6 +392,28 @@ package org.apache.hadoop.hive.metastore.api;
     __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __EXCLUSIVECTAS_ISSET_ID, value);
   }
 
+  public boolean isLocklessReadsEnabled() {
+    return this.locklessReadsEnabled;
+  }
+
+  public void setLocklessReadsEnabled(boolean locklessReadsEnabled) {
+    this.locklessReadsEnabled = locklessReadsEnabled;
+    setLocklessReadsEnabledIsSet(true);
+  }
+
+  public void unsetLocklessReadsEnabled() {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __LOCKLESSREADSENABLED_ISSET_ID);
+  }
+
+  /** Returns true if field locklessReadsEnabled is set (has been assigned a value) and false otherwise */
+  public boolean isSetLocklessReadsEnabled() {
+    return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __LOCKLESSREADSENABLED_ISSET_ID);
+  }
+
+  public void setLocklessReadsEnabledIsSet(boolean value) {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __LOCKLESSREADSENABLED_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
     switch (field) {
     case COMPONENT:
@@ -437,6 +472,14 @@ package org.apache.hadoop.hive.metastore.api;
       }
       break;
 
+    case LOCKLESS_READS_ENABLED:
+      if (value == null) {
+        unsetLocklessReadsEnabled();
+      } else {
+        setLocklessReadsEnabled((java.lang.Boolean)value);
+      }
+      break;
+
     }
   }
 
@@ -464,6 +507,9 @@ package org.apache.hadoop.hive.metastore.api;
     case EXCLUSIVE_CTAS:
       return isExclusiveCTAS();
 
+    case LOCKLESS_READS_ENABLED:
+      return isLocklessReadsEnabled();
+
     }
     throw new java.lang.IllegalStateException();
   }
@@ -489,6 +535,8 @@ package org.apache.hadoop.hive.metastore.api;
       return isSetZeroWaitReadEnabled();
     case EXCLUSIVE_CTAS:
       return isSetExclusiveCTAS();
+    case LOCKLESS_READS_ENABLED:
+      return isSetLocklessReadsEnabled();
     }
     throw new java.lang.IllegalStateException();
   }
@@ -569,6 +617,15 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
     }
 
+    boolean this_present_locklessReadsEnabled = true && this.isSetLocklessReadsEnabled();
+    boolean that_present_locklessReadsEnabled = true && that.isSetLocklessReadsEnabled();
+    if (this_present_locklessReadsEnabled || that_present_locklessReadsEnabled) {
+      if (!(this_present_locklessReadsEnabled && that_present_locklessReadsEnabled))
+        return false;
+      if (this.locklessReadsEnabled != that.locklessReadsEnabled)
+        return false;
+    }
+
     return true;
   }
 
@@ -604,6 +661,10 @@ package org.apache.hadoop.hive.metastore.api;
     if (isSetExclusiveCTAS())
       hashCode = hashCode * 8191 + ((exclusiveCTAS) ? 131071 : 524287);
 
+    hashCode = hashCode * 8191 + ((isSetLocklessReadsEnabled()) ? 131071 : 524287);
+    if (isSetLocklessReadsEnabled())
+      hashCode = hashCode * 8191 + ((locklessReadsEnabled) ? 131071 : 524287);
+
     return hashCode;
   }
 
@@ -685,6 +746,16 @@ package org.apache.hadoop.hive.metastore.api;
         return lastComparison;
       }
     }
+    lastComparison = java.lang.Boolean.compare(isSetLocklessReadsEnabled(), other.isSetLocklessReadsEnabled());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetLocklessReadsEnabled()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.locklessReadsEnabled, other.locklessReadsEnabled);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -757,6 +828,12 @@ package org.apache.hadoop.hive.metastore.api;
       sb.append(this.exclusiveCTAS);
       first = false;
     }
+    if (isSetLocklessReadsEnabled()) {
+      if (!first) sb.append(", ");
+      sb.append("locklessReadsEnabled:");
+      sb.append(this.locklessReadsEnabled);
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -881,6 +958,14 @@ package org.apache.hadoop.hive.metastore.api;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 8: // LOCKLESS_READS_ENABLED
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.locklessReadsEnabled = iprot.readBool();
+              struct.setLocklessReadsEnabledIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -938,6 +1023,11 @@ package org.apache.hadoop.hive.metastore.api;
         oprot.writeBool(struct.exclusiveCTAS);
         oprot.writeFieldEnd();
       }
+      if (struct.isSetLocklessReadsEnabled()) {
+        oprot.writeFieldBegin(LOCKLESS_READS_ENABLED_FIELD_DESC);
+        oprot.writeBool(struct.locklessReadsEnabled);
+        oprot.writeFieldEnd();
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -977,7 +1067,10 @@ package org.apache.hadoop.hive.metastore.api;
       if (struct.isSetExclusiveCTAS()) {
         optionals.set(3);
       }
-      oprot.writeBitSet(optionals, 4);
+      if (struct.isSetLocklessReadsEnabled()) {
+        optionals.set(4);
+      }
+      oprot.writeBitSet(optionals, 5);
       if (struct.isSetTxnid()) {
         oprot.writeI64(struct.txnid);
       }
@@ -990,6 +1083,9 @@ package org.apache.hadoop.hive.metastore.api;
       if (struct.isSetExclusiveCTAS()) {
         oprot.writeBool(struct.exclusiveCTAS);
       }
+      if (struct.isSetLocklessReadsEnabled()) {
+        oprot.writeBool(struct.locklessReadsEnabled);
+      }
     }
 
     @Override
@@ -1011,7 +1107,7 @@ package org.apache.hadoop.hive.metastore.api;
       struct.setUserIsSet(true);
       struct.hostname = iprot.readString();
       struct.setHostnameIsSet(true);
-      java.util.BitSet incoming = iprot.readBitSet(4);
+      java.util.BitSet incoming = iprot.readBitSet(5);
       if (incoming.get(0)) {
         struct.txnid = iprot.readI64();
         struct.setTxnidIsSet(true);
@@ -1028,6 +1124,10 @@ package org.apache.hadoop.hive.metastore.api;
         struct.exclusiveCTAS = iprot.readBool();
         struct.setExclusiveCTASIsSet(true);
       }
+      if (incoming.get(4)) {
+        struct.locklessReadsEnabled = iprot.readBool();
+        struct.setLocklessReadsEnabledIsSet(true);
+      }
     }
   }
 
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 6d5163d0efe..f7e1459bd96 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
@@ -373,6 +373,8 @@ package org.apache.hadoop.hive.metastore.api;
 
     public GetValidWriteIdsResponse get_valid_write_ids(GetValidWriteIdsRequest rqst) throws NoSuchTxnException, MetaException, org.apache.thrift.TException;
 
+    public void add_write_ids_to_min_history(long txnId, java.util.Map<java.lang.String,java.lang.Long> writeIds) throws MetaException, org.apache.thrift.TException;
+
     public AllocateTableWriteIdsResponse allocate_table_write_ids(AllocateTableWriteIdsRequest rqst) throws NoSuchTxnException, TxnAbortedException, MetaException, org.apache.thrift.TException;
 
     public MaxAllocatedTableWriteIdResponse get_max_allocated_table_write_id(MaxAllocatedTableWriteIdRequest rqst) throws MetaException, org.apache.thrift.TException;
@@ -921,6 +923,8 @@ package org.apache.hadoop.hive.metastore.api;
 
     public void get_valid_write_ids(GetValidWriteIdsRequest rqst, org.apache.thrift.async.AsyncMethodCallback<GetValidWriteIdsResponse> resultHandler) throws org.apache.thrift.TException;
 
+    public void add_write_ids_to_min_history(long txnId, java.util.Map<java.lang.String,java.lang.Long> writeIds, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
     public void allocate_table_write_ids(AllocateTableWriteIdsRequest rqst, org.apache.thrift.async.AsyncMethodCallback<AllocateTableWriteIdsResponse> resultHandler) throws org.apache.thrift.TException;
 
     public void get_max_allocated_table_write_id(MaxAllocatedTableWriteIdRequest rqst, org.apache.thrift.async.AsyncMethodCallback<MaxAllocatedTableWriteIdResponse> resultHandler) throws org.apache.thrift.TException;
@@ -6351,6 +6355,30 @@ package org.apache.hadoop.hive.metastore.api;
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_valid_write_ids failed: unknown result");
     }
 
+    public void add_write_ids_to_min_history(long txnId, java.util.Map<java.lang.String,java.lang.Long> writeIds) throws MetaException, org.apache.thrift.TException
+    {
+      send_add_write_ids_to_min_history(txnId, writeIds);
+      recv_add_write_ids_to_min_history();
+    }
+
+    public void send_add_write_ids_to_min_history(long txnId, java.util.Map<java.lang.String,java.lang.Long> writeIds) throws org.apache.thrift.TException
+    {
+      add_write_ids_to_min_history_args args = new add_write_ids_to_min_history_args();
+      args.setTxnId(txnId);
+      args.setWriteIds(writeIds);
+      sendBase("add_write_ids_to_min_history", args);
+    }
+
+    public void recv_add_write_ids_to_min_history() throws MetaException, org.apache.thrift.TException
+    {
+      add_write_ids_to_min_history_result result = new add_write_ids_to_min_history_result();
+      receiveBase(result, "add_write_ids_to_min_history");
+      if (result.o2 != null) {
+        throw result.o2;
+      }
+      return;
+    }
+
     public AllocateTableWriteIdsResponse allocate_table_write_ids(AllocateTableWriteIdsRequest rqst) throws NoSuchTxnException, TxnAbortedException, MetaException, org.apache.thrift.TException
     {
       send_allocate_table_write_ids(rqst);
@@ -15054,6 +15082,41 @@ package org.apache.hadoop.hive.metastore.api;
       }
     }
 
+    public void add_write_ids_to_min_history(long txnId, java.util.Map<java.lang.String,java.lang.Long> writeIds, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      add_write_ids_to_min_history_call method_call = new add_write_ids_to_min_history_call(txnId, writeIds, 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 add_write_ids_to_min_history_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+      private long txnId;
+      private java.util.Map<java.lang.String,java.lang.Long> writeIds;
+      public add_write_ids_to_min_history_call(long txnId, java.util.Map<java.lang.String,java.lang.Long> writeIds, org.apache.thrift.async.AsyncMethodCallback<Void> 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;
+        this.writeIds = writeIds;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("add_write_ids_to_min_history", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        add_write_ids_to_min_history_args args = new add_write_ids_to_min_history_args();
+        args.setTxnId(txnId);
+        args.setWriteIds(writeIds);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public Void getResult() throws MetaException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new java.lang.IllegalStateException("Method call not finished!");
+        }
+        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 null;
+      }
+    }
+
     public void allocate_table_write_ids(AllocateTableWriteIdsRequest rqst, org.apache.thrift.async.AsyncMethodCallback<AllocateTableWriteIdsResponse> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       allocate_table_write_ids_call method_call = new allocate_table_write_ids_call(rqst, resultHandler, this, ___protocolFactory, ___transport);
@@ -18237,6 +18300,7 @@ package org.apache.hadoop.hive.metastore.api;
       processMap.put("get_latest_txnid_in_conflict", new get_latest_txnid_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("add_write_ids_to_min_history", new add_write_ids_to_min_history());
       processMap.put("allocate_table_write_ids", new allocate_table_write_ids());
       processMap.put("get_max_allocated_table_write_id", new get_max_allocated_table_write_id());
       processMap.put("seed_write_id", new seed_write_id());
@@ -23886,6 +23950,35 @@ package org.apache.hadoop.hive.metastore.api;
       }
     }
 
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_write_ids_to_min_history<I extends Iface> extends org.apache.thrift.ProcessFunction<I, add_write_ids_to_min_history_args> {
+      public add_write_ids_to_min_history() {
+        super("add_write_ids_to_min_history");
+      }
+
+      public add_write_ids_to_min_history_args getEmptyArgsInstance() {
+        return new add_write_ids_to_min_history_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      public add_write_ids_to_min_history_result getResult(I iface, add_write_ids_to_min_history_args args) throws org.apache.thrift.TException {
+        add_write_ids_to_min_history_result result = new add_write_ids_to_min_history_result();
+        try {
+          iface.add_write_ids_to_min_history(args.txnId, args.writeIds);
+        } catch (MetaException o2) {
+          result.o2 = o2;
+        }
+        return result;
+      }
+    }
+
     @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class allocate_table_write_ids<I extends Iface> extends org.apache.thrift.ProcessFunction<I, allocate_table_write_ids_args> {
       public allocate_table_write_ids() {
         super("allocate_table_write_ids");
@@ -26814,6 +26907,7 @@ package org.apache.hadoop.hive.metastore.api;
       processMap.put("get_latest_txnid_in_conflict", new get_latest_txnid_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("add_write_ids_to_min_history", new add_write_ids_to_min_history());
       processMap.put("allocate_table_write_ids", new allocate_table_write_ids());
       processMap.put("get_max_allocated_table_write_id", new get_max_allocated_table_write_id());
       processMap.put("seed_write_id", new seed_write_id());
@@ -39244,158 +39338,20 @@ package org.apache.hadoop.hive.metastore.api;
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class allocate_table_write_ids<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, allocate_table_write_ids_args, AllocateTableWriteIdsResponse> {
-      public allocate_table_write_ids() {
-        super("allocate_table_write_ids");
-      }
-
-      public allocate_table_write_ids_args getEmptyArgsInstance() {
-        return new allocate_table_write_ids_args();
-      }
-
-      public org.apache.thrift.async.AsyncMethodCallback<AllocateTableWriteIdsResponse> 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<AllocateTableWriteIdsResponse>() { 
-          public void onComplete(AllocateTableWriteIdsResponse o) {
-            allocate_table_write_ids_result result = new allocate_table_write_ids_result();
-            result.success = o;
-            try {
-              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
-            } catch (org.apache.thrift.transport.TTransportException e) {
-              _LOGGER.error("TTransportException writing to internal frame buffer", e);
-              fb.close();
-            } catch (java.lang.Exception e) {
-              _LOGGER.error("Exception writing to internal frame buffer", e);
-              onError(e);
-            }
-          }
-          public void onError(java.lang.Exception e) {
-            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
-            org.apache.thrift.TSerializable msg;
-            allocate_table_write_ids_result result = new allocate_table_write_ids_result();
-            if (e instanceof NoSuchTxnException) {
-              result.o1 = (NoSuchTxnException) e;
-              result.setO1IsSet(true);
-              msg = result;
-            } else if (e instanceof TxnAbortedException) {
-              result.o2 = (TxnAbortedException) e;
-              result.setO2IsSet(true);
-              msg = result;
-            } else if (e instanceof MetaException) {
-              result.o3 = (MetaException) e;
-              result.setO3IsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
-              _LOGGER.error("TTransportException inside handler", e);
-              fb.close();
-              return;
-            } else if (e instanceof org.apache.thrift.TApplicationException) {
-              _LOGGER.error("TApplicationException inside handler", e);
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = (org.apache.thrift.TApplicationException)e;
-            } else {
-              _LOGGER.error("Exception inside handler", e);
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
-            }
-            try {
-              fcall.sendResponse(fb,msg,msgType,seqid);
-            } catch (java.lang.Exception ex) {
-              _LOGGER.error("Exception writing to internal frame buffer", ex);
-              fb.close();
-            }
-          }
-        };
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public void start(I iface, allocate_table_write_ids_args args, org.apache.thrift.async.AsyncMethodCallback<AllocateTableWriteIdsResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.allocate_table_write_ids(args.rqst,resultHandler);
-      }
-    }
-
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_max_allocated_table_write_id<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_max_allocated_table_write_id_args, MaxAllocatedTableWriteIdResponse> {
-      public get_max_allocated_table_write_id() {
-        super("get_max_allocated_table_write_id");
-      }
-
-      public get_max_allocated_table_write_id_args getEmptyArgsInstance() {
-        return new get_max_allocated_table_write_id_args();
-      }
-
-      public org.apache.thrift.async.AsyncMethodCallback<MaxAllocatedTableWriteIdResponse> 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<MaxAllocatedTableWriteIdResponse>() { 
-          public void onComplete(MaxAllocatedTableWriteIdResponse o) {
-            get_max_allocated_table_write_id_result result = new get_max_allocated_table_write_id_result();
-            result.success = o;
-            try {
-              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
-            } catch (org.apache.thrift.transport.TTransportException e) {
-              _LOGGER.error("TTransportException writing to internal frame buffer", e);
-              fb.close();
-            } catch (java.lang.Exception e) {
-              _LOGGER.error("Exception writing to internal frame buffer", e);
-              onError(e);
-            }
-          }
-          public void onError(java.lang.Exception e) {
-            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
-            org.apache.thrift.TSerializable msg;
-            get_max_allocated_table_write_id_result result = new get_max_allocated_table_write_id_result();
-            if (e instanceof MetaException) {
-              result.o1 = (MetaException) e;
-              result.setO1IsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
-              _LOGGER.error("TTransportException inside handler", e);
-              fb.close();
-              return;
-            } else if (e instanceof org.apache.thrift.TApplicationException) {
-              _LOGGER.error("TApplicationException inside handler", e);
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = (org.apache.thrift.TApplicationException)e;
-            } else {
-              _LOGGER.error("Exception inside handler", e);
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
-            }
-            try {
-              fcall.sendResponse(fb,msg,msgType,seqid);
-            } catch (java.lang.Exception ex) {
-              _LOGGER.error("Exception writing to internal frame buffer", ex);
-              fb.close();
-            }
-          }
-        };
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public void start(I iface, get_max_allocated_table_write_id_args args, org.apache.thrift.async.AsyncMethodCallback<MaxAllocatedTableWriteIdResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.get_max_allocated_table_write_id(args.rqst,resultHandler);
-      }
-    }
-
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class seed_write_id<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, seed_write_id_args, Void> {
-      public seed_write_id() {
-        super("seed_write_id");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_write_ids_to_min_history<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_write_ids_to_min_history_args, Void> {
+      public add_write_ids_to_min_history() {
+        super("add_write_ids_to_min_history");
       }
 
-      public seed_write_id_args getEmptyArgsInstance() {
-        return new seed_write_id_args();
+      public add_write_ids_to_min_history_args getEmptyArgsInstance() {
+        return new add_write_ids_to_min_history_args();
       }
 
       public org.apache.thrift.async.AsyncMethodCallback<Void> 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<Void>() { 
           public void onComplete(Void o) {
-            seed_write_id_result result = new seed_write_id_result();
+            add_write_ids_to_min_history_result result = new add_write_ids_to_min_history_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -39409,74 +39365,10 @@ 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;
-            seed_write_id_result result = new seed_write_id_result();
+            add_write_ids_to_min_history_result result = new add_write_ids_to_min_history_result();
             if (e instanceof MetaException) {
-              result.o1 = (MetaException) e;
-              result.setO1IsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
-              _LOGGER.error("TTransportException inside handler", e);
-              fb.close();
-              return;
-            } else if (e instanceof org.apache.thrift.TApplicationException) {
-              _LOGGER.error("TApplicationException inside handler", e);
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = (org.apache.thrift.TApplicationException)e;
-            } else {
-              _LOGGER.error("Exception inside handler", e);
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
-            }
-            try {
-              fcall.sendResponse(fb,msg,msgType,seqid);
-            } catch (java.lang.Exception ex) {
-              _LOGGER.error("Exception writing to internal frame buffer", ex);
-              fb.close();
-            }
-          }
-        };
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public void start(I iface, seed_write_id_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.seed_write_id(args.rqst,resultHandler);
-      }
-    }
-
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class seed_txn_id<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, seed_txn_id_args, Void> {
-      public seed_txn_id() {
-        super("seed_txn_id");
-      }
-
-      public seed_txn_id_args getEmptyArgsInstance() {
-        return new seed_txn_id_args();
-      }
-
-      public org.apache.thrift.async.AsyncMethodCallback<Void> 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<Void>() { 
-          public void onComplete(Void o) {
-            seed_txn_id_result result = new seed_txn_id_result();
-            try {
-              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
-            } catch (org.apache.thrift.transport.TTransportException e) {
-              _LOGGER.error("TTransportException writing to internal frame buffer", e);
-              fb.close();
-            } catch (java.lang.Exception e) {
-              _LOGGER.error("Exception writing to internal frame buffer", e);
-              onError(e);
-            }
-          }
-          public void onError(java.lang.Exception e) {
-            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
-            org.apache.thrift.TSerializable msg;
-            seed_txn_id_result result = new seed_txn_id_result();
-            if (e instanceof MetaException) {
-              result.o1 = (MetaException) e;
-              result.setO1IsSet(true);
+              result.o2 = (MetaException) e;
+              result.setO2IsSet(true);
               msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
@@ -39505,25 +39397,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, seed_txn_id_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.seed_txn_id(args.rqst,resultHandler);
+      public void start(I iface, add_write_ids_to_min_history_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.add_write_ids_to_min_history(args.txnId, args.writeIds,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class lock<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, lock_args, LockResponse> {
-      public lock() {
-        super("lock");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class allocate_table_write_ids<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, allocate_table_write_ids_args, AllocateTableWriteIdsResponse> {
+      public allocate_table_write_ids() {
+        super("allocate_table_write_ids");
       }
 
-      public lock_args getEmptyArgsInstance() {
-        return new lock_args();
+      public allocate_table_write_ids_args getEmptyArgsInstance() {
+        return new allocate_table_write_ids_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<LockResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<AllocateTableWriteIdsResponse> 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<LockResponse>() { 
-          public void onComplete(LockResponse o) {
-            lock_result result = new lock_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<AllocateTableWriteIdsResponse>() { 
+          public void onComplete(AllocateTableWriteIdsResponse o) {
+            allocate_table_write_ids_result result = new allocate_table_write_ids_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -39538,7 +39430,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;
-            lock_result result = new lock_result();
+            allocate_table_write_ids_result result = new allocate_table_write_ids_result();
             if (e instanceof NoSuchTxnException) {
               result.o1 = (NoSuchTxnException) e;
               result.setO1IsSet(true);
@@ -39547,6 +39439,10 @@ package org.apache.hadoop.hive.metastore.api;
               result.o2 = (TxnAbortedException) e;
               result.setO2IsSet(true);
               msg = result;
+            } else if (e instanceof MetaException) {
+              result.o3 = (MetaException) e;
+              result.setO3IsSet(true);
+              msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -39574,25 +39470,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, lock_args args, org.apache.thrift.async.AsyncMethodCallback<LockResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.lock(args.rqst,resultHandler);
+      public void start(I iface, allocate_table_write_ids_args args, org.apache.thrift.async.AsyncMethodCallback<AllocateTableWriteIdsResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.allocate_table_write_ids(args.rqst,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class check_lock<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, check_lock_args, LockResponse> {
-      public check_lock() {
-        super("check_lock");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_max_allocated_table_write_id<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_max_allocated_table_write_id_args, MaxAllocatedTableWriteIdResponse> {
+      public get_max_allocated_table_write_id() {
+        super("get_max_allocated_table_write_id");
       }
 
-      public check_lock_args getEmptyArgsInstance() {
-        return new check_lock_args();
+      public get_max_allocated_table_write_id_args getEmptyArgsInstance() {
+        return new get_max_allocated_table_write_id_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<LockResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<MaxAllocatedTableWriteIdResponse> 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<LockResponse>() { 
-          public void onComplete(LockResponse o) {
-            check_lock_result result = new check_lock_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<MaxAllocatedTableWriteIdResponse>() { 
+          public void onComplete(MaxAllocatedTableWriteIdResponse o) {
+            get_max_allocated_table_write_id_result result = new get_max_allocated_table_write_id_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -39607,19 +39503,11 @@ 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;
-            check_lock_result result = new check_lock_result();
-            if (e instanceof NoSuchTxnException) {
-              result.o1 = (NoSuchTxnException) e;
+            get_max_allocated_table_write_id_result result = new get_max_allocated_table_write_id_result();
+            if (e instanceof MetaException) {
+              result.o1 = (MetaException) e;
               result.setO1IsSet(true);
               msg = result;
-            } else if (e instanceof TxnAbortedException) {
-              result.o2 = (TxnAbortedException) e;
-              result.setO2IsSet(true);
-              msg = result;
-            } else if (e instanceof NoSuchLockException) {
-              result.o3 = (NoSuchLockException) e;
-              result.setO3IsSet(true);
-              msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -39647,25 +39535,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, check_lock_args args, org.apache.thrift.async.AsyncMethodCallback<LockResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.check_lock(args.rqst,resultHandler);
+      public void start(I iface, get_max_allocated_table_write_id_args args, org.apache.thrift.async.AsyncMethodCallback<MaxAllocatedTableWriteIdResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.get_max_allocated_table_write_id(args.rqst,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class unlock<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, unlock_args, Void> {
-      public unlock() {
-        super("unlock");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class seed_write_id<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, seed_write_id_args, Void> {
+      public seed_write_id() {
+        super("seed_write_id");
       }
 
-      public unlock_args getEmptyArgsInstance() {
-        return new unlock_args();
+      public seed_write_id_args getEmptyArgsInstance() {
+        return new seed_write_id_args();
       }
 
       public org.apache.thrift.async.AsyncMethodCallback<Void> 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<Void>() { 
           public void onComplete(Void o) {
-            unlock_result result = new unlock_result();
+            seed_write_id_result result = new seed_write_id_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -39679,15 +39567,11 @@ 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;
-            unlock_result result = new unlock_result();
-            if (e instanceof NoSuchLockException) {
-              result.o1 = (NoSuchLockException) e;
+            seed_write_id_result result = new seed_write_id_result();
+            if (e instanceof MetaException) {
+              result.o1 = (MetaException) e;
               result.setO1IsSet(true);
               msg = result;
-            } else if (e instanceof TxnOpenException) {
-              result.o2 = (TxnOpenException) e;
-              result.setO2IsSet(true);
-              msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -39715,26 +39599,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, unlock_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.unlock(args.rqst,resultHandler);
+      public void start(I iface, seed_write_id_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.seed_write_id(args.rqst,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class show_locks<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, show_locks_args, ShowLocksResponse> {
-      public show_locks() {
-        super("show_locks");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class seed_txn_id<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, seed_txn_id_args, Void> {
+      public seed_txn_id() {
+        super("seed_txn_id");
       }
 
-      public show_locks_args getEmptyArgsInstance() {
-        return new show_locks_args();
+      public seed_txn_id_args getEmptyArgsInstance() {
+        return new seed_txn_id_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<ShowLocksResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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<ShowLocksResponse>() { 
-          public void onComplete(ShowLocksResponse o) {
-            show_locks_result result = new show_locks_result();
-            result.success = o;
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            seed_txn_id_result result = new seed_txn_id_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -39748,8 +39631,12 @@ 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;
-            show_locks_result result = new show_locks_result();
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
+            seed_txn_id_result result = new seed_txn_id_result();
+            if (e instanceof MetaException) {
+              result.o1 = (MetaException) e;
+              result.setO1IsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -39776,25 +39663,26 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, show_locks_args args, org.apache.thrift.async.AsyncMethodCallback<ShowLocksResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.show_locks(args.rqst,resultHandler);
+      public void start(I iface, seed_txn_id_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.seed_txn_id(args.rqst,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class heartbeat<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, heartbeat_args, Void> {
-      public heartbeat() {
-        super("heartbeat");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class lock<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, lock_args, LockResponse> {
+      public lock() {
+        super("lock");
       }
 
-      public heartbeat_args getEmptyArgsInstance() {
-        return new heartbeat_args();
+      public lock_args getEmptyArgsInstance() {
+        return new lock_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<LockResponse> 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<Void>() { 
-          public void onComplete(Void o) {
-            heartbeat_result result = new heartbeat_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<LockResponse>() { 
+          public void onComplete(LockResponse o) {
+            lock_result result = new lock_result();
+            result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -39808,18 +39696,14 @@ 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;
-            heartbeat_result result = new heartbeat_result();
-            if (e instanceof NoSuchLockException) {
-              result.o1 = (NoSuchLockException) e;
+            lock_result result = new lock_result();
+            if (e instanceof NoSuchTxnException) {
+              result.o1 = (NoSuchTxnException) e;
               result.setO1IsSet(true);
               msg = result;
-            } else if (e instanceof NoSuchTxnException) {
-              result.o2 = (NoSuchTxnException) e;
-              result.setO2IsSet(true);
-              msg = result;
             } else if (e instanceof TxnAbortedException) {
-              result.o3 = (TxnAbortedException) e;
-              result.setO3IsSet(true);
+              result.o2 = (TxnAbortedException) e;
+              result.setO2IsSet(true);
               msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
@@ -39848,25 +39732,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, heartbeat_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.heartbeat(args.ids,resultHandler);
+      public void start(I iface, lock_args args, org.apache.thrift.async.AsyncMethodCallback<LockResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.lock(args.rqst,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class heartbeat_txn_range<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, heartbeat_txn_range_args, HeartbeatTxnRangeResponse> {
-      public heartbeat_txn_range() {
-        super("heartbeat_txn_range");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class check_lock<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, check_lock_args, LockResponse> {
+      public check_lock() {
+        super("check_lock");
       }
 
-      public heartbeat_txn_range_args getEmptyArgsInstance() {
-        return new heartbeat_txn_range_args();
+      public check_lock_args getEmptyArgsInstance() {
+        return new check_lock_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<HeartbeatTxnRangeResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<LockResponse> 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<HeartbeatTxnRangeResponse>() { 
-          public void onComplete(HeartbeatTxnRangeResponse o) {
-            heartbeat_txn_range_result result = new heartbeat_txn_range_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<LockResponse>() { 
+          public void onComplete(LockResponse o) {
+            check_lock_result result = new check_lock_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -39881,8 +39765,20 @@ 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;
-            heartbeat_txn_range_result result = new heartbeat_txn_range_result();
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
+            check_lock_result result = new check_lock_result();
+            if (e instanceof NoSuchTxnException) {
+              result.o1 = (NoSuchTxnException) e;
+              result.setO1IsSet(true);
+              msg = result;
+            } else if (e instanceof TxnAbortedException) {
+              result.o2 = (TxnAbortedException) e;
+              result.setO2IsSet(true);
+              msg = result;
+            } else if (e instanceof NoSuchLockException) {
+              result.o3 = (NoSuchLockException) e;
+              result.setO3IsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -39909,25 +39805,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, heartbeat_txn_range_args args, org.apache.thrift.async.AsyncMethodCallback<HeartbeatTxnRangeResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.heartbeat_txn_range(args.txns,resultHandler);
+      public void start(I iface, check_lock_args args, org.apache.thrift.async.AsyncMethodCallback<LockResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.check_lock(args.rqst,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class compact<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, compact_args, Void> {
-      public compact() {
-        super("compact");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class unlock<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, unlock_args, Void> {
+      public unlock() {
+        super("unlock");
       }
 
-      public compact_args getEmptyArgsInstance() {
-        return new compact_args();
+      public unlock_args getEmptyArgsInstance() {
+        return new unlock_args();
       }
 
       public org.apache.thrift.async.AsyncMethodCallback<Void> 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<Void>() { 
           public void onComplete(Void o) {
-            compact_result result = new compact_result();
+            unlock_result result = new unlock_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -39941,8 +39837,16 @@ 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;
-            compact_result result = new compact_result();
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
+            unlock_result result = new unlock_result();
+            if (e instanceof NoSuchLockException) {
+              result.o1 = (NoSuchLockException) e;
+              result.setO1IsSet(true);
+              msg = result;
+            } else if (e instanceof TxnOpenException) {
+              result.o2 = (TxnOpenException) e;
+              result.setO2IsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -39969,25 +39873,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, compact_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.compact(args.rqst,resultHandler);
+      public void start(I iface, unlock_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.unlock(args.rqst,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class compact2<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, compact2_args, CompactionResponse> {
-      public compact2() {
-        super("compact2");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class show_locks<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, show_locks_args, ShowLocksResponse> {
+      public show_locks() {
+        super("show_locks");
       }
 
-      public compact2_args getEmptyArgsInstance() {
-        return new compact2_args();
+      public show_locks_args getEmptyArgsInstance() {
+        return new show_locks_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<CompactionResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<ShowLocksResponse> 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<CompactionResponse>() { 
-          public void onComplete(CompactionResponse o) {
-            compact2_result result = new compact2_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<ShowLocksResponse>() { 
+          public void onComplete(ShowLocksResponse o) {
+            show_locks_result result = new show_locks_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -40002,7 +39906,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;
-            compact2_result result = new compact2_result();
+            show_locks_result result = new show_locks_result();
             if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -40030,26 +39934,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, compact2_args args, org.apache.thrift.async.AsyncMethodCallback<CompactionResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.compact2(args.rqst,resultHandler);
+      public void start(I iface, show_locks_args args, org.apache.thrift.async.AsyncMethodCallback<ShowLocksResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.show_locks(args.rqst,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class show_compact<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, show_compact_args, ShowCompactResponse> {
-      public show_compact() {
-        super("show_compact");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class heartbeat<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, heartbeat_args, Void> {
+      public heartbeat() {
+        super("heartbeat");
       }
 
-      public show_compact_args getEmptyArgsInstance() {
-        return new show_compact_args();
+      public heartbeat_args getEmptyArgsInstance() {
+        return new heartbeat_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<ShowCompactResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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<ShowCompactResponse>() { 
-          public void onComplete(ShowCompactResponse o) {
-            show_compact_result result = new show_compact_result();
-            result.success = o;
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            heartbeat_result result = new heartbeat_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -40063,8 +39966,20 @@ 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;
-            show_compact_result result = new show_compact_result();
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
+            heartbeat_result result = new heartbeat_result();
+            if (e instanceof NoSuchLockException) {
+              result.o1 = (NoSuchLockException) e;
+              result.setO1IsSet(true);
+              msg = result;
+            } else if (e instanceof NoSuchTxnException) {
+              result.o2 = (NoSuchTxnException) e;
+              result.setO2IsSet(true);
+              msg = result;
+            } else if (e instanceof TxnAbortedException) {
+              result.o3 = (TxnAbortedException) e;
+              result.setO3IsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -40091,27 +40006,26 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, show_compact_args args, org.apache.thrift.async.AsyncMethodCallback<ShowCompactResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.show_compact(args.rqst,resultHandler);
+      public void start(I iface, heartbeat_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.heartbeat(args.ids,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class submit_for_cleanup<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, submit_for_cleanup_args, java.lang.Boolean> {
-      public submit_for_cleanup() {
-        super("submit_for_cleanup");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class heartbeat_txn_range<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, heartbeat_txn_range_args, HeartbeatTxnRangeResponse> {
+      public heartbeat_txn_range() {
+        super("heartbeat_txn_range");
       }
 
-      public submit_for_cleanup_args getEmptyArgsInstance() {
-        return new submit_for_cleanup_args();
+      public heartbeat_txn_range_args getEmptyArgsInstance() {
+        return new heartbeat_txn_range_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<HeartbeatTxnRangeResponse> 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.Boolean>() { 
-          public void onComplete(java.lang.Boolean o) {
-            submit_for_cleanup_result result = new submit_for_cleanup_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<HeartbeatTxnRangeResponse>() { 
+          public void onComplete(HeartbeatTxnRangeResponse o) {
+            heartbeat_txn_range_result result = new heartbeat_txn_range_result();
             result.success = o;
-            result.setSuccessIsSet(true);
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -40125,12 +40039,8 @@ 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;
-            submit_for_cleanup_result result = new submit_for_cleanup_result();
-            if (e instanceof MetaException) {
-              result.o1 = (MetaException) e;
-              result.setO1IsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+            heartbeat_txn_range_result result = new heartbeat_txn_range_result();
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -40157,25 +40067,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, submit_for_cleanup_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
-        iface.submit_for_cleanup(args.o1, args.o2, args.o3,resultHandler);
+      public void start(I iface, heartbeat_txn_range_args args, org.apache.thrift.async.AsyncMethodCallback<HeartbeatTxnRangeResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.heartbeat_txn_range(args.txns,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_dynamic_partitions<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_dynamic_partitions_args, Void> {
-      public add_dynamic_partitions() {
-        super("add_dynamic_partitions");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class compact<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, compact_args, Void> {
+      public compact() {
+        super("compact");
       }
 
-      public add_dynamic_partitions_args getEmptyArgsInstance() {
-        return new add_dynamic_partitions_args();
+      public compact_args getEmptyArgsInstance() {
+        return new compact_args();
       }
 
       public org.apache.thrift.async.AsyncMethodCallback<Void> 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<Void>() { 
           public void onComplete(Void o) {
-            add_dynamic_partitions_result result = new add_dynamic_partitions_result();
+            compact_result result = new compact_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -40189,16 +40099,8 @@ 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;
-            add_dynamic_partitions_result result = new add_dynamic_partitions_result();
-            if (e instanceof NoSuchTxnException) {
-              result.o1 = (NoSuchTxnException) e;
-              result.setO1IsSet(true);
-              msg = result;
-            } else if (e instanceof TxnAbortedException) {
-              result.o2 = (TxnAbortedException) e;
-              result.setO2IsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+            compact_result result = new compact_result();
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -40225,25 +40127,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, add_dynamic_partitions_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.add_dynamic_partitions(args.rqst,resultHandler);
+      public void start(I iface, compact_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.compact(args.rqst,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class find_next_compact<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, find_next_compact_args, OptionalCompactionInfoStruct> {
-      public find_next_compact() {
-        super("find_next_compact");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class compact2<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, compact2_args, CompactionResponse> {
+      public compact2() {
+        super("compact2");
       }
 
-      public find_next_compact_args getEmptyArgsInstance() {
-        return new find_next_compact_args();
+      public compact2_args getEmptyArgsInstance() {
+        return new compact2_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<OptionalCompactionInfoStruct> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<CompactionResponse> 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<OptionalCompactionInfoStruct>() { 
-          public void onComplete(OptionalCompactionInfoStruct o) {
-            find_next_compact_result result = new find_next_compact_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<CompactionResponse>() { 
+          public void onComplete(CompactionResponse o) {
+            compact2_result result = new compact2_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -40258,12 +40160,8 @@ 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;
-            find_next_compact_result result = new find_next_compact_result();
-            if (e instanceof MetaException) {
-              result.o1 = (MetaException) e;
-              result.setO1IsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+            compact2_result result = new compact2_result();
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -40290,25 +40188,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, find_next_compact_args args, org.apache.thrift.async.AsyncMethodCallback<OptionalCompactionInfoStruct> resultHandler) throws org.apache.thrift.TException {
-        iface.find_next_compact(args.workerId,resultHandler);
+      public void start(I iface, compact2_args args, org.apache.thrift.async.AsyncMethodCallback<CompactionResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.compact2(args.rqst,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class find_next_compact2<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, find_next_compact2_args, OptionalCompactionInfoStruct> {
-      public find_next_compact2() {
-        super("find_next_compact2");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class show_compact<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, show_compact_args, ShowCompactResponse> {
+      public show_compact() {
+        super("show_compact");
       }
 
-      public find_next_compact2_args getEmptyArgsInstance() {
-        return new find_next_compact2_args();
+      public show_compact_args getEmptyArgsInstance() {
+        return new show_compact_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<OptionalCompactionInfoStruct> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<ShowCompactResponse> 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<OptionalCompactionInfoStruct>() { 
-          public void onComplete(OptionalCompactionInfoStruct o) {
-            find_next_compact2_result result = new find_next_compact2_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<ShowCompactResponse>() { 
+          public void onComplete(ShowCompactResponse o) {
+            show_compact_result result = new show_compact_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -40323,12 +40221,8 @@ 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;
-            find_next_compact2_result result = new find_next_compact2_result();
-            if (e instanceof MetaException) {
-              result.o1 = (MetaException) e;
-              result.setO1IsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+            show_compact_result result = new show_compact_result();
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -40355,25 +40249,27 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, find_next_compact2_args args, org.apache.thrift.async.AsyncMethodCallback<OptionalCompactionInfoStruct> resultHandler) throws org.apache.thrift.TException {
-        iface.find_next_compact2(args.rqst,resultHandler);
+      public void start(I iface, show_compact_args args, org.apache.thrift.async.AsyncMethodCallback<ShowCompactResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.show_compact(args.rqst,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class update_compactor_state<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, update_compactor_state_args, Void> {
-      public update_compactor_state() {
-        super("update_compactor_state");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class submit_for_cleanup<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, submit_for_cleanup_args, java.lang.Boolean> {
+      public submit_for_cleanup() {
+        super("submit_for_cleanup");
       }
 
-      public update_compactor_state_args getEmptyArgsInstance() {
-        return new update_compactor_state_args();
+      public submit_for_cleanup_args getEmptyArgsInstance() {
+        return new submit_for_cleanup_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> 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<Void>() { 
-          public void onComplete(Void o) {
-            update_compactor_state_result result = new update_compactor_state_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean>() { 
+          public void onComplete(java.lang.Boolean o) {
+            submit_for_cleanup_result result = new submit_for_cleanup_result();
+            result.success = o;
+            result.setSuccessIsSet(true);
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -40387,8 +40283,12 @@ 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;
-            update_compactor_state_result result = new update_compactor_state_result();
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
+            submit_for_cleanup_result result = new submit_for_cleanup_result();
+            if (e instanceof MetaException) {
+              result.o1 = (MetaException) e;
+              result.setO1IsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -40415,26 +40315,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, update_compactor_state_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.update_compactor_state(args.cr, args.txn_id,resultHandler);
+      public void start(I iface, submit_for_cleanup_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
+        iface.submit_for_cleanup(args.o1, args.o2, args.o3,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class find_columns_with_stats<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, find_columns_with_stats_args, java.util.List<java.lang.String>> {
-      public find_columns_with_stats() {
-        super("find_columns_with_stats");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_dynamic_partitions<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_dynamic_partitions_args, Void> {
+      public add_dynamic_partitions() {
+        super("add_dynamic_partitions");
       }
 
-      public find_columns_with_stats_args getEmptyArgsInstance() {
-        return new find_columns_with_stats_args();
+      public add_dynamic_partitions_args getEmptyArgsInstance() {
+        return new add_dynamic_partitions_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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.util.List<java.lang.String>>() { 
-          public void onComplete(java.util.List<java.lang.String> o) {
-            find_columns_with_stats_result result = new find_columns_with_stats_result();
-            result.success = o;
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            add_dynamic_partitions_result result = new add_dynamic_partitions_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -40448,8 +40347,16 @@ 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;
-            find_columns_with_stats_result result = new find_columns_with_stats_result();
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
+            add_dynamic_partitions_result result = new add_dynamic_partitions_result();
+            if (e instanceof NoSuchTxnException) {
+              result.o1 = (NoSuchTxnException) e;
+              result.setO1IsSet(true);
+              msg = result;
+            } else if (e instanceof TxnAbortedException) {
+              result.o2 = (TxnAbortedException) e;
+              result.setO2IsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -40476,25 +40383,26 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, find_columns_with_stats_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler) throws org.apache.thrift.TException {
-        iface.find_columns_with_stats(args.cr,resultHandler);
+      public void start(I iface, add_dynamic_partitions_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.add_dynamic_partitions(args.rqst,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class mark_cleaned<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, mark_cleaned_args, Void> {
-      public mark_cleaned() {
-        super("mark_cleaned");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class find_next_compact<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, find_next_compact_args, OptionalCompactionInfoStruct> {
+      public find_next_compact() {
+        super("find_next_compact");
       }
 
-      public mark_cleaned_args getEmptyArgsInstance() {
-        return new mark_cleaned_args();
+      public find_next_compact_args getEmptyArgsInstance() {
+        return new find_next_compact_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<OptionalCompactionInfoStruct> 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<Void>() { 
-          public void onComplete(Void o) {
-            mark_cleaned_result result = new mark_cleaned_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<OptionalCompactionInfoStruct>() { 
+          public void onComplete(OptionalCompactionInfoStruct o) {
+            find_next_compact_result result = new find_next_compact_result();
+            result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -40508,7 +40416,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;
-            mark_cleaned_result result = new mark_cleaned_result();
+            find_next_compact_result result = new find_next_compact_result();
             if (e instanceof MetaException) {
               result.o1 = (MetaException) e;
               result.setO1IsSet(true);
@@ -40540,25 +40448,26 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, mark_cleaned_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.mark_cleaned(args.cr,resultHandler);
+      public void start(I iface, find_next_compact_args args, org.apache.thrift.async.AsyncMethodCallback<OptionalCompactionInfoStruct> resultHandler) throws org.apache.thrift.TException {
+        iface.find_next_compact(args.workerId,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class mark_compacted<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, mark_compacted_args, Void> {
-      public mark_compacted() {
-        super("mark_compacted");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class find_next_compact2<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, find_next_compact2_args, OptionalCompactionInfoStruct> {
+      public find_next_compact2() {
+        super("find_next_compact2");
       }
 
-      public mark_compacted_args getEmptyArgsInstance() {
-        return new mark_compacted_args();
+      public find_next_compact2_args getEmptyArgsInstance() {
+        return new find_next_compact2_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<OptionalCompactionInfoStruct> 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<Void>() { 
-          public void onComplete(Void o) {
-            mark_compacted_result result = new mark_compacted_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<OptionalCompactionInfoStruct>() { 
+          public void onComplete(OptionalCompactionInfoStruct o) {
+            find_next_compact2_result result = new find_next_compact2_result();
+            result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -40572,7 +40481,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;
-            mark_compacted_result result = new mark_compacted_result();
+            find_next_compact2_result result = new find_next_compact2_result();
             if (e instanceof MetaException) {
               result.o1 = (MetaException) e;
               result.setO1IsSet(true);
@@ -40604,25 +40513,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, mark_compacted_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.mark_compacted(args.cr,resultHandler);
+      public void start(I iface, find_next_compact2_args args, org.apache.thrift.async.AsyncMethodCallback<OptionalCompactionInfoStruct> resultHandler) throws org.apache.thrift.TException {
+        iface.find_next_compact2(args.rqst,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class mark_failed<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, mark_failed_args, Void> {
-      public mark_failed() {
-        super("mark_failed");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class update_compactor_state<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, update_compactor_state_args, Void> {
+      public update_compactor_state() {
+        super("update_compactor_state");
       }
 
-      public mark_failed_args getEmptyArgsInstance() {
-        return new mark_failed_args();
+      public update_compactor_state_args getEmptyArgsInstance() {
+        return new update_compactor_state_args();
       }
 
       public org.apache.thrift.async.AsyncMethodCallback<Void> 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<Void>() { 
           public void onComplete(Void o) {
-            mark_failed_result result = new mark_failed_result();
+            update_compactor_state_result result = new update_compactor_state_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -40636,12 +40545,8 @@ 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;
-            mark_failed_result result = new mark_failed_result();
-            if (e instanceof MetaException) {
-              result.o1 = (MetaException) e;
-              result.setO1IsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+            update_compactor_state_result result = new update_compactor_state_result();
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -40668,25 +40573,26 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, mark_failed_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.mark_failed(args.cr,resultHandler);
+      public void start(I iface, update_compactor_state_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.update_compactor_state(args.cr, args.txn_id,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class mark_refused<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, mark_refused_args, Void> {
-      public mark_refused() {
-        super("mark_refused");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class find_columns_with_stats<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, find_columns_with_stats_args, java.util.List<java.lang.String>> {
+      public find_columns_with_stats() {
+        super("find_columns_with_stats");
       }
 
-      public mark_refused_args getEmptyArgsInstance() {
-        return new mark_refused_args();
+      public find_columns_with_stats_args getEmptyArgsInstance() {
+        return new find_columns_with_stats_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> 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<Void>() { 
-          public void onComplete(Void o) {
-            mark_refused_result result = new mark_refused_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>>() { 
+          public void onComplete(java.util.List<java.lang.String> o) {
+            find_columns_with_stats_result result = new find_columns_with_stats_result();
+            result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -40700,12 +40606,8 @@ 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;
-            mark_refused_result result = new mark_refused_result();
-            if (e instanceof MetaException) {
-              result.o1 = (MetaException) e;
-              result.setO1IsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+            find_columns_with_stats_result result = new find_columns_with_stats_result();
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -40732,27 +40634,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, mark_refused_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.mark_refused(args.cr,resultHandler);
+      public void start(I iface, find_columns_with_stats_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler) throws org.apache.thrift.TException {
+        iface.find_columns_with_stats(args.cr,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class update_compaction_metrics_data<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, update_compaction_metrics_data_args, java.lang.Boolean> {
-      public update_compaction_metrics_data() {
-        super("update_compaction_metrics_data");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class mark_cleaned<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, mark_cleaned_args, Void> {
+      public mark_cleaned() {
+        super("mark_cleaned");
       }
 
-      public update_compaction_metrics_data_args getEmptyArgsInstance() {
-        return new update_compaction_metrics_data_args();
+      public mark_cleaned_args getEmptyArgsInstance() {
+        return new mark_cleaned_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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.Boolean>() { 
-          public void onComplete(java.lang.Boolean o) {
-            update_compaction_metrics_data_result result = new update_compaction_metrics_data_result();
-            result.success = o;
-            result.setSuccessIsSet(true);
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            mark_cleaned_result result = new mark_cleaned_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -40766,7 +40666,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;
-            update_compaction_metrics_data_result result = new update_compaction_metrics_data_result();
+            mark_cleaned_result result = new mark_cleaned_result();
             if (e instanceof MetaException) {
               result.o1 = (MetaException) e;
               result.setO1IsSet(true);
@@ -40798,25 +40698,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, update_compaction_metrics_data_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
-        iface.update_compaction_metrics_data(args.data,resultHandler);
+      public void start(I iface, mark_cleaned_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.mark_cleaned(args.cr,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class remove_compaction_metrics_data<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, remove_compaction_metrics_data_args, Void> {
-      public remove_compaction_metrics_data() {
-        super("remove_compaction_metrics_data");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class mark_compacted<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, mark_compacted_args, Void> {
+      public mark_compacted() {
+        super("mark_compacted");
       }
 
-      public remove_compaction_metrics_data_args getEmptyArgsInstance() {
-        return new remove_compaction_metrics_data_args();
+      public mark_compacted_args getEmptyArgsInstance() {
+        return new mark_compacted_args();
       }
 
       public org.apache.thrift.async.AsyncMethodCallback<Void> 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<Void>() { 
           public void onComplete(Void o) {
-            remove_compaction_metrics_data_result result = new remove_compaction_metrics_data_result();
+            mark_compacted_result result = new mark_compacted_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -40830,7 +40730,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;
-            remove_compaction_metrics_data_result result = new remove_compaction_metrics_data_result();
+            mark_compacted_result result = new mark_compacted_result();
             if (e instanceof MetaException) {
               result.o1 = (MetaException) e;
               result.setO1IsSet(true);
@@ -40862,25 +40762,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, remove_compaction_metrics_data_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.remove_compaction_metrics_data(args.request,resultHandler);
+      public void start(I iface, mark_compacted_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.mark_compacted(args.cr,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class set_hadoop_jobid<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, set_hadoop_jobid_args, Void> {
-      public set_hadoop_jobid() {
-        super("set_hadoop_jobid");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class mark_failed<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, mark_failed_args, Void> {
+      public mark_failed() {
+        super("mark_failed");
       }
 
-      public set_hadoop_jobid_args getEmptyArgsInstance() {
-        return new set_hadoop_jobid_args();
+      public mark_failed_args getEmptyArgsInstance() {
+        return new mark_failed_args();
       }
 
       public org.apache.thrift.async.AsyncMethodCallback<Void> 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<Void>() { 
           public void onComplete(Void o) {
-            set_hadoop_jobid_result result = new set_hadoop_jobid_result();
+            mark_failed_result result = new mark_failed_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -40894,8 +40794,12 @@ 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;
-            set_hadoop_jobid_result result = new set_hadoop_jobid_result();
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
+            mark_failed_result result = new mark_failed_result();
+            if (e instanceof MetaException) {
+              result.o1 = (MetaException) e;
+              result.setO1IsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -40922,26 +40826,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, set_hadoop_jobid_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.set_hadoop_jobid(args.jobId, args.cq_id,resultHandler);
+      public void start(I iface, mark_failed_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.mark_failed(args.cr,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_latest_committed_compaction_info<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_latest_committed_compaction_info_args, GetLatestCommittedCompactionInfoResponse> {
-      public get_latest_committed_compaction_info() {
-        super("get_latest_committed_compaction_info");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class mark_refused<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, mark_refused_args, Void> {
+      public mark_refused() {
+        super("mark_refused");
       }
 
-      public get_latest_committed_compaction_info_args getEmptyArgsInstance() {
-        return new get_latest_committed_compaction_info_args();
+      public mark_refused_args getEmptyArgsInstance() {
+        return new mark_refused_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<GetLatestCommittedCompactionInfoResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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<GetLatestCommittedCompactionInfoResponse>() { 
-          public void onComplete(GetLatestCommittedCompactionInfoResponse o) {
-            get_latest_committed_compaction_info_result result = new get_latest_committed_compaction_info_result();
-            result.success = o;
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            mark_refused_result result = new mark_refused_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -40955,8 +40858,12 @@ 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_committed_compaction_info_result result = new get_latest_committed_compaction_info_result();
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
+            mark_refused_result result = new mark_refused_result();
+            if (e instanceof MetaException) {
+              result.o1 = (MetaException) e;
+              result.setO1IsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -40983,26 +40890,27 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, get_latest_committed_compaction_info_args args, org.apache.thrift.async.AsyncMethodCallback<GetLatestCommittedCompactionInfoResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.get_latest_committed_compaction_info(args.rqst,resultHandler);
+      public void start(I iface, mark_refused_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.mark_refused(args.cr,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_next_notification<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_next_notification_args, NotificationEventResponse> {
-      public get_next_notification() {
-        super("get_next_notification");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class update_compaction_metrics_data<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, update_compaction_metrics_data_args, java.lang.Boolean> {
+      public update_compaction_metrics_data() {
+        super("update_compaction_metrics_data");
       }
 
-      public get_next_notification_args getEmptyArgsInstance() {
-        return new get_next_notification_args();
+      public update_compaction_metrics_data_args getEmptyArgsInstance() {
+        return new update_compaction_metrics_data_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<NotificationEventResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> 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<NotificationEventResponse>() { 
-          public void onComplete(NotificationEventResponse o) {
-            get_next_notification_result result = new get_next_notification_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean>() { 
+          public void onComplete(java.lang.Boolean o) {
+            update_compaction_metrics_data_result result = new update_compaction_metrics_data_result();
             result.success = o;
+            result.setSuccessIsSet(true);
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -41016,8 +40924,12 @@ 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_next_notification_result result = new get_next_notification_result();
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
+            update_compaction_metrics_data_result result = new update_compaction_metrics_data_result();
+            if (e instanceof MetaException) {
+              result.o1 = (MetaException) e;
+              result.setO1IsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -41044,26 +40956,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, get_next_notification_args args, org.apache.thrift.async.AsyncMethodCallback<NotificationEventResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.get_next_notification(args.rqst,resultHandler);
+      public void start(I iface, update_compaction_metrics_data_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
+        iface.update_compaction_metrics_data(args.data,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_current_notificationEventId<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_current_notificationEventId_args, CurrentNotificationEventId> {
-      public get_current_notificationEventId() {
-        super("get_current_notificationEventId");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class remove_compaction_metrics_data<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, remove_compaction_metrics_data_args, Void> {
+      public remove_compaction_metrics_data() {
+        super("remove_compaction_metrics_data");
       }
 
-      public get_current_notificationEventId_args getEmptyArgsInstance() {
-        return new get_current_notificationEventId_args();
+      public remove_compaction_metrics_data_args getEmptyArgsInstance() {
+        return new remove_compaction_metrics_data_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<CurrentNotificationEventId> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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<CurrentNotificationEventId>() { 
-          public void onComplete(CurrentNotificationEventId o) {
-            get_current_notificationEventId_result result = new get_current_notificationEventId_result();
-            result.success = o;
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            remove_compaction_metrics_data_result result = new remove_compaction_metrics_data_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -41077,8 +40988,12 @@ 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_current_notificationEventId_result result = new get_current_notificationEventId_result();
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
+            remove_compaction_metrics_data_result result = new remove_compaction_metrics_data_result();
+            if (e instanceof MetaException) {
+              result.o1 = (MetaException) e;
+              result.setO1IsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -41105,26 +41020,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, get_current_notificationEventId_args args, org.apache.thrift.async.AsyncMethodCallback<CurrentNotificationEventId> resultHandler) throws org.apache.thrift.TException {
-        iface.get_current_notificationEventId(resultHandler);
+      public void start(I iface, remove_compaction_metrics_data_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.remove_compaction_metrics_data(args.request,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_notification_events_count<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_notification_events_count_args, NotificationEventsCountResponse> {
-      public get_notification_events_count() {
-        super("get_notification_events_count");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class set_hadoop_jobid<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, set_hadoop_jobid_args, Void> {
+      public set_hadoop_jobid() {
+        super("set_hadoop_jobid");
       }
 
-      public get_notification_events_count_args getEmptyArgsInstance() {
-        return new get_notification_events_count_args();
+      public set_hadoop_jobid_args getEmptyArgsInstance() {
+        return new set_hadoop_jobid_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<NotificationEventsCountResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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<NotificationEventsCountResponse>() { 
-          public void onComplete(NotificationEventsCountResponse o) {
-            get_notification_events_count_result result = new get_notification_events_count_result();
-            result.success = o;
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            set_hadoop_jobid_result result = new set_hadoop_jobid_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -41138,7 +41052,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_notification_events_count_result result = new get_notification_events_count_result();
+            set_hadoop_jobid_result result = new set_hadoop_jobid_result();
             if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -41166,25 +41080,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, get_notification_events_count_args args, org.apache.thrift.async.AsyncMethodCallback<NotificationEventsCountResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.get_notification_events_count(args.rqst,resultHandler);
+      public void start(I iface, set_hadoop_jobid_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.set_hadoop_jobid(args.jobId, args.cq_id,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class fire_listener_event<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, fire_listener_event_args, FireEventResponse> {
-      public fire_listener_event() {
-        super("fire_listener_event");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_latest_committed_compaction_info<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_latest_committed_compaction_info_args, GetLatestCommittedCompactionInfoResponse> {
+      public get_latest_committed_compaction_info() {
+        super("get_latest_committed_compaction_info");
       }
 
-      public fire_listener_event_args getEmptyArgsInstance() {
-        return new fire_listener_event_args();
+      public get_latest_committed_compaction_info_args getEmptyArgsInstance() {
+        return new get_latest_committed_compaction_info_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<FireEventResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<GetLatestCommittedCompactionInfoResponse> 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<FireEventResponse>() { 
-          public void onComplete(FireEventResponse o) {
-            fire_listener_event_result result = new fire_listener_event_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<GetLatestCommittedCompactionInfoResponse>() { 
+          public void onComplete(GetLatestCommittedCompactionInfoResponse o) {
+            get_latest_committed_compaction_info_result result = new get_latest_committed_compaction_info_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -41199,7 +41113,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;
-            fire_listener_event_result result = new fire_listener_event_result();
+            get_latest_committed_compaction_info_result result = new get_latest_committed_compaction_info_result();
             if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -41227,25 +41141,26 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, fire_listener_event_args args, org.apache.thrift.async.AsyncMethodCallback<FireEventResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.fire_listener_event(args.rqst,resultHandler);
+      public void start(I iface, get_latest_committed_compaction_info_args args, org.apache.thrift.async.AsyncMethodCallback<GetLatestCommittedCompactionInfoResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.get_latest_committed_compaction_info(args.rqst,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class flushCache<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, flushCache_args, Void> {
-      public flushCache() {
-        super("flushCache");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_next_notification<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_next_notification_args, NotificationEventResponse> {
+      public get_next_notification() {
+        super("get_next_notification");
       }
 
-      public flushCache_args getEmptyArgsInstance() {
-        return new flushCache_args();
+      public get_next_notification_args getEmptyArgsInstance() {
+        return new get_next_notification_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<NotificationEventResponse> 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<Void>() { 
-          public void onComplete(Void o) {
-            flushCache_result result = new flushCache_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<NotificationEventResponse>() { 
+          public void onComplete(NotificationEventResponse o) {
+            get_next_notification_result result = new get_next_notification_result();
+            result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -41259,7 +41174,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;
-            flushCache_result result = new flushCache_result();
+            get_next_notification_result result = new get_next_notification_result();
             if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -41287,25 +41202,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, flushCache_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.flushCache(resultHandler);
+      public void start(I iface, get_next_notification_args args, org.apache.thrift.async.AsyncMethodCallback<NotificationEventResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.get_next_notification(args.rqst,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_write_notification_log<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_write_notification_log_args, WriteNotificationLogResponse> {
-      public add_write_notification_log() {
-        super("add_write_notification_log");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_current_notificationEventId<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_current_notificationEventId_args, CurrentNotificationEventId> {
+      public get_current_notificationEventId() {
+        super("get_current_notificationEventId");
       }
 
-      public add_write_notification_log_args getEmptyArgsInstance() {
-        return new add_write_notification_log_args();
+      public get_current_notificationEventId_args getEmptyArgsInstance() {
+        return new get_current_notificationEventId_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<WriteNotificationLogResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<CurrentNotificationEventId> 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<WriteNotificationLogResponse>() { 
-          public void onComplete(WriteNotificationLogResponse o) {
-            add_write_notification_log_result result = new add_write_notification_log_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<CurrentNotificationEventId>() { 
+          public void onComplete(CurrentNotificationEventId o) {
+            get_current_notificationEventId_result result = new get_current_notificationEventId_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -41320,7 +41235,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;
-            add_write_notification_log_result result = new add_write_notification_log_result();
+            get_current_notificationEventId_result result = new get_current_notificationEventId_result();
             if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -41348,25 +41263,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, add_write_notification_log_args args, org.apache.thrift.async.AsyncMethodCallback<WriteNotificationLogResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.add_write_notification_log(args.rqst,resultHandler);
+      public void start(I iface, get_current_notificationEventId_args args, org.apache.thrift.async.AsyncMethodCallback<CurrentNotificationEventId> resultHandler) throws org.apache.thrift.TException {
+        iface.get_current_notificationEventId(resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_write_notification_log_in_batch<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_write_notification_log_in_batch_args, WriteNotificationLogBatchResponse> {
-      public add_write_notification_log_in_batch() {
-        super("add_write_notification_log_in_batch");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_notification_events_count<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_notification_events_count_args, NotificationEventsCountResponse> {
+      public get_notification_events_count() {
+        super("get_notification_events_count");
       }
 
-      public add_write_notification_log_in_batch_args getEmptyArgsInstance() {
-        return new add_write_notification_log_in_batch_args();
+      public get_notification_events_count_args getEmptyArgsInstance() {
+        return new get_notification_events_count_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<WriteNotificationLogBatchResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<NotificationEventsCountResponse> 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<WriteNotificationLogBatchResponse>() { 
-          public void onComplete(WriteNotificationLogBatchResponse o) {
-            add_write_notification_log_in_batch_result result = new add_write_notification_log_in_batch_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<NotificationEventsCountResponse>() { 
+          public void onComplete(NotificationEventsCountResponse o) {
+            get_notification_events_count_result result = new get_notification_events_count_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -41381,7 +41296,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;
-            add_write_notification_log_in_batch_result result = new add_write_notification_log_in_batch_result();
+            get_notification_events_count_result result = new get_notification_events_count_result();
             if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -41409,25 +41324,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, add_write_notification_log_in_batch_args args, org.apache.thrift.async.AsyncMethodCallback<WriteNotificationLogBatchResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.add_write_notification_log_in_batch(args.rqst,resultHandler);
+      public void start(I iface, get_notification_events_count_args args, org.apache.thrift.async.AsyncMethodCallback<NotificationEventsCountResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.get_notification_events_count(args.rqst,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class cm_recycle<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, cm_recycle_args, CmRecycleResponse> {
-      public cm_recycle() {
-        super("cm_recycle");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class fire_listener_event<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, fire_listener_event_args, FireEventResponse> {
+      public fire_listener_event() {
+        super("fire_listener_event");
       }
 
-      public cm_recycle_args getEmptyArgsInstance() {
-        return new cm_recycle_args();
+      public fire_listener_event_args getEmptyArgsInstance() {
+        return new fire_listener_event_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<CmRecycleResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<FireEventResponse> 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<CmRecycleResponse>() { 
-          public void onComplete(CmRecycleResponse o) {
-            cm_recycle_result result = new cm_recycle_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<FireEventResponse>() { 
+          public void onComplete(FireEventResponse o) {
+            fire_listener_event_result result = new fire_listener_event_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -41442,12 +41357,8 @@ 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;
-            cm_recycle_result result = new cm_recycle_result();
-            if (e instanceof MetaException) {
-              result.o1 = (MetaException) e;
-              result.setO1IsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+            fire_listener_event_result result = new fire_listener_event_result();
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -41474,26 +41385,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, cm_recycle_args args, org.apache.thrift.async.AsyncMethodCallback<CmRecycleResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.cm_recycle(args.request,resultHandler);
+      public void start(I iface, fire_listener_event_args args, org.apache.thrift.async.AsyncMethodCallback<FireEventResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.fire_listener_event(args.rqst,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_file_metadata_by_expr<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_file_metadata_by_expr_args, GetFileMetadataByExprResult> {
-      public get_file_metadata_by_expr() {
-        super("get_file_metadata_by_expr");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class flushCache<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, flushCache_args, Void> {
+      public flushCache() {
+        super("flushCache");
       }
 
-      public get_file_metadata_by_expr_args getEmptyArgsInstance() {
-        return new get_file_metadata_by_expr_args();
+      public flushCache_args getEmptyArgsInstance() {
+        return new flushCache_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<GetFileMetadataByExprResult> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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<GetFileMetadataByExprResult>() { 
-          public void onComplete(GetFileMetadataByExprResult o) {
-            get_file_metadata_by_expr_result result = new get_file_metadata_by_expr_result();
-            result.success = o;
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            flushCache_result result = new flushCache_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -41507,7 +41417,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_file_metadata_by_expr_result result = new get_file_metadata_by_expr_result();
+            flushCache_result result = new flushCache_result();
             if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -41535,25 +41445,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, get_file_metadata_by_expr_args args, org.apache.thrift.async.AsyncMethodCallback<GetFileMetadataByExprResult> resultHandler) throws org.apache.thrift.TException {
-        iface.get_file_metadata_by_expr(args.req,resultHandler);
+      public void start(I iface, flushCache_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.flushCache(resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_file_metadata<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_file_metadata_args, GetFileMetadataResult> {
-      public get_file_metadata() {
-        super("get_file_metadata");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_write_notification_log<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_write_notification_log_args, WriteNotificationLogResponse> {
+      public add_write_notification_log() {
+        super("add_write_notification_log");
       }
 
-      public get_file_metadata_args getEmptyArgsInstance() {
-        return new get_file_metadata_args();
+      public add_write_notification_log_args getEmptyArgsInstance() {
+        return new add_write_notification_log_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<GetFileMetadataResult> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<WriteNotificationLogResponse> 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<GetFileMetadataResult>() { 
-          public void onComplete(GetFileMetadataResult o) {
-            get_file_metadata_result result = new get_file_metadata_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<WriteNotificationLogResponse>() { 
+          public void onComplete(WriteNotificationLogResponse o) {
+            add_write_notification_log_result result = new add_write_notification_log_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -41568,7 +41478,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_file_metadata_result result = new get_file_metadata_result();
+            add_write_notification_log_result result = new add_write_notification_log_result();
             if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -41596,25 +41506,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, get_file_metadata_args args, org.apache.thrift.async.AsyncMethodCallback<GetFileMetadataResult> resultHandler) throws org.apache.thrift.TException {
-        iface.get_file_metadata(args.req,resultHandler);
+      public void start(I iface, add_write_notification_log_args args, org.apache.thrift.async.AsyncMethodCallback<WriteNotificationLogResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.add_write_notification_log(args.rqst,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class put_file_metadata<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, put_file_metadata_args, PutFileMetadataResult> {
-      public put_file_metadata() {
-        super("put_file_metadata");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_write_notification_log_in_batch<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_write_notification_log_in_batch_args, WriteNotificationLogBatchResponse> {
+      public add_write_notification_log_in_batch() {
+        super("add_write_notification_log_in_batch");
       }
 
-      public put_file_metadata_args getEmptyArgsInstance() {
-        return new put_file_metadata_args();
+      public add_write_notification_log_in_batch_args getEmptyArgsInstance() {
+        return new add_write_notification_log_in_batch_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<PutFileMetadataResult> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<WriteNotificationLogBatchResponse> 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<PutFileMetadataResult>() { 
-          public void onComplete(PutFileMetadataResult o) {
-            put_file_metadata_result result = new put_file_metadata_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<WriteNotificationLogBatchResponse>() { 
+          public void onComplete(WriteNotificationLogBatchResponse o) {
+            add_write_notification_log_in_batch_result result = new add_write_notification_log_in_batch_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -41629,7 +41539,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;
-            put_file_metadata_result result = new put_file_metadata_result();
+            add_write_notification_log_in_batch_result result = new add_write_notification_log_in_batch_result();
             if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -41657,25 +41567,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, put_file_metadata_args args, org.apache.thrift.async.AsyncMethodCallback<PutFileMetadataResult> resultHandler) throws org.apache.thrift.TException {
-        iface.put_file_metadata(args.req,resultHandler);
+      public void start(I iface, add_write_notification_log_in_batch_args args, org.apache.thrift.async.AsyncMethodCallback<WriteNotificationLogBatchResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.add_write_notification_log_in_batch(args.rqst,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class clear_file_metadata<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, clear_file_metadata_args, ClearFileMetadataResult> {
-      public clear_file_metadata() {
-        super("clear_file_metadata");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class cm_recycle<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, cm_recycle_args, CmRecycleResponse> {
+      public cm_recycle() {
+        super("cm_recycle");
       }
 
-      public clear_file_metadata_args getEmptyArgsInstance() {
-        return new clear_file_metadata_args();
+      public cm_recycle_args getEmptyArgsInstance() {
+        return new cm_recycle_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<ClearFileMetadataResult> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<CmRecycleResponse> 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<ClearFileMetadataResult>() { 
-          public void onComplete(ClearFileMetadataResult o) {
-            clear_file_metadata_result result = new clear_file_metadata_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<CmRecycleResponse>() { 
+          public void onComplete(CmRecycleResponse o) {
+            cm_recycle_result result = new cm_recycle_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -41690,8 +41600,12 @@ 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;
-            clear_file_metadata_result result = new clear_file_metadata_result();
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
+            cm_recycle_result result = new cm_recycle_result();
+            if (e instanceof MetaException) {
+              result.o1 = (MetaException) e;
+              result.setO1IsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -41718,25 +41632,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, clear_file_metadata_args args, org.apache.thrift.async.AsyncMethodCallback<ClearFileMetadataResult> resultHandler) throws org.apache.thrift.TException {
-        iface.clear_file_metadata(args.req,resultHandler);
+      public void start(I iface, cm_recycle_args args, org.apache.thrift.async.AsyncMethodCallback<CmRecycleResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.cm_recycle(args.request,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class cache_file_metadata<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, cache_file_metadata_args, CacheFileMetadataResult> {
-      public cache_file_metadata() {
-        super("cache_file_metadata");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_file_metadata_by_expr<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_file_metadata_by_expr_args, GetFileMetadataByExprResult> {
+      public get_file_metadata_by_expr() {
+        super("get_file_metadata_by_expr");
       }
 
-      public cache_file_metadata_args getEmptyArgsInstance() {
-        return new cache_file_metadata_args();
+      public get_file_metadata_by_expr_args getEmptyArgsInstance() {
+        return new get_file_metadata_by_expr_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<CacheFileMetadataResult> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<GetFileMetadataByExprResult> 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<CacheFileMetadataResult>() { 
-          public void onComplete(CacheFileMetadataResult o) {
-            cache_file_metadata_result result = new cache_file_metadata_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<GetFileMetadataByExprResult>() { 
+          public void onComplete(GetFileMetadataByExprResult o) {
+            get_file_metadata_by_expr_result result = new get_file_metadata_by_expr_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -41751,7 +41665,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;
-            cache_file_metadata_result result = new cache_file_metadata_result();
+            get_file_metadata_by_expr_result result = new get_file_metadata_by_expr_result();
             if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -41779,25 +41693,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, cache_file_metadata_args args, org.apache.thrift.async.AsyncMethodCallback<CacheFileMetadataResult> resultHandler) throws org.apache.thrift.TException {
-        iface.cache_file_metadata(args.req,resultHandler);
+      public void start(I iface, get_file_metadata_by_expr_args args, org.apache.thrift.async.AsyncMethodCallback<GetFileMetadataByExprResult> resultHandler) throws org.apache.thrift.TException {
+        iface.get_file_metadata_by_expr(args.req,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_metastore_db_uuid<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_metastore_db_uuid_args, java.lang.String> {
-      public get_metastore_db_uuid() {
-        super("get_metastore_db_uuid");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_file_metadata<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_file_metadata_args, GetFileMetadataResult> {
+      public get_file_metadata() {
+        super("get_file_metadata");
       }
 
-      public get_metastore_db_uuid_args getEmptyArgsInstance() {
-        return new get_metastore_db_uuid_args();
+      public get_file_metadata_args getEmptyArgsInstance() {
+        return new get_file_metadata_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<java.lang.String> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<GetFileMetadataResult> 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.String>() { 
-          public void onComplete(java.lang.String o) {
-            get_metastore_db_uuid_result result = new get_metastore_db_uuid_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<GetFileMetadataResult>() { 
+          public void onComplete(GetFileMetadataResult o) {
+            get_file_metadata_result result = new get_file_metadata_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -41812,12 +41726,8 @@ 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_metastore_db_uuid_result result = new get_metastore_db_uuid_result();
-            if (e instanceof MetaException) {
-              result.o1 = (MetaException) e;
-              result.setO1IsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+            get_file_metadata_result result = new get_file_metadata_result();
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -41844,25 +41754,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, get_metastore_db_uuid_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException {
-        iface.get_metastore_db_uuid(resultHandler);
+      public void start(I iface, get_file_metadata_args args, org.apache.thrift.async.AsyncMethodCallback<GetFileMetadataResult> resultHandler) throws org.apache.thrift.TException {
+        iface.get_file_metadata(args.req,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class create_resource_plan<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, create_resource_plan_args, WMCreateResourcePlanResponse> {
-      public create_resource_plan() {
-        super("create_resource_plan");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class put_file_metadata<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, put_file_metadata_args, PutFileMetadataResult> {
+      public put_file_metadata() {
+        super("put_file_metadata");
       }
 
-      public create_resource_plan_args getEmptyArgsInstance() {
-        return new create_resource_plan_args();
+      public put_file_metadata_args getEmptyArgsInstance() {
+        return new put_file_metadata_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<WMCreateResourcePlanResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<PutFileMetadataResult> 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<WMCreateResourcePlanResponse>() { 
-          public void onComplete(WMCreateResourcePlanResponse o) {
-            create_resource_plan_result result = new create_resource_plan_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<PutFileMetadataResult>() { 
+          public void onComplete(PutFileMetadataResult o) {
+            put_file_metadata_result result = new put_file_metadata_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -41877,20 +41787,8 @@ 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;
-            create_resource_plan_result result = new create_resource_plan_result();
-            if (e instanceof AlreadyExistsException) {
-              result.o1 = (AlreadyExistsException) e;
-              result.setO1IsSet(true);
-              msg = result;
-            } else if (e instanceof InvalidObjectException) {
-              result.o2 = (InvalidObjectException) e;
-              result.setO2IsSet(true);
-              msg = result;
-            } else if (e instanceof MetaException) {
-              result.o3 = (MetaException) e;
-              result.setO3IsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+            put_file_metadata_result result = new put_file_metadata_result();
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -41917,25 +41815,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, create_resource_plan_args args, org.apache.thrift.async.AsyncMethodCallback<WMCreateResourcePlanResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.create_resource_plan(args.request,resultHandler);
+      public void start(I iface, put_file_metadata_args args, org.apache.thrift.async.AsyncMethodCallback<PutFileMetadataResult> resultHandler) throws org.apache.thrift.TException {
+        iface.put_file_metadata(args.req,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_resource_plan<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_resource_plan_args, WMGetResourcePlanResponse> {
-      public get_resource_plan() {
-        super("get_resource_plan");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class clear_file_metadata<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, clear_file_metadata_args, ClearFileMetadataResult> {
+      public clear_file_metadata() {
+        super("clear_file_metadata");
       }
 
-      public get_resource_plan_args getEmptyArgsInstance() {
-        return new get_resource_plan_args();
+      public clear_file_metadata_args getEmptyArgsInstance() {
+        return new clear_file_metadata_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<WMGetResourcePlanResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<ClearFileMetadataResult> 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<WMGetResourcePlanResponse>() { 
-          public void onComplete(WMGetResourcePlanResponse o) {
-            get_resource_plan_result result = new get_resource_plan_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<ClearFileMetadataResult>() { 
+          public void onComplete(ClearFileMetadataResult o) {
+            clear_file_metadata_result result = new clear_file_metadata_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -41950,16 +41848,8 @@ 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_resource_plan_result result = new get_resource_plan_result();
-            if (e instanceof NoSuchObjectException) {
-              result.o1 = (NoSuchObjectException) e;
-              result.setO1IsSet(true);
-              msg = result;
-            } else if (e instanceof MetaException) {
-              result.o2 = (MetaException) e;
-              result.setO2IsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+            clear_file_metadata_result result = new clear_file_metadata_result();
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -41986,25 +41876,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, get_resource_plan_args args, org.apache.thrift.async.AsyncMethodCallback<WMGetResourcePlanResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.get_resource_plan(args.request,resultHandler);
+      public void start(I iface, clear_file_metadata_args args, org.apache.thrift.async.AsyncMethodCallback<ClearFileMetadataResult> resultHandler) throws org.apache.thrift.TException {
+        iface.clear_file_metadata(args.req,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_active_resource_plan<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_active_resource_plan_args, WMGetActiveResourcePlanResponse> {
-      public get_active_resource_plan() {
-        super("get_active_resource_plan");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class cache_file_metadata<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, cache_file_metadata_args, CacheFileMetadataResult> {
+      public cache_file_metadata() {
+        super("cache_file_metadata");
       }
 
-      public get_active_resource_plan_args getEmptyArgsInstance() {
-        return new get_active_resource_plan_args();
+      public cache_file_metadata_args getEmptyArgsInstance() {
+        return new cache_file_metadata_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<WMGetActiveResourcePlanResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<CacheFileMetadataResult> 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<WMGetActiveResourcePlanResponse>() { 
-          public void onComplete(WMGetActiveResourcePlanResponse o) {
-            get_active_resource_plan_result result = new get_active_resource_plan_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<CacheFileMetadataResult>() { 
+          public void onComplete(CacheFileMetadataResult o) {
+            cache_file_metadata_result result = new cache_file_metadata_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -42019,12 +41909,8 @@ 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_active_resource_plan_result result = new get_active_resource_plan_result();
-            if (e instanceof MetaException) {
-              result.o2 = (MetaException) e;
-              result.setO2IsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+            cache_file_metadata_result result = new cache_file_metadata_result();
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -42051,25 +41937,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, get_active_resource_plan_args args, org.apache.thrift.async.AsyncMethodCallback<WMGetActiveResourcePlanResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.get_active_resource_plan(args.request,resultHandler);
+      public void start(I iface, cache_file_metadata_args args, org.apache.thrift.async.AsyncMethodCallback<CacheFileMetadataResult> resultHandler) throws org.apache.thrift.TException {
+        iface.cache_file_metadata(args.req,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_all_resource_plans<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_all_resource_plans_args, WMGetAllResourcePlanResponse> {
-      public get_all_resource_plans() {
-        super("get_all_resource_plans");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_metastore_db_uuid<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_metastore_db_uuid_args, java.lang.String> {
+      public get_metastore_db_uuid() {
+        super("get_metastore_db_uuid");
       }
 
-      public get_all_resource_plans_args getEmptyArgsInstance() {
-        return new get_all_resource_plans_args();
+      public get_metastore_db_uuid_args getEmptyArgsInstance() {
+        return new get_metastore_db_uuid_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<WMGetAllResourcePlanResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<java.lang.String> 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<WMGetAllResourcePlanResponse>() { 
-          public void onComplete(WMGetAllResourcePlanResponse o) {
-            get_all_resource_plans_result result = new get_all_resource_plans_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<java.lang.String>() { 
+          public void onComplete(java.lang.String o) {
+            get_metastore_db_uuid_result result = new get_metastore_db_uuid_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -42084,7 +41970,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_all_resource_plans_result result = new get_all_resource_plans_result();
+            get_metastore_db_uuid_result result = new get_metastore_db_uuid_result();
             if (e instanceof MetaException) {
               result.o1 = (MetaException) e;
               result.setO1IsSet(true);
@@ -42116,25 +42002,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, get_all_resource_plans_args args, org.apache.thrift.async.AsyncMethodCallback<WMGetAllResourcePlanResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.get_all_resource_plans(args.request,resultHandler);
+      public void start(I iface, get_metastore_db_uuid_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException {
+        iface.get_metastore_db_uuid(resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class alter_resource_plan<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, alter_resource_plan_args, WMAlterResourcePlanResponse> {
-      public alter_resource_plan() {
-        super("alter_resource_plan");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class create_resource_plan<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, create_resource_plan_args, WMCreateResourcePlanResponse> {
+      public create_resource_plan() {
+        super("create_resource_plan");
       }
 
-      public alter_resource_plan_args getEmptyArgsInstance() {
-        return new alter_resource_plan_args();
+      public create_resource_plan_args getEmptyArgsInstance() {
+        return new create_resource_plan_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<WMAlterResourcePlanResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<WMCreateResourcePlanResponse> 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<WMAlterResourcePlanResponse>() { 
-          public void onComplete(WMAlterResourcePlanResponse o) {
-            alter_resource_plan_result result = new alter_resource_plan_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<WMCreateResourcePlanResponse>() { 
+          public void onComplete(WMCreateResourcePlanResponse o) {
+            create_resource_plan_result result = new create_resource_plan_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -42149,13 +42035,13 @@ 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;
-            alter_resource_plan_result result = new alter_resource_plan_result();
-            if (e instanceof NoSuchObjectException) {
-              result.o1 = (NoSuchObjectException) e;
+            create_resource_plan_result result = new create_resource_plan_result();
+            if (e instanceof AlreadyExistsException) {
+              result.o1 = (AlreadyExistsException) e;
               result.setO1IsSet(true);
               msg = result;
-            } else if (e instanceof InvalidOperationException) {
-              result.o2 = (InvalidOperationException) e;
+            } else if (e instanceof InvalidObjectException) {
+              result.o2 = (InvalidObjectException) e;
               result.setO2IsSet(true);
               msg = result;
             } else if (e instanceof MetaException) {
@@ -42189,25 +42075,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, alter_resource_plan_args args, org.apache.thrift.async.AsyncMethodCallback<WMAlterResourcePlanResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.alter_resource_plan(args.request,resultHandler);
+      public void start(I iface, create_resource_plan_args args, org.apache.thrift.async.AsyncMethodCallback<WMCreateResourcePlanResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.create_resource_plan(args.request,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class validate_resource_plan<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, validate_resource_plan_args, WMValidateResourcePlanResponse> {
-      public validate_resource_plan() {
-        super("validate_resource_plan");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_resource_plan<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_resource_plan_args, WMGetResourcePlanResponse> {
+      public get_resource_plan() {
+        super("get_resource_plan");
       }
 
-      public validate_resource_plan_args getEmptyArgsInstance() {
-        return new validate_resource_plan_args();
+      public get_resource_plan_args getEmptyArgsInstance() {
+        return new get_resource_plan_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<WMValidateResourcePlanResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<WMGetResourcePlanResponse> 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<WMValidateResourcePlanResponse>() { 
-          public void onComplete(WMValidateResourcePlanResponse o) {
-            validate_resource_plan_result result = new validate_resource_plan_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<WMGetResourcePlanResponse>() { 
+          public void onComplete(WMGetResourcePlanResponse o) {
+            get_resource_plan_result result = new get_resource_plan_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -42222,7 +42108,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;
-            validate_resource_plan_result result = new validate_resource_plan_result();
+            get_resource_plan_result result = new get_resource_plan_result();
             if (e instanceof NoSuchObjectException) {
               result.o1 = (NoSuchObjectException) e;
               result.setO1IsSet(true);
@@ -42258,25 +42144,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, validate_resource_plan_args args, org.apache.thrift.async.AsyncMethodCallback<WMValidateResourcePlanResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.validate_resource_plan(args.request,resultHandler);
+      public void start(I iface, get_resource_plan_args args, org.apache.thrift.async.AsyncMethodCallback<WMGetResourcePlanResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.get_resource_plan(args.request,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class drop_resource_plan<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drop_resource_plan_args, WMDropResourcePlanResponse> {
-      public drop_resource_plan() {
-        super("drop_resource_plan");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_active_resource_plan<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_active_resource_plan_args, WMGetActiveResourcePlanResponse> {
+      public get_active_resource_plan() {
+        super("get_active_resource_plan");
       }
 
-      public drop_resource_plan_args getEmptyArgsInstance() {
-        return new drop_resource_plan_args();
+      public get_active_resource_plan_args getEmptyArgsInstance() {
+        return new get_active_resource_plan_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<WMDropResourcePlanResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<WMGetActiveResourcePlanResponse> 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<WMDropResourcePlanResponse>() { 
-          public void onComplete(WMDropResourcePlanResponse o) {
-            drop_resource_plan_result result = new drop_resource_plan_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<WMGetActiveResourcePlanResponse>() { 
+          public void onComplete(WMGetActiveResourcePlanResponse o) {
+            get_active_resource_plan_result result = new get_active_resource_plan_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -42291,19 +42177,11 @@ 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;
-            drop_resource_plan_result result = new drop_resource_plan_result();
-            if (e instanceof NoSuchObjectException) {
-              result.o1 = (NoSuchObjectException) e;
-              result.setO1IsSet(true);
-              msg = result;
-            } else if (e instanceof InvalidOperationException) {
-              result.o2 = (InvalidOperationException) e;
+            get_active_resource_plan_result result = new get_active_resource_plan_result();
+            if (e instanceof MetaException) {
+              result.o2 = (MetaException) e;
               result.setO2IsSet(true);
               msg = result;
-            } else if (e instanceof MetaException) {
-              result.o3 = (MetaException) e;
-              result.setO3IsSet(true);
-              msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -42331,25 +42209,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, drop_resource_plan_args args, org.apache.thrift.async.AsyncMethodCallback<WMDropResourcePlanResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.drop_resource_plan(args.request,resultHandler);
+      public void start(I iface, get_active_resource_plan_args args, org.apache.thrift.async.AsyncMethodCallback<WMGetActiveResourcePlanResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.get_active_resource_plan(args.request,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class create_wm_trigger<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, create_wm_trigger_args, WMCreateTriggerResponse> {
-      public create_wm_trigger() {
-        super("create_wm_trigger");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_all_resource_plans<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_all_resource_plans_args, WMGetAllResourcePlanResponse> {
+      public get_all_resource_plans() {
+        super("get_all_resource_plans");
       }
 
-      public create_wm_trigger_args getEmptyArgsInstance() {
-        return new create_wm_trigger_args();
+      public get_all_resource_plans_args getEmptyArgsInstance() {
+        return new get_all_resource_plans_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<WMCreateTriggerResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<WMGetAllResourcePlanResponse> 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<WMCreateTriggerResponse>() { 
-          public void onComplete(WMCreateTriggerResponse o) {
-            create_wm_trigger_result result = new create_wm_trigger_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<WMGetAllResourcePlanResponse>() { 
+          public void onComplete(WMGetAllResourcePlanResponse o) {
+            get_all_resource_plans_result result = new get_all_resource_plans_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -42364,23 +42242,11 @@ 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;
-            create_wm_trigger_result result = new create_wm_trigger_result();
-            if (e instanceof AlreadyExistsException) {
-              result.o1 = (AlreadyExistsException) e;
+            get_all_resource_plans_result result = new get_all_resource_plans_result();
+            if (e instanceof MetaException) {
+              result.o1 = (MetaException) e;
               result.setO1IsSet(true);
               msg = result;
-            } else if (e instanceof NoSuchObjectException) {
-              result.o2 = (NoSuchObjectException) e;
-              result.setO2IsSet(true);
-              msg = result;
-            } else if (e instanceof InvalidObjectException) {
-              result.o3 = (InvalidObjectException) e;
-              result.setO3IsSet(true);
-              msg = result;
-            } else if (e instanceof MetaException) {
-              result.o4 = (MetaException) e;
-              result.setO4IsSet(true);
-              msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -42408,25 +42274,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, create_wm_trigger_args args, org.apache.thrift.async.AsyncMethodCallback<WMCreateTriggerResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.create_wm_trigger(args.request,resultHandler);
+      public void start(I iface, get_all_resource_plans_args args, org.apache.thrift.async.AsyncMethodCallback<WMGetAllResourcePlanResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.get_all_resource_plans(args.request,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class alter_wm_trigger<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, alter_wm_trigger_args, WMAlterTriggerResponse> {
-      public alter_wm_trigger() {
-        super("alter_wm_trigger");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class alter_resource_plan<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, alter_resource_plan_args, WMAlterResourcePlanResponse> {
+      public alter_resource_plan() {
+        super("alter_resource_plan");
       }
 
-      public alter_wm_trigger_args getEmptyArgsInstance() {
-        return new alter_wm_trigger_args();
+      public alter_resource_plan_args getEmptyArgsInstance() {
+        return new alter_resource_plan_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<WMAlterTriggerResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<WMAlterResourcePlanResponse> 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<WMAlterTriggerResponse>() { 
-          public void onComplete(WMAlterTriggerResponse o) {
-            alter_wm_trigger_result result = new alter_wm_trigger_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<WMAlterResourcePlanResponse>() { 
+          public void onComplete(WMAlterResourcePlanResponse o) {
+            alter_resource_plan_result result = new alter_resource_plan_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -42441,13 +42307,13 @@ 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;
-            alter_wm_trigger_result result = new alter_wm_trigger_result();
+            alter_resource_plan_result result = new alter_resource_plan_result();
             if (e instanceof NoSuchObjectException) {
               result.o1 = (NoSuchObjectException) e;
               result.setO1IsSet(true);
               msg = result;
-            } else if (e instanceof InvalidObjectException) {
-              result.o2 = (InvalidObjectException) e;
+            } else if (e instanceof InvalidOperationException) {
+              result.o2 = (InvalidOperationException) e;
               result.setO2IsSet(true);
               msg = result;
             } else if (e instanceof MetaException) {
@@ -42481,25 +42347,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, alter_wm_trigger_args args, org.apache.thrift.async.AsyncMethodCallback<WMAlterTriggerResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.alter_wm_trigger(args.request,resultHandler);
+      public void start(I iface, alter_resource_plan_args args, org.apache.thrift.async.AsyncMethodCallback<WMAlterResourcePlanResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.alter_resource_plan(args.request,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class drop_wm_trigger<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drop_wm_trigger_args, WMDropTriggerResponse> {
-      public drop_wm_trigger() {
-        super("drop_wm_trigger");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class validate_resource_plan<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, validate_resource_plan_args, WMValidateResourcePlanResponse> {
+      public validate_resource_plan() {
+        super("validate_resource_plan");
       }
 
-      public drop_wm_trigger_args getEmptyArgsInstance() {
-        return new drop_wm_trigger_args();
+      public validate_resource_plan_args getEmptyArgsInstance() {
+        return new validate_resource_plan_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<WMDropTriggerResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<WMValidateResourcePlanResponse> 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<WMDropTriggerResponse>() { 
-          public void onComplete(WMDropTriggerResponse o) {
-            drop_wm_trigger_result result = new drop_wm_trigger_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<WMValidateResourcePlanResponse>() { 
+          public void onComplete(WMValidateResourcePlanResponse o) {
+            validate_resource_plan_result result = new validate_resource_plan_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -42514,18 +42380,14 @@ 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;
-            drop_wm_trigger_result result = new drop_wm_trigger_result();
+            validate_resource_plan_result result = new validate_resource_plan_result();
             if (e instanceof NoSuchObjectException) {
               result.o1 = (NoSuchObjectException) e;
               result.setO1IsSet(true);
               msg = result;
-            } else if (e instanceof InvalidOperationException) {
-              result.o2 = (InvalidOperationException) e;
-              result.setO2IsSet(true);
-              msg = result;
             } else if (e instanceof MetaException) {
-              result.o3 = (MetaException) e;
-              result.setO3IsSet(true);
+              result.o2 = (MetaException) e;
+              result.setO2IsSet(true);
               msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
@@ -42554,25 +42416,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, drop_wm_trigger_args args, org.apache.thrift.async.AsyncMethodCallback<WMDropTriggerResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.drop_wm_trigger(args.request,resultHandler);
+      public void start(I iface, validate_resource_plan_args args, org.apache.thrift.async.AsyncMethodCallback<WMValidateResourcePlanResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.validate_resource_plan(args.request,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_triggers_for_resourceplan<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_triggers_for_resourceplan_args, WMGetTriggersForResourePlanResponse> {
-      public get_triggers_for_resourceplan() {
-        super("get_triggers_for_resourceplan");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class drop_resource_plan<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drop_resource_plan_args, WMDropResourcePlanResponse> {
+      public drop_resource_plan() {
+        super("drop_resource_plan");
       }
 
-      public get_triggers_for_resourceplan_args getEmptyArgsInstance() {
-        return new get_triggers_for_resourceplan_args();
+      public drop_resource_plan_args getEmptyArgsInstance() {
+        return new drop_resource_plan_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<WMGetTriggersForResourePlanResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<WMDropResourcePlanResponse> 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<WMGetTriggersForResourePlanResponse>() { 
-          public void onComplete(WMGetTriggersForResourePlanResponse o) {
-            get_triggers_for_resourceplan_result result = new get_triggers_for_resourceplan_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<WMDropResourcePlanResponse>() { 
+          public void onComplete(WMDropResourcePlanResponse o) {
+            drop_resource_plan_result result = new drop_resource_plan_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -42587,15 +42449,19 @@ 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_triggers_for_resourceplan_result result = new get_triggers_for_resourceplan_result();
+            drop_resource_plan_result result = new drop_resource_plan_result();
             if (e instanceof NoSuchObjectException) {
               result.o1 = (NoSuchObjectException) e;
               result.setO1IsSet(true);
               msg = result;
-            } else if (e instanceof MetaException) {
-              result.o2 = (MetaException) e;
+            } else if (e instanceof InvalidOperationException) {
+              result.o2 = (InvalidOperationException) e;
               result.setO2IsSet(true);
               msg = result;
+            } else if (e instanceof MetaException) {
+              result.o3 = (MetaException) e;
+              result.setO3IsSet(true);
+              msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -42623,25 +42489,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, get_triggers_for_resourceplan_args args, org.apache.thrift.async.AsyncMethodCallback<WMGetTriggersForResourePlanResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.get_triggers_for_resourceplan(args.request,resultHandler);
+      public void start(I iface, drop_resource_plan_args args, org.apache.thrift.async.AsyncMethodCallback<WMDropResourcePlanResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.drop_resource_plan(args.request,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class create_wm_pool<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, create_wm_pool_args, WMCreatePoolResponse> {
-      public create_wm_pool() {
-        super("create_wm_pool");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class create_wm_trigger<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, create_wm_trigger_args, WMCreateTriggerResponse> {
+      public create_wm_trigger() {
+        super("create_wm_trigger");
       }
 
-      public create_wm_pool_args getEmptyArgsInstance() {
-        return new create_wm_pool_args();
+      public create_wm_trigger_args getEmptyArgsInstance() {
+        return new create_wm_trigger_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<WMCreatePoolResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<WMCreateTriggerResponse> 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<WMCreatePoolResponse>() { 
-          public void onComplete(WMCreatePoolResponse o) {
-            create_wm_pool_result result = new create_wm_pool_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<WMCreateTriggerResponse>() { 
+          public void onComplete(WMCreateTriggerResponse o) {
+            create_wm_trigger_result result = new create_wm_trigger_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -42656,7 +42522,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;
-            create_wm_pool_result result = new create_wm_pool_result();
+            create_wm_trigger_result result = new create_wm_trigger_result();
             if (e instanceof AlreadyExistsException) {
               result.o1 = (AlreadyExistsException) e;
               result.setO1IsSet(true);
@@ -42700,25 +42566,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, create_wm_pool_args args, org.apache.thrift.async.AsyncMethodCallback<WMCreatePoolResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.create_wm_pool(args.request,resultHandler);
+      public void start(I iface, create_wm_trigger_args args, org.apache.thrift.async.AsyncMethodCallback<WMCreateTriggerResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.create_wm_trigger(args.request,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class alter_wm_pool<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, alter_wm_pool_args, WMAlterPoolResponse> {
-      public alter_wm_pool() {
-        super("alter_wm_pool");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class alter_wm_trigger<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, alter_wm_trigger_args, WMAlterTriggerResponse> {
+      public alter_wm_trigger() {
+        super("alter_wm_trigger");
       }
 
-      public alter_wm_pool_args getEmptyArgsInstance() {
-        return new alter_wm_pool_args();
+      public alter_wm_trigger_args getEmptyArgsInstance() {
+        return new alter_wm_trigger_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<WMAlterPoolResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<WMAlterTriggerResponse> 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<WMAlterPoolResponse>() { 
-          public void onComplete(WMAlterPoolResponse o) {
-            alter_wm_pool_result result = new alter_wm_pool_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<WMAlterTriggerResponse>() { 
+          public void onComplete(WMAlterTriggerResponse o) {
+            alter_wm_trigger_result result = new alter_wm_trigger_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -42733,22 +42599,18 @@ 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;
-            alter_wm_pool_result result = new alter_wm_pool_result();
-            if (e instanceof AlreadyExistsException) {
-              result.o1 = (AlreadyExistsException) e;
+            alter_wm_trigger_result result = new alter_wm_trigger_result();
+            if (e instanceof NoSuchObjectException) {
+              result.o1 = (NoSuchObjectException) e;
               result.setO1IsSet(true);
               msg = result;
-            } else if (e instanceof NoSuchObjectException) {
-              result.o2 = (NoSuchObjectException) e;
-              result.setO2IsSet(true);
-              msg = result;
             } else if (e instanceof InvalidObjectException) {
-              result.o3 = (InvalidObjectException) e;
-              result.setO3IsSet(true);
+              result.o2 = (InvalidObjectException) e;
+              result.setO2IsSet(true);
               msg = result;
             } else if (e instanceof MetaException) {
-              result.o4 = (MetaException) e;
-              result.setO4IsSet(true);
+              result.o3 = (MetaException) e;
+              result.setO3IsSet(true);
               msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
@@ -42777,25 +42639,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, alter_wm_pool_args args, org.apache.thrift.async.AsyncMethodCallback<WMAlterPoolResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.alter_wm_pool(args.request,resultHandler);
+      public void start(I iface, alter_wm_trigger_args args, org.apache.thrift.async.AsyncMethodCallback<WMAlterTriggerResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.alter_wm_trigger(args.request,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class drop_wm_pool<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drop_wm_pool_args, WMDropPoolResponse> {
-      public drop_wm_pool() {
-        super("drop_wm_pool");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class drop_wm_trigger<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drop_wm_trigger_args, WMDropTriggerResponse> {
+      public drop_wm_trigger() {
+        super("drop_wm_trigger");
       }
 
-      public drop_wm_pool_args getEmptyArgsInstance() {
-        return new drop_wm_pool_args();
+      public drop_wm_trigger_args getEmptyArgsInstance() {
+        return new drop_wm_trigger_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<WMDropPoolResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<WMDropTriggerResponse> 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<WMDropPoolResponse>() { 
-          public void onComplete(WMDropPoolResponse o) {
-            drop_wm_pool_result result = new drop_wm_pool_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<WMDropTriggerResponse>() { 
+          public void onComplete(WMDropTriggerResponse o) {
+            drop_wm_trigger_result result = new drop_wm_trigger_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -42810,7 +42672,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;
-            drop_wm_pool_result result = new drop_wm_pool_result();
+            drop_wm_trigger_result result = new drop_wm_trigger_result();
             if (e instanceof NoSuchObjectException) {
               result.o1 = (NoSuchObjectException) e;
               result.setO1IsSet(true);
@@ -42850,25 +42712,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, drop_wm_pool_args args, org.apache.thrift.async.AsyncMethodCallback<WMDropPoolResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.drop_wm_pool(args.request,resultHandler);
+      public void start(I iface, drop_wm_trigger_args args, org.apache.thrift.async.AsyncMethodCallback<WMDropTriggerResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.drop_wm_trigger(args.request,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class create_or_update_wm_mapping<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, create_or_update_wm_mapping_args, WMCreateOrUpdateMappingResponse> {
-      public create_or_update_wm_mapping() {
-        super("create_or_update_wm_mapping");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_triggers_for_resourceplan<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_triggers_for_resourceplan_args, WMGetTriggersForResourePlanResponse> {
+      public get_triggers_for_resourceplan() {
+        super("get_triggers_for_resourceplan");
       }
 
-      public create_or_update_wm_mapping_args getEmptyArgsInstance() {
-        return new create_or_update_wm_mapping_args();
+      public get_triggers_for_resourceplan_args getEmptyArgsInstance() {
+        return new get_triggers_for_resourceplan_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<WMCreateOrUpdateMappingResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<WMGetTriggersForResourePlanResponse> 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<WMCreateOrUpdateMappingResponse>() { 
-          public void onComplete(WMCreateOrUpdateMappingResponse o) {
-            create_or_update_wm_mapping_result result = new create_or_update_wm_mapping_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<WMGetTriggersForResourePlanResponse>() { 
+          public void onComplete(WMGetTriggersForResourePlanResponse o) {
+            get_triggers_for_resourceplan_result result = new get_triggers_for_resourceplan_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -42883,22 +42745,14 @@ 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;
-            create_or_update_wm_mapping_result result = new create_or_update_wm_mapping_result();
-            if (e instanceof AlreadyExistsException) {
-              result.o1 = (AlreadyExistsException) e;
+            get_triggers_for_resourceplan_result result = new get_triggers_for_resourceplan_result();
+            if (e instanceof NoSuchObjectException) {
+              result.o1 = (NoSuchObjectException) e;
               result.setO1IsSet(true);
               msg = result;
-            } else if (e instanceof NoSuchObjectException) {
-              result.o2 = (NoSuchObjectException) e;
-              result.setO2IsSet(true);
-              msg = result;
-            } else if (e instanceof InvalidObjectException) {
-              result.o3 = (InvalidObjectException) e;
-              result.setO3IsSet(true);
-              msg = result;
             } else if (e instanceof MetaException) {
-              result.o4 = (MetaException) e;
-              result.setO4IsSet(true);
+              result.o2 = (MetaException) e;
+              result.setO2IsSet(true);
               msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
@@ -42927,25 +42781,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, create_or_update_wm_mapping_args args, org.apache.thrift.async.AsyncMethodCallback<WMCreateOrUpdateMappingResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.create_or_update_wm_mapping(args.request,resultHandler);
+      public void start(I iface, get_triggers_for_resourceplan_args args, org.apache.thrift.async.AsyncMethodCallback<WMGetTriggersForResourePlanResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.get_triggers_for_resourceplan(args.request,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class drop_wm_mapping<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drop_wm_mapping_args, WMDropMappingResponse> {
-      public drop_wm_mapping() {
-        super("drop_wm_mapping");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class create_wm_pool<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, create_wm_pool_args, WMCreatePoolResponse> {
+      public create_wm_pool() {
+        super("create_wm_pool");
       }
 
-      public drop_wm_mapping_args getEmptyArgsInstance() {
-        return new drop_wm_mapping_args();
+      public create_wm_pool_args getEmptyArgsInstance() {
+        return new create_wm_pool_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<WMDropMappingResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<WMCreatePoolResponse> 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<WMDropMappingResponse>() { 
-          public void onComplete(WMDropMappingResponse o) {
-            drop_wm_mapping_result result = new drop_wm_mapping_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<WMCreatePoolResponse>() { 
+          public void onComplete(WMCreatePoolResponse o) {
+            create_wm_pool_result result = new create_wm_pool_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -42960,19 +42814,23 @@ 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;
-            drop_wm_mapping_result result = new drop_wm_mapping_result();
-            if (e instanceof NoSuchObjectException) {
-              result.o1 = (NoSuchObjectException) e;
+            create_wm_pool_result result = new create_wm_pool_result();
+            if (e instanceof AlreadyExistsException) {
+              result.o1 = (AlreadyExistsException) e;
               result.setO1IsSet(true);
               msg = result;
-            } else if (e instanceof InvalidOperationException) {
-              result.o2 = (InvalidOperationException) e;
+            } else if (e instanceof NoSuchObjectException) {
+              result.o2 = (NoSuchObjectException) e;
               result.setO2IsSet(true);
               msg = result;
-            } else if (e instanceof MetaException) {
-              result.o3 = (MetaException) e;
+            } else if (e instanceof InvalidObjectException) {
+              result.o3 = (InvalidObjectException) e;
               result.setO3IsSet(true);
               msg = result;
+            } else if (e instanceof MetaException) {
+              result.o4 = (MetaException) e;
+              result.setO4IsSet(true);
+              msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -43000,25 +42858,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, drop_wm_mapping_args args, org.apache.thrift.async.AsyncMethodCallback<WMDropMappingResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.drop_wm_mapping(args.request,resultHandler);
+      public void start(I iface, create_wm_pool_args args, org.apache.thrift.async.AsyncMethodCallback<WMCreatePoolResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.create_wm_pool(args.request,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class create_or_drop_wm_trigger_to_pool_mapping<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, create_or_drop_wm_trigger_to_pool_mapping_args, WMCreateOrDropTriggerToPoolMappingResponse> {
-      public create_or_drop_wm_trigger_to_pool_mapping() {
-        super("create_or_drop_wm_trigger_to_pool_mapping");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class alter_wm_pool<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, alter_wm_pool_args, WMAlterPoolResponse> {
+      public alter_wm_pool() {
+        super("alter_wm_pool");
       }
 
-      public create_or_drop_wm_trigger_to_pool_mapping_args getEmptyArgsInstance() {
-        return new create_or_drop_wm_trigger_to_pool_mapping_args();
+      public alter_wm_pool_args getEmptyArgsInstance() {
+        return new alter_wm_pool_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<WMCreateOrDropTriggerToPoolMappingResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<WMAlterPoolResponse> 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<WMCreateOrDropTriggerToPoolMappingResponse>() { 
-          public void onComplete(WMCreateOrDropTriggerToPoolMappingResponse o) {
-            create_or_drop_wm_trigger_to_pool_mapping_result result = new create_or_drop_wm_trigger_to_pool_mapping_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<WMAlterPoolResponse>() { 
+          public void onComplete(WMAlterPoolResponse o) {
+            alter_wm_pool_result result = new alter_wm_pool_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -43033,7 +42891,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;
-            create_or_drop_wm_trigger_to_pool_mapping_result result = new create_or_drop_wm_trigger_to_pool_mapping_result();
+            alter_wm_pool_result result = new alter_wm_pool_result();
             if (e instanceof AlreadyExistsException) {
               result.o1 = (AlreadyExistsException) e;
               result.setO1IsSet(true);
@@ -43077,25 +42935,26 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, create_or_drop_wm_trigger_to_pool_mapping_args args, org.apache.thrift.async.AsyncMethodCallback<WMCreateOrDropTriggerToPoolMappingResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.create_or_drop_wm_trigger_to_pool_mapping(args.request,resultHandler);
+      public void start(I iface, alter_wm_pool_args args, org.apache.thrift.async.AsyncMethodCallback<WMAlterPoolResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.alter_wm_pool(args.request,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class create_ischema<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, create_ischema_args, Void> {
-      public create_ischema() {
-        super("create_ischema");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class drop_wm_pool<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drop_wm_pool_args, WMDropPoolResponse> {
+      public drop_wm_pool() {
+        super("drop_wm_pool");
       }
 
-      public create_ischema_args getEmptyArgsInstance() {
-        return new create_ischema_args();
+      public drop_wm_pool_args getEmptyArgsInstance() {
+        return new drop_wm_pool_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<WMDropPoolResponse> 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<Void>() { 
-          public void onComplete(Void o) {
-            create_ischema_result result = new create_ischema_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<WMDropPoolResponse>() { 
+          public void onComplete(WMDropPoolResponse o) {
+            drop_wm_pool_result result = new drop_wm_pool_result();
+            result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -43109,13 +42968,13 @@ 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;
-            create_ischema_result result = new create_ischema_result();
-            if (e instanceof AlreadyExistsException) {
-              result.o1 = (AlreadyExistsException) e;
+            drop_wm_pool_result result = new drop_wm_pool_result();
+            if (e instanceof NoSuchObjectException) {
+              result.o1 = (NoSuchObjectException) e;
               result.setO1IsSet(true);
               msg = result;
-            } else if (e instanceof NoSuchObjectException) {
-              result.o2 = (NoSuchObjectException) e;
+            } else if (e instanceof InvalidOperationException) {
+              result.o2 = (InvalidOperationException) e;
               result.setO2IsSet(true);
               msg = result;
             } else if (e instanceof MetaException) {
@@ -43149,25 +43008,26 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, create_ischema_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.create_ischema(args.schema,resultHandler);
+      public void start(I iface, drop_wm_pool_args args, org.apache.thrift.async.AsyncMethodCallback<WMDropPoolResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.drop_wm_pool(args.request,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class alter_ischema<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, alter_ischema_args, Void> {
-      public alter_ischema() {
-        super("alter_ischema");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class create_or_update_wm_mapping<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, create_or_update_wm_mapping_args, WMCreateOrUpdateMappingResponse> {
+      public create_or_update_wm_mapping() {
+        super("create_or_update_wm_mapping");
       }
 
-      public alter_ischema_args getEmptyArgsInstance() {
-        return new alter_ischema_args();
+      public create_or_update_wm_mapping_args getEmptyArgsInstance() {
+        return new create_or_update_wm_mapping_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<WMCreateOrUpdateMappingResponse> 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<Void>() { 
-          public void onComplete(Void o) {
-            alter_ischema_result result = new alter_ischema_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<WMCreateOrUpdateMappingResponse>() { 
+          public void onComplete(WMCreateOrUpdateMappingResponse o) {
+            create_or_update_wm_mapping_result result = new create_or_update_wm_mapping_result();
+            result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -43181,15 +43041,23 @@ 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;
-            alter_ischema_result result = new alter_ischema_result();
-            if (e instanceof NoSuchObjectException) {
-              result.o1 = (NoSuchObjectException) e;
+            create_or_update_wm_mapping_result result = new create_or_update_wm_mapping_result();
+            if (e instanceof AlreadyExistsException) {
+              result.o1 = (AlreadyExistsException) e;
               result.setO1IsSet(true);
               msg = result;
-            } else if (e instanceof MetaException) {
-              result.o2 = (MetaException) e;
+            } else if (e instanceof NoSuchObjectException) {
+              result.o2 = (NoSuchObjectException) e;
               result.setO2IsSet(true);
               msg = result;
+            } else if (e instanceof InvalidObjectException) {
+              result.o3 = (InvalidObjectException) e;
+              result.setO3IsSet(true);
+              msg = result;
+            } else if (e instanceof MetaException) {
+              result.o4 = (MetaException) e;
+              result.setO4IsSet(true);
+              msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -43217,25 +43085,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, alter_ischema_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.alter_ischema(args.rqst,resultHandler);
+      public void start(I iface, create_or_update_wm_mapping_args args, org.apache.thrift.async.AsyncMethodCallback<WMCreateOrUpdateMappingResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.create_or_update_wm_mapping(args.request,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_ischema<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_ischema_args, ISchema> {
-      public get_ischema() {
-        super("get_ischema");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class drop_wm_mapping<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drop_wm_mapping_args, WMDropMappingResponse> {
+      public drop_wm_mapping() {
+        super("drop_wm_mapping");
       }
 
-      public get_ischema_args getEmptyArgsInstance() {
-        return new get_ischema_args();
+      public drop_wm_mapping_args getEmptyArgsInstance() {
+        return new drop_wm_mapping_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<ISchema> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<WMDropMappingResponse> 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<ISchema>() { 
-          public void onComplete(ISchema o) {
-            get_ischema_result result = new get_ischema_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<WMDropMappingResponse>() { 
+          public void onComplete(WMDropMappingResponse o) {
+            drop_wm_mapping_result result = new drop_wm_mapping_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -43250,15 +43118,19 @@ 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_ischema_result result = new get_ischema_result();
+            drop_wm_mapping_result result = new drop_wm_mapping_result();
             if (e instanceof NoSuchObjectException) {
               result.o1 = (NoSuchObjectException) e;
               result.setO1IsSet(true);
               msg = result;
-            } else if (e instanceof MetaException) {
-              result.o2 = (MetaException) e;
+            } else if (e instanceof InvalidOperationException) {
+              result.o2 = (InvalidOperationException) e;
               result.setO2IsSet(true);
               msg = result;
+            } else if (e instanceof MetaException) {
+              result.o3 = (MetaException) e;
+              result.setO3IsSet(true);
+              msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -43286,25 +43158,26 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, get_ischema_args args, org.apache.thrift.async.AsyncMethodCallback<ISchema> resultHandler) throws org.apache.thrift.TException {
-        iface.get_ischema(args.name,resultHandler);
+      public void start(I iface, drop_wm_mapping_args args, org.apache.thrift.async.AsyncMethodCallback<WMDropMappingResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.drop_wm_mapping(args.request,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class drop_ischema<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drop_ischema_args, Void> {
-      public drop_ischema() {
-        super("drop_ischema");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class create_or_drop_wm_trigger_to_pool_mapping<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, create_or_drop_wm_trigger_to_pool_mapping_args, WMCreateOrDropTriggerToPoolMappingResponse> {
+      public create_or_drop_wm_trigger_to_pool_mapping() {
+        super("create_or_drop_wm_trigger_to_pool_mapping");
       }
 
-      public drop_ischema_args getEmptyArgsInstance() {
-        return new drop_ischema_args();
+      public create_or_drop_wm_trigger_to_pool_mapping_args getEmptyArgsInstance() {
+        return new create_or_drop_wm_trigger_to_pool_mapping_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<WMCreateOrDropTriggerToPoolMappingResponse> 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<Void>() { 
-          public void onComplete(Void o) {
-            drop_ischema_result result = new drop_ischema_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<WMCreateOrDropTriggerToPoolMappingResponse>() { 
+          public void onComplete(WMCreateOrDropTriggerToPoolMappingResponse o) {
+            create_or_drop_wm_trigger_to_pool_mapping_result result = new create_or_drop_wm_trigger_to_pool_mapping_result();
+            result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -43318,19 +43191,23 @@ 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;
-            drop_ischema_result result = new drop_ischema_result();
-            if (e instanceof NoSuchObjectException) {
-              result.o1 = (NoSuchObjectException) e;
+            create_or_drop_wm_trigger_to_pool_mapping_result result = new create_or_drop_wm_trigger_to_pool_mapping_result();
+            if (e instanceof AlreadyExistsException) {
+              result.o1 = (AlreadyExistsException) e;
               result.setO1IsSet(true);
               msg = result;
-            } else if (e instanceof InvalidOperationException) {
-              result.o2 = (InvalidOperationException) e;
+            } else if (e instanceof NoSuchObjectException) {
+              result.o2 = (NoSuchObjectException) e;
               result.setO2IsSet(true);
               msg = result;
-            } else if (e instanceof MetaException) {
-              result.o3 = (MetaException) e;
+            } else if (e instanceof InvalidObjectException) {
+              result.o3 = (InvalidObjectException) e;
               result.setO3IsSet(true);
               msg = result;
+            } else if (e instanceof MetaException) {
+              result.o4 = (MetaException) e;
+              result.setO4IsSet(true);
+              msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -43358,25 +43235,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, drop_ischema_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.drop_ischema(args.name,resultHandler);
+      public void start(I iface, create_or_drop_wm_trigger_to_pool_mapping_args args, org.apache.thrift.async.AsyncMethodCallback<WMCreateOrDropTriggerToPoolMappingResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.create_or_drop_wm_trigger_to_pool_mapping(args.request,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_schema_version<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_schema_version_args, Void> {
-      public add_schema_version() {
-        super("add_schema_version");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class create_ischema<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, create_ischema_args, Void> {
+      public create_ischema() {
+        super("create_ischema");
       }
 
-      public add_schema_version_args getEmptyArgsInstance() {
-        return new add_schema_version_args();
+      public create_ischema_args getEmptyArgsInstance() {
+        return new create_ischema_args();
       }
 
       public org.apache.thrift.async.AsyncMethodCallback<Void> 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<Void>() { 
           public void onComplete(Void o) {
-            add_schema_version_result result = new add_schema_version_result();
+            create_ischema_result result = new create_ischema_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -43390,7 +43267,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;
-            add_schema_version_result result = new add_schema_version_result();
+            create_ischema_result result = new create_ischema_result();
             if (e instanceof AlreadyExistsException) {
               result.o1 = (AlreadyExistsException) e;
               result.setO1IsSet(true);
@@ -43430,26 +43307,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, add_schema_version_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.add_schema_version(args.schemaVersion,resultHandler);
+      public void start(I iface, create_ischema_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.create_ischema(args.schema,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_schema_version<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_schema_version_args, SchemaVersion> {
-      public get_schema_version() {
-        super("get_schema_version");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class alter_ischema<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, alter_ischema_args, Void> {
+      public alter_ischema() {
+        super("alter_ischema");
       }
 
-      public get_schema_version_args getEmptyArgsInstance() {
-        return new get_schema_version_args();
+      public alter_ischema_args getEmptyArgsInstance() {
+        return new alter_ischema_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<SchemaVersion> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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<SchemaVersion>() { 
-          public void onComplete(SchemaVersion o) {
-            get_schema_version_result result = new get_schema_version_result();
-            result.success = o;
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            alter_ischema_result result = new alter_ischema_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -43463,7 +43339,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_schema_version_result result = new get_schema_version_result();
+            alter_ischema_result result = new alter_ischema_result();
             if (e instanceof NoSuchObjectException) {
               result.o1 = (NoSuchObjectException) e;
               result.setO1IsSet(true);
@@ -43499,25 +43375,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, get_schema_version_args args, org.apache.thrift.async.AsyncMethodCallback<SchemaVersion> resultHandler) throws org.apache.thrift.TException {
-        iface.get_schema_version(args.schemaVersion,resultHandler);
+      public void start(I iface, alter_ischema_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.alter_ischema(args.rqst,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_schema_latest_version<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_schema_latest_version_args, SchemaVersion> {
-      public get_schema_latest_version() {
-        super("get_schema_latest_version");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_ischema<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_ischema_args, ISchema> {
+      public get_ischema() {
+        super("get_ischema");
       }
 
-      public get_schema_latest_version_args getEmptyArgsInstance() {
-        return new get_schema_latest_version_args();
+      public get_ischema_args getEmptyArgsInstance() {
+        return new get_ischema_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<SchemaVersion> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<ISchema> 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<SchemaVersion>() { 
-          public void onComplete(SchemaVersion o) {
-            get_schema_latest_version_result result = new get_schema_latest_version_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<ISchema>() { 
+          public void onComplete(ISchema o) {
+            get_ischema_result result = new get_ischema_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -43532,7 +43408,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_schema_latest_version_result result = new get_schema_latest_version_result();
+            get_ischema_result result = new get_ischema_result();
             if (e instanceof NoSuchObjectException) {
               result.o1 = (NoSuchObjectException) e;
               result.setO1IsSet(true);
@@ -43568,26 +43444,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, get_schema_latest_version_args args, org.apache.thrift.async.AsyncMethodCallback<SchemaVersion> resultHandler) throws org.apache.thrift.TException {
-        iface.get_schema_latest_version(args.schemaName,resultHandler);
+      public void start(I iface, get_ischema_args args, org.apache.thrift.async.AsyncMethodCallback<ISchema> resultHandler) throws org.apache.thrift.TException {
+        iface.get_ischema(args.name,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_schema_all_versions<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_schema_all_versions_args, java.util.List<SchemaVersion>> {
-      public get_schema_all_versions() {
-        super("get_schema_all_versions");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class drop_ischema<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drop_ischema_args, Void> {
+      public drop_ischema() {
+        super("drop_ischema");
       }
 
-      public get_schema_all_versions_args getEmptyArgsInstance() {
-        return new get_schema_all_versions_args();
+      public drop_ischema_args getEmptyArgsInstance() {
+        return new drop_ischema_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<SchemaVersion>> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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.util.List<SchemaVersion>>() { 
-          public void onComplete(java.util.List<SchemaVersion> o) {
-            get_schema_all_versions_result result = new get_schema_all_versions_result();
-            result.success = o;
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            drop_ischema_result result = new drop_ischema_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -43601,15 +43476,19 @@ 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_schema_all_versions_result result = new get_schema_all_versions_result();
+            drop_ischema_result result = new drop_ischema_result();
             if (e instanceof NoSuchObjectException) {
               result.o1 = (NoSuchObjectException) e;
               result.setO1IsSet(true);
               msg = result;
-            } else if (e instanceof MetaException) {
-              result.o2 = (MetaException) e;
+            } else if (e instanceof InvalidOperationException) {
+              result.o2 = (InvalidOperationException) e;
               result.setO2IsSet(true);
               msg = result;
+            } else if (e instanceof MetaException) {
+              result.o3 = (MetaException) e;
+              result.setO3IsSet(true);
+              msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -43637,25 +43516,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, get_schema_all_versions_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<SchemaVersion>> resultHandler) throws org.apache.thrift.TException {
-        iface.get_schema_all_versions(args.schemaName,resultHandler);
+      public void start(I iface, drop_ischema_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.drop_ischema(args.name,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class drop_schema_version<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drop_schema_version_args, Void> {
-      public drop_schema_version() {
-        super("drop_schema_version");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_schema_version<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_schema_version_args, Void> {
+      public add_schema_version() {
+        super("add_schema_version");
       }
 
-      public drop_schema_version_args getEmptyArgsInstance() {
-        return new drop_schema_version_args();
+      public add_schema_version_args getEmptyArgsInstance() {
+        return new add_schema_version_args();
       }
 
       public org.apache.thrift.async.AsyncMethodCallback<Void> 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<Void>() { 
           public void onComplete(Void o) {
-            drop_schema_version_result result = new drop_schema_version_result();
+            add_schema_version_result result = new add_schema_version_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -43669,15 +43548,19 @@ 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;
-            drop_schema_version_result result = new drop_schema_version_result();
-            if (e instanceof NoSuchObjectException) {
-              result.o1 = (NoSuchObjectException) e;
+            add_schema_version_result result = new add_schema_version_result();
+            if (e instanceof AlreadyExistsException) {
+              result.o1 = (AlreadyExistsException) e;
               result.setO1IsSet(true);
               msg = result;
-            } else if (e instanceof MetaException) {
-              result.o2 = (MetaException) e;
+            } else if (e instanceof NoSuchObjectException) {
+              result.o2 = (NoSuchObjectException) e;
               result.setO2IsSet(true);
               msg = result;
+            } else if (e instanceof MetaException) {
+              result.o3 = (MetaException) e;
+              result.setO3IsSet(true);
+              msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -43705,25 +43588,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, drop_schema_version_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.drop_schema_version(args.schemaVersion,resultHandler);
+      public void start(I iface, add_schema_version_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.add_schema_version(args.schemaVersion,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_schemas_by_cols<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_schemas_by_cols_args, FindSchemasByColsResp> {
-      public get_schemas_by_cols() {
-        super("get_schemas_by_cols");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_schema_version<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_schema_version_args, SchemaVersion> {
+      public get_schema_version() {
+        super("get_schema_version");
       }
 
-      public get_schemas_by_cols_args getEmptyArgsInstance() {
-        return new get_schemas_by_cols_args();
+      public get_schema_version_args getEmptyArgsInstance() {
+        return new get_schema_version_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<FindSchemasByColsResp> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<SchemaVersion> 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<FindSchemasByColsResp>() { 
-          public void onComplete(FindSchemasByColsResp o) {
-            get_schemas_by_cols_result result = new get_schemas_by_cols_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<SchemaVersion>() { 
+          public void onComplete(SchemaVersion o) {
+            get_schema_version_result result = new get_schema_version_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -43738,11 +43621,15 @@ 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_schemas_by_cols_result result = new get_schemas_by_cols_result();
-            if (e instanceof MetaException) {
-              result.o1 = (MetaException) e;
+            get_schema_version_result result = new get_schema_version_result();
+            if (e instanceof NoSuchObjectException) {
+              result.o1 = (NoSuchObjectException) e;
               result.setO1IsSet(true);
               msg = result;
+            } else if (e instanceof MetaException) {
+              result.o2 = (MetaException) e;
+              result.setO2IsSet(true);
+              msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -43770,25 +43657,26 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, get_schemas_by_cols_args args, org.apache.thrift.async.AsyncMethodCallback<FindSchemasByColsResp> resultHandler) throws org.apache.thrift.TException {
-        iface.get_schemas_by_cols(args.rqst,resultHandler);
+      public void start(I iface, get_schema_version_args args, org.apache.thrift.async.AsyncMethodCallback<SchemaVersion> resultHandler) throws org.apache.thrift.TException {
+        iface.get_schema_version(args.schemaVersion,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class map_schema_version_to_serde<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, map_schema_version_to_serde_args, Void> {
-      public map_schema_version_to_serde() {
-        super("map_schema_version_to_serde");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_schema_latest_version<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_schema_latest_version_args, SchemaVersion> {
+      public get_schema_latest_version() {
+        super("get_schema_latest_version");
       }
 
-      public map_schema_version_to_serde_args getEmptyArgsInstance() {
-        return new map_schema_version_to_serde_args();
+      public get_schema_latest_version_args getEmptyArgsInstance() {
+        return new get_schema_latest_version_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<SchemaVersion> 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<Void>() { 
-          public void onComplete(Void o) {
-            map_schema_version_to_serde_result result = new map_schema_version_to_serde_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<SchemaVersion>() { 
+          public void onComplete(SchemaVersion o) {
+            get_schema_latest_version_result result = new get_schema_latest_version_result();
+            result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -43802,7 +43690,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;
-            map_schema_version_to_serde_result result = new map_schema_version_to_serde_result();
+            get_schema_latest_version_result result = new get_schema_latest_version_result();
             if (e instanceof NoSuchObjectException) {
               result.o1 = (NoSuchObjectException) e;
               result.setO1IsSet(true);
@@ -43838,25 +43726,26 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, map_schema_version_to_serde_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.map_schema_version_to_serde(args.rqst,resultHandler);
+      public void start(I iface, get_schema_latest_version_args args, org.apache.thrift.async.AsyncMethodCallback<SchemaVersion> resultHandler) throws org.apache.thrift.TException {
+        iface.get_schema_latest_version(args.schemaName,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class set_schema_version_state<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, set_schema_version_state_args, Void> {
-      public set_schema_version_state() {
-        super("set_schema_version_state");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_schema_all_versions<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_schema_all_versions_args, java.util.List<SchemaVersion>> {
+      public get_schema_all_versions() {
+        super("get_schema_all_versions");
       }
 
-      public set_schema_version_state_args getEmptyArgsInstance() {
-        return new set_schema_version_state_args();
+      public get_schema_all_versions_args getEmptyArgsInstance() {
+        return new get_schema_all_versions_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<SchemaVersion>> 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<Void>() { 
-          public void onComplete(Void o) {
-            set_schema_version_state_result result = new set_schema_version_state_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<java.util.List<SchemaVersion>>() { 
+          public void onComplete(java.util.List<SchemaVersion> o) {
+            get_schema_all_versions_result result = new get_schema_all_versions_result();
+            result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -43870,18 +43759,14 @@ 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;
-            set_schema_version_state_result result = new set_schema_version_state_result();
+            get_schema_all_versions_result result = new get_schema_all_versions_result();
             if (e instanceof NoSuchObjectException) {
               result.o1 = (NoSuchObjectException) e;
               result.setO1IsSet(true);
               msg = result;
-            } else if (e instanceof InvalidOperationException) {
-              result.o2 = (InvalidOperationException) e;
-              result.setO2IsSet(true);
-              msg = result;
             } else if (e instanceof MetaException) {
-              result.o3 = (MetaException) e;
-              result.setO3IsSet(true);
+              result.o2 = (MetaException) e;
+              result.setO2IsSet(true);
               msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
@@ -43910,25 +43795,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, set_schema_version_state_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.set_schema_version_state(args.rqst,resultHandler);
+      public void start(I iface, get_schema_all_versions_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<SchemaVersion>> resultHandler) throws org.apache.thrift.TException {
+        iface.get_schema_all_versions(args.schemaName,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_serde<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_serde_args, Void> {
-      public add_serde() {
-        super("add_serde");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class drop_schema_version<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drop_schema_version_args, Void> {
+      public drop_schema_version() {
+        super("drop_schema_version");
       }
 
-      public add_serde_args getEmptyArgsInstance() {
-        return new add_serde_args();
+      public drop_schema_version_args getEmptyArgsInstance() {
+        return new drop_schema_version_args();
       }
 
       public org.apache.thrift.async.AsyncMethodCallback<Void> 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<Void>() { 
           public void onComplete(Void o) {
-            add_serde_result result = new add_serde_result();
+            drop_schema_version_result result = new drop_schema_version_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -43942,9 +43827,9 @@ 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;
-            add_serde_result result = new add_serde_result();
-            if (e instanceof AlreadyExistsException) {
-              result.o1 = (AlreadyExistsException) e;
+            drop_schema_version_result result = new drop_schema_version_result();
+            if (e instanceof NoSuchObjectException) {
+              result.o1 = (NoSuchObjectException) e;
               result.setO1IsSet(true);
               msg = result;
             } else if (e instanceof MetaException) {
@@ -43978,25 +43863,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, add_serde_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.add_serde(args.serde,resultHandler);
+      public void start(I iface, drop_schema_version_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.drop_schema_version(args.schemaVersion,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_serde<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_serde_args, SerDeInfo> {
-      public get_serde() {
-        super("get_serde");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_schemas_by_cols<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_schemas_by_cols_args, FindSchemasByColsResp> {
+      public get_schemas_by_cols() {
+        super("get_schemas_by_cols");
       }
 
-      public get_serde_args getEmptyArgsInstance() {
-        return new get_serde_args();
+      public get_schemas_by_cols_args getEmptyArgsInstance() {
+        return new get_schemas_by_cols_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<SerDeInfo> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<FindSchemasByColsResp> 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<SerDeInfo>() { 
-          public void onComplete(SerDeInfo o) {
-            get_serde_result result = new get_serde_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<FindSchemasByColsResp>() { 
+          public void onComplete(FindSchemasByColsResp o) {
+            get_schemas_by_cols_result result = new get_schemas_by_cols_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -44011,15 +43896,11 @@ 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_serde_result result = new get_serde_result();
-            if (e instanceof NoSuchObjectException) {
-              result.o1 = (NoSuchObjectException) e;
+            get_schemas_by_cols_result result = new get_schemas_by_cols_result();
+            if (e instanceof MetaException) {
+              result.o1 = (MetaException) e;
               result.setO1IsSet(true);
               msg = result;
-            } else if (e instanceof MetaException) {
-              result.o2 = (MetaException) e;
-              result.setO2IsSet(true);
-              msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -44047,26 +43928,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, get_serde_args args, org.apache.thrift.async.AsyncMethodCallback<SerDeInfo> resultHandler) throws org.apache.thrift.TException {
-        iface.get_serde(args.rqst,resultHandler);
+      public void start(I iface, get_schemas_by_cols_args args, org.apache.thrift.async.AsyncMethodCallback<FindSchemasByColsResp> resultHandler) throws org.apache.thrift.TException {
+        iface.get_schemas_by_cols(args.rqst,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_lock_materialization_rebuild<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_lock_materialization_rebuild_args, LockResponse> {
-      public get_lock_materialization_rebuild() {
-        super("get_lock_materialization_rebuild");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class map_schema_version_to_serde<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, map_schema_version_to_serde_args, Void> {
+      public map_schema_version_to_serde() {
+        super("map_schema_version_to_serde");
       }
 
-      public get_lock_materialization_rebuild_args getEmptyArgsInstance() {
-        return new get_lock_materialization_rebuild_args();
+      public map_schema_version_to_serde_args getEmptyArgsInstance() {
+        return new map_schema_version_to_serde_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<LockResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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<LockResponse>() { 
-          public void onComplete(LockResponse o) {
-            get_lock_materialization_rebuild_result result = new get_lock_materialization_rebuild_result();
-            result.success = o;
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            map_schema_version_to_serde_result result = new map_schema_version_to_serde_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -44080,8 +43960,16 @@ 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_lock_materialization_rebuild_result result = new get_lock_materialization_rebuild_result();
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
+            map_schema_version_to_serde_result result = new map_schema_version_to_serde_result();
+            if (e instanceof NoSuchObjectException) {
+              result.o1 = (NoSuchObjectException) e;
+              result.setO1IsSet(true);
+              msg = result;
+            } else if (e instanceof MetaException) {
+              result.o2 = (MetaException) e;
+              result.setO2IsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -44108,27 +43996,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, get_lock_materialization_rebuild_args args, org.apache.thrift.async.AsyncMethodCallback<LockResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.get_lock_materialization_rebuild(args.dbName, args.tableName, args.txnId,resultHandler);
+      public void start(I iface, map_schema_version_to_serde_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.map_schema_version_to_serde(args.rqst,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class heartbeat_lock_materialization_rebuild<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, heartbeat_lock_materialization_rebuild_args, java.lang.Boolean> {
-      public heartbeat_lock_materialization_rebuild() {
-        super("heartbeat_lock_materialization_rebuild");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class set_schema_version_state<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, set_schema_version_state_args, Void> {
+      public set_schema_version_state() {
+        super("set_schema_version_state");
       }
 
-      public heartbeat_lock_materialization_rebuild_args getEmptyArgsInstance() {
-        return new heartbeat_lock_materialization_rebuild_args();
+      public set_schema_version_state_args getEmptyArgsInstance() {
+        return new set_schema_version_state_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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.Boolean>() { 
-          public void onComplete(java.lang.Boolean o) {
-            heartbeat_lock_materialization_rebuild_result result = new heartbeat_lock_materialization_rebuild_result();
-            result.success = o;
-            result.setSuccessIsSet(true);
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            set_schema_version_state_result result = new set_schema_version_state_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -44142,8 +44028,20 @@ 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;
-            heartbeat_lock_materialization_rebuild_result result = new heartbeat_lock_materialization_rebuild_result();
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
+            set_schema_version_state_result result = new set_schema_version_state_result();
+            if (e instanceof NoSuchObjectException) {
+              result.o1 = (NoSuchObjectException) e;
+              result.setO1IsSet(true);
+              msg = result;
+            } else if (e instanceof InvalidOperationException) {
+              result.o2 = (InvalidOperationException) e;
+              result.setO2IsSet(true);
+              msg = result;
+            } else if (e instanceof MetaException) {
+              result.o3 = (MetaException) e;
+              result.setO3IsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -44170,25 +44068,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, heartbeat_lock_materialization_rebuild_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
-        iface.heartbeat_lock_materialization_rebuild(args.dbName, args.tableName, args.txnId,resultHandler);
+      public void start(I iface, set_schema_version_state_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.set_schema_version_state(args.rqst,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_runtime_stats<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_runtime_stats_args, Void> {
-      public add_runtime_stats() {
-        super("add_runtime_stats");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_serde<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_serde_args, Void> {
+      public add_serde() {
+        super("add_serde");
       }
 
-      public add_runtime_stats_args getEmptyArgsInstance() {
-        return new add_runtime_stats_args();
+      public add_serde_args getEmptyArgsInstance() {
+        return new add_serde_args();
       }
 
       public org.apache.thrift.async.AsyncMethodCallback<Void> 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<Void>() { 
           public void onComplete(Void o) {
-            add_runtime_stats_result result = new add_runtime_stats_result();
+            add_serde_result result = new add_serde_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -44202,11 +44100,15 @@ 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;
-            add_runtime_stats_result result = new add_runtime_stats_result();
-            if (e instanceof MetaException) {
-              result.o1 = (MetaException) e;
+            add_serde_result result = new add_serde_result();
+            if (e instanceof AlreadyExistsException) {
+              result.o1 = (AlreadyExistsException) e;
               result.setO1IsSet(true);
               msg = result;
+            } else if (e instanceof MetaException) {
+              result.o2 = (MetaException) e;
+              result.setO2IsSet(true);
+              msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -44234,25 +44136,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, add_runtime_stats_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.add_runtime_stats(args.stat,resultHandler);
+      public void start(I iface, add_serde_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.add_serde(args.serde,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_runtime_stats<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_runtime_stats_args, java.util.List<RuntimeStat>> {
-      public get_runtime_stats() {
-        super("get_runtime_stats");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_serde<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_serde_args, SerDeInfo> {
+      public get_serde() {
+        super("get_serde");
       }
 
-      public get_runtime_stats_args getEmptyArgsInstance() {
-        return new get_runtime_stats_args();
+      public get_serde_args getEmptyArgsInstance() {
+        return new get_serde_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<RuntimeStat>> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<SerDeInfo> 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.util.List<RuntimeStat>>() { 
-          public void onComplete(java.util.List<RuntimeStat> o) {
-            get_runtime_stats_result result = new get_runtime_stats_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<SerDeInfo>() { 
+          public void onComplete(SerDeInfo o) {
+            get_serde_result result = new get_serde_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -44267,11 +44169,15 @@ 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_runtime_stats_result result = new get_runtime_stats_result();
-            if (e instanceof MetaException) {
-              result.o1 = (MetaException) e;
+            get_serde_result result = new get_serde_result();
+            if (e instanceof NoSuchObjectException) {
+              result.o1 = (NoSuchObjectException) e;
               result.setO1IsSet(true);
               msg = result;
+            } else if (e instanceof MetaException) {
+              result.o2 = (MetaException) e;
+              result.setO2IsSet(true);
+              msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -44299,25 +44205,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, get_runtime_stats_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<RuntimeStat>> resultHandler) throws org.apache.thrift.TException {
-        iface.get_runtime_stats(args.rqst,resultHandler);
+      public void start(I iface, get_serde_args args, org.apache.thrift.async.AsyncMethodCallback<SerDeInfo> resultHandler) throws org.apache.thrift.TException {
+        iface.get_serde(args.rqst,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_partitions_with_specs<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_partitions_with_specs_args, GetPartitionsResponse> {
-      public get_partitions_with_specs() {
-        super("get_partitions_with_specs");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_lock_materialization_rebuild<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_lock_materialization_rebuild_args, LockResponse> {
+      public get_lock_materialization_rebuild() {
+        super("get_lock_materialization_rebuild");
       }
 
-      public get_partitions_with_specs_args getEmptyArgsInstance() {
-        return new get_partitions_with_specs_args();
+      public get_lock_materialization_rebuild_args getEmptyArgsInstance() {
+        return new get_lock_materialization_rebuild_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<GetPartitionsResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<LockResponse> 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<GetPartitionsResponse>() { 
-          public void onComplete(GetPartitionsResponse o) {
-            get_partitions_with_specs_result result = new get_partitions_with_specs_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<LockResponse>() { 
+          public void onComplete(LockResponse o) {
+            get_lock_materialization_rebuild_result result = new get_lock_materialization_rebuild_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -44332,12 +44238,8 @@ 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_partitions_with_specs_result result = new get_partitions_with_specs_result();
-            if (e instanceof MetaException) {
-              result.o1 = (MetaException) e;
-              result.setO1IsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+            get_lock_materialization_rebuild_result result = new get_lock_materialization_rebuild_result();
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -44364,26 +44266,27 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, get_partitions_with_specs_args args, org.apache.thrift.async.AsyncMethodCallback<GetPartitionsResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.get_partitions_with_specs(args.request,resultHandler);
+      public void start(I iface, get_lock_materialization_rebuild_args args, org.apache.thrift.async.AsyncMethodCallback<LockResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.get_lock_materialization_rebuild(args.dbName, args.tableName, args.txnId,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class scheduled_query_poll<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, scheduled_query_poll_args, ScheduledQueryPollResponse> {
-      public scheduled_query_poll() {
-        super("scheduled_query_poll");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class heartbeat_lock_materialization_rebuild<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, heartbeat_lock_materialization_rebuild_args, java.lang.Boolean> {
+      public heartbeat_lock_materialization_rebuild() {
+        super("heartbeat_lock_materialization_rebuild");
       }
 
-      public scheduled_query_poll_args getEmptyArgsInstance() {
-        return new scheduled_query_poll_args();
+      public heartbeat_lock_materialization_rebuild_args getEmptyArgsInstance() {
+        return new heartbeat_lock_materialization_rebuild_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<ScheduledQueryPollResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> 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<ScheduledQueryPollResponse>() { 
-          public void onComplete(ScheduledQueryPollResponse o) {
-            scheduled_query_poll_result result = new scheduled_query_poll_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean>() { 
+          public void onComplete(java.lang.Boolean o) {
+            heartbeat_lock_materialization_rebuild_result result = new heartbeat_lock_materialization_rebuild_result();
             result.success = o;
+            result.setSuccessIsSet(true);
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -44397,12 +44300,8 @@ 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;
-            scheduled_query_poll_result result = new scheduled_query_poll_result();
-            if (e instanceof MetaException) {
-              result.o1 = (MetaException) e;
-              result.setO1IsSet(true);
-              msg = result;
-            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+            heartbeat_lock_materialization_rebuild_result result = new heartbeat_lock_materialization_rebuild_result();
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -44429,25 +44328,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, scheduled_query_poll_args args, org.apache.thrift.async.AsyncMethodCallback<ScheduledQueryPollResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.scheduled_query_poll(args.request,resultHandler);
+      public void start(I iface, heartbeat_lock_materialization_rebuild_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
+        iface.heartbeat_lock_materialization_rebuild(args.dbName, args.tableName, args.txnId,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class scheduled_query_maintenance<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, scheduled_query_maintenance_args, Void> {
-      public scheduled_query_maintenance() {
-        super("scheduled_query_maintenance");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_runtime_stats<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_runtime_stats_args, Void> {
+      public add_runtime_stats() {
+        super("add_runtime_stats");
       }
 
-      public scheduled_query_maintenance_args getEmptyArgsInstance() {
-        return new scheduled_query_maintenance_args();
+      public add_runtime_stats_args getEmptyArgsInstance() {
+        return new add_runtime_stats_args();
       }
 
       public org.apache.thrift.async.AsyncMethodCallback<Void> 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<Void>() { 
           public void onComplete(Void o) {
-            scheduled_query_maintenance_result result = new scheduled_query_maintenance_result();
+            add_runtime_stats_result result = new add_runtime_stats_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -44461,23 +44360,11 @@ 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;
-            scheduled_query_maintenance_result result = new scheduled_query_maintenance_result();
+            add_runtime_stats_result result = new add_runtime_stats_result();
             if (e instanceof MetaException) {
               result.o1 = (MetaException) e;
               result.setO1IsSet(true);
               msg = result;
-            } else if (e instanceof NoSuchObjectException) {
-              result.o2 = (NoSuchObjectException) e;
-              result.setO2IsSet(true);
-              msg = result;
-            } else if (e instanceof AlreadyExistsException) {
-              result.o3 = (AlreadyExistsException) e;
-              result.setO3IsSet(true);
-              msg = result;
-            } else if (e instanceof InvalidInputException) {
-              result.o4 = (InvalidInputException) e;
-              result.setO4IsSet(true);
-              msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -44505,25 +44392,26 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, scheduled_query_maintenance_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.scheduled_query_maintenance(args.request,resultHandler);
+      public void start(I iface, add_runtime_stats_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.add_runtime_stats(args.stat,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class scheduled_query_progress<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, scheduled_query_progress_args, Void> {
-      public scheduled_query_progress() {
-        super("scheduled_query_progress");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_runtime_stats<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_runtime_stats_args, java.util.List<RuntimeStat>> {
+      public get_runtime_stats() {
+        super("get_runtime_stats");
       }
 
-      public scheduled_query_progress_args getEmptyArgsInstance() {
-        return new scheduled_query_progress_args();
+      public get_runtime_stats_args getEmptyArgsInstance() {
+        return new get_runtime_stats_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<RuntimeStat>> 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<Void>() { 
-          public void onComplete(Void o) {
-            scheduled_query_progress_result result = new scheduled_query_progress_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<java.util.List<RuntimeStat>>() { 
+          public void onComplete(java.util.List<RuntimeStat> o) {
+            get_runtime_stats_result result = new get_runtime_stats_result();
+            result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -44537,15 +44425,11 @@ 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;
-            scheduled_query_progress_result result = new scheduled_query_progress_result();
+            get_runtime_stats_result result = new get_runtime_stats_result();
             if (e instanceof MetaException) {
               result.o1 = (MetaException) e;
               result.setO1IsSet(true);
               msg = result;
-            } else if (e instanceof InvalidOperationException) {
-              result.o2 = (InvalidOperationException) e;
-              result.setO2IsSet(true);
-              msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -44573,25 +44457,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, scheduled_query_progress_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.scheduled_query_progress(args.info,resultHandler);
+      public void start(I iface, get_runtime_stats_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<RuntimeStat>> resultHandler) throws org.apache.thrift.TException {
+        iface.get_runtime_stats(args.rqst,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_scheduled_query<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_scheduled_query_args, ScheduledQuery> {
-      public get_scheduled_query() {
-        super("get_scheduled_query");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_partitions_with_specs<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_partitions_with_specs_args, GetPartitionsResponse> {
+      public get_partitions_with_specs() {
+        super("get_partitions_with_specs");
       }
 
-      public get_scheduled_query_args getEmptyArgsInstance() {
-        return new get_scheduled_query_args();
+      public get_partitions_with_specs_args getEmptyArgsInstance() {
+        return new get_partitions_with_specs_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<ScheduledQuery> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<GetPartitionsResponse> 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<ScheduledQuery>() { 
-          public void onComplete(ScheduledQuery o) {
-            get_scheduled_query_result result = new get_scheduled_query_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<GetPartitionsResponse>() { 
+          public void onComplete(GetPartitionsResponse o) {
+            get_partitions_with_specs_result result = new get_partitions_with_specs_result();
             result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -44606,15 +44490,11 @@ 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_scheduled_query_result result = new get_scheduled_query_result();
+            get_partitions_with_specs_result result = new get_partitions_with_specs_result();
             if (e instanceof MetaException) {
               result.o1 = (MetaException) e;
               result.setO1IsSet(true);
               msg = result;
-            } else if (e instanceof NoSuchObjectException) {
-              result.o2 = (NoSuchObjectException) e;
-              result.setO2IsSet(true);
-              msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -44642,25 +44522,26 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, get_scheduled_query_args args, org.apache.thrift.async.AsyncMethodCallback<ScheduledQuery> resultHandler) throws org.apache.thrift.TException {
-        iface.get_scheduled_query(args.scheduleKey,resultHandler);
+      public void start(I iface, get_partitions_with_specs_args args, org.apache.thrift.async.AsyncMethodCallback<GetPartitionsResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.get_partitions_with_specs(args.request,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_replication_metrics<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_replication_metrics_args, Void> {
-      public add_replication_metrics() {
-        super("add_replication_metrics");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class scheduled_query_poll<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, scheduled_query_poll_args, ScheduledQueryPollResponse> {
+      public scheduled_query_poll() {
+        super("scheduled_query_poll");
       }
 
-      public add_replication_metrics_args getEmptyArgsInstance() {
-        return new add_replication_metrics_args();
+      public scheduled_query_poll_args getEmptyArgsInstance() {
+        return new scheduled_query_poll_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<ScheduledQueryPollResponse> 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<Void>() { 
-          public void onComplete(Void o) {
-            add_replication_metrics_result result = new add_replication_metrics_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<ScheduledQueryPollResponse>() { 
+          public void onComplete(ScheduledQueryPollResponse o) {
+            scheduled_query_poll_result result = new scheduled_query_poll_result();
+            result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -44674,7 +44555,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;
-            add_replication_metrics_result result = new add_replication_metrics_result();
+            scheduled_query_poll_result result = new scheduled_query_poll_result();
             if (e instanceof MetaException) {
               result.o1 = (MetaException) e;
               result.setO1IsSet(true);
@@ -44706,26 +44587,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, add_replication_metrics_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.add_replication_metrics(args.replicationMetricList,resultHandler);
+      public void start(I iface, scheduled_query_poll_args args, org.apache.thrift.async.AsyncMethodCallback<ScheduledQueryPollResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.scheduled_query_poll(args.request,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_replication_metrics<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_replication_metrics_args, ReplicationMetricList> {
-      public get_replication_metrics() {
-        super("get_replication_metrics");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class scheduled_query_maintenance<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, scheduled_query_maintenance_args, Void> {
+      public scheduled_query_maintenance() {
+        super("scheduled_query_maintenance");
       }
 
-      public get_replication_metrics_args getEmptyArgsInstance() {
-        return new get_replication_metrics_args();
+      public scheduled_query_maintenance_args getEmptyArgsInstance() {
+        return new scheduled_query_maintenance_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<ReplicationMetricList> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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<ReplicationMetricList>() { 
-          public void onComplete(ReplicationMetricList o) {
-            get_replication_metrics_result result = new get_replication_metrics_result();
-            result.success = o;
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            scheduled_query_maintenance_result result = new scheduled_query_maintenance_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -44739,11 +44619,23 @@ 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_replication_metrics_result result = new get_replication_metrics_result();
+            scheduled_query_maintenance_result result = new scheduled_query_maintenance_result();
             if (e instanceof MetaException) {
               result.o1 = (MetaException) e;
               result.setO1IsSet(true);
               msg = result;
+            } else if (e instanceof NoSuchObjectException) {
+              result.o2 = (NoSuchObjectException) e;
+              result.setO2IsSet(true);
+              msg = result;
+            } else if (e instanceof AlreadyExistsException) {
+              result.o3 = (AlreadyExistsException) e;
+              result.setO3IsSet(true);
+              msg = result;
+            } else if (e instanceof InvalidInputException) {
+              result.o4 = (InvalidInputException) e;
+              result.setO4IsSet(true);
+              msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -44771,26 +44663,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, get_replication_metrics_args args, org.apache.thrift.async.AsyncMethodCallback<ReplicationMetricList> resultHandler) throws org.apache.thrift.TException {
-        iface.get_replication_metrics(args.rqst,resultHandler);
+      public void start(I iface, scheduled_query_maintenance_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.scheduled_query_maintenance(args.request,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_open_txns_req<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_open_txns_req_args, GetOpenTxnsResponse> {
-      public get_open_txns_req() {
-        super("get_open_txns_req");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class scheduled_query_progress<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, scheduled_query_progress_args, Void> {
+      public scheduled_query_progress() {
+        super("scheduled_query_progress");
       }
 
-      public get_open_txns_req_args getEmptyArgsInstance() {
-        return new get_open_txns_req_args();
+      public scheduled_query_progress_args getEmptyArgsInstance() {
+        return new scheduled_query_progress_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<GetOpenTxnsResponse> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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<GetOpenTxnsResponse>() { 
-          public void onComplete(GetOpenTxnsResponse o) {
-            get_open_txns_req_result result = new get_open_txns_req_result();
-            result.success = o;
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            scheduled_query_progress_result result = new scheduled_query_progress_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -44804,8 +44695,16 @@ 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_open_txns_req_result result = new get_open_txns_req_result();
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
+            scheduled_query_progress_result result = new scheduled_query_progress_result();
+            if (e instanceof MetaException) {
+              result.o1 = (MetaException) e;
+              result.setO1IsSet(true);
+              msg = result;
+            } else if (e instanceof InvalidOperationException) {
+              result.o2 = (InvalidOperationException) e;
+              result.setO2IsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -44832,25 +44731,26 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, get_open_txns_req_args args, org.apache.thrift.async.AsyncMethodCallback<GetOpenTxnsResponse> resultHandler) throws org.apache.thrift.TException {
-        iface.get_open_txns_req(args.getOpenTxnsRequest,resultHandler);
+      public void start(I iface, scheduled_query_progress_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.scheduled_query_progress(args.info,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class create_stored_procedure<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, create_stored_procedure_args, Void> {
-      public create_stored_procedure() {
-        super("create_stored_procedure");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_scheduled_query<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_scheduled_query_args, ScheduledQuery> {
+      public get_scheduled_query() {
+        super("get_scheduled_query");
       }
 
-      public create_stored_procedure_args getEmptyArgsInstance() {
-        return new create_stored_procedure_args();
+      public get_scheduled_query_args getEmptyArgsInstance() {
+        return new get_scheduled_query_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<ScheduledQuery> 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<Void>() { 
-          public void onComplete(Void o) {
-            create_stored_procedure_result result = new create_stored_procedure_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<ScheduledQuery>() { 
+          public void onComplete(ScheduledQuery o) {
+            get_scheduled_query_result result = new get_scheduled_query_result();
+            result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -44864,13 +44764,13 @@ 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;
-            create_stored_procedure_result result = new create_stored_procedure_result();
-            if (e instanceof NoSuchObjectException) {
-              result.o1 = (NoSuchObjectException) e;
+            get_scheduled_query_result result = new get_scheduled_query_result();
+            if (e instanceof MetaException) {
+              result.o1 = (MetaException) e;
               result.setO1IsSet(true);
               msg = result;
-            } else if (e instanceof MetaException) {
-              result.o2 = (MetaException) e;
+            } else if (e instanceof NoSuchObjectException) {
+              result.o2 = (NoSuchObjectException) e;
               result.setO2IsSet(true);
               msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
@@ -44900,26 +44800,25 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, create_stored_procedure_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.create_stored_procedure(args.proc,resultHandler);
+      public void start(I iface, get_scheduled_query_args args, org.apache.thrift.async.AsyncMethodCallback<ScheduledQuery> resultHandler) throws org.apache.thrift.TException {
+        iface.get_scheduled_query(args.scheduleKey,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_stored_procedure<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_stored_procedure_args, StoredProcedure> {
-      public get_stored_procedure() {
-        super("get_stored_procedure");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_replication_metrics<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_replication_metrics_args, Void> {
+      public add_replication_metrics() {
+        super("add_replication_metrics");
       }
 
-      public get_stored_procedure_args getEmptyArgsInstance() {
-        return new get_stored_procedure_args();
+      public add_replication_metrics_args getEmptyArgsInstance() {
+        return new add_replication_metrics_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<StoredProcedure> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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<StoredProcedure>() { 
-          public void onComplete(StoredProcedure o) {
-            get_stored_procedure_result result = new get_stored_procedure_result();
-            result.success = o;
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            add_replication_metrics_result result = new add_replication_metrics_result();
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -44933,15 +44832,11 @@ 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_stored_procedure_result result = new get_stored_procedure_result();
+            add_replication_metrics_result result = new add_replication_metrics_result();
             if (e instanceof MetaException) {
               result.o1 = (MetaException) e;
               result.setO1IsSet(true);
               msg = result;
-            } else if (e instanceof NoSuchObjectException) {
-              result.o2 = (NoSuchObjectException) e;
-              result.setO2IsSet(true);
-              msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -44969,25 +44864,26 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
       }
 
-      public void start(I iface, get_stored_procedure_args args, org.apache.thrift.async.AsyncMethodCallback<StoredProcedure> resultHandler) throws org.apache.thrift.TException {
-        iface.get_stored_procedure(args.request,resultHandler);
+      public void start(I iface, add_replication_metrics_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.add_replication_metrics(args.replicationMetricList,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class drop_stored_procedure<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drop_stored_procedure_args, Void> {
-      public drop_stored_procedure() {
-        super("drop_stored_procedure");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_replication_metrics<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_replication_metrics_args, ReplicationMetricList> {
+      public get_replication_metrics() {
+        super("get_replication_metrics");
       }
 
-      public drop_stored_procedure_args getEmptyArgsInstance() {
-        return new drop_stored_procedure_args();
+      public get_replication_metrics_args getEmptyArgsInstance() {
+        return new get_replication_metrics_args();
       }
 
-      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+      public org.apache.thrift.async.AsyncMethodCallback<ReplicationMetricList> 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<Void>() { 
-          public void onComplete(Void o) {
-            drop_stored_procedure_result result = new drop_stored_procedure_result();
+        return new org.apache.thrift.async.AsyncMethodCallback<ReplicationMetricList>() { 
+          public void onComplete(ReplicationMetricList o) {
+            get_replication_metrics_result result = new get_replication_metrics_result();
+            result.success = o;
             try {
               fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -45001,7 +44897,269 @@ 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;
-            drop_stored_procedure_result result = new drop_stored_procedure_result();
+            get_replication_metrics_result result = new get_replication_metrics_result();
+            if (e instanceof MetaException) {
+              result.o1 = (MetaException) e;
+              result.setO1IsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
+            } else {
+              _LOGGER.error("Exception inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+            } catch (java.lang.Exception ex) {
+              _LOGGER.error("Exception writing to internal frame buffer", ex);
+              fb.close();
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, get_replication_metrics_args args, org.apache.thrift.async.AsyncMethodCallback<ReplicationMetricList> resultHandler) throws org.apache.thrift.TException {
+        iface.get_replication_metrics(args.rqst,resultHandler);
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_open_txns_req<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_open_txns_req_args, GetOpenTxnsResponse> {
+      public get_open_txns_req() {
+        super("get_open_txns_req");
+      }
+
+      public get_open_txns_req_args getEmptyArgsInstance() {
+        return new get_open_txns_req_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<GetOpenTxnsResponse> 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<GetOpenTxnsResponse>() { 
+          public void onComplete(GetOpenTxnsResponse o) {
+            get_open_txns_req_result result = new get_open_txns_req_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+            } catch (org.apache.thrift.transport.TTransportException e) {
+              _LOGGER.error("TTransportException writing to internal frame buffer", e);
+              fb.close();
+            } catch (java.lang.Exception e) {
+              _LOGGER.error("Exception writing to internal frame buffer", e);
+              onError(e);
+            }
+          }
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            get_open_txns_req_result result = new get_open_txns_req_result();
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
+            } else {
+              _LOGGER.error("Exception inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+            } catch (java.lang.Exception ex) {
+              _LOGGER.error("Exception writing to internal frame buffer", ex);
+              fb.close();
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, get_open_txns_req_args args, org.apache.thrift.async.AsyncMethodCallback<GetOpenTxnsResponse> resultHandler) throws org.apache.thrift.TException {
+        iface.get_open_txns_req(args.getOpenTxnsRequest,resultHandler);
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class create_stored_procedure<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, create_stored_procedure_args, Void> {
+      public create_stored_procedure() {
+        super("create_stored_procedure");
+      }
+
+      public create_stored_procedure_args getEmptyArgsInstance() {
+        return new create_stored_procedure_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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<Void>() { 
+          public void onComplete(Void o) {
+            create_stored_procedure_result result = new create_stored_procedure_result();
+            try {
+              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+            } catch (org.apache.thrift.transport.TTransportException e) {
+              _LOGGER.error("TTransportException writing to internal frame buffer", e);
+              fb.close();
+            } catch (java.lang.Exception e) {
+              _LOGGER.error("Exception writing to internal frame buffer", e);
+              onError(e);
+            }
+          }
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            create_stored_procedure_result result = new create_stored_procedure_result();
+            if (e instanceof NoSuchObjectException) {
+              result.o1 = (NoSuchObjectException) e;
+              result.setO1IsSet(true);
+              msg = result;
+            } else if (e instanceof MetaException) {
+              result.o2 = (MetaException) e;
+              result.setO2IsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
+            } else {
+              _LOGGER.error("Exception inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+            } catch (java.lang.Exception ex) {
+              _LOGGER.error("Exception writing to internal frame buffer", ex);
+              fb.close();
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, create_stored_procedure_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.create_stored_procedure(args.proc,resultHandler);
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_stored_procedure<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_stored_procedure_args, StoredProcedure> {
+      public get_stored_procedure() {
+        super("get_stored_procedure");
+      }
+
+      public get_stored_procedure_args getEmptyArgsInstance() {
+        return new get_stored_procedure_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<StoredProcedure> 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<StoredProcedure>() { 
+          public void onComplete(StoredProcedure o) {
+            get_stored_procedure_result result = new get_stored_procedure_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+            } catch (org.apache.thrift.transport.TTransportException e) {
+              _LOGGER.error("TTransportException writing to internal frame buffer", e);
+              fb.close();
+            } catch (java.lang.Exception e) {
+              _LOGGER.error("Exception writing to internal frame buffer", e);
+              onError(e);
+            }
+          }
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            get_stored_procedure_result result = new get_stored_procedure_result();
+            if (e instanceof MetaException) {
+              result.o1 = (MetaException) e;
+              result.setO1IsSet(true);
+              msg = result;
+            } else if (e instanceof NoSuchObjectException) {
+              result.o2 = (NoSuchObjectException) e;
+              result.setO2IsSet(true);
+              msg = result;
+            } else if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
+            } else {
+              _LOGGER.error("Exception inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+            } catch (java.lang.Exception ex) {
+              _LOGGER.error("Exception writing to internal frame buffer", ex);
+              fb.close();
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, get_stored_procedure_args args, org.apache.thrift.async.AsyncMethodCallback<StoredProcedure> resultHandler) throws org.apache.thrift.TException {
+        iface.get_stored_procedure(args.request,resultHandler);
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class drop_stored_procedure<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drop_stored_procedure_args, Void> {
+      public drop_stored_procedure() {
+        super("drop_stored_procedure");
+      }
+
+      public drop_stored_procedure_args getEmptyArgsInstance() {
+        return new drop_stored_procedure_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<Void> 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<Void>() { 
+          public void onComplete(Void o) {
+            drop_stored_procedure_result result = new drop_stored_procedure_result();
+            try {
+              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+            } catch (org.apache.thrift.transport.TTransportException e) {
+              _LOGGER.error("TTransportException writing to internal frame buffer", e);
+              fb.close();
+            } catch (java.lang.Exception e) {
+              _LOGGER.error("Exception writing to internal frame buffer", e);
+              onError(e);
+            }
+          }
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            drop_stored_procedure_result result = new drop_stored_procedure_result();
             if (e instanceof MetaException) {
               result.o1 = (MetaException) e;
               result.setO1IsSet(true);
@@ -229954,6 +230112,883 @@ package org.apache.hadoop.hive.metastore.api;
     }
   }
 
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_write_ids_to_min_history_args implements org.apache.thrift.TBase<add_write_ids_to_min_history_args, add_write_ids_to_min_history_args._Fields>, java.io.Serializable, Cloneable, Comparable<add_write_ids_to_min_history_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("add_write_ids_to_min_history_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.protocol.TField WRITE_IDS_FIELD_DESC = new org.apache.thrift.protocol.TField("writeIds", org.apache.thrift.protocol.TType.MAP, (short)2);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new add_write_ids_to_min_history_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new add_write_ids_to_min_history_argsTupleSchemeFactory();
+
+    private long txnId; // required
+    private @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.Long> writeIds; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      TXN_ID((short)1, "txnId"),
+      WRITE_IDS((short)2, "writeIds");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // TXN_ID
+            return TXN_ID;
+          case 2: // WRITE_IDS
+            return WRITE_IDS;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByName(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    private static final int __TXNID_ISSET_ID = 0;
+    private byte __isset_bitfield = 0;
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      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)));
+      tmpMap.put(_Fields.WRITE_IDS, new org.apache.thrift.meta_data.FieldMetaData("writeIds", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+              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(add_write_ids_to_min_history_args.class, metaDataMap);
+    }
+
+    public add_write_ids_to_min_history_args() {
+    }
+
+    public add_write_ids_to_min_history_args(
+      long txnId,
+      java.util.Map<java.lang.String,java.lang.Long> writeIds)
+    {
+      this();
+      this.txnId = txnId;
+      setTxnIdIsSet(true);
+      this.writeIds = writeIds;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public add_write_ids_to_min_history_args(add_write_ids_to_min_history_args other) {
+      __isset_bitfield = other.__isset_bitfield;
+      this.txnId = other.txnId;
+      if (other.isSetWriteIds()) {
+        java.util.Map<java.lang.String,java.lang.Long> __this__writeIds = new java.util.HashMap<java.lang.String,java.lang.Long>(other.writeIds);
+        this.writeIds = __this__writeIds;
+      }
+    }
+
+    public add_write_ids_to_min_history_args deepCopy() {
+      return new add_write_ids_to_min_history_args(this);
+    }
+
+    @Override
+    public void clear() {
+      setTxnIdIsSet(false);
+      this.txnId = 0;
+      this.writeIds = null;
+    }
+
+    public long getTxnId() {
+      return this.txnId;
+    }
+
+    public void setTxnId(long txnId) {
+      this.txnId = txnId;
+      setTxnIdIsSet(true);
+    }
+
+    public void unsetTxnId() {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __TXNID_ISSET_ID);
+    }
+
+    /** Returns true if field txnId is set (has been assigned a value) and false otherwise */
+    public boolean isSetTxnId() {
+      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __TXNID_ISSET_ID);
+    }
+
+    public void setTxnIdIsSet(boolean value) {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __TXNID_ISSET_ID, value);
+    }
+
+    public int getWriteIdsSize() {
+      return (this.writeIds == null) ? 0 : this.writeIds.size();
+    }
+
+    public void putToWriteIds(java.lang.String key, long val) {
+      if (this.writeIds == null) {
+        this.writeIds = new java.util.HashMap<java.lang.String,java.lang.Long>();
+      }
+      this.writeIds.put(key, val);
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.Map<java.lang.String,java.lang.Long> getWriteIds() {
+      return this.writeIds;
+    }
+
+    public void setWriteIds(@org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.Long> writeIds) {
+      this.writeIds = writeIds;
+    }
+
+    public void unsetWriteIds() {
+      this.writeIds = null;
+    }
+
+    /** Returns true if field writeIds is set (has been assigned a value) and false otherwise */
+    public boolean isSetWriteIds() {
+      return this.writeIds != null;
+    }
+
+    public void setWriteIdsIsSet(boolean value) {
+      if (!value) {
+        this.writeIds = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case TXN_ID:
+        if (value == null) {
+          unsetTxnId();
+        } else {
+          setTxnId((java.lang.Long)value);
+        }
+        break;
+
+      case WRITE_IDS:
+        if (value == null) {
+          unsetWriteIds();
+        } else {
+          setWriteIds((java.util.Map<java.lang.String,java.lang.Long>)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TXN_ID:
+        return getTxnId();
+
+      case WRITE_IDS:
+        return getWriteIds();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TXN_ID:
+        return isSetTxnId();
+      case WRITE_IDS:
+        return isSetWriteIds();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof add_write_ids_to_min_history_args)
+        return this.equals((add_write_ids_to_min_history_args)that);
+      return false;
+    }
+
+    public boolean equals(add_write_ids_to_min_history_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_txnId = true;
+      boolean that_present_txnId = true;
+      if (this_present_txnId || that_present_txnId) {
+        if (!(this_present_txnId && that_present_txnId))
+          return false;
+        if (this.txnId != that.txnId)
+          return false;
+      }
+
+      boolean this_present_writeIds = true && this.isSetWriteIds();
+      boolean that_present_writeIds = true && that.isSetWriteIds();
+      if (this_present_writeIds || that_present_writeIds) {
+        if (!(this_present_writeIds && that_present_writeIds))
+          return false;
+        if (!this.writeIds.equals(that.writeIds))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(txnId);
+
+      hashCode = hashCode * 8191 + ((isSetWriteIds()) ? 131071 : 524287);
+      if (isSetWriteIds())
+        hashCode = hashCode * 8191 + writeIds.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(add_write_ids_to_min_history_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.compare(isSetTxnId(), other.isSetTxnId());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTxnId()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.txnId, other.txnId);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetWriteIds(), other.isSetWriteIds());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetWriteIds()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.writeIds, other.writeIds);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      scheme(oprot).write(oprot, this);
+    }
+
+    @Override
+    public java.lang.String toString() {
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("add_write_ids_to_min_history_args(");
+      boolean first = true;
+
+      sb.append("txnId:");
+      sb.append(this.txnId);
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("writeIds:");
+      if (this.writeIds == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.writeIds);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+      try {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bitfield = 0;
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class add_write_ids_to_min_history_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public add_write_ids_to_min_history_argsStandardScheme getScheme() {
+        return new add_write_ids_to_min_history_argsStandardScheme();
+      }
+    }
+
+    private static class add_write_ids_to_min_history_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<add_write_ids_to_min_history_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, add_write_ids_to_min_history_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // TXN_ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+                struct.txnId = iprot.readI64();
+                struct.setTxnIdIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // WRITE_IDS
+              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+                {
+                  org.apache.thrift.protocol.TMap _map2028 = iprot.readMapBegin();
+                  struct.writeIds = new java.util.HashMap<java.lang.String,java.lang.Long>(2*_map2028.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _key2029;
+                  long _val2030;
+                  for (int _i2031 = 0; _i2031 < _map2028.size; ++_i2031)
+                  {
+                    _key2029 = iprot.readString();
+                    _val2030 = iprot.readI64();
+                    struct.writeIds.put(_key2029, _val2030);
+                  }
+                  iprot.readMapEnd();
+                }
+                struct.setWriteIdsIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, add_write_ids_to_min_history_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        oprot.writeFieldBegin(TXN_ID_FIELD_DESC);
+        oprot.writeI64(struct.txnId);
+        oprot.writeFieldEnd();
+        if (struct.writeIds != null) {
+          oprot.writeFieldBegin(WRITE_IDS_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, struct.writeIds.size()));
+            for (java.util.Map.Entry<java.lang.String, java.lang.Long> _iter2032 : struct.writeIds.entrySet())
+            {
+              oprot.writeString(_iter2032.getKey());
+              oprot.writeI64(_iter2032.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class add_write_ids_to_min_history_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public add_write_ids_to_min_history_argsTupleScheme getScheme() {
+        return new add_write_ids_to_min_history_argsTupleScheme();
+      }
+    }
+
+    private static class add_write_ids_to_min_history_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<add_write_ids_to_min_history_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, add_write_ids_to_min_history_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()) {
+          optionals.set(0);
+        }
+        if (struct.isSetWriteIds()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetTxnId()) {
+          oprot.writeI64(struct.txnId);
+        }
+        if (struct.isSetWriteIds()) {
+          {
+            oprot.writeI32(struct.writeIds.size());
+            for (java.util.Map.Entry<java.lang.String, java.lang.Long> _iter2033 : struct.writeIds.entrySet())
+            {
+              oprot.writeString(_iter2033.getKey());
+              oprot.writeI64(_iter2033.getValue());
+            }
+          }
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, add_write_ids_to_min_history_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(2);
+        if (incoming.get(0)) {
+          struct.txnId = iprot.readI64();
+          struct.setTxnIdIsSet(true);
+        }
+        if (incoming.get(1)) {
+          {
+            org.apache.thrift.protocol.TMap _map2034 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64); 
+            struct.writeIds = new java.util.HashMap<java.lang.String,java.lang.Long>(2*_map2034.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _key2035;
+            long _val2036;
+            for (int _i2037 = 0; _i2037 < _map2034.size; ++_i2037)
+            {
+              _key2035 = iprot.readString();
+              _val2036 = iprot.readI64();
+              struct.writeIds.put(_key2035, _val2036);
+            }
+          }
+          struct.setWriteIdsIsSet(true);
+        }
+      }
+    }
+
+    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+    }
+  }
+
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_write_ids_to_min_history_result implements org.apache.thrift.TBase<add_write_ids_to_min_history_result, add_write_ids_to_min_history_result._Fields>, java.io.Serializable, Cloneable, Comparable<add_write_ids_to_min_history_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("add_write_ids_to_min_history_result");
+
+    private static final org.apache.thrift.protocol.TField O2_FIELD_DESC = new org.apache.thrift.protocol.TField("o2", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new add_write_ids_to_min_history_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new add_write_ids_to_min_history_resultTupleSchemeFactory();
+
+    private @org.apache.thrift.annotation.Nullable MetaException o2; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      O2((short)1, "o2");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // O2
+            return O2;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByName(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.O2, new org.apache.thrift.meta_data.FieldMetaData("o2", 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(add_write_ids_to_min_history_result.class, metaDataMap);
+    }
+
+    public add_write_ids_to_min_history_result() {
+    }
+
+    public add_write_ids_to_min_history_result(
+      MetaException o2)
+    {
+      this();
+      this.o2 = o2;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public add_write_ids_to_min_history_result(add_write_ids_to_min_history_result other) {
+      if (other.isSetO2()) {
+        this.o2 = new MetaException(other.o2);
+      }
+    }
+
+    public add_write_ids_to_min_history_result deepCopy() {
+      return new add_write_ids_to_min_history_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.o2 = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public MetaException getO2() {
+      return this.o2;
+    }
+
+    public void setO2(@org.apache.thrift.annotation.Nullable MetaException o2) {
+      this.o2 = o2;
+    }
+
+    public void unsetO2() {
+      this.o2 = null;
+    }
+
+    /** Returns true if field o2 is set (has been assigned a value) and false otherwise */
+    public boolean isSetO2() {
+      return this.o2 != null;
+    }
+
+    public void setO2IsSet(boolean value) {
+      if (!value) {
+        this.o2 = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case O2:
+        if (value == null) {
+          unsetO2();
+        } else {
+          setO2((MetaException)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case O2:
+        return getO2();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case O2:
+        return isSetO2();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof add_write_ids_to_min_history_result)
+        return this.equals((add_write_ids_to_min_history_result)that);
+      return false;
+    }
+
+    public boolean equals(add_write_ids_to_min_history_result that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_o2 = true && this.isSetO2();
+      boolean that_present_o2 = true && that.isSetO2();
+      if (this_present_o2 || that_present_o2) {
+        if (!(this_present_o2 && that_present_o2))
+          return false;
+        if (!this.o2.equals(that.o2))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetO2()) ? 131071 : 524287);
+      if (isSetO2())
+        hashCode = hashCode * 8191 + o2.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(add_write_ids_to_min_history_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.compare(isSetO2(), other.isSetO2());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetO2()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o2, other.o2);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      scheme(iprot).read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      scheme(oprot).write(oprot, this);
+      }
+
+    @Override
+    public java.lang.String toString() {
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("add_write_ids_to_min_history_result(");
+      boolean first = true;
+
+      sb.append("o2:");
+      if (this.o2 == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.o2);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class add_write_ids_to_min_history_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public add_write_ids_to_min_history_resultStandardScheme getScheme() {
+        return new add_write_ids_to_min_history_resultStandardScheme();
+      }
+    }
+
+    private static class add_write_ids_to_min_history_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<add_write_ids_to_min_history_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, add_write_ids_to_min_history_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // O2
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.o2 = new MetaException();
+                struct.o2.read(iprot);
+                struct.setO2IsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, add_write_ids_to_min_history_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.o2 != null) {
+          oprot.writeFieldBegin(O2_FIELD_DESC);
+          struct.o2.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class add_write_ids_to_min_history_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public add_write_ids_to_min_history_resultTupleScheme getScheme() {
+        return new add_write_ids_to_min_history_resultTupleScheme();
+      }
+    }
+
+    private static class add_write_ids_to_min_history_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<add_write_ids_to_min_history_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, add_write_ids_to_min_history_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.isSetO2()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetO2()) {
+          struct.o2.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, add_write_ids_to_min_history_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(1);
+        if (incoming.get(0)) {
+          struct.o2 = new MetaException();
+          struct.o2.read(iprot);
+          struct.setO2IsSet(true);
+        }
+      }
+    }
+
+    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+    }
+  }
+
   @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class allocate_table_write_ids_args implements org.apache.thrift.TBase<allocate_table_write_ids_args, allocate_table_write_ids_args._Fields>, java.io.Serializable, Cloneable, Comparable<allocate_table_write_ids_args>   {
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("allocate_table_write_ids_args");
 
@@ -245574,13 +246609,13 @@ package org.apache.hadoop.hive.metastore.api;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list2028 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<java.lang.String>(_list2028.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem2029;
-                  for (int _i2030 = 0; _i2030 < _list2028.size; ++_i2030)
+                  org.apache.thrift.protocol.TList _list2038 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<java.lang.String>(_list2038.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem2039;
+                  for (int _i2040 = 0; _i2040 < _list2038.size; ++_i2040)
                   {
-                    _elem2029 = iprot.readString();
-                    struct.success.add(_elem2029);
+                    _elem2039 = iprot.readString();
+                    struct.success.add(_elem2039);
                   }
                   iprot.readListEnd();
                 }
@@ -245606,9 +246641,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (java.lang.String _iter2031 : struct.success)
+            for (java.lang.String _iter2041 : struct.success)
             {
-              oprot.writeString(_iter2031);
+              oprot.writeString(_iter2041);
             }
             oprot.writeListEnd();
           }
@@ -245639,9 +246674,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (java.lang.String _iter2032 : struct.success)
+            for (java.lang.String _iter2042 : struct.success)
             {
-              oprot.writeString(_iter2032);
+              oprot.writeString(_iter2042);
             }
           }
         }
@@ -245653,13 +246688,13 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list2033 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.success = new java.util.ArrayList<java.lang.String>(_list2033.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem2034;
-            for (int _i2035 = 0; _i2035 < _list2033.size; ++_i2035)
+            org.apache.thrift.protocol.TList _list2043 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.success = new java.util.ArrayList<java.lang.String>(_list2043.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem2044;
+            for (int _i2045 = 0; _i2045 < _list2043.size; ++_i2045)
             {
-              _elem2034 = iprot.readString();
-              struct.success.add(_elem2034);
+              _elem2044 = iprot.readString();
+              struct.success.add(_elem2044);
             }
           }
           struct.setSuccessIsSet(true);
@@ -286641,14 +287676,14 @@ package org.apache.hadoop.hive.metastore.api;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list2036 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<SchemaVersion>(_list2036.size);
-                  @org.apache.thrift.annotation.Nullable SchemaVersion _elem2037;
-                  for (int _i2038 = 0; _i2038 < _list2036.size; ++_i2038)
+                  org.apache.thrift.protocol.TList _list2046 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<SchemaVersion>(_list2046.size);
+                  @org.apache.thrift.annotation.Nullable SchemaVersion _elem2047;
+                  for (int _i2048 = 0; _i2048 < _list2046.size; ++_i2048)
                   {
-                    _elem2037 = new SchemaVersion();
-                    _elem2037.read(iprot);
-                    struct.success.add(_elem2037);
+                    _elem2047 = new SchemaVersion();
+                    _elem2047.read(iprot);
+                    struct.success.add(_elem2047);
                   }
                   iprot.readListEnd();
                 }
@@ -286692,9 +287727,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (SchemaVersion _iter2039 : struct.success)
+            for (SchemaVersion _iter2049 : struct.success)
             {
-              _iter2039.write(oprot);
+              _iter2049.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -286741,9 +287776,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (SchemaVersion _iter2040 : struct.success)
+            for (SchemaVersion _iter2050 : struct.success)
             {
-              _iter2040.write(oprot);
+              _iter2050.write(oprot);
             }
           }
         }
@@ -286761,14 +287796,14 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list2041 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.success = new java.util.ArrayList<SchemaVersion>(_list2041.size);
-            @org.apache.thrift.annotation.Nullable SchemaVersion _elem2042;
-            for (int _i2043 = 0; _i2043 < _list2041.size; ++_i2043)
+            org.apache.thrift.protocol.TList _list2051 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.success = new java.util.ArrayList<SchemaVersion>(_list2051.size);
+            @org.apache.thrift.annotation.Nullable SchemaVersion _elem2052;
+            for (int _i2053 = 0; _i2053 < _list2051.size; ++_i2053)
             {
-              _elem2042 = new SchemaVersion();
-              _elem2042.read(iprot);
-              struct.success.add(_elem2042);
+              _elem2052 = new SchemaVersion();
+              _elem2052.read(iprot);
+              struct.success.add(_elem2052);
             }
           }
           struct.setSuccessIsSet(true);
@@ -295383,14 +296418,14 @@ package org.apache.hadoop.hive.metastore.api;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list2044 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<RuntimeStat>(_list2044.size);
-                  @org.apache.thrift.annotation.Nullable RuntimeStat _elem2045;
-                  for (int _i2046 = 0; _i2046 < _list2044.size; ++_i2046)
+                  org.apache.thrift.protocol.TList _list2054 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<RuntimeStat>(_list2054.size);
+                  @org.apache.thrift.annotation.Nullable RuntimeStat _elem2055;
+                  for (int _i2056 = 0; _i2056 < _list2054.size; ++_i2056)
                   {
-                    _elem2045 = new RuntimeStat();
-                    _elem2045.read(iprot);
-                    struct.success.add(_elem2045);
+                    _elem2055 = new RuntimeStat();
+                    _elem2055.read(iprot);
+                    struct.success.add(_elem2055);
                   }
                   iprot.readListEnd();
                 }
@@ -295425,9 +296460,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (RuntimeStat _iter2047 : struct.success)
+            for (RuntimeStat _iter2057 : struct.success)
             {
-              _iter2047.write(oprot);
+              _iter2057.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -295466,9 +296501,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (RuntimeStat _iter2048 : struct.success)
+            for (RuntimeStat _iter2058 : struct.success)
             {
-              _iter2048.write(oprot);
+              _iter2058.write(oprot);
             }
           }
         }
@@ -295483,14 +296518,14 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list2049 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.success = new java.util.ArrayList<RuntimeStat>(_list2049.size);
-            @org.apache.thrift.annotation.Nullable RuntimeStat _elem2050;
-            for (int _i2051 = 0; _i2051 < _list2049.size; ++_i2051)
+            org.apache.thrift.protocol.TList _list2059 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.success = new java.util.ArrayList<RuntimeStat>(_list2059.size);
+            @org.apache.thrift.annotation.Nullable RuntimeStat _elem2060;
+            for (int _i2061 = 0; _i2061 < _list2059.size; ++_i2061)
             {
-              _elem2050 = new RuntimeStat();
-              _elem2050.read(iprot);
-              struct.success.add(_elem2050);
+              _elem2060 = new RuntimeStat();
+              _elem2060.read(iprot);
+              struct.success.add(_elem2060);
             }
           }
           struct.setSuccessIsSet(true);
@@ -305601,13 +306636,13 @@ package org.apache.hadoop.hive.metastore.api;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list2052 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<java.lang.String>(_list2052.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem2053;
-                  for (int _i2054 = 0; _i2054 < _list2052.size; ++_i2054)
+                  org.apache.thrift.protocol.TList _list2062 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<java.lang.String>(_list2062.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem2063;
+                  for (int _i2064 = 0; _i2064 < _list2062.size; ++_i2064)
                   {
-                    _elem2053 = iprot.readString();
-                    struct.success.add(_elem2053);
+                    _elem2063 = iprot.readString();
+                    struct.success.add(_elem2063);
                   }
                   iprot.readListEnd();
                 }
@@ -305642,9 +306677,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (java.lang.String _iter2055 : struct.success)
+            for (java.lang.String _iter2065 : struct.success)
             {
-              oprot.writeString(_iter2055);
+              oprot.writeString(_iter2065);
             }
             oprot.writeListEnd();
           }
@@ -305683,9 +306718,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (java.lang.String _iter2056 : struct.success)
+            for (java.lang.String _iter2066 : struct.success)
             {
-              oprot.writeString(_iter2056);
+              oprot.writeString(_iter2066);
             }
           }
         }
@@ -305700,13 +306735,13 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list2057 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.success = new java.util.ArrayList<java.lang.String>(_list2057.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem2058;
-            for (int _i2059 = 0; _i2059 < _list2057.size; ++_i2059)
+            org.apache.thrift.protocol.TList _list2067 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.success = new java.util.ArrayList<java.lang.String>(_list2067.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem2068;
+            for (int _i2069 = 0; _i2069 < _list2067.size; ++_i2069)
             {
-              _elem2058 = iprot.readString();
-              struct.success.add(_elem2058);
+              _elem2068 = iprot.readString();
+              struct.success.add(_elem2068);
             }
           }
           struct.setSuccessIsSet(true);
@@ -308163,13 +309198,13 @@ package org.apache.hadoop.hive.metastore.api;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list2060 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<java.lang.String>(_list2060.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem2061;
-                  for (int _i2062 = 0; _i2062 < _list2060.size; ++_i2062)
+                  org.apache.thrift.protocol.TList _list2070 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<java.lang.String>(_list2070.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem2071;
+                  for (int _i2072 = 0; _i2072 < _list2070.size; ++_i2072)
                   {
-                    _elem2061 = iprot.readString();
-                    struct.success.add(_elem2061);
+                    _elem2071 = iprot.readString();
+                    struct.success.add(_elem2071);
                   }
                   iprot.readListEnd();
                 }
@@ -308204,9 +309239,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (java.lang.String _iter2063 : struct.success)
+            for (java.lang.String _iter2073 : struct.success)
             {
-              oprot.writeString(_iter2063);
+              oprot.writeString(_iter2073);
             }
             oprot.writeListEnd();
           }
@@ -308245,9 +309280,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (java.lang.String _iter2064 : struct.success)
+            for (java.lang.String _iter2074 : struct.success)
             {
-              oprot.writeString(_iter2064);
+              oprot.writeString(_iter2074);
             }
           }
         }
@@ -308262,13 +309297,13 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list2065 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.success = new java.util.ArrayList<java.lang.String>(_list2065.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem2066;
-            for (int _i2067 = 0; _i2067 < _list2065.size; ++_i2067)
+            org.apache.thrift.protocol.TList _list2075 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.success = new java.util.ArrayList<java.lang.String>(_list2075.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem2076;
+            for (int _i2077 = 0; _i2077 < _list2075.size; ++_i2077)
             {
-              _elem2066 = iprot.readString();
-              struct.success.add(_elem2066);
+              _elem2076 = iprot.readString();
+              struct.success.add(_elem2076);
             }
           }
           struct.setSuccessIsSet(true);
@@ -309782,14 +310817,14 @@ package org.apache.hadoop.hive.metastore.api;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list2068 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<WriteEventInfo>(_list2068.size);
-                  @org.apache.thrift.annotation.Nullable WriteEventInfo _elem2069;
-                  for (int _i2070 = 0; _i2070 < _list2068.size; ++_i2070)
+                  org.apache.thrift.protocol.TList _list2078 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<WriteEventInfo>(_list2078.size);
+                  @org.apache.thrift.annotation.Nullable WriteEventInfo _elem2079;
+                  for (int _i2080 = 0; _i2080 < _list2078.size; ++_i2080)
                   {
-                    _elem2069 = new WriteEventInfo();
-                    _elem2069.read(iprot);
-                    struct.success.add(_elem2069);
+                    _elem2079 = new WriteEventInfo();
+                    _elem2079.read(iprot);
+                    struct.success.add(_elem2079);
                   }
                   iprot.readListEnd();
                 }
@@ -309824,9 +310859,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (WriteEventInfo _iter2071 : struct.success)
+            for (WriteEventInfo _iter2081 : struct.success)
             {
-              _iter2071.write(oprot);
+              _iter2081.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -309865,9 +310900,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (WriteEventInfo _iter2072 : struct.success)
+            for (WriteEventInfo _iter2082 : struct.success)
             {
-              _iter2072.write(oprot);
+              _iter2082.write(oprot);
             }
           }
         }
@@ -309882,14 +310917,14 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list2073 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.success = new java.util.ArrayList<WriteEventInfo>(_list2073.size);
-            @org.apache.thrift.annotation.Nullable WriteEventInfo _elem2074;
-            for (int _i2075 = 0; _i2075 < _list2073.size; ++_i2075)
+            org.apache.thrift.protocol.TList _list2083 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.success = new java.util.ArrayList<WriteEventInfo>(_list2083.size);
+            @org.apache.thrift.annotation.Nullable WriteEventInfo _elem2084;
+            for (int _i2085 = 0; _i2085 < _list2083.size; ++_i2085)
             {
-              _elem2074 = new WriteEventInfo();
-              _elem2074.read(iprot);
-              struct.success.add(_elem2074);
+              _elem2084 = new WriteEventInfo();
+              _elem2084.read(iprot);
+              struct.success.add(_elem2084);
             }
           }
           struct.setSuccessIsSet(true);
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/LockRequest.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/LockRequest.php
index ffc31b11ca4..24b29bd929a 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/LockRequest.php
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/LockRequest.php
@@ -61,6 +61,11 @@ class LockRequest
             'isRequired' => false,
             'type' => TType::BOOL,
         ),
+        8 => array(
+            'var' => 'locklessReadsEnabled',
+            'isRequired' => false,
+            'type' => TType::BOOL,
+        ),
     );
 
     /**
@@ -91,6 +96,10 @@ class LockRequest
      * @var bool
      */
     public $exclusiveCTAS = false;
+    /**
+     * @var bool
+     */
+    public $locklessReadsEnabled = false;
 
     public function __construct($vals = null)
     {
@@ -116,6 +125,9 @@ class LockRequest
             if (isset($vals['exclusiveCTAS'])) {
                 $this->exclusiveCTAS = $vals['exclusiveCTAS'];
             }
+            if (isset($vals['locklessReadsEnabled'])) {
+                $this->locklessReadsEnabled = $vals['locklessReadsEnabled'];
+            }
         }
     }
 
@@ -197,6 +209,13 @@ class LockRequest
                         $xfer += $input->skip($ftype);
                     }
                     break;
+                case 8:
+                    if ($ftype == TType::BOOL) {
+                        $xfer += $input->readBool($this->locklessReadsEnabled);
+                    } else {
+                        $xfer += $input->skip($ftype);
+                    }
+                    break;
                 default:
                     $xfer += $input->skip($ftype);
                     break;
@@ -253,6 +272,11 @@ class LockRequest
             $xfer += $output->writeBool($this->exclusiveCTAS);
             $xfer += $output->writeFieldEnd();
         }
+        if ($this->locklessReadsEnabled !== null) {
+            $xfer += $output->writeFieldBegin('locklessReadsEnabled', TType::BOOL, 8);
+            $xfer += $output->writeBool($this->locklessReadsEnabled);
+            $xfer += $output->writeFieldEnd();
+        }
         $xfer += $output->writeFieldStop();
         $xfer += $output->writeStructEnd();
         return $xfer;
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 bfe22501bb1..9fa6942be90 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
@@ -11690,6 +11690,66 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
         throw new \Exception("get_valid_write_ids failed: unknown result");
     }
 
+    public function add_write_ids_to_min_history($txnId, array $writeIds)
+    {
+        $this->send_add_write_ids_to_min_history($txnId, $writeIds);
+        $this->recv_add_write_ids_to_min_history();
+    }
+
+    public function send_add_write_ids_to_min_history($txnId, array $writeIds)
+    {
+        $args = new \metastore\ThriftHiveMetastore_add_write_ids_to_min_history_args();
+        $args->txnId = $txnId;
+        $args->writeIds = $writeIds;
+        $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+        if ($bin_accel) {
+            thrift_protocol_write_binary(
+                $this->output_,
+                'add_write_ids_to_min_history',
+                TMessageType::CALL,
+                $args,
+                $this->seqid_,
+                $this->output_->isStrictWrite()
+            );
+        } else {
+            $this->output_->writeMessageBegin('add_write_ids_to_min_history', TMessageType::CALL, $this->seqid_);
+            $args->write($this->output_);
+            $this->output_->writeMessageEnd();
+            $this->output_->getTransport()->flush();
+        }
+    }
+
+    public function recv_add_write_ids_to_min_history()
+    {
+        $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+        if ($bin_accel) {
+            $result = thrift_protocol_read_binary(
+                $this->input_,
+                '\metastore\ThriftHiveMetastore_add_write_ids_to_min_history_result',
+                $this->input_->isStrictRead()
+            );
+        } else {
+            $rseqid = 0;
+            $fname = null;
+            $mtype = 0;
+
+            $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+            if ($mtype == TMessageType::EXCEPTION) {
+                $x = new TApplicationException();
+                $x->read($this->input_);
+                $this->input_->readMessageEnd();
+                throw $x;
+            }
+            $result = new \metastore\ThriftHiveMetastore_add_write_ids_to_min_history_result();
+            $result->read($this->input_);
+            $this->input_->readMessageEnd();
+        }
+        if ($result->o2 !== null) {
+            throw $result->o2;
+        }
+        return;
+    }
+
     public function allocate_table_write_ids(\metastore\AllocateTableWriteIdsRequest $rqst)
     {
         $this->send_allocate_table_write_ids($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 84e798249c0..e9bfcd476b4 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
@@ -1395,6 +1395,12 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf
      * @throws \metastore\MetaException
      */
     public function get_valid_write_ids(\metastore\GetValidWriteIdsRequest $rqst);
+    /**
+     * @param int $txnId
+     * @param array $writeIds
+     * @throws \metastore\MetaException
+     */
+    public function add_write_ids_to_min_history($txnId, array $writeIds);
     /**
      * @param \metastore\AllocateTableWriteIdsRequest $rqst
      * @return \metastore\AllocateTableWriteIdsResponse
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_packages_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_write_ids_to_min_history_args.php
similarity index 50%
copy from standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_packages_result.php
copy to standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_write_ids_to_min_history_args.php
index d2881abd070..150a657331a 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_packages_result.php
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_write_ids_to_min_history_args.php
@@ -16,52 +16,55 @@ use Thrift\Protocol\TProtocol;
 use Thrift\Protocol\TBinaryProtocolAccelerated;
 use Thrift\Exception\TApplicationException;
 
-class ThriftHiveMetastore_get_all_packages_result
+class ThriftHiveMetastore_add_write_ids_to_min_history_args
 {
     static public $isValidate = false;
 
     static public $_TSPEC = array(
-        0 => array(
-            'var' => 'success',
+        1 => array(
+            'var' => 'txnId',
             'isRequired' => false,
-            'type' => TType::LST,
-            'etype' => TType::STRING,
-            'elem' => array(
-                'type' => TType::STRING,
-                ),
+            'type' => TType::I64,
         ),
-        1 => array(
-            'var' => 'o1',
+        2 => array(
+            'var' => 'writeIds',
             'isRequired' => false,
-            'type' => TType::STRUCT,
-            'class' => '\metastore\MetaException',
+            'type' => TType::MAP,
+            'ktype' => TType::STRING,
+            'vtype' => TType::I64,
+            'key' => array(
+                'type' => TType::STRING,
+            ),
+            'val' => array(
+                'type' => TType::I64,
+                ),
         ),
     );
 
     /**
-     * @var string[]
+     * @var int
      */
-    public $success = null;
+    public $txnId = null;
     /**
-     * @var \metastore\MetaException
+     * @var array
      */
-    public $o1 = null;
+    public $writeIds = null;
 
     public function __construct($vals = null)
     {
         if (is_array($vals)) {
-            if (isset($vals['success'])) {
-                $this->success = $vals['success'];
+            if (isset($vals['txnId'])) {
+                $this->txnId = $vals['txnId'];
             }
-            if (isset($vals['o1'])) {
-                $this->o1 = $vals['o1'];
+            if (isset($vals['writeIds'])) {
+                $this->writeIds = $vals['writeIds'];
             }
         }
     }
 
     public function getName()
     {
-        return 'ThriftHiveMetastore_get_all_packages_result';
+        return 'ThriftHiveMetastore_add_write_ids_to_min_history_args';
     }
 
 
@@ -78,26 +81,28 @@ class ThriftHiveMetastore_get_all_packages_result
                 break;
             }
             switch ($fid) {
-                case 0:
-                    if ($ftype == TType::LST) {
-                        $this->success = array();
-                        $_size1812 = 0;
-                        $_etype1815 = 0;
-                        $xfer += $input->readListBegin($_etype1815, $_size1812);
-                        for ($_i1816 = 0; $_i1816 < $_size1812; ++$_i1816) {
-                            $elem1817 = null;
-                            $xfer += $input->readString($elem1817);
-                            $this->success []= $elem1817;
-                        }
-                        $xfer += $input->readListEnd();
+                case 1:
+                    if ($ftype == TType::I64) {
+                        $xfer += $input->readI64($this->txnId);
                     } else {
                         $xfer += $input->skip($ftype);
                     }
                     break;
-                case 1:
-                    if ($ftype == TType::STRUCT) {
-                        $this->o1 = new \metastore\MetaException();
-                        $xfer += $this->o1->read($input);
+                case 2:
+                    if ($ftype == TType::MAP) {
+                        $this->writeIds = array();
+                        $_size1784 = 0;
+                        $_ktype1785 = 0;
+                        $_vtype1786 = 0;
+                        $xfer += $input->readMapBegin($_ktype1785, $_vtype1786, $_size1784);
+                        for ($_i1788 = 0; $_i1788 < $_size1784; ++$_i1788) {
+                            $key1789 = '';
+                            $val1790 = 0;
+                            $xfer += $input->readString($key1789);
+                            $xfer += $input->readI64($val1790);
+                            $this->writeIds[$key1789] = $val1790;
+                        }
+                        $xfer += $input->readMapEnd();
                     } else {
                         $xfer += $input->skip($ftype);
                     }
@@ -115,22 +120,23 @@ class ThriftHiveMetastore_get_all_packages_result
     public function write($output)
     {
         $xfer = 0;
-        $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_all_packages_result');
-        if ($this->success !== null) {
-            if (!is_array($this->success)) {
+        $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_write_ids_to_min_history_args');
+        if ($this->txnId !== null) {
+            $xfer += $output->writeFieldBegin('txnId', TType::I64, 1);
+            $xfer += $output->writeI64($this->txnId);
+            $xfer += $output->writeFieldEnd();
+        }
+        if ($this->writeIds !== null) {
+            if (!is_array($this->writeIds)) {
                 throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
             }
-            $xfer += $output->writeFieldBegin('success', TType::LST, 0);
-            $output->writeListBegin(TType::STRING, count($this->success));
-            foreach ($this->success as $iter1818) {
-                $xfer += $output->writeString($iter1818);
+            $xfer += $output->writeFieldBegin('writeIds', TType::MAP, 2);
+            $output->writeMapBegin(TType::STRING, TType::I64, count($this->writeIds));
+            foreach ($this->writeIds as $kiter1791 => $viter1792) {
+                $xfer += $output->writeString($kiter1791);
+                $xfer += $output->writeI64($viter1792);
             }
-            $output->writeListEnd();
-            $xfer += $output->writeFieldEnd();
-        }
-        if ($this->o1 !== null) {
-            $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
-            $xfer += $this->o1->write($output);
+            $output->writeMapEnd();
             $xfer += $output->writeFieldEnd();
         }
         $xfer += $output->writeFieldStop();
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_find_columns_with_stats_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_write_ids_to_min_history_result.php
similarity index 52%
copy from standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_find_columns_with_stats_result.php
copy to standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_write_ids_to_min_history_result.php
index 01ffe2c71b8..1368fc94462 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_find_columns_with_stats_result.php
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_add_write_ids_to_min_history_result.php
@@ -16,39 +16,36 @@ use Thrift\Protocol\TProtocol;
 use Thrift\Protocol\TBinaryProtocolAccelerated;
 use Thrift\Exception\TApplicationException;
 
-class ThriftHiveMetastore_find_columns_with_stats_result
+class ThriftHiveMetastore_add_write_ids_to_min_history_result
 {
     static public $isValidate = false;
 
     static public $_TSPEC = array(
-        0 => array(
-            'var' => 'success',
+        1 => array(
+            'var' => 'o2',
             'isRequired' => false,
-            'type' => TType::LST,
-            'etype' => TType::STRING,
-            'elem' => array(
-                'type' => TType::STRING,
-                ),
+            'type' => TType::STRUCT,
+            'class' => '\metastore\MetaException',
         ),
     );
 
     /**
-     * @var string[]
+     * @var \metastore\MetaException
      */
-    public $success = null;
+    public $o2 = null;
 
     public function __construct($vals = null)
     {
         if (is_array($vals)) {
-            if (isset($vals['success'])) {
-                $this->success = $vals['success'];
+            if (isset($vals['o2'])) {
+                $this->o2 = $vals['o2'];
             }
         }
     }
 
     public function getName()
     {
-        return 'ThriftHiveMetastore_find_columns_with_stats_result';
+        return 'ThriftHiveMetastore_add_write_ids_to_min_history_result';
     }
 
 
@@ -65,18 +62,10 @@ class ThriftHiveMetastore_find_columns_with_stats_result
                 break;
             }
             switch ($fid) {
-                case 0:
-                    if ($ftype == TType::LST) {
-                        $this->success = array();
-                        $_size1784 = 0;
-                        $_etype1787 = 0;
-                        $xfer += $input->readListBegin($_etype1787, $_size1784);
-                        for ($_i1788 = 0; $_i1788 < $_size1784; ++$_i1788) {
-                            $elem1789 = null;
-                            $xfer += $input->readString($elem1789);
-                            $this->success []= $elem1789;
-                        }
-                        $xfer += $input->readListEnd();
+                case 1:
+                    if ($ftype == TType::STRUCT) {
+                        $this->o2 = new \metastore\MetaException();
+                        $xfer += $this->o2->read($input);
                     } else {
                         $xfer += $input->skip($ftype);
                     }
@@ -94,17 +83,10 @@ class ThriftHiveMetastore_find_columns_with_stats_result
     public function write($output)
     {
         $xfer = 0;
-        $xfer += $output->writeStructBegin('ThriftHiveMetastore_find_columns_with_stats_result');
-        if ($this->success !== null) {
-            if (!is_array($this->success)) {
-                throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
-            }
-            $xfer += $output->writeFieldBegin('success', TType::LST, 0);
-            $output->writeListBegin(TType::STRING, count($this->success));
-            foreach ($this->success as $iter1790) {
-                $xfer += $output->writeString($iter1790);
-            }
-            $output->writeListEnd();
+        $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_write_ids_to_min_history_result');
+        if ($this->o2 !== null) {
+            $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 1);
+            $xfer += $this->o2->write($output);
             $xfer += $output->writeFieldEnd();
         }
         $xfer += $output->writeFieldStop();
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_find_columns_with_stats_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_find_columns_with_stats_result.php
index 01ffe2c71b8..b2a297a6749 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_find_columns_with_stats_result.php
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_find_columns_with_stats_result.php
@@ -68,13 +68,13 @@ class ThriftHiveMetastore_find_columns_with_stats_result
                 case 0:
                     if ($ftype == TType::LST) {
                         $this->success = array();
-                        $_size1784 = 0;
-                        $_etype1787 = 0;
-                        $xfer += $input->readListBegin($_etype1787, $_size1784);
-                        for ($_i1788 = 0; $_i1788 < $_size1784; ++$_i1788) {
-                            $elem1789 = null;
-                            $xfer += $input->readString($elem1789);
-                            $this->success []= $elem1789;
+                        $_size1793 = 0;
+                        $_etype1796 = 0;
+                        $xfer += $input->readListBegin($_etype1796, $_size1793);
+                        for ($_i1797 = 0; $_i1797 < $_size1793; ++$_i1797) {
+                            $elem1798 = null;
+                            $xfer += $input->readString($elem1798);
+                            $this->success []= $elem1798;
                         }
                         $xfer += $input->readListEnd();
                     } else {
@@ -101,8 +101,8 @@ class ThriftHiveMetastore_find_columns_with_stats_result
             }
             $xfer += $output->writeFieldBegin('success', TType::LST, 0);
             $output->writeListBegin(TType::STRING, count($this->success));
-            foreach ($this->success as $iter1790) {
-                $xfer += $output->writeString($iter1790);
+            foreach ($this->success as $iter1799) {
+                $xfer += $output->writeString($iter1799);
             }
             $output->writeListEnd();
             $xfer += $output->writeFieldEnd();
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_packages_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_packages_result.php
index d2881abd070..fcd94e0fa34 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_packages_result.php
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_packages_result.php
@@ -81,13 +81,13 @@ class ThriftHiveMetastore_get_all_packages_result
                 case 0:
                     if ($ftype == TType::LST) {
                         $this->success = array();
-                        $_size1812 = 0;
-                        $_etype1815 = 0;
-                        $xfer += $input->readListBegin($_etype1815, $_size1812);
-                        for ($_i1816 = 0; $_i1816 < $_size1812; ++$_i1816) {
-                            $elem1817 = null;
-                            $xfer += $input->readString($elem1817);
-                            $this->success []= $elem1817;
+                        $_size1821 = 0;
+                        $_etype1824 = 0;
+                        $xfer += $input->readListBegin($_etype1824, $_size1821);
+                        for ($_i1825 = 0; $_i1825 < $_size1821; ++$_i1825) {
+                            $elem1826 = null;
+                            $xfer += $input->readString($elem1826);
+                            $this->success []= $elem1826;
                         }
                         $xfer += $input->readListEnd();
                     } else {
@@ -122,8 +122,8 @@ class ThriftHiveMetastore_get_all_packages_result
             }
             $xfer += $output->writeFieldBegin('success', TType::LST, 0);
             $output->writeListBegin(TType::STRING, count($this->success));
-            foreach ($this->success as $iter1818) {
-                $xfer += $output->writeString($iter1818);
+            foreach ($this->success as $iter1827) {
+                $xfer += $output->writeString($iter1827);
             }
             $output->writeListEnd();
             $xfer += $output->writeFieldEnd();
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_stored_procedures_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_stored_procedures_result.php
index 9be4c3c8f29..e94412ce11a 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_stored_procedures_result.php
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_stored_procedures_result.php
@@ -81,13 +81,13 @@ class ThriftHiveMetastore_get_all_stored_procedures_result
                 case 0:
                     if ($ftype == TType::LST) {
                         $this->success = array();
-                        $_size1805 = 0;
-                        $_etype1808 = 0;
-                        $xfer += $input->readListBegin($_etype1808, $_size1805);
-                        for ($_i1809 = 0; $_i1809 < $_size1805; ++$_i1809) {
-                            $elem1810 = null;
-                            $xfer += $input->readString($elem1810);
-                            $this->success []= $elem1810;
+                        $_size1814 = 0;
+                        $_etype1817 = 0;
+                        $xfer += $input->readListBegin($_etype1817, $_size1814);
+                        for ($_i1818 = 0; $_i1818 < $_size1814; ++$_i1818) {
+                            $elem1819 = null;
+                            $xfer += $input->readString($elem1819);
+                            $this->success []= $elem1819;
                         }
                         $xfer += $input->readListEnd();
                     } else {
@@ -122,8 +122,8 @@ class ThriftHiveMetastore_get_all_stored_procedures_result
             }
             $xfer += $output->writeFieldBegin('success', TType::LST, 0);
             $output->writeListBegin(TType::STRING, count($this->success));
-            foreach ($this->success as $iter1811) {
-                $xfer += $output->writeString($iter1811);
+            foreach ($this->success as $iter1820) {
+                $xfer += $output->writeString($iter1820);
             }
             $output->writeListEnd();
             $xfer += $output->writeFieldEnd();
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_write_event_info_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_write_event_info_result.php
index 21fb4c4d539..52c34503d80 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_write_event_info_result.php
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_all_write_event_info_result.php
@@ -82,14 +82,14 @@ class ThriftHiveMetastore_get_all_write_event_info_result
                 case 0:
                     if ($ftype == TType::LST) {
                         $this->success = array();
-                        $_size1819 = 0;
-                        $_etype1822 = 0;
-                        $xfer += $input->readListBegin($_etype1822, $_size1819);
-                        for ($_i1823 = 0; $_i1823 < $_size1819; ++$_i1823) {
-                            $elem1824 = null;
-                            $elem1824 = new \metastore\WriteEventInfo();
-                            $xfer += $elem1824->read($input);
-                            $this->success []= $elem1824;
+                        $_size1828 = 0;
+                        $_etype1831 = 0;
+                        $xfer += $input->readListBegin($_etype1831, $_size1828);
+                        for ($_i1832 = 0; $_i1832 < $_size1828; ++$_i1832) {
+                            $elem1833 = null;
+                            $elem1833 = new \metastore\WriteEventInfo();
+                            $xfer += $elem1833->read($input);
+                            $this->success []= $elem1833;
                         }
                         $xfer += $input->readListEnd();
                     } else {
@@ -124,8 +124,8 @@ class ThriftHiveMetastore_get_all_write_event_info_result
             }
             $xfer += $output->writeFieldBegin('success', TType::LST, 0);
             $output->writeListBegin(TType::STRUCT, count($this->success));
-            foreach ($this->success as $iter1825) {
-                $xfer += $iter1825->write($output);
+            foreach ($this->success as $iter1834) {
+                $xfer += $iter1834->write($output);
             }
             $output->writeListEnd();
             $xfer += $output->writeFieldEnd();
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_runtime_stats_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_runtime_stats_result.php
index 22101e55b5f..2f1893c70e5 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_runtime_stats_result.php
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_runtime_stats_result.php
@@ -82,14 +82,14 @@ class ThriftHiveMetastore_get_runtime_stats_result
                 case 0:
                     if ($ftype == TType::LST) {
                         $this->success = array();
-                        $_size1798 = 0;
-                        $_etype1801 = 0;
-                        $xfer += $input->readListBegin($_etype1801, $_size1798);
-                        for ($_i1802 = 0; $_i1802 < $_size1798; ++$_i1802) {
-                            $elem1803 = null;
-                            $elem1803 = new \metastore\RuntimeStat();
-                            $xfer += $elem1803->read($input);
-                            $this->success []= $elem1803;
+                        $_size1807 = 0;
+                        $_etype1810 = 0;
+                        $xfer += $input->readListBegin($_etype1810, $_size1807);
+                        for ($_i1811 = 0; $_i1811 < $_size1807; ++$_i1811) {
+                            $elem1812 = null;
+                            $elem1812 = new \metastore\RuntimeStat();
+                            $xfer += $elem1812->read($input);
+                            $this->success []= $elem1812;
                         }
                         $xfer += $input->readListEnd();
                     } else {
@@ -124,8 +124,8 @@ class ThriftHiveMetastore_get_runtime_stats_result
             }
             $xfer += $output->writeFieldBegin('success', TType::LST, 0);
             $output->writeListBegin(TType::STRUCT, count($this->success));
-            foreach ($this->success as $iter1804) {
-                $xfer += $iter1804->write($output);
+            foreach ($this->success as $iter1813) {
+                $xfer += $iter1813->write($output);
             }
             $output->writeListEnd();
             $xfer += $output->writeFieldEnd();
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_schema_all_versions_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_schema_all_versions_result.php
index 1ae1d11edf4..1181bfaba10 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_schema_all_versions_result.php
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_get_schema_all_versions_result.php
@@ -95,14 +95,14 @@ class ThriftHiveMetastore_get_schema_all_versions_result
                 case 0:
                     if ($ftype == TType::LST) {
                         $this->success = array();
-                        $_size1791 = 0;
-                        $_etype1794 = 0;
-                        $xfer += $input->readListBegin($_etype1794, $_size1791);
-                        for ($_i1795 = 0; $_i1795 < $_size1791; ++$_i1795) {
-                            $elem1796 = null;
-                            $elem1796 = new \metastore\SchemaVersion();
-                            $xfer += $elem1796->read($input);
-                            $this->success []= $elem1796;
+                        $_size1800 = 0;
+                        $_etype1803 = 0;
+                        $xfer += $input->readListBegin($_etype1803, $_size1800);
+                        for ($_i1804 = 0; $_i1804 < $_size1800; ++$_i1804) {
+                            $elem1805 = null;
+                            $elem1805 = new \metastore\SchemaVersion();
+                            $xfer += $elem1805->read($input);
+                            $this->success []= $elem1805;
                         }
                         $xfer += $input->readListEnd();
                     } else {
@@ -145,8 +145,8 @@ class ThriftHiveMetastore_get_schema_all_versions_result
             }
             $xfer += $output->writeFieldBegin('success', TType::LST, 0);
             $output->writeListBegin(TType::STRUCT, count($this->success));
-            foreach ($this->success as $iter1797) {
-                $xfer += $iter1797->write($output);
+            foreach ($this->success as $iter1806) {
+                $xfer += $iter1806->write($output);
             }
             $output->writeListEnd();
             $xfer += $output->writeFieldEnd();
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 62dd5fa9842..477b0a96c30 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
@@ -203,6 +203,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
     print('  i64 get_latest_txnid_in_conflict(i64 txnId)')
     print('  void repl_tbl_writeid_state(ReplTblWriteIdStateRequest rqst)')
     print('  GetValidWriteIdsResponse get_valid_write_ids(GetValidWriteIdsRequest rqst)')
+    print('  void add_write_ids_to_min_history(i64 txnId,  writeIds)')
     print('  AllocateTableWriteIdsResponse allocate_table_write_ids(AllocateTableWriteIdsRequest rqst)')
     print('  MaxAllocatedTableWriteIdResponse get_max_allocated_table_write_id(MaxAllocatedTableWriteIdRequest rqst)')
     print('  void seed_write_id(SeedTableWriteIdsRequest rqst)')
@@ -1462,6 +1463,12 @@ elif cmd == 'get_valid_write_ids':
         sys.exit(1)
     pp.pprint(client.get_valid_write_ids(eval(args[0]),))
 
+elif cmd == 'add_write_ids_to_min_history':
+    if len(args) != 2:
+        print('add_write_ids_to_min_history requires 2 args')
+        sys.exit(1)
+    pp.pprint(client.add_write_ids_to_min_history(eval(args[0]), eval(args[1]),))
+
 elif cmd == 'allocate_table_write_ids':
     if len(args) != 1:
         print('allocate_table_write_ids requires 1 args')
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 f8a2f3a8a1c..b77c76b9c1c 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
@@ -1582,6 +1582,15 @@ class Iface(fb303.FacebookService.Iface):
         """
         pass
 
+    def add_write_ids_to_min_history(self, txnId, writeIds):
+        """
+        Parameters:
+         - txnId
+         - writeIds
+
+        """
+        pass
+
     def allocate_table_write_ids(self, rqst):
         """
         Parameters:
@@ -8974,6 +8983,40 @@ class Client(fb303.FacebookService.Client, Iface):
             raise result.o2
         raise TApplicationException(TApplicationException.MISSING_RESULT, "get_valid_write_ids failed: unknown result")
 
+    def add_write_ids_to_min_history(self, txnId, writeIds):
+        """
+        Parameters:
+         - txnId
+         - writeIds
+
+        """
+        self.send_add_write_ids_to_min_history(txnId, writeIds)
+        self.recv_add_write_ids_to_min_history()
+
+    def send_add_write_ids_to_min_history(self, txnId, writeIds):
+        self._oprot.writeMessageBegin('add_write_ids_to_min_history', TMessageType.CALL, self._seqid)
+        args = add_write_ids_to_min_history_args()
+        args.txnId = txnId
+        args.writeIds = writeIds
+        args.write(self._oprot)
+        self._oprot.writeMessageEnd()
+        self._oprot.trans.flush()
+
+    def recv_add_write_ids_to_min_history(self):
+        iprot = self._iprot
+        (fname, mtype, rseqid) = iprot.readMessageBegin()
+        if mtype == TMessageType.EXCEPTION:
+            x = TApplicationException()
+            x.read(iprot)
+            iprot.readMessageEnd()
+            raise x
+        result = add_write_ids_to_min_history_result()
+        result.read(iprot)
+        iprot.readMessageEnd()
+        if result.o2 is not None:
+            raise result.o2
+        return
+
     def allocate_table_write_ids(self, rqst):
         """
         Parameters:
@@ -12341,6 +12384,7 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
         self._processMap["get_latest_txnid_in_conflict"] = Processor.process_get_latest_txnid_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["add_write_ids_to_min_history"] = Processor.process_add_write_ids_to_min_history
         self._processMap["allocate_table_write_ids"] = Processor.process_allocate_table_write_ids
         self._processMap["get_max_allocated_table_write_id"] = Processor.process_get_max_allocated_table_write_id
         self._processMap["seed_write_id"] = Processor.process_seed_write_id
@@ -17647,6 +17691,32 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
         oprot.writeMessageEnd()
         oprot.trans.flush()
 
+    def process_add_write_ids_to_min_history(self, seqid, iprot, oprot):
+        args = add_write_ids_to_min_history_args()
+        args.read(iprot)
+        iprot.readMessageEnd()
+        result = add_write_ids_to_min_history_result()
+        try:
+            self._handler.add_write_ids_to_min_history(args.txnId, args.writeIds)
+            msg_type = TMessageType.REPLY
+        except TTransport.TTransportException:
+            raise
+        except MetaException as o2:
+            msg_type = TMessageType.REPLY
+            result.o2 = o2
+        except TApplicationException as ex:
+            logging.exception('TApplication exception in handler')
+            msg_type = TMessageType.EXCEPTION
+            result = ex
+        except Exception:
+            logging.exception('Unexpected exception in handler')
+            msg_type = TMessageType.EXCEPTION
+            result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+        oprot.writeMessageBegin("add_write_ids_to_min_history", msg_type, seqid)
+        result.write(oprot)
+        oprot.writeMessageEnd()
+        oprot.trans.flush()
+
     def process_allocate_table_write_ids(self, seqid, iprot, oprot):
         args = allocate_table_write_ids_args()
         args.read(iprot)
@@ -48711,6 +48781,152 @@ get_valid_write_ids_result.thrift_spec = (
 )
 
 
+class add_write_ids_to_min_history_args(object):
+    """
+    Attributes:
+     - txnId
+     - writeIds
+
+    """
+
+
+    def __init__(self, txnId=None, writeIds=None,):
+        self.txnId = txnId
+        self.writeIds = writeIds
+
+    def read(self, iprot):
+        if iprot._fast_decode is not None and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None:
+            iprot._fast_decode(self, iprot, [self.__class__, self.thrift_spec])
+            return
+        iprot.readStructBegin()
+        while True:
+            (fname, ftype, fid) = iprot.readFieldBegin()
+            if ftype == TType.STOP:
+                break
+            if fid == 1:
+                if ftype == TType.I64:
+                    self.txnId = iprot.readI64()
+                else:
+                    iprot.skip(ftype)
+            elif fid == 2:
+                if ftype == TType.MAP:
+                    self.writeIds = {}
+                    (_ktype1782, _vtype1783, _size1781) = iprot.readMapBegin()
+                    for _i1785 in range(_size1781):
+                        _key1786 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString()
+                        _val1787 = iprot.readI64()
+                        self.writeIds[_key1786] = _val1787
+                    iprot.readMapEnd()
+                else:
+                    iprot.skip(ftype)
+            else:
+                iprot.skip(ftype)
+            iprot.readFieldEnd()
+        iprot.readStructEnd()
+
+    def write(self, oprot):
+        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('add_write_ids_to_min_history_args')
+        if self.txnId is not None:
+            oprot.writeFieldBegin('txnId', TType.I64, 1)
+            oprot.writeI64(self.txnId)
+            oprot.writeFieldEnd()
+        if self.writeIds is not None:
+            oprot.writeFieldBegin('writeIds', TType.MAP, 2)
+            oprot.writeMapBegin(TType.STRING, TType.I64, len(self.writeIds))
+            for kiter1788, viter1789 in self.writeIds.items():
+                oprot.writeString(kiter1788.encode('utf-8') if sys.version_info[0] == 2 else kiter1788)
+                oprot.writeI64(viter1789)
+            oprot.writeMapEnd()
+            oprot.writeFieldEnd()
+        oprot.writeFieldStop()
+        oprot.writeStructEnd()
+
+    def validate(self):
+        return
+
+    def __repr__(self):
+        L = ['%s=%r' % (key, value)
+             for key, value in self.__dict__.items()]
+        return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+    def __eq__(self, other):
+        return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+    def __ne__(self, other):
+        return not (self == other)
+all_structs.append(add_write_ids_to_min_history_args)
+add_write_ids_to_min_history_args.thrift_spec = (
+    None,  # 0
+    (1, TType.I64, 'txnId', None, None, ),  # 1
+    (2, TType.MAP, 'writeIds', (TType.STRING, 'UTF8', TType.I64, None, False), None, ),  # 2
+)
+
+
+class add_write_ids_to_min_history_result(object):
+    """
+    Attributes:
+     - o2
+
+    """
+
+
+    def __init__(self, o2=None,):
+        self.o2 = o2
+
+    def read(self, iprot):
+        if iprot._fast_decode is not None and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None:
+            iprot._fast_decode(self, iprot, [self.__class__, self.thrift_spec])
+            return
+        iprot.readStructBegin()
+        while True:
+            (fname, ftype, fid) = iprot.readFieldBegin()
+            if ftype == TType.STOP:
+                break
+            if fid == 1:
+                if ftype == TType.STRUCT:
+                    self.o2 = MetaException.read(iprot)
+                else:
+                    iprot.skip(ftype)
+            else:
+                iprot.skip(ftype)
+            iprot.readFieldEnd()
+        iprot.readStructEnd()
+
+    def write(self, oprot):
+        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('add_write_ids_to_min_history_result')
+        if self.o2 is not None:
+            oprot.writeFieldBegin('o2', TType.STRUCT, 1)
+            self.o2.write(oprot)
+            oprot.writeFieldEnd()
+        oprot.writeFieldStop()
+        oprot.writeStructEnd()
+
+    def validate(self):
+        return
+
+    def __repr__(self):
+        L = ['%s=%r' % (key, value)
+             for key, value in self.__dict__.items()]
+        return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+    def __eq__(self, other):
+        return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+    def __ne__(self, other):
+        return not (self == other)
+all_structs.append(add_write_ids_to_min_history_result)
+add_write_ids_to_min_history_result.thrift_spec = (
+    None,  # 0
+    (1, TType.STRUCT, 'o2', [MetaException, None], None, ),  # 1
+)
+
+
 class allocate_table_write_ids_args(object):
     """
     Attributes:
@@ -51235,10 +51451,10 @@ class find_columns_with_stats_result(object):
             if fid == 0:
                 if ftype == TType.LIST:
                     self.success = []
-                    (_etype1784, _size1781) = iprot.readListBegin()
-                    for _i1785 in range(_size1781):
-                        _elem1786 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString()
-                        self.success.append(_elem1786)
+                    (_etype1793, _size1790) = iprot.readListBegin()
+                    for _i1794 in range(_size1790):
+                        _elem1795 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString()
+                        self.success.append(_elem1795)
                     iprot.readListEnd()
                 else:
                     iprot.skip(ftype)
@@ -51255,8 +51471,8 @@ class find_columns_with_stats_result(object):
         if self.success is not None:
             oprot.writeFieldBegin('success', TType.LIST, 0)
             oprot.writeListBegin(TType.STRING, len(self.success))
-            for iter1787 in self.success:
-                oprot.writeString(iter1787.encode('utf-8') if sys.version_info[0] == 2 else iter1787)
+            for iter1796 in self.success:
+                oprot.writeString(iter1796.encode('utf-8') if sys.version_info[0] == 2 else iter1796)
             oprot.writeListEnd()
             oprot.writeFieldEnd()
         oprot.writeFieldStop()
@@ -57812,11 +58028,11 @@ class get_schema_all_versions_result(object):
             if fid == 0:
                 if ftype == TType.LIST:
                     self.success = []
-                    (_etype1791, _size1788) = iprot.readListBegin()
-                    for _i1792 in range(_size1788):
-                        _elem1793 = SchemaVersion()
-                        _elem1793.read(iprot)
-                        self.success.append(_elem1793)
+                    (_etype1800, _size1797) = iprot.readListBegin()
+                    for _i1801 in range(_size1797):
+                        _elem1802 = SchemaVersion()
+                        _elem1802.read(iprot)
+                        self.success.append(_elem1802)
                     iprot.readListEnd()
                 else:
                     iprot.skip(ftype)
@@ -57843,8 +58059,8 @@ class get_schema_all_versions_result(object):
         if self.success is not None:
             oprot.writeFieldBegin('success', TType.LIST, 0)
             oprot.writeListBegin(TType.STRUCT, len(self.success))
-            for iter1794 in self.success:
-                iter1794.write(oprot)
+            for iter1803 in self.success:
+                iter1803.write(oprot)
             oprot.writeListEnd()
             oprot.writeFieldEnd()
         if self.o1 is not None:
@@ -59233,11 +59449,11 @@ class get_runtime_stats_result(object):
             if fid == 0:
                 if ftype == TType.LIST:
                     self.success = []
-                    (_etype1798, _size1795) = iprot.readListBegin()
-                    for _i1799 in range(_size1795):
-                        _elem1800 = RuntimeStat()
-                        _elem1800.read(iprot)
-                        self.success.append(_elem1800)
+                    (_etype1807, _size1804) = iprot.readListBegin()
+                    for _i1808 in range(_size1804):
+                        _elem1809 = RuntimeStat()
+                        _elem1809.read(iprot)
+                        self.success.append(_elem1809)
                     iprot.readListEnd()
                 else:
                     iprot.skip(ftype)
@@ -59259,8 +59475,8 @@ class get_runtime_stats_result(object):
         if self.success is not None:
             oprot.writeFieldBegin('success', TType.LIST, 0)
             oprot.writeListBegin(TType.STRUCT, len(self.success))
-            for iter1801 in self.success:
-                iter1801.write(oprot)
+            for iter1810 in self.success:
+                iter1810.write(oprot)
             oprot.writeListEnd()
             oprot.writeFieldEnd()
         if self.o1 is not None:
@@ -60897,10 +61113,10 @@ class get_all_stored_procedures_result(object):
             if fid == 0:
                 if ftype == TType.LIST:
                     self.success = []
-                    (_etype1805, _size1802) = iprot.readListBegin()
-                    for _i1806 in range(_size1802):
-                        _elem1807 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString()
-                        self.success.append(_elem1807)
+                    (_etype1814, _size1811) = iprot.readListBegin()
+                    for _i1815 in range(_size1811):
+                        _elem1816 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString()
+                        self.success.append(_elem1816)
                     iprot.readListEnd()
                 else:
                     iprot.skip(ftype)
@@ -60922,8 +61138,8 @@ class get_all_stored_procedures_result(object):
         if self.success is not None:
             oprot.writeFieldBegin('success', TType.LIST, 0)
             oprot.writeListBegin(TType.STRING, len(self.success))
-            for iter1808 in self.success:
-                oprot.writeString(iter1808.encode('utf-8') if sys.version_info[0] == 2 else iter1808)
+            for iter1817 in self.success:
+                oprot.writeString(iter1817.encode('utf-8') if sys.version_info[0] == 2 else iter1817)
             oprot.writeListEnd()
             oprot.writeFieldEnd()
         if self.o1 is not None:
@@ -61315,10 +61531,10 @@ class get_all_packages_result(object):
             if fid == 0:
                 if ftype == TType.LIST:
                     self.success = []
-                    (_etype1812, _size1809) = iprot.readListBegin()
-                    for _i1813 in range(_size1809):
-                        _elem1814 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString()
-                        self.success.append(_elem1814)
+                    (_etype1821, _size1818) = iprot.readListBegin()
+                    for _i1822 in range(_size1818):
+                        _elem1823 = iprot.readString().decode('utf-8', errors='replace') if sys.version_info[0] == 2 else iprot.readString()
+                        self.success.append(_elem1823)
                     iprot.readListEnd()
                 else:
                     iprot.skip(ftype)
@@ -61340,8 +61556,8 @@ class get_all_packages_result(object):
         if self.success is not None:
             oprot.writeFieldBegin('success', TType.LIST, 0)
             oprot.writeListBegin(TType.STRING, len(self.success))
-            for iter1815 in self.success:
-                oprot.writeString(iter1815.encode('utf-8') if sys.version_info[0] == 2 else iter1815)
+            for iter1824 in self.success:
+                oprot.writeString(iter1824.encode('utf-8') if sys.version_info[0] == 2 else iter1824)
             oprot.writeListEnd()
             oprot.writeFieldEnd()
         if self.o1 is not None:
@@ -61584,11 +61800,11 @@ class get_all_write_event_info_result(object):
             if fid == 0:
                 if ftype == TType.LIST:
                     self.success = []
-                    (_etype1819, _size1816) = iprot.readListBegin()
-                    for _i1820 in range(_size1816):
-                        _elem1821 = WriteEventInfo()
-                        _elem1821.read(iprot)
-                        self.success.append(_elem1821)
+                    (_etype1828, _size1825) = iprot.readListBegin()
+                    for _i1829 in range(_size1825):
+                        _elem1830 = WriteEventInfo()
+                        _elem1830.read(iprot)
+                        self.success.append(_elem1830)
                     iprot.readListEnd()
                 else:
                     iprot.skip(ftype)
@@ -61610,8 +61826,8 @@ class get_all_write_event_info_result(object):
         if self.success is not None:
             oprot.writeFieldBegin('success', TType.LIST, 0)
             oprot.writeListBegin(TType.STRUCT, len(self.success))
-            for iter1822 in self.success:
-                iter1822.write(oprot)
+            for iter1831 in self.success:
+                iter1831.write(oprot)
             oprot.writeListEnd()
             oprot.writeFieldEnd()
         if self.o1 is not None:
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index 6744067c5b2..6f66497ca19 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -14349,11 +14349,12 @@ class LockRequest(object):
      - agentInfo
      - zeroWaitReadEnabled
      - exclusiveCTAS
+     - locklessReadsEnabled
 
     """
 
 
-    def __init__(self, component=None, txnid=None, user=None, hostname=None, agentInfo="Unknown", zeroWaitReadEnabled=False, exclusiveCTAS=False,):
+    def __init__(self, component=None, txnid=None, user=None, hostname=None, agentInfo="Unknown", zeroWaitReadEnabled=False, exclusiveCTAS=False, locklessReadsEnabled=False,):
         self.component = component
         self.txnid = txnid
         self.user = user
@@ -14361,6 +14362,7 @@ class LockRequest(object):
         self.agentInfo = agentInfo
         self.zeroWaitReadEnabled = zeroWaitReadEnabled
         self.exclusiveCTAS = exclusiveCTAS
+        self.locklessReadsEnabled = locklessReadsEnabled
 
     def read(self, iprot):
         if iprot._fast_decode is not None and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None:
@@ -14412,6 +14414,11 @@ class LockRequest(object):
                     self.exclusiveCTAS = iprot.readBool()
                 else:
                     iprot.skip(ftype)
+            elif fid == 8:
+                if ftype == TType.BOOL:
+                    self.locklessReadsEnabled = iprot.readBool()
+                else:
+                    iprot.skip(ftype)
             else:
                 iprot.skip(ftype)
             iprot.readFieldEnd()
@@ -14453,6 +14460,10 @@ class LockRequest(object):
             oprot.writeFieldBegin('exclusiveCTAS', TType.BOOL, 7)
             oprot.writeBool(self.exclusiveCTAS)
             oprot.writeFieldEnd()
+        if self.locklessReadsEnabled is not None:
+            oprot.writeFieldBegin('locklessReadsEnabled', TType.BOOL, 8)
+            oprot.writeBool(self.locklessReadsEnabled)
+            oprot.writeFieldEnd()
         oprot.writeFieldStop()
         oprot.writeStructEnd()
 
@@ -31570,6 +31581,7 @@ LockRequest.thrift_spec = (
     (5, TType.STRING, 'agentInfo', 'UTF8', "Unknown", ),  # 5
     (6, TType.BOOL, 'zeroWaitReadEnabled', None, False, ),  # 6
     (7, TType.BOOL, 'exclusiveCTAS', None, False, ),  # 7
+    (8, TType.BOOL, 'locklessReadsEnabled', None, False, ),  # 8
 )
 all_structs.append(LockResponse)
 LockResponse.thrift_spec = (
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb
index a16515e0019..98ba4f54dd5 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -4225,6 +4225,7 @@ class LockRequest
   AGENTINFO = 5
   ZEROWAITREADENABLED = 6
   EXCLUSIVECTAS = 7
+  LOCKLESSREADSENABLED = 8
 
   FIELDS = {
     COMPONENT => {:type => ::Thrift::Types::LIST, :name => 'component', :element => {:type => ::Thrift::Types::STRUCT, :class => ::LockComponent}},
@@ -4233,7 +4234,8 @@ class LockRequest
     HOSTNAME => {:type => ::Thrift::Types::STRING, :name => 'hostname'},
     AGENTINFO => {:type => ::Thrift::Types::STRING, :name => 'agentInfo', :default => %q"Unknown", :optional => true},
     ZEROWAITREADENABLED => {:type => ::Thrift::Types::BOOL, :name => 'zeroWaitReadEnabled', :default => false, :optional => true},
-    EXCLUSIVECTAS => {:type => ::Thrift::Types::BOOL, :name => 'exclusiveCTAS', :default => false, :optional => true}
+    EXCLUSIVECTAS => {:type => ::Thrift::Types::BOOL, :name => 'exclusiveCTAS', :default => false, :optional => true},
+    LOCKLESSREADSENABLED => {:type => ::Thrift::Types::BOOL, :name => 'locklessReadsEnabled', :default => false, :optional => true}
   }
 
   def struct_fields; FIELDS; end
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 2cee350a156..60fca527fe1 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
@@ -3010,6 +3010,21 @@ module ThriftHiveMetastore
       raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_valid_write_ids failed: unknown result')
     end
 
+    def add_write_ids_to_min_history(txnId, writeIds)
+      send_add_write_ids_to_min_history(txnId, writeIds)
+      recv_add_write_ids_to_min_history()
+    end
+
+    def send_add_write_ids_to_min_history(txnId, writeIds)
+      send_message('add_write_ids_to_min_history', Add_write_ids_to_min_history_args, :txnId => txnId, :writeIds => writeIds)
+    end
+
+    def recv_add_write_ids_to_min_history()
+      result = receive_message(Add_write_ids_to_min_history_result)
+      raise result.o2 unless result.o2.nil?
+      return
+    end
+
     def allocate_table_write_ids(rqst)
       send_allocate_table_write_ids(rqst)
       return recv_allocate_table_write_ids()
@@ -6818,6 +6833,17 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'get_valid_write_ids', seqid)
     end
 
+    def process_add_write_ids_to_min_history(seqid, iprot, oprot)
+      args = read_args(iprot, Add_write_ids_to_min_history_args)
+      result = Add_write_ids_to_min_history_result.new()
+      begin
+        @handler.add_write_ids_to_min_history(args.txnId, args.writeIds)
+      rescue ::MetaException => o2
+        result.o2 = o2
+      end
+      write_result(result, oprot, 'add_write_ids_to_min_history', seqid)
+    end
+
     def process_allocate_table_write_ids(seqid, iprot, oprot)
       args = read_args(iprot, Allocate_table_write_ids_args)
       result = Allocate_table_write_ids_result.new()
@@ -14599,6 +14625,40 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
+  class Add_write_ids_to_min_history_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    TXNID = 1
+    WRITEIDS = 2
+
+    FIELDS = {
+      TXNID => {:type => ::Thrift::Types::I64, :name => 'txnId'},
+      WRITEIDS => {:type => ::Thrift::Types::MAP, :name => 'writeIds', :key => {:type => ::Thrift::Types::STRING}, :value => {:type => ::Thrift::Types::I64}}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Add_write_ids_to_min_history_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    O2 = 1
+
+    FIELDS = {
+      O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
   class Allocate_table_write_ids_args
     include ::Thrift::Struct, ::Thrift::Struct_Union
     RQST = 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 fed5d54de46..50412dd17be 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
@@ -3947,6 +3947,11 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     return client.get_valid_write_ids(rqst);
   }
 
+  @Override
+  public void addWriteIdsToMinHistory(long txnId, Map<String, Long> writeIds) throws TException {
+    client.add_write_ids_to_min_history(txnId, writeIds);
+  }
+
   @Override
   public long openTxn(String user) throws TException {
     OpenTxnsResponse txns = openTxnsIntr(user, 1, null, null, null);
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 c71e2afc59b..44bd7e37dfa 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
@@ -3078,6 +3078,13 @@ public interface IMetaStoreClient {
   List<TableValidWriteIds> getValidWriteIds(List<String> tablesList, String validTxnList)
           throws TException;
 
+  /**
+   * Persists minOpenWriteId list to identify obsolete directories eligible for cleanup
+   * @param txnId transaction identifier
+   * @param writeIds list of minOpenWriteId
+   */
+  void addWriteIdsToMinHistory(long txnId, Map<String, Long> writeIds) throws TException;
+    
   /**
    * Initiate a transaction.
    * @param user User who is opening this transaction.  This is the Hive user,
@@ -3581,6 +3588,11 @@ public interface IMetaStoreClient {
    */
   void insertTable(Table table, boolean overwrite) throws MetaException;
 
+  /**
+   * Checks if there is a conflicting transaction
+   * @param txnId
+   * @return latest txnId in conflict
+   */
   long getLatestTxnIdInConflict(long txnId) throws TException;
 
   /**
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
index 4af26205dcb..0578e4b1c00 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
@@ -1544,7 +1544,10 @@ public class MetastoreConf {
     TXN_OPENTXN_TIMEOUT("metastore.txn.opentxn.timeout", "hive.txn.opentxn.timeout", 1000, TimeUnit.MILLISECONDS,
         "Time before an open transaction operation should persist, otherwise it is considered invalid and rolled back"),
     TXN_USE_MIN_HISTORY_LEVEL("metastore.txn.use.minhistorylevel", "hive.txn.use.minhistorylevel", true,
-        "Set this to false, for the TxnHandler and Cleaner to not use MinHistoryLevel table and take advantage of openTxn optimisation.\n"
+        "Set this to false, for the TxnHandler and Cleaner to not use MIN_HISTORY_LEVEL table and take advantage of openTxn optimisation.\n"
+            + "If the table is dropped HMS will switch this flag to false."),
+    TXN_USE_MIN_HISTORY_WRITE_ID("metastore.txn.use.minhistorywriteid", "hive.txn.use.minhistorywriteid", false,
+      "Set this to true, to avoid global minOpenTxn check in Cleaner.\n"
             + "If the table is dropped HMS will switch this flag to false."),
     LOCK_NUMRETRIES("metastore.lock.numretries", "hive.lock.numretries", 100,
         "The number of times you want to try to get all the locks"),
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 c14b12153c6..af20b469d48 100644
--- a/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
+++ b/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
@@ -1217,7 +1217,8 @@ struct LockRequest {
     4: required string hostname, // used in 'show locks' to help admins find who has open locks
     5: optional string agentInfo = "Unknown",
     6: optional bool zeroWaitReadEnabled = false,
-    7: optional bool exclusiveCTAS = false
+    7: optional bool exclusiveCTAS = false,
+    8: optional bool locklessReadsEnabled = false
 }
 
 struct LockResponse {
@@ -3003,6 +3004,7 @@ PartitionsResponse get_partitions_req(1:PartitionsRequest req)
   void repl_tbl_writeid_state(1: ReplTblWriteIdStateRequest rqst)
   GetValidWriteIdsResponse get_valid_write_ids(1:GetValidWriteIdsRequest rqst)
       throws (1:NoSuchTxnException o1, 2:MetaException o2)
+  void add_write_ids_to_min_history(1:i64 txnId, 2: map<string, i64> writeIds) throws (1:MetaException o2)
   AllocateTableWriteIdsResponse allocate_table_write_ids(1:AllocateTableWriteIdsRequest rqst)
     throws (1:NoSuchTxnException o1, 2:TxnAbortedException o2, 3:MetaException o3)
   MaxAllocatedTableWriteIdResponse get_max_allocated_table_write_id(1:MaxAllocatedTableWriteIdRequest rqst)
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java
index 0e38a5cf4be..4857634d1e4 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java
@@ -8790,6 +8790,11 @@ public class HMSHandler extends FacebookBase implements IHMSHandler {
     return getTxnHandler().getValidWriteIds(rqst);
   }
 
+  @Override
+  public void add_write_ids_to_min_history(long txnId, Map<String, Long> validWriteIds) throws TException {
+     getTxnHandler().addWriteIdsToMinHistory(txnId, validWriteIds);
+  }
+
   @Override
   public void set_hadoop_jobid(String jobId, long cqId) {
     getTxnHandler().setHadoopJobId(jobId, cqId);
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 a5c6b05a3df..46e88a3d6b0 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
@@ -60,6 +60,7 @@ public class CompactionInfo implements Comparable<CompactionInfo> {
   public boolean hasOldAbort = false;
   public long retryRetention = 0;
   public long nextTxnId = 0;
+  public long minOpenWriteId = -1;
   public long txnId = 0;
   public long commitTime = 0;
   public String poolName;
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 f6529a5895c..927833f8d48 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
@@ -392,29 +392,47 @@ class CompactionTxnHandler extends TxnHandler {
          * By filtering on minOpenTxnWaterMark, we will only cleanup after every transaction is committed, that could see
          * the uncompacted deltas. This way the cleaner can clean up everything that was made obsolete by this compaction.
          */
-        String whereClause = " WHERE \"CQ_STATE\" = '" + READY_FOR_CLEANING + "'";
-        if (minOpenTxnWaterMark > 0) {
+        String whereClause = " WHERE \"CQ_STATE\" = " + quoteChar(READY_FOR_CLEANING) + 
+          " AND (\"CQ_COMMIT_TIME\" < (" + getEpochFn(dbProduct) + " - \"CQ_RETRY_RETENTION\" - " + retentionTime + ") OR \"CQ_COMMIT_TIME\" IS NULL)";
+        
+        String queryStr = 
+          "SELECT \"CQ_ID\", \"cq1\".\"CQ_DATABASE\", \"cq1\".\"CQ_TABLE\", \"cq1\".\"CQ_PARTITION\"," + 
+          "  \"CQ_TYPE\", \"CQ_RUN_AS\", \"CQ_HIGHEST_WRITE_ID\", \"CQ_TBLPROPERTIES\", \"CQ_RETRY_RETENTION\", " +
+          "  \"CQ_NEXT_TXN_ID\"";
+        if (useMinHistoryWriteId) {
+          queryStr += ", \"MIN_OPEN_WRITE_ID\"";
+        }
+        queryStr +=
+          "  FROM \"COMPACTION_QUEUE\" \"cq1\" " +
+          "INNER JOIN (" +
+          "  SELECT MIN(\"CQ_HIGHEST_WRITE_ID\") \"MIN_WRITE_ID_HWM\", \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\"" +
+          "  FROM \"COMPACTION_QUEUE\"" 
+          + whereClause +
+          "  GROUP BY \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\") \"cq2\" " +
+          "ON \"cq1\".\"CQ_DATABASE\" = \"cq2\".\"CQ_DATABASE\""+
+          "  AND \"cq1\".\"CQ_TABLE\" = \"cq2\".\"CQ_TABLE\""+
+          "  AND (\"cq1\".\"CQ_PARTITION\" = \"cq2\".\"CQ_PARTITION\"" +
+          "    OR \"cq1\".\"CQ_PARTITION\" IS NULL AND \"cq2\".\"CQ_PARTITION\" IS NULL)" +
+          "  AND \"CQ_HIGHEST_WRITE_ID\" = \"MIN_WRITE_ID_HWM\" ";
+        
+        if (useMinHistoryWriteId) {
+          queryStr += 
+            "LEFT JOIN (" +
+            "  SELECT MIN(\"MH_WRITEID\") \"MIN_OPEN_WRITE_ID\", \"MH_DATABASE\", \"MH_TABLE\"" +
+            "  FROM \"MIN_HISTORY_WRITE_ID\"" +
+            "  GROUP BY \"MH_DATABASE\", \"MH_TABLE\") \"hwm\" " +
+            "ON \"cq1\".\"CQ_DATABASE\" = \"hwm\".\"MH_DATABASE\"" +
+            "  AND \"cq1\".\"CQ_TABLE\" = \"hwm\".\"MH_TABLE\"";
+          
+          whereClause += " AND (\"CQ_HIGHEST_WRITE_ID\" < \"MIN_OPEN_WRITE_ID\" OR \"MIN_OPEN_WRITE_ID\" IS NULL)";
+          
+        } else if (minOpenTxnWaterMark > 0) {
           whereClause += " AND (\"CQ_NEXT_TXN_ID\" <= " + minOpenTxnWaterMark + " OR \"CQ_NEXT_TXN_ID\" IS NULL)";
         }
-        whereClause += " AND (\"CQ_COMMIT_TIME\" < (" + getEpochFn(dbProduct) + " - \"CQ_RETRY_RETENTION\" - " + retentionTime + ") OR \"CQ_COMMIT_TIME\" IS NULL)";
-        String s = "SELECT \"CQ_ID\", \"cq1\".\"CQ_DATABASE\", \"cq1\".\"CQ_TABLE\", \"cq1\".\"CQ_PARTITION\"," +
-            "   \"CQ_TYPE\", \"CQ_RUN_AS\", \"CQ_HIGHEST_WRITE_ID\", \"CQ_TBLPROPERTIES\", \"CQ_RETRY_RETENTION\" " +
-            "  FROM \"COMPACTION_QUEUE\" \"cq1\" " +
-            "INNER JOIN (" +
-            "  SELECT MIN(\"CQ_HIGHEST_WRITE_ID\") \"WRITE_ID\", \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\"" +
-            "  FROM \"COMPACTION_QUEUE\""
-            + whereClause +
-            "  GROUP BY \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\") \"cq2\" " +
-            "ON \"cq1\".\"CQ_DATABASE\" = \"cq2\".\"CQ_DATABASE\""+
-            "  AND \"cq1\".\"CQ_TABLE\" = \"cq2\".\"CQ_TABLE\""+
-            "  AND (\"cq1\".\"CQ_PARTITION\" = \"cq2\".\"CQ_PARTITION\"" +
-            "    OR \"cq1\".\"CQ_PARTITION\" IS NULL AND \"cq2\".\"CQ_PARTITION\" IS NULL)"
-            + whereClause +
-            "  AND \"CQ_HIGHEST_WRITE_ID\" = \"WRITE_ID\"" +
-            "  ORDER BY \"CQ_ID\"";
-        LOG.debug("Going to execute query <{}>", s);
+        queryStr += whereClause + " ORDER BY \"CQ_ID\"";
+        LOG.debug("Going to execute query <{}>", queryStr);
 
-        try (ResultSet rs = stmt.executeQuery(s)) {
+        try (ResultSet rs = stmt.executeQuery(queryStr)) {
           while (rs.next()) {
             CompactionInfo info = new CompactionInfo();
             info.id = rs.getLong(1);
@@ -426,6 +444,10 @@ class CompactionTxnHandler extends TxnHandler {
             info.highestWriteId = rs.getLong(7);
             info.properties = rs.getString(8);
             info.retryRetention = rs.getInt(9);
+            info.nextTxnId = rs.getLong(10);
+            if (useMinHistoryWriteId) {
+              info.minOpenWriteId = rs.getLong(11);
+            }
             LOG.debug("Found ready to clean: {}", info);
             rc.add(info);
           }
@@ -434,8 +456,7 @@ class CompactionTxnHandler extends TxnHandler {
       } catch (SQLException e) {
         LOG.error("Unable to select next element for cleaning, " + e.getMessage());
         checkRetryable(e, "findReadyToClean");
-        throw new MetaException("Unable to connect to transaction database " +
-            e.getMessage());
+        throw new MetaException("Unable to connect to transaction database " + e.getMessage());
       }
     } catch (RetryException e) {
       return findReadyToClean(minOpenTxnWaterMark, retentionTime);
@@ -717,7 +738,7 @@ class CompactionTxnHandler extends TxnHandler {
         if (!rs.next()) {
           throw new MetaException("Transaction tables not properly initialized, no record found in TXNS");
         }
-        long minUncommitedTxnid = minTxnIdSeenOpen < 0 ? rs.getLong(1) : Math.min(rs.getLong(1), minTxnIdSeenOpen);
+        long minUncommitedTxnid = Math.min(rs.getLong(1), minTxnIdSeenOpen);
 
         // As all txns below min_uncommitted_txnid are either committed or empty_aborted, we are allowed
         // to cleanup the entries less than min_uncommitted_txnid from the TXN_TO_WRITE_ID table.
@@ -1535,20 +1556,17 @@ class CompactionTxnHandler extends TxnHandler {
   @Override
   @RetrySemantics.Idempotent
   public long findMinOpenTxnIdForCleaner() throws MetaException {
-    Connection dbConn = null;
+    if (useMinHistoryWriteId) {
+      return Long.MAX_VALUE;
+    }
     try {
-      try {
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED, connPoolCompaction);
+      try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED, connPoolCompaction)) {
         return getMinOpenTxnIdWaterMark(dbConn);
       } catch (SQLException e) {
-        LOG.error("Unable to getMinOpenTxnIdForCleaner", e);
-        rollbackDBConn(dbConn);
-        checkRetryable(e, "getMinOpenTxnForCleaner");
-        throw new MetaException("Unable to execute getMinOpenTxnIfForCleaner() " +
-            e.getMessage());
-      } finally {
-        closeDbConn(dbConn);
-      }
+        LOG.error("Unable to fetch minOpenTxnId for Cleaner", e);
+        checkRetryable(e, "findMinOpenTxnIdForCleaner");
+        throw new MetaException("Unable to execute findMinOpenTxnIdForCleaner() " + e.getMessage());
+      } 
     } catch (RetryException e) {
       return findMinOpenTxnIdForCleaner();
     }
@@ -1564,13 +1582,11 @@ class CompactionTxnHandler extends TxnHandler {
   @RetrySemantics.Idempotent
   @Deprecated
   public long findMinTxnIdSeenOpen() throws MetaException {
-    if (!useMinHistoryLevel) {
-      return -1L;
+    if (!useMinHistoryLevel || useMinHistoryWriteId) {
+      return Long.MAX_VALUE;
     }
-    Connection dbConn = null;
     try {
-      try {
-        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED, connPoolCompaction);
+      try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED, connPoolCompaction)) {
         long minOpenTxn;
         try (Statement stmt = dbConn.createStatement()) {
           try (ResultSet rs = stmt.executeQuery("SELECT MIN(\"MHL_MIN_OPEN_TXNID\") FROM \"MIN_HISTORY_LEVEL\"")) {
@@ -1579,24 +1595,20 @@ class CompactionTxnHandler extends TxnHandler {
             }
             minOpenTxn = rs.getLong(1);
             if (rs.wasNull()) {
-              minOpenTxn = -1L;
+              minOpenTxn = Long.MAX_VALUE;
             }
           }
         }
-        dbConn.rollback();
         return minOpenTxn;
       } catch (SQLException e) {
         if (dbProduct.isTableNotExistsError(e)) {
           useMinHistoryLevel = false;
-          return -1L;
+          return Long.MAX_VALUE;
         } else {
           LOG.error("Unable to execute findMinTxnIdSeenOpen", e);
-          rollbackDBConn(dbConn);
           checkRetryable(e, "findMinTxnIdSeenOpen");
           throw new MetaException("Unable to execute findMinTxnIdSeenOpen() " + e.getMessage());
         }
-      } finally {
-        closeDbConn(dbConn);
       }
     } catch (RetryException e) {
       return findMinTxnIdSeenOpen();
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 044e46b5a07..0d22f5fd9af 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
@@ -285,6 +285,8 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
       "WHERE \"HL_LAST_HEARTBEAT\" < %s - ? AND \"HL_TXNID\" = 0";
   private static final String TXN_TO_WRITE_ID_INSERT_QUERY = "INSERT INTO \"TXN_TO_WRITE_ID\" (\"T2W_TXNID\", " +
       "\"T2W_DATABASE\", \"T2W_TABLE\", \"T2W_WRITEID\") VALUES (?, ?, ?, ?)";
+  private static final String MIN_HISTORY_WRITE_ID_INSERT_QUERY = "INSERT INTO \"MIN_HISTORY_WRITE_ID\" (\"MH_TXNID\", " +
+      "\"MH_DATABASE\", \"MH_TABLE\", \"MH_WRITEID\") VALUES (?, ?, ?, ?)";
   private static final String SELECT_NWI_NEXT_FROM_NEXT_WRITE_ID =
       "SELECT \"NWI_NEXT\" FROM \"NEXT_WRITE_ID\" WHERE \"NWI_DATABASE\" = ? AND \"NWI_TABLE\" = ?";
   private static final String SELECT_METRICS_INFO_QUERY =
@@ -298,7 +300,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         TxnStatus.ABORTED + "') \"A\" CROSS JOIN (" +
       "SELECT COUNT(*), ({0} - MIN(\"HL_ACQUIRED_AT\"))/1000 FROM \"HIVE_LOCKS\") \"HL\" CROSS JOIN (" +
       "SELECT ({0} - MIN(\"CQ_COMMIT_TIME\"))/1000 from \"COMPACTION_QUEUE\" WHERE " +
-          "\"CQ_STATE\"=''" + Character.toString(READY_FOR_CLEANING) + "'') OLDEST_CLEAN";
+          "\"CQ_STATE\"=''" + READY_FOR_CLEANING + "'') OLDEST_CLEAN";
   private static final String SELECT_TABLES_WITH_X_ABORTED_TXNS =
       "SELECT \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" FROM \"TXN_COMPONENTS\" " +
           "INNER JOIN \"TXNS\" ON \"TC_TXNID\" = \"TXN_ID\" WHERE \"TXN_STATE\" = " + TxnStatus.ABORTED +
@@ -340,6 +342,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
   // Whether to use min_history_level table or not.
   // At startup we read it from the config, but set it to false if min_history_level does nto exists.
   static boolean useMinHistoryLevel;
+  static boolean useMinHistoryWriteId;
 
   /**
    * Derby specific concurrency control
@@ -411,12 +414,15 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
     maxBatchSize = MetastoreConf.getIntVar(conf, ConfVars.JDBC_MAX_BATCH_SIZE);
 
     openTxnTimeOutMillis = MetastoreConf.getTimeVar(conf, ConfVars.TXN_OPENTXN_TIMEOUT, TimeUnit.MILLISECONDS);
-
+    
     try {
-      boolean minHistoryConfig = MetastoreConf.getBoolVar(conf, ConfVars.TXN_USE_MIN_HISTORY_LEVEL);
+      useMinHistoryWriteId = checkIfTableIsUsable("MIN_HISTORY_WRITE_ID", 
+        MetastoreConf.getBoolVar(conf, ConfVars.TXN_USE_MIN_HISTORY_WRITE_ID));
+      
       // override the config if table does not exists anymore
       // this helps to roll out his feature when multiple HMS is accessing the same backend DB
-      useMinHistoryLevel = checkMinHistoryLevelTable(minHistoryConfig);
+      useMinHistoryLevel = checkIfTableIsUsable("MIN_HISTORY_LEVEL",
+        MetastoreConf.getBoolVar(conf, ConfVars.TXN_USE_MIN_HISTORY_LEVEL));
     } catch (MetaException e) {
       String msg = "Error during TxnHandler startup, " + e.getMessage();
       LOG.error(msg);
@@ -435,11 +441,11 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
   }
 
   /**
-   * Check if min_history_level table is usable
+   * Check if provided table is usable
    * @return
    * @throws MetaException
    */
-  private boolean checkMinHistoryLevelTable(boolean configValue) throws MetaException {
+  private boolean checkIfTableIsUsable(String tableName, boolean configValue) throws MetaException {
     if (!configValue) {
       // don't check it if disabled
       return false;
@@ -450,17 +456,17 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
       dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
       try (Statement stmt = dbConn.createStatement()) {
         // Dummy query to see if table exists
-        try (ResultSet rs = stmt.executeQuery("SELECT 1 FROM \"MIN_HISTORY_LEVEL\"")) {
+        try (ResultSet rs = stmt.executeQuery("SELECT 1 FROM \"" + tableName + "\"")) {
           rs.next();
         }
       }
     } catch (SQLException e) {
-      LOG.debug("Catching sql exception in min history level check", e);
+      LOG.debug("Catching sql exception in " + tableName + " check", e);
       if (dbProduct.isTableNotExistsError(e)) {
         tableExists = false;
       } else {
         throw new MetaException(
-            "Unable to select from transaction database: " + getMessage(e) + StringUtils.stringifyException(e));
+          "Unable to select from transaction database: " + getMessage(e) + StringUtils.stringifyException(e));
       }
     } finally {
       closeDbConn(dbConn);
@@ -1591,6 +1597,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         }
         updateWSCommitIdAndCleanUpMetadata(stmt, txnid, txnType, commitId, tempCommitId);
         removeTxnsFromMinHistoryLevel(dbConn, ImmutableList.of(txnid));
+        removeWriteIdsFromMinHistory(dbConn, ImmutableList.of(txnid));
         if (rqst.isSetKeyValue()) {
... 435 lines suppressed ...