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 2022/02/17 13:52:31 UTC

[hive] branch master updated: HIVE-25915: Query based MINOR compaction fails with NPE if the data is loaded into the ACID table (Laszlo Vegh, reviewed by Denys Kuzmenko)

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 93c1dbb  HIVE-25915: Query based MINOR compaction fails with NPE if the data is loaded into the ACID table (Laszlo Vegh, reviewed by Denys Kuzmenko)
93c1dbb is described below

commit 93c1dbb8bff3e587effbda3f76bee179d876f18b
Author: veghlaci05 <90...@users.noreply.github.com>
AuthorDate: Thu Feb 17 14:51:30 2022 +0100

    HIVE-25915: Query based MINOR compaction fails with NPE if the data is loaded into the ACID table (Laszlo Vegh, reviewed by Denys Kuzmenko)
    
    Closes #3000
---
 .../ql/txn/compactor/TestCrudCompactorOnTez.java   |  146 +-
 .../org.mockito.plugins.MockMaker                  |    1 +
 .../upgrade/hive/hive-schema-4.0.0.hive.sql        |    2 +-
 .../upgrade/hive/upgrade-3.1.0-to-4.0.0.hive.sql   |    2 +-
 .../hive/ql/txn/compactor/CompactorThread.java     |   22 +-
 .../hadoop/hive/ql/txn/compactor/Initiator.java    |   14 +-
 .../hadoop/hive/ql/txn/compactor/Worker.java       |   24 +-
 .../src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp |  381 ++
 .../src/gen/thrift/gen-cpp/ThriftHiveMetastore.h   |  125 +
 .../ThriftHiveMetastore_server.skeleton.cpp        |    5 +
 .../hive/metastore/api/ThriftHiveMetastore.java    | 3927 ++++++++++++--------
 .../metastore/ThriftHiveMetastoreClient.php        |   59 +
 .../gen-php/metastore/ThriftHiveMetastoreIf.php    |    5 +
 .../ThriftHiveMetastore_mark_refused_args.php      |   99 +
 .../ThriftHiveMetastore_mark_refused_result.php    |   96 +
 .../hive_metastore/ThriftHiveMetastore-remote      |    7 +
 .../gen-py/hive_metastore/ThriftHiveMetastore.py   |  192 +
 .../src/gen/thrift/gen-rb/thrift_hive_metastore.rb |   58 +
 .../hadoop/hive/metastore/HiveMetaStoreClient.java |    5 +
 .../hadoop/hive/metastore/IMetaStoreClient.java    |    8 +
 .../src/main/thrift/hive_metastore.thrift          |    1 +
 .../apache/hadoop/hive/metastore/HMSHandler.java   |    5 +
 .../hive/metastore/txn/CompactionTxnHandler.java   |   78 +-
 .../hadoop/hive/metastore/txn/TxnHandler.java      |    2 +
 .../apache/hadoop/hive/metastore/txn/TxnStore.java |    9 +
 .../metastore/HiveMetaStoreClientPreCatalog.java   |    5 +
 26 files changed, 3708 insertions(+), 1570 deletions(-)

diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCrudCompactorOnTez.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCrudCompactorOnTez.java
index f1b8b8a..02fa7c9 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCrudCompactorOnTez.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCrudCompactorOnTez.java
@@ -24,6 +24,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import com.google.common.collect.Lists;
 import org.apache.commons.lang3.StringUtils;
@@ -35,6 +36,7 @@ import org.apache.hadoop.hive.common.ValidWriteIdList;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.CompactionRequest;
 import org.apache.hadoop.hive.metastore.api.CompactionType;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.ShowCompactRequest;
@@ -46,6 +48,7 @@ import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.TxnCommandsBaseForTests;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.hooks.HiveProtoLoggingHook;
@@ -69,16 +72,155 @@ import org.apache.orc.impl.RecordReaderImpl;
 import org.apache.tez.dag.history.logging.proto.ProtoMessageReader;
 import org.junit.Assert;
 import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+import org.mockito.internal.util.reflection.FieldSetter;
 
+import static org.apache.hadoop.hive.ql.TxnCommandsBaseForTests.runWorker;
 import static org.apache.hadoop.hive.ql.txn.compactor.TestCompactor.execSelectAndDumpData;
 import static org.apache.hadoop.hive.ql.txn.compactor.TestCompactor.executeStatementOnDriver;
 import static org.apache.hadoop.hive.ql.txn.compactor.CompactorTestUtil.executeStatementOnDriverAndReturnResults;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.*;
 
 @SuppressWarnings("deprecation")
 public class TestCrudCompactorOnTez extends CompactorOnTezTest {
 
+  @Test
+  public void testMinorCompactionShouldBeRefusedOnTablesWithOriginalFiles() throws Exception {
+    conf.setBoolVar(HiveConf.ConfVars.COMPACTOR_CRUD_QUERY_BASED, true);
+    // Set delta numbuer threshold to 2 to avoid skipping compaction because of too few deltas
+    conf.setIntVar(HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_NUM_THRESHOLD, 2);
+    // Set delta percentage to a high value to suppress selecting major compression based on that
+    conf.setFloatVar(HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_PCT_THRESHOLD, 1000f);
+
+    final String dbName = "default";
+    final String tableName = "compaction_test";
+    executeStatementOnDriver("drop table if exists " + tableName, driver);
+    executeStatementOnDriver("CREATE TABLE " + tableName + "(id string, value string) CLUSTERED BY(id) " +
+            "INTO 10 BUCKETS STORED AS ORC TBLPROPERTIES('transactional'='false')", driver);
+
+    executeStatementOnDriver("INSERT INTO TABLE " + tableName + " values ('1','one'),('2','two'),('3','three')," +
+            "('4','four'),('5','five'),('6','six'),('7','seven'),('8','eight'),('9','nine'),('10','ten')," +
+            "('11','eleven'),('12','twelve'),('13','thirteen'),('14','fourteen'),('15','fifteen'),('16','sixteen')," +
+            "('17','seventeen'),('18','eighteen'),('19','nineteen'),('20','twenty')", driver);
+
+    executeStatementOnDriver("alter table " + tableName + " set TBLPROPERTIES('transactional'='true')", driver);
+
+    executeStatementOnDriver("insert into " + tableName + " values ('21', 'value21'),('84', 'value84')," +
+            "('66', 'value66'),('54', 'value54')", driver);
+    executeStatementOnDriver("insert into " + tableName + " values ('22', 'value22'),('34', 'value34')," +
+            "('35', 'value35')", driver);
+    executeStatementOnDriver("insert into " + tableName + " values ('75', 'value75'),('99', 'value99')", driver);
+
+    execSelectAndDumpData("select * from " + tableName, driver, "Dumping data for " +
+            tableName + " after load:");
+
+    TxnStore txnHandler = TxnUtils.getTxnStore(conf);
+
+    //Prevent initiator from submitting the compaction requests
+    TxnStore mockedHandler = spy(txnHandler);
+    doThrow(new RuntimeException("")).when(mockedHandler).compact(nullable(CompactionRequest.class));
+    Initiator initiator = new Initiator();
+    initiator.setConf(conf);
+    initiator.init(new AtomicBoolean(true));
+    FieldSetter.setField(initiator, MetaStoreCompactorThread.class.getDeclaredField("txnHandler"), mockedHandler);
+
+    //Run initiator and capture compaction requests
+    initiator.run();
+
+    //Check captured compaction request and if the type for the table was MAJOR
+    ArgumentCaptor<CompactionRequest> requests = ArgumentCaptor.forClass(CompactionRequest.class);
+    verify(mockedHandler).compact(requests.capture());
+    Assert.assertTrue(requests.getAllValues().stream().anyMatch(r -> r.getTablename().equals(tableName) && r.getType().equals(CompactionType.MAJOR)));
+
+    //Try to do a minor compaction directly
+    CompactionRequest rqst = new CompactionRequest(dbName, tableName, CompactionType.MINOR);
+    txnHandler.compact(rqst);
+
+    runWorker(conf);
+
+    //Check if both compactions were failed with the expected error message
+    ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
+    List<ShowCompactResponseElement> compacts = rsp.getCompacts();
+    if (2 != compacts.size()) {
+      Assert.fail("Expecting 2 rows and found " + compacts.size() + " files " + compacts);
+    }
+    Assert.assertEquals("did not initiate", compacts.get(0).getState());
+    Assert.assertTrue(compacts.get(0).getErrorMessage()
+            .startsWith("Caught exception while trying to determine if we should compact"));
+    Assert.assertEquals("refused", compacts.get(1).getState());
+    Assert.assertTrue(compacts.get(1).getErrorMessage()
+            .startsWith("Query based Minor compaction is not possible for full acid tables having raw format (non-acid) data in them."));
+  }
+
+  @Test
+  public void testMinorCompactionShouldBeRefusedOnTablesWithRawData() throws Exception {
+    conf.setBoolVar(HiveConf.ConfVars.COMPACTOR_CRUD_QUERY_BASED, true);
+    // Set delta numbuer threshold to 2 to avoid skipping compaction because of too few deltas
+    conf.setIntVar(HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_NUM_THRESHOLD, 2);
+    // Set delta percentage to a high value to suppress selecting major compression based on that
+    conf.setFloatVar(HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_PCT_THRESHOLD, 1000f);
+
+    TxnStore txnHandler = TxnUtils.getTxnStore(conf);
+
+    final String dbName = "default";
+    final String origTableName = "compaction_test";
+    final String testTableName = "imported";
+    executeStatementOnDriver("drop table if exists " + origTableName, driver);
+    executeStatementOnDriver("drop table if exists " + testTableName, driver);
+    executeStatementOnDriver("CREATE TABLE " + origTableName + "(id string, value string) CLUSTERED BY(id) " +
+            "INTO 10 BUCKETS STORED AS ORC TBLPROPERTIES('transactional'='true')", driver);
+
+    executeStatementOnDriver("INSERT INTO TABLE " + origTableName + " values ('1','one'),('2','two'),('3','three')," +
+            "('4','four'),('5','five'),('6','six'),('7','seven'),('8','eight'),('9','nine'),('10','ten')," +
+            "('11','eleven'),('12','twelve'),('13','thirteen'),('14','fourteen'),('15','fifteen'),('16','sixteen')," +
+            "('17','seventeen'),('18','eighteen'),('19','nineteen'),('20','twenty')", driver);
+
+    execSelectAndDumpData("select * from " + origTableName, driver, "Dumping data for " +
+            origTableName + " after load:");
+
+    executeStatementOnDriver("export table " + origTableName + " to '/tmp/temp_acid'", driver);
+    executeStatementOnDriver("import table " + testTableName + " from '/tmp/temp_acid'", driver);
+    executeStatementOnDriver("insert into " + testTableName + " values ('21', 'value21'),('84', 'value84')," +
+            "('66', 'value66'),('54', 'value54')", driver);
+    executeStatementOnDriver("insert into " + testTableName + " values ('22', 'value22'),('34', 'value34')," +
+            "('35', 'value35')", driver);
+    executeStatementOnDriver("insert into " + testTableName + " values ('75', 'value75'),('99', 'value99')", driver);
+
+    //Prevent initiator from submitting the compaction requests
+    TxnStore mockedHandler = spy(txnHandler);
+    doThrow(new RuntimeException("")).when(mockedHandler).compact(nullable(CompactionRequest.class));
+    Initiator initiator = new Initiator();
+    initiator.setConf(conf);
+    initiator.init(new AtomicBoolean(true));
+    FieldSetter.setField(initiator, MetaStoreCompactorThread.class.getDeclaredField("txnHandler"), mockedHandler);
+
+    //Run initiator and capture compaction requests
+    initiator.run();
+
+    //Check captured compaction request and if the type for the table was MAJOR
+    ArgumentCaptor<CompactionRequest> requests = ArgumentCaptor.forClass(CompactionRequest.class);
+    verify(mockedHandler).compact(requests.capture());
+    Assert.assertTrue(requests.getAllValues().stream().anyMatch(r -> r.getTablename().equals(testTableName) && r.getType().equals(CompactionType.MAJOR)));
+
+    //Try to do a minor compaction directly
+    CompactionRequest rqst = new CompactionRequest(dbName, testTableName, CompactionType.MINOR);
+    txnHandler.compact(rqst);
+
+    runWorker(conf);
+
+    //Check if both compactions were failed with the expected error message
+    ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
+    List<ShowCompactResponseElement> compacts = rsp.getCompacts();
+    if (2 != compacts.size()) {
+      Assert.fail("Expecting 2 rows and found " + compacts.size() + " files " + compacts);
+    }
+    Assert.assertEquals("did not initiate", compacts.get(0).getState());
+    Assert.assertTrue(compacts.get(0).getErrorMessage().startsWith("Caught exception while trying to determine if we should compact"));
+    Assert.assertEquals("refused", compacts.get(1).getState());
+    Assert.assertTrue(compacts.get(1).getErrorMessage().startsWith("Query based Minor compaction is not possible for full acid tables having raw format (non-acid) data in them."));
+  }
+
   /**
    * After each major compaction, stats need to be updated on the table
    * 1. create an ORC backed table (Orc is currently required by ACID)
diff --git a/itests/hive-unit/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/itests/hive-unit/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker
new file mode 100644
index 0000000..ca6ee9c
--- /dev/null
+++ b/itests/hive-unit/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker
@@ -0,0 +1 @@
+mock-maker-inline
\ No newline at end of file
diff --git a/metastore/scripts/upgrade/hive/hive-schema-4.0.0.hive.sql b/metastore/scripts/upgrade/hive/hive-schema-4.0.0.hive.sql
index dfc1f99..522229c 100644
--- a/metastore/scripts/upgrade/hive/hive-schema-4.0.0.hive.sql
+++ b/metastore/scripts/upgrade/hive/hive-schema-4.0.0.hive.sql
@@ -1223,7 +1223,7 @@ SELECT
   CQ_TABLE,
   CQ_PARTITION,
   CASE WHEN CQ_TYPE = 'i' THEN 'minor' WHEN CQ_TYPE = 'a' THEN 'major' ELSE 'UNKNOWN' END,
-  CASE WHEN CQ_STATE = 'i' THEN 'initiated' WHEN CQ_STATE = 'w' THEN 'working' WHEN CQ_STATE = 'r' THEN 'ready for cleaning' ELSE 'UNKNOWN' END,
+  CASE WHEN CQ_STATE = 'i' THEN 'initiated' WHEN CQ_STATE = 'w' THEN 'working' WHEN CQ_STATE = 'r' THEN 'ready for cleaning' WHEN CQ_STATE = 'c' THEN 'refused' ELSE 'UNKNOWN' END,
   CASE WHEN CQ_WORKER_ID IS NULL THEN NULL ELSE split(CQ_WORKER_ID,"-")[0] END,
   CASE WHEN CQ_WORKER_ID IS NULL THEN NULL ELSE split(CQ_WORKER_ID,"-")[size(split(CQ_WORKER_ID,"-"))-1] END,
   CQ_WORKER_VERSION,
diff --git a/metastore/scripts/upgrade/hive/upgrade-3.1.0-to-4.0.0.hive.sql b/metastore/scripts/upgrade/hive/upgrade-3.1.0-to-4.0.0.hive.sql
index ca2d839..eb3a997 100644
--- a/metastore/scripts/upgrade/hive/upgrade-3.1.0-to-4.0.0.hive.sql
+++ b/metastore/scripts/upgrade/hive/upgrade-3.1.0-to-4.0.0.hive.sql
@@ -341,7 +341,7 @@ SELECT
   CQ_TABLE,
   CQ_PARTITION,
   CASE WHEN CQ_TYPE = 'i' THEN 'minor' WHEN CQ_TYPE = 'a' THEN 'major' ELSE 'UNKNOWN' END,
-  CASE WHEN CQ_STATE = 'i' THEN 'initiated' WHEN CQ_STATE = 'w' THEN 'working' WHEN CQ_STATE = 'r' THEN 'ready for cleaning' ELSE 'UNKNOWN' END,
+  CASE WHEN CQ_STATE = 'i' THEN 'initiated' WHEN CQ_STATE = 'w' THEN 'working' WHEN CQ_STATE = 'r' THEN 'ready for cleaning' WHEN CQ_STATE = 'c' THEN 'refused' ELSE 'UNKNOWN' END,
   CASE WHEN CQ_WORKER_ID IS NULL THEN NULL ELSE split(CQ_WORKER_ID,"-")[0] END,
   CASE WHEN CQ_WORKER_ID IS NULL THEN NULL ELSE split(CQ_WORKER_ID,"-")[size(split(CQ_WORKER_ID,"-"))-1] END,
   CQ_WORKER_VERSION,
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java
index 473c186..9e825eb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java
@@ -35,6 +35,8 @@ import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
 
+import org.apache.hadoop.hive.ql.io.AcidDirectory;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -42,6 +44,7 @@ import org.apache.hadoop.hive.ql.exec.repl.util.ReplUtils;
 
 import java.util.Arrays;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -138,6 +141,23 @@ public abstract class CompactorThread extends Thread implements Configurable {
   }
 
   /**
+   * Check for that special case when minor compaction is supported or not.
+   * <ul>
+   *   <li>The table is Insert-only OR</li>
+   *   <li>Query based compaction is not enabled OR</li>
+   *   <li>The table has only acid data in it.</li>
+   * </ul>
+   * @param tblproperties The properties of the table to check
+   * @param dir The {@link AcidDirectory} instance pointing to the table's folder on the filesystem.
+   * @return Returns true if minor compaction is supported based on the given parameters, false otherwise.
+   */
+  protected boolean isMinorCompactionSupported(Map<String, String> tblproperties, AcidDirectory dir) {
+    //Query based Minor compaction is not possible for full acid tables having raw format (non-acid) data in them.
+    return AcidUtils.isInsertOnlyTable(tblproperties) || !conf.getBoolVar(HiveConf.ConfVars.COMPACTOR_CRUD_QUERY_BASED)
+            || !(dir.getOriginalFiles().size() > 0 || dir.getCurrentDirectories().stream().anyMatch(AcidUtils.ParsedDelta::isRawFormat));
+  }
+
+  /**
    * Get the storage descriptor for a compaction.
    * @param t table from {@link #resolveTable(org.apache.hadoop.hive.metastore.txn.CompactionInfo)}
    * @param p table from {@link #resolvePartition(org.apache.hadoop.hive.metastore.txn.CompactionInfo)}
@@ -151,7 +171,7 @@ public abstract class CompactorThread extends Thread implements Configurable {
    * Determine whether to run this job as the current user or whether we need a doAs to switch
    * users.
    * @param owner of the directory we will be working in, as determined by
-   * {@link TxnUtils#findUserToRunAs(String, org.apache.hadoop.hive.metastore.api.Table)}
+   * {@link org.apache.hadoop.hive.metastore.txn.TxnUtils#findUserToRunAs(String, Table, Configuration)}
    * @return true if the job should run as the current user, false if a doAs is needed.
    */
   protected boolean runJobAsSelf(String owner) {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
index 4cc54da..325492f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hive.ql.txn.compactor;
 
-import com.codahale.metrics.Counter;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.cache.Cache;
 import com.google.common.cache.CacheBuilder;
@@ -243,7 +242,8 @@ public class Initiator extends MetaStoreCompactorThread {
       ValidWriteIdList validWriteIds = resolveValidWriteIds(t);
       CompactionType type = checkForCompaction(ci, validWriteIds, sd, t.getParameters(), runAs);
       if (type != null) {
-        requestCompaction(ci, runAs, type);
+        ci.type = type;
+        requestCompaction(ci, runAs);
       }
     } catch (Throwable ex) {
       String errorMessage = "Caught exception while trying to determine if we should compact " + ci + ". Marking "
@@ -431,7 +431,7 @@ public class Initiator extends MetaStoreCompactorThread {
   }
 
   private CompactionType determineCompactionType(CompactionInfo ci, AcidDirectory dir, Map<String,
-      String> tblproperties, long baseSize, long deltaSize) throws IOException {
+      String> tblproperties, long baseSize, long deltaSize) {
     boolean noBase = false;
     List<AcidUtils.ParsedDelta> deltas = dir.getCurrentDirectories();
     if (baseSize == 0 && deltaSize > 0) {
@@ -490,7 +490,9 @@ public class Initiator extends MetaStoreCompactorThread {
     // If there's no base file, do a major compaction
     LOG.debug("Found " + deltas.size() + " delta files, and " + (noBase ? "no" : "has") + " base," +
         "requesting " + (noBase ? "major" : "minor") + " compaction");
-    return noBase ? CompactionType.MAJOR : CompactionType.MINOR;
+
+    return noBase || !isMinorCompactionSupported(tblproperties, dir) ?
+            CompactionType.MAJOR : CompactionType.MINOR;
   }
 
   private long getBaseSize(AcidDirectory dir) throws IOException {
@@ -513,8 +515,8 @@ public class Initiator extends MetaStoreCompactorThread {
     return size;
   }
 
-  private void requestCompaction(CompactionInfo ci, String runAs, CompactionType type) throws MetaException {
-    CompactionRequest rqst = new CompactionRequest(ci.dbname, ci.tableName, type);
+  private void requestCompaction(CompactionInfo ci, String runAs) throws MetaException {
+    CompactionRequest rqst = new CompactionRequest(ci.dbname, ci.tableName, ci.type);
     if (ci.partName != null) rqst.setPartitionname(ci.partName);
     rqst.setRunas(runAs);
     rqst.setInitiatorId(getInitiatorId(Thread.currentThread().getId()));
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
index 859f4c1..8cdbcd2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
@@ -21,6 +21,7 @@ import com.google.common.annotations.VisibleForTesting;
 import java.util.concurrent.ScheduledExecutorService;
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.ValidCompactorWriteIdList;
@@ -456,7 +457,18 @@ public class Worker extends RemoteCompactorThread implements MetaStoreThread {
         compactionTxn.wasSuccessful();
         return false;
       }
-
+      if (!ci.isMajorCompaction() && !isMinorCompactionSupported(t.getParameters(), dir)) {
+        ci.errorMessage = String.format("Query based Minor compaction is not possible for full acid tables having raw " +
+                "format (non-acid) data in them. Compaction type: %s, Partition: %s, Compaction id: %d",
+                ci.type.toString(), ci.getFullPartitionName(), ci.id);
+        LOG.error(ci.errorMessage);
+        try {
+          msc.markRefused(CompactionInfo.compactionInfoToStruct(ci));
+        } catch (Throwable tr) {
+          LOG.error("Caught an exception while trying to mark compaction {} as failed: {}", ci, tr);
+        }
+        return false;
+      }
       checkInterrupt();
 
       try {
@@ -495,7 +507,7 @@ public class Worker extends RemoteCompactorThread implements MetaStoreThread {
         LOG.error("Caught exception while trying to compact " + ci +
             ". Marking failed to avoid repeated failures", e);
         final CompactionType ctype = ci.type;
-        markFailed(ci, e);
+        markFailed(ci, e.getMessage());
 
         if (runJobAsSelf(ci.runAs)) {
           cleanupResultDirs(sd, tblValidWriteIds, ctype, dir);
@@ -517,7 +529,7 @@ public class Worker extends RemoteCompactorThread implements MetaStoreThread {
       }
     } catch (TException | IOException t) {
       LOG.error("Caught an exception in the main loop of compactor worker " + workerName, t);
-      markFailed(ci, t);
+      markFailed(ci, t.getMessage());
       if (msc != null) {
         msc.close();
         msc = null;
@@ -615,9 +627,9 @@ public class Worker extends RemoteCompactorThread implements MetaStoreThread {
     return new CompactorMR();
   }
 
-  private void markFailed(CompactionInfo ci, Throwable e) {
-    if (ci != null) {
-      ci.errorMessage = e.getMessage();
+  private void markFailed(CompactionInfo ci, String errorMessage) {
+    if (ci != null && StringUtils.isNotBlank(errorMessage)) {
+      ci.errorMessage = errorMessage;
     }
     if (msc == null) {
       LOG.warn("Metastore client was null. Could not mark failed: {}", ci);
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 26218bf..d98a0d3 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
@@ -48352,6 +48352,193 @@ uint32_t ThriftHiveMetastore_mark_failed_presult::read(::apache::thrift::protoco
 }
 
 
+ThriftHiveMetastore_mark_refused_args::~ThriftHiveMetastore_mark_refused_args() noexcept {
+}
+
+
+uint32_t ThriftHiveMetastore_mark_refused_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_STRUCT) {
+          xfer += this->cr.read(iprot);
+          this->__isset.cr = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t ThriftHiveMetastore_mark_refused_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_mark_refused_args");
+
+  xfer += oprot->writeFieldBegin("cr", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += this->cr.write(oprot);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_mark_refused_pargs::~ThriftHiveMetastore_mark_refused_pargs() noexcept {
+}
+
+
+uint32_t ThriftHiveMetastore_mark_refused_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_mark_refused_pargs");
+
+  xfer += oprot->writeFieldBegin("cr", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += (*(this->cr)).write(oprot);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_mark_refused_result::~ThriftHiveMetastore_mark_refused_result() noexcept {
+}
+
+
+uint32_t ThriftHiveMetastore_mark_refused_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->o1.read(iprot);
+          this->__isset.o1 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t ThriftHiveMetastore_mark_refused_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+
+  uint32_t xfer = 0;
+
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_mark_refused_result");
+
+  if (this->__isset.o1) {
+    xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
+    xfer += this->o1.write(oprot);
+    xfer += oprot->writeFieldEnd();
+  }
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_mark_refused_presult::~ThriftHiveMetastore_mark_refused_presult() noexcept {
+}
+
+
+uint32_t ThriftHiveMetastore_mark_refused_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->o1.read(iprot);
+          this->__isset.o1 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+
 ThriftHiveMetastore_update_compaction_metrics_data_args::~ThriftHiveMetastore_update_compaction_metrics_data_args() noexcept {
 }
 
@@ -75931,6 +76118,62 @@ void ThriftHiveMetastoreClient::recv_mark_failed()
   return;
 }
 
+void ThriftHiveMetastoreClient::mark_refused(const CompactionInfoStruct& cr)
+{
+  send_mark_refused(cr);
+  recv_mark_refused();
+}
+
+void ThriftHiveMetastoreClient::send_mark_refused(const CompactionInfoStruct& cr)
+{
+  int32_t cseqid = 0;
+  oprot_->writeMessageBegin("mark_refused", ::apache::thrift::protocol::T_CALL, cseqid);
+
+  ThriftHiveMetastore_mark_refused_pargs args;
+  args.cr = &cr;
+  args.write(oprot_);
+
+  oprot_->writeMessageEnd();
+  oprot_->getTransport()->writeEnd();
+  oprot_->getTransport()->flush();
+}
+
+void ThriftHiveMetastoreClient::recv_mark_refused()
+{
+
+  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("mark_refused") != 0) {
+    iprot_->skip(::apache::thrift::protocol::T_STRUCT);
+    iprot_->readMessageEnd();
+    iprot_->getTransport()->readEnd();
+  }
+  ThriftHiveMetastore_mark_refused_presult result;
+  result.read(iprot_);
+  iprot_->readMessageEnd();
+  iprot_->getTransport()->readEnd();
+
+  if (result.__isset.o1) {
+    throw result.o1;
+  }
+  return;
+}
+
 bool ThriftHiveMetastoreClient::update_compaction_metrics_data(const CompactionMetricsDataStruct& data)
 {
   send_update_compaction_metrics_data(data);
@@ -92010,6 +92253,62 @@ void ThriftHiveMetastoreProcessor::process_mark_failed(int32_t seqid, ::apache::
   }
 }
 
+void ThriftHiveMetastoreProcessor::process_mark_refused(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.mark_refused", callContext);
+  }
+  ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.mark_refused");
+
+  if (this->eventHandler_.get() != nullptr) {
+    this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.mark_refused");
+  }
+
+  ThriftHiveMetastore_mark_refused_args args;
+  args.read(iprot);
+  iprot->readMessageEnd();
+  uint32_t bytes = iprot->getTransport()->readEnd();
+
+  if (this->eventHandler_.get() != nullptr) {
+    this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.mark_refused", bytes);
+  }
+
+  ThriftHiveMetastore_mark_refused_result result;
+  try {
+    iface_->mark_refused(args.cr);
+  } catch (MetaException &o1) {
+    result.o1 = o1;
+    result.__isset.o1 = true;
+  } catch (const std::exception& e) {
+    if (this->eventHandler_.get() != nullptr) {
+      this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.mark_refused");
+    }
+
+    ::apache::thrift::TApplicationException x(e.what());
+    oprot->writeMessageBegin("mark_refused", ::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.mark_refused");
+  }
+
+  oprot->writeMessageBegin("mark_refused", ::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.mark_refused", bytes);
+  }
+}
+
 void ThriftHiveMetastoreProcessor::process_update_compaction_metrics_data(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext)
 {
   void* ctx = nullptr;
@@ -114095,6 +114394,88 @@ void ThriftHiveMetastoreConcurrentClient::recv_mark_failed(const int32_t seqid)
   } // end while(true)
 }
 
+void ThriftHiveMetastoreConcurrentClient::mark_refused(const CompactionInfoStruct& cr)
+{
+  int32_t seqid = send_mark_refused(cr);
+  recv_mark_refused(seqid);
+}
+
+int32_t ThriftHiveMetastoreConcurrentClient::send_mark_refused(const CompactionInfoStruct& cr)
+{
+  int32_t cseqid = this->sync_->generateSeqId();
+  ::apache::thrift::async::TConcurrentSendSentry sentry(this->sync_.get());
+  oprot_->writeMessageBegin("mark_refused", ::apache::thrift::protocol::T_CALL, cseqid);
+
+  ThriftHiveMetastore_mark_refused_pargs args;
+  args.cr = &cr;
+  args.write(oprot_);
+
+  oprot_->writeMessageEnd();
+  oprot_->getTransport()->writeEnd();
+  oprot_->getTransport()->flush();
+
+  sentry.commit();
+  return cseqid;
+}
+
+void ThriftHiveMetastoreConcurrentClient::recv_mark_refused(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("mark_refused") != 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_mark_refused_presult result;
+      result.read(iprot_);
+      iprot_->readMessageEnd();
+      iprot_->getTransport()->readEnd();
+
+      if (result.__isset.o1) {
+        sentry.commit();
+        throw result.o1;
+      }
+      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)
+}
+
 bool ThriftHiveMetastoreConcurrentClient::update_compaction_metrics_data(const CompactionMetricsDataStruct& data)
 {
   int32_t seqid = send_update_compaction_metrics_data(data);
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 f16153b..6be3ba6 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
@@ -224,6 +224,7 @@ class ThriftHiveMetastoreIf : virtual public  ::facebook::fb303::FacebookService
   virtual void mark_cleaned(const CompactionInfoStruct& cr) = 0;
   virtual void mark_compacted(const CompactionInfoStruct& cr) = 0;
   virtual void mark_failed(const CompactionInfoStruct& cr) = 0;
+  virtual void mark_refused(const CompactionInfoStruct& cr) = 0;
   virtual bool update_compaction_metrics_data(const CompactionMetricsDataStruct& data) = 0;
   virtual void remove_compaction_metrics_data(const CompactionMetricsDataRequest& request) = 0;
   virtual void set_hadoop_jobid(const std::string& jobId, const int64_t cq_id) = 0;
@@ -945,6 +946,9 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p
   void mark_failed(const CompactionInfoStruct& /* cr */) {
     return;
   }
+  void mark_refused(const CompactionInfoStruct& /* cr */) {
+    return;
+  }
   bool update_compaction_metrics_data(const CompactionMetricsDataStruct& /* data */) {
     bool _return = false;
     return _return;
@@ -25488,6 +25492,110 @@ class ThriftHiveMetastore_mark_failed_presult {
 
 };
 
+typedef struct _ThriftHiveMetastore_mark_refused_args__isset {
+  _ThriftHiveMetastore_mark_refused_args__isset() : cr(false) {}
+  bool cr :1;
+} _ThriftHiveMetastore_mark_refused_args__isset;
+
+class ThriftHiveMetastore_mark_refused_args {
+ public:
+
+  ThriftHiveMetastore_mark_refused_args(const ThriftHiveMetastore_mark_refused_args&);
+  ThriftHiveMetastore_mark_refused_args& operator=(const ThriftHiveMetastore_mark_refused_args&);
+  ThriftHiveMetastore_mark_refused_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_mark_refused_args() noexcept;
+  CompactionInfoStruct cr;
+
+  _ThriftHiveMetastore_mark_refused_args__isset __isset;
+
+  void __set_cr(const CompactionInfoStruct& val);
+
+  bool operator == (const ThriftHiveMetastore_mark_refused_args & rhs) const
+  {
+    if (!(cr == rhs.cr))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_mark_refused_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_mark_refused_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_mark_refused_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_mark_refused_pargs() noexcept;
+  const CompactionInfoStruct* cr;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_mark_refused_result__isset {
+  _ThriftHiveMetastore_mark_refused_result__isset() : o1(false) {}
+  bool o1 :1;
+} _ThriftHiveMetastore_mark_refused_result__isset;
+
+class ThriftHiveMetastore_mark_refused_result {
+ public:
+
+  ThriftHiveMetastore_mark_refused_result(const ThriftHiveMetastore_mark_refused_result&);
+  ThriftHiveMetastore_mark_refused_result& operator=(const ThriftHiveMetastore_mark_refused_result&);
+  ThriftHiveMetastore_mark_refused_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_mark_refused_result() noexcept;
+  MetaException o1;
+
+  _ThriftHiveMetastore_mark_refused_result__isset __isset;
+
+  void __set_o1(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_mark_refused_result & rhs) const
+  {
+    if (!(o1 == rhs.o1))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_mark_refused_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_mark_refused_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_mark_refused_presult__isset {
+  _ThriftHiveMetastore_mark_refused_presult__isset() : o1(false) {}
+  bool o1 :1;
+} _ThriftHiveMetastore_mark_refused_presult__isset;
+
+class ThriftHiveMetastore_mark_refused_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_mark_refused_presult() noexcept;
+  MetaException o1;
+
+  _ThriftHiveMetastore_mark_refused_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
 typedef struct _ThriftHiveMetastore_update_compaction_metrics_data_args__isset {
   _ThriftHiveMetastore_update_compaction_metrics_data_args__isset() : data(false) {}
   bool data :1;
@@ -34080,6 +34188,9 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public
   void mark_failed(const CompactionInfoStruct& cr);
   void send_mark_failed(const CompactionInfoStruct& cr);
   void recv_mark_failed();
+  void mark_refused(const CompactionInfoStruct& cr);
+  void send_mark_refused(const CompactionInfoStruct& cr);
+  void recv_mark_refused();
   bool update_compaction_metrics_data(const CompactionMetricsDataStruct& data);
   void send_update_compaction_metrics_data(const CompactionMetricsDataStruct& data);
   bool recv_update_compaction_metrics_data();
@@ -34498,6 +34609,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
   void process_mark_cleaned(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_mark_compacted(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_mark_failed(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
+  void process_mark_refused(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_update_compaction_metrics_data(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_remove_compaction_metrics_data(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_set_hadoop_jobid(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
@@ -34770,6 +34882,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
     processMap_["mark_cleaned"] = &ThriftHiveMetastoreProcessor::process_mark_cleaned;
     processMap_["mark_compacted"] = &ThriftHiveMetastoreProcessor::process_mark_compacted;
     processMap_["mark_failed"] = &ThriftHiveMetastoreProcessor::process_mark_failed;
+    processMap_["mark_refused"] = &ThriftHiveMetastoreProcessor::process_mark_refused;
     processMap_["update_compaction_metrics_data"] = &ThriftHiveMetastoreProcessor::process_update_compaction_metrics_data;
     processMap_["remove_compaction_metrics_data"] = &ThriftHiveMetastoreProcessor::process_remove_compaction_metrics_data;
     processMap_["set_hadoop_jobid"] = &ThriftHiveMetastoreProcessor::process_set_hadoop_jobid;
@@ -36771,6 +36884,15 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi
     ifaces_[i]->mark_failed(cr);
   }
 
+  void mark_refused(const CompactionInfoStruct& cr) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->mark_refused(cr);
+    }
+    ifaces_[i]->mark_refused(cr);
+  }
+
   bool update_compaction_metrics_data(const CompactionMetricsDataStruct& data) {
     size_t sz = ifaces_.size();
     size_t i = 0;
@@ -38060,6 +38182,9 @@ class ThriftHiveMetastoreConcurrentClient : virtual public ThriftHiveMetastoreIf
   void mark_failed(const CompactionInfoStruct& cr);
   int32_t send_mark_failed(const CompactionInfoStruct& cr);
   void recv_mark_failed(const int32_t seqid);
+  void mark_refused(const CompactionInfoStruct& cr);
+  int32_t send_mark_refused(const CompactionInfoStruct& cr);
+  void recv_mark_refused(const int32_t seqid);
   bool update_compaction_metrics_data(const CompactionMetricsDataStruct& data);
   int32_t send_update_compaction_metrics_data(const CompactionMetricsDataStruct& data);
   bool recv_update_compaction_metrics_data(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 028cf9c..023a59c 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
@@ -1010,6 +1010,11 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("mark_failed\n");
   }
 
+  void mark_refused(const CompactionInfoStruct& cr) {
+    // Your implementation goes here
+    printf("mark_refused\n");
+  }
+
   bool update_compaction_metrics_data(const CompactionMetricsDataStruct& data) {
     // Your implementation goes here
     printf("update_compaction_metrics_data\n");
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 fed8336..1c26a38 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
@@ -411,6 +411,8 @@ package org.apache.hadoop.hive.metastore.api;
 
     public void mark_failed(CompactionInfoStruct cr) throws MetaException, org.apache.thrift.TException;
 
+    public void mark_refused(CompactionInfoStruct cr) throws MetaException, org.apache.thrift.TException;
+
     public boolean update_compaction_metrics_data(CompactionMetricsDataStruct data) throws MetaException, org.apache.thrift.TException;
 
     public void remove_compaction_metrics_data(CompactionMetricsDataRequest request) throws MetaException, org.apache.thrift.TException;
@@ -951,6 +953,8 @@ package org.apache.hadoop.hive.metastore.api;
 
     public void mark_failed(CompactionInfoStruct cr, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
+    public void mark_refused(CompactionInfoStruct cr, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
     public void update_compaction_metrics_data(CompactionMetricsDataStruct data, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
 
     public void remove_compaction_metrics_data(CompactionMetricsDataRequest request, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
@@ -6806,6 +6810,29 @@ package org.apache.hadoop.hive.metastore.api;
       return;
     }
 
+    public void mark_refused(CompactionInfoStruct cr) throws MetaException, org.apache.thrift.TException
+    {
+      send_mark_refused(cr);
+      recv_mark_refused();
+    }
+
+    public void send_mark_refused(CompactionInfoStruct cr) throws org.apache.thrift.TException
+    {
+      mark_refused_args args = new mark_refused_args();
+      args.setCr(cr);
+      sendBase("mark_refused", args);
+    }
+
+    public void recv_mark_refused() throws MetaException, org.apache.thrift.TException
+    {
+      mark_refused_result result = new mark_refused_result();
+      receiveBase(result, "mark_refused");
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      return;
+    }
+
     public boolean update_compaction_metrics_data(CompactionMetricsDataStruct data) throws MetaException, org.apache.thrift.TException
     {
       send_update_compaction_metrics_data(data);
@@ -15546,6 +15573,38 @@ package org.apache.hadoop.hive.metastore.api;
       }
     }
 
+    public void mark_refused(CompactionInfoStruct cr, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      mark_refused_call method_call = new mark_refused_call(cr, 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 mark_refused_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+      private CompactionInfoStruct cr;
+      public mark_refused_call(CompactionInfoStruct cr, 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.cr = cr;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("mark_refused", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        mark_refused_args args = new mark_refused_args();
+        args.setCr(cr);
+        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 update_compaction_metrics_data(CompactionMetricsDataStruct data, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       update_compaction_metrics_data_call method_call = new update_compaction_metrics_data_call(data, resultHandler, this, ___protocolFactory, ___transport);
@@ -18003,6 +18062,7 @@ package org.apache.hadoop.hive.metastore.api;
       processMap.put("mark_cleaned", new mark_cleaned());
       processMap.put("mark_compacted", new mark_compacted());
       processMap.put("mark_failed", new mark_failed());
+      processMap.put("mark_refused", new mark_refused());
       processMap.put("update_compaction_metrics_data", new update_compaction_metrics_data());
       processMap.put("remove_compaction_metrics_data", new remove_compaction_metrics_data());
       processMap.put("set_hadoop_jobid", new set_hadoop_jobid());
@@ -24170,6 +24230,35 @@ package org.apache.hadoop.hive.metastore.api;
       }
     }
 
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class mark_refused<I extends Iface> extends org.apache.thrift.ProcessFunction<I, mark_refused_args> {
+      public mark_refused() {
+        super("mark_refused");
+      }
+
+      public mark_refused_args getEmptyArgsInstance() {
+        return new mark_refused_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      public mark_refused_result getResult(I iface, mark_refused_args args) throws org.apache.thrift.TException {
+        mark_refused_result result = new mark_refused_result();
+        try {
+          iface.mark_refused(args.cr);
+        } catch (MetaException o1) {
+          result.o1 = o1;
+        }
+        return result;
+      }
+    }
+
     @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class update_compaction_metrics_data<I extends Iface> extends org.apache.thrift.ProcessFunction<I, update_compaction_metrics_data_args> {
       public update_compaction_metrics_data() {
         super("update_compaction_metrics_data");
@@ -26459,6 +26548,7 @@ package org.apache.hadoop.hive.metastore.api;
       processMap.put("mark_cleaned", new mark_cleaned());
       processMap.put("mark_compacted", new mark_compacted());
       processMap.put("mark_failed", new mark_failed());
+      processMap.put("mark_refused", new mark_refused());
       processMap.put("update_compaction_metrics_data", new update_compaction_metrics_data());
       processMap.put("remove_compaction_metrics_data", new remove_compaction_metrics_data());
       processMap.put("set_hadoop_jobid", new set_hadoop_jobid());
@@ -40096,22 +40186,20 @@ package org.apache.hadoop.hive.metastore.api;
       }
     }
 
-    @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_refused<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, mark_refused_args, Void> {
+      public mark_refused() {
+        super("mark_refused");
       }
 
-      public update_compaction_metrics_data_args getEmptyArgsInstance() {
-        return new update_compaction_metrics_data_args();
+      public mark_refused_args getEmptyArgsInstance() {
+        return new mark_refused_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_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) {
@@ -40125,7 +40213,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_refused_result result = new mark_refused_result();
             if (e instanceof MetaException) {
               result.o1 = (MetaException) e;
               result.setO1IsSet(true);
@@ -40157,25 +40245,27 @@ 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_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 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 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 remove_compaction_metrics_data_args getEmptyArgsInstance() {
-        return new remove_compaction_metrics_data_args();
+      public update_compaction_metrics_data_args getEmptyArgsInstance() {
+        return new update_compaction_metrics_data_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) {
-            remove_compaction_metrics_data_result result = new remove_compaction_metrics_data_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) {
@@ -40189,7 +40279,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();
+            update_compaction_metrics_data_result result = new update_compaction_metrics_data_result();
             if (e instanceof MetaException) {
               result.o1 = (MetaException) e;
               result.setO1IsSet(true);
@@ -40221,147 +40311,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, 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 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 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 set_hadoop_jobid_args getEmptyArgsInstance() {
-        return new set_hadoop_jobid_args();
+      public remove_compaction_metrics_data_args getEmptyArgsInstance() {
+        return new remove_compaction_metrics_data_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();
-            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;
-            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();
-              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, 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 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 get_latest_committed_compaction_info_args getEmptyArgsInstance() {
-        return new get_latest_committed_compaction_info_args();
-      }
-
-      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<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);
-            } 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_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();
-              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_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 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 get_next_notification_args getEmptyArgsInstance() {
-        return new get_next_notification_args();
-      }
-
-      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<NotificationEventResponse>() { 
-          public void onComplete(NotificationEventResponse o) {
-            get_next_notification_result result = new get_next_notification_result();
-            result.success = 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) {
@@ -40375,8 +40343,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) {
+            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;
@@ -40403,26 +40375,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, 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_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 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_current_notificationEventId_args getEmptyArgsInstance() {
-        return new get_current_notificationEventId_args();
+      public set_hadoop_jobid_args getEmptyArgsInstance() {
+        return new set_hadoop_jobid_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) {
+            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) {
@@ -40436,7 +40407,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_current_notificationEventId_result result = new get_current_notificationEventId_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();
@@ -40464,25 +40435,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, 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 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 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 get_notification_events_count_args getEmptyArgsInstance() {
-        return new get_notification_events_count_args();
+      public get_latest_committed_compaction_info_args getEmptyArgsInstance() {
+        return new get_latest_committed_compaction_info_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<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<NotificationEventsCountResponse>() { 
-          public void onComplete(NotificationEventsCountResponse o) {
-            get_notification_events_count_result result = new get_notification_events_count_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);
@@ -40497,7 +40468,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();
+            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();
@@ -40525,25 +40496,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, 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 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_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 fire_listener_event_args getEmptyArgsInstance() {
-        return new fire_listener_event_args();
+      public get_next_notification_args getEmptyArgsInstance() {
+        return new get_next_notification_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<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<FireEventResponse>() { 
-          public void onComplete(FireEventResponse o) {
-            fire_listener_event_result result = new fire_listener_event_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);
@@ -40558,7 +40529,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_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();
@@ -40586,25 +40557,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_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 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_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 flushCache_args getEmptyArgsInstance() {
-        return new flushCache_args();
+      public get_current_notificationEventId_args getEmptyArgsInstance() {
+        return new get_current_notificationEventId_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<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<Void>() { 
-          public void onComplete(Void o) {
-            flushCache_result result = new flushCache_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);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -40618,7 +40590,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_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();
@@ -40646,25 +40618,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_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<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_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_args getEmptyArgsInstance() {
-        return new add_write_notification_log_args();
+      public get_notification_events_count_args getEmptyArgsInstance() {
+        return new get_notification_events_count_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<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<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<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);
@@ -40679,7 +40651,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_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();
@@ -40707,25 +40679,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_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 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 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 add_write_notification_log_in_batch_args getEmptyArgsInstance() {
-        return new add_write_notification_log_in_batch_args();
+      public fire_listener_event_args getEmptyArgsInstance() {
+        return new fire_listener_event_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<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<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<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);
@@ -40740,7 +40712,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();
+            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();
@@ -40768,26 +40740,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, 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 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 flushCache<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, flushCache_args, Void> {
+      public flushCache() {
+        super("flushCache");
       }
 
-      public cm_recycle_args getEmptyArgsInstance() {
-        return new cm_recycle_args();
+      public flushCache_args getEmptyArgsInstance() {
+        return new flushCache_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<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<CmRecycleResponse>() { 
-          public void onComplete(CmRecycleResponse o) {
-            cm_recycle_result result = new cm_recycle_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) {
@@ -40801,12 +40772,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) {
+            flushCache_result result = new flushCache_result();
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
               return;
@@ -40833,25 +40800,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, 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_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 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_by_expr_args getEmptyArgsInstance() {
-        return new get_file_metadata_by_expr_args();
+      public add_write_notification_log_args getEmptyArgsInstance() {
+        return new add_write_notification_log_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<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<GetFileMetadataByExprResult>() { 
-          public void onComplete(GetFileMetadataByExprResult o) {
-            get_file_metadata_by_expr_result result = new get_file_metadata_by_expr_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);
@@ -40866,7 +40833,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();
+            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();
@@ -40894,25 +40861,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, 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 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_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 get_file_metadata_args getEmptyArgsInstance() {
-        return new get_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<GetFileMetadataResult> 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<GetFileMetadataResult>() { 
-          public void onComplete(GetFileMetadataResult o) {
-            get_file_metadata_result result = new get_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);
@@ -40927,7 +40894,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_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();
@@ -40955,25 +40922,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_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 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 cm_recycle<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, cm_recycle_args, CmRecycleResponse> {
+      public cm_recycle() {
+        super("cm_recycle");
       }
 
-      public put_file_metadata_args getEmptyArgsInstance() {
-        return new put_file_metadata_args();
+      public cm_recycle_args getEmptyArgsInstance() {
+        return new cm_recycle_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<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<PutFileMetadataResult>() { 
-          public void onComplete(PutFileMetadataResult o) {
-            put_file_metadata_result result = new put_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);
@@ -40988,8 +40955,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;
-            put_file_metadata_result result = new put_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;
@@ -41016,25 +40987,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, 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 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 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 clear_file_metadata_args getEmptyArgsInstance() {
-        return new clear_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<ClearFileMetadataResult> 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<ClearFileMetadataResult>() { 
-          public void onComplete(ClearFileMetadataResult o) {
-            clear_file_metadata_result result = new clear_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);
@@ -41049,7 +41020,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;
-            clear_file_metadata_result result = new clear_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();
@@ -41077,25 +41048,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, 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 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<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_file_metadata_args, GetFileMetadataResult> {
+      public get_file_metadata() {
+        super("get_file_metadata");
       }
 
-      public cache_file_metadata_args getEmptyArgsInstance() {
-        return new cache_file_metadata_args();
+      public get_file_metadata_args getEmptyArgsInstance() {
+        return new get_file_metadata_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<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<CacheFileMetadataResult>() { 
-          public void onComplete(CacheFileMetadataResult o) {
-            cache_file_metadata_result result = new cache_file_metadata_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);
@@ -41110,7 +41081,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_result result = new get_file_metadata_result();
             if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -41138,25 +41109,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_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 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 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 get_metastore_db_uuid_args getEmptyArgsInstance() {
-        return new get_metastore_db_uuid_args();
+      public put_file_metadata_args getEmptyArgsInstance() {
+        return new put_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<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<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<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);
@@ -41171,12 +41142,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) {
+            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;
@@ -41203,25 +41170,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, 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 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 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 create_resource_plan_args getEmptyArgsInstance() {
-        return new create_resource_plan_args();
+      public clear_file_metadata_args getEmptyArgsInstance() {
+        return new clear_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<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<WMCreateResourcePlanResponse>() { 
-          public void onComplete(WMCreateResourcePlanResponse o) {
-            create_resource_plan_result result = new create_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);
@@ -41236,20 +41203,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) {
+            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;
@@ -41276,25 +41231,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, 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_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 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_resource_plan_args getEmptyArgsInstance() {
-        return new get_resource_plan_args();
+      public cache_file_metadata_args getEmptyArgsInstance() {
+        return new cache_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<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<WMGetResourcePlanResponse>() { 
-          public void onComplete(WMGetResourcePlanResponse o) {
-            get_resource_plan_result result = new get_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);
@@ -41309,16 +41264,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) {
+            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;
@@ -41345,25 +41292,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, 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_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 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_active_resource_plan_args getEmptyArgsInstance() {
-        return new get_active_resource_plan_args();
+      public get_metastore_db_uuid_args getEmptyArgsInstance() {
+        return new get_metastore_db_uuid_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<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<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<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);
@@ -41378,10 +41325,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;
-            get_active_resource_plan_result result = new get_active_resource_plan_result();
+            get_metastore_db_uuid_result result = new get_metastore_db_uuid_result();
             if (e instanceof MetaException) {
-              result.o2 = (MetaException) e;
-              result.setO2IsSet(true);
+              result.o1 = (MetaException) e;
+              result.setO1IsSet(true);
               msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
@@ -41410,25 +41357,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, 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 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 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 get_all_resource_plans_args getEmptyArgsInstance() {
-        return new get_all_resource_plans_args();
+      public create_resource_plan_args getEmptyArgsInstance() {
+        return new create_resource_plan_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<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<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<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);
@@ -41443,11 +41390,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_all_resource_plans_result result = new get_all_resource_plans_result();
-            if (e instanceof MetaException) {
-              result.o1 = (MetaException) 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 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) {
               _LOGGER.error("TTransportException inside handler", e);
               fb.close();
@@ -41475,25 +41430,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, 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 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 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 alter_resource_plan_args getEmptyArgsInstance() {
-        return new alter_resource_plan_args();
+      public get_resource_plan_args getEmptyArgsInstance() {
+        return new get_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<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<WMAlterResourcePlanResponse>() { 
-          public void onComplete(WMAlterResourcePlanResponse o) {
-            alter_resource_plan_result result = new alter_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);
@@ -41508,18 +41463,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;
-            alter_resource_plan_result result = new alter_resource_plan_result();
+            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 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);
@@ -41548,25 +41499,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, 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 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_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 validate_resource_plan_args getEmptyArgsInstance() {
-        return new validate_resource_plan_args();
+      public get_active_resource_plan_args getEmptyArgsInstance() {
+        return new get_active_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<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<WMValidateResourcePlanResponse>() { 
-          public void onComplete(WMValidateResourcePlanResponse o) {
-            validate_resource_plan_result result = new validate_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);
@@ -41581,12 +41532,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;
-            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 MetaException) {
+            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;
@@ -41617,25 +41564,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_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 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_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 drop_resource_plan_args getEmptyArgsInstance() {
-        return new drop_resource_plan_args();
+      public get_all_resource_plans_args getEmptyArgsInstance() {
+        return new get_all_resource_plans_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<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<WMDropResourcePlanResponse>() { 
-          public void onComplete(WMDropResourcePlanResponse o) {
-            drop_resource_plan_result result = new drop_resource_plan_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);
@@ -41650,19 +41597,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;
+            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 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();
@@ -41690,25 +41629,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_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 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 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 create_wm_trigger_args getEmptyArgsInstance() {
-        return new create_wm_trigger_args();
+      public alter_resource_plan_args getEmptyArgsInstance() {
+        return new alter_resource_plan_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<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<WMCreateTriggerResponse>() { 
-          public void onComplete(WMCreateTriggerResponse o) {
-            create_wm_trigger_result result = new create_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);
@@ -41723,22 +41662,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;
-            create_wm_trigger_result result = new create_wm_trigger_result();
-            if (e instanceof AlreadyExistsException) {
-              result.o1 = (AlreadyExistsException) e;
+            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 NoSuchObjectException) {
-              result.o2 = (NoSuchObjectException) e;
+            } else if (e instanceof InvalidOperationException) {
+              result.o2 = (InvalidOperationException) 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.o3 = (MetaException) e;
+              result.setO3IsSet(true);
               msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
@@ -41767,25 +41702,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, 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 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 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 alter_wm_trigger_args getEmptyArgsInstance() {
-        return new alter_wm_trigger_args();
+      public validate_resource_plan_args getEmptyArgsInstance() {
+        return new validate_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<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<WMAlterTriggerResponse>() { 
-          public void onComplete(WMAlterTriggerResponse o) {
-            alter_wm_trigger_result result = new alter_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);
@@ -41800,18 +41735,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;
-            alter_wm_trigger_result result = new alter_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 InvalidObjectException) {
-              result.o2 = (InvalidObjectException) 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);
@@ -41840,25 +41771,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, 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 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 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 drop_wm_trigger_args getEmptyArgsInstance() {
-        return new drop_wm_trigger_args();
+      public drop_resource_plan_args getEmptyArgsInstance() {
+        return new drop_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<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<WMDropTriggerResponse>() { 
-          public void onComplete(WMDropTriggerResponse o) {
-            drop_wm_trigger_result result = new drop_wm_trigger_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);
@@ -41873,7 +41804,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_trigger_result result = new drop_wm_trigger_result();
+            drop_resource_plan_result result = new drop_resource_plan_result();
             if (e instanceof NoSuchObjectException) {
               result.o1 = (NoSuchObjectException) e;
               result.setO1IsSet(true);
@@ -41913,25 +41844,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, 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 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 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 get_triggers_for_resourceplan_args getEmptyArgsInstance() {
-        return new get_triggers_for_resourceplan_args();
+      public create_wm_trigger_args getEmptyArgsInstance() {
+        return new create_wm_trigger_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<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<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<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);
@@ -41946,15 +41877,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_triggers_for_resourceplan_result result = new get_triggers_for_resourceplan_result();
-            if (e instanceof NoSuchObjectException) {
-              result.o1 = (NoSuchObjectException) e;
+            create_wm_trigger_result result = new create_wm_trigger_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();
@@ -41982,25 +41921,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, 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 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 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 create_wm_pool_args getEmptyArgsInstance() {
-        return new create_wm_pool_args();
+      public alter_wm_trigger_args getEmptyArgsInstance() {
+        return new alter_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<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<WMCreatePoolResponse>() { 
-          public void onComplete(WMCreatePoolResponse o) {
-            create_wm_pool_result result = new create_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);
@@ -42015,22 +41954,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;
-            create_wm_pool_result result = new create_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);
@@ -42059,25 +41994,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, 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 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 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 alter_wm_pool_args getEmptyArgsInstance() {
-        return new alter_wm_pool_args();
+      public drop_wm_trigger_args getEmptyArgsInstance() {
+        return new drop_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<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<WMAlterPoolResponse>() { 
-          public void onComplete(WMAlterPoolResponse o) {
-            alter_wm_pool_result result = new alter_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);
@@ -42092,22 +42027,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;
+            drop_wm_trigger_result result = new drop_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;
+            } else if (e instanceof InvalidOperationException) {
+              result.o2 = (InvalidOperationException) 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.o3 = (MetaException) e;
+              result.setO3IsSet(true);
               msg = result;
             } else if (e instanceof org.apache.thrift.transport.TTransportException) {
               _LOGGER.error("TTransportException inside handler", e);
@@ -42136,25 +42067,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, 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 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 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 drop_wm_pool_args getEmptyArgsInstance() {
-        return new drop_wm_pool_args();
+      public get_triggers_for_resourceplan_args getEmptyArgsInstance() {
+        return new get_triggers_for_resourceplan_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<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<WMDropPoolResponse>() { 
-          public void onComplete(WMDropPoolResponse o) {
-            drop_wm_pool_result result = new drop_wm_pool_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);
@@ -42169,18 +42100,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_pool_result result = new drop_wm_pool_result();
+            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 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);
@@ -42209,25 +42136,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, 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 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 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 create_or_update_wm_mapping_args getEmptyArgsInstance() {
-        return new create_or_update_wm_mapping_args();
+      public create_wm_pool_args getEmptyArgsInstance() {
+        return new create_wm_pool_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<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<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<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);
@@ -42242,7 +42169,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_update_wm_mapping_result result = new create_or_update_wm_mapping_result();
+            create_wm_pool_result result = new create_wm_pool_result();
             if (e instanceof AlreadyExistsException) {
               result.o1 = (AlreadyExistsException) e;
               result.setO1IsSet(true);
@@ -42286,25 +42213,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, 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 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 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 drop_wm_mapping_args getEmptyArgsInstance() {
-        return new drop_wm_mapping_args();
+      public alter_wm_pool_args getEmptyArgsInstance() {
+        return new alter_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<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<WMDropMappingResponse>() { 
-          public void onComplete(WMDropMappingResponse o) {
-            drop_wm_mapping_result result = new drop_wm_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);
@@ -42319,19 +42246,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;
+            alter_wm_pool_result result = new alter_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();
@@ -42359,25 +42290,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, 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_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 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_or_drop_wm_trigger_to_pool_mapping_args getEmptyArgsInstance() {
-        return new create_or_drop_wm_trigger_to_pool_mapping_args();
+      public drop_wm_pool_args getEmptyArgsInstance() {
+        return new drop_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<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<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<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);
@@ -42392,22 +42323,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;
-            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;
+            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 InvalidObjectException) {
-              result.o3 = (InvalidObjectException) e;
-              result.setO3IsSet(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);
@@ -42436,25 +42363,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, 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 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 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 create_ischema_args getEmptyArgsInstance() {
-        return new create_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) {
-            create_ischema_result result = new create_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) {
@@ -42468,7 +42396,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_ischema_result result = new create_ischema_result();
+            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);
@@ -42477,10 +42405,14 @@ package org.apache.hadoop.hive.metastore.api;
               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();
@@ -42508,25 +42440,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, 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 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 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 alter_ischema_args getEmptyArgsInstance() {
-        return new alter_ischema_args();
+      public drop_wm_mapping_args getEmptyArgsInstance() {
+        return new drop_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<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<Void>() { 
-          public void onComplete(Void o) {
-            alter_ischema_result result = new alter_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);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -42540,15 +42473,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;
-            alter_ischema_result result = new alter_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();
@@ -42576,25 +42513,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, 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 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 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 get_ischema_args getEmptyArgsInstance() {
-        return new get_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<ISchema> 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<ISchema>() { 
-          public void onComplete(ISchema o) {
-            get_ischema_result result = new get_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);
@@ -42609,15 +42546,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_ischema_result result = new get_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 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();
@@ -42645,25 +42590,25 @@ 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, 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 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_ischema<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, create_ischema_args, Void> {
+      public create_ischema() {
+        super("create_ischema");
       }
 
-      public drop_ischema_args getEmptyArgsInstance() {
-        return new drop_ischema_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) {
-            drop_ischema_result result = new drop_ischema_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) {
@@ -42677,13 +42622,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;
-            drop_ischema_result result = new drop_ischema_result();
-            if (e instanceof NoSuchObjectException) {
-              result.o1 = (NoSuchObjectException) e;
+            create_ischema_result result = new create_ischema_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) {
@@ -42717,25 +42662,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_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 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 alter_ischema<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, alter_ischema_args, Void> {
+      public alter_ischema() {
+        super("alter_ischema");
       }
 
-      public add_schema_version_args getEmptyArgsInstance() {
-        return new add_schema_version_args();
+      public alter_ischema_args getEmptyArgsInstance() {
+        return new alter_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();
+            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) {
@@ -42749,18 +42694,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;
-            add_schema_version_result result = new add_schema_version_result();
-            if (e instanceof AlreadyExistsException) {
-              result.o1 = (AlreadyExistsException) e;
+            alter_ischema_result result = new alter_ischema_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 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);
@@ -42789,25 +42730,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, 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_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 get_ischema<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_ischema_args, ISchema> {
+      public get_ischema() {
+        super("get_ischema");
       }
 
-      public get_schema_version_args getEmptyArgsInstance() {
-        return new get_schema_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_version_result result = new get_schema_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);
@@ -42822,7 +42763,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();
+            get_ischema_result result = new get_ischema_result();
             if (e instanceof NoSuchObjectException) {
               result.o1 = (NoSuchObjectException) e;
               result.setO1IsSet(true);
@@ -42858,26 +42799,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, 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_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 drop_ischema<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drop_ischema_args, Void> {
+      public drop_ischema() {
+        super("drop_ischema");
       }
 
-      public get_schema_latest_version_args getEmptyArgsInstance() {
-        return new get_schema_latest_version_args();
+      public drop_ischema_args getEmptyArgsInstance() {
+        return new drop_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_latest_version_result result = new get_schema_latest_version_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) {
@@ -42891,15 +42831,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_latest_version_result result = new get_schema_latest_version_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();
@@ -42927,26 +42871,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, 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 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 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 get_schema_all_versions_args getEmptyArgsInstance() {
-        return new get_schema_all_versions_args();
+      public add_schema_version_args getEmptyArgsInstance() {
+        return new add_schema_version_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) {
+            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) {
@@ -42960,15 +42903,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();
-            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();
@@ -42996,25 +42943,26 @@ 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, 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 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 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 drop_schema_version_args getEmptyArgsInstance() {
-        return new drop_schema_version_args();
+      public get_schema_version_args getEmptyArgsInstance() {
+        return new get_schema_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) {
-            drop_schema_version_result result = new drop_schema_version_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);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -43028,7 +42976,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_schema_version_result result = new drop_schema_version_result();
+            get_schema_version_result result = new get_schema_version_result();
             if (e instanceof NoSuchObjectException) {
               result.o1 = (NoSuchObjectException) e;
               result.setO1IsSet(true);
@@ -43064,25 +43012,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, 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 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_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 get_schemas_by_cols_args getEmptyArgsInstance() {
-        return new get_schemas_by_cols_args();
+      public get_schema_latest_version_args getEmptyArgsInstance() {
+        return new get_schema_latest_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_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);
@@ -43097,11 +43045,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_latest_version_result result = new get_schema_latest_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();
@@ -43129,25 +43081,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_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 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_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 map_schema_version_to_serde_args getEmptyArgsInstance() {
-        return new map_schema_version_to_serde_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) {
-            map_schema_version_to_serde_result result = new map_schema_version_to_serde_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) {
@@ -43161,7 +43114,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_all_versions_result result = new get_schema_all_versions_result();
             if (e instanceof NoSuchObjectException) {
               result.o1 = (NoSuchObjectException) e;
               result.setO1IsSet(true);
@@ -43197,25 +43150,25 @@ 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_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 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 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 set_schema_version_state_args getEmptyArgsInstance() {
-        return new set_schema_version_state_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) {
-            set_schema_version_state_result result = new set_schema_version_state_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) {
@@ -43229,18 +43182,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();
+            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 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);
@@ -43269,25 +43218,26 @@ 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, 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 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 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 add_serde_args getEmptyArgsInstance() {
-        return new add_serde_args();
+      public get_schemas_by_cols_args getEmptyArgsInstance() {
+        return new get_schemas_by_cols_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<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<Void>() { 
-          public void onComplete(Void o) {
-            add_serde_result result = new add_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);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -43301,15 +43251,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;
-            add_serde_result result = new add_serde_result();
-            if (e instanceof AlreadyExistsException) {
-              result.o1 = (AlreadyExistsException) 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();
@@ -43337,26 +43283,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, 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_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 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_serde_args getEmptyArgsInstance() {
-        return new get_serde_args();
+      public map_schema_version_to_serde_args getEmptyArgsInstance() {
+        return new map_schema_version_to_serde_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<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<SerDeInfo>() { 
-          public void onComplete(SerDeInfo o) {
-            get_serde_result result = new get_serde_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) {
@@ -43370,7 +43315,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_serde_result result = new get_serde_result();
+            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);
@@ -43406,26 +43351,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, 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 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 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 get_lock_materialization_rebuild_args getEmptyArgsInstance() {
-        return new get_lock_materialization_rebuild_args();
+      public set_schema_version_state_args getEmptyArgsInstance() {
+        return new set_schema_version_state_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) {
+            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) {
@@ -43439,8 +43383,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;
-            get_lock_materialization_rebuild_result result = new get_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;
@@ -43467,27 +43423,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, 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 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 add_serde<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_serde_args, Void> {
+      public add_serde() {
+        super("add_serde");
       }
 
-      public heartbeat_lock_materialization_rebuild_args getEmptyArgsInstance() {
-        return new heartbeat_lock_materialization_rebuild_args();
+      public add_serde_args getEmptyArgsInstance() {
+        return new add_serde_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) {
+            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) {
@@ -43501,8 +43455,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;
-            heartbeat_lock_materialization_rebuild_result result = new heartbeat_lock_materialization_rebuild_result();
-            if (e instanceof org.apache.thrift.transport.TTransportException) {
+            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();
               return;
@@ -43529,25 +43491,26 @@ 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, 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 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 get_serde<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_serde_args, SerDeInfo> {
+      public get_serde() {
+        super("get_serde");
       }
 
-      public add_runtime_stats_args getEmptyArgsInstance() {
-        return new add_runtime_stats_args();
+      public get_serde_args getEmptyArgsInstance() {
+        return new get_serde_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<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<Void>() { 
-          public void onComplete(Void o) {
-            add_runtime_stats_result result = new add_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);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -43561,11 +43524,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;
+            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();
@@ -43593,25 +43560,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, 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_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_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_runtime_stats_args getEmptyArgsInstance() {
-        return new get_runtime_stats_args();
+      public get_lock_materialization_rebuild_args getEmptyArgsInstance() {
+        return new get_lock_materialization_rebuild_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<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<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<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);
@@ -43626,12 +43593,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_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 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;
@@ -43658,26 +43621,27 @@ 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_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 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 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 get_partitions_with_specs_args getEmptyArgsInstance() {
-        return new get_partitions_with_specs_args();
+      public heartbeat_lock_materialization_rebuild_args getEmptyArgsInstance() {
+        return new heartbeat_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<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<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<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) {
@@ -43691,12 +43655,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) {
+            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;
@@ -43723,26 +43683,25 @@ 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, 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_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 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_poll_args getEmptyArgsInstance() {
-        return new scheduled_query_poll_args();
+      public add_runtime_stats_args getEmptyArgsInstance() {
+        return new add_runtime_stats_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<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<ScheduledQueryPollResponse>() { 
-          public void onComplete(ScheduledQueryPollResponse o) {
-            scheduled_query_poll_result result = new scheduled_query_poll_result();
-            result.success = o;
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            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) {
@@ -43756,7 +43715,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;
-            scheduled_query_poll_result result = new scheduled_query_poll_result();
+            add_runtime_stats_result result = new add_runtime_stats_result();
             if (e instanceof MetaException) {
               result.o1 = (MetaException) e;
               result.setO1IsSet(true);
@@ -43788,25 +43747,26 @@ 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, 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_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 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_maintenance_args getEmptyArgsInstance() {
-        return new scheduled_query_maintenance_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_maintenance_result result = new scheduled_query_maintenance_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) {
@@ -43820,23 +43780,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();
+            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 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();
@@ -43864,25 +43812,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, 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 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_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 scheduled_query_progress_args getEmptyArgsInstance() {
-        return new scheduled_query_progress_args();
+      public get_partitions_with_specs_args getEmptyArgsInstance() {
+        return new get_partitions_with_specs_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<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<Void>() { 
-          public void onComplete(Void o) {
-            scheduled_query_progress_result result = new scheduled_query_progress_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);
             } catch (org.apache.thrift.transport.TTransportException e) {
@@ -43896,15 +43845,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_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 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();
@@ -43932,25 +43877,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_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 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 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 get_scheduled_query_args getEmptyArgsInstance() {
-        return new get_scheduled_query_args();
+      public scheduled_query_poll_args getEmptyArgsInstance() {
+        return new scheduled_query_poll_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<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<ScheduledQuery>() { 
-          public void onComplete(ScheduledQuery o) {
-            get_scheduled_query_result result = new get_scheduled_query_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);
@@ -43965,15 +43910,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();
+            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 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();
@@ -44001,25 +43942,25 @@ 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, 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 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_maintenance<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, scheduled_query_maintenance_args, Void> {
+      public scheduled_query_maintenance() {
+        super("scheduled_query_maintenance");
       }
 
-      public add_replication_metrics_args getEmptyArgsInstance() {
-        return new add_replication_metrics_args();
+      public scheduled_query_maintenance_args getEmptyArgsInstance() {
+        return new scheduled_query_maintenance_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_replication_metrics_result result = new add_replication_metrics_result();
+            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) {
@@ -44033,11 +43974,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;
-            add_replication_metrics_result result = new add_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();
@@ -44065,26 +44018,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_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_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_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_replication_metrics_args getEmptyArgsInstance() {
-        return new get_replication_metrics_args();
+      public scheduled_query_progress_args getEmptyArgsInstance() {
+        return new scheduled_query_progress_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_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) {
@@ -44098,11 +44050,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_replication_metrics_result result = new get_replication_metrics_result();
+            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();
@@ -44130,25 +44086,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_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 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 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 get_open_txns_req_args getEmptyArgsInstance() {
-        return new get_open_txns_req_args();
+      public get_scheduled_query_args getEmptyArgsInstance() {
+        return new get_scheduled_query_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<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<GetOpenTxnsResponse>() { 
-          public void onComplete(GetOpenTxnsResponse o) {
-            get_open_txns_req_result result = new get_open_txns_req_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);
@@ -44163,8 +44119,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) {
+            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 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;
@@ -44191,25 +44155,25 @@ 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, 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 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 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 create_stored_procedure_args getEmptyArgsInstance() {
-        return new create_stored_procedure_args();
+      public add_replication_metrics_args getEmptyArgsInstance() {
+        return new add_replication_metrics_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();
+            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) {
@@ -44223,15 +44187,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_stored_procedure_result result = new create_stored_procedure_result();
-            if (e instanceof NoSuchObjectException) {
-              result.o1 = (NoSuchObjectException) e;
+            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 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();
@@ -44259,25 +44219,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, 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 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 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 get_stored_procedure_args getEmptyArgsInstance() {
-        return new get_stored_procedure_args();
+      public get_replication_metrics_args getEmptyArgsInstance() {
+        return new get_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<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<StoredProcedure>() { 
-          public void onComplete(StoredProcedure o) {
-            get_stored_procedure_result result = new get_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);
@@ -44292,15 +44252,209 @@ 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();
+            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 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_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();
@@ -240909,52 +241063,752 @@ package org.apache.hadoop.hive.metastore.api;
     }
 
     // isset id assignments
-    private static final int __TXN_ID_ISSET_ID = 0;
-    private byte __isset_bitfield = 0;
+    private static final int __TXN_ID_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.CR, new org.apache.thrift.meta_data.FieldMetaData("cr", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, CompactionInfoStruct.class)));
+      tmpMap.put(_Fields.TXN_ID, new org.apache.thrift.meta_data.FieldMetaData("txn_id", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(update_compactor_state_args.class, metaDataMap);
+    }
+
+    public update_compactor_state_args() {
+    }
+
+    public update_compactor_state_args(
+      CompactionInfoStruct cr,
+      long txn_id)
+    {
+      this();
+      this.cr = cr;
+      this.txn_id = txn_id;
+      setTxn_idIsSet(true);
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public update_compactor_state_args(update_compactor_state_args other) {
+      __isset_bitfield = other.__isset_bitfield;
+      if (other.isSetCr()) {
+        this.cr = new CompactionInfoStruct(other.cr);
+      }
+      this.txn_id = other.txn_id;
+    }
+
+    public update_compactor_state_args deepCopy() {
+      return new update_compactor_state_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.cr = null;
+      setTxn_idIsSet(false);
+      this.txn_id = 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public CompactionInfoStruct getCr() {
+      return this.cr;
+    }
+
+    public void setCr(@org.apache.thrift.annotation.Nullable CompactionInfoStruct cr) {
+      this.cr = cr;
+    }
+
+    public void unsetCr() {
+      this.cr = null;
+    }
+
+    /** Returns true if field cr is set (has been assigned a value) and false otherwise */
+    public boolean isSetCr() {
+      return this.cr != null;
+    }
+
+    public void setCrIsSet(boolean value) {
+      if (!value) {
+        this.cr = null;
+      }
+    }
+
+    public long getTxn_id() {
+      return this.txn_id;
+    }
+
+    public void setTxn_id(long txn_id) {
+      this.txn_id = txn_id;
+      setTxn_idIsSet(true);
+    }
+
+    public void unsetTxn_id() {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __TXN_ID_ISSET_ID);
+    }
+
+    /** Returns true if field txn_id is set (has been assigned a value) and false otherwise */
+    public boolean isSetTxn_id() {
+      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __TXN_ID_ISSET_ID);
+    }
+
+    public void setTxn_idIsSet(boolean value) {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __TXN_ID_ISSET_ID, value);
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      case CR:
+        if (value == null) {
+          unsetCr();
+        } else {
+          setCr((CompactionInfoStruct)value);
+        }
+        break;
+
+      case TXN_ID:
+        if (value == null) {
+          unsetTxn_id();
+        } else {
+          setTxn_id((java.lang.Long)value);
+        }
+        break;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case CR:
+        return getCr();
+
+      case TXN_ID:
+        return getTxn_id();
+
+      }
+      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 CR:
+        return isSetCr();
+      case TXN_ID:
+        return isSetTxn_id();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof update_compactor_state_args)
+        return this.equals((update_compactor_state_args)that);
+      return false;
+    }
+
+    public boolean equals(update_compactor_state_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_cr = true && this.isSetCr();
+      boolean that_present_cr = true && that.isSetCr();
+      if (this_present_cr || that_present_cr) {
+        if (!(this_present_cr && that_present_cr))
+          return false;
+        if (!this.cr.equals(that.cr))
+          return false;
+      }
+
+      boolean this_present_txn_id = true;
+      boolean that_present_txn_id = true;
+      if (this_present_txn_id || that_present_txn_id) {
+        if (!(this_present_txn_id && that_present_txn_id))
+          return false;
+        if (this.txn_id != that.txn_id)
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetCr()) ? 131071 : 524287);
+      if (isSetCr())
+        hashCode = hashCode * 8191 + cr.hashCode();
+
+      hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(txn_id);
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(update_compactor_state_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.compare(isSetCr(), other.isSetCr());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCr()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.cr, other.cr);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetTxn_id(), other.isSetTxn_id());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTxn_id()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.txn_id, other.txn_id);
+        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("update_compactor_state_args(");
+      boolean first = true;
+
+      sb.append("cr:");
+      if (this.cr == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.cr);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("txn_id:");
+      sb.append(this.txn_id);
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (cr != null) {
+        cr.validate();
+      }
+    }
+
+    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 update_compactor_state_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public update_compactor_state_argsStandardScheme getScheme() {
+        return new update_compactor_state_argsStandardScheme();
+      }
+    }
+
+    private static class update_compactor_state_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<update_compactor_state_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, update_compactor_state_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: // CR
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.cr = new CompactionInfoStruct();
+                struct.cr.read(iprot);
+                struct.setCrIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // TXN_ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+                struct.txn_id = iprot.readI64();
+                struct.setTxn_idIsSet(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, update_compactor_state_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.cr != null) {
+          oprot.writeFieldBegin(CR_FIELD_DESC);
+          struct.cr.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldBegin(TXN_ID_FIELD_DESC);
+        oprot.writeI64(struct.txn_id);
+        oprot.writeFieldEnd();
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class update_compactor_state_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public update_compactor_state_argsTupleScheme getScheme() {
+        return new update_compactor_state_argsTupleScheme();
+      }
+    }
+
+    private static class update_compactor_state_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<update_compactor_state_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, update_compactor_state_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.isSetCr()) {
+          optionals.set(0);
+        }
+        if (struct.isSetTxn_id()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetCr()) {
+          struct.cr.write(oprot);
+        }
+        if (struct.isSetTxn_id()) {
+          oprot.writeI64(struct.txn_id);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, update_compactor_state_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.cr = new CompactionInfoStruct();
+          struct.cr.read(iprot);
+          struct.setCrIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.txn_id = iprot.readI64();
+          struct.setTxn_idIsSet(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 update_compactor_state_result implements org.apache.thrift.TBase<update_compactor_state_result, update_compactor_state_result._Fields>, java.io.Serializable, Cloneable, Comparable<update_compactor_state_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("update_compactor_state_result");
+
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new update_compactor_state_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new update_compactor_state_resultTupleSchemeFactory();
+
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+;
+
+      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) {
+          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;
+      }
+    }
+    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);
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(update_compactor_state_result.class, metaDataMap);
+    }
+
+    public update_compactor_state_result() {
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public update_compactor_state_result(update_compactor_state_result other) {
+    }
+
+    public update_compactor_state_result deepCopy() {
+      return new update_compactor_state_result(this);
+    }
+
+    @Override
+    public void clear() {
+    }
+
+    public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
+      switch (field) {
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      }
+      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) {
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof update_compactor_state_result)
+        return this.equals((update_compactor_state_result)that);
+      return false;
+    }
+
+    public boolean equals(update_compactor_state_result that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(update_compactor_state_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      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("update_compactor_state_result(");
+      boolean first = true;
+
+      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 update_compactor_state_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public update_compactor_state_resultStandardScheme getScheme() {
+        return new update_compactor_state_resultStandardScheme();
+      }
+    }
+
+    private static class update_compactor_state_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<update_compactor_state_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, update_compactor_state_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) {
+            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, update_compactor_state_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class update_compactor_state_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public update_compactor_state_resultTupleScheme getScheme() {
+        return new update_compactor_state_resultTupleScheme();
+      }
+    }
+
+    private static class update_compactor_state_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<update_compactor_state_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, update_compactor_state_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, update_compactor_state_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+      }
+    }
+
+    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 find_columns_with_stats_args implements org.apache.thrift.TBase<find_columns_with_stats_args, find_columns_with_stats_args._Fields>, java.io.Serializable, Cloneable, Comparable<find_columns_with_stats_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("find_columns_with_stats_args");
+
+    private static final org.apache.thrift.protocol.TField CR_FIELD_DESC = new org.apache.thrift.protocol.TField("cr", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new find_columns_with_stats_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new find_columns_with_stats_argsTupleSchemeFactory();
+
+    private @org.apache.thrift.annotation.Nullable CompactionInfoStruct cr; // 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 {
+      CR((short)1, "cr");
+
+      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: // CR
+            return CR;
+          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.CR, new org.apache.thrift.meta_data.FieldMetaData("cr", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, CompactionInfoStruct.class)));
-      tmpMap.put(_Fields.TXN_ID, new org.apache.thrift.meta_data.FieldMetaData("txn_id", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(update_compactor_state_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(find_columns_with_stats_args.class, metaDataMap);
     }
 
-    public update_compactor_state_args() {
+    public find_columns_with_stats_args() {
     }
 
-    public update_compactor_state_args(
-      CompactionInfoStruct cr,
-      long txn_id)
+    public find_columns_with_stats_args(
+      CompactionInfoStruct cr)
     {
       this();
       this.cr = cr;
-      this.txn_id = txn_id;
-      setTxn_idIsSet(true);
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public update_compactor_state_args(update_compactor_state_args other) {
-      __isset_bitfield = other.__isset_bitfield;
+    public find_columns_with_stats_args(find_columns_with_stats_args other) {
       if (other.isSetCr()) {
         this.cr = new CompactionInfoStruct(other.cr);
       }
-      this.txn_id = other.txn_id;
     }
 
-    public update_compactor_state_args deepCopy() {
-      return new update_compactor_state_args(this);
+    public find_columns_with_stats_args deepCopy() {
+      return new find_columns_with_stats_args(this);
     }
 
     @Override
     public void clear() {
       this.cr = null;
-      setTxn_idIsSet(false);
-      this.txn_id = 0;
     }
 
     @org.apache.thrift.annotation.Nullable
@@ -240981,28 +241835,6 @@ package org.apache.hadoop.hive.metastore.api;
       }
     }
 
-    public long getTxn_id() {
-      return this.txn_id;
-    }
-
-    public void setTxn_id(long txn_id) {
-      this.txn_id = txn_id;
-      setTxn_idIsSet(true);
-    }
-
-    public void unsetTxn_id() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __TXN_ID_ISSET_ID);
-    }
-
-    /** Returns true if field txn_id is set (has been assigned a value) and false otherwise */
-    public boolean isSetTxn_id() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __TXN_ID_ISSET_ID);
-    }
-
-    public void setTxn_idIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __TXN_ID_ISSET_ID, value);
-    }
-
     public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
       switch (field) {
       case CR:
@@ -241013,14 +241845,6 @@ package org.apache.hadoop.hive.metastore.api;
         }
         break;
 
-      case TXN_ID:
-        if (value == null) {
-          unsetTxn_id();
-        } else {
-          setTxn_id((java.lang.Long)value);
-        }
-        break;
-
       }
     }
 
@@ -241030,9 +241854,6 @@ package org.apache.hadoop.hive.metastore.api;
       case CR:
         return getCr();
 
-      case TXN_ID:
-        return getTxn_id();
-
       }
       throw new java.lang.IllegalStateException();
     }
@@ -241046,20 +241867,18 @@ package org.apache.hadoop.hive.metastore.api;
       switch (field) {
       case CR:
         return isSetCr();
-      case TXN_ID:
-        return isSetTxn_id();
       }
       throw new java.lang.IllegalStateException();
     }
 
     @Override
     public boolean equals(java.lang.Object that) {
-      if (that instanceof update_compactor_state_args)
-        return this.equals((update_compactor_state_args)that);
+      if (that instanceof find_columns_with_stats_args)
+        return this.equals((find_columns_with_stats_args)that);
       return false;
     }
 
-    public boolean equals(update_compactor_state_args that) {
+    public boolean equals(find_columns_with_stats_args that) {
       if (that == null)
         return false;
       if (this == that)
@@ -241074,15 +241893,6 @@ package org.apache.hadoop.hive.metastore.api;
           return false;
       }
 
-      boolean this_present_txn_id = true;
-      boolean that_present_txn_id = true;
-      if (this_present_txn_id || that_present_txn_id) {
-        if (!(this_present_txn_id && that_present_txn_id))
-          return false;
-        if (this.txn_id != that.txn_id)
-          return false;
-      }
-
       return true;
     }
 
@@ -241094,13 +241904,11 @@ package org.apache.hadoop.hive.metastore.api;
       if (isSetCr())
         hashCode = hashCode * 8191 + cr.hashCode();
 
-      hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(txn_id);
-
       return hashCode;
     }
 
     @Override
-    public int compareTo(update_compactor_state_args other) {
+    public int compareTo(find_columns_with_stats_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -241117,16 +241925,6 @@ package org.apache.hadoop.hive.metastore.api;
           return lastComparison;
         }
       }
-      lastComparison = java.lang.Boolean.compare(isSetTxn_id(), other.isSetTxn_id());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetTxn_id()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.txn_id, other.txn_id);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
       return 0;
     }
 
@@ -241145,7 +241943,7 @@ package org.apache.hadoop.hive.metastore.api;
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("update_compactor_state_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("find_columns_with_stats_args(");
       boolean first = true;
 
       sb.append("cr:");
@@ -241155,10 +241953,6 @@ package org.apache.hadoop.hive.metastore.api;
         sb.append(this.cr);
       }
       first = false;
-      if (!first) sb.append(", ");
-      sb.append("txn_id:");
-      sb.append(this.txn_id);
-      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -241181,23 +241975,21 @@ package org.apache.hadoop.hive.metastore.api;
 
     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 update_compactor_state_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public update_compactor_state_argsStandardScheme getScheme() {
-        return new update_compactor_state_argsStandardScheme();
+    private static class find_columns_with_stats_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public find_columns_with_stats_argsStandardScheme getScheme() {
+        return new find_columns_with_stats_argsStandardScheme();
       }
     }
 
-    private static class update_compactor_state_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<update_compactor_state_args> {
+    private static class find_columns_with_stats_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<find_columns_with_stats_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, update_compactor_state_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, find_columns_with_stats_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -241216,14 +242008,6 @@ package org.apache.hadoop.hive.metastore.api;
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
-            case 2: // TXN_ID
-              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
-                struct.txn_id = iprot.readI64();
-                struct.setTxn_idIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
             default:
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
           }
@@ -241233,7 +242017,7 @@ package org.apache.hadoop.hive.metastore.api;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, update_compactor_state_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, find_columns_with_stats_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -241242,55 +242026,42 @@ package org.apache.hadoop.hive.metastore.api;
           struct.cr.write(oprot);
           oprot.writeFieldEnd();
         }
-        oprot.writeFieldBegin(TXN_ID_FIELD_DESC);
-        oprot.writeI64(struct.txn_id);
-        oprot.writeFieldEnd();
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
 
     }
 
-    private static class update_compactor_state_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public update_compactor_state_argsTupleScheme getScheme() {
-        return new update_compactor_state_argsTupleScheme();
+    private static class find_columns_with_stats_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public find_columns_with_stats_argsTupleScheme getScheme() {
+        return new find_columns_with_stats_argsTupleScheme();
       }
     }
 
-    private static class update_compactor_state_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<update_compactor_state_args> {
+    private static class find_columns_with_stats_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<find_columns_with_stats_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, update_compactor_state_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, find_columns_with_stats_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.isSetCr()) {
           optionals.set(0);
         }
-        if (struct.isSetTxn_id()) {
-          optionals.set(1);
-        }
-        oprot.writeBitSet(optionals, 2);
+        oprot.writeBitSet(optionals, 1);
         if (struct.isSetCr()) {
           struct.cr.write(oprot);
         }
-        if (struct.isSetTxn_id()) {
-          oprot.writeI64(struct.txn_id);
-        }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, update_compactor_state_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, find_columns_with_stats_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);
+        java.util.BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           struct.cr = new CompactionInfoStruct();
           struct.cr.read(iprot);
           struct.setCrIsSet(true);
         }
-        if (incoming.get(1)) {
-          struct.txn_id = iprot.readI64();
-          struct.setTxn_idIsSet(true);
-        }
       }
     }
 
@@ -241299,17 +242070,19 @@ package org.apache.hadoop.hive.metastore.api;
     }
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class update_compactor_state_result implements org.apache.thrift.TBase<update_compactor_state_result, update_compactor_state_result._Fields>, java.io.Serializable, Cloneable, Comparable<update_compactor_state_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("update_compactor_state_result");
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class find_columns_with_stats_result implements org.apache.thrift.TBase<find_columns_with_stats_result, find_columns_with_stats_result._Fields>, java.io.Serializable, Cloneable, Comparable<find_columns_with_stats_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("find_columns_with_stats_result");
 
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new update_compactor_state_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new update_compactor_state_resultTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new find_columns_with_stats_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new find_columns_with_stats_resultTupleSchemeFactory();
 
+    private @org.apache.thrift.annotation.Nullable java.util.List<java.lang.String> success; // 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 {
-;
+      SUCCESS((short)0, "success");
 
       private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
 
@@ -241325,6 +242098,8 @@ package org.apache.hadoop.hive.metastore.api;
       @org.apache.thrift.annotation.Nullable
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
           default:
             return null;
         }
@@ -241364,38 +242139,106 @@ package org.apache.hadoop.hive.metastore.api;
         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.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(update_compactor_state_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(find_columns_with_stats_result.class, metaDataMap);
     }
 
-    public update_compactor_state_result() {
+    public find_columns_with_stats_result() {
+    }
+
+    public find_columns_with_stats_result(
+      java.util.List<java.lang.String> success)
+    {
+      this();
+      this.success = success;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public update_compactor_state_result(update_compactor_state_result other) {
+    public find_columns_with_stats_result(find_columns_with_stats_result other) {
+      if (other.isSetSuccess()) {
+        java.util.List<java.lang.String> __this__success = new java.util.ArrayList<java.lang.String>(other.success);
+        this.success = __this__success;
+      }
     }
 
-    public update_compactor_state_result deepCopy() {
-      return new update_compactor_state_result(this);
+    public find_columns_with_stats_result deepCopy() {
+      return new find_columns_with_stats_result(this);
     }
 
     @Override
     public void clear() {
+      this.success = null;
+    }
+
+    public int getSuccessSize() {
+      return (this.success == null) ? 0 : this.success.size();
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.Iterator<java.lang.String> getSuccessIterator() {
+      return (this.success == null) ? null : this.success.iterator();
+    }
+
+    public void addToSuccess(java.lang.String elem) {
+      if (this.success == null) {
+        this.success = new java.util.ArrayList<java.lang.String>();
+      }
+      this.success.add(elem);
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.List<java.lang.String> getSuccess() {
+      return this.success;
+    }
+
+    public void setSuccess(@org.apache.thrift.annotation.Nullable java.util.List<java.lang.String> success) {
+      this.success = success;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
     }
 
     public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
       switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((java.util.List<java.lang.String>)value);
+        }
+        break;
+
       }
     }
 
     @org.apache.thrift.annotation.Nullable
     public java.lang.Object getFieldValue(_Fields field) {
       switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
       }
       throw new java.lang.IllegalStateException();
     }
@@ -241407,23 +242250,34 @@ package org.apache.hadoop.hive.metastore.api;
       }
 
       switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
       }
       throw new java.lang.IllegalStateException();
     }
 
     @Override
     public boolean equals(java.lang.Object that) {
-      if (that instanceof update_compactor_state_result)
-        return this.equals((update_compactor_state_result)that);
+      if (that instanceof find_columns_with_stats_result)
+        return this.equals((find_columns_with_stats_result)that);
       return false;
     }
 
-    public boolean equals(update_compactor_state_result that) {
+    public boolean equals(find_columns_with_stats_result that) {
       if (that == null)
         return false;
       if (this == that)
         return true;
 
+      boolean this_present_success = true && this.isSetSuccess();
+      boolean that_present_success = true && that.isSetSuccess();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
       return true;
     }
 
@@ -241431,17 +242285,31 @@ package org.apache.hadoop.hive.metastore.api;
     public int hashCode() {
       int hashCode = 1;
 
+      hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
+      if (isSetSuccess())
+        hashCode = hashCode * 8191 + success.hashCode();
+
       return hashCode;
     }
 
     @Override
-    public int compareTo(update_compactor_state_result other) {
+    public int compareTo(find_columns_with_stats_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
 
+      lastComparison = java.lang.Boolean.compare(isSetSuccess(), other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -241460,9 +242328,16 @@ package org.apache.hadoop.hive.metastore.api;
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("update_compactor_state_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("find_columns_with_stats_result(");
       boolean first = true;
 
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -241488,15 +242363,15 @@ package org.apache.hadoop.hive.metastore.api;
       }
     }
 
-    private static class update_compactor_state_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public update_compactor_state_resultStandardScheme getScheme() {
-        return new update_compactor_state_resultStandardScheme();
+    private static class find_columns_with_stats_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public find_columns_with_stats_resultStandardScheme getScheme() {
+        return new find_columns_with_stats_resultStandardScheme();
       }
     }
 
-    private static class update_compactor_state_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<update_compactor_state_result> {
+    private static class find_columns_with_stats_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<find_columns_with_stats_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, update_compactor_state_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, find_columns_with_stats_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -241506,6 +242381,24 @@ package org.apache.hadoop.hive.metastore.api;
             break;
           }
           switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+                {
+                  org.apache.thrift.protocol.TList _list2010 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<java.lang.String>(_list2010.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem2011;
+                  for (int _i2012 = 0; _i2012 < _list2010.size; ++_i2012)
+                  {
+                    _elem2011 = iprot.readString();
+                    struct.success.add(_elem2011);
+                  }
+                  iprot.readListEnd();
+                }
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
             default:
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
           }
@@ -241515,32 +242408,72 @@ package org.apache.hadoop.hive.metastore.api;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, update_compactor_state_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, find_columns_with_stats_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          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 _iter2013 : struct.success)
+            {
+              oprot.writeString(_iter2013);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
 
     }
 
-    private static class update_compactor_state_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public update_compactor_state_resultTupleScheme getScheme() {
-        return new update_compactor_state_resultTupleScheme();
+    private static class find_columns_with_stats_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public find_columns_with_stats_resultTupleScheme getScheme() {
+        return new find_columns_with_stats_resultTupleScheme();
       }
     }
 
-    private static class update_compactor_state_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<update_compactor_state_result> {
+    private static class find_columns_with_stats_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<find_columns_with_stats_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, update_compactor_state_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, find_columns_with_stats_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetSuccess()) {
+          {
+            oprot.writeI32(struct.success.size());
+            for (java.lang.String _iter2014 : struct.success)
+            {
+              oprot.writeString(_iter2014);
+            }
+          }
+        }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, update_compactor_state_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, find_columns_with_stats_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)) {
+          {
+            org.apache.thrift.protocol.TList _list2015 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.success = new java.util.ArrayList<java.lang.String>(_list2015.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem2016;
+            for (int _i2017 = 0; _i2017 < _list2015.size; ++_i2017)
+            {
+              _elem2016 = iprot.readString();
+              struct.success.add(_elem2016);
+            }
+          }
+          struct.setSuccessIsSet(true);
+        }
       }
     }
 
@@ -241549,13 +242482,13 @@ package org.apache.hadoop.hive.metastore.api;
     }
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class find_columns_with_stats_args implements org.apache.thrift.TBase<find_columns_with_stats_args, find_columns_with_stats_args._Fields>, java.io.Serializable, Cloneable, Comparable<find_columns_with_stats_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("find_columns_with_stats_args");
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class mark_cleaned_args implements org.apache.thrift.TBase<mark_cleaned_args, mark_cleaned_args._Fields>, java.io.Serializable, Cloneable, Comparable<mark_cleaned_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("mark_cleaned_args");
 
     private static final org.apache.thrift.protocol.TField CR_FIELD_DESC = new org.apache.thrift.protocol.TField("cr", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new find_columns_with_stats_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new find_columns_with_stats_argsTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new mark_cleaned_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new mark_cleaned_argsTupleSchemeFactory();
 
     private @org.apache.thrift.annotation.Nullable CompactionInfoStruct cr; // required
 
@@ -241626,13 +242559,13 @@ package org.apache.hadoop.hive.metastore.api;
       tmpMap.put(_Fields.CR, new org.apache.thrift.meta_data.FieldMetaData("cr", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, CompactionInfoStruct.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(find_columns_with_stats_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(mark_cleaned_args.class, metaDataMap);
     }
 
-    public find_columns_with_stats_args() {
+    public mark_cleaned_args() {
     }
 
-    public find_columns_with_stats_args(
+    public mark_cleaned_args(
       CompactionInfoStruct cr)
     {
       this();
@@ -241642,14 +242575,14 @@ package org.apache.hadoop.hive.metastore.api;
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public find_columns_with_stats_args(find_columns_with_stats_args other) {
+    public mark_cleaned_args(mark_cleaned_args other) {
       if (other.isSetCr()) {
         this.cr = new CompactionInfoStruct(other.cr);
       }
     }
 
-    public find_columns_with_stats_args deepCopy() {
-      return new find_columns_with_stats_args(this);
+    public mark_cleaned_args deepCopy() {
+      return new mark_cleaned_args(this);
     }
 
     @Override
@@ -241719,12 +242652,12 @@ package org.apache.hadoop.hive.metastore.api;
 
     @Override
     public boolean equals(java.lang.Object that) {
-      if (that instanceof find_columns_with_stats_args)
-        return this.equals((find_columns_with_stats_args)that);
+      if (that instanceof mark_cleaned_args)
+        return this.equals((mark_cleaned_args)that);
       return false;
     }
 
-    public boolean equals(find_columns_with_stats_args that) {
+    public boolean equals(mark_cleaned_args that) {
       if (that == null)
         return false;
       if (this == that)
@@ -241754,7 +242687,7 @@ package org.apache.hadoop.hive.metastore.api;
     }
 
     @Override
-    public int compareTo(find_columns_with_stats_args other) {
+    public int compareTo(mark_cleaned_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -241789,7 +242722,7 @@ package org.apache.hadoop.hive.metastore.api;
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("find_columns_with_stats_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("mark_cleaned_args(");
       boolean first = true;
 
       sb.append("cr:");
@@ -241827,15 +242760,15 @@ package org.apache.hadoop.hive.metastore.api;
       }
     }
 
-    private static class find_columns_with_stats_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public find_columns_with_stats_argsStandardScheme getScheme() {
-        return new find_columns_with_stats_argsStandardScheme();
+    private static class mark_cleaned_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public mark_cleaned_argsStandardScheme getScheme() {
+        return new mark_cleaned_argsStandardScheme();
       }
     }
 
-    private static class find_columns_with_stats_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<find_columns_with_stats_args> {
+    private static class mark_cleaned_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<mark_cleaned_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, find_columns_with_stats_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, mark_cleaned_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -241863,7 +242796,7 @@ package org.apache.hadoop.hive.metastore.api;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, find_columns_with_stats_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, mark_cleaned_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -241878,16 +242811,16 @@ package org.apache.hadoop.hive.metastore.api;
 
     }
 
-    private static class find_columns_with_stats_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public find_columns_with_stats_argsTupleScheme getScheme() {
-        return new find_columns_with_stats_argsTupleScheme();
+    private static class mark_cleaned_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public mark_cleaned_argsTupleScheme getScheme() {
+        return new mark_cleaned_argsTupleScheme();
       }
     }
 
-    private static class find_columns_with_stats_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<find_columns_with_stats_args> {
+    private static class mark_cleaned_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<mark_cleaned_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, find_columns_with_stats_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, mark_cleaned_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.isSetCr()) {
@@ -241900,7 +242833,7 @@ package org.apache.hadoop.hive.metastore.api;
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, find_columns_with_stats_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, mark_cleaned_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
@@ -241916,19 +242849,19 @@ package org.apache.hadoop.hive.metastore.api;
     }
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class find_columns_with_stats_result implements org.apache.thrift.TBase<find_columns_with_stats_result, find_columns_with_stats_result._Fields>, java.io.Serializable, Cloneable, Comparable<find_columns_with_stats_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("find_columns_with_stats_result");
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class mark_cleaned_result implements org.apache.thrift.TBase<mark_cleaned_result, mark_cleaned_result._Fields>, java.io.Serializable, Cloneable, Comparable<mark_cleaned_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("mark_cleaned_result");
 
-    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0);
+    private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new find_columns_with_stats_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new find_columns_with_stats_resultTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new mark_cleaned_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new mark_cleaned_resultTupleSchemeFactory();
 
-    private @org.apache.thrift.annotation.Nullable java.util.List<java.lang.String> success; // required
+    private @org.apache.thrift.annotation.Nullable MetaException o1; // 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 {
-      SUCCESS((short)0, "success");
+      O1((short)1, "o1");
 
       private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
 
@@ -241944,8 +242877,8 @@ package org.apache.hadoop.hive.metastore.api;
       @org.apache.thrift.annotation.Nullable
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
-          case 0: // SUCCESS
-            return SUCCESS;
+          case 1: // O1
+            return O1;
           default:
             return null;
         }
@@ -241990,89 +242923,71 @@ package org.apache.hadoop.hive.metastore.api;
     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.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
-              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+      tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, MetaException.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(find_columns_with_stats_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(mark_cleaned_result.class, metaDataMap);
     }
 
-    public find_columns_with_stats_result() {
+    public mark_cleaned_result() {
     }
 
-    public find_columns_with_stats_result(
-      java.util.List<java.lang.String> success)
+    public mark_cleaned_result(
+      MetaException o1)
     {
       this();
-      this.success = success;
+      this.o1 = o1;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public find_columns_with_stats_result(find_columns_with_stats_result other) {
-      if (other.isSetSuccess()) {
-        java.util.List<java.lang.String> __this__success = new java.util.ArrayList<java.lang.String>(other.success);
-        this.success = __this__success;
+    public mark_cleaned_result(mark_cleaned_result other) {
+      if (other.isSetO1()) {
+        this.o1 = new MetaException(other.o1);
       }
     }
 
-    public find_columns_with_stats_result deepCopy() {
-      return new find_columns_with_stats_result(this);
+    public mark_cleaned_result deepCopy() {
+      return new mark_cleaned_result(this);
     }
 
     @Override
     public void clear() {
-      this.success = null;
-    }
-
-    public int getSuccessSize() {
-      return (this.success == null) ? 0 : this.success.size();
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.Iterator<java.lang.String> getSuccessIterator() {
-      return (this.success == null) ? null : this.success.iterator();
-    }
-
-    public void addToSuccess(java.lang.String elem) {
-      if (this.success == null) {
-        this.success = new java.util.ArrayList<java.lang.String>();
-      }
-      this.success.add(elem);
+      this.o1 = null;
     }
 
     @org.apache.thrift.annotation.Nullable
-    public java.util.List<java.lang.String> getSuccess() {
-      return this.success;
+    public MetaException getO1() {
+      return this.o1;
     }
 
-    public void setSuccess(@org.apache.thrift.annotation.Nullable java.util.List<java.lang.String> success) {
-      this.success = success;
+    public void setO1(@org.apache.thrift.annotation.Nullable MetaException o1) {
+      this.o1 = o1;
     }
 
-    public void unsetSuccess() {
-      this.success = null;
+    public void unsetO1() {
+      this.o1 = null;
     }
 
-    /** Returns true if field success is set (has been assigned a value) and false otherwise */
-    public boolean isSetSuccess() {
-      return this.success != null;
+    /** Returns true if field o1 is set (has been assigned a value) and false otherwise */
+    public boolean isSetO1() {
+      return this.o1 != null;
     }
 
-    public void setSuccessIsSet(boolean value) {
+    public void setO1IsSet(boolean value) {
       if (!value) {
-        this.success = null;
+        this.o1 = null;
       }
     }
 
     public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
       switch (field) {
-      case SUCCESS:
+      case O1:
         if (value == null) {
-          unsetSuccess();
+          unsetO1();
         } else {
-          setSuccess((java.util.List<java.lang.String>)value);
+          setO1((MetaException)value);
         }
         break;
 
@@ -242082,8 +242997,8 @@ package org.apache.hadoop.hive.metastore.api;
     @org.apache.thrift.annotation.Nullable
     public java.lang.Object getFieldValue(_Fields field) {
       switch (field) {
-      case SUCCESS:
-        return getSuccess();
+      case O1:
+        return getO1();
 
       }
       throw new java.lang.IllegalStateException();
@@ -242096,31 +243011,31 @@ package org.apache.hadoop.hive.metastore.api;
       }
 
       switch (field) {
-      case SUCCESS:
-        return isSetSuccess();
+      case O1:
+        return isSetO1();
       }
       throw new java.lang.IllegalStateException();
     }
 
     @Override
     public boolean equals(java.lang.Object that) {
-      if (that instanceof find_columns_with_stats_result)
-        return this.equals((find_columns_with_stats_result)that);
+      if (that instanceof mark_cleaned_result)
+        return this.equals((mark_cleaned_result)that);
       return false;
     }
 
-    public boolean equals(find_columns_with_stats_result that) {
+    public boolean equals(mark_cleaned_result that) {
       if (that == null)
         return false;
       if (this == that)
         return true;
 
-      boolean this_present_success = true && this.isSetSuccess();
-      boolean that_present_success = true && that.isSetSuccess();
-      if (this_present_success || that_present_success) {
-        if (!(this_present_success && that_present_success))
+      boolean this_present_o1 = true && this.isSetO1();
+      boolean that_present_o1 = true && that.isSetO1();
+      if (this_present_o1 || that_present_o1) {
+        if (!(this_present_o1 && that_present_o1))
           return false;
-        if (!this.success.equals(that.success))
+        if (!this.o1.equals(that.o1))
           return false;
       }
 
@@ -242131,27 +243046,27 @@ package org.apache.hadoop.hive.metastore.api;
     public int hashCode() {
       int hashCode = 1;
 
-      hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
-      if (isSetSuccess())
-        hashCode = hashCode * 8191 + success.hashCode();
+      hashCode = hashCode * 8191 + ((isSetO1()) ? 131071 : 524287);
+      if (isSetO1())
+        hashCode = hashCode * 8191 + o1.hashCode();
 
       return hashCode;
     }
 
     @Override
-    public int compareTo(find_columns_with_stats_result other) {
+    public int compareTo(mark_cleaned_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
 
-      lastComparison = java.lang.Boolean.compare(isSetSuccess(), other.isSetSuccess());
+      lastComparison = java.lang.Boolean.compare(isSetO1(), other.isSetO1());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetSuccess()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+      if (isSetO1()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, other.o1);
         if (lastComparison != 0) {
           return lastComparison;
         }
@@ -242174,14 +243089,14 @@ package org.apache.hadoop.hive.metastore.api;
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("find_columns_with_stats_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("mark_cleaned_result(");
       boolean first = true;
 
-      sb.append("success:");
-      if (this.success == null) {
+      sb.append("o1:");
+      if (this.o1 == null) {
         sb.append("null");
       } else {
-        sb.append(this.success);
+        sb.append(this.o1);
       }
       first = false;
       sb.append(")");
@@ -242209,15 +243124,15 @@ package org.apache.hadoop.hive.metastore.api;
       }
     }
 
-    private static class find_columns_with_stats_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public find_columns_with_stats_resultStandardScheme getScheme() {
-        return new find_columns_with_stats_resultStandardScheme();
+    private static class mark_cleaned_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public mark_cleaned_resultStandardScheme getScheme() {
+        return new mark_cleaned_resultStandardScheme();
       }
     }
 
-    private static class find_columns_with_stats_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<find_columns_with_stats_result> {
+    private static class mark_cleaned_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<mark_cleaned_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, find_columns_with_stats_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, mark_cleaned_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -242227,20 +243142,11 @@ package org.apache.hadoop.hive.metastore.api;
             break;
           }
           switch (schemeField.id) {
-            case 0: // SUCCESS
-              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
-                {
-                  org.apache.thrift.protocol.TList _list2010 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<java.lang.String>(_list2010.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem2011;
-                  for (int _i2012 = 0; _i2012 < _list2010.size; ++_i2012)
-                  {
-                    _elem2011 = iprot.readString();
-                    struct.success.add(_elem2011);
-                  }
-                  iprot.readListEnd();
-                }
-                struct.setSuccessIsSet(true);
+            case 1: // O1
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.o1 = new MetaException();
+                struct.o1.read(iprot);
+                struct.setO1IsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
@@ -242254,20 +243160,13 @@ package org.apache.hadoop.hive.metastore.api;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, find_columns_with_stats_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, mark_cleaned_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.success != null) {
-          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 _iter2013 : struct.success)
-            {
-              oprot.writeString(_iter2013);
-            }
-            oprot.writeListEnd();
-          }
+        if (struct.o1 != null) {
+          oprot.writeFieldBegin(O1_FIELD_DESC);
+          struct.o1.write(oprot);
           oprot.writeFieldEnd();
         }
         oprot.writeFieldStop();
@@ -242276,49 +243175,35 @@ package org.apache.hadoop.hive.metastore.api;
 
     }
 
-    private static class find_columns_with_stats_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public find_columns_with_stats_resultTupleScheme getScheme() {
-        return new find_columns_with_stats_resultTupleScheme();
+    private static class mark_cleaned_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public mark_cleaned_resultTupleScheme getScheme() {
+        return new mark_cleaned_resultTupleScheme();
       }
     }
 
-    private static class find_columns_with_stats_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<find_columns_with_stats_result> {
+    private static class mark_cleaned_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<mark_cleaned_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, find_columns_with_stats_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, mark_cleaned_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet optionals = new java.util.BitSet();
-        if (struct.isSetSuccess()) {
+        if (struct.isSetO1()) {
           optionals.set(0);
         }
         oprot.writeBitSet(optionals, 1);
-        if (struct.isSetSuccess()) {
-          {
-            oprot.writeI32(struct.success.size());
-            for (java.lang.String _iter2014 : struct.success)
-            {
-              oprot.writeString(_iter2014);
-            }
-          }
+        if (struct.isSetO1()) {
+          struct.o1.write(oprot);
         }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, find_columns_with_stats_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, mark_cleaned_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)) {
-          {
-            org.apache.thrift.protocol.TList _list2015 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.success = new java.util.ArrayList<java.lang.String>(_list2015.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem2016;
-            for (int _i2017 = 0; _i2017 < _list2015.size; ++_i2017)
-            {
-              _elem2016 = iprot.readString();
-              struct.success.add(_elem2016);
-            }
-          }
-          struct.setSuccessIsSet(true);
+          struct.o1 = new MetaException();
+          struct.o1.read(iprot);
+          struct.setO1IsSet(true);
         }
       }
     }
@@ -242328,13 +243213,13 @@ package org.apache.hadoop.hive.metastore.api;
     }
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class mark_cleaned_args implements org.apache.thrift.TBase<mark_cleaned_args, mark_cleaned_args._Fields>, java.io.Serializable, Cloneable, Comparable<mark_cleaned_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("mark_cleaned_args");
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class mark_compacted_args implements org.apache.thrift.TBase<mark_compacted_args, mark_compacted_args._Fields>, java.io.Serializable, Cloneable, Comparable<mark_compacted_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("mark_compacted_args");
 
     private static final org.apache.thrift.protocol.TField CR_FIELD_DESC = new org.apache.thrift.protocol.TField("cr", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new mark_cleaned_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new mark_cleaned_argsTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new mark_compacted_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new mark_compacted_argsTupleSchemeFactory();
 
     private @org.apache.thrift.annotation.Nullable CompactionInfoStruct cr; // required
 
@@ -242405,13 +243290,13 @@ package org.apache.hadoop.hive.metastore.api;
       tmpMap.put(_Fields.CR, new org.apache.thrift.meta_data.FieldMetaData("cr", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, CompactionInfoStruct.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(mark_cleaned_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(mark_compacted_args.class, metaDataMap);
     }
 
-    public mark_cleaned_args() {
+    public mark_compacted_args() {
     }
 
-    public mark_cleaned_args(
+    public mark_compacted_args(
       CompactionInfoStruct cr)
     {
       this();
@@ -242421,14 +243306,14 @@ package org.apache.hadoop.hive.metastore.api;
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public mark_cleaned_args(mark_cleaned_args other) {
+    public mark_compacted_args(mark_compacted_args other) {
       if (other.isSetCr()) {
         this.cr = new CompactionInfoStruct(other.cr);
       }
     }
 
-    public mark_cleaned_args deepCopy() {
-      return new mark_cleaned_args(this);
+    public mark_compacted_args deepCopy() {
+      return new mark_compacted_args(this);
     }
 
     @Override
@@ -242498,12 +243383,12 @@ package org.apache.hadoop.hive.metastore.api;
 
     @Override
     public boolean equals(java.lang.Object that) {
-      if (that instanceof mark_cleaned_args)
-        return this.equals((mark_cleaned_args)that);
+      if (that instanceof mark_compacted_args)
+        return this.equals((mark_compacted_args)that);
       return false;
     }
 
-    public boolean equals(mark_cleaned_args that) {
+    public boolean equals(mark_compacted_args that) {
       if (that == null)
         return false;
       if (this == that)
@@ -242533,7 +243418,7 @@ package org.apache.hadoop.hive.metastore.api;
     }
 
     @Override
-    public int compareTo(mark_cleaned_args other) {
+    public int compareTo(mark_compacted_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -242568,7 +243453,7 @@ package org.apache.hadoop.hive.metastore.api;
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("mark_cleaned_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("mark_compacted_args(");
       boolean first = true;
 
       sb.append("cr:");
@@ -242606,15 +243491,15 @@ package org.apache.hadoop.hive.metastore.api;
       }
     }
 
-    private static class mark_cleaned_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public mark_cleaned_argsStandardScheme getScheme() {
-        return new mark_cleaned_argsStandardScheme();
+    private static class mark_compacted_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public mark_compacted_argsStandardScheme getScheme() {
+        return new mark_compacted_argsStandardScheme();
       }
     }
 
-    private static class mark_cleaned_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<mark_cleaned_args> {
+    private static class mark_compacted_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<mark_compacted_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, mark_cleaned_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, mark_compacted_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -242642,7 +243527,7 @@ package org.apache.hadoop.hive.metastore.api;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, mark_cleaned_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, mark_compacted_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -242657,16 +243542,16 @@ package org.apache.hadoop.hive.metastore.api;
 
     }
 
-    private static class mark_cleaned_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public mark_cleaned_argsTupleScheme getScheme() {
-        return new mark_cleaned_argsTupleScheme();
+    private static class mark_compacted_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public mark_compacted_argsTupleScheme getScheme() {
+        return new mark_compacted_argsTupleScheme();
       }
     }
 
-    private static class mark_cleaned_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<mark_cleaned_args> {
+    private static class mark_compacted_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<mark_compacted_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, mark_cleaned_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, mark_compacted_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.isSetCr()) {
@@ -242679,7 +243564,7 @@ package org.apache.hadoop.hive.metastore.api;
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, mark_cleaned_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, mark_compacted_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
@@ -242695,13 +243580,13 @@ package org.apache.hadoop.hive.metastore.api;
     }
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class mark_cleaned_result implements org.apache.thrift.TBase<mark_cleaned_result, mark_cleaned_result._Fields>, java.io.Serializable, Cloneable, Comparable<mark_cleaned_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("mark_cleaned_result");
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class mark_compacted_result implements org.apache.thrift.TBase<mark_compacted_result, mark_compacted_result._Fields>, java.io.Serializable, Cloneable, Comparable<mark_compacted_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("mark_compacted_result");
 
     private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new mark_cleaned_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new mark_cleaned_resultTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new mark_compacted_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new mark_compacted_resultTupleSchemeFactory();
 
     private @org.apache.thrift.annotation.Nullable MetaException o1; // required
 
@@ -242772,13 +243657,13 @@ package org.apache.hadoop.hive.metastore.api;
       tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, MetaException.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(mark_cleaned_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(mark_compacted_result.class, metaDataMap);
     }
 
-    public mark_cleaned_result() {
+    public mark_compacted_result() {
     }
 
-    public mark_cleaned_result(
+    public mark_compacted_result(
       MetaException o1)
     {
       this();
@@ -242788,14 +243673,14 @@ package org.apache.hadoop.hive.metastore.api;
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public mark_cleaned_result(mark_cleaned_result other) {
+    public mark_compacted_result(mark_compacted_result other) {
       if (other.isSetO1()) {
         this.o1 = new MetaException(other.o1);
       }
     }
 
-    public mark_cleaned_result deepCopy() {
-      return new mark_cleaned_result(this);
+    public mark_compacted_result deepCopy() {
+      return new mark_compacted_result(this);
     }
 
     @Override
@@ -242865,12 +243750,12 @@ package org.apache.hadoop.hive.metastore.api;
 
     @Override
     public boolean equals(java.lang.Object that) {
-      if (that instanceof mark_cleaned_result)
-        return this.equals((mark_cleaned_result)that);
+      if (that instanceof mark_compacted_result)
+        return this.equals((mark_compacted_result)that);
       return false;
     }
 
-    public boolean equals(mark_cleaned_result that) {
+    public boolean equals(mark_compacted_result that) {
       if (that == null)
         return false;
       if (this == that)
@@ -242900,7 +243785,7 @@ package org.apache.hadoop.hive.metastore.api;
     }
 
     @Override
-    public int compareTo(mark_cleaned_result other) {
+    public int compareTo(mark_compacted_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -242935,7 +243820,7 @@ package org.apache.hadoop.hive.metastore.api;
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("mark_cleaned_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("mark_compacted_result(");
       boolean first = true;
 
       sb.append("o1:");
@@ -242970,15 +243855,15 @@ package org.apache.hadoop.hive.metastore.api;
       }
     }
 
-    private static class mark_cleaned_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public mark_cleaned_resultStandardScheme getScheme() {
-        return new mark_cleaned_resultStandardScheme();
+    private static class mark_compacted_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public mark_compacted_resultStandardScheme getScheme() {
+        return new mark_compacted_resultStandardScheme();
       }
     }
 
-    private static class mark_cleaned_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<mark_cleaned_result> {
+    private static class mark_compacted_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<mark_compacted_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, mark_cleaned_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, mark_compacted_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -243006,7 +243891,7 @@ package org.apache.hadoop.hive.metastore.api;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, mark_cleaned_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, mark_compacted_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -243021,16 +243906,16 @@ package org.apache.hadoop.hive.metastore.api;
 
     }
 
-    private static class mark_cleaned_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public mark_cleaned_resultTupleScheme getScheme() {
-        return new mark_cleaned_resultTupleScheme();
+    private static class mark_compacted_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public mark_compacted_resultTupleScheme getScheme() {
+        return new mark_compacted_resultTupleScheme();
       }
     }
 
-    private static class mark_cleaned_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<mark_cleaned_result> {
+    private static class mark_compacted_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<mark_compacted_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, mark_cleaned_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, mark_compacted_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.isSetO1()) {
@@ -243043,7 +243928,7 @@ package org.apache.hadoop.hive.metastore.api;
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, mark_cleaned_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, mark_compacted_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)) {
@@ -243059,13 +243944,13 @@ package org.apache.hadoop.hive.metastore.api;
     }
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class mark_compacted_args implements org.apache.thrift.TBase<mark_compacted_args, mark_compacted_args._Fields>, java.io.Serializable, Cloneable, Comparable<mark_compacted_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("mark_compacted_args");
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class mark_failed_args implements org.apache.thrift.TBase<mark_failed_args, mark_failed_args._Fields>, java.io.Serializable, Cloneable, Comparable<mark_failed_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("mark_failed_args");
 
     private static final org.apache.thrift.protocol.TField CR_FIELD_DESC = new org.apache.thrift.protocol.TField("cr", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new mark_compacted_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new mark_compacted_argsTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new mark_failed_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new mark_failed_argsTupleSchemeFactory();
 
     private @org.apache.thrift.annotation.Nullable CompactionInfoStruct cr; // required
 
@@ -243136,13 +244021,13 @@ package org.apache.hadoop.hive.metastore.api;
       tmpMap.put(_Fields.CR, new org.apache.thrift.meta_data.FieldMetaData("cr", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, CompactionInfoStruct.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(mark_compacted_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(mark_failed_args.class, metaDataMap);
     }
 
-    public mark_compacted_args() {
+    public mark_failed_args() {
     }
 
-    public mark_compacted_args(
+    public mark_failed_args(
       CompactionInfoStruct cr)
     {
       this();
@@ -243152,14 +244037,14 @@ package org.apache.hadoop.hive.metastore.api;
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public mark_compacted_args(mark_compacted_args other) {
+    public mark_failed_args(mark_failed_args other) {
       if (other.isSetCr()) {
         this.cr = new CompactionInfoStruct(other.cr);
       }
     }
 
-    public mark_compacted_args deepCopy() {
-      return new mark_compacted_args(this);
+    public mark_failed_args deepCopy() {
+      return new mark_failed_args(this);
     }
 
     @Override
@@ -243229,12 +244114,12 @@ package org.apache.hadoop.hive.metastore.api;
 
     @Override
     public boolean equals(java.lang.Object that) {
-      if (that instanceof mark_compacted_args)
-        return this.equals((mark_compacted_args)that);
+      if (that instanceof mark_failed_args)
+        return this.equals((mark_failed_args)that);
       return false;
     }
 
-    public boolean equals(mark_compacted_args that) {
+    public boolean equals(mark_failed_args that) {
       if (that == null)
         return false;
       if (this == that)
@@ -243264,7 +244149,7 @@ package org.apache.hadoop.hive.metastore.api;
     }
 
     @Override
-    public int compareTo(mark_compacted_args other) {
+    public int compareTo(mark_failed_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -243299,7 +244184,7 @@ package org.apache.hadoop.hive.metastore.api;
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("mark_compacted_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("mark_failed_args(");
       boolean first = true;
 
       sb.append("cr:");
@@ -243337,15 +244222,15 @@ package org.apache.hadoop.hive.metastore.api;
       }
     }
 
-    private static class mark_compacted_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public mark_compacted_argsStandardScheme getScheme() {
-        return new mark_compacted_argsStandardScheme();
+    private static class mark_failed_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public mark_failed_argsStandardScheme getScheme() {
+        return new mark_failed_argsStandardScheme();
       }
     }
 
-    private static class mark_compacted_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<mark_compacted_args> {
+    private static class mark_failed_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<mark_failed_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, mark_compacted_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, mark_failed_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -243373,7 +244258,7 @@ package org.apache.hadoop.hive.metastore.api;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, mark_compacted_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, mark_failed_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -243388,16 +244273,16 @@ package org.apache.hadoop.hive.metastore.api;
 
     }
 
-    private static class mark_compacted_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public mark_compacted_argsTupleScheme getScheme() {
-        return new mark_compacted_argsTupleScheme();
+    private static class mark_failed_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public mark_failed_argsTupleScheme getScheme() {
+        return new mark_failed_argsTupleScheme();
       }
     }
 
-    private static class mark_compacted_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<mark_compacted_args> {
+    private static class mark_failed_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<mark_failed_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, mark_compacted_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, mark_failed_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.isSetCr()) {
@@ -243410,7 +244295,7 @@ package org.apache.hadoop.hive.metastore.api;
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, mark_compacted_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, mark_failed_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
@@ -243426,13 +244311,13 @@ package org.apache.hadoop.hive.metastore.api;
     }
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class mark_compacted_result implements org.apache.thrift.TBase<mark_compacted_result, mark_compacted_result._Fields>, java.io.Serializable, Cloneable, Comparable<mark_compacted_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("mark_compacted_result");
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class mark_failed_result implements org.apache.thrift.TBase<mark_failed_result, mark_failed_result._Fields>, java.io.Serializable, Cloneable, Comparable<mark_failed_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("mark_failed_result");
 
     private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new mark_compacted_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new mark_compacted_resultTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new mark_failed_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new mark_failed_resultTupleSchemeFactory();
 
     private @org.apache.thrift.annotation.Nullable MetaException o1; // required
 
@@ -243503,13 +244388,13 @@ package org.apache.hadoop.hive.metastore.api;
       tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, MetaException.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(mark_compacted_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(mark_failed_result.class, metaDataMap);
     }
 
-    public mark_compacted_result() {
+    public mark_failed_result() {
     }
 
-    public mark_compacted_result(
+    public mark_failed_result(
       MetaException o1)
     {
       this();
@@ -243519,14 +244404,14 @@ package org.apache.hadoop.hive.metastore.api;
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public mark_compacted_result(mark_compacted_result other) {
+    public mark_failed_result(mark_failed_result other) {
       if (other.isSetO1()) {
         this.o1 = new MetaException(other.o1);
       }
     }
 
-    public mark_compacted_result deepCopy() {
-      return new mark_compacted_result(this);
+    public mark_failed_result deepCopy() {
+      return new mark_failed_result(this);
     }
 
     @Override
@@ -243596,12 +244481,12 @@ package org.apache.hadoop.hive.metastore.api;
 
     @Override
     public boolean equals(java.lang.Object that) {
-      if (that instanceof mark_compacted_result)
-        return this.equals((mark_compacted_result)that);
+      if (that instanceof mark_failed_result)
+        return this.equals((mark_failed_result)that);
       return false;
     }
 
-    public boolean equals(mark_compacted_result that) {
+    public boolean equals(mark_failed_result that) {
       if (that == null)
         return false;
       if (this == that)
@@ -243631,7 +244516,7 @@ package org.apache.hadoop.hive.metastore.api;
     }
 
     @Override
-    public int compareTo(mark_compacted_result other) {
+    public int compareTo(mark_failed_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -243666,7 +244551,7 @@ package org.apache.hadoop.hive.metastore.api;
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("mark_compacted_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("mark_failed_result(");
       boolean first = true;
 
       sb.append("o1:");
@@ -243701,15 +244586,15 @@ package org.apache.hadoop.hive.metastore.api;
       }
     }
 
-    private static class mark_compacted_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public mark_compacted_resultStandardScheme getScheme() {
-        return new mark_compacted_resultStandardScheme();
+    private static class mark_failed_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public mark_failed_resultStandardScheme getScheme() {
+        return new mark_failed_resultStandardScheme();
       }
     }
 
-    private static class mark_compacted_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<mark_compacted_result> {
+    private static class mark_failed_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<mark_failed_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, mark_compacted_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, mark_failed_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -243737,7 +244622,7 @@ package org.apache.hadoop.hive.metastore.api;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, mark_compacted_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, mark_failed_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -243752,16 +244637,16 @@ package org.apache.hadoop.hive.metastore.api;
 
     }
 
-    private static class mark_compacted_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public mark_compacted_resultTupleScheme getScheme() {
-        return new mark_compacted_resultTupleScheme();
+    private static class mark_failed_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public mark_failed_resultTupleScheme getScheme() {
+        return new mark_failed_resultTupleScheme();
       }
     }
 
-    private static class mark_compacted_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<mark_compacted_result> {
+    private static class mark_failed_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<mark_failed_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, mark_compacted_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, mark_failed_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.isSetO1()) {
@@ -243774,7 +244659,7 @@ package org.apache.hadoop.hive.metastore.api;
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, mark_compacted_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, mark_failed_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)) {
@@ -243790,13 +244675,13 @@ package org.apache.hadoop.hive.metastore.api;
     }
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class mark_failed_args implements org.apache.thrift.TBase<mark_failed_args, mark_failed_args._Fields>, java.io.Serializable, Cloneable, Comparable<mark_failed_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("mark_failed_args");
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class mark_refused_args implements org.apache.thrift.TBase<mark_refused_args, mark_refused_args._Fields>, java.io.Serializable, Cloneable, Comparable<mark_refused_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("mark_refused_args");
 
     private static final org.apache.thrift.protocol.TField CR_FIELD_DESC = new org.apache.thrift.protocol.TField("cr", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new mark_failed_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new mark_failed_argsTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new mark_refused_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new mark_refused_argsTupleSchemeFactory();
 
     private @org.apache.thrift.annotation.Nullable CompactionInfoStruct cr; // required
 
@@ -243867,13 +244752,13 @@ package org.apache.hadoop.hive.metastore.api;
       tmpMap.put(_Fields.CR, new org.apache.thrift.meta_data.FieldMetaData("cr", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, CompactionInfoStruct.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(mark_failed_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(mark_refused_args.class, metaDataMap);
     }
 
-    public mark_failed_args() {
+    public mark_refused_args() {
     }
 
-    public mark_failed_args(
+    public mark_refused_args(
       CompactionInfoStruct cr)
     {
       this();
@@ -243883,14 +244768,14 @@ package org.apache.hadoop.hive.metastore.api;
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public mark_failed_args(mark_failed_args other) {
+    public mark_refused_args(mark_refused_args other) {
       if (other.isSetCr()) {
         this.cr = new CompactionInfoStruct(other.cr);
       }
     }
 
-    public mark_failed_args deepCopy() {
-      return new mark_failed_args(this);
+    public mark_refused_args deepCopy() {
+      return new mark_refused_args(this);
     }
 
     @Override
@@ -243960,12 +244845,12 @@ package org.apache.hadoop.hive.metastore.api;
 
     @Override
     public boolean equals(java.lang.Object that) {
-      if (that instanceof mark_failed_args)
-        return this.equals((mark_failed_args)that);
+      if (that instanceof mark_refused_args)
+        return this.equals((mark_refused_args)that);
       return false;
     }
 
-    public boolean equals(mark_failed_args that) {
+    public boolean equals(mark_refused_args that) {
       if (that == null)
         return false;
       if (this == that)
@@ -243995,7 +244880,7 @@ package org.apache.hadoop.hive.metastore.api;
     }
 
     @Override
-    public int compareTo(mark_failed_args other) {
+    public int compareTo(mark_refused_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -244030,7 +244915,7 @@ package org.apache.hadoop.hive.metastore.api;
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("mark_failed_args(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("mark_refused_args(");
       boolean first = true;
 
       sb.append("cr:");
@@ -244068,15 +244953,15 @@ package org.apache.hadoop.hive.metastore.api;
       }
     }
 
-    private static class mark_failed_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public mark_failed_argsStandardScheme getScheme() {
-        return new mark_failed_argsStandardScheme();
+    private static class mark_refused_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public mark_refused_argsStandardScheme getScheme() {
+        return new mark_refused_argsStandardScheme();
       }
     }
 
-    private static class mark_failed_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<mark_failed_args> {
+    private static class mark_refused_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<mark_refused_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, mark_failed_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, mark_refused_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -244104,7 +244989,7 @@ package org.apache.hadoop.hive.metastore.api;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, mark_failed_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, mark_refused_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -244119,16 +245004,16 @@ package org.apache.hadoop.hive.metastore.api;
 
     }
 
-    private static class mark_failed_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public mark_failed_argsTupleScheme getScheme() {
-        return new mark_failed_argsTupleScheme();
+    private static class mark_refused_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public mark_refused_argsTupleScheme getScheme() {
+        return new mark_refused_argsTupleScheme();
       }
     }
 
-    private static class mark_failed_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<mark_failed_args> {
+    private static class mark_refused_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<mark_refused_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, mark_failed_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, mark_refused_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.isSetCr()) {
@@ -244141,7 +245026,7 @@ package org.apache.hadoop.hive.metastore.api;
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, mark_failed_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, mark_refused_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
@@ -244157,13 +245042,13 @@ package org.apache.hadoop.hive.metastore.api;
     }
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class mark_failed_result implements org.apache.thrift.TBase<mark_failed_result, mark_failed_result._Fields>, java.io.Serializable, Cloneable, Comparable<mark_failed_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("mark_failed_result");
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class mark_refused_result implements org.apache.thrift.TBase<mark_refused_result, mark_refused_result._Fields>, java.io.Serializable, Cloneable, Comparable<mark_refused_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("mark_refused_result");
 
     private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new mark_failed_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new mark_failed_resultTupleSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new mark_refused_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new mark_refused_resultTupleSchemeFactory();
 
     private @org.apache.thrift.annotation.Nullable MetaException o1; // required
 
@@ -244234,13 +245119,13 @@ package org.apache.hadoop.hive.metastore.api;
       tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, MetaException.class)));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(mark_failed_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(mark_refused_result.class, metaDataMap);
     }
 
-    public mark_failed_result() {
+    public mark_refused_result() {
     }
 
-    public mark_failed_result(
+    public mark_refused_result(
       MetaException o1)
     {
       this();
@@ -244250,14 +245135,14 @@ package org.apache.hadoop.hive.metastore.api;
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public mark_failed_result(mark_failed_result other) {
+    public mark_refused_result(mark_refused_result other) {
       if (other.isSetO1()) {
         this.o1 = new MetaException(other.o1);
       }
     }
 
-    public mark_failed_result deepCopy() {
-      return new mark_failed_result(this);
+    public mark_refused_result deepCopy() {
+      return new mark_refused_result(this);
     }
 
     @Override
@@ -244327,12 +245212,12 @@ package org.apache.hadoop.hive.metastore.api;
 
     @Override
     public boolean equals(java.lang.Object that) {
-      if (that instanceof mark_failed_result)
-        return this.equals((mark_failed_result)that);
+      if (that instanceof mark_refused_result)
+        return this.equals((mark_refused_result)that);
       return false;
     }
 
-    public boolean equals(mark_failed_result that) {
+    public boolean equals(mark_refused_result that) {
       if (that == null)
         return false;
       if (this == that)
@@ -244362,7 +245247,7 @@ package org.apache.hadoop.hive.metastore.api;
     }
 
     @Override
-    public int compareTo(mark_failed_result other) {
+    public int compareTo(mark_refused_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -244397,7 +245282,7 @@ package org.apache.hadoop.hive.metastore.api;
 
     @Override
     public java.lang.String toString() {
-      java.lang.StringBuilder sb = new java.lang.StringBuilder("mark_failed_result(");
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("mark_refused_result(");
       boolean first = true;
 
       sb.append("o1:");
@@ -244432,15 +245317,15 @@ package org.apache.hadoop.hive.metastore.api;
       }
     }
 
-    private static class mark_failed_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public mark_failed_resultStandardScheme getScheme() {
-        return new mark_failed_resultStandardScheme();
+    private static class mark_refused_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public mark_refused_resultStandardScheme getScheme() {
+        return new mark_refused_resultStandardScheme();
       }
     }
 
-    private static class mark_failed_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<mark_failed_result> {
+    private static class mark_refused_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<mark_refused_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, mark_failed_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, mark_refused_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -244468,7 +245353,7 @@ package org.apache.hadoop.hive.metastore.api;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, mark_failed_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, mark_refused_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -244483,16 +245368,16 @@ package org.apache.hadoop.hive.metastore.api;
 
     }
 
-    private static class mark_failed_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      public mark_failed_resultTupleScheme getScheme() {
-        return new mark_failed_resultTupleScheme();
+    private static class mark_refused_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public mark_refused_resultTupleScheme getScheme() {
+        return new mark_refused_resultTupleScheme();
       }
     }
 
-    private static class mark_failed_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<mark_failed_result> {
+    private static class mark_refused_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<mark_refused_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, mark_failed_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, mark_refused_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.isSetO1()) {
@@ -244505,7 +245390,7 @@ package org.apache.hadoop.hive.metastore.api;
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, mark_failed_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, mark_refused_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)) {
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 61f767f..48966ca 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
@@ -12845,6 +12845,65 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
         return;
     }
 
+    public function mark_refused(\metastore\CompactionInfoStruct $cr)
+    {
+        $this->send_mark_refused($cr);
+        $this->recv_mark_refused();
+    }
+
+    public function send_mark_refused(\metastore\CompactionInfoStruct $cr)
+    {
+        $args = new \metastore\ThriftHiveMetastore_mark_refused_args();
+        $args->cr = $cr;
+        $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+        if ($bin_accel) {
+            thrift_protocol_write_binary(
+                $this->output_,
+                'mark_refused',
+                TMessageType::CALL,
+                $args,
+                $this->seqid_,
+                $this->output_->isStrictWrite()
+            );
+        } else {
+            $this->output_->writeMessageBegin('mark_refused', TMessageType::CALL, $this->seqid_);
+            $args->write($this->output_);
+            $this->output_->writeMessageEnd();
+            $this->output_->getTransport()->flush();
+        }
+    }
+
+    public function recv_mark_refused()
+    {
+        $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+        if ($bin_accel) {
+            $result = thrift_protocol_read_binary(
+                $this->input_,
+                '\metastore\ThriftHiveMetastore_mark_refused_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_mark_refused_result();
+            $result->read($this->input_);
+            $this->input_->readMessageEnd();
+        }
+        if ($result->o1 !== null) {
+            throw $result->o1;
+        }
+        return;
+    }
+
     public function update_compaction_metrics_data(\metastore\CompactionMetricsDataStruct $data)
     {
         $this->send_update_compaction_metrics_data($data);
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 9c1a4c6..a224d2b 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
@@ -1503,6 +1503,11 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf
      */
     public function mark_failed(\metastore\CompactionInfoStruct $cr);
     /**
+     * @param \metastore\CompactionInfoStruct $cr
+     * @throws \metastore\MetaException
+     */
+    public function mark_refused(\metastore\CompactionInfoStruct $cr);
+    /**
      * @param \metastore\CompactionMetricsDataStruct $data
      * @return bool
      * @throws \metastore\MetaException
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_mark_refused_args.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_mark_refused_args.php
new file mode 100644
index 0000000..23e92f6
--- /dev/null
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_mark_refused_args.php
@@ -0,0 +1,99 @@
+<?php
+namespace metastore;
+
+/**
+ * Autogenerated by Thrift Compiler (0.14.1)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+use Thrift\Base\TBase;
+use Thrift\Type\TType;
+use Thrift\Type\TMessageType;
+use Thrift\Exception\TException;
+use Thrift\Exception\TProtocolException;
+use Thrift\Protocol\TProtocol;
+use Thrift\Protocol\TBinaryProtocolAccelerated;
+use Thrift\Exception\TApplicationException;
+
+class ThriftHiveMetastore_mark_refused_args
+{
+    static public $isValidate = false;
+
+    static public $_TSPEC = array(
+        1 => array(
+            'var' => 'cr',
+            'isRequired' => false,
+            'type' => TType::STRUCT,
+            'class' => '\metastore\CompactionInfoStruct',
+        ),
+    );
+
+    /**
+     * @var \metastore\CompactionInfoStruct
+     */
+    public $cr = null;
+
+    public function __construct($vals = null)
+    {
+        if (is_array($vals)) {
+            if (isset($vals['cr'])) {
+                $this->cr = $vals['cr'];
+            }
+        }
+    }
+
+    public function getName()
+    {
+        return 'ThriftHiveMetastore_mark_refused_args';
+    }
+
+
+    public function read($input)
+    {
+        $xfer = 0;
+        $fname = null;
+        $ftype = 0;
+        $fid = 0;
+        $xfer += $input->readStructBegin($fname);
+        while (true) {
+            $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+            if ($ftype == TType::STOP) {
+                break;
+            }
+            switch ($fid) {
+                case 1:
+                    if ($ftype == TType::STRUCT) {
+                        $this->cr = new \metastore\CompactionInfoStruct();
+                        $xfer += $this->cr->read($input);
+                    } else {
+                        $xfer += $input->skip($ftype);
+                    }
+                    break;
+                default:
+                    $xfer += $input->skip($ftype);
+                    break;
+            }
+            $xfer += $input->readFieldEnd();
+        }
+        $xfer += $input->readStructEnd();
+        return $xfer;
+    }
+
+    public function write($output)
+    {
+        $xfer = 0;
+        $xfer += $output->writeStructBegin('ThriftHiveMetastore_mark_refused_args');
+        if ($this->cr !== null) {
+            if (!is_object($this->cr)) {
+                throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+            }
+            $xfer += $output->writeFieldBegin('cr', TType::STRUCT, 1);
+            $xfer += $this->cr->write($output);
+            $xfer += $output->writeFieldEnd();
+        }
+        $xfer += $output->writeFieldStop();
+        $xfer += $output->writeStructEnd();
+        return $xfer;
+    }
+}
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_mark_refused_result.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_mark_refused_result.php
new file mode 100644
index 0000000..74a9dee
--- /dev/null
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore_mark_refused_result.php
@@ -0,0 +1,96 @@
+<?php
+namespace metastore;
+
+/**
+ * Autogenerated by Thrift Compiler (0.14.1)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+use Thrift\Base\TBase;
+use Thrift\Type\TType;
+use Thrift\Type\TMessageType;
+use Thrift\Exception\TException;
+use Thrift\Exception\TProtocolException;
+use Thrift\Protocol\TProtocol;
+use Thrift\Protocol\TBinaryProtocolAccelerated;
+use Thrift\Exception\TApplicationException;
+
+class ThriftHiveMetastore_mark_refused_result
+{
+    static public $isValidate = false;
+
+    static public $_TSPEC = array(
+        1 => array(
+            'var' => 'o1',
+            'isRequired' => false,
+            'type' => TType::STRUCT,
+            'class' => '\metastore\MetaException',
+        ),
+    );
+
+    /**
+     * @var \metastore\MetaException
+     */
+    public $o1 = null;
+
+    public function __construct($vals = null)
+    {
+        if (is_array($vals)) {
+            if (isset($vals['o1'])) {
+                $this->o1 = $vals['o1'];
+            }
+        }
+    }
+
+    public function getName()
+    {
+        return 'ThriftHiveMetastore_mark_refused_result';
+    }
+
+
+    public function read($input)
+    {
+        $xfer = 0;
+        $fname = null;
+        $ftype = 0;
+        $fid = 0;
+        $xfer += $input->readStructBegin($fname);
+        while (true) {
+            $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+            if ($ftype == TType::STOP) {
+                break;
+            }
+            switch ($fid) {
+                case 1:
+                    if ($ftype == TType::STRUCT) {
+                        $this->o1 = new \metastore\MetaException();
+                        $xfer += $this->o1->read($input);
+                    } else {
+                        $xfer += $input->skip($ftype);
+                    }
+                    break;
+                default:
+                    $xfer += $input->skip($ftype);
+                    break;
+            }
+            $xfer += $input->readFieldEnd();
+        }
+        $xfer += $input->readStructEnd();
+        return $xfer;
+    }
+
+    public function write($output)
+    {
+        $xfer = 0;
+        $xfer += $output->writeStructBegin('ThriftHiveMetastore_mark_refused_result');
+        if ($this->o1 !== null) {
+            $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+            $xfer += $this->o1->write($output);
+            $xfer += $output->writeFieldEnd();
+        }
+        $xfer += $output->writeFieldStop();
+        $xfer += $output->writeStructEnd();
+        return $xfer;
+    }
+}
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 5706286..2c545d1 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
@@ -222,6 +222,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
     print('  void mark_cleaned(CompactionInfoStruct cr)')
     print('  void mark_compacted(CompactionInfoStruct cr)')
     print('  void mark_failed(CompactionInfoStruct cr)')
+    print('  void mark_refused(CompactionInfoStruct cr)')
     print('  bool update_compaction_metrics_data(CompactionMetricsDataStruct data)')
     print('  void remove_compaction_metrics_data(CompactionMetricsDataRequest request)')
     print('  void set_hadoop_jobid(string jobId, i64 cq_id)')
@@ -1572,6 +1573,12 @@ elif cmd == 'mark_failed':
         sys.exit(1)
     pp.pprint(client.mark_failed(eval(args[0]),))
 
+elif cmd == 'mark_refused':
+    if len(args) != 1:
+        print('mark_refused requires 1 args')
+        sys.exit(1)
+    pp.pprint(client.mark_refused(eval(args[0]),))
+
 elif cmd == 'update_compaction_metrics_data':
     if len(args) != 1:
         print('update_compaction_metrics_data 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 7a568ed..1617258 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
@@ -1735,6 +1735,14 @@ class Iface(fb303.FacebookService.Iface):
         """
         pass
 
+    def mark_refused(self, cr):
+        """
+        Parameters:
+         - cr
+
+        """
+        pass
+
     def update_compaction_metrics_data(self, data):
         """
         Parameters:
@@ -9572,6 +9580,38 @@ class Client(fb303.FacebookService.Client, Iface):
             raise result.o1
         return
 
+    def mark_refused(self, cr):
+        """
+        Parameters:
+         - cr
+
+        """
+        self.send_mark_refused(cr)
+        self.recv_mark_refused()
+
+    def send_mark_refused(self, cr):
+        self._oprot.writeMessageBegin('mark_refused', TMessageType.CALL, self._seqid)
+        args = mark_refused_args()
+        args.cr = cr
+        args.write(self._oprot)
+        self._oprot.writeMessageEnd()
+        self._oprot.trans.flush()
+
+    def recv_mark_refused(self):
+        iprot = self._iprot
+        (fname, mtype, rseqid) = iprot.readMessageBegin()
+        if mtype == TMessageType.EXCEPTION:
+            x = TApplicationException()
+            x.read(iprot)
+            iprot.readMessageEnd()
+            raise x
+        result = mark_refused_result()
+        result.read(iprot)
+        iprot.readMessageEnd()
+        if result.o1 is not None:
+            raise result.o1
+        return
+
     def update_compaction_metrics_data(self, data):
         """
         Parameters:
@@ -12188,6 +12228,7 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
         self._processMap["mark_cleaned"] = Processor.process_mark_cleaned
         self._processMap["mark_compacted"] = Processor.process_mark_compacted
         self._processMap["mark_failed"] = Processor.process_mark_failed
+        self._processMap["mark_refused"] = Processor.process_mark_refused
         self._processMap["update_compaction_metrics_data"] = Processor.process_update_compaction_metrics_data
         self._processMap["remove_compaction_metrics_data"] = Processor.process_remove_compaction_metrics_data
         self._processMap["set_hadoop_jobid"] = Processor.process_set_hadoop_jobid
@@ -17968,6 +18009,32 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
         oprot.writeMessageEnd()
         oprot.trans.flush()
 
+    def process_mark_refused(self, seqid, iprot, oprot):
+        args = mark_refused_args()
+        args.read(iprot)
+        iprot.readMessageEnd()
+        result = mark_refused_result()
+        try:
+            self._handler.mark_refused(args.cr)
+            msg_type = TMessageType.REPLY
+        except TTransport.TTransportException:
+            raise
+        except MetaException as o1:
+            msg_type = TMessageType.REPLY
+            result.o1 = o1
+        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("mark_refused", msg_type, seqid)
+        result.write(oprot)
+        oprot.writeMessageEnd()
+        oprot.trans.flush()
+
     def process_update_compaction_metrics_data(self, seqid, iprot, oprot):
         args = update_compaction_metrics_data_args()
         args.read(iprot)
@@ -50934,6 +51001,131 @@ mark_failed_result.thrift_spec = (
 )
 
 
+class mark_refused_args(object):
+    """
+    Attributes:
+     - cr
+
+    """
+
+
+    def __init__(self, cr=None,):
+        self.cr = cr
+
+    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.cr = CompactionInfoStruct()
+                    self.cr.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('mark_refused_args')
+        if self.cr is not None:
+            oprot.writeFieldBegin('cr', TType.STRUCT, 1)
+            self.cr.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(mark_refused_args)
+mark_refused_args.thrift_spec = (
+    None,  # 0
+    (1, TType.STRUCT, 'cr', [CompactionInfoStruct, None], None, ),  # 1
+)
+
+
+class mark_refused_result(object):
+    """
+    Attributes:
+     - o1
+
+    """
+
+
+    def __init__(self, o1=None,):
+        self.o1 = o1
+
+    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.o1 = 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('mark_refused_result')
+        if self.o1 is not None:
+            oprot.writeFieldBegin('o1', TType.STRUCT, 1)
+            self.o1.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(mark_refused_result)
+mark_refused_result.thrift_spec = (
+    None,  # 0
+    (1, TType.STRUCT, 'o1', [MetaException, None], None, ),  # 1
+)
+
+
 class update_compaction_metrics_data_args(object):
     """
     Attributes:
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 19e6238..c6e3602 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
@@ -3306,6 +3306,21 @@ module ThriftHiveMetastore
       return
     end
 
+    def mark_refused(cr)
+      send_mark_refused(cr)
+      recv_mark_refused()
+    end
+
+    def send_mark_refused(cr)
+      send_message('mark_refused', Mark_refused_args, :cr => cr)
+    end
+
+    def recv_mark_refused()
+      result = receive_message(Mark_refused_result)
+      raise result.o1 unless result.o1.nil?
+      return
+    end
+
     def update_compaction_metrics_data(data)
       send_update_compaction_metrics_data(data)
       return recv_update_compaction_metrics_data()
@@ -6954,6 +6969,17 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'mark_failed', seqid)
     end
 
+    def process_mark_refused(seqid, iprot, oprot)
+      args = read_args(iprot, Mark_refused_args)
+      result = Mark_refused_result.new()
+      begin
+        @handler.mark_refused(args.cr)
+      rescue ::MetaException => o1
+        result.o1 = o1
+      end
+      write_result(result, oprot, 'mark_refused', seqid)
+    end
+
     def process_update_compaction_metrics_data(seqid, iprot, oprot)
       args = read_args(iprot, Update_compaction_metrics_data_args)
       result = Update_compaction_metrics_data_result.new()
@@ -15126,6 +15152,38 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
+  class Mark_refused_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    CR = 1
+
+    FIELDS = {
+      CR => {:type => ::Thrift::Types::STRUCT, :name => 'cr', :class => ::CompactionInfoStruct}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Mark_refused_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    O1 = 1
+
+    FIELDS = {
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
   class Update_compaction_metrics_data_args
     include ::Thrift::Struct, ::Thrift::Struct_Union
     DATA = 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 50558a8..dbfac3c 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
@@ -4981,6 +4981,11 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   }
 
   @Override
+  public void markRefused(CompactionInfoStruct cr) throws MetaException, TException {
+    client.mark_refused(cr);
+  }
+
+  @Override
   public boolean updateCompactionMetricsData(CompactionMetricsDataStruct struct)
       throws MetaException, TException {
     return client.update_compaction_metrics_data(struct);
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 f46b276..2a175bd 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
@@ -4309,6 +4309,14 @@ public interface IMetaStoreClient {
   void markFailed(CompactionInfoStruct cr) throws MetaException, TException;
 
   /**
+   * Mark a compaction as refused (to run).
+   * @param cr compaction job.
+   * @throws MetaException
+   * @throws TException
+   */
+  void markRefused(CompactionInfoStruct cr) throws MetaException, TException;
+
+  /**
    * Create, update or delete one record in the compaction metrics cache.
    * <p>
    * If the metric is not found in the metrics cache, it will be created.
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 3c33c3f..7093587 100644
--- a/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
+++ b/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
@@ -2955,6 +2955,7 @@ PartitionsResponse get_partitions_req(1:PartitionsRequest req)
   void mark_cleaned(1:CompactionInfoStruct cr) throws(1:MetaException o1)
   void mark_compacted(1: CompactionInfoStruct cr) throws(1:MetaException o1)
   void mark_failed(1: CompactionInfoStruct cr) throws(1:MetaException o1)
+  void mark_refused(1: CompactionInfoStruct cr) throws(1:MetaException o1)
   bool update_compaction_metrics_data(1: CompactionMetricsDataStruct data) throws(1:MetaException o1)
   void remove_compaction_metrics_data(1: CompactionMetricsDataRequest request) throws(1:MetaException o1)
   void set_hadoop_jobid(1: string jobId, 2: i64 cq_id)
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 2d57e58..9991d21 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
@@ -8793,6 +8793,11 @@ public class HMSHandler extends FacebookBase implements IHMSHandler {
   }
 
   @Override
+  public void mark_refused(CompactionInfoStruct cr) throws MetaException {
+    getTxnHandler().markRefused(CompactionInfo.compactionStructToInfo(cr));
+  }
+
+  @Override
   public boolean update_compaction_metrics_data(CompactionMetricsDataStruct struct) throws MetaException, TException {
       return getTxnHandler().updateCompactionMetricsData(CompactionMetricsDataConverter.structToData(struct));
   }
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 64679a0..730b5ec 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
@@ -1323,15 +1323,8 @@ class CompactionTxnHandler extends TxnHandler {
     }
   }
 
-  /**
-   * If there is an entry in compaction_queue with ci.id, remove it
-   * Make entry in completed_compactions with status 'f'.
-   * If there is no entry in compaction_queue, it means Initiator failed to even schedule a compaction,
-   * which we record as DID_NOT_INITIATE entry in history.
-   */
-  @Override
-  @RetrySemantics.CannotRetry
-  public void markFailed(CompactionInfo ci) throws MetaException {//todo: this should not throw
+
+  private void updateStatus(CompactionInfo ci) throws MetaException {
     if (LOG.isDebugEnabled()) {
       LOG.debug("Marking as failed: CompactionInfo: " + ci.toString());
     }
@@ -1340,26 +1333,29 @@ class CompactionTxnHandler extends TxnHandler {
       Statement stmt = null;
       PreparedStatement pStmt = null;
       ResultSet rs = null;
-      // the error message related to the failure is wrapped inside CompactionInfo
-      // fetch this info, since ci will be reused in subsequent queries
-      String errorMessage = ci.errorMessage;
       try {
         dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
         stmt = dbConn.createStatement();
         pStmt = dbConn.prepareStatement("SELECT \"CQ_ID\", \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\", "
-            + "\"CQ_STATE\", \"CQ_TYPE\", \"CQ_TBLPROPERTIES\", \"CQ_WORKER_ID\", \"CQ_START\", \"CQ_RUN_AS\", "
-            + "\"CQ_HIGHEST_WRITE_ID\", \"CQ_META_INFO\", \"CQ_HADOOP_JOB_ID\", \"CQ_ERROR_MESSAGE\", "
-            + "\"CQ_ENQUEUE_TIME\", \"CQ_WORKER_VERSION\", \"CQ_INITIATOR_ID\", \"CQ_INITIATOR_VERSION\" "
-            + "FROM \"COMPACTION_QUEUE\" WHERE \"CQ_ID\" = ?");
+                + "\"CQ_STATE\", \"CQ_TYPE\", \"CQ_TBLPROPERTIES\", \"CQ_WORKER_ID\", \"CQ_START\", \"CQ_RUN_AS\", "
+                + "\"CQ_HIGHEST_WRITE_ID\", \"CQ_META_INFO\", \"CQ_HADOOP_JOB_ID\", \"CQ_ERROR_MESSAGE\", "
+                + "\"CQ_ENQUEUE_TIME\", \"CQ_WORKER_VERSION\", \"CQ_INITIATOR_ID\", \"CQ_INITIATOR_VERSION\" "
+                + "FROM \"COMPACTION_QUEUE\" WHERE \"CQ_ID\" = ?");
         pStmt.setLong(1, ci.id);
         rs = pStmt.executeQuery();
         if (rs.next()) {
+          //preserve errorMessage and state
+          String errorMessage = ci.errorMessage;
+          char state = ci.state;
           ci = CompactionInfo.loadFullFromCompactionQueue(rs);
+          ci.errorMessage = errorMessage;
+          ci.state = state;
+
           String s = "DELETE FROM \"COMPACTION_QUEUE\" WHERE \"CQ_ID\" = ?";
           pStmt = dbConn.prepareStatement(s);
           pStmt.setLong(1, ci.id);
           LOG.debug("Going to execute update <" + s + ">");
-          int updCnt = pStmt.executeUpdate();
+          pStmt.executeUpdate();
         }
         else {
           if(ci.id > 0) {
@@ -1371,31 +1367,23 @@ class CompactionTxnHandler extends TxnHandler {
           //The failure occurred before we even made an entry in COMPACTION_QUEUE
           //generate ID so that we can make an entry in COMPLETED_COMPACTIONS
           ci.id = generateCompactionQueueId(stmt);
-          //mostly this indicates that the Initiator is paying attention to some table even though
-          //compactions are not happening.
-          ci.state = DID_NOT_INITIATE;
           //this is not strictly accurate, but 'type' cannot be null.
           if(ci.type == null) {
             ci.type = CompactionType.MINOR;
           }
           ci.start = getDbTime(dbConn);
           LOG.debug("The failure occurred before we even made an entry in COMPACTION_QUEUE. Generated ID so that we "
-              + "can make an entry in COMPLETED_COMPACTIONS. New Id: " + ci.id);
-        } else {
-          ci.state = FAILED_STATE;
+                  + "can make an entry in COMPLETED_COMPACTIONS. New Id: " + ci.id);
         }
         close(rs, stmt, null);
         closeStmt(pStmt);
 
         pStmt = dbConn.prepareStatement("INSERT INTO \"COMPLETED_COMPACTIONS\" "
-            + "(\"CC_ID\", \"CC_DATABASE\", \"CC_TABLE\", \"CC_PARTITION\", \"CC_STATE\", \"CC_TYPE\", "
-            + "\"CC_TBLPROPERTIES\", \"CC_WORKER_ID\", \"CC_START\", \"CC_END\", \"CC_RUN_AS\", "
-            + "\"CC_HIGHEST_WRITE_ID\", \"CC_META_INFO\", \"CC_HADOOP_JOB_ID\", \"CC_ERROR_MESSAGE\", "
-            + "\"CC_ENQUEUE_TIME\", \"CC_WORKER_VERSION\", \"CC_INITIATOR_ID\", \"CC_INITIATOR_VERSION\") "
-            + "VALUES(?,?,?,?,?, ?,?,?,?,?, ?,?,?,?,?,?,?,?,?)");
-        if (errorMessage != null) {
-          ci.errorMessage = errorMessage;
-        }
+                + "(\"CC_ID\", \"CC_DATABASE\", \"CC_TABLE\", \"CC_PARTITION\", \"CC_STATE\", \"CC_TYPE\", "
+                + "\"CC_TBLPROPERTIES\", \"CC_WORKER_ID\", \"CC_START\", \"CC_END\", \"CC_RUN_AS\", "
+                + "\"CC_HIGHEST_WRITE_ID\", \"CC_META_INFO\", \"CC_HADOOP_JOB_ID\", \"CC_ERROR_MESSAGE\", "
+                + "\"CC_ENQUEUE_TIME\", \"CC_WORKER_VERSION\", \"CC_INITIATOR_ID\", \"CC_INITIATOR_VERSION\") "
+                + "VALUES(?,?,?,?,?, ?,?,?,?,?, ?,?,?,?,?,?,?,?,?)");
         CompactionInfo.insertIntoCompletedCompactions(pStmt, ci, getDbTime(dbConn));
         int updCount = pStmt.executeUpdate();
         LOG.debug("Inserted " + updCount + " entries into COMPLETED_COMPACTIONS");
@@ -1413,9 +1401,35 @@ class CompactionTxnHandler extends TxnHandler {
         close(null, pStmt, dbConn);
       }
     } catch (RetryException e) {
-      markFailed(ci);
+      updateStatus(ci);
     }
   }
+
+  /**
+   * If there is an entry in compaction_queue with ci.id, remove it
+   * Make entry in completed_compactions with status 'f'.
+   * If there is no entry in compaction_queue, it means Initiator failed to even schedule a compaction,
+   * which we record as DID_NOT_INITIATE entry in history.
+   */
+  @Override
+  @RetrySemantics.CannotRetry
+  public void markFailed(CompactionInfo ci) throws MetaException {
+    ci.state = ci.id == 0 ? DID_NOT_INITIATE : FAILED_STATE;
+    updateStatus(ci);
+  }
+
+  /**
+   * Mark a compaction as refused (to run).
+   * @param info compaction job.
+   * @throws MetaException
+   */
+  @Override
+  @RetrySemantics.CannotRetry
+  public void markRefused(CompactionInfo info) throws MetaException {
+    info.state = REFUSED_STATE;
+    updateStatus(info);
+  }
+
   @Override
   @RetrySemantics.Idempotent
   public void setHadoopJobId(String hadoopJobId, long id) {
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 7d2b0ad..bf61d08 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
@@ -239,6 +239,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
   static final char FAILED_STATE = 'f';
   static final char SUCCEEDED_STATE = 's';
   static final char DID_NOT_INITIATE = 'a';
+  static final char REFUSED_STATE = 'c';
 
   // Compactor types
   static final protected char MAJOR_TYPE = 'a';
@@ -3888,6 +3889,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
       case FAILED_STATE: return FAILED_RESPONSE;
       case SUCCEEDED_STATE: return SUCCEEDED_RESPONSE;
       case DID_NOT_INITIATE: return DID_NOT_INITIATE_RESPONSE;
+      case REFUSED_STATE: return REFUSED_RESPONSE;
       default:
         return Character.toString(s);
     }
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
index c59ff40..c55f534 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
@@ -102,6 +102,7 @@ public interface TxnStore extends Configurable {
   String FAILED_RESPONSE = "failed";
   String SUCCEEDED_RESPONSE = "succeeded";
   String DID_NOT_INITIATE_RESPONSE = "did not initiate";
+  String REFUSED_RESPONSE = "refused";
 
   String[] COMPACTION_STATES = new String[] {INITIATED_RESPONSE, WORKING_RESPONSE, CLEANING_RESPONSE, FAILED_RESPONSE,
       SUCCEEDED_RESPONSE, DID_NOT_INITIATE_RESPONSE };
@@ -514,6 +515,14 @@ public interface TxnStore extends Configurable {
   void markFailed(CompactionInfo info) throws MetaException;
 
   /**
+   * Mark a compaction as refused (to run). This can happen if a manual compaction is requested by the user,
+   * but for some reason, the table is not suitable for compation.
+   * @param info compaction job.
+   * @throws MetaException
+   */
+  void markRefused(CompactionInfo info) throws MetaException;
+
+  /**
    * Clean up entries from TXN_TO_WRITE_ID table less than min_uncommited_txnid as found by
    * min(max(TXNS.txn_id), min(WRITE_SET.WS_COMMIT_ID), min(Aborted TXNS.txn_id)).
    */
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
index de22897..652bca0 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
@@ -3958,6 +3958,11 @@ public class HiveMetaStoreClientPreCatalog implements IMetaStoreClient, AutoClos
   }
 
   @Override
+  public void markRefused(CompactionInfoStruct cr) throws MetaException, TException {
+    client.mark_refused(cr);
+  }
+
+  @Override
   public boolean updateCompactionMetricsData(CompactionMetricsDataStruct struct)
       throws MetaException, TException {
     return client.update_compaction_metrics_data(struct);