You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by lp...@apache.org on 2022/01/21 15:00:54 UTC

[hive] branch master updated: HIVE-25842: Reimplement delta file metric collection. (#2916) (Laszlo Pinter, reviewed by Karen Coppage and Denys Kuzmenko)

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

lpinter 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 8409416  HIVE-25842: Reimplement delta file metric collection. (#2916) (Laszlo Pinter, reviewed by Karen Coppage and Denys Kuzmenko)
8409416 is described below

commit 8409416d2104c5700893552c9ba8614c3024d1ad
Author: László Pintér <47...@users.noreply.github.com>
AuthorDate: Fri Jan 21 16:00:46 2022 +0100

    HIVE-25842: Reimplement delta file metric collection. (#2916) (Laszlo Pinter, reviewed by Karen Coppage and Denys Kuzmenko)
---
 .../hive/common/metrics/common/MetricsFactory.java |     6 +-
 .../java/org/apache/hadoop/hive/conf/HiveConf.java |    30 +-
 .../txn/compactor/TestCompactionMetricsOnTez.java  |   278 -
 .../hive/ql/exec/tez/HiveSplitGenerator.java       |    22 -
 .../apache/hadoop/hive/ql/exec/tez/TezTask.java    |     5 -
 .../apache/hadoop/hive/ql/io/HiveInputFormat.java  |     1 -
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java      |    27 -
 .../hadoop/hive/ql/txn/compactor/Cleaner.java      |    47 +-
 .../hadoop/hive/ql/txn/compactor/Initiator.java    |    72 +-
 .../hadoop/hive/ql/txn/compactor/Worker.java       |    15 +
 .../metrics/DeltaFilesMetricReporter.java          |   635 +-
 .../ql/txn/compactor/TestCompactionMetrics.java    |     1 +
 .../ql/txn/compactor/TestDeltaFilesMetrics.java    |   482 +-
 .../apache/hive/service/server/HiveServer2.java    |     4 -
 .../src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp | 13284 ++++++++++---------
 .../src/gen/thrift/gen-cpp/ThriftHiveMetastore.h   |   259 +
 .../ThriftHiveMetastore_server.skeleton.cpp        |    10 +
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp    |  5061 +++----
 .../src/gen/thrift/gen-cpp/hive_metastore_types.h  |   192 +
 .../api/CompactionMetricsDataRequest.java          |   681 +
 .../api/CompactionMetricsDataResponse.java         |   375 +
 .../metastore/api/CompactionMetricsDataStruct.java |   878 ++
 .../metastore/api/CompactionMetricsMetricType.java |    46 +
 .../hive/metastore/api/ThriftHiveMetastore.java    |  3341 ++++-
 .../metastore/CompactionMetricsDataRequest.php     |   167 +
 .../metastore/CompactionMetricsDataResponse.php    |    99 +
 .../metastore/CompactionMetricsDataStruct.php      |   215 +
 .../metastore/CompactionMetricsMetricType.php      |    33 +
 .../metastore/ThriftHiveMetastoreClient.php        |   121 +
 .../gen-php/metastore/ThriftHiveMetastoreIf.php    |    11 +
 ...tastore_remove_compaction_metrics_data_args.php |    99 +
 ...store_remove_compaction_metrics_data_result.php |    96 +
 ...tastore_update_compaction_metrics_data_args.php |    99 +
 ...store_update_compaction_metrics_data_result.php |   120 +
 .../hive_metastore/ThriftHiveMetastore-remote      |    14 +
 .../gen-py/hive_metastore/ThriftHiveMetastore.py   |   397 +
 .../src/gen/thrift/gen-py/hive_metastore/ttypes.py |   319 +
 .../src/gen/thrift/gen-rb/hive_metastore_types.rb  |    93 +
 .../src/gen/thrift/gen-rb/thrift_hive_metastore.rb |   119 +
 .../hadoop/hive/metastore/HiveMetaStoreClient.java |    11 +
 .../hadoop/hive/metastore/IMetaStoreClient.java    |    18 +
 .../hadoop/hive/metastore/conf/MetastoreConf.java  |    24 +
 .../src/main/thrift/hive_metastore.thrift          |    42 +-
 .../apache/hadoop/hive/metastore/HMSHandler.java   |    13 +
 .../hive/metastore/txn/CompactionMetricsData.java  |   133 +
 .../txn/CompactionMetricsDataConverter.java        |    79 +
 .../hive/metastore/txn/CompactionTxnHandler.java   |   194 +
 .../hadoop/hive/metastore/txn/TxnHandler.java      |    42 +-
 .../apache/hadoop/hive/metastore/txn/TxnStore.java |    47 +
 .../src/main/sql/derby/hive-schema-4.0.0.derby.sql |    10 +
 .../sql/derby/upgrade-3.2.0-to-4.0.0.derby.sql     |    10 +
 .../src/main/sql/mssql/hive-schema-4.0.0.mssql.sql |    10 +
 .../sql/mssql/upgrade-3.2.0-to-4.0.0.mssql.sql     |    10 +
 .../src/main/sql/mysql/hive-schema-4.0.0.mysql.sql |    10 +
 .../sql/mysql/upgrade-3.2.0-to-4.0.0.mysql.sql     |    10 +
 .../main/sql/oracle/hive-schema-4.0.0.oracle.sql   |    10 +
 .../sql/oracle/upgrade-3.2.0-to-4.0.0.oracle.sql   |    10 +
 .../sql/postgres/hive-schema-4.0.0.postgres.sql    |    10 +
 .../postgres/upgrade-3.2.0-to-4.0.0.postgres.sql   |    10 +
 .../metastore/HiveMetaStoreClientPreCatalog.java   |    11 +
 .../hadoop/hive/metastore/utils/TestTxnDbUtil.java |     1 +
 .../upgrade-3.1.3000-to-4.0.0.postgres.sql         |    10 +
 62 files changed, 18332 insertions(+), 10147 deletions(-)

diff --git a/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsFactory.java b/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsFactory.java
index b8e9a01..334683a 100644
--- a/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsFactory.java
+++ b/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsFactory.java
@@ -17,8 +17,8 @@
  */
 package org.apache.hadoop.hive.common.metrics.common;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.util.ReflectionUtils;
 
 import java.lang.reflect.Constructor;
 
@@ -34,10 +34,10 @@ public class MetricsFactory {
   /**
    * Initializes static Metrics instance.
    */
-  public synchronized static void init(HiveConf conf) throws Exception {
+  public synchronized static void init(Configuration conf) throws Exception {
     if (metrics == null) {
       Class metricsClass = conf.getClassByName(
-        conf.getVar(HiveConf.ConfVars.HIVE_METRICS_CLASS));
+        conf.get(HiveConf.ConfVars.HIVE_METRICS_CLASS.varname));
       Constructor constructor = metricsClass.getConstructor(HiveConf.class);
       metrics = (Metrics) constructor.newInstance(conf);
     }
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 6e4bbcc7..e0fcf3c 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -3085,24 +3085,38 @@ public class HiveConf extends Configuration {
     
     // Configs having to do with DeltaFilesMetricReporter, which collects lists of most recently active tables
     // with the most number of active/obsolete deltas.
+    /**
+     * @deprecated use MetastoreConf.METASTORE_DELTAMETRICS_MAX_CACHE_SIZE
+     */
+    @Deprecated
     HIVE_TXN_ACID_METRICS_MAX_CACHE_SIZE("hive.txn.acid.metrics.max.cache.size", 100,
         new RangeValidator(0, 500),
         "Size of the ACID metrics cache, i.e. max number of partitions and unpartitioned tables with the "
             + "most deltas that will be included in the lists of active, obsolete and small deltas. "
             + "Allowed range is 0 to 500."),
-    HIVE_TXN_ACID_METRICS_CACHE_DURATION("hive.txn.acid.metrics.cache.duration", "7200s",
-        new TimeValidator(TimeUnit.SECONDS),
-        "Maximum lifetime in seconds for an entry in the ACID metrics cache."),
+    /**
+     * @deprecated use MetastoreConf.METASTORE_DELTAMETRICS_REPORTING_INTERVAL
+     */
+    @Deprecated
     HIVE_TXN_ACID_METRICS_REPORTING_INTERVAL("hive.txn.acid.metrics.reporting.interval", "30s",
         new TimeValidator(TimeUnit.SECONDS),
         "Reporting period for ACID metrics in seconds."),
+    /**
+     * @deprecated use MetastoreConf.METASTORE_DELTAMETRICS_DELTA_NUM_THRESHOLD
+     */
+    @Deprecated
     HIVE_TXN_ACID_METRICS_DELTA_NUM_THRESHOLD("hive.txn.acid.metrics.delta.num.threshold", 100,
         "The minimum number of active delta files a table/partition must have in order to be included in the ACID metrics report."),
+    /**
+     * @deprecated use MetastoreConf.METASTORE_DELTAMETRICS_OBSOLETE_DELTA_NUM_THRESHOLD
+     */
+    @Deprecated
     HIVE_TXN_ACID_METRICS_OBSOLETE_DELTA_NUM_THRESHOLD("hive.txn.acid.metrics.obsolete.delta.num.threshold", 100,
         "The minimum number of obsolete delta files a table/partition must have in order to be included in the ACID metrics report."),
-    HIVE_TXN_ACID_METRICS_DELTA_CHECK_THRESHOLD("hive.txn.acid.metrics.delta.check.threshold", "300s",
-        new TimeValidator(TimeUnit.SECONDS),
-        "Deltas not older than this value will not be included in the ACID metrics report."),
+    /**
+     * @deprecated use MetastoreConf.METASTORE_DELTAMETRICS_DELTA_PCT_THRESHOLD
+     */
+    @Deprecated
     HIVE_TXN_ACID_METRICS_DELTA_PCT_THRESHOLD("hive.txn.acid.metrics.delta.pct.threshold", 0.01f,
         "Percentage (fractional) size of the delta files relative to the base directory. Deltas smaller than this threshold " +
         "count as small deltas. Default 0.01 = 1%.)"),
@@ -3255,6 +3269,10 @@ public class HiveConf extends Configuration {
         "If the number of small delta directories under a table/partition passes this threshold, a " +
             "warning message will be logged."),
 
+    /**
+     * @deprecated use MetastoreConf.METASTORE_DELTAMETRICS_LOGGER_FREQUENCY
+     */
+    @Deprecated
     HIVE_COMPACTOR_ACID_METRICS_LOGGER_FREQUENCY(
         "hive.compactor.acid.metrics.logger.frequency",
         "360m", new TimeValidator(TimeUnit.MINUTES),
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactionMetricsOnTez.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactionMetricsOnTez.java
deleted file mode 100644
index c0c29e9..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactionMetricsOnTez.java
+++ /dev/null
@@ -1,278 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hive.ql.txn.compactor;
-
-import org.apache.commons.lang3.RandomStringUtils;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.common.metrics.common.MetricsFactory;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.api.CompactionType;
-import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
-import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
-import org.apache.hadoop.hive.ql.txn.compactor.metrics.DeltaFilesMetricReporter;
-import org.apache.tez.dag.api.TezConfiguration;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.text.MessageFormat;
-import java.util.HashMap;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.hadoop.hive.ql.txn.compactor.TestDeltaFilesMetrics.gaugeToMap;
-import static org.apache.hadoop.hive.ql.txn.compactor.TestDeltaFilesMetrics.verifyMetricsMatch;
-import static org.apache.hadoop.hive.ql.txn.compactor.TestCompactor.executeStatementOnDriver;
-
-public class TestCompactionMetricsOnTez extends CompactorOnTezTest {
-
-  /**
-   * Use {@link CompactorOnTezTest#setupWithConf(org.apache.hadoop.hive.conf.HiveConf)} when HiveConf is
-   * configured to your liking.
-   */
-  @Override
-  public void setup() {
-  }
-
-  @After
-  public void tearDown() {
-    DeltaFilesMetricReporter.close();
-  }
-
-  /**
-   * Note: Does not initialize DeltaFilesMetricReporter.
-   * @param hiveConf
-   * @throws Exception
-   */
-  @Override
-  protected void setupWithConf(HiveConf hiveConf) throws Exception {
-    HiveConf.setIntVar(hiveConf, HiveConf.ConfVars.HIVE_TXN_ACID_METRICS_OBSOLETE_DELTA_NUM_THRESHOLD, 0);
-    HiveConf.setIntVar(hiveConf, HiveConf.ConfVars.HIVE_TXN_ACID_METRICS_DELTA_NUM_THRESHOLD, 0);
-    HiveConf.setTimeVar(hiveConf, HiveConf.ConfVars.HIVE_TXN_ACID_METRICS_REPORTING_INTERVAL, 1, TimeUnit.SECONDS);
-    HiveConf.setTimeVar(hiveConf, HiveConf.ConfVars.HIVE_TXN_ACID_METRICS_DELTA_CHECK_THRESHOLD, 0, TimeUnit.SECONDS);
-    HiveConf.setFloatVar(hiveConf, HiveConf.ConfVars.HIVE_TXN_ACID_METRICS_DELTA_PCT_THRESHOLD, 0.7f);
-    super.setupWithConf(hiveConf);
-    MetricsFactory.close();
-    MetricsFactory.init(hiveConf);
-  }
-
-  @Test
-  public void testDeltaFilesMetric() throws Exception {
-    HiveConf conf = new HiveConf();
-    HiveConf.setBoolVar(conf, HiveConf.ConfVars.HIVE_SERVER2_METRICS_ENABLED, true);
-    setupWithConf(conf);
-    DeltaFilesMetricReporter.init(conf);
-
-    String dbName = "default", tableName = "test_metrics";
-    String partitionToday = "ds=today", partitionTomorrow = "ds=tomorrow", partitionYesterday = "ds=yesterday";
-
-    CompactorOnTezTest.TestDataProvider testDataProvider = new CompactorOnTezTest.TestDataProvider();
-    testDataProvider.createFullAcidTable(tableName, true, false);
-    testDataProvider.insertTestDataPartitioned(tableName);
-
-    executeStatementOnDriver("select avg(b) from " + tableName, driver);
-    Thread.sleep(1000);
-
-    verifyMetricsMatch(new HashMap<String, String>() {{
-      put("default." + tableName + Path.SEPARATOR + partitionTomorrow, "3");
-      put("default." + tableName + Path.SEPARATOR + partitionYesterday, "4");
-      put("default." + tableName + Path.SEPARATOR + partitionToday, "5");
-        }}, gaugeToMap(MetricsConstants.COMPACTION_NUM_DELTAS));
-
-    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_OBSOLETE_DELTAS).size());
-    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_SMALL_DELTAS).size());
-
-    CompactorTestUtil.runCompaction(conf, dbName, tableName, CompactionType.MAJOR, true,
-        partitionToday, partitionTomorrow, partitionYesterday);
-
-    executeStatementOnDriver("select avg(b) from " + tableName, driver);
-    Thread.sleep(1000);
-
-    verifyMetricsMatch(new HashMap<String, String>() {{
-      put("default." + tableName + Path.SEPARATOR + partitionTomorrow, "3");
-      put("default." + tableName + Path.SEPARATOR + partitionYesterday, "4");
-      put("default." + tableName + Path.SEPARATOR + partitionToday, "5");
-        }}, gaugeToMap(MetricsConstants.COMPACTION_NUM_OBSOLETE_DELTAS));
-
-    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_DELTAS).size());
-
-    String insertQry = MessageFormat.format("insert into " + tableName + " partition (ds=''today'') values " +
-      "(''{0}'',1),(''{0}'',2),(''{0}'',3),(''{0}'',4),(''{0}'',5),(''{0}'',6),(''{0}'',7), (''{0}'',8),(''{0}'',9)," +
-      "(''{0}'',10),(''{0}'',11),(''{0}'',12)", RandomStringUtils.random(4096, false, true));
-    for (int i = 0; i < 10; i++) {
-      executeStatementOnDriver(insertQry, driver);
-    }
-    CompactorTestUtil.runCompaction(conf, dbName, tableName, CompactionType.MAJOR, true,
-        partitionToday);
-    executeStatementOnDriver("insert into " + tableName + " partition (ds='today') values('1',2)", driver);
-
-    executeStatementOnDriver("select avg(b) from " + tableName, driver);
-    Thread.sleep(1000);
-
-    verifyMetricsMatch(new HashMap<String, String>() {{
-          put("default." + tableName + Path.SEPARATOR + partitionToday, "1");
-        }}, gaugeToMap(MetricsConstants.COMPACTION_NUM_SMALL_DELTAS));
-  }
-
-  @Test
-  public void testDeltaFilesMetricMultiPartitioned() throws Exception {
-    HiveConf conf = new HiveConf();
-    HiveConf.setBoolVar(conf, HiveConf.ConfVars.HIVE_SERVER2_METRICS_ENABLED, true);
-    setupWithConf(conf);
-    DeltaFilesMetricReporter.init(conf);
-
-    String dbName = "default", tableName = "test_metrics";
-    String partitionToday = "ds=today", partitionTomorrow = "ds=tomorrow", partitionYesterday = "ds=yesterday";
-    String partitionB2 = "b=2", partitionB3 = "b=3", partitionB4 = "b=4";
-
-    CompactorOnTezTest.TestDataProvider testDataProvider = new CompactorOnTezTest.TestDataProvider();
-    testDataProvider.createFullAcidTable(tableName, true, false);
-    executeStatementOnDriver("drop table if exists " + tableName, driver);
-    executeStatementOnDriver("create table " + tableName + " (a string)"
-        + " partitioned by (b int, ds string)"
-        + " clustered by (a) into 2 buckets stored as orc "
-        + "TBLPROPERTIES('transactional'='true', 'transactional_properties'='default')", driver);
-    testDataProvider.insertTestDataPartitioned(tableName);
-
-    executeStatementOnDriver("select avg(b) from " + tableName, driver);
-    Thread.sleep(1000);
-
-    verifyMetricsMatch(new HashMap<String, String>() {{
-      put("default." + tableName + Path.SEPARATOR + partitionB2 + Path.SEPARATOR + partitionToday, "4");
-      put("default." + tableName + Path.SEPARATOR + partitionB2 + Path.SEPARATOR + partitionTomorrow, "3");
-      put("default." + tableName + Path.SEPARATOR + partitionB2 + Path.SEPARATOR + partitionYesterday, "1");
-
-      put("default." + tableName + Path.SEPARATOR + partitionB3 + Path.SEPARATOR + partitionToday, "4");
-      put("default." + tableName + Path.SEPARATOR + partitionB3 + Path.SEPARATOR + partitionTomorrow, "1");
-      put("default." + tableName + Path.SEPARATOR + partitionB3 + Path.SEPARATOR + partitionYesterday, "2");
-
-      put("default." + tableName + Path.SEPARATOR + partitionB4 + Path.SEPARATOR + partitionToday, "3");
-      put("default." + tableName + Path.SEPARATOR + partitionB4 + Path.SEPARATOR + partitionYesterday, "3");
-
-    }}, gaugeToMap(MetricsConstants.COMPACTION_NUM_DELTAS));
-
-    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_OBSOLETE_DELTAS).size());
-    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_SMALL_DELTAS).size());
-
-    CompactorTestUtil.runCompaction(conf, dbName, tableName, CompactionType.MAJOR, true,
-        partitionB2 + Path.SEPARATOR + partitionToday,
-        partitionB2 + Path.SEPARATOR + partitionTomorrow,
-        partitionB2 + Path.SEPARATOR + partitionYesterday,
-        partitionB3 + Path.SEPARATOR + partitionToday,
-        partitionB3 + Path.SEPARATOR + partitionTomorrow,
-        partitionB3 + Path.SEPARATOR + partitionYesterday,
-        partitionB4 + Path.SEPARATOR + partitionToday,
-        partitionB4 + Path.SEPARATOR + partitionTomorrow,
-        partitionB4 + Path.SEPARATOR + partitionYesterday
-        );
-
-    executeStatementOnDriver("select avg(b) from " + tableName, driver);
-    Thread.sleep(1000);
-
-    verifyMetricsMatch(new HashMap<String, String>() {{
-      put("default." + tableName + Path.SEPARATOR + partitionB2 + Path.SEPARATOR + partitionToday, "4");
-      // TODO uncomment after HIVE-25492 is fixed:
-//      put("default." + tableName + Path.SEPARATOR + partitionB2 + Path.SEPARATOR + partitionTomorrow, "3");
-      // 1 delta file isn't enough to compact partitionB2/partitionYesterday
-
-      put("default." + tableName + Path.SEPARATOR + partitionB3 + Path.SEPARATOR + partitionToday, "4");
-      // 1 delta file isn't enough to compact partitionB3/partitionTomorrow
-      put("default." + tableName + Path.SEPARATOR + partitionB3 + Path.SEPARATOR + partitionYesterday, "2");
-
-      put("default." + tableName + Path.SEPARATOR + partitionB4 + Path.SEPARATOR + partitionToday, "3");
-      put("default." + tableName + Path.SEPARATOR + partitionB4 + Path.SEPARATOR + partitionYesterday, "3");
-    }}, gaugeToMap(MetricsConstants.COMPACTION_NUM_OBSOLETE_DELTAS));
-
-    verifyMetricsMatch(new HashMap<String, String>() {{
-      // TODO remove after HIVE-25492 is fixed:
-      put("default." + tableName + Path.SEPARATOR + partitionB2 + Path.SEPARATOR + partitionTomorrow, "3");
-
-      put("default." + tableName + Path.SEPARATOR + partitionB2 + Path.SEPARATOR + partitionYesterday, "1");
-      put("default." + tableName + Path.SEPARATOR + partitionB3 + Path.SEPARATOR + partitionTomorrow, "1");
-    }}, gaugeToMap(MetricsConstants.COMPACTION_NUM_DELTAS));
-  }
-
-  /**
-   * Queries shouldn't fail, but metrics should be 0, if tez.counters.max limit is passed.
-   * @throws Exception
-   */
-  @Test
-  public void testDeltaFilesMetricTezMaxCounters() throws Exception {
-    HiveConf conf = new HiveConf();
-    conf.setInt(TezConfiguration.TEZ_COUNTERS_MAX, 50);
-    HiveConf.setBoolVar(conf, HiveConf.ConfVars.HIVE_SERVER2_METRICS_ENABLED, true);
-    setupWithConf(conf);
-
-    DeltaFilesMetricReporter.init(conf);
-
-    String tableName = "test_metrics";
-    CompactorOnTezTest.TestDataProvider testDataProvider = new CompactorOnTezTest.TestDataProvider();
-    testDataProvider.createFullAcidTable(tableName, true, false);
-    // Create 51 partitions
-    for (int i = 0; i < 51; i++) {
-      executeStatementOnDriver("insert into " + tableName + " values('1', " + i * i + ", '" + i + "')", driver);
-    }
-
-    // Touch all partitions
-    executeStatementOnDriver("select avg(b) from " + tableName, driver);
-    Thread.sleep(1000);
-
-    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_DELTAS).size());
-    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_OBSOLETE_DELTAS).size());
-    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_SMALL_DELTAS).size());
-  }
-
-  /**
-   * Queries should succeed if additional acid metrics are disabled.
-   * @throws Exception
-   */
-  @Test(expected = javax.management.InstanceNotFoundException.class)
-  public void testDeltaFilesMetricWithMetricsDisabled() throws Exception {
-    HiveConf conf = new HiveConf();
-    HiveConf.setBoolVar(conf, HiveConf.ConfVars.HIVE_SERVER2_METRICS_ENABLED, false);
-    MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON, true);
-
-    verifyQueryRuns(conf);
-  }
-
-
-  /**
-   * Queries should succeed if extended metrics are disabled.
-   * @throws Exception
-   */
-  @Test(expected = javax.management.InstanceNotFoundException.class)
-  public void testDeltaFilesMetricWithExtMetricsDisabled() throws Exception {
-    HiveConf conf = new HiveConf();
-    HiveConf.setBoolVar(conf, HiveConf.ConfVars.HIVE_SERVER2_METRICS_ENABLED, true);
-    MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON, false);
-
-    verifyQueryRuns(conf);
-  }
-
-  private void verifyQueryRuns(HiveConf conf) throws Exception {
-    setupWithConf(conf);
-    // DeltaFilesMetricReporter is not instantiated because either metrics or extended metrics are disabled.
-
-    String tableName = "test_metrics";
-    TestDataProvider testDataProvider = new TestDataProvider();
-    testDataProvider.createFullAcidTable(tableName, true, false);
-    testDataProvider.insertTestDataPartitioned(tableName);
-
-    executeStatementOnDriver("select avg(b) from " + tableName, driver);
-    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_DELTAS).size());
-  }
-}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
index db51e30..be817c8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
@@ -38,11 +38,8 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.ql.exec.Operator;
-import org.apache.hadoop.hive.ql.exec.SerializationUtilities;
 import org.apache.hadoop.hive.ql.io.HiveInputFormat;
 import org.apache.hadoop.hive.ql.plan.PartitionDesc;
-import org.apache.hadoop.hive.ql.plan.TableDesc;
-import org.apache.hadoop.hive.ql.txn.compactor.metrics.DeltaFilesMetricReporter;
 import org.apache.tez.common.counters.TezCounters;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -229,24 +226,6 @@ public class HiveSplitGenerator extends InputInitializer {
 
         }
 
-        HashMap<Path, DeltaFilesMetricReporter.DeltaFilesMetadata> deltaFilesMetadata = new HashMap();
-
-        work.getPathToPartitionInfo().entrySet().forEach(e -> {
-          DeltaFilesMetricReporter.DeltaFilesMetadata metadata = new DeltaFilesMetricReporter.DeltaFilesMetadata();
-          TableDesc tableDesc = e.getValue().getTableDesc();
-          metadata.dbName = tableDesc.getDbName();
-          metadata.tableName = tableDesc.getTableName();
-          LinkedHashMap<String, String> partSpec = e.getValue().getPartSpec();
-          if (partSpec != null && !partSpec.isEmpty()) {
-            metadata.partitionName = Joiner.on(Path.SEPARATOR).join(partSpec.entrySet());
-          }
-          deltaFilesMetadata.put(e.getKey(), metadata);
-        });
-
-        String serializedMetadata = SerializationUtilities.serializeObject(deltaFilesMetadata);
-        jobConf.set(DeltaFilesMetricReporter.JOB_CONF_DELTA_FILES_METRICS_METADATA, serializedMetadata);
-
-
         InputSplit[] splits;
         if (generateSingleSplit &&
           conf.get(HiveConf.ConfVars.HIVETEZINPUTFORMAT.varname).equals(HiveInputFormat.class.getName())) {
@@ -319,7 +298,6 @@ public class HiveSplitGenerator extends InputInitializer {
             }
             counterName = Utilities.getVertexCounterName(HiveInputCounters.INPUT_FILES.name(), vertexName);
             tezCounters.findCounter(groupName, counterName).increment(files.size());
-            DeltaFilesMetricReporter.createCountersForAcidMetrics(tezCounters, jobConf);
           } catch (Exception e) {
             LOG.warn("Caught exception while trying to update Tez counters", e);
           }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
index 6e76a33..2235904 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
@@ -63,7 +63,6 @@ import org.apache.hadoop.hive.ql.plan.TezWork;
 import org.apache.hadoop.hive.ql.plan.UnionWork;
 import org.apache.hadoop.hive.ql.plan.api.StageType;
 import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.hadoop.hive.ql.txn.compactor.metrics.DeltaFilesMetricReporter;
 import org.apache.hadoop.hive.ql.wm.WmContext;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -266,10 +265,6 @@ public class TezTask extends Task<TezWork> {
           Set<StatusGetOpts> statusGetOpts = EnumSet.of(StatusGetOpts.GET_COUNTERS);
           TezCounters dagCounters = dagClient.getDAGStatus(statusGetOpts).getDAGCounters();
 
-          if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_SERVER2_METRICS_ENABLED) &&
-              MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON)) {
-            DeltaFilesMetricReporter.getInstance().submit(dagCounters, queryPlan.getInputs());
-          }
           // if initial counters exists, merge it with dag counters to get aggregated view
           TezCounters mergedCounters = counters == null ? dagCounters : Utils.mergeTezCounters(dagCounters, counters);
           counters = mergedCounters;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
index 3331228..787b627 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
@@ -862,7 +862,6 @@ public class HiveInputFormat<K extends WritableComparable, V extends Writable>
           currentInputFormatClass, currentDirs.size()*(numSplits / dirs.length),
           currentTable, result);
     }
-    DeltaFilesMetricReporter.backPropagateAcidMetrics(job, newjob);
 
     Utilities.clearWorkMapForConf(job);
     LOG.info("number of splits " + result.size());
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
index b65efc7..a2936b3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
@@ -31,23 +31,19 @@ import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
-import java.util.EnumMap;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
 import java.util.Set;
-import java.util.Queue;
 import java.util.TreeMap;
-import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CompletionService;
 import java.util.concurrent.ExecutorCompletionService;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
-import java.util.concurrent.PriorityBlockingQueue;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -62,13 +58,11 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
 import org.apache.hadoop.hive.common.ValidWriteIdList;
-import org.apache.hadoop.hive.conf.Constants;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.Metastore;
 import org.apache.hadoop.hive.metastore.Metastore.SplitInfos;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
-import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedInputFormatInterface;
@@ -104,8 +98,6 @@ import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.TruthValue;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
-import org.apache.hadoop.hive.ql.txn.compactor.metrics.DeltaFilesMetricReporter;
-import org.apache.hadoop.hive.ql.txn.compactor.metrics.DeltaFilesMetricReporter.DeltaFilesMetricType;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
 import org.apache.hadoop.hive.serde2.SerDeStats;
@@ -1792,17 +1784,6 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
         " reader schema " + (readerSchema == null ? "NULL" : readerSchema.toString()) +
         " ACID scan property " + isAcidTableScan);
     }
-    boolean metricsEnabled = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_SERVER2_METRICS_ENABLED) &&
-        MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON);
-    long checkThresholdInSec = HiveConf.getTimeVar(conf,
-        HiveConf.ConfVars.HIVE_TXN_ACID_METRICS_DELTA_CHECK_THRESHOLD, TimeUnit.SECONDS);
-    float deltaPctThreshold = HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVE_TXN_ACID_METRICS_DELTA_PCT_THRESHOLD);
-    int deltasThreshold = HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVE_TXN_ACID_METRICS_DELTA_NUM_THRESHOLD);
-    int obsoleteDeltasThreshold = HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVE_TXN_ACID_METRICS_OBSOLETE_DELTA_NUM_THRESHOLD);
-    int maxCacheSize = HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVE_TXN_ACID_METRICS_MAX_CACHE_SIZE);
-
-    EnumMap<DeltaFilesMetricType, Queue<Pair<String, Integer>>> deltaFilesStats =
-        new EnumMap<>(DeltaFilesMetricType.class);
 
     // complete path futures and schedule split generation
     try {
@@ -1829,10 +1810,6 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
           continue;
         }
 
-        if (metricsEnabled && directory instanceof AcidDirectory) {
-          DeltaFilesMetricReporter.mergeDeltaFilesStats((AcidDirectory) directory, checkThresholdInSec,
-              deltaPctThreshold, deltasThreshold, obsoleteDeltasThreshold, maxCacheSize, deltaFilesStats, conf);
-        }
         // We have received a new directory information, make split strategies.
         --resultsLeft;
         if (directory.getFiles().isEmpty()) {
@@ -1862,9 +1839,6 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
           }
         }
       }
-      if (metricsEnabled) {
-        DeltaFilesMetricReporter.addAcidMetricsToConfObj(deltaFilesStats, conf);
-      }
 
       // Run the last combined strategy, if any.
       if (combinedCtx != null && combinedCtx.combined != null) {
@@ -1973,7 +1947,6 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
     }
     List<OrcSplit> result = generateSplitsInfo(conf,
         new Context(conf, numSplits, createExternalCaches()));
-    DeltaFilesMetricReporter.backPropagateAcidMetrics(job, conf);
 
     long end = System.currentTimeMillis();
     LOG.info("getSplits finished (#splits: {}). duration: {} ms", result.size(), (end - start));
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
index bca9a29..db65250 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
@@ -20,10 +20,24 @@ package org.apache.hadoop.hive.ql.txn.compactor;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.metastore.ReplChangeManager;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
+import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsRequest;
 import org.apache.hadoop.hive.metastore.api.GetValidWriteIdsResponse;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.api.TxnOpenException;
+import org.apache.hadoop.hive.metastore.api.UnlockRequest;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.metrics.Metrics;
 import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
@@ -35,6 +49,7 @@ import org.apache.hadoop.hive.metastore.utils.FileUtils;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.io.AcidDirectory;
 import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil.ThrowingRunnable;
+import org.apache.hadoop.hive.ql.txn.compactor.metrics.DeltaFilesMetricReporter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.fs.FileSystem;
@@ -43,20 +58,6 @@ import org.apache.hadoop.hive.common.ValidWriteIdList;
 import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
 import org.apache.hadoop.hive.common.StringableMap;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.api.Database;
-import org.apache.hadoop.hive.metastore.api.DataOperationType;
-import org.apache.hadoop.hive.metastore.api.LockRequest;
-import org.apache.hadoop.hive.metastore.api.LockResponse;
-import org.apache.hadoop.hive.metastore.api.LockState;
-import org.apache.hadoop.hive.metastore.api.LockType;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
-import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
-import org.apache.hadoop.hive.metastore.api.TxnOpenException;
-import org.apache.hadoop.hive.metastore.api.UnlockRequest;
 import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -91,6 +92,7 @@ public class Cleaner extends MetaStoreCompactorThread {
   static final private String CLASS_NAME = Cleaner.class.getName();
   static final private Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
   private long cleanerCheckInterval = 0;
+  private boolean metricsEnabled = false;
 
   private ReplChangeManager replChangeManager;
   private ExecutorService cleanerExecutor;
@@ -104,14 +106,15 @@ public class Cleaner extends MetaStoreCompactorThread {
     cleanerExecutor = CompactorUtil.createExecutorWithThreadFactory(
             conf.getIntVar(HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_THREADS_NUM),
             COMPACTOR_CLEANER_THREAD_NAME_FORMAT);
+    metricsEnabled = MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METRICS_ENABLED) &&
+        MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON) &&
+        MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.COMPACTOR_INITIATOR_ON);
   }
 
   @Override
   public void run() {
     LOG.info("Starting Cleaner thread");
     try {
-      boolean metricsEnabled = MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METRICS_ENABLED) &&
-          MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON);
       Counter failuresCounter = Metrics.getOrCreateCounter(MetricsConstants.COMPACTION_CLEANER_FAILURE_COUNTER);
       do {
         TxnStore.MutexAPI.LockHandle handle = null;
@@ -188,6 +191,12 @@ public class Cleaner extends MetaStoreCompactorThread {
     }
   }
 
+  private void updateDeltaFilesMetrics(String dbName, String tableName, String partName, List<Path> obsoleteDirs) {
+    if (metricsEnabled) {
+      DeltaFilesMetricReporter.updateMetricsFromCleaner(dbName, tableName, partName, obsoleteDirs, conf, txnHandler);
+    }
+  }
+
   private void clean(CompactionInfo ci, long minOpenTxnGLB, boolean metricsEnabled) throws MetaException {
     LOG.info("Starting cleaning for " + ci);
     PerfLogger perfLogger = PerfLogger.getPerfLogger(false);
@@ -412,7 +421,11 @@ public class Cleaner extends MetaStoreCompactorThread {
     }
     StringBuilder extraDebugInfo = new StringBuilder("[").append(obsoleteDirs.stream()
         .map(Path::getName).collect(Collectors.joining(",")));
-    return remove(location, ci, obsoleteDirs, true, fs, extraDebugInfo);
+    boolean success = remove(location, ci, obsoleteDirs, true, fs, extraDebugInfo);
+    if (dir.getObsolete().size() > 0) {
+      updateDeltaFilesMetrics(ci.dbname, ci.tableName, ci.partName, dir.getObsolete());
+    }
+    return success;
   }
 
   private boolean removeFiles(String location, CompactionInfo ci)
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 ad52182..d8a3bcb 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
@@ -29,6 +29,7 @@ import org.apache.hadoop.hive.common.ServerUtils;
 import org.apache.hadoop.hive.common.ValidReadTxnList;
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.common.ValidWriteIdList;
+import org.apache.hadoop.hive.common.metrics.common.MetricsFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.CompactionRequest;
 import org.apache.hadoop.hive.metastore.api.CompactionResponse;
@@ -54,6 +55,7 @@ import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.apache.hadoop.hive.ql.io.AcidDirectory;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.io.AcidUtils.ParsedDirectory;
+import org.apache.hadoop.hive.ql.txn.compactor.metrics.DeltaFilesMetricReporter;
 import org.apache.hadoop.hive.shims.HadoopShims.HdfsFileStatusWithId;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
@@ -95,6 +97,7 @@ public class Initiator extends MetaStoreCompactorThread {
   private long checkInterval;
   private ExecutorService compactionExecutor;
   private Optional<Cache<String, Table>> tableCache = Optional.empty();
+  private boolean metricsEnabled;
 
   @Override
   public void run() {
@@ -109,8 +112,6 @@ public class Initiator extends MetaStoreCompactorThread {
       long abortedTimeThreshold = HiveConf
           .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD,
               TimeUnit.MILLISECONDS);
-      boolean metricsEnabled = MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METRICS_ENABLED) &&
-          MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON);
       Counter failuresCounter = Metrics.getOrCreateCounter(MetricsConstants.COMPACTION_INITIATOR_FAILURE_COUNTER);
 
       // Make sure we run through the loop once before checking to stop as this makes testing
@@ -299,6 +300,13 @@ public class Initiator extends MetaStoreCompactorThread {
     if (tableCacheOn) {
       this.tableCache = Optional.of(CacheBuilder.newBuilder().softValues().build());
     }
+    metricsEnabled = MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METRICS_ENABLED) &&
+        MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON) &&
+        MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.COMPACTOR_INITIATOR_ON);
+    if (metricsEnabled) {
+      MetricsFactory.init(conf);
+      DeltaFilesMetricReporter.init(conf, txnHandler);
+    }
   }
 
   private void recoverFailedCompactions(boolean remoteOnly) throws MetaException {
@@ -355,6 +363,14 @@ public class Initiator extends MetaStoreCompactorThread {
     }
     return false;
   }
+  
+  private void updateDeltaFilesMetrics(AcidDirectory directory, String dbName, String tableName, String partName,
+      long baseSize, Map<Path, Long> deltaSizes) {
+    if (metricsEnabled) {
+      DeltaFilesMetricReporter.updateMetricsFromInitiator(directory, dbName, tableName, partName, conf, txnHandler,
+          baseSize, deltaSizes);
+    }
+  }
 
   private CompactionType checkForCompaction(final CompactionInfo ci,
                                             final ValidWriteIdList writeIds,
@@ -378,9 +394,18 @@ public class Initiator extends MetaStoreCompactorThread {
           + "Initiating minor compaction.");
       return CompactionType.MINOR;
     }
+    AcidDirectory acidDirectory = getAcidDirectory(sd, writeIds);
+    long baseSize = getBaseSize(acidDirectory);
+    FileSystem fs = acidDirectory.getFs();
+    Map<Path, Long> deltaSizes = new HashMap<>();
+    for (AcidUtils.ParsedDelta delta : acidDirectory.getCurrentDirectories()) {
+      deltaSizes.put(delta.getPath(), getDirSize(fs, delta));
+    }
+    long deltaSize = deltaSizes.values().stream().reduce(0L, Long::sum);
+    updateDeltaFilesMetrics(acidDirectory, ci.dbname, ci.tableName, ci.partName, baseSize, deltaSizes);
 
     if (runJobAsSelf(runAs)) {
-      return determineCompactionType(ci, writeIds, sd, tblproperties);
+      return determineCompactionType(ci, acidDirectory, tblproperties, baseSize, deltaSize);
     } else {
       LOG.info("Going to initiate as user " + runAs + " for " + ci.getFullPartitionName());
       UserGroupInformation ugi = UserGroupInformation.createProxyUser(runAs,
@@ -390,7 +415,7 @@ public class Initiator extends MetaStoreCompactorThread {
         compactionType = ugi.doAs(new PrivilegedExceptionAction<CompactionType>() {
           @Override
           public CompactionType run() throws Exception {
-            return determineCompactionType(ci, writeIds, sd, tblproperties);
+            return determineCompactionType(ci, acidDirectory, tblproperties, baseSize, deltaSize);
           }
         });
       } finally {
@@ -405,29 +430,16 @@ public class Initiator extends MetaStoreCompactorThread {
     }
   }
 
-  private CompactionType determineCompactionType(CompactionInfo ci, ValidWriteIdList writeIds,
-                                                 StorageDescriptor sd, Map<String, String> tblproperties)
-      throws IOException {
-
-    boolean noBase = false;
+  private AcidDirectory getAcidDirectory(StorageDescriptor sd, ValidWriteIdList writeIds) throws IOException {
     Path location = new Path(sd.getLocation());
     FileSystem fs = location.getFileSystem(conf);
-    AcidDirectory dir = AcidUtils.getAcidState(fs, location, conf, writeIds, Ref.from(false), false);
-    long baseSize = 0;
-    if (dir.getBase() != null) {
-      baseSize = sumDirSize(fs, dir.getBase());
-    } else {
-      for (HdfsFileStatusWithId origStat : dir.getOriginalFiles()) {
-        baseSize += origStat.getFileStatus().getLen();
-      }
-    }
+    return AcidUtils.getAcidState(fs, location, conf, writeIds, Ref.from(false), false);
+  }
 
-    long deltaSize = 0;
+  private CompactionType determineCompactionType(CompactionInfo ci, AcidDirectory dir, Map<String,
+      String> tblproperties, long baseSize, long deltaSize) throws IOException {
+    boolean noBase = false;
     List<AcidUtils.ParsedDelta> deltas = dir.getCurrentDirectories();
-    for (AcidUtils.ParsedDelta delta : deltas) {
-      deltaSize += sumDirSize(fs, delta);
-    }
-
     if (baseSize == 0 && deltaSize > 0) {
       noBase = true;
     } else {
@@ -487,7 +499,19 @@ public class Initiator extends MetaStoreCompactorThread {
     return noBase ? CompactionType.MAJOR : CompactionType.MINOR;
   }
 
-  private long sumDirSize(FileSystem fs, ParsedDirectory dir) throws IOException {
+  private long getBaseSize(AcidDirectory dir) throws IOException {
+    long baseSize = 0;
+    if (dir.getBase() != null) {
+      baseSize = getDirSize(dir.getFs(), dir.getBase());
+    } else {
+      for (HdfsFileStatusWithId origStat : dir.getOriginalFiles()) {
+        baseSize += origStat.getFileStatus().getLen();
+      }
+    }
+    return baseSize;
+  }
+
+  private long getDirSize(FileSystem fs, ParsedDirectory dir) throws IOException {
     long size = dir.getFiles(fs, Ref.from(false)).stream()
         .map(HdfsFileStatusWithId::getFileStatus)
         .mapToLong(FileStatus::getLen)
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 d4aa6a1..ff3b3f6 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
@@ -46,6 +46,7 @@ import org.apache.hadoop.hive.ql.DriverUtils;
 import org.apache.hadoop.hive.ql.io.AcidDirectory;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.log.PerfLogger;
+import org.apache.hadoop.hive.ql.txn.compactor.metrics.DeltaFilesMetricReporter;
 import org.apache.hive.common.util.Ref;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.thrift.TException;
@@ -86,6 +87,7 @@ public class Worker extends RemoteCompactorThread implements MetaStoreThread {
   static final private long SLEEP_TIME = 10000;
 
   private String workerName;
+  private boolean metricsEnabled;
 
   // TODO: this doesn't check if compaction is already running (even though Initiator does but we
   //  don't go through Initiator for user initiated compactions)
@@ -144,6 +146,9 @@ public class Worker extends RemoteCompactorThread implements MetaStoreThread {
     super.init(stop);
     this.workerName = getWorkerId();
     setName(workerName);
+    // To enable delta metrics collection, initiator must be enabled on HMS side
+    metricsEnabled = MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METRICS_ENABLED) &&
+        MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON);
   }
 
   @VisibleForTesting
@@ -493,6 +498,9 @@ public class Worker extends RemoteCompactorThread implements MetaStoreThread {
             + compactionTxn + ", marking as compacted.");
         msc.markCompacted(CompactionInfo.compactionInfoToStruct(ci));
         compactionTxn.wasSuccessful();
+
+        updateDeltaFilesMetrics(dir, ci.dbname, ci.tableName, ci.partName, ci.type);
+
       } catch (Throwable e) {
         LOG.error("Caught exception while trying to compact " + ci +
             ". Marking failed to avoid repeated failures", e);
@@ -650,6 +658,13 @@ public class Worker extends RemoteCompactorThread implements MetaStoreThread {
     return name.toString();
   }
 
+  private void updateDeltaFilesMetrics(AcidDirectory directory, String dbName, String tableName, String partName,
+      CompactionType type) {
+    if (metricsEnabled) {
+      DeltaFilesMetricReporter.updateMetricsFromWorker(directory, dbName, tableName, partName, type, conf, msc);
+    }
+  }
+
   /**
    * Keep track of the compaction's transaction and its operations.
    */
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/metrics/DeltaFilesMetricReporter.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/metrics/DeltaFilesMetricReporter.java
index 311d5ba..2cc66ef 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/metrics/DeltaFilesMetricReporter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/metrics/DeltaFilesMetricReporter.java
@@ -17,116 +17,69 @@
  */
 package org.apache.hadoop.hive.ql.txn.compactor.metrics;
 
-import com.google.common.base.Joiner;
-import com.google.common.base.Splitter;
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
-
-import com.google.common.cache.RemovalNotification;
-import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.AcidConstants;
 import org.apache.hadoop.hive.common.metrics.MetricsMBeanImpl;
 import org.apache.hadoop.hive.common.metrics.common.Metrics;
 import org.apache.hadoop.hive.common.metrics.common.MetricsFactory;
-import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.CompactionMetricsDataRequest;
+import org.apache.hadoop.hive.metastore.api.CompactionMetricsDataStruct;
+import org.apache.hadoop.hive.metastore.api.CompactionMetricsMetricType;
+import org.apache.hadoop.hive.metastore.api.CompactionType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
-import org.apache.hadoop.hive.ql.exec.SerializationUtilities;
-import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.metastore.txn.CompactionMetricsData;
+import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.ql.io.AcidDirectory;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
-
-import org.apache.hadoop.hive.shims.HadoopShims;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hive.common.util.Ref;
-import org.apache.tez.common.counters.CounterGroup;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.common.counters.TezCounters;
+import org.apache.thrift.TException;
 import org.jetbrains.annotations.NotNull;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
-import java.io.IOException;
-import java.io.Serializable;
 import java.lang.management.ManagementFactory;
-import java.util.Arrays;
 import java.util.ArrayList;
-import java.util.Comparator;
-import java.util.Date;
-import java.util.EnumMap;
-import java.util.HashMap;
 import java.util.List;
-import java.util.PriorityQueue;
-import java.util.Queue;
-
-import java.util.Set;
+import java.util.Map;
 import java.util.concurrent.Executors;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.PriorityBlockingQueue;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 import static org.apache.hadoop.hive.metastore.metrics.MetricsConstants.COMPACTION_NUM_DELTAS;
 import static org.apache.hadoop.hive.metastore.metrics.MetricsConstants.COMPACTION_NUM_OBSOLETE_DELTAS;
 import static org.apache.hadoop.hive.metastore.metrics.MetricsConstants.COMPACTION_NUM_SMALL_DELTAS;
-
-import static org.apache.hadoop.hive.ql.txn.compactor.metrics.DeltaFilesMetricReporter.DeltaFilesMetricType.NUM_DELTAS;
-import static org.apache.hadoop.hive.ql.txn.compactor.metrics.DeltaFilesMetricReporter.DeltaFilesMetricType.NUM_OBSOLETE_DELTAS;
-import static org.apache.hadoop.hive.ql.txn.compactor.metrics.DeltaFilesMetricReporter.DeltaFilesMetricType.NUM_SMALL_DELTAS;
+import static org.apache.hadoop.hive.metastore.txn.CompactionMetricsData.MetricType.NUM_DELTAS;
+import static org.apache.hadoop.hive.metastore.txn.CompactionMetricsData.MetricType.NUM_SMALL_DELTAS;
 
 /**
  * Collects and publishes ACID compaction related metrics.
- * Everything should be behind 2 feature flags: {@link HiveConf.ConfVars#HIVE_SERVER2_METRICS_ENABLED} and
+ * Everything should be behind 2 feature flags: {@link MetastoreConf.ConfVars#METRICS_ENABLED} and
  * {@link MetastoreConf.ConfVars#METASTORE_ACIDMETRICS_EXT_ON}.
- * First we store the information in the jobConf, then in Tez Counters, then in a cache stored here, then in a custom
- * MBean.
+ * First we store the information in the HMS backend DB COMPACTION_METRICS_CACHE table, then in a custom MBean.
+ * The contents if the backend table is logged out by a separate thread in a
+ * {@link MetastoreConf.ConfVars#METASTORE_DELTAMETRICS_LOGGER_FREQUENCY} frequency.
  */
 public class DeltaFilesMetricReporter {
 
   private static final Logger LOG = LoggerFactory.getLogger(AcidUtils.class);
 
   public static final String OBJECT_NAME_PREFIX = "metrics:type=compaction,name=";
-  public static final String JOB_CONF_DELTA_FILES_METRICS_METADATA = "delta.files.metrics.metadata";
-  public static final char ENTRY_SEPARATOR = ';';
-  public static final String KEY_VALUE_SEPARATOR = "->";
-
-  private static long lastSuccessfulLoggingTime = 0;
-  private String hiveEntitySeparator;
-
-  public enum DeltaFilesMetricType {
-    NUM_OBSOLETE_DELTAS("HIVE_ACID_NUM_OBSOLETE_DELTAS"),
-    NUM_DELTAS("HIVE_ACID_NUM_DELTAS"),
-    NUM_SMALL_DELTAS("HIVE_ACID_NUM_SMALL_DELTAS");
-
-    private final String value;
-
-    DeltaFilesMetricType(String value) {
-      this.value = value;
-    }
-
-    @Override
-    public String toString() {
-      return value;
-    }
-  }
 
-  private Cache<String, Integer> deltaCache, smallDeltaCache;
-  private Cache<String, Integer> obsoleteDeltaCache;
+  private static boolean initialized = false;
 
   private MetricsMBeanImpl deltaObject, smallDeltaObject, obsoleteDeltaObject;
-  private List<ObjectName> registeredObjects = new ArrayList<>();
-
-  private BlockingQueue<Pair<String, Integer>> deltaTopN, smallDeltaTopN;
-  private BlockingQueue<Pair<String, Integer>> obsoleteDeltaTopN;
+  private final List<ObjectName> registeredObjects = new ArrayList<>();
 
-  private ScheduledExecutorService executorService;
+  private ScheduledExecutorService reporterExecutorService;
+  private ScheduledExecutorService loggerExecutorService;
+  private int maxCacheSize;
 
   private static class InstanceHolder {
     public static DeltaFilesMetricReporter instance = new DeltaFilesMetricReporter();
@@ -139,157 +92,37 @@ public class DeltaFilesMetricReporter {
     return InstanceHolder.instance;
   }
 
-  public static synchronized void init(HiveConf conf) throws Exception {
-    getInstance().configure(conf);
+  public static synchronized void init(Configuration conf, TxnStore txnHandler) throws Exception {
+    if (!initialized) {
+      getInstance().configure(conf, txnHandler);
+      initialized = true;
+    }
   }
 
-  private void configure(HiveConf conf) throws Exception {
+  private void configure(Configuration conf, TxnStore txnHandler) throws Exception {
     long reportingInterval =
-        HiveConf.getTimeVar(conf, HiveConf.ConfVars.HIVE_TXN_ACID_METRICS_REPORTING_INTERVAL, TimeUnit.SECONDS);
-    hiveEntitySeparator = conf.getVar(HiveConf.ConfVars.HIVE_ENTITY_SEPARATOR);
+        MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.METASTORE_DELTAMETRICS_REPORTING_INTERVAL, TimeUnit.SECONDS);
+
+    maxCacheSize = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.METASTORE_DELTAMETRICS_MAX_CACHE_SIZE);
 
-    initCachesForMetrics(conf);
     initObjectsForMetrics();
 
     ThreadFactory threadFactory =
         new ThreadFactoryBuilder().setDaemon(true).setNameFormat("DeltaFilesMetricReporter %d").build();
-    executorService = Executors.newSingleThreadScheduledExecutor(threadFactory);
-    executorService.scheduleAtFixedRate(new ReportingTask(), 0, reportingInterval, TimeUnit.SECONDS);
+    reporterExecutorService = Executors.newSingleThreadScheduledExecutor(threadFactory);
+    reporterExecutorService.scheduleAtFixedRate(new ReportingTask(txnHandler), 0, reportingInterval, TimeUnit.SECONDS);
 
     LOG.info("Started DeltaFilesMetricReporter thread");
-  }
-
-  public void submit(TezCounters counters, Set<ReadEntity> inputs) {
-    try {
-      updateMetrics(NUM_OBSOLETE_DELTAS, obsoleteDeltaCache, obsoleteDeltaTopN, counters, inputs);
-      updateMetrics(NUM_DELTAS, deltaCache, deltaTopN, counters, inputs);
-      updateMetrics(NUM_SMALL_DELTAS, smallDeltaCache, smallDeltaTopN, counters, inputs);
-    } catch (Exception e) {
-      LOG.warn("Caught exception while trying to update delta metrics cache. Invalidating cache", e);
-      try {
-        obsoleteDeltaCache.invalidateAll();
-        deltaCache.invalidateAll();
-        smallDeltaCache.invalidateAll();
-      } catch (Exception x) {
-        LOG.warn("Caught exception while trying to invalidate cache", x);
-      }
-    }
-  }
-
-  /**
-   * Copy counters to caches.
-   */
-  private void updateMetrics(DeltaFilesMetricType metric, Cache<String, Integer> cache,
-      Queue<Pair<String, Integer>> topN, TezCounters counters, Set<ReadEntity> inputs) {
-
-    // Create list of paths affected by the query
-    List<String> inputPaths = Lists.newArrayList();
-    if (inputs != null) {
-      inputs.stream().map(readEntity -> readEntity.getName().split(hiveEntitySeparator)).forEach(inputNames -> {
-        String dbName = inputNames[0];
-        String tableName = inputNames[1];
-        String partitionName = inputNames.length > 2 ? inputNames[2] : null;
-        inputPaths.add(getDeltaCountKey(dbName, tableName, partitionName));
-      });
-    }
-
-    // Invalidate from cache if the counter value differs from the cache value, or if the query touched the partition
-    // in question but no counter was collected
-    CounterGroup group = counters.getGroup(metric.value);
-    for (String key : inputPaths) {
-      Integer prev = cache.getIfPresent(key);
-      if (prev != null) {
-        TezCounter counter = counters.findCounter(group.getName(), key);
-        if (counter != null && (counter.getValue() == 0 || counter.getValue() != prev)) {
-          cache.invalidate(key);
-        }
-      }
-    }
-
-    // Add all counter values to the cache
-    for (TezCounter counter : group) {
-      if (counter.getValue() != 0) {
-        topN.add(Pair.of(counter.getName(), (int) counter.getValue()));
-        cache.put(counter.getName(), (int) counter.getValue());
-      }
-    }
-  }
-
-  /**
-   * Update EnumMap<DeltaFilesMetricType, Queue<Pair<String, Integer>>> deltaFilesStats with {@link AcidDirectory}
-   * contents
-   */
-  public static void mergeDeltaFilesStats(AcidDirectory dir, long checkThresholdInSec, float deltaPctThreshold,
-      int deltasThreshold, int obsoleteDeltasThreshold, int maxCacheSize,
-      EnumMap<DeltaFilesMetricType, Queue<Pair<String, Integer>>> deltaFilesStats, Configuration conf)
-      throws IOException {
-
-    try {
-      long baseSize = getBaseSize(dir);
-      int numObsoleteDeltas = getNumObsoleteDeltas(dir, checkThresholdInSec);
-
-      int numDeltas = 0;
-      int numSmallDeltas = 0;
-
-      long now = new Date().getTime();
-
-      for (AcidUtils.ParsedDelta delta : dir.getCurrentDirectories()) {
-        if (now - getModificationTime(delta, dir.getFs()) >= checkThresholdInSec * 1000) {
-          numDeltas++;
-
-          long deltaSize = getDirSize(delta, dir.getFs());
-          if (baseSize != 0 && deltaSize / (float) baseSize < deltaPctThreshold) {
-            numSmallDeltas++;
-          }
-        }
-      }
 
-      logDeltaDirMetrics(dir, conf, numObsoleteDeltas, numDeltas, numSmallDeltas);
+    long loggerFrequency = MetastoreConf
+        .getTimeVar(conf, MetastoreConf.ConfVars.METASTORE_DELTAMETRICS_LOGGER_FREQUENCY, TimeUnit.MINUTES);
+    loggerExecutorService = Executors.newSingleThreadScheduledExecutor(
+        new ThreadFactoryBuilder().setDaemon(true).setNameFormat("DeltaFilesMetricLogger %d").build());
+    loggerExecutorService.scheduleAtFixedRate(new LoggerTask(txnHandler), loggerFrequency, loggerFrequency, TimeUnit.MINUTES);
 
-      String serializedMetadata = conf.get(JOB_CONF_DELTA_FILES_METRICS_METADATA);
-      if (serializedMetadata == null) {
-        LOG.warn("delta.files.metrics.metadata is missing from config. Delta metrics can't be updated.");
-        return;
-      }
-      HashMap<Path, DeltaFilesMetadata> pathToMetadata = new HashMap<>();
-      pathToMetadata = SerializationUtilities.deserializeObject(serializedMetadata, pathToMetadata.getClass());
-      if (pathToMetadata == null) {
-        LOG.warn("Delta metrics can't be updated since the metadata is null.");
-        return;
-      }
-      DeltaFilesMetadata metadata = pathToMetadata.get(dir.getPath());
-      filterAndAddToDeltaFilesStats(NUM_DELTAS, numDeltas, deltasThreshold, deltaFilesStats, metadata, maxCacheSize);
-      filterAndAddToDeltaFilesStats(NUM_OBSOLETE_DELTAS, numObsoleteDeltas, obsoleteDeltasThreshold, deltaFilesStats,
-          metadata, maxCacheSize);
-      filterAndAddToDeltaFilesStats(NUM_SMALL_DELTAS, numSmallDeltas, deltasThreshold, deltaFilesStats, metadata,
-          maxCacheSize);
-    } catch (Throwable t) {
-      LOG.warn("Unknown throwable caught while updating delta metrics. Metrics will not be updated.", t);
-    }
+    LOG.info("Started DeltaFilesMetricLogger thread");
   }
 
-  /**
-   * Add partition and delta count to deltaFilesStats if the delta count is over the recording threshold and it is in
-   * the top {@link HiveConf.ConfVars#HIVE_TXN_ACID_METRICS_MAX_CACHE_SIZE} deltas.
-   */
-  private static void filterAndAddToDeltaFilesStats(DeltaFilesMetricType type, int deltaCount, int deltasThreshold,
-      EnumMap<DeltaFilesMetricType, Queue<Pair<String, Integer>>> deltaFilesStats, DeltaFilesMetadata metadata,
-      int maxCacheSize) {
-    if (deltaCount > deltasThreshold) {
-      Queue<Pair<String,Integer>> pairQueue = deltaFilesStats.get(type);
-      if (pairQueue != null && pairQueue.size() == maxCacheSize) {
-        Pair<String, Integer> lowest = pairQueue.peek();
-        if (lowest != null && deltaCount > lowest.getValue()) {
-          pairQueue.poll();
-        }
-      }
-      if (pairQueue == null || pairQueue.size() < maxCacheSize) {
-        String deltaCountKey = getDeltaCountKey(metadata.dbName, metadata.tableName, metadata.partitionName);
-        deltaFilesStats.computeIfAbsent(type,
-            v -> (new PriorityQueue<>(maxCacheSize, getComparator()))).add(Pair.of(deltaCountKey, deltaCount));
-      }
-    }
-  }
 
   private static String getDeltaCountKey(String dbName, String tableName, String partitionName) {
     StringBuilder key = new StringBuilder();
@@ -310,109 +143,6 @@ public class DeltaFilesMetricReporter {
     return key.toString();
   }
 
-  private static void logDeltaDirMetrics(AcidDirectory dir, Configuration conf, int numObsoleteDeltas, int numDeltas,
-      int numSmallDeltas) {
-    long loggerFrequency = HiveConf
-        .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ACID_METRICS_LOGGER_FREQUENCY, TimeUnit.MILLISECONDS);
-    if (loggerFrequency <= 0) {
-      return;
-    }
-    long currentTime = System.currentTimeMillis();
-    if (lastSuccessfulLoggingTime == 0 || currentTime >= lastSuccessfulLoggingTime + loggerFrequency) {
-      lastSuccessfulLoggingTime = currentTime;
-      if (numDeltas >= HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ACTIVE_DELTA_DIR_THRESHOLD)) {
-        LOG.warn("Directory " + dir.getPath() + " contains " + numDeltas + " active delta directories. This can " +
-            "cause performance degradation.");
-      }
-
-      if (numObsoleteDeltas >=
-          HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_OBSOLETE_DELTA_DIR_THRESHOLD)) {
-        LOG.warn("Directory " + dir.getPath() + " contains " + numDeltas + " obsolete delta directories. This can " +
-            "indicate compaction cleaner issues.");
-      }
-
-      if (numSmallDeltas >= HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_SMALL_DELTA_DIR_THRESHOLD)) {
-        LOG.warn("Directory " + dir.getPath() + " contains " + numDeltas + " small delta directories. This can " +
-            "indicate performance degradation and there might be a problem with your streaming setup.");
-      }
-    }
-  }
-
-  private static int getNumObsoleteDeltas(AcidDirectory dir, long checkThresholdInSec) throws IOException {
-    int numObsoleteDeltas = 0;
-    for (Path obsolete : dir.getObsolete()) {
-      FileStatus stat = dir.getFs().getFileStatus(obsolete);
-      if (System.currentTimeMillis() - stat.getModificationTime() >= checkThresholdInSec * 1000) {
-        numObsoleteDeltas++;
-      }
-    }
-    return numObsoleteDeltas;
-  }
-
-  public static void createCountersForAcidMetrics(TezCounters tezCounters, JobConf jobConf) {
-    if (HiveConf.getBoolVar(jobConf, HiveConf.ConfVars.HIVE_SERVER2_METRICS_ENABLED) &&
-      MetastoreConf.getBoolVar(jobConf, MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON)) {
-
-      Arrays.stream(DeltaFilesMetricType.values())
-        .filter(type -> jobConf.get(type.name()) != null)
-        .forEach(type ->
-            Splitter.on(ENTRY_SEPARATOR).withKeyValueSeparator(KEY_VALUE_SEPARATOR).split(jobConf.get(type.name())).forEach(
-              (path, cnt) -> tezCounters.findCounter(type.value, path).setValue(Long.parseLong(cnt))
-            )
-        );
-    }
-  }
-
-  public static void addAcidMetricsToConfObj(EnumMap<DeltaFilesMetricType,
-      Queue<Pair<String, Integer>>> deltaFilesStats, Configuration conf) {
-    try {
-      deltaFilesStats.forEach((type, value) -> conf
-          .set(type.name(), Joiner.on(ENTRY_SEPARATOR).withKeyValueSeparator(KEY_VALUE_SEPARATOR).join(value)));
-
-    } catch (Exception e) {
-      LOG.warn("Couldn't add Delta metrics to conf object", e);
-    }
-  }
-
-  public static void backPropagateAcidMetrics(JobConf jobConf, Configuration conf) {
-    if (HiveConf.getBoolVar(jobConf, HiveConf.ConfVars.HIVE_SERVER2_METRICS_ENABLED) &&
-      MetastoreConf.getBoolVar(jobConf, MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON)) {
-      try {
-        Arrays.stream(DeltaFilesMetricType.values()).filter(type -> conf.get(type.name()) != null)
-            .forEach(type -> jobConf.set(type.name(), conf.get(type.name())));
-      } catch (Exception e) {
-        LOG.warn("Couldn't back propagate Delta metrics to jobConf object", e);
-      }
-    }
-  }
-
-  private static long getBaseSize(AcidDirectory dir) throws IOException {
-    long baseSize = 0;
-    if (dir.getBase() != null) {
-      baseSize = getDirSize(dir.getBase(), dir.getFs());
-    } else {
-      for (HadoopShims.HdfsFileStatusWithId origStat : dir.getOriginalFiles()) {
-        baseSize += origStat.getFileStatus().getLen();
-      }
-    }
-    return baseSize;
-  }
-
-  private static long getModificationTime(AcidUtils.ParsedDirectory dir, FileSystem fs) throws IOException {
-    return dir.getFiles(fs, Ref.from(false)).stream()
-      .map(HadoopShims.HdfsFileStatusWithId::getFileStatus)
-      .mapToLong(FileStatus::getModificationTime)
-      .max()
-      .orElse(new Date().getTime());
-  }
-
-  private static long getDirSize(AcidUtils.ParsedDirectory dir, FileSystem fs) throws IOException {
-    return dir.getFiles(fs, Ref.from(false)).stream()
-      .map(HadoopShims.HdfsFileStatusWithId::getFileStatus)
-      .mapToLong(FileStatus::getLen)
-      .sum();
-  }
-
   private void initObjectsForMetrics() throws Exception {
     MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
 
@@ -435,55 +165,71 @@ public class DeltaFilesMetricReporter {
         .getObjectName());
   }
 
-  private void initCachesForMetrics(HiveConf conf) {
-    int maxCacheSize = HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVE_TXN_ACID_METRICS_MAX_CACHE_SIZE);
-    long duration = HiveConf.getTimeVar(conf,
-        HiveConf.ConfVars.HIVE_TXN_ACID_METRICS_CACHE_DURATION, TimeUnit.SECONDS);
-
-    deltaTopN = new PriorityBlockingQueue<>(maxCacheSize, getComparator());
-    smallDeltaTopN = new PriorityBlockingQueue<>(maxCacheSize, getComparator());
-    obsoleteDeltaTopN = new PriorityBlockingQueue<>(maxCacheSize, getComparator());
-
-    deltaCache = CacheBuilder.newBuilder()
-      .expireAfterWrite(duration, TimeUnit.SECONDS)
-      .removalListener(notification -> removalPredicate(deltaTopN, notification))
-      .softValues()
-      .build();
-
-    smallDeltaCache = CacheBuilder.newBuilder()
-      .expireAfterWrite(duration, TimeUnit.SECONDS)
-      .removalListener(notification -> removalPredicate(smallDeltaTopN, notification))
-      .softValues()
-      .build();
-
-    obsoleteDeltaCache = CacheBuilder.newBuilder()
-      .expireAfterWrite(duration, TimeUnit.SECONDS)
-      .removalListener(notification -> removalPredicate(obsoleteDeltaTopN, notification))
-      .softValues()
-      .build();
-  }
-
-  private static Comparator<Pair<String, Integer>> getComparator() {
-    return Comparator.comparing(Pair::getValue);
-  }
+  private final class ReportingTask implements Runnable {
 
-  private void removalPredicate(BlockingQueue<Pair<String, Integer>> topN, RemovalNotification notification) {
-    topN.removeIf(item -> item.getKey().equals(notification.getKey()));
-  }
+    private final TxnStore txnHandler;
 
-  private final class ReportingTask implements Runnable {
+    private ReportingTask(TxnStore txnHandler) {
+      this.txnHandler = txnHandler;
+    }
     @Override
     public void run() {
       Metrics metrics = MetricsFactory.getInstance();
       if (metrics != null) {
-        obsoleteDeltaCache.cleanUp();
-        obsoleteDeltaObject.updateAll(obsoleteDeltaCache.asMap());
+        try {
+          LOG.debug("Called reporting task.");
+          List<CompactionMetricsData> deltas = txnHandler.getTopCompactionMetricsDataPerType(maxCacheSize);
+          Map<String, Integer> deltasMap = deltas.stream()
+              .filter(d -> d.getMetricType() == NUM_DELTAS).collect(
+              Collectors.toMap(item -> getDeltaCountKey(item.getDbName(), item.getTblName(), item.getPartitionName()),
+                  CompactionMetricsData::getMetricValue));
+          deltaObject.updateAll(deltasMap);
+
+          Map<String, Integer> smallDeltasMap = deltas.stream()
+              .filter(d -> d.getMetricType() == NUM_SMALL_DELTAS).collect(
+              Collectors.toMap(item -> getDeltaCountKey(item.getDbName(), item.getTblName(), item.getPartitionName()),
+                  CompactionMetricsData::getMetricValue));
+          smallDeltaObject.updateAll(smallDeltasMap);
+
+          Map<String, Integer> obsoleteDeltasMap = deltas.stream()
+              .filter(d -> d.getMetricType() == CompactionMetricsData.MetricType.NUM_OBSOLETE_DELTAS).collect(
+              Collectors.toMap(item -> getDeltaCountKey(item.getDbName(), item.getTblName(), item.getPartitionName()),
+                  CompactionMetricsData::getMetricValue));
+          obsoleteDeltaObject.updateAll(obsoleteDeltasMap);
+        } catch (Throwable e) {
+          LOG.warn("Caught exception while trying to fetch compaction metrics from metastore backend db.", e);
+        }
+      }
+    }
+  }
+
+  private final class LoggerTask implements Runnable {
 
-        deltaCache.cleanUp();
-        deltaObject.updateAll(deltaCache.asMap());
+    private final TxnStore txnHandler;
 
-        smallDeltaCache.cleanUp();
-        smallDeltaObject.updateAll(smallDeltaCache.asMap());
+    private LoggerTask(TxnStore txnHandler) {
+      this.txnHandler = txnHandler;
+    }
+    @Override
+    public void run() {
+      try {
+        List<CompactionMetricsData> deltas =
+            txnHandler.getTopCompactionMetricsDataPerType(maxCacheSize);
+        deltas.stream().filter(d -> d.getMetricType() == NUM_DELTAS).forEach(d ->
+          LOG.warn(String.format("Directory %s contains %d active delta directories. " +
+              "This can cause performance degradation.",
+              getDeltaCountKey(d.getDbName(), d.getTblName(), d.getPartitionName()), d.getMetricValue())));
+        deltas.stream().filter(d -> d.getMetricType() == NUM_SMALL_DELTAS).forEach(d ->
+            LOG.warn(String.format("Directory %s contains %d small delta directories. " +
+                "This can indicate performance degradation and there might be a problem with your streaming setup.",
+                getDeltaCountKey(d.getDbName(), d.getTblName(), d.getPartitionName()), d.getMetricValue())));
+        deltas.stream().filter(d -> d.getMetricType() == CompactionMetricsData.MetricType.NUM_OBSOLETE_DELTAS)
+            .forEach(d ->
+            LOG.warn(String.format("Directory %s contains %d obsolete delta directories. " +
+                "This can indicate compaction cleaner issues.",
+                getDeltaCountKey(d.getDbName(), d.getTblName(), d.getPartitionName()), d.getMetricValue())));
+      } catch (MetaException e) {
+        LOG.warn("Caught exception while trying to log delta metrics data.", e);
       }
     }
   }
@@ -492,12 +238,17 @@ public class DeltaFilesMetricReporter {
   public static void close() {
     if (getInstance() != null) {
       getInstance().shutdown();
+      initialized = false;
     }
   }
 
   private void shutdown() {
-    if (executorService != null) {
-      executorService.shutdownNow();
+    if (reporterExecutorService != null) {
+      reporterExecutorService.shutdownNow();
+    }
+
+    if (loggerExecutorService != null) {
+      loggerExecutorService.shutdownNow();
     }
 
     MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
@@ -512,7 +263,181 @@ public class DeltaFilesMetricReporter {
     }
   }
 
-  public static class DeltaFilesMetadata implements Serializable {
-    public String dbName, tableName, partitionName;
+  public static void updateMetricsFromInitiator(AcidDirectory dir, String dbName, String tableName, String partitionName,
+      Configuration conf, TxnStore txnHandler, long baseSize, Map<Path, Long> deltaSizes) {
+    LOG.debug("Updating delta file metrics from initiator");
+    double deltaPctThreshold = MetastoreConf.getDoubleVar(conf, MetastoreConf.ConfVars.METASTORE_DELTAMETRICS_DELTA_PCT_THRESHOLD);
+    int deltasThreshold = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.METASTORE_DELTAMETRICS_DELTA_NUM_THRESHOLD);
+    int obsoleteDeltasThreshold = MetastoreConf.getIntVar(conf,
+        MetastoreConf.ConfVars.METASTORE_DELTAMETRICS_OBSOLETE_DELTA_NUM_THRESHOLD);
+    try {
+      // We have an AcidDir from the initiator, therefore we can use that to calculate active,small, obsolete delta
+      // count
+
+      int numDeltas = dir.getCurrentDirectories().size();
+      int numSmallDeltas = 0;
+
+      for (AcidUtils.ParsedDelta delta : dir.getCurrentDirectories()) {
+        if (deltaSizes.containsKey(delta.getPath())) {
+          long deltaSize = deltaSizes.get(delta.getPath());
+          if (baseSize != 0 && deltaSize / (float) baseSize < deltaPctThreshold) {
+            numSmallDeltas++;
+          }
+        }
+      }
+
+      int numObsoleteDeltas = filterOutBaseAndOriginalFiles(dir.getObsolete()).size();
+
+      if (numDeltas >= deltasThreshold) {
+        updateMetrics(dbName, tableName, partitionName, NUM_DELTAS, numDeltas, txnHandler);
+      } else {
+        removeMetrics(dbName, tableName, partitionName, NUM_DELTAS, txnHandler);
+      }
+
+      if (numSmallDeltas >= deltasThreshold) {
+        updateMetrics(dbName, tableName, partitionName, NUM_SMALL_DELTAS, numSmallDeltas, txnHandler);
+      } else {
+        removeMetrics(dbName, tableName, partitionName, NUM_SMALL_DELTAS, txnHandler);
+      }
+
+      if (numObsoleteDeltas >= obsoleteDeltasThreshold) {
+        updateMetrics(dbName, tableName, partitionName, CompactionMetricsData.MetricType.NUM_OBSOLETE_DELTAS, numObsoleteDeltas, txnHandler);
+      } else {
+        removeMetrics(dbName, tableName, partitionName, CompactionMetricsData.MetricType.NUM_OBSOLETE_DELTAS, txnHandler);
+      }
+
+      LOG.debug("Finished updating delta file metrics from initiator.\n deltaPctThreshold = {}, deltasThreshold = {}, "
+          + "obsoleteDeltasThreshold = {}, numDeltas = {}, numSmallDeltas = {},  numObsoleteDeltas = {}",
+          deltaPctThreshold, deltasThreshold, obsoleteDeltasThreshold, numDeltas, numSmallDeltas, numObsoleteDeltas);
+
+    } catch (Throwable t) {
+      LOG.warn("Unknown throwable caught while updating delta metrics. Metrics will not be updated.", t);
+    }
+  }
+
+  public static void updateMetricsFromWorker(AcidDirectory directory, String dbName, String tableName, String partitionName,
+      CompactionType type, Configuration conf, IMetaStoreClient client) {
+    LOG.debug("Updating delta file metrics from worker");
+    int deltasThreshold = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.METASTORE_DELTAMETRICS_DELTA_NUM_THRESHOLD);
+    int obsoleteDeltasThreshold = MetastoreConf.getIntVar(conf,
+        MetastoreConf.ConfVars.METASTORE_DELTAMETRICS_OBSOLETE_DELTA_NUM_THRESHOLD);
+    try {
+      // we have an instance of the AcidDirectory before the compaction worker was started
+      // from this we can get how many delta directories existed
+      // the previously active delta directories are now moved to obsolete
+      int numObsoleteDeltas = directory.getCurrentDirectories().size();
+      if (numObsoleteDeltas >= obsoleteDeltasThreshold) {
+        updateMetrics(dbName, tableName, partitionName, CompactionMetricsMetricType.NUM_OBSOLETE_DELTAS,
+            numObsoleteDeltas, client);
+      } else {
+        removeMetrics(dbName, tableName, partitionName, CompactionMetricsMetricType.NUM_OBSOLETE_DELTAS, client);
+      }
+
+      // We don't know the size of the newly create delta directories, that would require a fresh AcidDirectory
+      // Clear the small delta num counter from the cache for this key
+      client.removeCompactionMetricsData(new CompactionMetricsDataRequest(dbName, tableName, partitionName,
+          CompactionMetricsMetricType.NUM_SMALL_DELTAS));
+
+      // The new number of active delta dirs are either 0, 1 or 2.
+      // If we ran MAJOR compaction, no new delta is created, just base dir
+      // If we ran MINOR compaction, we can have 1 or 2 new delta dirs, depending on whether we had deltas or
+      // delete deltas.
+      if (type == CompactionType.MAJOR) {
+        client.removeCompactionMetricsData(new CompactionMetricsDataRequest(dbName, tableName, partitionName,
+            CompactionMetricsMetricType.NUM_DELTAS));
+      } else {
+        int numNewDeltas = 0;
+        // check whether we had deltas
+        if (directory.getDeleteDeltas().size() > 0) {
+          numNewDeltas++;
+        }
+
+        // if the size of the current dirs is bigger than the size of delete deltas, it means we have active deltas
+        if (directory.getCurrentDirectories().size() > directory.getDeleteDeltas().size()) {
+          numNewDeltas++;
+        }
+
+        // recalculate the delta count
+        if (numNewDeltas >= deltasThreshold) {
+          updateMetrics(dbName, tableName, partitionName, CompactionMetricsMetricType.NUM_DELTAS, numNewDeltas, client);
+        } else {
+          removeMetrics(dbName, tableName, partitionName, CompactionMetricsMetricType.NUM_DELTAS, client);
+        }
+      }
+
+      LOG.debug("Finished updating delta file metrics from worker.\n deltasThreshold = {}, "
+              + "obsoleteDeltasThreshold = {}, numObsoleteDeltas = {}",
+          deltasThreshold, obsoleteDeltasThreshold, numObsoleteDeltas);
+
+    } catch (Throwable t) {
+      LOG.warn("Unknown throwable caught while updating delta metrics. Metrics will not be updated.", t);
+    }
+  }
+
+  public static void updateMetricsFromCleaner(String dbName, String tableName, String partitionName,
+      List<Path> deletedFiles, Configuration conf, TxnStore txnHandler) {
+    LOG.debug("Updating delta file metrics from cleaner");
+    int obsoleteDeltasThreshold = MetastoreConf.getIntVar(conf,
+        MetastoreConf.ConfVars.METASTORE_DELTAMETRICS_DELTA_NUM_THRESHOLD);
+    try {
+      CompactionMetricsData prevObsoleteDelta =
+          txnHandler.getCompactionMetricsData(dbName, tableName, partitionName,
+              CompactionMetricsData.MetricType.NUM_OBSOLETE_DELTAS);
+      int numObsoleteDeltas = 0;
+      if (prevObsoleteDelta != null) {
+        numObsoleteDeltas = prevObsoleteDelta.getMetricValue() - filterOutBaseAndOriginalFiles(deletedFiles).size();
+        if (numObsoleteDeltas >= obsoleteDeltasThreshold) {
+          updateMetrics(dbName, tableName, partitionName, CompactionMetricsData.MetricType.NUM_OBSOLETE_DELTAS,
+              numObsoleteDeltas, txnHandler);
+        } else {
+          txnHandler.removeCompactionMetricsData(dbName, tableName, partitionName,
+              CompactionMetricsData.MetricType.NUM_OBSOLETE_DELTAS);
+        }
+      }
+
+      LOG.debug("Finished updating delta file metrics from cleaner.\n obsoleteDeltasThreshold = {}, "
+              + "numObsoleteDeltas = {}", obsoleteDeltasThreshold, numObsoleteDeltas);
+
+    } catch (Throwable t) {
+      LOG.warn("Unknown throwable caught while updating delta metrics. Metrics will not be updated.", t);
+    }
+  }
+
+  private static void updateMetrics(String dbName, String tblName, String partitionName,
+      CompactionMetricsData.MetricType type, int numDeltas, TxnStore txnHandler) throws MetaException {
+    CompactionMetricsData data = new CompactionMetricsData.Builder()
+        .dbName(dbName).tblName(tblName).partitionName(partitionName).metricType(type).metricValue(numDeltas).build();
+    if (!txnHandler.updateCompactionMetricsData(data)) {
+      LOG.warn("Compaction metric data cannot be updated because of version mismatch.");
+    }
+  }
+
+  private static void updateMetrics(String dbName, String tblName, String partitionName,
+      CompactionMetricsMetricType type, int numDeltas, IMetaStoreClient client) throws TException {
+    CompactionMetricsDataStruct struct = new CompactionMetricsDataStruct();
+    struct.setDbname(dbName);
+    struct.setTblname(tblName);
+    struct.setPartitionname(partitionName);
+    struct.setType(type);
+    struct.setMetricvalue(numDeltas);
+    if (!client.updateCompactionMetricsData(struct)) {
+      LOG.warn("Compaction metric data cannot be updated because of version mismatch.");
+    }
+  }
+
+  private static void removeMetrics(String dbName, String tblName, String partitionName,
+      CompactionMetricsData.MetricType type, TxnStore txnHandler) throws MetaException {
+    txnHandler.removeCompactionMetricsData(dbName, tblName, partitionName, type);
+  }
+
+  private static void removeMetrics(String dbName, String tblName, String partitionName,
+      CompactionMetricsMetricType type, IMetaStoreClient client) throws TException {
+    CompactionMetricsDataRequest request = new CompactionMetricsDataRequest(dbName, tblName, partitionName, type);
+    client.removeCompactionMetricsData(request);
+  }
+
+  private static List<Path> filterOutBaseAndOriginalFiles(List<Path> paths) {
+    return paths.stream().filter(p -> p.getName().startsWith(AcidConstants.DELTA_PREFIX) || p.getName()
+        .startsWith(AcidConstants.DELETE_DELTA_PREFIX)).collect(Collectors.toList());
   }
 }
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCompactionMetrics.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCompactionMetrics.java
index eda09c1..90eff22 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCompactionMetrics.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCompactionMetrics.java
@@ -81,6 +81,7 @@ public class TestCompactionMetrics  extends CompactorTest {
   public void setUp() throws Exception {
     MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.METRICS_ENABLED, true);
     MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.TXN_USE_MIN_HISTORY_LEVEL, true);
+    MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.COMPACTOR_INITIATOR_ON, true);
     // re-initialize metrics
     Metrics.shutdown();
     Metrics.initialize(conf);
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestDeltaFilesMetrics.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestDeltaFilesMetrics.java
index e321f33..08604f6 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestDeltaFilesMetrics.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestDeltaFilesMetrics.java
@@ -17,19 +17,26 @@
  */
 package org.apache.hadoop.hive.ql.txn.compactor;
 
-import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Maps;
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.metrics.common.MetricsFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.CommitTxnRequest;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
+import org.apache.hadoop.hive.metastore.api.LockComponent;
+import org.apache.hadoop.hive.metastore.api.LockLevel;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.ShowCompactRequest;
+import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
+import org.apache.hadoop.hive.metastore.api.ShowCompactResponseElement;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
-import org.apache.hadoop.hive.ql.hooks.ReadEntity;
-import org.apache.hadoop.hive.ql.io.AcidDirectory;
 import org.apache.hadoop.hive.ql.txn.compactor.metrics.DeltaFilesMetricReporter;
-import org.apache.tez.common.counters.TezCounters;
-import org.jetbrains.annotations.NotNull;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Test;
@@ -39,199 +46,396 @@ import javax.management.MBeanInfo;
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
 import java.lang.management.ManagementFactory;
-import java.util.EnumMap;
+import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
-import java.util.Queue;
 import java.util.concurrent.TimeUnit;
 
-import static org.apache.hadoop.hive.ql.txn.compactor.metrics.DeltaFilesMetricReporter.DeltaFilesMetricType.NUM_DELTAS;
-import static org.apache.hadoop.hive.ql.txn.compactor.metrics.DeltaFilesMetricReporter.DeltaFilesMetricType.NUM_OBSOLETE_DELTAS;
-import static org.apache.hadoop.hive.ql.txn.compactor.metrics.DeltaFilesMetricReporter.DeltaFilesMetricType.NUM_SMALL_DELTAS;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
 public class TestDeltaFilesMetrics extends CompactorTest  {
 
   private void setUpHiveConf() {
-    HiveConf.setBoolVar(conf, HiveConf.ConfVars.HIVE_SERVER2_METRICS_ENABLED, true);
-    HiveConf.setIntVar(conf, HiveConf.ConfVars.HIVE_TXN_ACID_METRICS_MAX_CACHE_SIZE, 2);
-    HiveConf.setTimeVar(conf, HiveConf.ConfVars.HIVE_TXN_ACID_METRICS_CACHE_DURATION, 7200, TimeUnit.SECONDS);
-    HiveConf.setIntVar(conf, HiveConf.ConfVars.HIVE_TXN_ACID_METRICS_OBSOLETE_DELTA_NUM_THRESHOLD, 100);
-    HiveConf.setIntVar(conf, HiveConf.ConfVars.HIVE_TXN_ACID_METRICS_DELTA_NUM_THRESHOLD, 100);
-    HiveConf.setTimeVar(conf, HiveConf.ConfVars.HIVE_TXN_ACID_METRICS_REPORTING_INTERVAL, 1, TimeUnit.SECONDS);
+    MetastoreConf.setLongVar(conf, MetastoreConf.ConfVars.METASTORE_DELTAMETRICS_DELTA_NUM_THRESHOLD, 1);
+    MetastoreConf.setLongVar(conf, MetastoreConf.ConfVars.METASTORE_DELTAMETRICS_OBSOLETE_DELTA_NUM_THRESHOLD, 1);
+    MetastoreConf.setTimeVar(conf, MetastoreConf.ConfVars.METASTORE_DELTAMETRICS_REPORTING_INTERVAL, 1,
+        TimeUnit.SECONDS);
+    MetastoreConf.setDoubleVar(conf, MetastoreConf.ConfVars.METASTORE_DELTAMETRICS_DELTA_PCT_THRESHOLD, 0.15f);
+    MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.METRICS_ENABLED, true);
+    MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON, true);
+    MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.COMPACTOR_INITIATOR_ON, true);
+    HiveConf.setBoolVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_GATHER_STATS, false);
   }
 
-  private void initAndCollectFirstMetrics() throws Exception {
+  @After
+  public void tearDown() throws Exception {
     MetricsFactory.close();
-    MetricsFactory.init(conf);
+    DeltaFilesMetricReporter.close();
+  }
 
-    DeltaFilesMetricReporter.init(conf);
 
-    TezCounters tezCounters = new TezCounters();
-    tezCounters.findCounter(NUM_OBSOLETE_DELTAS + "", "default.acid/p=1").setValue(200);
-    tezCounters.findCounter(NUM_OBSOLETE_DELTAS + "", "default.acid/p=2").setValue(100);
-    tezCounters.findCounter(NUM_OBSOLETE_DELTAS + "", "default.acid/p=3").setValue(150);
-    tezCounters.findCounter(NUM_OBSOLETE_DELTAS + "", "default.acid_v2").setValue(250);
+  static void verifyMetricsMatch(Map<String, String> expected, Map<String, String> actual) {
+    Assert.assertTrue("Actual metrics " + actual + " don't match expected: " + expected,
+        equivalent(expected, actual));
+  }
 
-    tezCounters.findCounter(NUM_DELTAS + "", "default.acid/p=1").setValue(150);
-    tezCounters.findCounter(NUM_DELTAS + "", "default.acid/p=2").setValue(100);
-    tezCounters.findCounter(NUM_DELTAS + "", "default.acid/p=3").setValue(250);
-    tezCounters.findCounter(NUM_DELTAS + "", "default.acid_v2").setValue(200);
+  private static boolean equivalent(Map<String, String> lhs, Map<String, String> rhs) {
+    return lhs.size() == rhs.size() && Maps.difference(lhs, rhs).areEqual();
+  }
 
-    tezCounters.findCounter(NUM_SMALL_DELTAS + "", "default.acid/p=1").setValue(250);
-    tezCounters.findCounter(NUM_SMALL_DELTAS + "", "default.acid/p=2").setValue(200);
-    tezCounters.findCounter(NUM_SMALL_DELTAS + "", "default.acid/p=3").setValue(150);
-    tezCounters.findCounter(NUM_SMALL_DELTAS + "", "default.acid_v2").setValue(100);
+  static Map<String, String> gaugeToMap(String metric) throws Exception {
+    MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+    ObjectName oname = new ObjectName(DeltaFilesMetricReporter.OBJECT_NAME_PREFIX + metric);
+    MBeanInfo mbeanInfo = mbs.getMBeanInfo(oname);
 
-    DeltaFilesMetricReporter.getInstance().submit(tezCounters, null);
-    Thread.sleep(1000);
+    Map<String, String> result = new HashMap<>();
+    for (MBeanAttributeInfo attr : mbeanInfo.getAttributes()) {
+      result.put(attr.getName(), String.valueOf(mbs.getAttribute(oname, attr.getName())));
+    }
+    return result;
   }
 
-  @After
-  public void tearDown() {
-    DeltaFilesMetricReporter.close();
+  @Override
+  boolean useHive130DeltaDirName() {
+    return false;
   }
 
   @Test
-  public void testDeltaFilesMetric() throws Exception {
+  public void testDeltaFileMetricPartitionedTable() throws Exception {
     setUpHiveConf();
-    initAndCollectFirstMetrics();
+    String dbName = "default";
+    String tblName = "dp";
+    String partName = "ds=part1";
+
+    Table t = newTable(dbName, tblName, true);
+    List<LockComponent> components = new ArrayList<>();
 
+    Partition p = newPartition(t, "part1");
+    addBaseFile(t, p, 20L, 20);
+    addDeltaFile(t, p, 21L, 22L, 2);
+    addDeltaFile(t, p, 23L, 24L, 20);
+
+    components.add(createLockComponent(dbName, tblName, partName));
+
+    burnThroughTransactions(dbName, tblName, 23);
+    long txnid = openTxn();
+
+    LockRequest req = new LockRequest(components, "me", "localhost");
+    req.setTxnid(txnid);
+    LockResponse res = txnHandler.lock(req);
+    Assert.assertEquals(LockState.ACQUIRED, res.getState());
+
+    long writeid = allocateWriteId(dbName, tblName, txnid);
+    Assert.assertEquals(24, writeid);
+    txnHandler.commitTxn(new CommitTxnRequest(txnid));
+
+    startInitiator();
+
+    TimeUnit.SECONDS.sleep(2);
+    // 2 active deltas
+    // 1 small delta
+    // 0 obsolete deltas
     verifyMetricsMatch(new HashMap<String, String>() {{
-      put("default.acid/p=1", "200");
-      put("default.acid/p=2", "100");
-      put("default.acid/p=3", "150");
-      put("default.acid_v2", "250");
-    }}, gaugeToMap(MetricsConstants.COMPACTION_NUM_OBSOLETE_DELTAS));
+      put(dbName + "." + tblName + Path.SEPARATOR + partName, "2");}}, gaugeToMap(MetricsConstants.COMPACTION_NUM_DELTAS));
+    verifyMetricsMatch(new HashMap<String, String>() {{
+      put(dbName + "." + tblName + Path.SEPARATOR + partName, "1");}}, gaugeToMap(MetricsConstants.COMPACTION_NUM_SMALL_DELTAS));
+    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_OBSOLETE_DELTAS).size());
+
+    startWorker();
 
+    TimeUnit.SECONDS.sleep(2);
+    // 0 active deltas
+    // 0 small delta
+    // 2 obsolete deltas
+    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_DELTAS).size());
+    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_SMALL_DELTAS).size());
     verifyMetricsMatch(new HashMap<String, String>() {{
-      put("default.acid/p=1", "150");
-      put("default.acid/p=2", "100");
-      put("default.acid/p=3", "250");
-      put("default.acid_v2", "200");
-    }}, gaugeToMap(MetricsConstants.COMPACTION_NUM_DELTAS));
+      put(dbName + "." + tblName + Path.SEPARATOR + partName, "2");}}, gaugeToMap(MetricsConstants.COMPACTION_NUM_OBSOLETE_DELTAS));
+
+
+    addDeltaFile(t, p, 25L, 26L, 2);
+    addDeltaFile(t, p, 27L, 28L, 20);
+    addDeltaFile(t, p, 29L, 30L, 2);
+
+    burnThroughTransactions(dbName, tblName, 30);
+    txnid = openTxn();
 
+    req = new LockRequest(components, "me", "localhost");
+    req.setTxnid(txnid);
+    res = txnHandler.lock(req);
+    Assert.assertEquals(LockState.ACQUIRED, res.getState());
+
+    writeid = allocateWriteId(dbName, tblName, txnid);
+    Assert.assertEquals(55, writeid);
+    txnHandler.commitTxn(new CommitTxnRequest(txnid));
+    // Change these params to initiate MINOR compaction
+    HiveConf.setFloatVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_PCT_THRESHOLD, 1.8f);
+    HiveConf.setIntVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_NUM_THRESHOLD, 2);
+    startInitiator();
+
+    TimeUnit.SECONDS.sleep(2);
+    // 3 active deltas
+    // 2 small deltas
+    // 2 obsolete deltas
     verifyMetricsMatch(new HashMap<String, String>() {{
-      put("default.acid/p=1", "250");
-      put("default.acid/p=2", "200");
-      put("default.acid/p=3", "150");
-      put("default.acid_v2", "100");
-    }}, gaugeToMap(MetricsConstants.COMPACTION_NUM_SMALL_DELTAS));
+      put(dbName + "." + tblName + Path.SEPARATOR + partName, "3");}}, gaugeToMap(MetricsConstants.COMPACTION_NUM_DELTAS));
+    verifyMetricsMatch(new HashMap<String, String>() {{
+      put(dbName + "." + tblName + Path.SEPARATOR + partName, "2");}}, gaugeToMap(MetricsConstants.COMPACTION_NUM_SMALL_DELTAS));
+    verifyMetricsMatch(new HashMap<String, String>() {{
+      put(dbName + "." + tblName + Path.SEPARATOR + partName, "2");}}, gaugeToMap(MetricsConstants.COMPACTION_NUM_OBSOLETE_DELTAS));
+
+    startCleaner();
+
+    TimeUnit.SECONDS.sleep(2);
+    // 3 active deltas
+    // 2 small deltas
+    // 0 obsolete delta
+    verifyMetricsMatch(new HashMap<String, String>() {{
+      put(dbName + "." + tblName + Path.SEPARATOR + partName, "3");}}, gaugeToMap(MetricsConstants.COMPACTION_NUM_DELTAS));
+    verifyMetricsMatch(new HashMap<String, String>() {{
+      put(dbName + "." + tblName + Path.SEPARATOR + partName, "2");}}, gaugeToMap(MetricsConstants.COMPACTION_NUM_SMALL_DELTAS));
+    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_OBSOLETE_DELTAS).size());
+
+    startWorker();
+
+    TimeUnit.SECONDS.sleep(2);
+    // 1 active delta
+    // 0 small delta
+    // 3 obsolete deltas
+    verifyMetricsMatch(new HashMap<String, String>() {{
+      put(dbName + "." + tblName + Path.SEPARATOR + partName, "1");}}, gaugeToMap(MetricsConstants.COMPACTION_NUM_DELTAS));
+    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_SMALL_DELTAS).size());
+    verifyMetricsMatch(new HashMap<String, String>() {{
+      put(dbName + "." + tblName + Path.SEPARATOR + partName, "3");}}, gaugeToMap(MetricsConstants.COMPACTION_NUM_OBSOLETE_DELTAS));
+
+    startCleaner();
+
+    TimeUnit.SECONDS.sleep(2);
+    verifyMetricsMatch(new HashMap<String, String>() {{
+      put(dbName + "." + tblName + Path.SEPARATOR + partName, "1");}}, gaugeToMap(MetricsConstants.COMPACTION_NUM_DELTAS));
+    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_SMALL_DELTAS).size());
+    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_OBSOLETE_DELTAS).size());
   }
 
   @Test
-  public void testDeltaFilesMetricUpdate() throws Exception {
+  public void testDeltaFileMetricMultiPartitionedTable() throws Exception {
     setUpHiveConf();
-    initAndCollectFirstMetrics();
-
-    TezCounters tezCounters = new TezCounters();
-    tezCounters.findCounter(NUM_OBSOLETE_DELTAS + "", "default.acid/p=1").setValue(50);
-    tezCounters.findCounter(NUM_DELTAS + "", "default.acid/p=1").setValue(50);
-    tezCounters.findCounter(NUM_DELTAS + "", "default.acid/p=3").setValue(0);
-    tezCounters.findCounter(NUM_SMALL_DELTAS + "", "default.acid/p=1").setValue(50);
-    tezCounters.findCounter(NUM_SMALL_DELTAS + "", "default.acid/p=2").setValue(0);
-    tezCounters.findCounter(NUM_SMALL_DELTAS + "", "default.acid/p=3").setValue(50);
+    String dbName = "default";
+    String tblName = "dp";
+    String part1Name = "ds=part1";
+    String part2Name = "ds=part2";
+    String part3Name = "ds=part3";
+    Table t = newTable(dbName, tblName, true);
+    List<LockComponent> components = new ArrayList<>();
+
+
+    Partition p1 = newPartition(t, "part1");
+    addDeltaFile(t, p1, 1L, 2L, 2);
+    addDeltaFile(t, p1, 3L, 4L, 4);
+
+    Partition p2 = newPartition(t, "part2");
+    addBaseFile(t, p2, 5L, 20);
+    addDeltaFile(t, p2, 6L, 7L, 2);
+    addDeltaFile(t, p2, 8L, 9L, 3);
+    addDeltaFile(t, p2, 10L, 11L, 1);
+
+    Partition p3 = newPartition(t, "part3");
+    addDeltaFile(t, p3, 12L, 13L, 3);
+    addDeltaFile(t, p3, 14L, 15L, 20);
+    addDeltaFile(t, p3, 16L, 17L, 50);
+    addDeltaFile(t, p3, 18L, 19L, 2);
+
+    components.add(createLockComponent(dbName, tblName, part1Name));
+    components.add(createLockComponent(dbName, tblName, part2Name));
+    components.add(createLockComponent(dbName, tblName, part3Name));
+
+    burnThroughTransactions(dbName, tblName, 19);
+    long txnId = openTxn();
+
+    LockRequest req = new LockRequest(components, "me", "localhost");
+    req.setTxnid(txnId);
+    LockResponse res = txnHandler.lock(req);
+    Assert.assertEquals(LockState.ACQUIRED, res.getState());
+
+    allocateWriteId(dbName, tblName, txnId);
+    txnHandler.commitTxn(new CommitTxnRequest(txnId));
+
+    HiveConf.setIntVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_NUM_THRESHOLD, 2);
+    HiveConf.setFloatVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_PCT_THRESHOLD, 0.4f);
+    startInitiator();
+
+    TimeUnit.SECONDS.sleep(2);
+    verifyMetricsMatch(new HashMap<String, String>() {{
+      put(dbName + "." + tblName + Path.SEPARATOR + part1Name, "2");
+      put(dbName + "." + tblName + Path.SEPARATOR + part2Name, "3");
+      put(dbName + "." + tblName + Path.SEPARATOR + part3Name, "4");
+    }}, gaugeToMap(MetricsConstants.COMPACTION_NUM_DELTAS));
+    verifyMetricsMatch(new HashMap<String, String>() {{
+      put(dbName + "." + tblName + Path.SEPARATOR + part2Name, "2");
+    }}, gaugeToMap(MetricsConstants.COMPACTION_NUM_SMALL_DELTAS));
+    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_OBSOLETE_DELTAS).size());
 
-    // the next pass will be from a query that touches only acid/p=1 and acid/p=3
-    ReadEntity p1 = getReadEntity("default@acid@p=1");
-    // don't update p2
-    ReadEntity p3 = getReadEntity("default@acid@p=3");
+    ShowCompactResponse showCompactResponse = txnHandler.showCompact(new ShowCompactRequest());
+    List<ShowCompactResponseElement> compacts = showCompactResponse.getCompacts();
+    Assert.assertEquals(2, compacts.size());
 
-    DeltaFilesMetricReporter.getInstance().submit(tezCounters, ImmutableSet.of(p1, p3));
-    Thread.sleep(1000);
+    // Need to run two worker sessions, to compact all resources in the compaction queue
+    startWorker();
+    startWorker();
 
+    TimeUnit.SECONDS.sleep(2);
+    verifyMetricsMatch(new HashMap<String, String>() {{
+      put(dbName + "." + tblName + Path.SEPARATOR + part1Name, "2");
+      put(dbName + "." + tblName + Path.SEPARATOR + part2Name, "1");
+    }}, gaugeToMap(MetricsConstants.COMPACTION_NUM_DELTAS));
+    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_SMALL_DELTAS).size());
     verifyMetricsMatch(new HashMap<String, String>() {{
-      put("default.acid/p=1", "50"); // updated
-      put("default.acid/p=2", "100");
-      // p=3 was removed since the query touched it and it didn't have enough deltas to be included in counters
-      put("default.acid_v2", "250");
+      put(dbName + "." + tblName + Path.SEPARATOR + part2Name, "3");
+      put(dbName + "." + tblName + Path.SEPARATOR + part3Name, "4");
     }}, gaugeToMap(MetricsConstants.COMPACTION_NUM_OBSOLETE_DELTAS));
 
-    verifyMetricsMatch(new HashMap<String, String>() {{
-      put("default.acid/p=1", "50"); // updated
-      put("default.acid/p=2", "100");
-      // p=3 was removed since the query touched it and it didn't have enough deltas (0) to be included in counters
-      put("default.acid_v2", "200");
-    }}, gaugeToMap(MetricsConstants.COMPACTION_NUM_DELTAS));
+    startCleaner();
+    startCleaner();
 
+    TimeUnit.SECONDS.sleep(2);
     verifyMetricsMatch(new HashMap<String, String>() {{
-      put("default.acid/p=1", "50");  // updated
-      put("default.acid/p=2", "200"); // not updated since the query didn't touch p=2
-      put("default.acid/p=3", "50");  // updated
-      put("default.acid_v2", "100");
-    }}, gaugeToMap(MetricsConstants.COMPACTION_NUM_SMALL_DELTAS));
+      put(dbName + "." + tblName + Path.SEPARATOR + part1Name, "2");
+      put(dbName + "." + tblName + Path.SEPARATOR + part2Name, "1");
+    }}, gaugeToMap(MetricsConstants.COMPACTION_NUM_DELTAS));
+    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_SMALL_DELTAS).size());
+    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_OBSOLETE_DELTAS).size());
   }
 
   @Test
-  public void testDeltaFilesMetricTimeout() throws Exception {
+  public void testDeltaFileMetricUnpartitionedTable() throws Exception {
     setUpHiveConf();
-    HiveConf.setTimeVar(conf, HiveConf.ConfVars.HIVE_TXN_ACID_METRICS_CACHE_DURATION, 5, TimeUnit.SECONDS);
-    initAndCollectFirstMetrics();
-    Thread.sleep(5000);
+    String dbName = "default";
+    String tblName = "dp";
+    Table t = newTable(dbName, tblName, false);
+    List<LockComponent> components = new ArrayList<>();
+
+    addBaseFile(t, null, 20L, 20);
+    addDeltaFile(t, null, 21L, 22L, 2);
+    addDeltaFile(t, null, 23L, 24L, 20);
+
+    components.add(createLockComponent(dbName, tblName, null));
+    burnThroughTransactions(dbName, tblName, 24);
+    long txnId = openTxn();
+
+    LockRequest req = new LockRequest(components, "me", "localhost");
+    req.setTxnid(txnId);
+    LockResponse res = txnHandler.lock(req);
+    Assert.assertEquals(LockState.ACQUIRED, res.getState());
+
+    long writeId = allocateWriteId(dbName, tblName, txnId);
+    Assert.assertEquals(25, writeId);
+    txnHandler.commitTxn(new CommitTxnRequest(txnId));
+
+    startInitiator();
+
+    TimeUnit.SECONDS.sleep(2);
+    // 2 active deltas
+    // 1 small delta
+    // 0 obsolete deltas
+    verifyMetricsMatch(new HashMap<String, String>() {{
+      put(dbName + "." + tblName, "2");}},  gaugeToMap(MetricsConstants.COMPACTION_NUM_DELTAS));
+    verifyMetricsMatch(new HashMap<String, String>() {{
+      put(dbName + "." + tblName, "1");}},  gaugeToMap(MetricsConstants.COMPACTION_NUM_SMALL_DELTAS));
+    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_OBSOLETE_DELTAS).size());
 
-    TezCounters tezCounters = new TezCounters();
-    tezCounters.findCounter(NUM_DELTAS + "", "default.acid/p=2").setValue(150);
-    DeltaFilesMetricReporter.getInstance().submit(tezCounters, null);
-    Thread.sleep(1000);
+    startWorker();
 
+    TimeUnit.SECONDS.sleep(2);
+    // 0 active delta
+    // 0 small delta
+    // 2 obsolete delta
+    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_DELTAS).size());
+    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_SMALL_DELTAS).size());
     verifyMetricsMatch(new HashMap<String, String>() {{
-      put("default.acid/p=2", "150");
-    }}, gaugeToMap(MetricsConstants.COMPACTION_NUM_DELTAS));
+      put(dbName + "." + tblName, "2");}}, gaugeToMap(MetricsConstants.COMPACTION_NUM_OBSOLETE_DELTAS));
+
+    startCleaner();
+
+    TimeUnit.SECONDS.sleep(2);
+    // 0 active delta
+    // 0 small delta
+    // 0 obsolete delta
+    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_DELTAS).size());
+    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_SMALL_DELTAS).size());
+    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_OBSOLETE_DELTAS).size());
   }
 
-  @Test
-  public void testMergeDeltaFilesStatsNullData() throws Exception {
+  @Test(expected = javax.management.InstanceNotFoundException.class)
+  public void testDeltaFilesMetricFromInitiatorWithMetricsDisabled() throws Exception {
     setUpHiveConf();
-    MetricsFactory.close();
-    MetricsFactory.init(conf);
-    DeltaFilesMetricReporter.init(conf);
+    conf.setBoolean(MetastoreConf.ConfVars.METRICS_ENABLED.getVarname(), false);
+    startInitiator();
+    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_DELTAS).size());
+  }
 
-    AcidDirectory dir = new AcidDirectory(new Path("/"), FileSystem.get(conf), null);
-    long checkThresholdInSec = HiveConf.getTimeVar(conf,
-        HiveConf.ConfVars.HIVE_TXN_ACID_METRICS_DELTA_CHECK_THRESHOLD, TimeUnit.SECONDS);
-    float deltaPctThreshold = HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVE_TXN_ACID_METRICS_DELTA_PCT_THRESHOLD);
-    int deltasThreshold = HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVE_TXN_ACID_METRICS_DELTA_NUM_THRESHOLD);
-    int obsoleteDeltasThreshold = HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVE_TXN_ACID_METRICS_OBSOLETE_DELTA_NUM_THRESHOLD);
-    int maxCacheSize = HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVE_TXN_ACID_METRICS_MAX_CACHE_SIZE);
-    EnumMap<DeltaFilesMetricReporter.DeltaFilesMetricType, Queue<Pair<String, Integer>>> deltaFilesStats =
-        new EnumMap<>(DeltaFilesMetricReporter.DeltaFilesMetricType.class);
+  @Test(expected = javax.management.InstanceNotFoundException.class)
+  public void testDeltaFilesMetricFromWorkerWithMetricsDisabled() throws Exception {
+    setUpHiveConf();
+    conf.setBoolean(MetastoreConf.ConfVars.METRICS_ENABLED.getVarname(), false);
+    startWorker();
+    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_SMALL_DELTAS).size());
+  }
 
-    //conf.get(JOB_CONF_DELTA_FILES_METRICS_METADATA) will not have a value assigned; this test checks for an NPE
-    DeltaFilesMetricReporter.mergeDeltaFilesStats(dir,checkThresholdInSec, deltaPctThreshold, deltasThreshold,
-        obsoleteDeltasThreshold, maxCacheSize, deltaFilesStats, conf);
+  @Test(expected = javax.management.InstanceNotFoundException.class)
+  public void testDeltaFilesMetricFromCleanerWithMetricsDisabled() throws Exception {
+    setUpHiveConf();
+    conf.setBoolean(MetastoreConf.ConfVars.METRICS_ENABLED.getVarname(), false);
+    startCleaner();
+    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_OBSOLETE_DELTAS).size());
   }
 
-  static void verifyMetricsMatch(Map<String, String> expected, Map<String, String> actual) {
-    Assert.assertTrue("Actual metrics " + actual + " don't match expected: " + expected,
-        equivalent(expected, actual));
+  @Test(expected = javax.management.InstanceNotFoundException.class)
+  public void testDeltaFilesMetricFromInitiatorWithExtMetricsDisabled() throws Exception {
+    setUpHiveConf();
+    conf.setBoolean(MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON.getVarname(), false);
+    startInitiator();
+    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_DELTAS).size());
   }
 
-  private static boolean equivalent(Map<String, String> lhs, Map<String, String> rhs) {
-    return lhs.size() == rhs.size() && Maps.difference(lhs, rhs).areEqual();
+  @Test(expected = javax.management.InstanceNotFoundException.class)
+  public void testDeltaFilesMetricFromWorkerWithExtMetricsDisabled() throws Exception {
+    setUpHiveConf();
+    conf.setBoolean(MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON.getVarname(), false);
+    startWorker();
+    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_SMALL_DELTAS).size());
   }
 
-  static Map<String, String> gaugeToMap(String metric) throws Exception {
-    MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
-    ObjectName oname = new ObjectName(DeltaFilesMetricReporter.OBJECT_NAME_PREFIX + metric);
-    MBeanInfo mbeanInfo = mbs.getMBeanInfo(oname);
+  @Test(expected = javax.management.InstanceNotFoundException.class)
+  public void testDeltaFilesMetricFromCleanerWithExtMetricsDisabled() throws Exception {
+    setUpHiveConf();
+    conf.setBoolean(MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON.getVarname(), false);
+    startCleaner();
+    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_OBSOLETE_DELTAS).size());
+  }
 
-    Map<String, String> result = new HashMap<>();
-    for (MBeanAttributeInfo attr : mbeanInfo.getAttributes()) {
-      result.put(attr.getName(), String.valueOf(mbs.getAttribute(oname, attr.getName())));
-    }
-    return result;
+  @Test(expected = javax.management.InstanceNotFoundException.class)
+  public void testDeltaFilesMetricFromInitiatorWithInitiatorOff() throws Exception {
+    setUpHiveConf();
+    conf.setBoolean(MetastoreConf.ConfVars.COMPACTOR_INITIATOR_ON.getVarname(), false);
+    startInitiator();
+    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_DELTAS).size());
   }
 
-  @NotNull
-  private ReadEntity getReadEntity(String s) {
-    ReadEntity p3 = mock(ReadEntity.class);
-    when(p3.getName()).thenReturn(s);
-    return p3;
+  @Test(expected = javax.management.InstanceNotFoundException.class)
+  public void testDeltaFilesMetricFromCleanerWithInitiatorOff() throws Exception {
+    setUpHiveConf();
+    conf.setBoolean(MetastoreConf.ConfVars.COMPACTOR_INITIATOR_ON.getVarname(), false);
+    startCleaner();
+    Assert.assertEquals(0, gaugeToMap(MetricsConstants.COMPACTION_NUM_OBSOLETE_DELTAS).size());
   }
 
-  @Override
-  boolean useHive130DeltaDirName() {
-    return false;
+  private LockComponent createLockComponent(String dbName, String tblName, String partName) {
+    LockComponent component = new LockComponent(LockType.SHARED_WRITE, LockLevel.PARTITION, dbName);
+    component.setTablename(tblName);
+    if (partName != null) {
+      component.setPartitionname(partName);
+    }
+    component.setOperationType(DataOperationType.UPDATE);
+    return component;
   }
+
 }
diff --git a/service/src/java/org/apache/hive/service/server/HiveServer2.java b/service/src/java/org/apache/hive/service/server/HiveServer2.java
index a642ae5..5ce6cf9 100644
--- a/service/src/java/org/apache/hive/service/server/HiveServer2.java
+++ b/service/src/java/org/apache/hive/service/server/HiveServer2.java
@@ -214,9 +214,6 @@ public class HiveServer2 extends CompositeService {
     try {
       if (hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_METRICS_ENABLED)) {
         MetricsFactory.init(hiveConf);
-        if (MetastoreConf.getBoolVar(hiveConf, MetastoreConf.ConfVars.METASTORE_ACIDMETRICS_EXT_ON)) {
-          DeltaFilesMetricReporter.init(hiveConf);
-        }
       }
     } catch (Throwable t) {
       LOG.warn("Could not initiate the HiveServer2 Metrics system.  Metrics may not be reported.", t);
@@ -937,7 +934,6 @@ public class HiveServer2 extends CompositeService {
         LOG.error("error in Metrics deinit: " + e.getClass().getName() + " "
           + e.getMessage(), e);
       }
-      DeltaFilesMetricReporter.close();
     }
     // Remove this server instance from ZooKeeper if dynamic service discovery is set
     if (serviceDiscovery && !activePassiveHA) {
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 d08962f..2c187b0 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
@@ -2561,14 +2561,14 @@ uint32_t ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1745;
-            ::apache::thrift::protocol::TType _etype1748;
-            xfer += iprot->readListBegin(_etype1748, _size1745);
-            this->success.resize(_size1745);
-            uint32_t _i1749;
-            for (_i1749 = 0; _i1749 < _size1745; ++_i1749)
+            uint32_t _size1753;
+            ::apache::thrift::protocol::TType _etype1756;
+            xfer += iprot->readListBegin(_etype1756, _size1753);
+            this->success.resize(_size1753);
+            uint32_t _i1757;
+            for (_i1757 = 0; _i1757 < _size1753; ++_i1757)
             {
-              xfer += iprot->readString(this->success[_i1749]);
+              xfer += iprot->readString(this->success[_i1757]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2607,10 +2607,10 @@ uint32_t ThriftHiveMetastore_get_databases_result::write(::apache::thrift::proto
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1750;
-      for (_iter1750 = this->success.begin(); _iter1750 != this->success.end(); ++_iter1750)
+      std::vector<std::string> ::const_iterator _iter1758;
+      for (_iter1758 = this->success.begin(); _iter1758 != this->success.end(); ++_iter1758)
       {
-        xfer += oprot->writeString((*_iter1750));
+        xfer += oprot->writeString((*_iter1758));
       }
       xfer += oprot->writeListEnd();
     }
@@ -2655,14 +2655,14 @@ uint32_t ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1751;
-            ::apache::thrift::protocol::TType _etype1754;
-            xfer += iprot->readListBegin(_etype1754, _size1751);
-            (*(this->success)).resize(_size1751);
-            uint32_t _i1755;
-            for (_i1755 = 0; _i1755 < _size1751; ++_i1755)
+            uint32_t _size1759;
+            ::apache::thrift::protocol::TType _etype1762;
+            xfer += iprot->readListBegin(_etype1762, _size1759);
+            (*(this->success)).resize(_size1759);
+            uint32_t _i1763;
+            for (_i1763 = 0; _i1763 < _size1759; ++_i1763)
             {
-              xfer += iprot->readString((*(this->success))[_i1755]);
+              xfer += iprot->readString((*(this->success))[_i1763]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2779,14 +2779,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1756;
-            ::apache::thrift::protocol::TType _etype1759;
-            xfer += iprot->readListBegin(_etype1759, _size1756);
-            this->success.resize(_size1756);
-            uint32_t _i1760;
-            for (_i1760 = 0; _i1760 < _size1756; ++_i1760)
+            uint32_t _size1764;
+            ::apache::thrift::protocol::TType _etype1767;
+            xfer += iprot->readListBegin(_etype1767, _size1764);
+            this->success.resize(_size1764);
+            uint32_t _i1768;
+            for (_i1768 = 0; _i1768 < _size1764; ++_i1768)
             {
-              xfer += iprot->readString(this->success[_i1760]);
+              xfer += iprot->readString(this->success[_i1768]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2825,10 +2825,10 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::write(::apache::thrift::p
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1761;
-      for (_iter1761 = this->success.begin(); _iter1761 != this->success.end(); ++_iter1761)
+      std::vector<std::string> ::const_iterator _iter1769;
+      for (_iter1769 = this->success.begin(); _iter1769 != this->success.end(); ++_iter1769)
       {
-        xfer += oprot->writeString((*_iter1761));
+        xfer += oprot->writeString((*_iter1769));
       }
       xfer += oprot->writeListEnd();
     }
@@ -2873,14 +2873,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1762;
-            ::apache::thrift::protocol::TType _etype1765;
-            xfer += iprot->readListBegin(_etype1765, _size1762);
-            (*(this->success)).resize(_size1762);
-            uint32_t _i1766;
-            for (_i1766 = 0; _i1766 < _size1762; ++_i1766)
+            uint32_t _size1770;
+            ::apache::thrift::protocol::TType _etype1773;
+            xfer += iprot->readListBegin(_etype1773, _size1770);
+            (*(this->success)).resize(_size1770);
+            uint32_t _i1774;
+            for (_i1774 = 0; _i1774 < _size1770; ++_i1774)
             {
-              xfer += iprot->readString((*(this->success))[_i1766]);
+              xfer += iprot->readString((*(this->success))[_i1774]);
             }
             xfer += iprot->readListEnd();
           }
@@ -3933,14 +3933,14 @@ uint32_t ThriftHiveMetastore_get_dataconnectors_result::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1767;
-            ::apache::thrift::protocol::TType _etype1770;
-            xfer += iprot->readListBegin(_etype1770, _size1767);
-            this->success.resize(_size1767);
-            uint32_t _i1771;
-            for (_i1771 = 0; _i1771 < _size1767; ++_i1771)
+            uint32_t _size1775;
+            ::apache::thrift::protocol::TType _etype1778;
+            xfer += iprot->readListBegin(_etype1778, _size1775);
+            this->success.resize(_size1775);
+            uint32_t _i1779;
+            for (_i1779 = 0; _i1779 < _size1775; ++_i1779)
             {
-              xfer += iprot->readString(this->success[_i1771]);
+              xfer += iprot->readString(this->success[_i1779]);
             }
             xfer += iprot->readListEnd();
           }
@@ -3979,10 +3979,10 @@ uint32_t ThriftHiveMetastore_get_dataconnectors_result::write(::apache::thrift::
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1772;
-      for (_iter1772 = this->success.begin(); _iter1772 != this->success.end(); ++_iter1772)
+      std::vector<std::string> ::const_iterator _iter1780;
+      for (_iter1780 = this->success.begin(); _iter1780 != this->success.end(); ++_iter1780)
       {
-        xfer += oprot->writeString((*_iter1772));
+        xfer += oprot->writeString((*_iter1780));
       }
       xfer += oprot->writeListEnd();
     }
@@ -4027,14 +4027,14 @@ uint32_t ThriftHiveMetastore_get_dataconnectors_presult::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1773;
-            ::apache::thrift::protocol::TType _etype1776;
-            xfer += iprot->readListBegin(_etype1776, _size1773);
-            (*(this->success)).resize(_size1773);
-            uint32_t _i1777;
-            for (_i1777 = 0; _i1777 < _size1773; ++_i1777)
+            uint32_t _size1781;
+            ::apache::thrift::protocol::TType _etype1784;
+            xfer += iprot->readListBegin(_etype1784, _size1781);
+            (*(this->success)).resize(_size1781);
+            uint32_t _i1785;
+            for (_i1785 = 0; _i1785 < _size1781; ++_i1785)
             {
-              xfer += iprot->readString((*(this->success))[_i1777]);
+              xfer += iprot->readString((*(this->success))[_i1785]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5096,17 +5096,17 @@ uint32_t ThriftHiveMetastore_get_type_all_result::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size1778;
-            ::apache::thrift::protocol::TType _ktype1779;
-            ::apache::thrift::protocol::TType _vtype1780;
-            xfer += iprot->readMapBegin(_ktype1779, _vtype1780, _size1778);
-            uint32_t _i1782;
-            for (_i1782 = 0; _i1782 < _size1778; ++_i1782)
+            uint32_t _size1786;
+            ::apache::thrift::protocol::TType _ktype1787;
+            ::apache::thrift::protocol::TType _vtype1788;
+            xfer += iprot->readMapBegin(_ktype1787, _vtype1788, _size1786);
+            uint32_t _i1790;
+            for (_i1790 = 0; _i1790 < _size1786; ++_i1790)
             {
-              std::string _key1783;
-              xfer += iprot->readString(_key1783);
-              Type& _val1784 = this->success[_key1783];
-              xfer += _val1784.read(iprot);
+              std::string _key1791;
+              xfer += iprot->readString(_key1791);
+              Type& _val1792 = this->success[_key1791];
+              xfer += _val1792.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -5145,11 +5145,11 @@ uint32_t ThriftHiveMetastore_get_type_all_result::write(::apache::thrift::protoc
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::map<std::string, Type> ::const_iterator _iter1785;
-      for (_iter1785 = this->success.begin(); _iter1785 != this->success.end(); ++_iter1785)
+      std::map<std::string, Type> ::const_iterator _iter1793;
+      for (_iter1793 = this->success.begin(); _iter1793 != this->success.end(); ++_iter1793)
       {
-        xfer += oprot->writeString(_iter1785->first);
-        xfer += _iter1785->second.write(oprot);
+        xfer += oprot->writeString(_iter1793->first);
+        xfer += _iter1793->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -5194,17 +5194,17 @@ uint32_t ThriftHiveMetastore_get_type_all_presult::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size1786;
-            ::apache::thrift::protocol::TType _ktype1787;
-            ::apache::thrift::protocol::TType _vtype1788;
-            xfer += iprot->readMapBegin(_ktype1787, _vtype1788, _size1786);
-            uint32_t _i1790;
-            for (_i1790 = 0; _i1790 < _size1786; ++_i1790)
+            uint32_t _size1794;
+            ::apache::thrift::protocol::TType _ktype1795;
+            ::apache::thrift::protocol::TType _vtype1796;
+            xfer += iprot->readMapBegin(_ktype1795, _vtype1796, _size1794);
+            uint32_t _i1798;
+            for (_i1798 = 0; _i1798 < _size1794; ++_i1798)
             {
-              std::string _key1791;
-              xfer += iprot->readString(_key1791);
-              Type& _val1792 = (*(this->success))[_key1791];
-              xfer += _val1792.read(iprot);
+              std::string _key1799;
+              xfer += iprot->readString(_key1799);
+              Type& _val1800 = (*(this->success))[_key1799];
+              xfer += _val1800.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -5358,14 +5358,14 @@ uint32_t ThriftHiveMetastore_get_fields_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1793;
-            ::apache::thrift::protocol::TType _etype1796;
-            xfer += iprot->readListBegin(_etype1796, _size1793);
-            this->success.resize(_size1793);
-            uint32_t _i1797;
-            for (_i1797 = 0; _i1797 < _size1793; ++_i1797)
+            uint32_t _size1801;
+            ::apache::thrift::protocol::TType _etype1804;
+            xfer += iprot->readListBegin(_etype1804, _size1801);
+            this->success.resize(_size1801);
+            uint32_t _i1805;
+            for (_i1805 = 0; _i1805 < _size1801; ++_i1805)
             {
-              xfer += this->success[_i1797].read(iprot);
+              xfer += this->success[_i1805].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5420,10 +5420,10 @@ uint32_t ThriftHiveMetastore_get_fields_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter1798;
-      for (_iter1798 = this->success.begin(); _iter1798 != this->success.end(); ++_iter1798)
+      std::vector<FieldSchema> ::const_iterator _iter1806;
+      for (_iter1806 = this->success.begin(); _iter1806 != this->success.end(); ++_iter1806)
       {
-        xfer += (*_iter1798).write(oprot);
+        xfer += (*_iter1806).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -5476,14 +5476,14 @@ uint32_t ThriftHiveMetastore_get_fields_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1799;
-            ::apache::thrift::protocol::TType _etype1802;
-            xfer += iprot->readListBegin(_etype1802, _size1799);
-            (*(this->success)).resize(_size1799);
-            uint32_t _i1803;
-            for (_i1803 = 0; _i1803 < _size1799; ++_i1803)
+            uint32_t _size1807;
+            ::apache::thrift::protocol::TType _etype1810;
+            xfer += iprot->readListBegin(_etype1810, _size1807);
+            (*(this->success)).resize(_size1807);
+            uint32_t _i1811;
+            for (_i1811 = 0; _i1811 < _size1807; ++_i1811)
             {
-              xfer += (*(this->success))[_i1803].read(iprot);
+              xfer += (*(this->success))[_i1811].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5669,14 +5669,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1804;
-            ::apache::thrift::protocol::TType _etype1807;
-            xfer += iprot->readListBegin(_etype1807, _size1804);
-            this->success.resize(_size1804);
-            uint32_t _i1808;
-            for (_i1808 = 0; _i1808 < _size1804; ++_i1808)
+            uint32_t _size1812;
+            ::apache::thrift::protocol::TType _etype1815;
+            xfer += iprot->readListBegin(_etype1815, _size1812);
+            this->success.resize(_size1812);
+            uint32_t _i1816;
+            for (_i1816 = 0; _i1816 < _size1812; ++_i1816)
             {
-              xfer += this->success[_i1808].read(iprot);
+              xfer += this->success[_i1816].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5731,10 +5731,10 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::write(:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter1809;
-      for (_iter1809 = this->success.begin(); _iter1809 != this->success.end(); ++_iter1809)
+      std::vector<FieldSchema> ::const_iterator _iter1817;
+      for (_iter1817 = this->success.begin(); _iter1817 != this->success.end(); ++_iter1817)
       {
-        xfer += (*_iter1809).write(oprot);
+        xfer += (*_iter1817).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -5787,14 +5787,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1810;
-            ::apache::thrift::protocol::TType _etype1813;
-            xfer += iprot->readListBegin(_etype1813, _size1810);
-            (*(this->success)).resize(_size1810);
-            uint32_t _i1814;
-            for (_i1814 = 0; _i1814 < _size1810; ++_i1814)
+            uint32_t _size1818;
+            ::apache::thrift::protocol::TType _etype1821;
+            xfer += iprot->readListBegin(_etype1821, _size1818);
+            (*(this->success)).resize(_size1818);
+            uint32_t _i1822;
+            for (_i1822 = 0; _i1822 < _size1818; ++_i1822)
             {
-              xfer += (*(this->success))[_i1814].read(iprot);
+              xfer += (*(this->success))[_i1822].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6211,14 +6211,14 @@ uint32_t ThriftHiveMetastore_get_schema_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1815;
-            ::apache::thrift::protocol::TType _etype1818;
-            xfer += iprot->readListBegin(_etype1818, _size1815);
-            this->success.resize(_size1815);
-            uint32_t _i1819;
-            for (_i1819 = 0; _i1819 < _size1815; ++_i1819)
+            uint32_t _size1823;
+            ::apache::thrift::protocol::TType _etype1826;
+            xfer += iprot->readListBegin(_etype1826, _size1823);
+            this->success.resize(_size1823);
+            uint32_t _i1827;
+            for (_i1827 = 0; _i1827 < _size1823; ++_i1827)
             {
-              xfer += this->success[_i1819].read(iprot);
+              xfer += this->success[_i1827].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6273,10 +6273,10 @@ uint32_t ThriftHiveMetastore_get_schema_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter1820;
-      for (_iter1820 = this->success.begin(); _iter1820 != this->success.end(); ++_iter1820)
+      std::vector<FieldSchema> ::const_iterator _iter1828;
+      for (_iter1828 = this->success.begin(); _iter1828 != this->success.end(); ++_iter1828)
       {
-        xfer += (*_iter1820).write(oprot);
+        xfer += (*_iter1828).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -6329,14 +6329,14 @@ uint32_t ThriftHiveMetastore_get_schema_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1821;
-            ::apache::thrift::protocol::TType _etype1824;
-            xfer += iprot->readListBegin(_etype1824, _size1821);
-            (*(this->success)).resize(_size1821);
-            uint32_t _i1825;
-            for (_i1825 = 0; _i1825 < _size1821; ++_i1825)
+            uint32_t _size1829;
+            ::apache::thrift::protocol::TType _etype1832;
+            xfer += iprot->readListBegin(_etype1832, _size1829);
+            (*(this->success)).resize(_size1829);
+            uint32_t _i1833;
+            for (_i1833 = 0; _i1833 < _size1829; ++_i1833)
             {
-              xfer += (*(this->success))[_i1825].read(iprot);
+              xfer += (*(this->success))[_i1833].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6522,14 +6522,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1826;
-            ::apache::thrift::protocol::TType _etype1829;
-            xfer += iprot->readListBegin(_etype1829, _size1826);
-            this->success.resize(_size1826);
-            uint32_t _i1830;
-            for (_i1830 = 0; _i1830 < _size1826; ++_i1830)
+            uint32_t _size1834;
+            ::apache::thrift::protocol::TType _etype1837;
+            xfer += iprot->readListBegin(_etype1837, _size1834);
+            this->success.resize(_size1834);
+            uint32_t _i1838;
+            for (_i1838 = 0; _i1838 < _size1834; ++_i1838)
             {
-              xfer += this->success[_i1830].read(iprot);
+              xfer += this->success[_i1838].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6584,10 +6584,10 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::write(:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter1831;
-      for (_iter1831 = this->success.begin(); _iter1831 != this->success.end(); ++_iter1831)
+      std::vector<FieldSchema> ::const_iterator _iter1839;
+      for (_iter1839 = this->success.begin(); _iter1839 != this->success.end(); ++_iter1839)
       {
-        xfer += (*_iter1831).write(oprot);
+        xfer += (*_iter1839).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -6640,14 +6640,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1832;
-            ::apache::thrift::protocol::TType _etype1835;
-            xfer += iprot->readListBegin(_etype1835, _size1832);
-            (*(this->success)).resize(_size1832);
-            uint32_t _i1836;
-            for (_i1836 = 0; _i1836 < _size1832; ++_i1836)
+            uint32_t _size1840;
+            ::apache::thrift::protocol::TType _etype1843;
+            xfer += iprot->readListBegin(_etype1843, _size1840);
+            (*(this->success)).resize(_size1840);
+            uint32_t _i1844;
+            for (_i1844 = 0; _i1844 < _size1840; ++_i1844)
             {
-              xfer += (*(this->success))[_i1836].read(iprot);
+              xfer += (*(this->success))[_i1844].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7487,14 +7487,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->primaryKeys.clear();
-            uint32_t _size1837;
-            ::apache::thrift::protocol::TType _etype1840;
-            xfer += iprot->readListBegin(_etype1840, _size1837);
-            this->primaryKeys.resize(_size1837);
-            uint32_t _i1841;
-            for (_i1841 = 0; _i1841 < _size1837; ++_i1841)
+            uint32_t _size1845;
+            ::apache::thrift::protocol::TType _etype1848;
+            xfer += iprot->readListBegin(_etype1848, _size1845);
+            this->primaryKeys.resize(_size1845);
+            uint32_t _i1849;
+            for (_i1849 = 0; _i1849 < _size1845; ++_i1849)
             {
-              xfer += this->primaryKeys[_i1841].read(iprot);
+              xfer += this->primaryKeys[_i1849].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7507,14 +7507,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->foreignKeys.clear();
-            uint32_t _size1842;
-            ::apache::thrift::protocol::TType _etype1845;
-            xfer += iprot->readListBegin(_etype1845, _size1842);
-            this->foreignKeys.resize(_size1842);
-            uint32_t _i1846;
-            for (_i1846 = 0; _i1846 < _size1842; ++_i1846)
+            uint32_t _size1850;
+            ::apache::thrift::protocol::TType _etype1853;
+            xfer += iprot->readListBegin(_etype1853, _size1850);
+            this->foreignKeys.resize(_size1850);
+            uint32_t _i1854;
+            for (_i1854 = 0; _i1854 < _size1850; ++_i1854)
             {
-              xfer += this->foreignKeys[_i1846].read(iprot);
+              xfer += this->foreignKeys[_i1854].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7527,14 +7527,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->uniqueConstraints.clear();
-            uint32_t _size1847;
-            ::apache::thrift::protocol::TType _etype1850;
-            xfer += iprot->readListBegin(_etype1850, _size1847);
-            this->uniqueConstraints.resize(_size1847);
-            uint32_t _i1851;
-            for (_i1851 = 0; _i1851 < _size1847; ++_i1851)
+            uint32_t _size1855;
+            ::apache::thrift::protocol::TType _etype1858;
+            xfer += iprot->readListBegin(_etype1858, _size1855);
+            this->uniqueConstraints.resize(_size1855);
+            uint32_t _i1859;
+            for (_i1859 = 0; _i1859 < _size1855; ++_i1859)
             {
-              xfer += this->uniqueConstraints[_i1851].read(iprot);
+              xfer += this->uniqueConstraints[_i1859].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7547,14 +7547,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->notNullConstraints.clear();
-            uint32_t _size1852;
-            ::apache::thrift::protocol::TType _etype1855;
-            xfer += iprot->readListBegin(_etype1855, _size1852);
-            this->notNullConstraints.resize(_size1852);
-            uint32_t _i1856;
-            for (_i1856 = 0; _i1856 < _size1852; ++_i1856)
+            uint32_t _size1860;
+            ::apache::thrift::protocol::TType _etype1863;
+            xfer += iprot->readListBegin(_etype1863, _size1860);
+            this->notNullConstraints.resize(_size1860);
+            uint32_t _i1864;
+            for (_i1864 = 0; _i1864 < _size1860; ++_i1864)
             {
-              xfer += this->notNullConstraints[_i1856].read(iprot);
+              xfer += this->notNullConstraints[_i1864].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7567,14 +7567,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->defaultConstraints.clear();
-            uint32_t _size1857;
-            ::apache::thrift::protocol::TType _etype1860;
-            xfer += iprot->readListBegin(_etype1860, _size1857);
-            this->defaultConstraints.resize(_size1857);
-            uint32_t _i1861;
-            for (_i1861 = 0; _i1861 < _size1857; ++_i1861)
+            uint32_t _size1865;
+            ::apache::thrift::protocol::TType _etype1868;
+            xfer += iprot->readListBegin(_etype1868, _size1865);
+            this->defaultConstraints.resize(_size1865);
+            uint32_t _i1869;
+            for (_i1869 = 0; _i1869 < _size1865; ++_i1869)
             {
-              xfer += this->defaultConstraints[_i1861].read(iprot);
+              xfer += this->defaultConstraints[_i1869].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7587,14 +7587,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->checkConstraints.clear();
-            uint32_t _size1862;
-            ::apache::thrift::protocol::TType _etype1865;
-            xfer += iprot->readListBegin(_etype1865, _size1862);
-            this->checkConstraints.resize(_size1862);
-            uint32_t _i1866;
-            for (_i1866 = 0; _i1866 < _size1862; ++_i1866)
+            uint32_t _size1870;
+            ::apache::thrift::protocol::TType _etype1873;
+            xfer += iprot->readListBegin(_etype1873, _size1870);
+            this->checkConstraints.resize(_size1870);
+            uint32_t _i1874;
+            for (_i1874 = 0; _i1874 < _size1870; ++_i1874)
             {
-              xfer += this->checkConstraints[_i1866].read(iprot);
+              xfer += this->checkConstraints[_i1874].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7627,10 +7627,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->primaryKeys.size()));
-    std::vector<SQLPrimaryKey> ::const_iterator _iter1867;
-    for (_iter1867 = this->primaryKeys.begin(); _iter1867 != this->primaryKeys.end(); ++_iter1867)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter1875;
+    for (_iter1875 = this->primaryKeys.begin(); _iter1875 != this->primaryKeys.end(); ++_iter1875)
     {
-      xfer += (*_iter1867).write(oprot);
+      xfer += (*_iter1875).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7639,10 +7639,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->foreignKeys.size()));
-    std::vector<SQLForeignKey> ::const_iterator _iter1868;
-    for (_iter1868 = this->foreignKeys.begin(); _iter1868 != this->foreignKeys.end(); ++_iter1868)
+    std::vector<SQLForeignKey> ::const_iterator _iter1876;
+    for (_iter1876 = this->foreignKeys.begin(); _iter1876 != this->foreignKeys.end(); ++_iter1876)
     {
-      xfer += (*_iter1868).write(oprot);
+      xfer += (*_iter1876).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7651,10 +7651,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 4);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->uniqueConstraints.size()));
-    std::vector<SQLUniqueConstraint> ::const_iterator _iter1869;
-    for (_iter1869 = this->uniqueConstraints.begin(); _iter1869 != this->uniqueConstraints.end(); ++_iter1869)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter1877;
+    for (_iter1877 = this->uniqueConstraints.begin(); _iter1877 != this->uniqueConstraints.end(); ++_iter1877)
     {
-      xfer += (*_iter1869).write(oprot);
+      xfer += (*_iter1877).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7663,10 +7663,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->notNullConstraints.size()));
-    std::vector<SQLNotNullConstraint> ::const_iterator _iter1870;
-    for (_iter1870 = this->notNullConstraints.begin(); _iter1870 != this->notNullConstraints.end(); ++_iter1870)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter1878;
+    for (_iter1878 = this->notNullConstraints.begin(); _iter1878 != this->notNullConstraints.end(); ++_iter1878)
     {
-      xfer += (*_iter1870).write(oprot);
+      xfer += (*_iter1878).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7675,10 +7675,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("defaultConstraints", ::apache::thrift::protocol::T_LIST, 6);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->defaultConstraints.size()));
-    std::vector<SQLDefaultConstraint> ::const_iterator _iter1871;
-    for (_iter1871 = this->defaultConstraints.begin(); _iter1871 != this->defaultConstraints.end(); ++_iter1871)
+    std::vector<SQLDefaultConstraint> ::const_iterator _iter1879;
+    for (_iter1879 = this->defaultConstraints.begin(); _iter1879 != this->defaultConstraints.end(); ++_iter1879)
     {
-      xfer += (*_iter1871).write(oprot);
+      xfer += (*_iter1879).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7687,10 +7687,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("checkConstraints", ::apache::thrift::protocol::T_LIST, 7);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->checkConstraints.size()));
-    std::vector<SQLCheckConstraint> ::const_iterator _iter1872;
-    for (_iter1872 = this->checkConstraints.begin(); _iter1872 != this->checkConstraints.end(); ++_iter1872)
+    std::vector<SQLCheckConstraint> ::const_iterator _iter1880;
+    for (_iter1880 = this->checkConstraints.begin(); _iter1880 != this->checkConstraints.end(); ++_iter1880)
     {
-      xfer += (*_iter1872).write(oprot);
+      xfer += (*_iter1880).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7718,10 +7718,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->primaryKeys)).size()));
-    std::vector<SQLPrimaryKey> ::const_iterator _iter1873;
-    for (_iter1873 = (*(this->primaryKeys)).begin(); _iter1873 != (*(this->primaryKeys)).end(); ++_iter1873)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter1881;
+    for (_iter1881 = (*(this->primaryKeys)).begin(); _iter1881 != (*(this->primaryKeys)).end(); ++_iter1881)
     {
-      xfer += (*_iter1873).write(oprot);
+      xfer += (*_iter1881).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7730,10 +7730,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->foreignKeys)).size()));
-    std::vector<SQLForeignKey> ::const_iterator _iter1874;
-    for (_iter1874 = (*(this->foreignKeys)).begin(); _iter1874 != (*(this->foreignKeys)).end(); ++_iter1874)
+    std::vector<SQLForeignKey> ::const_iterator _iter1882;
+    for (_iter1882 = (*(this->foreignKeys)).begin(); _iter1882 != (*(this->foreignKeys)).end(); ++_iter1882)
     {
-      xfer += (*_iter1874).write(oprot);
+      xfer += (*_iter1882).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7742,10 +7742,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 4);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->uniqueConstraints)).size()));
-    std::vector<SQLUniqueConstraint> ::const_iterator _iter1875;
-    for (_iter1875 = (*(this->uniqueConstraints)).begin(); _iter1875 != (*(this->uniqueConstraints)).end(); ++_iter1875)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter1883;
+    for (_iter1883 = (*(this->uniqueConstraints)).begin(); _iter1883 != (*(this->uniqueConstraints)).end(); ++_iter1883)
     {
-      xfer += (*_iter1875).write(oprot);
+      xfer += (*_iter1883).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7754,10 +7754,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->notNullConstraints)).size()));
-    std::vector<SQLNotNullConstraint> ::const_iterator _iter1876;
-    for (_iter1876 = (*(this->notNullConstraints)).begin(); _iter1876 != (*(this->notNullConstraints)).end(); ++_iter1876)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter1884;
+    for (_iter1884 = (*(this->notNullConstraints)).begin(); _iter1884 != (*(this->notNullConstraints)).end(); ++_iter1884)
     {
-      xfer += (*_iter1876).write(oprot);
+      xfer += (*_iter1884).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7766,10 +7766,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("defaultConstraints", ::apache::thrift::protocol::T_LIST, 6);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->defaultConstraints)).size()));
-    std::vector<SQLDefaultConstraint> ::const_iterator _iter1877;
-    for (_iter1877 = (*(this->defaultConstraints)).begin(); _iter1877 != (*(this->defaultConstraints)).end(); ++_iter1877)
+    std::vector<SQLDefaultConstraint> ::const_iterator _iter1885;
+    for (_iter1885 = (*(this->defaultConstraints)).begin(); _iter1885 != (*(this->defaultConstraints)).end(); ++_iter1885)
     {
-      xfer += (*_iter1877).write(oprot);
+      xfer += (*_iter1885).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7778,10 +7778,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("checkConstraints", ::apache::thrift::protocol::T_LIST, 7);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->checkConstraints)).size()));
-    std::vector<SQLCheckConstraint> ::const_iterator _iter1878;
-    for (_iter1878 = (*(this->checkConstraints)).begin(); _iter1878 != (*(this->checkConstraints)).end(); ++_iter1878)
+    std::vector<SQLCheckConstraint> ::const_iterator _iter1886;
+    for (_iter1886 = (*(this->checkConstraints)).begin(); _iter1886 != (*(this->checkConstraints)).end(); ++_iter1886)
     {
-      xfer += (*_iter1878).write(oprot);
+      xfer += (*_iter1886).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -10463,14 +10463,14 @@ uint32_t ThriftHiveMetastore_truncate_table_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partNames.clear();
-            uint32_t _size1879;
-            ::apache::thrift::protocol::TType _etype1882;
-            xfer += iprot->readListBegin(_etype1882, _size1879);
-            this->partNames.resize(_size1879);
-            uint32_t _i1883;
-            for (_i1883 = 0; _i1883 < _size1879; ++_i1883)
+            uint32_t _size1887;
+            ::apache::thrift::protocol::TType _etype1890;
+            xfer += iprot->readListBegin(_etype1890, _size1887);
+            this->partNames.resize(_size1887);
+            uint32_t _i1891;
+            for (_i1891 = 0; _i1891 < _size1887; ++_i1891)
             {
-              xfer += iprot->readString(this->partNames[_i1883]);
+              xfer += iprot->readString(this->partNames[_i1891]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10507,10 +10507,10 @@ uint32_t ThriftHiveMetastore_truncate_table_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partNames.size()));
-    std::vector<std::string> ::const_iterator _iter1884;
-    for (_iter1884 = this->partNames.begin(); _iter1884 != this->partNames.end(); ++_iter1884)
+    std::vector<std::string> ::const_iterator _iter1892;
+    for (_iter1892 = this->partNames.begin(); _iter1892 != this->partNames.end(); ++_iter1892)
     {
-      xfer += oprot->writeString((*_iter1884));
+      xfer += oprot->writeString((*_iter1892));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10542,10 +10542,10 @@ uint32_t ThriftHiveMetastore_truncate_table_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partNames)).size()));
-    std::vector<std::string> ::const_iterator _iter1885;
-    for (_iter1885 = (*(this->partNames)).begin(); _iter1885 != (*(this->partNames)).end(); ++_iter1885)
+    std::vector<std::string> ::const_iterator _iter1893;
+    for (_iter1893 = (*(this->partNames)).begin(); _iter1893 != (*(this->partNames)).end(); ++_iter1893)
     {
-      xfer += oprot->writeString((*_iter1885));
+      xfer += oprot->writeString((*_iter1893));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10996,14 +10996,14 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1886;
-            ::apache::thrift::protocol::TType _etype1889;
-            xfer += iprot->readListBegin(_etype1889, _size1886);
-            this->success.resize(_size1886);
-            uint32_t _i1890;
-            for (_i1890 = 0; _i1890 < _size1886; ++_i1890)
+            uint32_t _size1894;
+            ::apache::thrift::protocol::TType _etype1897;
+            xfer += iprot->readListBegin(_etype1897, _size1894);
+            this->success.resize(_size1894);
+            uint32_t _i1898;
+            for (_i1898 = 0; _i1898 < _size1894; ++_i1898)
             {
-              xfer += iprot->readString(this->success[_i1890]);
+              xfer += iprot->readString(this->success[_i1898]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11042,10 +11042,10 @@ uint32_t ThriftHiveMetastore_get_tables_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1891;
-      for (_iter1891 = this->success.begin(); _iter1891 != this->success.end(); ++_iter1891)
+      std::vector<std::string> ::const_iterator _iter1899;
+      for (_iter1899 = this->success.begin(); _iter1899 != this->success.end(); ++_iter1899)
       {
-        xfer += oprot->writeString((*_iter1891));
+        xfer += oprot->writeString((*_iter1899));
       }
       xfer += oprot->writeListEnd();
     }
@@ -11090,14 +11090,14 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1892;
-            ::apache::thrift::protocol::TType _etype1895;
-            xfer += iprot->readListBegin(_etype1895, _size1892);
-            (*(this->success)).resize(_size1892);
-            uint32_t _i1896;
-            for (_i1896 = 0; _i1896 < _size1892; ++_i1896)
+            uint32_t _size1900;
+            ::apache::thrift::protocol::TType _etype1903;
+            xfer += iprot->readListBegin(_etype1903, _size1900);
+            (*(this->success)).resize(_size1900);
+            uint32_t _i1904;
+            for (_i1904 = 0; _i1904 < _size1900; ++_i1904)
             {
-              xfer += iprot->readString((*(this->success))[_i1896]);
+              xfer += iprot->readString((*(this->success))[_i1904]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11267,14 +11267,14 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_result::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1897;
-            ::apache::thrift::protocol::TType _etype1900;
-            xfer += iprot->readListBegin(_etype1900, _size1897);
-            this->success.resize(_size1897);
-            uint32_t _i1901;
-            for (_i1901 = 0; _i1901 < _size1897; ++_i1901)
+            uint32_t _size1905;
+            ::apache::thrift::protocol::TType _etype1908;
+            xfer += iprot->readListBegin(_etype1908, _size1905);
+            this->success.resize(_size1905);
+            uint32_t _i1909;
+            for (_i1909 = 0; _i1909 < _size1905; ++_i1909)
             {
-              xfer += iprot->readString(this->success[_i1901]);
+              xfer += iprot->readString(this->success[_i1909]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11313,10 +11313,10 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_result::write(::apache::thrift::
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1902;
-      for (_iter1902 = this->success.begin(); _iter1902 != this->success.end(); ++_iter1902)
+      std::vector<std::string> ::const_iterator _iter1910;
+      for (_iter1910 = this->success.begin(); _iter1910 != this->success.end(); ++_iter1910)
       {
-        xfer += oprot->writeString((*_iter1902));
+        xfer += oprot->writeString((*_iter1910));
       }
       xfer += oprot->writeListEnd();
     }
@@ -11361,14 +11361,14 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_presult::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1903;
-            ::apache::thrift::protocol::TType _etype1906;
-            xfer += iprot->readListBegin(_etype1906, _size1903);
-            (*(this->success)).resize(_size1903);
-            uint32_t _i1907;
-            for (_i1907 = 0; _i1907 < _size1903; ++_i1907)
+            uint32_t _size1911;
+            ::apache::thrift::protocol::TType _etype1914;
+            xfer += iprot->readListBegin(_etype1914, _size1911);
+            (*(this->success)).resize(_size1911);
+            uint32_t _i1915;
+            for (_i1915 = 0; _i1915 < _size1911; ++_i1915)
             {
-              xfer += iprot->readString((*(this->success))[_i1907]);
+              xfer += iprot->readString((*(this->success))[_i1915]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11485,14 +11485,14 @@ uint32_t ThriftHiveMetastore_get_all_materialized_view_objects_for_rewriting_res
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1908;
-            ::apache::thrift::protocol::TType _etype1911;
-            xfer += iprot->readListBegin(_etype1911, _size1908);
-            this->success.resize(_size1908);
-            uint32_t _i1912;
-            for (_i1912 = 0; _i1912 < _size1908; ++_i1912)
+            uint32_t _size1916;
+            ::apache::thrift::protocol::TType _etype1919;
+            xfer += iprot->readListBegin(_etype1919, _size1916);
+            this->success.resize(_size1916);
+            uint32_t _i1920;
+            for (_i1920 = 0; _i1920 < _size1916; ++_i1920)
             {
-              xfer += this->success[_i1912].read(iprot);
+              xfer += this->success[_i1920].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -11531,10 +11531,10 @@ uint32_t ThriftHiveMetastore_get_all_materialized_view_objects_for_rewriting_res
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Table> ::const_iterator _iter1913;
-      for (_iter1913 = this->success.begin(); _iter1913 != this->success.end(); ++_iter1913)
+      std::vector<Table> ::const_iterator _iter1921;
+      for (_iter1921 = this->success.begin(); _iter1921 != this->success.end(); ++_iter1921)
       {
-        xfer += (*_iter1913).write(oprot);
+        xfer += (*_iter1921).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -11579,14 +11579,14 @@ uint32_t ThriftHiveMetastore_get_all_materialized_view_objects_for_rewriting_pre
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1914;
-            ::apache::thrift::protocol::TType _etype1917;
-            xfer += iprot->readListBegin(_etype1917, _size1914);
-            (*(this->success)).resize(_size1914);
-            uint32_t _i1918;
-            for (_i1918 = 0; _i1918 < _size1914; ++_i1918)
+            uint32_t _size1922;
+            ::apache::thrift::protocol::TType _etype1925;
+            xfer += iprot->readListBegin(_etype1925, _size1922);
+            (*(this->success)).resize(_size1922);
+            uint32_t _i1926;
+            for (_i1926 = 0; _i1926 < _size1922; ++_i1926)
             {
-              xfer += (*(this->success))[_i1918].read(iprot);
+              xfer += (*(this->success))[_i1926].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -11724,14 +11724,14 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_result::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1919;
-            ::apache::thrift::protocol::TType _etype1922;
-            xfer += iprot->readListBegin(_etype1922, _size1919);
-            this->success.resize(_size1919);
-            uint32_t _i1923;
-            for (_i1923 = 0; _i1923 < _size1919; ++_i1923)
+            uint32_t _size1927;
+            ::apache::thrift::protocol::TType _etype1930;
+            xfer += iprot->readListBegin(_etype1930, _size1927);
+            this->success.resize(_size1927);
+            uint32_t _i1931;
+            for (_i1931 = 0; _i1931 < _size1927; ++_i1931)
             {
-              xfer += iprot->readString(this->success[_i1923]);
+              xfer += iprot->readString(this->success[_i1931]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11770,10 +11770,10 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_result::write(
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1924;
-      for (_iter1924 = this->success.begin(); _iter1924 != this->success.end(); ++_iter1924)
+      std::vector<std::string> ::const_iterator _iter1932;
+      for (_iter1932 = this->success.begin(); _iter1932 != this->success.end(); ++_iter1932)
       {
-        xfer += oprot->writeString((*_iter1924));
+        xfer += oprot->writeString((*_iter1932));
       }
       xfer += oprot->writeListEnd();
     }
@@ -11818,14 +11818,14 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_presult::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1925;
-            ::apache::thrift::protocol::TType _etype1928;
-            xfer += iprot->readListBegin(_etype1928, _size1925);
-            (*(this->success)).resize(_size1925);
-            uint32_t _i1929;
-            for (_i1929 = 0; _i1929 < _size1925; ++_i1929)
+            uint32_t _size1933;
+            ::apache::thrift::protocol::TType _etype1936;
+            xfer += iprot->readListBegin(_etype1936, _size1933);
+            (*(this->success)).resize(_size1933);
+            uint32_t _i1937;
+            for (_i1937 = 0; _i1937 < _size1933; ++_i1937)
             {
-              xfer += iprot->readString((*(this->success))[_i1929]);
+              xfer += iprot->readString((*(this->success))[_i1937]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11900,14 +11900,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_types.clear();
-            uint32_t _size1930;
-            ::apache::thrift::protocol::TType _etype1933;
-            xfer += iprot->readListBegin(_etype1933, _size1930);
-            this->tbl_types.resize(_size1930);
-            uint32_t _i1934;
-            for (_i1934 = 0; _i1934 < _size1930; ++_i1934)
+            uint32_t _size1938;
+            ::apache::thrift::protocol::TType _etype1941;
+            xfer += iprot->readListBegin(_etype1941, _size1938);
+            this->tbl_types.resize(_size1938);
+            uint32_t _i1942;
+            for (_i1942 = 0; _i1942 < _size1938; ++_i1942)
             {
-              xfer += iprot->readString(this->tbl_types[_i1934]);
+              xfer += iprot->readString(this->tbl_types[_i1942]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11944,10 +11944,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_types.size()));
-    std::vector<std::string> ::const_iterator _iter1935;
-    for (_iter1935 = this->tbl_types.begin(); _iter1935 != this->tbl_types.end(); ++_iter1935)
+    std::vector<std::string> ::const_iterator _iter1943;
+    for (_iter1943 = this->tbl_types.begin(); _iter1943 != this->tbl_types.end(); ++_iter1943)
     {
-      xfer += oprot->writeString((*_iter1935));
+      xfer += oprot->writeString((*_iter1943));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11979,10 +11979,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_types)).size()));
-    std::vector<std::string> ::const_iterator _iter1936;
-    for (_iter1936 = (*(this->tbl_types)).begin(); _iter1936 != (*(this->tbl_types)).end(); ++_iter1936)
+    std::vector<std::string> ::const_iterator _iter1944;
+    for (_iter1944 = (*(this->tbl_types)).begin(); _iter1944 != (*(this->tbl_types)).end(); ++_iter1944)
     {
-      xfer += oprot->writeString((*_iter1936));
+      xfer += oprot->writeString((*_iter1944));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12023,14 +12023,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1937;
-            ::apache::thrift::protocol::TType _etype1940;
-            xfer += iprot->readListBegin(_etype1940, _size1937);
-            this->success.resize(_size1937);
-            uint32_t _i1941;
-            for (_i1941 = 0; _i1941 < _size1937; ++_i1941)
+            uint32_t _size1945;
+            ::apache::thrift::protocol::TType _etype1948;
+            xfer += iprot->readListBegin(_etype1948, _size1945);
+            this->success.resize(_size1945);
+            uint32_t _i1949;
+            for (_i1949 = 0; _i1949 < _size1945; ++_i1949)
             {
-              xfer += this->success[_i1941].read(iprot);
+              xfer += this->success[_i1949].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12069,10 +12069,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<TableMeta> ::const_iterator _iter1942;
-      for (_iter1942 = this->success.begin(); _iter1942 != this->success.end(); ++_iter1942)
+      std::vector<TableMeta> ::const_iterator _iter1950;
+      for (_iter1950 = this->success.begin(); _iter1950 != this->success.end(); ++_iter1950)
       {
-        xfer += (*_iter1942).write(oprot);
+        xfer += (*_iter1950).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -12117,14 +12117,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1943;
-            ::apache::thrift::protocol::TType _etype1946;
-            xfer += iprot->readListBegin(_etype1946, _size1943);
-            (*(this->success)).resize(_size1943);
-            uint32_t _i1947;
-            for (_i1947 = 0; _i1947 < _size1943; ++_i1947)
+            uint32_t _size1951;
+            ::apache::thrift::protocol::TType _etype1954;
+            xfer += iprot->readListBegin(_etype1954, _size1951);
+            (*(this->success)).resize(_size1951);
+            uint32_t _i1955;
+            for (_i1955 = 0; _i1955 < _size1951; ++_i1955)
             {
-              xfer += (*(this->success))[_i1947].read(iprot);
+              xfer += (*(this->success))[_i1955].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12262,14 +12262,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1948;
-            ::apache::thrift::protocol::TType _etype1951;
-            xfer += iprot->readListBegin(_etype1951, _size1948);
-            this->success.resize(_size1948);
-            uint32_t _i1952;
-            for (_i1952 = 0; _i1952 < _size1948; ++_i1952)
+            uint32_t _size1956;
+            ::apache::thrift::protocol::TType _etype1959;
+            xfer += iprot->readListBegin(_etype1959, _size1956);
+            this->success.resize(_size1956);
+            uint32_t _i1960;
+            for (_i1960 = 0; _i1960 < _size1956; ++_i1960)
             {
-              xfer += iprot->readString(this->success[_i1952]);
+              xfer += iprot->readString(this->success[_i1960]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12308,10 +12308,10 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1953;
-      for (_iter1953 = this->success.begin(); _iter1953 != this->success.end(); ++_iter1953)
+      std::vector<std::string> ::const_iterator _iter1961;
+      for (_iter1961 = this->success.begin(); _iter1961 != this->success.end(); ++_iter1961)
       {
-        xfer += oprot->writeString((*_iter1953));
+        xfer += oprot->writeString((*_iter1961));
       }
       xfer += oprot->writeListEnd();
     }
@@ -12356,14 +12356,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1954;
-            ::apache::thrift::protocol::TType _etype1957;
-            xfer += iprot->readListBegin(_etype1957, _size1954);
-            (*(this->success)).resize(_size1954);
-            uint32_t _i1958;
-            for (_i1958 = 0; _i1958 < _size1954; ++_i1958)
+            uint32_t _size1962;
+            ::apache::thrift::protocol::TType _etype1965;
+            xfer += iprot->readListBegin(_etype1965, _size1962);
+            (*(this->success)).resize(_size1962);
+            uint32_t _i1966;
+            for (_i1966 = 0; _i1966 < _size1962; ++_i1966)
             {
-              xfer += iprot->readString((*(this->success))[_i1958]);
+              xfer += iprot->readString((*(this->success))[_i1966]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12673,14 +12673,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_names.clear();
-            uint32_t _size1959;
-            ::apache::thrift::protocol::TType _etype1962;
-            xfer += iprot->readListBegin(_etype1962, _size1959);
-            this->tbl_names.resize(_size1959);
-            uint32_t _i1963;
-            for (_i1963 = 0; _i1963 < _size1959; ++_i1963)
+            uint32_t _size1967;
+            ::apache::thrift::protocol::TType _etype1970;
+            xfer += iprot->readListBegin(_etype1970, _size1967);
+            this->tbl_names.resize(_size1967);
+            uint32_t _i1971;
+            for (_i1971 = 0; _i1971 < _size1967; ++_i1971)
             {
-              xfer += iprot->readString(this->tbl_names[_i1963]);
+              xfer += iprot->readString(this->tbl_names[_i1971]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12713,10 +12713,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::write(::apache::thr
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_names.size()));
-    std::vector<std::string> ::const_iterator _iter1964;
-    for (_iter1964 = this->tbl_names.begin(); _iter1964 != this->tbl_names.end(); ++_iter1964)
+    std::vector<std::string> ::const_iterator _iter1972;
+    for (_iter1972 = this->tbl_names.begin(); _iter1972 != this->tbl_names.end(); ++_iter1972)
     {
-      xfer += oprot->writeString((*_iter1964));
+      xfer += oprot->writeString((*_iter1972));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12744,10 +12744,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_pargs::write(::apache::th
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_names)).size()));
-    std::vector<std::string> ::const_iterator _iter1965;
-    for (_iter1965 = (*(this->tbl_names)).begin(); _iter1965 != (*(this->tbl_names)).end(); ++_iter1965)
+    std::vector<std::string> ::const_iterator _iter1973;
+    for (_iter1973 = (*(this->tbl_names)).begin(); _iter1973 != (*(this->tbl_names)).end(); ++_iter1973)
     {
-      xfer += oprot->writeString((*_iter1965));
+      xfer += oprot->writeString((*_iter1973));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12788,14 +12788,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1966;
-            ::apache::thrift::protocol::TType _etype1969;
-            xfer += iprot->readListBegin(_etype1969, _size1966);
-            this->success.resize(_size1966);
-            uint32_t _i1970;
-            for (_i1970 = 0; _i1970 < _size1966; ++_i1970)
+            uint32_t _size1974;
+            ::apache::thrift::protocol::TType _etype1977;
+            xfer += iprot->readListBegin(_etype1977, _size1974);
+            this->success.resize(_size1974);
+            uint32_t _i1978;
+            for (_i1978 = 0; _i1978 < _size1974; ++_i1978)
             {
-              xfer += this->success[_i1970].read(iprot);
+              xfer += this->success[_i1978].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12826,10 +12826,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::write(::apache::t
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Table> ::const_iterator _iter1971;
-      for (_iter1971 = this->success.begin(); _iter1971 != this->success.end(); ++_iter1971)
+      std::vector<Table> ::const_iterator _iter1979;
+      for (_iter1979 = this->success.begin(); _iter1979 != this->success.end(); ++_iter1979)
       {
-        xfer += (*_iter1971).write(oprot);
+        xfer += (*_iter1979).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -12870,14 +12870,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1972;
-            ::apache::thrift::protocol::TType _etype1975;
-            xfer += iprot->readListBegin(_etype1975, _size1972);
-            (*(this->success)).resize(_size1972);
-            uint32_t _i1976;
-            for (_i1976 = 0; _i1976 < _size1972; ++_i1976)
+            uint32_t _size1980;
+            ::apache::thrift::protocol::TType _etype1983;
+            xfer += iprot->readListBegin(_etype1983, _size1980);
+            (*(this->success)).resize(_size1980);
+            uint32_t _i1984;
+            for (_i1984 = 0; _i1984 < _size1980; ++_i1984)
             {
-              xfer += (*(this->success))[_i1976].read(iprot);
+              xfer += (*(this->success))[_i1984].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13007,14 +13007,14 @@ uint32_t ThriftHiveMetastore_get_tables_ext_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1977;
-            ::apache::thrift::protocol::TType _etype1980;
-            xfer += iprot->readListBegin(_etype1980, _size1977);
-            this->success.resize(_size1977);
-            uint32_t _i1981;
-            for (_i1981 = 0; _i1981 < _size1977; ++_i1981)
+            uint32_t _size1985;
+            ::apache::thrift::protocol::TType _etype1988;
+            xfer += iprot->readListBegin(_etype1988, _size1985);
+            this->success.resize(_size1985);
+            uint32_t _i1989;
+            for (_i1989 = 0; _i1989 < _size1985; ++_i1989)
             {
-              xfer += this->success[_i1981].read(iprot);
+              xfer += this->success[_i1989].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13053,10 +13053,10 @@ uint32_t ThriftHiveMetastore_get_tables_ext_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<ExtendedTableInfo> ::const_iterator _iter1982;
-      for (_iter1982 = this->success.begin(); _iter1982 != this->success.end(); ++_iter1982)
+      std::vector<ExtendedTableInfo> ::const_iterator _iter1990;
+      for (_iter1990 = this->success.begin(); _iter1990 != this->success.end(); ++_iter1990)
       {
-        xfer += (*_iter1982).write(oprot);
+        xfer += (*_iter1990).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -13101,14 +13101,14 @@ uint32_t ThriftHiveMetastore_get_tables_ext_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1983;
-            ::apache::thrift::protocol::TType _etype1986;
-            xfer += iprot->readListBegin(_etype1986, _size1983);
-            (*(this->success)).resize(_size1983);
-            uint32_t _i1987;
-            for (_i1987 = 0; _i1987 < _size1983; ++_i1987)
+            uint32_t _size1991;
+            ::apache::thrift::protocol::TType _etype1994;
+            xfer += iprot->readListBegin(_etype1994, _size1991);
+            (*(this->success)).resize(_size1991);
+            uint32_t _i1995;
+            for (_i1995 = 0; _i1995 < _size1991; ++_i1995)
             {
-              xfer += (*(this->success))[_i1987].read(iprot);
+              xfer += (*(this->success))[_i1995].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14290,14 +14290,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1988;
-            ::apache::thrift::protocol::TType _etype1991;
-            xfer += iprot->readListBegin(_etype1991, _size1988);
-            this->success.resize(_size1988);
-            uint32_t _i1992;
-            for (_i1992 = 0; _i1992 < _size1988; ++_i1992)
+            uint32_t _size1996;
+            ::apache::thrift::protocol::TType _etype1999;
+            xfer += iprot->readListBegin(_etype1999, _size1996);
+            this->success.resize(_size1996);
+            uint32_t _i2000;
+            for (_i2000 = 0; _i2000 < _size1996; ++_i2000)
             {
-              xfer += iprot->readString(this->success[_i1992]);
+              xfer += iprot->readString(this->success[_i2000]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14352,10 +14352,10 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::write(::apache::t
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1993;
-      for (_iter1993 = this->success.begin(); _iter1993 != this->success.end(); ++_iter1993)
+      std::vector<std::string> ::const_iterator _iter2001;
+      for (_iter2001 = this->success.begin(); _iter2001 != this->success.end(); ++_iter2001)
       {
-        xfer += oprot->writeString((*_iter1993));
+        xfer += oprot->writeString((*_iter2001));
       }
       xfer += oprot->writeListEnd();
     }
@@ -14408,14 +14408,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1994;
-            ::apache::thrift::protocol::TType _etype1997;
-            xfer += iprot->readListBegin(_etype1997, _size1994);
-            (*(this->success)).resize(_size1994);
-            uint32_t _i1998;
-            for (_i1998 = 0; _i1998 < _size1994; ++_i1998)
+            uint32_t _size2002;
+            ::apache::thrift::protocol::TType _etype2005;
+            xfer += iprot->readListBegin(_etype2005, _size2002);
+            (*(this->success)).resize(_size2002);
+            uint32_t _i2006;
+            for (_i2006 = 0; _i2006 < _size2002; ++_i2006)
             {
-              xfer += iprot->readString((*(this->success))[_i1998]);
+              xfer += iprot->readString((*(this->success))[_i2006]);
             }
             xfer += iprot->readListEnd();
           }
@@ -15976,14 +15976,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size1999;
-            ::apache::thrift::protocol::TType _etype2002;
-            xfer += iprot->readListBegin(_etype2002, _size1999);
-            this->new_parts.resize(_size1999);
-            uint32_t _i2003;
-            for (_i2003 = 0; _i2003 < _size1999; ++_i2003)
+            uint32_t _size2007;
+            ::apache::thrift::protocol::TType _etype2010;
+            xfer += iprot->readListBegin(_etype2010, _size2007);
+            this->new_parts.resize(_size2007);
+            uint32_t _i2011;
+            for (_i2011 = 0; _i2011 < _size2007; ++_i2011)
             {
-              xfer += this->new_parts[_i2003].read(iprot);
+              xfer += this->new_parts[_i2011].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -16012,10 +16012,10 @@ uint32_t ThriftHiveMetastore_add_partitions_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->new_parts.size()));
-    std::vector<Partition> ::const_iterator _iter2004;
-    for (_iter2004 = this->new_parts.begin(); _iter2004 != this->new_parts.end(); ++_iter2004)
+    std::vector<Partition> ::const_iterator _iter2012;
+    for (_iter2012 = this->new_parts.begin(); _iter2012 != this->new_parts.end(); ++_iter2012)
     {
-      xfer += (*_iter2004).write(oprot);
+      xfer += (*_iter2012).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -16039,10 +16039,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->new_parts)).size()));
-    std::vector<Partition> ::const_iterator _iter2005;
-    for (_iter2005 = (*(this->new_parts)).begin(); _iter2005 != (*(this->new_parts)).end(); ++_iter2005)
+    std::vector<Partition> ::const_iterator _iter2013;
+    for (_iter2013 = (*(this->new_parts)).begin(); _iter2013 != (*(this->new_parts)).end(); ++_iter2013)
     {
-      xfer += (*_iter2005).write(oprot);
+      xfer += (*_iter2013).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -16251,14 +16251,14 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_args::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size2006;
-            ::apache::thrift::protocol::TType _etype2009;
-            xfer += iprot->readListBegin(_etype2009, _size2006);
-            this->new_parts.resize(_size2006);
-            uint32_t _i2010;
-            for (_i2010 = 0; _i2010 < _size2006; ++_i2010)
+            uint32_t _size2014;
+            ::apache::thrift::protocol::TType _etype2017;
+            xfer += iprot->readListBegin(_etype2017, _size2014);
+            this->new_parts.resize(_size2014);
+            uint32_t _i2018;
+            for (_i2018 = 0; _i2018 < _size2014; ++_i2018)
             {
-              xfer += this->new_parts[_i2010].read(iprot);
+              xfer += this->new_parts[_i2018].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -16287,10 +16287,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_args::write(::apache::thrift::
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->new_parts.size()));
-    std::vector<PartitionSpec> ::const_iterator _iter2011;
-    for (_iter2011 = this->new_parts.begin(); _iter2011 != this->new_parts.end(); ++_iter2011)
+    std::vector<PartitionSpec> ::const_iterator _iter2019;
+    for (_iter2019 = this->new_parts.begin(); _iter2019 != this->new_parts.end(); ++_iter2019)
     {
-      xfer += (*_iter2011).write(oprot);
+      xfer += (*_iter2019).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -16314,10 +16314,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_pargs::write(::apache::thrift:
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->new_parts)).size()));
-    std::vector<PartitionSpec> ::const_iterator _iter2012;
-    for (_iter2012 = (*(this->new_parts)).begin(); _iter2012 != (*(this->new_parts)).end(); ++_iter2012)
+    std::vector<PartitionSpec> ::const_iterator _iter2020;
+    for (_iter2020 = (*(this->new_parts)).begin(); _iter2020 != (*(this->new_parts)).end(); ++_iter2020)
     {
-      xfer += (*_iter2012).write(oprot);
+      xfer += (*_iter2020).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -16542,14 +16542,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size2013;
-            ::apache::thrift::protocol::TType _etype2016;
-            xfer += iprot->readListBegin(_etype2016, _size2013);
-            this->part_vals.resize(_size2013);
-            uint32_t _i2017;
-            for (_i2017 = 0; _i2017 < _size2013; ++_i2017)
+            uint32_t _size2021;
+            ::apache::thrift::protocol::TType _etype2024;
+            xfer += iprot->readListBegin(_etype2024, _size2021);
+            this->part_vals.resize(_size2021);
+            uint32_t _i2025;
+            for (_i2025 = 0; _i2025 < _size2021; ++_i2025)
             {
-              xfer += iprot->readString(this->part_vals[_i2017]);
+              xfer += iprot->readString(this->part_vals[_i2025]);
             }
             xfer += iprot->readListEnd();
           }
@@ -16586,10 +16586,10 @@ uint32_t ThriftHiveMetastore_append_partition_args::write(::apache::thrift::prot
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter2018;
-    for (_iter2018 = this->part_vals.begin(); _iter2018 != this->part_vals.end(); ++_iter2018)
+    std::vector<std::string> ::const_iterator _iter2026;
+    for (_iter2026 = this->part_vals.begin(); _iter2026 != this->part_vals.end(); ++_iter2026)
     {
-      xfer += oprot->writeString((*_iter2018));
+      xfer += oprot->writeString((*_iter2026));
     }
     xfer += oprot->writeListEnd();
   }
@@ -16621,10 +16621,10 @@ uint32_t ThriftHiveMetastore_append_partition_pargs::write(::apache::thrift::pro
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter2019;
-    for (_iter2019 = (*(this->part_vals)).begin(); _iter2019 != (*(this->part_vals)).end(); ++_iter2019)
+    std::vector<std::string> ::const_iterator _iter2027;
+    for (_iter2027 = (*(this->part_vals)).begin(); _iter2027 != (*(this->part_vals)).end(); ++_iter2027)
     {
-      xfer += oprot->writeString((*_iter2019));
+      xfer += oprot->writeString((*_iter2027));
     }
     xfer += oprot->writeListEnd();
   }
@@ -17096,14 +17096,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size2020;
-            ::apache::thrift::protocol::TType _etype2023;
-            xfer += iprot->readListBegin(_etype2023, _size2020);
-            this->part_vals.resize(_size2020);
-            uint32_t _i2024;
-            for (_i2024 = 0; _i2024 < _size2020; ++_i2024)
+            uint32_t _size2028;
+            ::apache::thrift::protocol::TType _etype2031;
+            xfer += iprot->readListBegin(_etype2031, _size2028);
+            this->part_vals.resize(_size2028);
+            uint32_t _i2032;
+            for (_i2032 = 0; _i2032 < _size2028; ++_i2032)
             {
-              xfer += iprot->readString(this->part_vals[_i2024]);
+              xfer += iprot->readString(this->part_vals[_i2032]);
             }
             xfer += iprot->readListEnd();
           }
@@ -17148,10 +17148,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::wri
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter2025;
-    for (_iter2025 = this->part_vals.begin(); _iter2025 != this->part_vals.end(); ++_iter2025)
+    std::vector<std::string> ::const_iterator _iter2033;
+    for (_iter2033 = this->part_vals.begin(); _iter2033 != this->part_vals.end(); ++_iter2033)
     {
-      xfer += oprot->writeString((*_iter2025));
+      xfer += oprot->writeString((*_iter2033));
     }
     xfer += oprot->writeListEnd();
   }
@@ -17187,10 +17187,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_pargs::wr
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter2026;
-    for (_iter2026 = (*(this->part_vals)).begin(); _iter2026 != (*(this->part_vals)).end(); ++_iter2026)
+    std::vector<std::string> ::const_iterator _iter2034;
+    for (_iter2034 = (*(this->part_vals)).begin(); _iter2034 != (*(this->part_vals)).end(); ++_iter2034)
     {
-      xfer += oprot->writeString((*_iter2026));
+      xfer += oprot->writeString((*_iter2034));
     }
     xfer += oprot->writeListEnd();
   }
@@ -17993,14 +17993,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size2027;
-            ::apache::thrift::protocol::TType _etype2030;
-            xfer += iprot->readListBegin(_etype2030, _size2027);
-            this->part_vals.resize(_size2027);
-            uint32_t _i2031;
-            for (_i2031 = 0; _i2031 < _size2027; ++_i2031)
+            uint32_t _size2035;
+            ::apache::thrift::protocol::TType _etype2038;
+            xfer += iprot->readListBegin(_etype2038, _size2035);
+            this->part_vals.resize(_size2035);
+            uint32_t _i2039;
+            for (_i2039 = 0; _i2039 < _size2035; ++_i2039)
             {
-              xfer += iprot->readString(this->part_vals[_i2031]);
+              xfer += iprot->readString(this->part_vals[_i2039]);
             }
             xfer += iprot->readListEnd();
           }
@@ -18045,10 +18045,10 @@ uint32_t ThriftHiveMetastore_drop_partition_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter2032;
-    for (_iter2032 = this->part_vals.begin(); _iter2032 != this->part_vals.end(); ++_iter2032)
+    std::vector<std::string> ::const_iterator _iter2040;
+    for (_iter2040 = this->part_vals.begin(); _iter2040 != this->part_vals.end(); ++_iter2040)
     {
-      xfer += oprot->writeString((*_iter2032));
+      xfer += oprot->writeString((*_iter2040));
     }
     xfer += oprot->writeListEnd();
   }
@@ -18084,10 +18084,10 @@ uint32_t ThriftHiveMetastore_drop_partition_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter2033;
-    for (_iter2033 = (*(this->part_vals)).begin(); _iter2033 != (*(this->part_vals)).end(); ++_iter2033)
+    std::vector<std::string> ::const_iterator _iter2041;
+    for (_iter2041 = (*(this->part_vals)).begin(); _iter2041 != (*(this->part_vals)).end(); ++_iter2041)
     {
-      xfer += oprot->writeString((*_iter2033));
+      xfer += oprot->writeString((*_iter2041));
     }
     xfer += oprot->writeListEnd();
   }
@@ -18296,14 +18296,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size2034;
-            ::apache::thrift::protocol::TType _etype2037;
-            xfer += iprot->readListBegin(_etype2037, _size2034);
-            this->part_vals.resize(_size2034);
-            uint32_t _i2038;
-            for (_i2038 = 0; _i2038 < _size2034; ++_i2038)
+            uint32_t _size2042;
+            ::apache::thrift::protocol::TType _etype2045;
+            xfer += iprot->readListBegin(_etype2045, _size2042);
+            this->part_vals.resize(_size2042);
+            uint32_t _i2046;
+            for (_i2046 = 0; _i2046 < _size2042; ++_i2046)
             {
-              xfer += iprot->readString(this->part_vals[_i2038]);
+              xfer += iprot->readString(this->part_vals[_i2046]);
             }
             xfer += iprot->readListEnd();
           }
@@ -18356,10 +18356,10 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::write
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter2039;
-    for (_iter2039 = this->part_vals.begin(); _iter2039 != this->part_vals.end(); ++_iter2039)
+    std::vector<std::string> ::const_iterator _iter2047;
+    for (_iter2047 = this->part_vals.begin(); _iter2047 != this->part_vals.end(); ++_iter2047)
     {
-      xfer += oprot->writeString((*_iter2039));
+      xfer += oprot->writeString((*_iter2047));
     }
     xfer += oprot->writeListEnd();
   }
@@ -18399,10 +18399,10 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_pargs::writ
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter2040;
-    for (_iter2040 = (*(this->part_vals)).begin(); _iter2040 != (*(this->part_vals)).end(); ++_iter2040)
+    std::vector<std::string> ::const_iterator _iter2048;
+    for (_iter2048 = (*(this->part_vals)).begin(); _iter2048 != (*(this->part_vals)).end(); ++_iter2048)
     {
-      xfer += oprot->writeString((*_iter2040));
+      xfer += oprot->writeString((*_iter2048));
     }
     xfer += oprot->writeListEnd();
   }
@@ -19408,14 +19408,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size2041;
-            ::apache::thrift::protocol::TType _etype2044;
-            xfer += iprot->readListBegin(_etype2044, _size2041);
-            this->part_vals.resize(_size2041);
-            uint32_t _i2045;
-            for (_i2045 = 0; _i2045 < _size2041; ++_i2045)
+            uint32_t _size2049;
+            ::apache::thrift::protocol::TType _etype2052;
+            xfer += iprot->readListBegin(_etype2052, _size2049);
+            this->part_vals.resize(_size2049);
+            uint32_t _i2053;
+            for (_i2053 = 0; _i2053 < _size2049; ++_i2053)
             {
-              xfer += iprot->readString(this->part_vals[_i2045]);
+              xfer += iprot->readString(this->part_vals[_i2053]);
             }
             xfer += iprot->readListEnd();
           }
@@ -19452,10 +19452,10 @@ uint32_t ThriftHiveMetastore_get_partition_args::write(::apache::thrift::protoco
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter2046;
-    for (_iter2046 = this->part_vals.begin(); _iter2046 != this->part_vals.end(); ++_iter2046)
+    std::vector<std::string> ::const_iterator _iter2054;
+    for (_iter2054 = this->part_vals.begin(); _iter2054 != this->part_vals.end(); ++_iter2054)
     {
-      xfer += oprot->writeString((*_iter2046));
+      xfer += oprot->writeString((*_iter2054));
     }
     xfer += oprot->writeListEnd();
   }
@@ -19487,10 +19487,10 @@ uint32_t ThriftHiveMetastore_get_partition_pargs::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter2047;
-    for (_iter2047 = (*(this->part_vals)).begin(); _iter2047 != (*(this->part_vals)).end(); ++_iter2047)
+    std::vector<std::string> ::const_iterator _iter2055;
+    for (_iter2055 = (*(this->part_vals)).begin(); _iter2055 != (*(this->part_vals)).end(); ++_iter2055)
     {
-      xfer += oprot->writeString((*_iter2047));
+      xfer += oprot->writeString((*_iter2055));
     }
     xfer += oprot->writeListEnd();
   }
@@ -19906,17 +19906,17 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size2048;
-            ::apache::thrift::protocol::TType _ktype2049;
-            ::apache::thrift::protocol::TType _vtype2050;
-            xfer += iprot->readMapBegin(_ktype2049, _vtype2050, _size2048);
-            uint32_t _i2052;
-            for (_i2052 = 0; _i2052 < _size2048; ++_i2052)
+            uint32_t _size2056;
+            ::apache::thrift::protocol::TType _ktype2057;
+            ::apache::thrift::protocol::TType _vtype2058;
+            xfer += iprot->readMapBegin(_ktype2057, _vtype2058, _size2056);
+            uint32_t _i2060;
+            for (_i2060 = 0; _i2060 < _size2056; ++_i2060)
             {
-              std::string _key2053;
-              xfer += iprot->readString(_key2053);
-              std::string& _val2054 = this->partitionSpecs[_key2053];
-              xfer += iprot->readString(_val2054);
+              std::string _key2061;
+              xfer += iprot->readString(_key2061);
+              std::string& _val2062 = this->partitionSpecs[_key2061];
+              xfer += iprot->readString(_val2062);
             }
             xfer += iprot->readMapEnd();
           }
@@ -19977,11 +19977,11 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::write(::apache::thrift::pr
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionSpecs.size()));
-    std::map<std::string, std::string> ::const_iterator _iter2055;
-    for (_iter2055 = this->partitionSpecs.begin(); _iter2055 != this->partitionSpecs.end(); ++_iter2055)
+    std::map<std::string, std::string> ::const_iterator _iter2063;
+    for (_iter2063 = this->partitionSpecs.begin(); _iter2063 != this->partitionSpecs.end(); ++_iter2063)
     {
-      xfer += oprot->writeString(_iter2055->first);
-      xfer += oprot->writeString(_iter2055->second);
+      xfer += oprot->writeString(_iter2063->first);
+      xfer += oprot->writeString(_iter2063->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -20021,11 +20021,11 @@ uint32_t ThriftHiveMetastore_exchange_partition_pargs::write(::apache::thrift::p
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partitionSpecs)).size()));
-    std::map<std::string, std::string> ::const_iterator _iter2056;
-    for (_iter2056 = (*(this->partitionSpecs)).begin(); _iter2056 != (*(this->partitionSpecs)).end(); ++_iter2056)
+    std::map<std::string, std::string> ::const_iterator _iter2064;
+    for (_iter2064 = (*(this->partitionSpecs)).begin(); _iter2064 != (*(this->partitionSpecs)).end(); ++_iter2064)
     {
-      xfer += oprot->writeString(_iter2056->first);
-      xfer += oprot->writeString(_iter2056->second);
+      xfer += oprot->writeString(_iter2064->first);
+      xfer += oprot->writeString(_iter2064->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -20270,17 +20270,17 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size2057;
-            ::apache::thrift::protocol::TType _ktype2058;
-            ::apache::thrift::protocol::TType _vtype2059;
-            xfer += iprot->readMapBegin(_ktype2058, _vtype2059, _size2057);
-            uint32_t _i2061;
-            for (_i2061 = 0; _i2061 < _size2057; ++_i2061)
+            uint32_t _size2065;
+            ::apache::thrift::protocol::TType _ktype2066;
+            ::apache::thrift::protocol::TType _vtype2067;
+            xfer += iprot->readMapBegin(_ktype2066, _vtype2067, _size2065);
+            uint32_t _i2069;
+            for (_i2069 = 0; _i2069 < _size2065; ++_i2069)
             {
-              std::string _key2062;
-              xfer += iprot->readString(_key2062);
-              std::string& _val2063 = this->partitionSpecs[_key2062];
-              xfer += iprot->readString(_val2063);
+              std::string _key2070;
+              xfer += iprot->readString(_key2070);
+              std::string& _val2071 = this->partitionSpecs[_key2070];
+              xfer += iprot->readString(_val2071);
             }
             xfer += iprot->readMapEnd();
           }
@@ -20341,11 +20341,11 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::write(::apache::thrift::p
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionSpecs.size()));
-    std::map<std::string, std::string> ::const_iterator _iter2064;
-    for (_iter2064 = this->partitionSpecs.begin(); _iter2064 != this->partitionSpecs.end(); ++_iter2064)
+    std::map<std::string, std::string> ::const_iterator _iter2072;
+    for (_iter2072 = this->partitionSpecs.begin(); _iter2072 != this->partitionSpecs.end(); ++_iter2072)
     {
-      xfer += oprot->writeString(_iter2064->first);
-      xfer += oprot->writeString(_iter2064->second);
+      xfer += oprot->writeString(_iter2072->first);
+      xfer += oprot->writeString(_iter2072->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -20385,11 +20385,11 @@ uint32_t ThriftHiveMetastore_exchange_partitions_pargs::write(::apache::thrift::
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partitionSpecs)).size()));
-    std::map<std::string, std::string> ::const_iterator _iter2065;
-    for (_iter2065 = (*(this->partitionSpecs)).begin(); _iter2065 != (*(this->partitionSpecs)).end(); ++_iter2065)
+    std::map<std::string, std::string> ::const_iterator _iter2073;
+    for (_iter2073 = (*(this->partitionSpecs)).begin(); _iter2073 != (*(this->partitionSpecs)).end(); ++_iter2073)
     {
-      xfer += oprot->writeString(_iter2065->first);
-      xfer += oprot->writeString(_iter2065->second);
+      xfer += oprot->writeString(_iter2073->first);
+      xfer += oprot->writeString(_iter2073->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -20446,14 +20446,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2066;
-            ::apache::thrift::protocol::TType _etype2069;
-            xfer += iprot->readListBegin(_etype2069, _size2066);
-            this->success.resize(_size2066);
-            uint32_t _i2070;
-            for (_i2070 = 0; _i2070 < _size2066; ++_i2070)
+            uint32_t _size2074;
+            ::apache::thrift::protocol::TType _etype2077;
+            xfer += iprot->readListBegin(_etype2077, _size2074);
+            this->success.resize(_size2074);
+            uint32_t _i2078;
+            for (_i2078 = 0; _i2078 < _size2074; ++_i2078)
             {
-              xfer += this->success[_i2070].read(iprot);
+              xfer += this->success[_i2078].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -20516,10 +20516,10 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::write(::apache::thrift:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter2071;
-      for (_iter2071 = this->success.begin(); _iter2071 != this->success.end(); ++_iter2071)
+      std::vector<Partition> ::const_iterator _iter2079;
+      for (_iter2079 = this->success.begin(); _iter2079 != this->success.end(); ++_iter2079)
       {
-        xfer += (*_iter2071).write(oprot);
+        xfer += (*_iter2079).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -20576,14 +20576,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_presult::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2072;
-            ::apache::thrift::protocol::TType _etype2075;
-            xfer += iprot->readListBegin(_etype2075, _size2072);
-            (*(this->success)).resize(_size2072);
-            uint32_t _i2076;
-            for (_i2076 = 0; _i2076 < _size2072; ++_i2076)
+            uint32_t _size2080;
+            ::apache::thrift::protocol::TType _etype2083;
+            xfer += iprot->readListBegin(_etype2083, _size2080);
+            (*(this->success)).resize(_size2080);
+            uint32_t _i2084;
+            for (_i2084 = 0; _i2084 < _size2080; ++_i2084)
             {
-              xfer += (*(this->success))[_i2076].read(iprot);
+              xfer += (*(this->success))[_i2084].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -20682,14 +20682,14 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size2077;
-            ::apache::thrift::protocol::TType _etype2080;
-            xfer += iprot->readListBegin(_etype2080, _size2077);
-            this->part_vals.resize(_size2077);
-            uint32_t _i2081;
-            for (_i2081 = 0; _i2081 < _size2077; ++_i2081)
+            uint32_t _size2085;
+            ::apache::thrift::protocol::TType _etype2088;
+            xfer += iprot->readListBegin(_etype2088, _size2085);
+            this->part_vals.resize(_size2085);
+            uint32_t _i2089;
+            for (_i2089 = 0; _i2089 < _size2085; ++_i2089)
             {
-              xfer += iprot->readString(this->part_vals[_i2081]);
+              xfer += iprot->readString(this->part_vals[_i2089]);
             }
             xfer += iprot->readListEnd();
           }
@@ -20710,14 +20710,14 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size2082;
-            ::apache::thrift::protocol::TType _etype2085;
-            xfer += iprot->readListBegin(_etype2085, _size2082);
-            this->group_names.resize(_size2082);
-            uint32_t _i2086;
-            for (_i2086 = 0; _i2086 < _size2082; ++_i2086)
+            uint32_t _size2090;
+            ::apache::thrift::protocol::TType _etype2093;
+            xfer += iprot->readListBegin(_etype2093, _size2090);
+            this->group_names.resize(_size2090);
+            uint32_t _i2094;
+            for (_i2094 = 0; _i2094 < _size2090; ++_i2094)
             {
-              xfer += iprot->readString(this->group_names[_i2086]);
+              xfer += iprot->readString(this->group_names[_i2094]);
             }
             xfer += iprot->readListEnd();
           }
@@ -20754,10 +20754,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrif
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter2087;
-    for (_iter2087 = this->part_vals.begin(); _iter2087 != this->part_vals.end(); ++_iter2087)
+    std::vector<std::string> ::const_iterator _iter2095;
+    for (_iter2095 = this->part_vals.begin(); _iter2095 != this->part_vals.end(); ++_iter2095)
     {
-      xfer += oprot->writeString((*_iter2087));
+      xfer += oprot->writeString((*_iter2095));
     }
     xfer += oprot->writeListEnd();
   }
@@ -20770,10 +20770,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrif
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter2088;
-    for (_iter2088 = this->group_names.begin(); _iter2088 != this->group_names.end(); ++_iter2088)
+    std::vector<std::string> ::const_iterator _iter2096;
+    for (_iter2096 = this->group_names.begin(); _iter2096 != this->group_names.end(); ++_iter2096)
     {
-      xfer += oprot->writeString((*_iter2088));
+      xfer += oprot->writeString((*_iter2096));
     }
     xfer += oprot->writeListEnd();
   }
@@ -20805,10 +20805,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thri
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter2089;
-    for (_iter2089 = (*(this->part_vals)).begin(); _iter2089 != (*(this->part_vals)).end(); ++_iter2089)
+    std::vector<std::string> ::const_iterator _iter2097;
+    for (_iter2097 = (*(this->part_vals)).begin(); _iter2097 != (*(this->part_vals)).end(); ++_iter2097)
     {
-      xfer += oprot->writeString((*_iter2089));
+      xfer += oprot->writeString((*_iter2097));
     }
     xfer += oprot->writeListEnd();
   }
@@ -20821,10 +20821,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thri
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter2090;
-    for (_iter2090 = (*(this->group_names)).begin(); _iter2090 != (*(this->group_names)).end(); ++_iter2090)
+    std::vector<std::string> ::const_iterator _iter2098;
+    for (_iter2098 = (*(this->group_names)).begin(); _iter2098 != (*(this->group_names)).end(); ++_iter2098)
     {
-      xfer += oprot->writeString((*_iter2090));
+      xfer += oprot->writeString((*_iter2098));
     }
     xfer += oprot->writeListEnd();
   }
@@ -21383,14 +21383,14 @@ uint32_t ThriftHiveMetastore_get_partitions_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2091;
-            ::apache::thrift::protocol::TType _etype2094;
-            xfer += iprot->readListBegin(_etype2094, _size2091);
-            this->success.resize(_size2091);
-            uint32_t _i2095;
-            for (_i2095 = 0; _i2095 < _size2091; ++_i2095)
+            uint32_t _size2099;
+            ::apache::thrift::protocol::TType _etype2102;
+            xfer += iprot->readListBegin(_etype2102, _size2099);
+            this->success.resize(_size2099);
+            uint32_t _i2103;
+            for (_i2103 = 0; _i2103 < _size2099; ++_i2103)
             {
-              xfer += this->success[_i2095].read(iprot);
+              xfer += this->success[_i2103].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -21437,10 +21437,10 @@ uint32_t ThriftHiveMetastore_get_partitions_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter2096;
-      for (_iter2096 = this->success.begin(); _iter2096 != this->success.end(); ++_iter2096)
+      std::vector<Partition> ::const_iterator _iter2104;
+      for (_iter2104 = this->success.begin(); _iter2104 != this->success.end(); ++_iter2104)
       {
-        xfer += (*_iter2096).write(oprot);
+        xfer += (*_iter2104).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -21489,14 +21489,14 @@ uint32_t ThriftHiveMetastore_get_partitions_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2097;
-            ::apache::thrift::protocol::TType _etype2100;
-            xfer += iprot->readListBegin(_etype2100, _size2097);
-            (*(this->success)).resize(_size2097);
-            uint32_t _i2101;
-            for (_i2101 = 0; _i2101 < _size2097; ++_i2101)
+            uint32_t _size2105;
+            ::apache::thrift::protocol::TType _etype2108;
+            xfer += iprot->readListBegin(_etype2108, _size2105);
+            (*(this->success)).resize(_size2105);
+            uint32_t _i2109;
+            for (_i2109 = 0; _i2109 < _size2105; ++_i2109)
             {
-              xfer += (*(this->success))[_i2101].read(iprot);
+              xfer += (*(this->success))[_i2109].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -21822,14 +21822,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::read(::apache::thrif
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size2102;
-            ::apache::thrift::protocol::TType _etype2105;
-            xfer += iprot->readListBegin(_etype2105, _size2102);
-            this->group_names.resize(_size2102);
-            uint32_t _i2106;
-            for (_i2106 = 0; _i2106 < _size2102; ++_i2106)
+            uint32_t _size2110;
+            ::apache::thrift::protocol::TType _etype2113;
+            xfer += iprot->readListBegin(_etype2113, _size2110);
+            this->group_names.resize(_size2110);
+            uint32_t _i2114;
+            for (_i2114 = 0; _i2114 < _size2110; ++_i2114)
             {
-              xfer += iprot->readString(this->group_names[_i2106]);
+              xfer += iprot->readString(this->group_names[_i2114]);
             }
             xfer += iprot->readListEnd();
           }
@@ -21874,10 +21874,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::write(::apache::thri
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter2107;
-    for (_iter2107 = this->group_names.begin(); _iter2107 != this->group_names.end(); ++_iter2107)
+    std::vector<std::string> ::const_iterator _iter2115;
+    for (_iter2115 = this->group_names.begin(); _iter2115 != this->group_names.end(); ++_iter2115)
     {
-      xfer += oprot->writeString((*_iter2107));
+      xfer += oprot->writeString((*_iter2115));
     }
     xfer += oprot->writeListEnd();
   }
@@ -21917,10 +21917,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_pargs::write(::apache::thr
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter2108;
-    for (_iter2108 = (*(this->group_names)).begin(); _iter2108 != (*(this->group_names)).end(); ++_iter2108)
+    std::vector<std::string> ::const_iterator _iter2116;
+    for (_iter2116 = (*(this->group_names)).begin(); _iter2116 != (*(this->group_names)).end(); ++_iter2116)
     {
-      xfer += oprot->writeString((*_iter2108));
+      xfer += oprot->writeString((*_iter2116));
     }
     xfer += oprot->writeListEnd();
   }
@@ -21961,14 +21961,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2109;
-            ::apache::thrift::protocol::TType _etype2112;
-            xfer += iprot->readListBegin(_etype2112, _size2109);
-            this->success.resize(_size2109);
-            uint32_t _i2113;
-            for (_i2113 = 0; _i2113 < _size2109; ++_i2113)
+            uint32_t _size2117;
+            ::apache::thrift::protocol::TType _etype2120;
+            xfer += iprot->readListBegin(_etype2120, _size2117);
+            this->success.resize(_size2117);
+            uint32_t _i2121;
+            for (_i2121 = 0; _i2121 < _size2117; ++_i2121)
             {
-              xfer += this->success[_i2113].read(iprot);
+              xfer += this->success[_i2121].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -22015,10 +22015,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::write(::apache::th
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter2114;
-      for (_iter2114 = this->success.begin(); _iter2114 != this->success.end(); ++_iter2114)
+      std::vector<Partition> ::const_iterator _iter2122;
+      for (_iter2122 = this->success.begin(); _iter2122 != this->success.end(); ++_iter2122)
       {
-        xfer += (*_iter2114).write(oprot);
+        xfer += (*_iter2122).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -22067,14 +22067,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_presult::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2115;
-            ::apache::thrift::protocol::TType _etype2118;
-            xfer += iprot->readListBegin(_etype2118, _size2115);
-            (*(this->success)).resize(_size2115);
-            uint32_t _i2119;
-            for (_i2119 = 0; _i2119 < _size2115; ++_i2119)
+            uint32_t _size2123;
+            ::apache::thrift::protocol::TType _etype2126;
+            xfer += iprot->readListBegin(_etype2126, _size2123);
+            (*(this->success)).resize(_size2123);
+            uint32_t _i2127;
+            for (_i2127 = 0; _i2127 < _size2123; ++_i2127)
             {
-              xfer += (*(this->success))[_i2119].read(iprot);
+              xfer += (*(this->success))[_i2127].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -22252,14 +22252,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2120;
-            ::apache::thrift::protocol::TType _etype2123;
-            xfer += iprot->readListBegin(_etype2123, _size2120);
-            this->success.resize(_size2120);
-            uint32_t _i2124;
-            for (_i2124 = 0; _i2124 < _size2120; ++_i2124)
+            uint32_t _size2128;
+            ::apache::thrift::protocol::TType _etype2131;
+            xfer += iprot->readListBegin(_etype2131, _size2128);
+            this->success.resize(_size2128);
+            uint32_t _i2132;
+            for (_i2132 = 0; _i2132 < _size2128; ++_i2132)
             {
-              xfer += this->success[_i2124].read(iprot);
+              xfer += this->success[_i2132].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -22306,10 +22306,10 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::write(::apache::thrift
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<PartitionSpec> ::const_iterator _iter2125;
-      for (_iter2125 = this->success.begin(); _iter2125 != this->success.end(); ++_iter2125)
+      std::vector<PartitionSpec> ::const_iterator _iter2133;
+      for (_iter2133 = this->success.begin(); _iter2133 != this->success.end(); ++_iter2133)
       {
-        xfer += (*_iter2125).write(oprot);
+        xfer += (*_iter2133).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -22358,14 +22358,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_presult::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2126;
-            ::apache::thrift::protocol::TType _etype2129;
-            xfer += iprot->readListBegin(_etype2129, _size2126);
-            (*(this->success)).resize(_size2126);
-            uint32_t _i2130;
-            for (_i2130 = 0; _i2130 < _size2126; ++_i2130)
+            uint32_t _size2134;
+            ::apache::thrift::protocol::TType _etype2137;
+            xfer += iprot->readListBegin(_etype2137, _size2134);
+            (*(this->success)).resize(_size2134);
+            uint32_t _i2138;
+            for (_i2138 = 0; _i2138 < _size2134; ++_i2138)
             {
-              xfer += (*(this->success))[_i2130].read(iprot);
+              xfer += (*(this->success))[_i2138].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -22543,14 +22543,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2131;
-            ::apache::thrift::protocol::TType _etype2134;
-            xfer += iprot->readListBegin(_etype2134, _size2131);
-            this->success.resize(_size2131);
-            uint32_t _i2135;
-            for (_i2135 = 0; _i2135 < _size2131; ++_i2135)
+            uint32_t _size2139;
+            ::apache::thrift::protocol::TType _etype2142;
+            xfer += iprot->readListBegin(_etype2142, _size2139);
+            this->success.resize(_size2139);
+            uint32_t _i2143;
+            for (_i2143 = 0; _i2143 < _size2139; ++_i2143)
             {
-              xfer += iprot->readString(this->success[_i2135]);
+              xfer += iprot->readString(this->success[_i2143]);
             }
             xfer += iprot->readListEnd();
           }
@@ -22597,10 +22597,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::write(::apache::thrift:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter2136;
-      for (_iter2136 = this->success.begin(); _iter2136 != this->success.end(); ++_iter2136)
+      std::vector<std::string> ::const_iterator _iter2144;
+      for (_iter2144 = this->success.begin(); _iter2144 != this->success.end(); ++_iter2144)
       {
-        xfer += oprot->writeString((*_iter2136));
+        xfer += oprot->writeString((*_iter2144));
       }
       xfer += oprot->writeListEnd();
     }
@@ -22649,14 +22649,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_presult::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2137;
-            ::apache::thrift::protocol::TType _etype2140;
-            xfer += iprot->readListBegin(_etype2140, _size2137);
-            (*(this->success)).resize(_size2137);
-            uint32_t _i2141;
-            for (_i2141 = 0; _i2141 < _size2137; ++_i2141)
+            uint32_t _size2145;
+            ::apache::thrift::protocol::TType _etype2148;
+            xfer += iprot->readListBegin(_etype2148, _size2145);
+            (*(this->success)).resize(_size2145);
+            uint32_t _i2149;
+            for (_i2149 = 0; _i2149 < _size2145; ++_i2149)
             {
-              xfer += iprot->readString((*(this->success))[_i2141]);
+              xfer += iprot->readString((*(this->success))[_i2149]);
             }
             xfer += iprot->readListEnd();
           }
@@ -22966,14 +22966,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_args::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size2142;
-            ::apache::thrift::protocol::TType _etype2145;
-            xfer += iprot->readListBegin(_etype2145, _size2142);
-            this->part_vals.resize(_size2142);
-            uint32_t _i2146;
-            for (_i2146 = 0; _i2146 < _size2142; ++_i2146)
+            uint32_t _size2150;
+            ::apache::thrift::protocol::TType _etype2153;
+            xfer += iprot->readListBegin(_etype2153, _size2150);
+            this->part_vals.resize(_size2150);
+            uint32_t _i2154;
+            for (_i2154 = 0; _i2154 < _size2150; ++_i2154)
             {
-              xfer += iprot->readString(this->part_vals[_i2146]);
+              xfer += iprot->readString(this->part_vals[_i2154]);
             }
             xfer += iprot->readListEnd();
           }
@@ -23018,10 +23018,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_args::write(::apache::thrift::pro
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter2147;
-    for (_iter2147 = this->part_vals.begin(); _iter2147 != this->part_vals.end(); ++_iter2147)
+    std::vector<std::string> ::const_iterator _iter2155;
+    for (_iter2155 = this->part_vals.begin(); _iter2155 != this->part_vals.end(); ++_iter2155)
     {
-      xfer += oprot->writeString((*_iter2147));
+      xfer += oprot->writeString((*_iter2155));
     }
     xfer += oprot->writeListEnd();
   }
@@ -23057,10 +23057,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_pargs::write(::apache::thrift::pr
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter2148;
-    for (_iter2148 = (*(this->part_vals)).begin(); _iter2148 != (*(this->part_vals)).end(); ++_iter2148)
+    std::vector<std::string> ::const_iterator _iter2156;
+    for (_iter2156 = (*(this->part_vals)).begin(); _iter2156 != (*(this->part_vals)).end(); ++_iter2156)
     {
-      xfer += oprot->writeString((*_iter2148));
+      xfer += oprot->writeString((*_iter2156));
     }
     xfer += oprot->writeListEnd();
   }
@@ -23105,14 +23105,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2149;
-            ::apache::thrift::protocol::TType _etype2152;
-            xfer += iprot->readListBegin(_etype2152, _size2149);
-            this->success.resize(_size2149);
-            uint32_t _i2153;
-            for (_i2153 = 0; _i2153 < _size2149; ++_i2153)
+            uint32_t _size2157;
+            ::apache::thrift::protocol::TType _etype2160;
+            xfer += iprot->readListBegin(_etype2160, _size2157);
+            this->success.resize(_size2157);
+            uint32_t _i2161;
+            for (_i2161 = 0; _i2161 < _size2157; ++_i2161)
             {
-              xfer += this->success[_i2153].read(iprot);
+              xfer += this->success[_i2161].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -23159,10 +23159,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_result::write(::apache::thrift::p
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter2154;
-      for (_iter2154 = this->success.begin(); _iter2154 != this->success.end(); ++_iter2154)
+      std::vector<Partition> ::const_iterator _iter2162;
+      for (_iter2162 = this->success.begin(); _iter2162 != this->success.end(); ++_iter2162)
       {
-        xfer += (*_iter2154).write(oprot);
+        xfer += (*_iter2162).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -23211,14 +23211,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2155;
-            ::apache::thrift::protocol::TType _etype2158;
-            xfer += iprot->readListBegin(_etype2158, _size2155);
-            (*(this->success)).resize(_size2155);
-            uint32_t _i2159;
-            for (_i2159 = 0; _i2159 < _size2155; ++_i2159)
+            uint32_t _size2163;
+            ::apache::thrift::protocol::TType _etype2166;
+            xfer += iprot->readListBegin(_etype2166, _size2163);
+            (*(this->success)).resize(_size2163);
+            uint32_t _i2167;
+            for (_i2167 = 0; _i2167 < _size2163; ++_i2167)
             {
-              xfer += (*(this->success))[_i2159].read(iprot);
+              xfer += (*(this->success))[_i2167].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -23301,14 +23301,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size2160;
-            ::apache::thrift::protocol::TType _etype2163;
-            xfer += iprot->readListBegin(_etype2163, _size2160);
-            this->part_vals.resize(_size2160);
-            uint32_t _i2164;
-            for (_i2164 = 0; _i2164 < _size2160; ++_i2164)
+            uint32_t _size2168;
+            ::apache::thrift::protocol::TType _etype2171;
+            xfer += iprot->readListBegin(_etype2171, _size2168);
+            this->part_vals.resize(_size2168);
+            uint32_t _i2172;
+            for (_i2172 = 0; _i2172 < _size2168; ++_i2172)
             {
-              xfer += iprot->readString(this->part_vals[_i2164]);
+              xfer += iprot->readString(this->part_vals[_i2172]);
             }
             xfer += iprot->readListEnd();
           }
@@ -23337,14 +23337,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size2165;
-            ::apache::thrift::protocol::TType _etype2168;
-            xfer += iprot->readListBegin(_etype2168, _size2165);
-            this->group_names.resize(_size2165);
-            uint32_t _i2169;
-            for (_i2169 = 0; _i2169 < _size2165; ++_i2169)
+            uint32_t _size2173;
+            ::apache::thrift::protocol::TType _etype2176;
+            xfer += iprot->readListBegin(_etype2176, _size2173);
+            this->group_names.resize(_size2173);
+            uint32_t _i2177;
+            for (_i2177 = 0; _i2177 < _size2173; ++_i2177)
             {
-              xfer += iprot->readString(this->group_names[_i2169]);
+              xfer += iprot->readString(this->group_names[_i2177]);
             }
             xfer += iprot->readListEnd();
           }
@@ -23381,10 +23381,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::write(::apache::t
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter2170;
-    for (_iter2170 = this->part_vals.begin(); _iter2170 != this->part_vals.end(); ++_iter2170)
+    std::vector<std::string> ::const_iterator _iter2178;
+    for (_iter2178 = this->part_vals.begin(); _iter2178 != this->part_vals.end(); ++_iter2178)
     {
-      xfer += oprot->writeString((*_iter2170));
+      xfer += oprot->writeString((*_iter2178));
     }
     xfer += oprot->writeListEnd();
   }
@@ -23401,10 +23401,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::write(::apache::t
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 6);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter2171;
-    for (_iter2171 = this->group_names.begin(); _iter2171 != this->group_names.end(); ++_iter2171)
+    std::vector<std::string> ::const_iterator _iter2179;
+    for (_iter2179 = this->group_names.begin(); _iter2179 != this->group_names.end(); ++_iter2179)
     {
-      xfer += oprot->writeString((*_iter2171));
+      xfer += oprot->writeString((*_iter2179));
     }
     xfer += oprot->writeListEnd();
   }
@@ -23436,10 +23436,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_pargs::write(::apache::
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter2172;
-    for (_iter2172 = (*(this->part_vals)).begin(); _iter2172 != (*(this->part_vals)).end(); ++_iter2172)
+    std::vector<std::string> ::const_iterator _iter2180;
+    for (_iter2180 = (*(this->part_vals)).begin(); _iter2180 != (*(this->part_vals)).end(); ++_iter2180)
     {
-      xfer += oprot->writeString((*_iter2172));
+      xfer += oprot->writeString((*_iter2180));
     }
     xfer += oprot->writeListEnd();
   }
@@ -23456,10 +23456,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_pargs::write(::apache::
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 6);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter2173;
-    for (_iter2173 = (*(this->group_names)).begin(); _iter2173 != (*(this->group_names)).end(); ++_iter2173)
+    std::vector<std::string> ::const_iterator _iter2181;
+    for (_iter2181 = (*(this->group_names)).begin(); _iter2181 != (*(this->group_names)).end(); ++_iter2181)
     {
-      xfer += oprot->writeString((*_iter2173));
+      xfer += oprot->writeString((*_iter2181));
     }
     xfer += oprot->writeListEnd();
   }
@@ -23500,14 +23500,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_result::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2174;
-            ::apache::thrift::protocol::TType _etype2177;
-            xfer += iprot->readListBegin(_etype2177, _size2174);
-            this->success.resize(_size2174);
-            uint32_t _i2178;
-            for (_i2178 = 0; _i2178 < _size2174; ++_i2178)
+            uint32_t _size2182;
+            ::apache::thrift::protocol::TType _etype2185;
+            xfer += iprot->readListBegin(_etype2185, _size2182);
+            this->success.resize(_size2182);
+            uint32_t _i2186;
+            for (_i2186 = 0; _i2186 < _size2182; ++_i2186)
             {
-              xfer += this->success[_i2178].read(iprot);
+              xfer += this->success[_i2186].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -23554,10 +23554,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_result::write(::apache:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter2179;
-      for (_iter2179 = this->success.begin(); _iter2179 != this->success.end(); ++_iter2179)
+      std::vector<Partition> ::const_iterator _iter2187;
+      for (_iter2187 = this->success.begin(); _iter2187 != this->success.end(); ++_iter2187)
       {
-        xfer += (*_iter2179).write(oprot);
+        xfer += (*_iter2187).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -23606,14 +23606,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_presult::read(::apache:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2180;
-            ::apache::thrift::protocol::TType _etype2183;
-            xfer += iprot->readListBegin(_etype2183, _size2180);
-            (*(this->success)).resize(_size2180);
-            uint32_t _i2184;
-            for (_i2184 = 0; _i2184 < _size2180; ++_i2184)
+            uint32_t _size2188;
+            ::apache::thrift::protocol::TType _etype2191;
+            xfer += iprot->readListBegin(_etype2191, _size2188);
+            (*(this->success)).resize(_size2188);
+            uint32_t _i2192;
+            for (_i2192 = 0; _i2192 < _size2188; ++_i2192)
             {
-              xfer += (*(this->success))[_i2184].read(iprot);
+              xfer += (*(this->success))[_i2192].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -23923,14 +23923,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_args::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size2185;
-            ::apache::thrift::protocol::TType _etype2188;
-            xfer += iprot->readListBegin(_etype2188, _size2185);
-            this->part_vals.resize(_size2185);
-            uint32_t _i2189;
-            for (_i2189 = 0; _i2189 < _size2185; ++_i2189)
+            uint32_t _size2193;
+            ::apache::thrift::protocol::TType _etype2196;
+            xfer += iprot->readListBegin(_etype2196, _size2193);
+            this->part_vals.resize(_size2193);
+            uint32_t _i2197;
+            for (_i2197 = 0; _i2197 < _size2193; ++_i2197)
             {
-              xfer += iprot->readString(this->part_vals[_i2189]);
+              xfer += iprot->readString(this->part_vals[_i2197]);
             }
             xfer += iprot->readListEnd();
           }
@@ -23975,10 +23975,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_args::write(::apache::thrift
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter2190;
-    for (_iter2190 = this->part_vals.begin(); _iter2190 != this->part_vals.end(); ++_iter2190)
+    std::vector<std::string> ::const_iterator _iter2198;
+    for (_iter2198 = this->part_vals.begin(); _iter2198 != this->part_vals.end(); ++_iter2198)
     {
-      xfer += oprot->writeString((*_iter2190));
+      xfer += oprot->writeString((*_iter2198));
     }
     xfer += oprot->writeListEnd();
   }
@@ -24014,10 +24014,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_pargs::write(::apache::thrif
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter2191;
-    for (_iter2191 = (*(this->part_vals)).begin(); _iter2191 != (*(this->part_vals)).end(); ++_iter2191)
+    std::vector<std::string> ::const_iterator _iter2199;
+    for (_iter2199 = (*(this->part_vals)).begin(); _iter2199 != (*(this->part_vals)).end(); ++_iter2199)
     {
-      xfer += oprot->writeString((*_iter2191));
+      xfer += oprot->writeString((*_iter2199));
     }
     xfer += oprot->writeListEnd();
   }
@@ -24062,14 +24062,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_result::read(::apache::thrif
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2192;
-            ::apache::thrift::protocol::TType _etype2195;
-            xfer += iprot->readListBegin(_etype2195, _size2192);
-            this->success.resize(_size2192);
-            uint32_t _i2196;
-            for (_i2196 = 0; _i2196 < _size2192; ++_i2196)
+            uint32_t _size2200;
+            ::apache::thrift::protocol::TType _etype2203;
+            xfer += iprot->readListBegin(_etype2203, _size2200);
+            this->success.resize(_size2200);
+            uint32_t _i2204;
+            for (_i2204 = 0; _i2204 < _size2200; ++_i2204)
             {
-              xfer += iprot->readString(this->success[_i2196]);
+              xfer += iprot->readString(this->success[_i2204]);
             }
             xfer += iprot->readListEnd();
           }
@@ -24116,10 +24116,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_result::write(::apache::thri
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter2197;
-      for (_iter2197 = this->success.begin(); _iter2197 != this->success.end(); ++_iter2197)
+      std::vector<std::string> ::const_iterator _iter2205;
+      for (_iter2205 = this->success.begin(); _iter2205 != this->success.end(); ++_iter2205)
       {
-        xfer += oprot->writeString((*_iter2197));
+        xfer += oprot->writeString((*_iter2205));
       }
       xfer += oprot->writeListEnd();
     }
@@ -24168,14 +24168,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_presult::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2198;
-            ::apache::thrift::protocol::TType _etype2201;
-            xfer += iprot->readListBegin(_etype2201, _size2198);
-            (*(this->success)).resize(_size2198);
-            uint32_t _i2202;
-            for (_i2202 = 0; _i2202 < _size2198; ++_i2202)
+            uint32_t _size2206;
+            ::apache::thrift::protocol::TType _etype2209;
+            xfer += iprot->readListBegin(_etype2209, _size2206);
+            (*(this->success)).resize(_size2206);
+            uint32_t _i2210;
+            for (_i2210 = 0; _i2210 < _size2206; ++_i2210)
             {
-              xfer += iprot->readString((*(this->success))[_i2202]);
+              xfer += iprot->readString((*(this->success))[_i2210]);
             }
             xfer += iprot->readListEnd();
           }
@@ -24548,14 +24548,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_req_result::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2203;
-            ::apache::thrift::protocol::TType _etype2206;
-            xfer += iprot->readListBegin(_etype2206, _size2203);
-            this->success.resize(_size2203);
-            uint32_t _i2207;
-            for (_i2207 = 0; _i2207 < _size2203; ++_i2207)
+            uint32_t _size2211;
+            ::apache::thrift::protocol::TType _etype2214;
+            xfer += iprot->readListBegin(_etype2214, _size2211);
+            this->success.resize(_size2211);
+            uint32_t _i2215;
+            for (_i2215 = 0; _i2215 < _size2211; ++_i2215)
             {
-              xfer += iprot->readString(this->success[_i2207]);
+              xfer += iprot->readString(this->success[_i2215]);
             }
             xfer += iprot->readListEnd();
           }
@@ -24602,10 +24602,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_req_result::write(::apache::thr
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter2208;
-      for (_iter2208 = this->success.begin(); _iter2208 != this->success.end(); ++_iter2208)
+      std::vector<std::string> ::const_iterator _iter2216;
+      for (_iter2216 = this->success.begin(); _iter2216 != this->success.end(); ++_iter2216)
       {
-        xfer += oprot->writeString((*_iter2208));
+        xfer += oprot->writeString((*_iter2216));
       }
       xfer += oprot->writeListEnd();
     }
@@ -24654,14 +24654,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_req_presult::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2209;
-            ::apache::thrift::protocol::TType _etype2212;
-            xfer += iprot->readListBegin(_etype2212, _size2209);
-            (*(this->success)).resize(_size2209);
-            uint32_t _i2213;
-            for (_i2213 = 0; _i2213 < _size2209; ++_i2213)
+            uint32_t _size2217;
+            ::apache::thrift::protocol::TType _etype2220;
+            xfer += iprot->readListBegin(_etype2220, _size2217);
+            (*(this->success)).resize(_size2217);
+            uint32_t _i2221;
+            for (_i2221 = 0; _i2221 < _size2217; ++_i2221)
             {
-              xfer += iprot->readString((*(this->success))[_i2213]);
+              xfer += iprot->readString((*(this->success))[_i2221]);
             }
             xfer += iprot->readListEnd();
           }
@@ -24855,14 +24855,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_filter_result::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2214;
-            ::apache::thrift::protocol::TType _etype2217;
-            xfer += iprot->readListBegin(_etype2217, _size2214);
-            this->success.resize(_size2214);
-            uint32_t _i2218;
-            for (_i2218 = 0; _i2218 < _size2214; ++_i2218)
+            uint32_t _size2222;
+            ::apache::thrift::protocol::TType _etype2225;
+            xfer += iprot->readListBegin(_etype2225, _size2222);
+            this->success.resize(_size2222);
+            uint32_t _i2226;
+            for (_i2226 = 0; _i2226 < _size2222; ++_i2226)
             {
-              xfer += this->success[_i2218].read(iprot);
+              xfer += this->success[_i2226].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -24909,10 +24909,10 @@ uint32_t ThriftHiveMetastore_get_partitions_by_filter_result::write(::apache::th
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter2219;
-      for (_iter2219 = this->success.begin(); _iter2219 != this->success.end(); ++_iter2219)
+      std::vector<Partition> ::const_iterator _iter2227;
+      for (_iter2227 = this->success.begin(); _iter2227 != this->success.end(); ++_iter2227)
       {
-        xfer += (*_iter2219).write(oprot);
+        xfer += (*_iter2227).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -24961,14 +24961,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_filter_presult::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2220;
-            ::apache::thrift::protocol::TType _etype2223;
-            xfer += iprot->readListBegin(_etype2223, _size2220);
-            (*(this->success)).resize(_size2220);
-            uint32_t _i2224;
-            for (_i2224 = 0; _i2224 < _size2220; ++_i2224)
+            uint32_t _size2228;
+            ::apache::thrift::protocol::TType _etype2231;
+            xfer += iprot->readListBegin(_etype2231, _size2228);
+            (*(this->success)).resize(_size2228);
+            uint32_t _i2232;
+            for (_i2232 = 0; _i2232 < _size2228; ++_i2232)
             {
-              xfer += (*(this->success))[_i2224].read(iprot);
+              xfer += (*(this->success))[_i2232].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -25162,14 +25162,14 @@ uint32_t ThriftHiveMetastore_get_part_specs_by_filter_result::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2225;
-            ::apache::thrift::protocol::TType _etype2228;
-            xfer += iprot->readListBegin(_etype2228, _size2225);
-            this->success.resize(_size2225);
-            uint32_t _i2229;
-            for (_i2229 = 0; _i2229 < _size2225; ++_i2229)
+            uint32_t _size2233;
+            ::apache::thrift::protocol::TType _etype2236;
+            xfer += iprot->readListBegin(_etype2236, _size2233);
+            this->success.resize(_size2233);
+            uint32_t _i2237;
+            for (_i2237 = 0; _i2237 < _size2233; ++_i2237)
             {
-              xfer += this->success[_i2229].read(iprot);
+              xfer += this->success[_i2237].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -25216,10 +25216,10 @@ uint32_t ThriftHiveMetastore_get_part_specs_by_filter_result::write(::apache::th
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<PartitionSpec> ::const_iterator _iter2230;
-      for (_iter2230 = this->success.begin(); _iter2230 != this->success.end(); ++_iter2230)
+      std::vector<PartitionSpec> ::const_iterator _iter2238;
+      for (_iter2238 = this->success.begin(); _iter2238 != this->success.end(); ++_iter2238)
       {
-        xfer += (*_iter2230).write(oprot);
+        xfer += (*_iter2238).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -25268,14 +25268,14 @@ uint32_t ThriftHiveMetastore_get_part_specs_by_filter_presult::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2231;
-            ::apache::thrift::protocol::TType _etype2234;
-            xfer += iprot->readListBegin(_etype2234, _size2231);
-            (*(this->success)).resize(_size2231);
-            uint32_t _i2235;
-            for (_i2235 = 0; _i2235 < _size2231; ++_i2235)
+            uint32_t _size2239;
+            ::apache::thrift::protocol::TType _etype2242;
+            xfer += iprot->readListBegin(_etype2242, _size2239);
+            (*(this->success)).resize(_size2239);
+            uint32_t _i2243;
+            for (_i2243 = 0; _i2243 < _size2239; ++_i2243)
             {
-              xfer += (*(this->success))[_i2235].read(iprot);
+              xfer += (*(this->success))[_i2243].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -26071,14 +26071,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_args::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->names.clear();
-            uint32_t _size2236;
-            ::apache::thrift::protocol::TType _etype2239;
-            xfer += iprot->readListBegin(_etype2239, _size2236);
-            this->names.resize(_size2236);
-            uint32_t _i2240;
-            for (_i2240 = 0; _i2240 < _size2236; ++_i2240)
+            uint32_t _size2244;
+            ::apache::thrift::protocol::TType _etype2247;
+            xfer += iprot->readListBegin(_etype2247, _size2244);
+            this->names.resize(_size2244);
+            uint32_t _i2248;
+            for (_i2248 = 0; _i2248 < _size2244; ++_i2248)
             {
-              xfer += iprot->readString(this->names[_i2240]);
+              xfer += iprot->readString(this->names[_i2248]);
             }
             xfer += iprot->readListEnd();
           }
@@ -26115,10 +26115,10 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_args::write(::apache::thrif
   xfer += oprot->writeFieldBegin("names", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->names.size()));
-    std::vector<std::string> ::const_iterator _iter2241;
-    for (_iter2241 = this->names.begin(); _iter2241 != this->names.end(); ++_iter2241)
+    std::vector<std::string> ::const_iterator _iter2249;
+    for (_iter2249 = this->names.begin(); _iter2249 != this->names.end(); ++_iter2249)
     {
-      xfer += oprot->writeString((*_iter2241));
+      xfer += oprot->writeString((*_iter2249));
     }
     xfer += oprot->writeListEnd();
   }
@@ -26150,10 +26150,10 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_pargs::write(::apache::thri
   xfer += oprot->writeFieldBegin("names", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->names)).size()));
-    std::vector<std::string> ::const_iterator _iter2242;
-    for (_iter2242 = (*(this->names)).begin(); _iter2242 != (*(this->names)).end(); ++_iter2242)
+    std::vector<std::string> ::const_iterator _iter2250;
+    for (_iter2250 = (*(this->names)).begin(); _iter2250 != (*(this->names)).end(); ++_iter2250)
     {
-      xfer += oprot->writeString((*_iter2242));
+      xfer += oprot->writeString((*_iter2250));
     }
     xfer += oprot->writeListEnd();
   }
@@ -26194,14 +26194,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_result::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2243;
-            ::apache::thrift::protocol::TType _etype2246;
-            xfer += iprot->readListBegin(_etype2246, _size2243);
-            this->success.resize(_size2243);
-            uint32_t _i2247;
-            for (_i2247 = 0; _i2247 < _size2243; ++_i2247)
+            uint32_t _size2251;
+            ::apache::thrift::protocol::TType _etype2254;
+            xfer += iprot->readListBegin(_etype2254, _size2251);
+            this->success.resize(_size2251);
+            uint32_t _i2255;
+            for (_i2255 = 0; _i2255 < _size2251; ++_i2255)
             {
-              xfer += this->success[_i2247].read(iprot);
+              xfer += this->success[_i2255].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -26248,10 +26248,10 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_result::write(::apache::thr
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter2248;
-      for (_iter2248 = this->success.begin(); _iter2248 != this->success.end(); ++_iter2248)
+      std::vector<Partition> ::const_iterator _iter2256;
+      for (_iter2256 = this->success.begin(); _iter2256 != this->success.end(); ++_iter2256)
       {
-        xfer += (*_iter2248).write(oprot);
+        xfer += (*_iter2256).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -26300,14 +26300,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_presult::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2249;
-            ::apache::thrift::protocol::TType _etype2252;
-            xfer += iprot->readListBegin(_etype2252, _size2249);
-            (*(this->success)).resize(_size2249);
-            uint32_t _i2253;
-            for (_i2253 = 0; _i2253 < _size2249; ++_i2253)
+            uint32_t _size2257;
+            ::apache::thrift::protocol::TType _etype2260;
+            xfer += iprot->readListBegin(_etype2260, _size2257);
+            (*(this->success)).resize(_size2257);
+            uint32_t _i2261;
+            for (_i2261 = 0; _i2261 < _size2257; ++_i2261)
             {
-              xfer += (*(this->success))[_i2253].read(iprot);
+              xfer += (*(this->success))[_i2261].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -26856,14 +26856,14 @@ uint32_t ThriftHiveMetastore_alter_partitions_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size2254;
-            ::apache::thrift::protocol::TType _etype2257;
-            xfer += iprot->readListBegin(_etype2257, _size2254);
-            this->new_parts.resize(_size2254);
-            uint32_t _i2258;
-            for (_i2258 = 0; _i2258 < _size2254; ++_i2258)
+            uint32_t _size2262;
+            ::apache::thrift::protocol::TType _etype2265;
+            xfer += iprot->readListBegin(_etype2265, _size2262);
+            this->new_parts.resize(_size2262);
+            uint32_t _i2266;
+            for (_i2266 = 0; _i2266 < _size2262; ++_i2266)
             {
-              xfer += this->new_parts[_i2258].read(iprot);
+              xfer += this->new_parts[_i2266].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -26900,10 +26900,10 @@ uint32_t ThriftHiveMetastore_alter_partitions_args::write(::apache::thrift::prot
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->new_parts.size()));
-    std::vector<Partition> ::const_iterator _iter2259;
-    for (_iter2259 = this->new_parts.begin(); _iter2259 != this->new_parts.end(); ++_iter2259)
+    std::vector<Partition> ::const_iterator _iter2267;
+    for (_iter2267 = this->new_parts.begin(); _iter2267 != this->new_parts.end(); ++_iter2267)
     {
-      xfer += (*_iter2259).write(oprot);
+      xfer += (*_iter2267).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -26935,10 +26935,10 @@ uint32_t ThriftHiveMetastore_alter_partitions_pargs::write(::apache::thrift::pro
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->new_parts)).size()));
-    std::vector<Partition> ::const_iterator _iter2260;
-    for (_iter2260 = (*(this->new_parts)).begin(); _iter2260 != (*(this->new_parts)).end(); ++_iter2260)
+    std::vector<Partition> ::const_iterator _iter2268;
+    for (_iter2268 = (*(this->new_parts)).begin(); _iter2268 != (*(this->new_parts)).end(); ++_iter2268)
     {
-      xfer += (*_iter2260).write(oprot);
+      xfer += (*_iter2268).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -27123,14 +27123,14 @@ uint32_t ThriftHiveMetastore_alter_partitions_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size2261;
-            ::apache::thrift::protocol::TType _etype2264;
-            xfer += iprot->readListBegin(_etype2264, _size2261);
-            this->new_parts.resize(_size2261);
-            uint32_t _i2265;
-            for (_i2265 = 0; _i2265 < _size2261; ++_i2265)
+            uint32_t _size2269;
+            ::apache::thrift::protocol::TType _etype2272;
+            xfer += iprot->readListBegin(_etype2272, _size2269);
+            this->new_parts.resize(_size2269);
+            uint32_t _i2273;
+            for (_i2273 = 0; _i2273 < _size2269; ++_i2273)
             {
-              xfer += this->new_parts[_i2265].read(iprot);
+              xfer += this->new_parts[_i2273].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -27175,10 +27175,10 @@ uint32_t ThriftHiveMetastore_alter_partitions_with_environment_context_args::wri
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->new_parts.size()));
-    std::vector<Partition> ::const_iterator _iter2266;
-    for (_iter2266 = this->new_parts.begin(); _iter2266 != this->new_parts.end(); ++_iter2266)
+    std::vector<Partition> ::const_iterator _iter2274;
+    for (_iter2274 = this->new_parts.begin(); _iter2274 != this->new_parts.end(); ++_iter2274)
     {
-      xfer += (*_iter2266).write(oprot);
+      xfer += (*_iter2274).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -27214,10 +27214,10 @@ uint32_t ThriftHiveMetastore_alter_partitions_with_environment_context_pargs::wr
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->new_parts)).size()));
-    std::vector<Partition> ::const_iterator _iter2267;
-    for (_iter2267 = (*(this->new_parts)).begin(); _iter2267 != (*(this->new_parts)).end(); ++_iter2267)
+    std::vector<Partition> ::const_iterator _iter2275;
+    for (_iter2275 = (*(this->new_parts)).begin(); _iter2275 != (*(this->new_parts)).end(); ++_iter2275)
     {
-      xfer += (*_iter2267).write(oprot);
+      xfer += (*_iter2275).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -27888,14 +27888,14 @@ uint32_t ThriftHiveMetastore_rename_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size2268;
-            ::apache::thrift::protocol::TType _etype2271;
-            xfer += iprot->readListBegin(_etype2271, _size2268);
-            this->part_vals.resize(_size2268);
-            uint32_t _i2272;
-            for (_i2272 = 0; _i2272 < _size2268; ++_i2272)
+            uint32_t _size2276;
+            ::apache::thrift::protocol::TType _etype2279;
+            xfer += iprot->readListBegin(_etype2279, _size2276);
+            this->part_vals.resize(_size2276);
+            uint32_t _i2280;
+            for (_i2280 = 0; _i2280 < _size2276; ++_i2280)
             {
-              xfer += iprot->readString(this->part_vals[_i2272]);
+              xfer += iprot->readString(this->part_vals[_i2280]);
             }
             xfer += iprot->readListEnd();
           }
@@ -27940,10 +27940,10 @@ uint32_t ThriftHiveMetastore_rename_partition_args::write(::apache::thrift::prot
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter2273;
-    for (_iter2273 = this->part_vals.begin(); _iter2273 != this->part_vals.end(); ++_iter2273)
+    std::vector<std::string> ::const_iterator _iter2281;
+    for (_iter2281 = this->part_vals.begin(); _iter2281 != this->part_vals.end(); ++_iter2281)
     {
-      xfer += oprot->writeString((*_iter2273));
+      xfer += oprot->writeString((*_iter2281));
     }
     xfer += oprot->writeListEnd();
   }
@@ -27979,10 +27979,10 @@ uint32_t ThriftHiveMetastore_rename_partition_pargs::write(::apache::thrift::pro
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter2274;
-    for (_iter2274 = (*(this->part_vals)).begin(); _iter2274 != (*(this->part_vals)).end(); ++_iter2274)
+    std::vector<std::string> ::const_iterator _iter2282;
+    for (_iter2282 = (*(this->part_vals)).begin(); _iter2282 != (*(this->part_vals)).end(); ++_iter2282)
     {
-      xfer += oprot->writeString((*_iter2274));
+      xfer += oprot->writeString((*_iter2282));
     }
     xfer += oprot->writeListEnd();
   }
@@ -28382,14 +28382,14 @@ uint32_t ThriftHiveMetastore_partition_name_has_valid_characters_args::read(::ap
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size2275;
-            ::apache::thrift::protocol::TType _etype2278;
-            xfer += iprot->readListBegin(_etype2278, _size2275);
-            this->part_vals.resize(_size2275);
-            uint32_t _i2279;
-            for (_i2279 = 0; _i2279 < _size2275; ++_i2279)
+            uint32_t _size2283;
+            ::apache::thrift::protocol::TType _etype2286;
+            xfer += iprot->readListBegin(_etype2286, _size2283);
+            this->part_vals.resize(_size2283);
+            uint32_t _i2287;
+            for (_i2287 = 0; _i2287 < _size2283; ++_i2287)
             {
-              xfer += iprot->readString(this->part_vals[_i2279]);
+              xfer += iprot->readString(this->part_vals[_i2287]);
             }
             xfer += iprot->readListEnd();
           }
@@ -28426,10 +28426,10 @@ uint32_t ThriftHiveMetastore_partition_name_has_valid_characters_args::write(::a
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter2280;
-    for (_iter2280 = this->part_vals.begin(); _iter2280 != this->part_vals.end(); ++_iter2280)
+    std::vector<std::string> ::const_iterator _iter2288;
+    for (_iter2288 = this->part_vals.begin(); _iter2288 != this->part_vals.end(); ++_iter2288)
     {
-      xfer += oprot->writeString((*_iter2280));
+      xfer += oprot->writeString((*_iter2288));
     }
     xfer += oprot->writeListEnd();
   }
@@ -28457,10 +28457,10 @@ uint32_t ThriftHiveMetastore_partition_name_has_valid_characters_pargs::write(::
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter2281;
-    for (_iter2281 = (*(this->part_vals)).begin(); _iter2281 != (*(this->part_vals)).end(); ++_iter2281)
+    std::vector<std::string> ::const_iterator _iter2289;
+    for (_iter2289 = (*(this->part_vals)).begin(); _iter2289 != (*(this->part_vals)).end(); ++_iter2289)
     {
-      xfer += oprot->writeString((*_iter2281));
+      xfer += oprot->writeString((*_iter2289));
     }
     xfer += oprot->writeListEnd();
   }
@@ -28935,14 +28935,14 @@ uint32_t ThriftHiveMetastore_partition_name_to_vals_result::read(::apache::thrif
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2282;
-            ::apache::thrift::protocol::TType _etype2285;
-            xfer += iprot->readListBegin(_etype2285, _size2282);
-            this->success.resize(_size2282);
-            uint32_t _i2286;
-            for (_i2286 = 0; _i2286 < _size2282; ++_i2286)
+            uint32_t _size2290;
+            ::apache::thrift::protocol::TType _etype2293;
+            xfer += iprot->readListBegin(_etype2293, _size2290);
+            this->success.resize(_size2290);
+            uint32_t _i2294;
+            for (_i2294 = 0; _i2294 < _size2290; ++_i2294)
             {
-              xfer += iprot->readString(this->success[_i2286]);
+              xfer += iprot->readString(this->success[_i2294]);
             }
             xfer += iprot->readListEnd();
           }
@@ -28981,10 +28981,10 @@ uint32_t ThriftHiveMetastore_partition_name_to_vals_result::write(::apache::thri
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter2287;
-      for (_iter2287 = this->success.begin(); _iter2287 != this->success.end(); ++_iter2287)
+      std::vector<std::string> ::const_iterator _iter2295;
+      for (_iter2295 = this->success.begin(); _iter2295 != this->success.end(); ++_iter2295)
       {
-        xfer += oprot->writeString((*_iter2287));
+        xfer += oprot->writeString((*_iter2295));
       }
       xfer += oprot->writeListEnd();
     }
@@ -29029,14 +29029,14 @@ uint32_t ThriftHiveMetastore_partition_name_to_vals_presult::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2288;
-            ::apache::thrift::protocol::TType _etype2291;
-            xfer += iprot->readListBegin(_etype2291, _size2288);
-            (*(this->success)).resize(_size2288);
-            uint32_t _i2292;
-            for (_i2292 = 0; _i2292 < _size2288; ++_i2292)
+            uint32_t _size2296;
+            ::apache::thrift::protocol::TType _etype2299;
+            xfer += iprot->readListBegin(_etype2299, _size2296);
+            (*(this->success)).resize(_size2296);
+            uint32_t _i2300;
+            for (_i2300 = 0; _i2300 < _size2296; ++_i2300)
             {
-              xfer += iprot->readString((*(this->success))[_i2292]);
+              xfer += iprot->readString((*(this->success))[_i2300]);
             }
             xfer += iprot->readListEnd();
           }
@@ -29174,17 +29174,17 @@ uint32_t ThriftHiveMetastore_partition_name_to_spec_result::read(::apache::thrif
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size2293;
-            ::apache::thrift::protocol::TType _ktype2294;
-            ::apache::thrift::protocol::TType _vtype2295;
-            xfer += iprot->readMapBegin(_ktype2294, _vtype2295, _size2293);
-            uint32_t _i2297;
-            for (_i2297 = 0; _i2297 < _size2293; ++_i2297)
+            uint32_t _size2301;
+            ::apache::thrift::protocol::TType _ktype2302;
+            ::apache::thrift::protocol::TType _vtype2303;
+            xfer += iprot->readMapBegin(_ktype2302, _vtype2303, _size2301);
+            uint32_t _i2305;
+            for (_i2305 = 0; _i2305 < _size2301; ++_i2305)
             {
-              std::string _key2298;
-              xfer += iprot->readString(_key2298);
-              std::string& _val2299 = this->success[_key2298];
-              xfer += iprot->readString(_val2299);
+              std::string _key2306;
+              xfer += iprot->readString(_key2306);
+              std::string& _val2307 = this->success[_key2306];
+              xfer += iprot->readString(_val2307);
             }
             xfer += iprot->readMapEnd();
           }
@@ -29223,11 +29223,11 @@ uint32_t ThriftHiveMetastore_partition_name_to_spec_result::write(::apache::thri
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::map<std::string, std::string> ::const_iterator _iter2300;
-      for (_iter2300 = this->success.begin(); _iter2300 != this->success.end(); ++_iter2300)
+      std::map<std::string, std::string> ::const_iterator _iter2308;
+      for (_iter2308 = this->success.begin(); _iter2308 != this->success.end(); ++_iter2308)
       {
-        xfer += oprot->writeString(_iter2300->first);
-        xfer += oprot->writeString(_iter2300->second);
+        xfer += oprot->writeString(_iter2308->first);
+        xfer += oprot->writeString(_iter2308->second);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -29272,17 +29272,17 @@ uint32_t ThriftHiveMetastore_partition_name_to_spec_presult::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size2301;
-            ::apache::thrift::protocol::TType _ktype2302;
-            ::apache::thrift::protocol::TType _vtype2303;
-            xfer += iprot->readMapBegin(_ktype2302, _vtype2303, _size2301);
-            uint32_t _i2305;
-            for (_i2305 = 0; _i2305 < _size2301; ++_i2305)
+            uint32_t _size2309;
+            ::apache::thrift::protocol::TType _ktype2310;
+            ::apache::thrift::protocol::TType _vtype2311;
+            xfer += iprot->readMapBegin(_ktype2310, _vtype2311, _size2309);
+            uint32_t _i2313;
+            for (_i2313 = 0; _i2313 < _size2309; ++_i2313)
             {
-              std::string _key2306;
-              xfer += iprot->readString(_key2306);
-              std::string& _val2307 = (*(this->success))[_key2306];
-              xfer += iprot->readString(_val2307);
+              std::string _key2314;
+              xfer += iprot->readString(_key2314);
+              std::string& _val2315 = (*(this->success))[_key2314];
+              xfer += iprot->readString(_val2315);
             }
             xfer += iprot->readMapEnd();
           }
@@ -29357,17 +29357,17 @@ uint32_t ThriftHiveMetastore_markPartitionForEvent_args::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->part_vals.clear();
-            uint32_t _size2308;
-            ::apache::thrift::protocol::TType _ktype2309;
-            ::apache::thrift::protocol::TType _vtype2310;
-            xfer += iprot->readMapBegin(_ktype2309, _vtype2310, _size2308);
-            uint32_t _i2312;
-            for (_i2312 = 0; _i2312 < _size2308; ++_i2312)
+            uint32_t _size2316;
+            ::apache::thrift::protocol::TType _ktype2317;
+            ::apache::thrift::protocol::TType _vtype2318;
+            xfer += iprot->readMapBegin(_ktype2317, _vtype2318, _size2316);
+            uint32_t _i2320;
+            for (_i2320 = 0; _i2320 < _size2316; ++_i2320)
             {
-              std::string _key2313;
-              xfer += iprot->readString(_key2313);
-              std::string& _val2314 = this->part_vals[_key2313];
-              xfer += iprot->readString(_val2314);
+              std::string _key2321;
+              xfer += iprot->readString(_key2321);
+              std::string& _val2322 = this->part_vals[_key2321];
+              xfer += iprot->readString(_val2322);
             }
             xfer += iprot->readMapEnd();
           }
@@ -29378,9 +29378,9 @@ uint32_t ThriftHiveMetastore_markPartitionForEvent_args::read(::apache::thrift::
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast2315;
-          xfer += iprot->readI32(ecast2315);
-          this->eventType = (PartitionEventType::type)ecast2315;
+          int32_t ecast2323;
+          xfer += iprot->readI32(ecast2323);
+          this->eventType = (PartitionEventType::type)ecast2323;
           this->__isset.eventType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -29414,11 +29414,11 @@ uint32_t ThriftHiveMetastore_markPartitionForEvent_args::write(::apache::thrift:
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_MAP, 3);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::map<std::string, std::string> ::const_iterator _iter2316;
-    for (_iter2316 = this->part_vals.begin(); _iter2316 != this->part_vals.end(); ++_iter2316)
+    std::map<std::string, std::string> ::const_iterator _iter2324;
+    for (_iter2324 = this->part_vals.begin(); _iter2324 != this->part_vals.end(); ++_iter2324)
     {
-      xfer += oprot->writeString(_iter2316->first);
-      xfer += oprot->writeString(_iter2316->second);
+      xfer += oprot->writeString(_iter2324->first);
+      xfer += oprot->writeString(_iter2324->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -29454,11 +29454,11 @@ uint32_t ThriftHiveMetastore_markPartitionForEvent_pargs::write(::apache::thrift
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_MAP, 3);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::map<std::string, std::string> ::const_iterator _iter2317;
-    for (_iter2317 = (*(this->part_vals)).begin(); _iter2317 != (*(this->part_vals)).end(); ++_iter2317)
+    std::map<std::string, std::string> ::const_iterator _iter2325;
+    for (_iter2325 = (*(this->part_vals)).begin(); _iter2325 != (*(this->part_vals)).end(); ++_iter2325)
     {
-      xfer += oprot->writeString(_iter2317->first);
-      xfer += oprot->writeString(_iter2317->second);
+      xfer += oprot->writeString(_iter2325->first);
+      xfer += oprot->writeString(_iter2325->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -29727,17 +29727,17 @@ uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_args::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->part_vals.clear();
-            uint32_t _size2318;
-            ::apache::thrift::protocol::TType _ktype2319;
-            ::apache::thrift::protocol::TType _vtype2320;
-            xfer += iprot->readMapBegin(_ktype2319, _vtype2320, _size2318);
-            uint32_t _i2322;
-            for (_i2322 = 0; _i2322 < _size2318; ++_i2322)
+            uint32_t _size2326;
+            ::apache::thrift::protocol::TType _ktype2327;
+            ::apache::thrift::protocol::TType _vtype2328;
+            xfer += iprot->readMapBegin(_ktype2327, _vtype2328, _size2326);
+            uint32_t _i2330;
+            for (_i2330 = 0; _i2330 < _size2326; ++_i2330)
             {
-              std::string _key2323;
-              xfer += iprot->readString(_key2323);
-              std::string& _val2324 = this->part_vals[_key2323];
-              xfer += iprot->readString(_val2324);
+              std::string _key2331;
+              xfer += iprot->readString(_key2331);
+              std::string& _val2332 = this->part_vals[_key2331];
+              xfer += iprot->readString(_val2332);
             }
             xfer += iprot->readMapEnd();
           }
@@ -29748,9 +29748,9 @@ uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_args::read(::apache::thri
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast2325;
-          xfer += iprot->readI32(ecast2325);
-          this->eventType = (PartitionEventType::type)ecast2325;
+          int32_t ecast2333;
+          xfer += iprot->readI32(ecast2333);
+          this->eventType = (PartitionEventType::type)ecast2333;
           this->__isset.eventType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -29784,11 +29784,11 @@ uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_args::write(::apache::thr
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_MAP, 3);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::map<std::string, std::string> ::const_iterator _iter2326;
-    for (_iter2326 = this->part_vals.begin(); _iter2326 != this->part_vals.end(); ++_iter2326)
+    std::map<std::string, std::string> ::const_iterator _iter2334;
+    for (_iter2334 = this->part_vals.begin(); _iter2334 != this->part_vals.end(); ++_iter2334)
     {
-      xfer += oprot->writeString(_iter2326->first);
-      xfer += oprot->writeString(_iter2326->second);
+      xfer += oprot->writeString(_iter2334->first);
+      xfer += oprot->writeString(_iter2334->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -29824,11 +29824,11 @@ uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_pargs::write(::apache::th
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_MAP, 3);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::map<std::string, std::string> ::const_iterator _iter2327;
-    for (_iter2327 = (*(this->part_vals)).begin(); _iter2327 != (*(this->part_vals)).end(); ++_iter2327)
+    std::map<std::string, std::string> ::const_iterator _iter2335;
+    for (_iter2335 = (*(this->part_vals)).begin(); _iter2335 != (*(this->part_vals)).end(); ++_iter2335)
     {
-      xfer += oprot->writeString(_iter2327->first);
-      xfer += oprot->writeString(_iter2327->second);
+      xfer += oprot->writeString(_iter2335->first);
+      xfer += oprot->writeString(_iter2335->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -35957,14 +35957,14 @@ uint32_t ThriftHiveMetastore_get_functions_result::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2328;
-            ::apache::thrift::protocol::TType _etype2331;
-            xfer += iprot->readListBegin(_etype2331, _size2328);
-            this->success.resize(_size2328);
-            uint32_t _i2332;
-            for (_i2332 = 0; _i2332 < _size2328; ++_i2332)
+            uint32_t _size2336;
+            ::apache::thrift::protocol::TType _etype2339;
+            xfer += iprot->readListBegin(_etype2339, _size2336);
+            this->success.resize(_size2336);
+            uint32_t _i2340;
+            for (_i2340 = 0; _i2340 < _size2336; ++_i2340)
             {
-              xfer += iprot->readString(this->success[_i2332]);
+              xfer += iprot->readString(this->success[_i2340]);
             }
             xfer += iprot->readListEnd();
           }
@@ -36003,10 +36003,10 @@ uint32_t ThriftHiveMetastore_get_functions_result::write(::apache::thrift::proto
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter2333;
-      for (_iter2333 = this->success.begin(); _iter2333 != this->success.end(); ++_iter2333)
+      std::vector<std::string> ::const_iterator _iter2341;
+      for (_iter2341 = this->success.begin(); _iter2341 != this->success.end(); ++_iter2341)
       {
-        xfer += oprot->writeString((*_iter2333));
+        xfer += oprot->writeString((*_iter2341));
       }
       xfer += oprot->writeListEnd();
     }
@@ -36051,14 +36051,14 @@ uint32_t ThriftHiveMetastore_get_functions_presult::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2334;
-            ::apache::thrift::protocol::TType _etype2337;
-            xfer += iprot->readListBegin(_etype2337, _size2334);
-            (*(this->success)).resize(_size2334);
-            uint32_t _i2338;
-            for (_i2338 = 0; _i2338 < _size2334; ++_i2338)
+            uint32_t _size2342;
+            ::apache::thrift::protocol::TType _etype2345;
+            xfer += iprot->readListBegin(_etype2345, _size2342);
+            (*(this->success)).resize(_size2342);
+            uint32_t _i2346;
+            for (_i2346 = 0; _i2346 < _size2342; ++_i2346)
             {
-              xfer += iprot->readString((*(this->success))[_i2338]);
+              xfer += iprot->readString((*(this->success))[_i2346]);
             }
             xfer += iprot->readListEnd();
           }
@@ -37018,14 +37018,14 @@ uint32_t ThriftHiveMetastore_get_role_names_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2339;
-            ::apache::thrift::protocol::TType _etype2342;
-            xfer += iprot->readListBegin(_etype2342, _size2339);
-            this->success.resize(_size2339);
-            uint32_t _i2343;
-            for (_i2343 = 0; _i2343 < _size2339; ++_i2343)
+            uint32_t _size2347;
+            ::apache::thrift::protocol::TType _etype2350;
+            xfer += iprot->readListBegin(_etype2350, _size2347);
+            this->success.resize(_size2347);
+            uint32_t _i2351;
+            for (_i2351 = 0; _i2351 < _size2347; ++_i2351)
             {
-              xfer += iprot->readString(this->success[_i2343]);
+              xfer += iprot->readString(this->success[_i2351]);
             }
             xfer += iprot->readListEnd();
           }
@@ -37064,10 +37064,10 @@ uint32_t ThriftHiveMetastore_get_role_names_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter2344;
-      for (_iter2344 = this->success.begin(); _iter2344 != this->success.end(); ++_iter2344)
+      std::vector<std::string> ::const_iterator _iter2352;
+      for (_iter2352 = this->success.begin(); _iter2352 != this->success.end(); ++_iter2352)
       {
-        xfer += oprot->writeString((*_iter2344));
+        xfer += oprot->writeString((*_iter2352));
       }
       xfer += oprot->writeListEnd();
     }
@@ -37112,14 +37112,14 @@ uint32_t ThriftHiveMetastore_get_role_names_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2345;
-            ::apache::thrift::protocol::TType _etype2348;
-            xfer += iprot->readListBegin(_etype2348, _size2345);
-            (*(this->success)).resize(_size2345);
-            uint32_t _i2349;
-            for (_i2349 = 0; _i2349 < _size2345; ++_i2349)
+            uint32_t _size2353;
+            ::apache::thrift::protocol::TType _etype2356;
+            xfer += iprot->readListBegin(_etype2356, _size2353);
+            (*(this->success)).resize(_size2353);
+            uint32_t _i2357;
+            for (_i2357 = 0; _i2357 < _size2353; ++_i2357)
             {
-              xfer += iprot->readString((*(this->success))[_i2349]);
+              xfer += iprot->readString((*(this->success))[_i2357]);
             }
             xfer += iprot->readListEnd();
           }
@@ -37192,9 +37192,9 @@ uint32_t ThriftHiveMetastore_grant_role_args::read(::apache::thrift::protocol::T
         break;
       case 3:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast2350;
-          xfer += iprot->readI32(ecast2350);
-          this->principal_type = (PrincipalType::type)ecast2350;
+          int32_t ecast2358;
+          xfer += iprot->readI32(ecast2358);
+          this->principal_type = (PrincipalType::type)ecast2358;
           this->__isset.principal_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -37210,9 +37210,9 @@ uint32_t ThriftHiveMetastore_grant_role_args::read(::apache::thrift::protocol::T
         break;
       case 5:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast2351;
-          xfer += iprot->readI32(ecast2351);
-          this->grantorType = (PrincipalType::type)ecast2351;
+          int32_t ecast2359;
+          xfer += iprot->readI32(ecast2359);
+          this->grantorType = (PrincipalType::type)ecast2359;
           this->__isset.grantorType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -37483,9 +37483,9 @@ uint32_t ThriftHiveMetastore_revoke_role_args::read(::apache::thrift::protocol::
         break;
       case 3:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast2352;
-          xfer += iprot->readI32(ecast2352);
-          this->principal_type = (PrincipalType::type)ecast2352;
+          int32_t ecast2360;
+          xfer += iprot->readI32(ecast2360);
+          this->principal_type = (PrincipalType::type)ecast2360;
           this->__isset.principal_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -37716,9 +37716,9 @@ uint32_t ThriftHiveMetastore_list_roles_args::read(::apache::thrift::protocol::T
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast2353;
-          xfer += iprot->readI32(ecast2353);
-          this->principal_type = (PrincipalType::type)ecast2353;
+          int32_t ecast2361;
+          xfer += iprot->readI32(ecast2361);
+          this->principal_type = (PrincipalType::type)ecast2361;
           this->__isset.principal_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -37807,14 +37807,14 @@ uint32_t ThriftHiveMetastore_list_roles_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2354;
-            ::apache::thrift::protocol::TType _etype2357;
-            xfer += iprot->readListBegin(_etype2357, _size2354);
-            this->success.resize(_size2354);
-            uint32_t _i2358;
-            for (_i2358 = 0; _i2358 < _size2354; ++_i2358)
+            uint32_t _size2362;
+            ::apache::thrift::protocol::TType _etype2365;
+            xfer += iprot->readListBegin(_etype2365, _size2362);
+            this->success.resize(_size2362);
+            uint32_t _i2366;
+            for (_i2366 = 0; _i2366 < _size2362; ++_i2366)
             {
-              xfer += this->success[_i2358].read(iprot);
+              xfer += this->success[_i2366].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -37853,10 +37853,10 @@ uint32_t ThriftHiveMetastore_list_roles_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Role> ::const_iterator _iter2359;
-      for (_iter2359 = this->success.begin(); _iter2359 != this->success.end(); ++_iter2359)
+      std::vector<Role> ::const_iterator _iter2367;
+      for (_iter2367 = this->success.begin(); _iter2367 != this->success.end(); ++_iter2367)
       {
-        xfer += (*_iter2359).write(oprot);
+        xfer += (*_iter2367).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -37901,14 +37901,14 @@ uint32_t ThriftHiveMetastore_list_roles_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2360;
-            ::apache::thrift::protocol::TType _etype2363;
-            xfer += iprot->readListBegin(_etype2363, _size2360);
-            (*(this->success)).resize(_size2360);
-            uint32_t _i2364;
-            for (_i2364 = 0; _i2364 < _size2360; ++_i2364)
+            uint32_t _size2368;
+            ::apache::thrift::protocol::TType _etype2371;
+            xfer += iprot->readListBegin(_etype2371, _size2368);
+            (*(this->success)).resize(_size2368);
+            uint32_t _i2372;
+            for (_i2372 = 0; _i2372 < _size2368; ++_i2372)
             {
-              xfer += (*(this->success))[_i2364].read(iprot);
+              xfer += (*(this->success))[_i2372].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -38604,14 +38604,14 @@ uint32_t ThriftHiveMetastore_get_privilege_set_args::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size2365;
-            ::apache::thrift::protocol::TType _etype2368;
-            xfer += iprot->readListBegin(_etype2368, _size2365);
-            this->group_names.resize(_size2365);
-            uint32_t _i2369;
-            for (_i2369 = 0; _i2369 < _size2365; ++_i2369)
+            uint32_t _size2373;
+            ::apache::thrift::protocol::TType _etype2376;
+            xfer += iprot->readListBegin(_etype2376, _size2373);
+            this->group_names.resize(_size2373);
+            uint32_t _i2377;
+            for (_i2377 = 0; _i2377 < _size2373; ++_i2377)
             {
-              xfer += iprot->readString(this->group_names[_i2369]);
+              xfer += iprot->readString(this->group_names[_i2377]);
             }
             xfer += iprot->readListEnd();
           }
@@ -38648,10 +38648,10 @@ uint32_t ThriftHiveMetastore_get_privilege_set_args::write(::apache::thrift::pro
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter2370;
-    for (_iter2370 = this->group_names.begin(); _iter2370 != this->group_names.end(); ++_iter2370)
+    std::vector<std::string> ::const_iterator _iter2378;
+    for (_iter2378 = this->group_names.begin(); _iter2378 != this->group_names.end(); ++_iter2378)
     {
-      xfer += oprot->writeString((*_iter2370));
+      xfer += oprot->writeString((*_iter2378));
     }
     xfer += oprot->writeListEnd();
   }
@@ -38683,10 +38683,10 @@ uint32_t ThriftHiveMetastore_get_privilege_set_pargs::write(::apache::thrift::pr
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter2371;
-    for (_iter2371 = (*(this->group_names)).begin(); _iter2371 != (*(this->group_names)).end(); ++_iter2371)
+    std::vector<std::string> ::const_iterator _iter2379;
+    for (_iter2379 = (*(this->group_names)).begin(); _iter2379 != (*(this->group_names)).end(); ++_iter2379)
     {
-      xfer += oprot->writeString((*_iter2371));
+      xfer += oprot->writeString((*_iter2379));
     }
     xfer += oprot->writeListEnd();
   }
@@ -38861,9 +38861,9 @@ uint32_t ThriftHiveMetastore_list_privileges_args::read(::apache::thrift::protoc
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast2372;
-          xfer += iprot->readI32(ecast2372);
-          this->principal_type = (PrincipalType::type)ecast2372;
+          int32_t ecast2380;
+          xfer += iprot->readI32(ecast2380);
+          this->principal_type = (PrincipalType::type)ecast2380;
           this->__isset.principal_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -38968,14 +38968,14 @@ uint32_t ThriftHiveMetastore_list_privileges_result::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2373;
-            ::apache::thrift::protocol::TType _etype2376;
-            xfer += iprot->readListBegin(_etype2376, _size2373);
-            this->success.resize(_size2373);
-            uint32_t _i2377;
-            for (_i2377 = 0; _i2377 < _size2373; ++_i2377)
+            uint32_t _size2381;
+            ::apache::thrift::protocol::TType _etype2384;
+            xfer += iprot->readListBegin(_etype2384, _size2381);
+            this->success.resize(_size2381);
+            uint32_t _i2385;
+            for (_i2385 = 0; _i2385 < _size2381; ++_i2385)
             {
-              xfer += this->success[_i2377].read(iprot);
+              xfer += this->success[_i2385].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -39014,10 +39014,10 @@ uint32_t ThriftHiveMetastore_list_privileges_result::write(::apache::thrift::pro
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<HiveObjectPrivilege> ::const_iterator _iter2378;
-      for (_iter2378 = this->success.begin(); _iter2378 != this->success.end(); ++_iter2378)
+      std::vector<HiveObjectPrivilege> ::const_iterator _iter2386;
+      for (_iter2386 = this->success.begin(); _iter2386 != this->success.end(); ++_iter2386)
       {
-        xfer += (*_iter2378).write(oprot);
+        xfer += (*_iter2386).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -39062,14 +39062,14 @@ uint32_t ThriftHiveMetastore_list_privileges_presult::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2379;
-            ::apache::thrift::protocol::TType _etype2382;
-            xfer += iprot->readListBegin(_etype2382, _size2379);
-            (*(this->success)).resize(_size2379);
-            uint32_t _i2383;
-            for (_i2383 = 0; _i2383 < _size2379; ++_i2383)
+            uint32_t _size2387;
+            ::apache::thrift::protocol::TType _etype2390;
+            xfer += iprot->readListBegin(_etype2390, _size2387);
+            (*(this->success)).resize(_size2387);
+            uint32_t _i2391;
+            for (_i2391 = 0; _i2391 < _size2387; ++_i2391)
             {
-              xfer += (*(this->success))[_i2383].read(iprot);
+              xfer += (*(this->success))[_i2391].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -39996,14 +39996,14 @@ uint32_t ThriftHiveMetastore_set_ugi_args::read(::apache::thrift::protocol::TPro
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size2384;
-            ::apache::thrift::protocol::TType _etype2387;
-            xfer += iprot->readListBegin(_etype2387, _size2384);
-            this->group_names.resize(_size2384);
-            uint32_t _i2388;
-            for (_i2388 = 0; _i2388 < _size2384; ++_i2388)
+            uint32_t _size2392;
+            ::apache::thrift::protocol::TType _etype2395;
+            xfer += iprot->readListBegin(_etype2395, _size2392);
+            this->group_names.resize(_size2392);
+            uint32_t _i2396;
+            for (_i2396 = 0; _i2396 < _size2392; ++_i2396)
             {
-              xfer += iprot->readString(this->group_names[_i2388]);
+              xfer += iprot->readString(this->group_names[_i2396]);
             }
             xfer += iprot->readListEnd();
           }
@@ -40036,10 +40036,10 @@ uint32_t ThriftHiveMetastore_set_ugi_args::write(::apache::thrift::protocol::TPr
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter2389;
-    for (_iter2389 = this->group_names.begin(); _iter2389 != this->group_names.end(); ++_iter2389)
+    std::vector<std::string> ::const_iterator _iter2397;
+    for (_iter2397 = this->group_names.begin(); _iter2397 != this->group_names.end(); ++_iter2397)
     {
-      xfer += oprot->writeString((*_iter2389));
+      xfer += oprot->writeString((*_iter2397));
     }
     xfer += oprot->writeListEnd();
   }
@@ -40067,10 +40067,10 @@ uint32_t ThriftHiveMetastore_set_ugi_pargs::write(::apache::thrift::protocol::TP
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter2390;
-    for (_iter2390 = (*(this->group_names)).begin(); _iter2390 != (*(this->group_names)).end(); ++_iter2390)
+    std::vector<std::string> ::const_iterator _iter2398;
+    for (_iter2398 = (*(this->group_names)).begin(); _iter2398 != (*(this->group_names)).end(); ++_iter2398)
     {
-      xfer += oprot->writeString((*_iter2390));
+      xfer += oprot->writeString((*_iter2398));
     }
     xfer += oprot->writeListEnd();
   }
@@ -40111,14 +40111,14 @@ uint32_t ThriftHiveMetastore_set_ugi_result::read(::apache::thrift::protocol::TP
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2391;
-            ::apache::thrift::protocol::TType _etype2394;
-            xfer += iprot->readListBegin(_etype2394, _size2391);
-            this->success.resize(_size2391);
-            uint32_t _i2395;
-            for (_i2395 = 0; _i2395 < _size2391; ++_i2395)
+            uint32_t _size2399;
+            ::apache::thrift::protocol::TType _etype2402;
+            xfer += iprot->readListBegin(_etype2402, _size2399);
+            this->success.resize(_size2399);
+            uint32_t _i2403;
+            for (_i2403 = 0; _i2403 < _size2399; ++_i2403)
             {
-              xfer += iprot->readString(this->success[_i2395]);
+              xfer += iprot->readString(this->success[_i2403]);
             }
             xfer += iprot->readListEnd();
           }
@@ -40157,10 +40157,10 @@ uint32_t ThriftHiveMetastore_set_ugi_result::write(::apache::thrift::protocol::T
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter2396;
-      for (_iter2396 = this->success.begin(); _iter2396 != this->success.end(); ++_iter2396)
+      std::vector<std::string> ::const_iterator _iter2404;
+      for (_iter2404 = this->success.begin(); _iter2404 != this->success.end(); ++_iter2404)
       {
-        xfer += oprot->writeString((*_iter2396));
+        xfer += oprot->writeString((*_iter2404));
       }
       xfer += oprot->writeListEnd();
     }
@@ -40205,14 +40205,14 @@ uint32_t ThriftHiveMetastore_set_ugi_presult::read(::apache::thrift::protocol::T
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2397;
-            ::apache::thrift::protocol::TType _etype2400;
-            xfer += iprot->readListBegin(_etype2400, _size2397);
-            (*(this->success)).resize(_size2397);
-            uint32_t _i2401;
-            for (_i2401 = 0; _i2401 < _size2397; ++_i2401)
+            uint32_t _size2405;
+            ::apache::thrift::protocol::TType _etype2408;
+            xfer += iprot->readListBegin(_etype2408, _size2405);
+            (*(this->success)).resize(_size2405);
+            uint32_t _i2409;
+            for (_i2409 = 0; _i2409 < _size2405; ++_i2409)
             {
-              xfer += iprot->readString((*(this->success))[_i2401]);
+              xfer += iprot->readString((*(this->success))[_i2409]);
             }
             xfer += iprot->readListEnd();
           }
@@ -41523,14 +41523,14 @@ uint32_t ThriftHiveMetastore_get_all_token_identifiers_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2402;
-            ::apache::thrift::protocol::TType _etype2405;
-            xfer += iprot->readListBegin(_etype2405, _size2402);
-            this->success.resize(_size2402);
-            uint32_t _i2406;
-            for (_i2406 = 0; _i2406 < _size2402; ++_i2406)
+            uint32_t _size2410;
+            ::apache::thrift::protocol::TType _etype2413;
+            xfer += iprot->readListBegin(_etype2413, _size2410);
+            this->success.resize(_size2410);
+            uint32_t _i2414;
+            for (_i2414 = 0; _i2414 < _size2410; ++_i2414)
             {
-              xfer += iprot->readString(this->success[_i2406]);
+              xfer += iprot->readString(this->success[_i2414]);
             }
             xfer += iprot->readListEnd();
           }
@@ -41561,10 +41561,10 @@ uint32_t ThriftHiveMetastore_get_all_token_identifiers_result::write(::apache::t
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter2407;
-      for (_iter2407 = this->success.begin(); _iter2407 != this->success.end(); ++_iter2407)
+      std::vector<std::string> ::const_iterator _iter2415;
+      for (_iter2415 = this->success.begin(); _iter2415 != this->success.end(); ++_iter2415)
       {
-        xfer += oprot->writeString((*_iter2407));
+        xfer += oprot->writeString((*_iter2415));
       }
       xfer += oprot->writeListEnd();
     }
@@ -41605,14 +41605,14 @@ uint32_t ThriftHiveMetastore_get_all_token_identifiers_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2408;
-            ::apache::thrift::protocol::TType _etype2411;
-            xfer += iprot->readListBegin(_etype2411, _size2408);
-            (*(this->success)).resize(_size2408);
-            uint32_t _i2412;
-            for (_i2412 = 0; _i2412 < _size2408; ++_i2412)
+            uint32_t _size2416;
+            ::apache::thrift::protocol::TType _etype2419;
+            xfer += iprot->readListBegin(_etype2419, _size2416);
+            (*(this->success)).resize(_size2416);
+            uint32_t _i2420;
+            for (_i2420 = 0; _i2420 < _size2416; ++_i2420)
             {
-              xfer += iprot->readString((*(this->success))[_i2412]);
+              xfer += iprot->readString((*(this->success))[_i2420]);
             }
             xfer += iprot->readListEnd();
           }
@@ -42338,14 +42338,14 @@ uint32_t ThriftHiveMetastore_get_master_keys_result::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2413;
-            ::apache::thrift::protocol::TType _etype2416;
-            xfer += iprot->readListBegin(_etype2416, _size2413);
-            this->success.resize(_size2413);
-            uint32_t _i2417;
-            for (_i2417 = 0; _i2417 < _size2413; ++_i2417)
+            uint32_t _size2421;
+            ::apache::thrift::protocol::TType _etype2424;
+            xfer += iprot->readListBegin(_etype2424, _size2421);
+            this->success.resize(_size2421);
+            uint32_t _i2425;
+            for (_i2425 = 0; _i2425 < _size2421; ++_i2425)
             {
-              xfer += iprot->readString(this->success[_i2417]);
+              xfer += iprot->readString(this->success[_i2425]);
             }
             xfer += iprot->readListEnd();
           }
@@ -42376,10 +42376,10 @@ uint32_t ThriftHiveMetastore_get_master_keys_result::write(::apache::thrift::pro
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter2418;
-      for (_iter2418 = this->success.begin(); _iter2418 != this->success.end(); ++_iter2418)
+      std::vector<std::string> ::const_iterator _iter2426;
+      for (_iter2426 = this->success.begin(); _iter2426 != this->success.end(); ++_iter2426)
       {
-        xfer += oprot->writeString((*_iter2418));
+        xfer += oprot->writeString((*_iter2426));
       }
       xfer += oprot->writeListEnd();
     }
@@ -42420,14 +42420,14 @@ uint32_t ThriftHiveMetastore_get_master_keys_presult::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2419;
-            ::apache::thrift::protocol::TType _etype2422;
-            xfer += iprot->readListBegin(_etype2422, _size2419);
-            (*(this->success)).resize(_size2419);
-            uint32_t _i2423;
-            for (_i2423 = 0; _i2423 < _size2419; ++_i2423)
+            uint32_t _size2427;
+            ::apache::thrift::protocol::TType _etype2430;
+            xfer += iprot->readListBegin(_etype2430, _size2427);
+            (*(this->success)).resize(_size2427);
+            uint32_t _i2431;
+            for (_i2431 = 0; _i2431 < _size2427; ++_i2431)
             {
-              xfer += iprot->readString((*(this->success))[_i2423]);
+              xfer += iprot->readString((*(this->success))[_i2431]);
             }
             xfer += iprot->readListEnd();
           }
@@ -47680,14 +47680,14 @@ uint32_t ThriftHiveMetastore_find_columns_with_stats_result::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2424;
-            ::apache::thrift::protocol::TType _etype2427;
-            xfer += iprot->readListBegin(_etype2427, _size2424);
-            this->success.resize(_size2424);
-            uint32_t _i2428;
-            for (_i2428 = 0; _i2428 < _size2424; ++_i2428)
+            uint32_t _size2432;
+            ::apache::thrift::protocol::TType _etype2435;
+            xfer += iprot->readListBegin(_etype2435, _size2432);
+            this->success.resize(_size2432);
+            uint32_t _i2436;
+            for (_i2436 = 0; _i2436 < _size2432; ++_i2436)
             {
-              xfer += iprot->readString(this->success[_i2428]);
+              xfer += iprot->readString(this->success[_i2436]);
             }
             xfer += iprot->readListEnd();
           }
@@ -47718,10 +47718,10 @@ uint32_t ThriftHiveMetastore_find_columns_with_stats_result::write(::apache::thr
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter2429;
-      for (_iter2429 = this->success.begin(); _iter2429 != this->success.end(); ++_iter2429)
+      std::vector<std::string> ::const_iterator _iter2437;
+      for (_iter2437 = this->success.begin(); _iter2437 != this->success.end(); ++_iter2437)
       {
-        xfer += oprot->writeString((*_iter2429));
+        xfer += oprot->writeString((*_iter2437));
       }
       xfer += oprot->writeListEnd();
     }
@@ -47762,14 +47762,14 @@ uint32_t ThriftHiveMetastore_find_columns_with_stats_presult::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2430;
-            ::apache::thrift::protocol::TType _etype2433;
-            xfer += iprot->readListBegin(_etype2433, _size2430);
-            (*(this->success)).resize(_size2430);
-            uint32_t _i2434;
-            for (_i2434 = 0; _i2434 < _size2430; ++_i2434)
+            uint32_t _size2438;
+            ::apache::thrift::protocol::TType _etype2441;
+            xfer += iprot->readListBegin(_etype2441, _size2438);
+            (*(this->success)).resize(_size2438);
+            uint32_t _i2442;
+            for (_i2442 = 0; _i2442 < _size2438; ++_i2442)
             {
-              xfer += iprot->readString((*(this->success))[_i2434]);
+              xfer += iprot->readString((*(this->success))[_i2442]);
             }
             xfer += iprot->readListEnd();
           }
@@ -48352,11 +48352,11 @@ uint32_t ThriftHiveMetastore_mark_failed_presult::read(::apache::thrift::protoco
 }
 
 
-ThriftHiveMetastore_set_hadoop_jobid_args::~ThriftHiveMetastore_set_hadoop_jobid_args() noexcept {
+ThriftHiveMetastore_update_compaction_metrics_data_args::~ThriftHiveMetastore_update_compaction_metrics_data_args() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_set_hadoop_jobid_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_update_compaction_metrics_data_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -48378,17 +48378,9 @@ uint32_t ThriftHiveMetastore_set_hadoop_jobid_args::read(::apache::thrift::proto
     switch (fid)
     {
       case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->jobId);
-          this->__isset.jobId = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 2:
-        if (ftype == ::apache::thrift::protocol::T_I64) {
-          xfer += iprot->readI64(this->cq_id);
-          this->__isset.cq_id = true;
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->data.read(iprot);
+          this->__isset.data = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -48405,17 +48397,13 @@ uint32_t ThriftHiveMetastore_set_hadoop_jobid_args::read(::apache::thrift::proto
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_set_hadoop_jobid_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_update_compaction_metrics_data_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_set_hadoop_jobid_args");
-
-  xfer += oprot->writeFieldBegin("jobId", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString(this->jobId);
-  xfer += oprot->writeFieldEnd();
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_update_compaction_metrics_data_args");
 
-  xfer += oprot->writeFieldBegin("cq_id", ::apache::thrift::protocol::T_I64, 2);
-  xfer += oprot->writeI64(this->cq_id);
+  xfer += oprot->writeFieldBegin("data", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += this->data.write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -48424,21 +48412,17 @@ uint32_t ThriftHiveMetastore_set_hadoop_jobid_args::write(::apache::thrift::prot
 }
 
 
-ThriftHiveMetastore_set_hadoop_jobid_pargs::~ThriftHiveMetastore_set_hadoop_jobid_pargs() noexcept {
+ThriftHiveMetastore_update_compaction_metrics_data_pargs::~ThriftHiveMetastore_update_compaction_metrics_data_pargs() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_set_hadoop_jobid_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_update_compaction_metrics_data_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_set_hadoop_jobid_pargs");
-
-  xfer += oprot->writeFieldBegin("jobId", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString((*(this->jobId)));
-  xfer += oprot->writeFieldEnd();
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_update_compaction_metrics_data_pargs");
 
-  xfer += oprot->writeFieldBegin("cq_id", ::apache::thrift::protocol::T_I64, 2);
-  xfer += oprot->writeI64((*(this->cq_id)));
+  xfer += oprot->writeFieldBegin("data", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += (*(this->data)).write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -48447,11 +48431,11 @@ uint32_t ThriftHiveMetastore_set_hadoop_jobid_pargs::write(::apache::thrift::pro
 }
 
 
-ThriftHiveMetastore_set_hadoop_jobid_result::~ThriftHiveMetastore_set_hadoop_jobid_result() noexcept {
+ThriftHiveMetastore_update_compaction_metrics_data_result::~ThriftHiveMetastore_update_compaction_metrics_data_result() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_set_hadoop_jobid_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_update_compaction_metrics_data_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -48470,7 +48454,28 @@ uint32_t ThriftHiveMetastore_set_hadoop_jobid_result::read(::apache::thrift::pro
     if (ftype == ::apache::thrift::protocol::T_STOP) {
       break;
     }
-    xfer += iprot->skip(ftype);
+    switch (fid)
+    {
+      case 0:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->success);
+          this->__isset.success = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      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();
   }
 
@@ -48479,23 +48484,32 @@ uint32_t ThriftHiveMetastore_set_hadoop_jobid_result::read(::apache::thrift::pro
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_set_hadoop_jobid_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_update_compaction_metrics_data_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_set_hadoop_jobid_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_update_compaction_metrics_data_result");
 
+  if (this->__isset.success) {
+    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0);
+    xfer += oprot->writeBool(this->success);
+    xfer += oprot->writeFieldEnd();
+  } else 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_set_hadoop_jobid_presult::~ThriftHiveMetastore_set_hadoop_jobid_presult() noexcept {
+ThriftHiveMetastore_update_compaction_metrics_data_presult::~ThriftHiveMetastore_update_compaction_metrics_data_presult() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_set_hadoop_jobid_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_update_compaction_metrics_data_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -48514,7 +48528,28 @@ uint32_t ThriftHiveMetastore_set_hadoop_jobid_presult::read(::apache::thrift::pr
     if (ftype == ::apache::thrift::protocol::T_STOP) {
       break;
     }
-    xfer += iprot->skip(ftype);
+    switch (fid)
+    {
+      case 0:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool((*(this->success)));
+          this->__isset.success = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      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();
   }
 
@@ -48524,11 +48559,11 @@ uint32_t ThriftHiveMetastore_set_hadoop_jobid_presult::read(::apache::thrift::pr
 }
 
 
-ThriftHiveMetastore_get_latest_committed_compaction_info_args::~ThriftHiveMetastore_get_latest_committed_compaction_info_args() noexcept {
+ThriftHiveMetastore_remove_compaction_metrics_data_args::~ThriftHiveMetastore_remove_compaction_metrics_data_args() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_latest_committed_compaction_info_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_remove_compaction_metrics_data_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -48551,8 +48586,8 @@ uint32_t ThriftHiveMetastore_get_latest_committed_compaction_info_args::read(::a
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->rqst.read(iprot);
-          this->__isset.rqst = true;
+          xfer += this->request.read(iprot);
+          this->__isset.request = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -48569,13 +48604,13 @@ uint32_t ThriftHiveMetastore_get_latest_committed_compaction_info_args::read(::a
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_latest_committed_compaction_info_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_remove_compaction_metrics_data_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_latest_committed_compaction_info_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_remove_compaction_metrics_data_args");
 
-  xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1);
-  xfer += this->rqst.write(oprot);
+  xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += this->request.write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -48584,17 +48619,17 @@ uint32_t ThriftHiveMetastore_get_latest_committed_compaction_info_args::write(::
 }
 
 
-ThriftHiveMetastore_get_latest_committed_compaction_info_pargs::~ThriftHiveMetastore_get_latest_committed_compaction_info_pargs() noexcept {
+ThriftHiveMetastore_remove_compaction_metrics_data_pargs::~ThriftHiveMetastore_remove_compaction_metrics_data_pargs() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_latest_committed_compaction_info_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_remove_compaction_metrics_data_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_latest_committed_compaction_info_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_remove_compaction_metrics_data_pargs");
 
-  xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1);
-  xfer += (*(this->rqst)).write(oprot);
+  xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += (*(this->request)).write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -48603,11 +48638,11 @@ uint32_t ThriftHiveMetastore_get_latest_committed_compaction_info_pargs::write(:
 }
 
 
-ThriftHiveMetastore_get_latest_committed_compaction_info_result::~ThriftHiveMetastore_get_latest_committed_compaction_info_result() noexcept {
+ThriftHiveMetastore_remove_compaction_metrics_data_result::~ThriftHiveMetastore_remove_compaction_metrics_data_result() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_latest_committed_compaction_info_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_remove_compaction_metrics_data_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -48628,10 +48663,10 @@ uint32_t ThriftHiveMetastore_get_latest_committed_compaction_info_result::read(:
     }
     switch (fid)
     {
-      case 0:
+      case 1:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->success.read(iprot);
-          this->__isset.success = true;
+          xfer += this->o1.read(iprot);
+          this->__isset.o1 = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -48648,15 +48683,15 @@ uint32_t ThriftHiveMetastore_get_latest_committed_compaction_info_result::read(:
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_latest_committed_compaction_info_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_remove_compaction_metrics_data_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_latest_committed_compaction_info_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_remove_compaction_metrics_data_result");
 
-  if (this->__isset.success) {
-    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
-    xfer += this->success.write(oprot);
+  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();
@@ -48665,57 +48700,11 @@ uint32_t ThriftHiveMetastore_get_latest_committed_compaction_info_result::write(
 }
 
 
-ThriftHiveMetastore_get_latest_committed_compaction_info_presult::~ThriftHiveMetastore_get_latest_committed_compaction_info_presult() noexcept {
+ThriftHiveMetastore_remove_compaction_metrics_data_presult::~ThriftHiveMetastore_remove_compaction_metrics_data_presult() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_latest_committed_compaction_info_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 0:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += (*(this->success)).read(iprot);
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-
-ThriftHiveMetastore_get_next_notification_args::~ThriftHiveMetastore_get_next_notification_args() noexcept {
-}
-
-
-uint32_t ThriftHiveMetastore_get_next_notification_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_remove_compaction_metrics_data_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -48738,8 +48727,8 @@ uint32_t ThriftHiveMetastore_get_next_notification_args::read(::apache::thrift::
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->rqst.read(iprot);
-          this->__isset.rqst = true;
+          xfer += this->o1.read(iprot);
+          this->__isset.o1 = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -48756,45 +48745,12 @@ uint32_t ThriftHiveMetastore_get_next_notification_args::read(::apache::thrift::
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_next_notification_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_next_notification_args");
-
-  xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1);
-  xfer += this->rqst.write(oprot);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHiveMetastore_get_next_notification_pargs::~ThriftHiveMetastore_get_next_notification_pargs() noexcept {
-}
-
 
-uint32_t ThriftHiveMetastore_get_next_notification_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_next_notification_pargs");
-
-  xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1);
-  xfer += (*(this->rqst)).write(oprot);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHiveMetastore_get_next_notification_result::~ThriftHiveMetastore_get_next_notification_result() noexcept {
+ThriftHiveMetastore_set_hadoop_jobid_args::~ThriftHiveMetastore_set_hadoop_jobid_args() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_next_notification_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_set_hadoop_jobid_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -48815,72 +48771,18 @@ uint32_t ThriftHiveMetastore_get_next_notification_result::read(::apache::thrift
     }
     switch (fid)
     {
-      case 0:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->success.read(iprot);
-          this->__isset.success = true;
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->jobId);
+          this->__isset.jobId = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHiveMetastore_get_next_notification_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
-
-  uint32_t xfer = 0;
-
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_next_notification_result");
-
-  if (this->__isset.success) {
-    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
-    xfer += this->success.write(oprot);
-    xfer += oprot->writeFieldEnd();
-  }
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHiveMetastore_get_next_notification_presult::~ThriftHiveMetastore_get_next_notification_presult() noexcept {
-}
-
-
-uint32_t ThriftHiveMetastore_get_next_notification_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 0:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += (*(this->success)).read(iprot);
-          this->__isset.success = true;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_I64) {
+          xfer += iprot->readI64(this->cq_id);
+          this->__isset.cq_id = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -48897,43 +48799,18 @@ uint32_t ThriftHiveMetastore_get_next_notification_presult::read(::apache::thrif
   return xfer;
 }
 
-
-ThriftHiveMetastore_get_current_notificationEventId_args::~ThriftHiveMetastore_get_current_notificationEventId_args() noexcept {
-}
-
-
-uint32_t ThriftHiveMetastore_get_current_notificationEventId_args::read(::apache::thrift::protocol::TProtocol* iprot) {
-
-  ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+uint32_t ThriftHiveMetastore_set_hadoop_jobid_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   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;
-    }
-    xfer += iprot->skip(ftype);
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
+  ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_set_hadoop_jobid_args");
 
-  return xfer;
-}
+  xfer += oprot->writeFieldBegin("jobId", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->jobId);
+  xfer += oprot->writeFieldEnd();
 
-uint32_t ThriftHiveMetastore_get_current_notificationEventId_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_current_notificationEventId_args");
+  xfer += oprot->writeFieldBegin("cq_id", ::apache::thrift::protocol::T_I64, 2);
+  xfer += oprot->writeI64(this->cq_id);
+  xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
@@ -48941,14 +48818,22 @@ uint32_t ThriftHiveMetastore_get_current_notificationEventId_args::write(::apach
 }
 
 
-ThriftHiveMetastore_get_current_notificationEventId_pargs::~ThriftHiveMetastore_get_current_notificationEventId_pargs() noexcept {
+ThriftHiveMetastore_set_hadoop_jobid_pargs::~ThriftHiveMetastore_set_hadoop_jobid_pargs() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_current_notificationEventId_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_set_hadoop_jobid_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_current_notificationEventId_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_set_hadoop_jobid_pargs");
+
+  xfer += oprot->writeFieldBegin("jobId", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString((*(this->jobId)));
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("cq_id", ::apache::thrift::protocol::T_I64, 2);
+  xfer += oprot->writeI64((*(this->cq_id)));
+  xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
@@ -48956,11 +48841,11 @@ uint32_t ThriftHiveMetastore_get_current_notificationEventId_pargs::write(::apac
 }
 
 
-ThriftHiveMetastore_get_current_notificationEventId_result::~ThriftHiveMetastore_get_current_notificationEventId_result() noexcept {
+ThriftHiveMetastore_set_hadoop_jobid_result::~ThriftHiveMetastore_set_hadoop_jobid_result() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_current_notificationEventId_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_set_hadoop_jobid_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -48979,20 +48864,7 @@ uint32_t ThriftHiveMetastore_get_current_notificationEventId_result::read(::apac
     if (ftype == ::apache::thrift::protocol::T_STOP) {
       break;
     }
-    switch (fid)
-    {
-      case 0:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->success.read(iprot);
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
+    xfer += iprot->skip(ftype);
     xfer += iprot->readFieldEnd();
   }
 
@@ -49001,28 +48873,23 @@ uint32_t ThriftHiveMetastore_get_current_notificationEventId_result::read(::apac
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_current_notificationEventId_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_set_hadoop_jobid_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_current_notificationEventId_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_set_hadoop_jobid_result");
 
-  if (this->__isset.success) {
-    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
-    xfer += this->success.write(oprot);
-    xfer += oprot->writeFieldEnd();
-  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
 }
 
 
-ThriftHiveMetastore_get_current_notificationEventId_presult::~ThriftHiveMetastore_get_current_notificationEventId_presult() noexcept {
+ThriftHiveMetastore_set_hadoop_jobid_presult::~ThriftHiveMetastore_set_hadoop_jobid_presult() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_current_notificationEventId_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_set_hadoop_jobid_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -49041,20 +48908,7 @@ uint32_t ThriftHiveMetastore_get_current_notificationEventId_presult::read(::apa
     if (ftype == ::apache::thrift::protocol::T_STOP) {
       break;
     }
-    switch (fid)
-    {
-      case 0:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += (*(this->success)).read(iprot);
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
+    xfer += iprot->skip(ftype);
     xfer += iprot->readFieldEnd();
   }
 
@@ -49064,11 +48918,11 @@ uint32_t ThriftHiveMetastore_get_current_notificationEventId_presult::read(::apa
 }
 
 
-ThriftHiveMetastore_get_notification_events_count_args::~ThriftHiveMetastore_get_notification_events_count_args() noexcept {
+ThriftHiveMetastore_get_latest_committed_compaction_info_args::~ThriftHiveMetastore_get_latest_committed_compaction_info_args() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_notification_events_count_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_latest_committed_compaction_info_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -49109,10 +48963,10 @@ uint32_t ThriftHiveMetastore_get_notification_events_count_args::read(::apache::
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_notification_events_count_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_latest_committed_compaction_info_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_notification_events_count_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_latest_committed_compaction_info_args");
 
   xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1);
   xfer += this->rqst.write(oprot);
@@ -49124,14 +48978,14 @@ uint32_t ThriftHiveMetastore_get_notification_events_count_args::write(::apache:
 }
 
 
-ThriftHiveMetastore_get_notification_events_count_pargs::~ThriftHiveMetastore_get_notification_events_count_pargs() noexcept {
+ThriftHiveMetastore_get_latest_committed_compaction_info_pargs::~ThriftHiveMetastore_get_latest_committed_compaction_info_pargs() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_notification_events_count_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_latest_committed_compaction_info_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_notification_events_count_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_latest_committed_compaction_info_pargs");
 
   xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1);
   xfer += (*(this->rqst)).write(oprot);
@@ -49143,11 +48997,11 @@ uint32_t ThriftHiveMetastore_get_notification_events_count_pargs::write(::apache
 }
 
 
-ThriftHiveMetastore_get_notification_events_count_result::~ThriftHiveMetastore_get_notification_events_count_result() noexcept {
+ThriftHiveMetastore_get_latest_committed_compaction_info_result::~ThriftHiveMetastore_get_latest_committed_compaction_info_result() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_notification_events_count_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_latest_committed_compaction_info_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -49188,11 +49042,11 @@ uint32_t ThriftHiveMetastore_get_notification_events_count_result::read(::apache
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_notification_events_count_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_latest_committed_compaction_info_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_notification_events_count_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_latest_committed_compaction_info_result");
 
   if (this->__isset.success) {
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
@@ -49205,11 +49059,11 @@ uint32_t ThriftHiveMetastore_get_notification_events_count_result::write(::apach
 }
 
 
-ThriftHiveMetastore_get_notification_events_count_presult::~ThriftHiveMetastore_get_notification_events_count_presult() noexcept {
+ThriftHiveMetastore_get_latest_committed_compaction_info_presult::~ThriftHiveMetastore_get_latest_committed_compaction_info_presult() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_notification_events_count_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_latest_committed_compaction_info_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -49251,11 +49105,11 @@ uint32_t ThriftHiveMetastore_get_notification_events_count_presult::read(::apach
 }
 
 
-ThriftHiveMetastore_fire_listener_event_args::~ThriftHiveMetastore_fire_listener_event_args() noexcept {
+ThriftHiveMetastore_get_next_notification_args::~ThriftHiveMetastore_get_next_notification_args() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_fire_listener_event_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_next_notification_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -49296,10 +49150,10 @@ uint32_t ThriftHiveMetastore_fire_listener_event_args::read(::apache::thrift::pr
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_fire_listener_event_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_next_notification_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_fire_listener_event_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_next_notification_args");
 
   xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1);
   xfer += this->rqst.write(oprot);
@@ -49311,14 +49165,14 @@ uint32_t ThriftHiveMetastore_fire_listener_event_args::write(::apache::thrift::p
 }
 
 
-ThriftHiveMetastore_fire_listener_event_pargs::~ThriftHiveMetastore_fire_listener_event_pargs() noexcept {
+ThriftHiveMetastore_get_next_notification_pargs::~ThriftHiveMetastore_get_next_notification_pargs() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_fire_listener_event_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_next_notification_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_fire_listener_event_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_next_notification_pargs");
 
   xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1);
   xfer += (*(this->rqst)).write(oprot);
@@ -49330,11 +49184,11 @@ uint32_t ThriftHiveMetastore_fire_listener_event_pargs::write(::apache::thrift::
 }
 
 
-ThriftHiveMetastore_fire_listener_event_result::~ThriftHiveMetastore_fire_listener_event_result() noexcept {
+ThriftHiveMetastore_get_next_notification_result::~ThriftHiveMetastore_get_next_notification_result() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_fire_listener_event_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_next_notification_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -49375,11 +49229,11 @@ uint32_t ThriftHiveMetastore_fire_listener_event_result::read(::apache::thrift::
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_fire_listener_event_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_next_notification_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_fire_listener_event_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_next_notification_result");
 
   if (this->__isset.success) {
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
@@ -49392,11 +49246,11 @@ uint32_t ThriftHiveMetastore_fire_listener_event_result::write(::apache::thrift:
 }
 
 
-ThriftHiveMetastore_fire_listener_event_presult::~ThriftHiveMetastore_fire_listener_event_presult() noexcept {
+ThriftHiveMetastore_get_next_notification_presult::~ThriftHiveMetastore_get_next_notification_presult() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_fire_listener_event_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_next_notification_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -49438,113 +49292,11 @@ uint32_t ThriftHiveMetastore_fire_listener_event_presult::read(::apache::thrift:
 }
 
 
-ThriftHiveMetastore_flushCache_args::~ThriftHiveMetastore_flushCache_args() noexcept {
-}
-
-
-uint32_t ThriftHiveMetastore_flushCache_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;
-    }
-    xfer += iprot->skip(ftype);
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHiveMetastore_flushCache_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_flushCache_args");
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHiveMetastore_flushCache_pargs::~ThriftHiveMetastore_flushCache_pargs() noexcept {
-}
-
-
-uint32_t ThriftHiveMetastore_flushCache_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_flushCache_pargs");
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHiveMetastore_flushCache_result::~ThriftHiveMetastore_flushCache_result() noexcept {
-}
-
-
-uint32_t ThriftHiveMetastore_flushCache_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;
-    }
-    xfer += iprot->skip(ftype);
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHiveMetastore_flushCache_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
-
-  uint32_t xfer = 0;
-
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_flushCache_result");
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHiveMetastore_flushCache_presult::~ThriftHiveMetastore_flushCache_presult() noexcept {
+ThriftHiveMetastore_get_current_notificationEventId_args::~ThriftHiveMetastore_get_current_notificationEventId_args() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_flushCache_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_current_notificationEventId_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -49572,60 +49324,10 @@ uint32_t ThriftHiveMetastore_flushCache_presult::read(::apache::thrift::protocol
   return xfer;
 }
 
-
-ThriftHiveMetastore_add_write_notification_log_args::~ThriftHiveMetastore_add_write_notification_log_args() noexcept {
-}
-
-
-uint32_t ThriftHiveMetastore_add_write_notification_log_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->rqst.read(iprot);
-          this->__isset.rqst = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHiveMetastore_add_write_notification_log_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_current_notificationEventId_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_write_notification_log_args");
-
-  xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1);
-  xfer += this->rqst.write(oprot);
-  xfer += oprot->writeFieldEnd();
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_current_notificationEventId_args");
 
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
@@ -49633,18 +49335,14 @@ uint32_t ThriftHiveMetastore_add_write_notification_log_args::write(::apache::th
 }
 
 
-ThriftHiveMetastore_add_write_notification_log_pargs::~ThriftHiveMetastore_add_write_notification_log_pargs() noexcept {
+ThriftHiveMetastore_get_current_notificationEventId_pargs::~ThriftHiveMetastore_get_current_notificationEventId_pargs() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_add_write_notification_log_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_current_notificationEventId_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_write_notification_log_pargs");
-
-  xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1);
-  xfer += (*(this->rqst)).write(oprot);
-  xfer += oprot->writeFieldEnd();
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_current_notificationEventId_pargs");
 
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
@@ -49652,11 +49350,11 @@ uint32_t ThriftHiveMetastore_add_write_notification_log_pargs::write(::apache::t
 }
 
 
-ThriftHiveMetastore_add_write_notification_log_result::~ThriftHiveMetastore_add_write_notification_log_result() noexcept {
+ThriftHiveMetastore_get_current_notificationEventId_result::~ThriftHiveMetastore_get_current_notificationEventId_result() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_add_write_notification_log_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_current_notificationEventId_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -49697,11 +49395,11 @@ uint32_t ThriftHiveMetastore_add_write_notification_log_result::read(::apache::t
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_add_write_notification_log_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_current_notificationEventId_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_write_notification_log_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_current_notificationEventId_result");
 
   if (this->__isset.success) {
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
@@ -49714,11 +49412,11 @@ uint32_t ThriftHiveMetastore_add_write_notification_log_result::write(::apache::
 }
 
 
-ThriftHiveMetastore_add_write_notification_log_presult::~ThriftHiveMetastore_add_write_notification_log_presult() noexcept {
+ThriftHiveMetastore_get_current_notificationEventId_presult::~ThriftHiveMetastore_get_current_notificationEventId_presult() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_add_write_notification_log_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_current_notificationEventId_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -49760,11 +49458,11 @@ uint32_t ThriftHiveMetastore_add_write_notification_log_presult::read(::apache::
 }
 
 
-ThriftHiveMetastore_add_write_notification_log_in_batch_args::~ThriftHiveMetastore_add_write_notification_log_in_batch_args() noexcept {
+ThriftHiveMetastore_get_notification_events_count_args::~ThriftHiveMetastore_get_notification_events_count_args() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_add_write_notification_log_in_batch_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_notification_events_count_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -49805,10 +49503,10 @@ uint32_t ThriftHiveMetastore_add_write_notification_log_in_batch_args::read(::ap
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_add_write_notification_log_in_batch_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_notification_events_count_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_write_notification_log_in_batch_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_notification_events_count_args");
 
   xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1);
   xfer += this->rqst.write(oprot);
@@ -49820,14 +49518,14 @@ uint32_t ThriftHiveMetastore_add_write_notification_log_in_batch_args::write(::a
 }
 
 
-ThriftHiveMetastore_add_write_notification_log_in_batch_pargs::~ThriftHiveMetastore_add_write_notification_log_in_batch_pargs() noexcept {
+ThriftHiveMetastore_get_notification_events_count_pargs::~ThriftHiveMetastore_get_notification_events_count_pargs() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_add_write_notification_log_in_batch_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_notification_events_count_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_write_notification_log_in_batch_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_notification_events_count_pargs");
 
   xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1);
   xfer += (*(this->rqst)).write(oprot);
@@ -49839,11 +49537,11 @@ uint32_t ThriftHiveMetastore_add_write_notification_log_in_batch_pargs::write(::
 }
 
 
-ThriftHiveMetastore_add_write_notification_log_in_batch_result::~ThriftHiveMetastore_add_write_notification_log_in_batch_result() noexcept {
+ThriftHiveMetastore_get_notification_events_count_result::~ThriftHiveMetastore_get_notification_events_count_result() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_add_write_notification_log_in_batch_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_notification_events_count_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -49884,11 +49582,11 @@ uint32_t ThriftHiveMetastore_add_write_notification_log_in_batch_result::read(::
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_add_write_notification_log_in_batch_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_notification_events_count_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_write_notification_log_in_batch_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_notification_events_count_result");
 
   if (this->__isset.success) {
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
@@ -49901,11 +49599,11 @@ uint32_t ThriftHiveMetastore_add_write_notification_log_in_batch_result::write(:
 }
 
 
-ThriftHiveMetastore_add_write_notification_log_in_batch_presult::~ThriftHiveMetastore_add_write_notification_log_in_batch_presult() noexcept {
+ThriftHiveMetastore_get_notification_events_count_presult::~ThriftHiveMetastore_get_notification_events_count_presult() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_add_write_notification_log_in_batch_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_notification_events_count_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -49947,11 +49645,11 @@ uint32_t ThriftHiveMetastore_add_write_notification_log_in_batch_presult::read(:
 }
 
 
-ThriftHiveMetastore_cm_recycle_args::~ThriftHiveMetastore_cm_recycle_args() noexcept {
+ThriftHiveMetastore_fire_listener_event_args::~ThriftHiveMetastore_fire_listener_event_args() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_cm_recycle_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_fire_listener_event_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -49974,8 +49672,8 @@ uint32_t ThriftHiveMetastore_cm_recycle_args::read(::apache::thrift::protocol::T
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->request.read(iprot);
-          this->__isset.request = true;
+          xfer += this->rqst.read(iprot);
+          this->__isset.rqst = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -49992,13 +49690,13 @@ uint32_t ThriftHiveMetastore_cm_recycle_args::read(::apache::thrift::protocol::T
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_cm_recycle_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_fire_listener_event_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_cm_recycle_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_fire_listener_event_args");
 
-  xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1);
-  xfer += this->request.write(oprot);
+  xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += this->rqst.write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -50007,17 +49705,17 @@ uint32_t ThriftHiveMetastore_cm_recycle_args::write(::apache::thrift::protocol::
 }
 
 
-ThriftHiveMetastore_cm_recycle_pargs::~ThriftHiveMetastore_cm_recycle_pargs() noexcept {
+ThriftHiveMetastore_fire_listener_event_pargs::~ThriftHiveMetastore_fire_listener_event_pargs() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_cm_recycle_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_fire_listener_event_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_cm_recycle_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_fire_listener_event_pargs");
 
-  xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1);
-  xfer += (*(this->request)).write(oprot);
+  xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += (*(this->rqst)).write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -50026,11 +49724,11 @@ uint32_t ThriftHiveMetastore_cm_recycle_pargs::write(::apache::thrift::protocol:
 }
 
 
-ThriftHiveMetastore_cm_recycle_result::~ThriftHiveMetastore_cm_recycle_result() noexcept {
+ThriftHiveMetastore_fire_listener_event_result::~ThriftHiveMetastore_fire_listener_event_result() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_cm_recycle_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_fire_listener_event_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -50059,14 +49757,6 @@ uint32_t ThriftHiveMetastore_cm_recycle_result::read(::apache::thrift::protocol:
           xfer += iprot->skip(ftype);
         }
         break;
-      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;
@@ -50079,20 +49769,16 @@ uint32_t ThriftHiveMetastore_cm_recycle_result::read(::apache::thrift::protocol:
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_cm_recycle_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_fire_listener_event_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_cm_recycle_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_fire_listener_event_result");
 
   if (this->__isset.success) {
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
     xfer += this->success.write(oprot);
     xfer += oprot->writeFieldEnd();
-  } else 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();
@@ -50100,11 +49786,11 @@ uint32_t ThriftHiveMetastore_cm_recycle_result::write(::apache::thrift::protocol
 }
 
 
-ThriftHiveMetastore_cm_recycle_presult::~ThriftHiveMetastore_cm_recycle_presult() noexcept {
+ThriftHiveMetastore_fire_listener_event_presult::~ThriftHiveMetastore_fire_listener_event_presult() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_cm_recycle_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_fire_listener_event_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -50133,14 +49819,6 @@ uint32_t ThriftHiveMetastore_cm_recycle_presult::read(::apache::thrift::protocol
           xfer += iprot->skip(ftype);
         }
         break;
-      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;
@@ -50154,11 +49832,146 @@ uint32_t ThriftHiveMetastore_cm_recycle_presult::read(::apache::thrift::protocol
 }
 
 
-ThriftHiveMetastore_get_file_metadata_by_expr_args::~ThriftHiveMetastore_get_file_metadata_by_expr_args() noexcept {
+ThriftHiveMetastore_flushCache_args::~ThriftHiveMetastore_flushCache_args() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_flushCache_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;
+    }
+    xfer += iprot->skip(ftype);
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t ThriftHiveMetastore_flushCache_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_flushCache_args");
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_flushCache_pargs::~ThriftHiveMetastore_flushCache_pargs() noexcept {
+}
+
+
+uint32_t ThriftHiveMetastore_flushCache_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_flushCache_pargs");
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_flushCache_result::~ThriftHiveMetastore_flushCache_result() noexcept {
+}
+
+
+uint32_t ThriftHiveMetastore_flushCache_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;
+    }
+    xfer += iprot->skip(ftype);
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t ThriftHiveMetastore_flushCache_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+
+  uint32_t xfer = 0;
+
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_flushCache_result");
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_flushCache_presult::~ThriftHiveMetastore_flushCache_presult() noexcept {
+}
+
+
+uint32_t ThriftHiveMetastore_flushCache_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;
+    }
+    xfer += iprot->skip(ftype);
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+
+ThriftHiveMetastore_add_write_notification_log_args::~ThriftHiveMetastore_add_write_notification_log_args() noexcept {
+}
+
+
+uint32_t ThriftHiveMetastore_add_write_notification_log_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -50181,8 +49994,8 @@ uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_args::read(::apache::thri
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->req.read(iprot);
-          this->__isset.req = true;
+          xfer += this->rqst.read(iprot);
+          this->__isset.rqst = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -50199,13 +50012,13 @@ uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_args::read(::apache::thri
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_add_write_notification_log_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_file_metadata_by_expr_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_write_notification_log_args");
 
-  xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1);
-  xfer += this->req.write(oprot);
+  xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += this->rqst.write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -50214,17 +50027,17 @@ uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_args::write(::apache::thr
 }
 
 
-ThriftHiveMetastore_get_file_metadata_by_expr_pargs::~ThriftHiveMetastore_get_file_metadata_by_expr_pargs() noexcept {
+ThriftHiveMetastore_add_write_notification_log_pargs::~ThriftHiveMetastore_add_write_notification_log_pargs() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_add_write_notification_log_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_file_metadata_by_expr_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_write_notification_log_pargs");
 
-  xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1);
-  xfer += (*(this->req)).write(oprot);
+  xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += (*(this->rqst)).write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -50233,11 +50046,11 @@ uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_pargs::write(::apache::th
 }
 
 
-ThriftHiveMetastore_get_file_metadata_by_expr_result::~ThriftHiveMetastore_get_file_metadata_by_expr_result() noexcept {
+ThriftHiveMetastore_add_write_notification_log_result::~ThriftHiveMetastore_add_write_notification_log_result() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_add_write_notification_log_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -50278,11 +50091,11 @@ uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_result::read(::apache::th
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_add_write_notification_log_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_file_metadata_by_expr_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_write_notification_log_result");
 
   if (this->__isset.success) {
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
@@ -50295,11 +50108,11 @@ uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_result::write(::apache::t
 }
 
 
-ThriftHiveMetastore_get_file_metadata_by_expr_presult::~ThriftHiveMetastore_get_file_metadata_by_expr_presult() noexcept {
+ThriftHiveMetastore_add_write_notification_log_presult::~ThriftHiveMetastore_add_write_notification_log_presult() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_add_write_notification_log_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -50341,11 +50154,11 @@ uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_presult::read(::apache::t
 }
 
 
-ThriftHiveMetastore_get_file_metadata_args::~ThriftHiveMetastore_get_file_metadata_args() noexcept {
+ThriftHiveMetastore_add_write_notification_log_in_batch_args::~ThriftHiveMetastore_add_write_notification_log_in_batch_args() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_file_metadata_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_add_write_notification_log_in_batch_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -50368,8 +50181,8 @@ uint32_t ThriftHiveMetastore_get_file_metadata_args::read(::apache::thrift::prot
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->req.read(iprot);
-          this->__isset.req = true;
+          xfer += this->rqst.read(iprot);
+          this->__isset.rqst = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -50386,13 +50199,13 @@ uint32_t ThriftHiveMetastore_get_file_metadata_args::read(::apache::thrift::prot
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_file_metadata_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_add_write_notification_log_in_batch_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_file_metadata_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_write_notification_log_in_batch_args");
 
-  xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1);
-  xfer += this->req.write(oprot);
+  xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += this->rqst.write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -50401,17 +50214,17 @@ uint32_t ThriftHiveMetastore_get_file_metadata_args::write(::apache::thrift::pro
 }
 
 
-ThriftHiveMetastore_get_file_metadata_pargs::~ThriftHiveMetastore_get_file_metadata_pargs() noexcept {
+ThriftHiveMetastore_add_write_notification_log_in_batch_pargs::~ThriftHiveMetastore_add_write_notification_log_in_batch_pargs() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_file_metadata_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_add_write_notification_log_in_batch_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_file_metadata_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_write_notification_log_in_batch_pargs");
 
-  xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1);
-  xfer += (*(this->req)).write(oprot);
+  xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += (*(this->rqst)).write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -50420,11 +50233,11 @@ uint32_t ThriftHiveMetastore_get_file_metadata_pargs::write(::apache::thrift::pr
 }
 
 
-ThriftHiveMetastore_get_file_metadata_result::~ThriftHiveMetastore_get_file_metadata_result() noexcept {
+ThriftHiveMetastore_add_write_notification_log_in_batch_result::~ThriftHiveMetastore_add_write_notification_log_in_batch_result() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_file_metadata_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_add_write_notification_log_in_batch_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -50465,11 +50278,11 @@ uint32_t ThriftHiveMetastore_get_file_metadata_result::read(::apache::thrift::pr
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_file_metadata_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_add_write_notification_log_in_batch_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_file_metadata_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_add_write_notification_log_in_batch_result");
 
   if (this->__isset.success) {
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
@@ -50482,11 +50295,11 @@ uint32_t ThriftHiveMetastore_get_file_metadata_result::write(::apache::thrift::p
 }
 
 
-ThriftHiveMetastore_get_file_metadata_presult::~ThriftHiveMetastore_get_file_metadata_presult() noexcept {
+ThriftHiveMetastore_add_write_notification_log_in_batch_presult::~ThriftHiveMetastore_add_write_notification_log_in_batch_presult() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_file_metadata_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_add_write_notification_log_in_batch_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -50528,11 +50341,11 @@ uint32_t ThriftHiveMetastore_get_file_metadata_presult::read(::apache::thrift::p
 }
 
 
-ThriftHiveMetastore_put_file_metadata_args::~ThriftHiveMetastore_put_file_metadata_args() noexcept {
+ThriftHiveMetastore_cm_recycle_args::~ThriftHiveMetastore_cm_recycle_args() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_put_file_metadata_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_cm_recycle_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -50555,8 +50368,8 @@ uint32_t ThriftHiveMetastore_put_file_metadata_args::read(::apache::thrift::prot
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->req.read(iprot);
-          this->__isset.req = true;
+          xfer += this->request.read(iprot);
+          this->__isset.request = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -50573,13 +50386,13 @@ uint32_t ThriftHiveMetastore_put_file_metadata_args::read(::apache::thrift::prot
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_put_file_metadata_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_cm_recycle_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_put_file_metadata_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_cm_recycle_args");
 
-  xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1);
-  xfer += this->req.write(oprot);
+  xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += this->request.write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -50588,17 +50401,17 @@ uint32_t ThriftHiveMetastore_put_file_metadata_args::write(::apache::thrift::pro
 }
 
 
-ThriftHiveMetastore_put_file_metadata_pargs::~ThriftHiveMetastore_put_file_metadata_pargs() noexcept {
+ThriftHiveMetastore_cm_recycle_pargs::~ThriftHiveMetastore_cm_recycle_pargs() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_put_file_metadata_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_cm_recycle_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_put_file_metadata_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_cm_recycle_pargs");
 
-  xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1);
-  xfer += (*(this->req)).write(oprot);
+  xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += (*(this->request)).write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -50607,11 +50420,11 @@ uint32_t ThriftHiveMetastore_put_file_metadata_pargs::write(::apache::thrift::pr
 }
 
 
-ThriftHiveMetastore_put_file_metadata_result::~ThriftHiveMetastore_put_file_metadata_result() noexcept {
+ThriftHiveMetastore_cm_recycle_result::~ThriftHiveMetastore_cm_recycle_result() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_put_file_metadata_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_cm_recycle_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -50640,6 +50453,14 @@ uint32_t ThriftHiveMetastore_put_file_metadata_result::read(::apache::thrift::pr
           xfer += iprot->skip(ftype);
         }
         break;
+      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;
@@ -50652,16 +50473,20 @@ uint32_t ThriftHiveMetastore_put_file_metadata_result::read(::apache::thrift::pr
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_put_file_metadata_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_cm_recycle_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_put_file_metadata_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_cm_recycle_result");
 
   if (this->__isset.success) {
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
     xfer += this->success.write(oprot);
     xfer += oprot->writeFieldEnd();
+  } else 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();
@@ -50669,11 +50494,11 @@ uint32_t ThriftHiveMetastore_put_file_metadata_result::write(::apache::thrift::p
 }
 
 
-ThriftHiveMetastore_put_file_metadata_presult::~ThriftHiveMetastore_put_file_metadata_presult() noexcept {
+ThriftHiveMetastore_cm_recycle_presult::~ThriftHiveMetastore_cm_recycle_presult() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_put_file_metadata_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_cm_recycle_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -50702,6 +50527,14 @@ uint32_t ThriftHiveMetastore_put_file_metadata_presult::read(::apache::thrift::p
           xfer += iprot->skip(ftype);
         }
         break;
+      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;
@@ -50715,11 +50548,11 @@ uint32_t ThriftHiveMetastore_put_file_metadata_presult::read(::apache::thrift::p
 }
 
 
-ThriftHiveMetastore_clear_file_metadata_args::~ThriftHiveMetastore_clear_file_metadata_args() noexcept {
+ThriftHiveMetastore_get_file_metadata_by_expr_args::~ThriftHiveMetastore_get_file_metadata_by_expr_args() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_clear_file_metadata_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -50760,10 +50593,10 @@ uint32_t ThriftHiveMetastore_clear_file_metadata_args::read(::apache::thrift::pr
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_clear_file_metadata_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_clear_file_metadata_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_file_metadata_by_expr_args");
 
   xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1);
   xfer += this->req.write(oprot);
@@ -50775,14 +50608,14 @@ uint32_t ThriftHiveMetastore_clear_file_metadata_args::write(::apache::thrift::p
 }
 
 
-ThriftHiveMetastore_clear_file_metadata_pargs::~ThriftHiveMetastore_clear_file_metadata_pargs() noexcept {
+ThriftHiveMetastore_get_file_metadata_by_expr_pargs::~ThriftHiveMetastore_get_file_metadata_by_expr_pargs() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_clear_file_metadata_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_clear_file_metadata_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_file_metadata_by_expr_pargs");
 
   xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1);
   xfer += (*(this->req)).write(oprot);
@@ -50794,11 +50627,11 @@ uint32_t ThriftHiveMetastore_clear_file_metadata_pargs::write(::apache::thrift::
 }
 
 
-ThriftHiveMetastore_clear_file_metadata_result::~ThriftHiveMetastore_clear_file_metadata_result() noexcept {
+ThriftHiveMetastore_get_file_metadata_by_expr_result::~ThriftHiveMetastore_get_file_metadata_by_expr_result() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_clear_file_metadata_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -50839,11 +50672,11 @@ uint32_t ThriftHiveMetastore_clear_file_metadata_result::read(::apache::thrift::
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_clear_file_metadata_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_clear_file_metadata_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_file_metadata_by_expr_result");
 
   if (this->__isset.success) {
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
@@ -50856,11 +50689,11 @@ uint32_t ThriftHiveMetastore_clear_file_metadata_result::write(::apache::thrift:
 }
 
 
-ThriftHiveMetastore_clear_file_metadata_presult::~ThriftHiveMetastore_clear_file_metadata_presult() noexcept {
+ThriftHiveMetastore_get_file_metadata_by_expr_presult::~ThriftHiveMetastore_get_file_metadata_by_expr_presult() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_clear_file_metadata_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_file_metadata_by_expr_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -50902,11 +50735,11 @@ uint32_t ThriftHiveMetastore_clear_file_metadata_presult::read(::apache::thrift:
 }
 
 
-ThriftHiveMetastore_cache_file_metadata_args::~ThriftHiveMetastore_cache_file_metadata_args() noexcept {
+ThriftHiveMetastore_get_file_metadata_args::~ThriftHiveMetastore_get_file_metadata_args() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_cache_file_metadata_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_file_metadata_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -50947,10 +50780,10 @@ uint32_t ThriftHiveMetastore_cache_file_metadata_args::read(::apache::thrift::pr
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_cache_file_metadata_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_file_metadata_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_cache_file_metadata_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_file_metadata_args");
 
   xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1);
   xfer += this->req.write(oprot);
@@ -50962,14 +50795,14 @@ uint32_t ThriftHiveMetastore_cache_file_metadata_args::write(::apache::thrift::p
 }
 
 
-ThriftHiveMetastore_cache_file_metadata_pargs::~ThriftHiveMetastore_cache_file_metadata_pargs() noexcept {
+ThriftHiveMetastore_get_file_metadata_pargs::~ThriftHiveMetastore_get_file_metadata_pargs() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_cache_file_metadata_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_file_metadata_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_cache_file_metadata_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_file_metadata_pargs");
 
   xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1);
   xfer += (*(this->req)).write(oprot);
@@ -50981,11 +50814,11 @@ uint32_t ThriftHiveMetastore_cache_file_metadata_pargs::write(::apache::thrift::
 }
 
 
-ThriftHiveMetastore_cache_file_metadata_result::~ThriftHiveMetastore_cache_file_metadata_result() noexcept {
+ThriftHiveMetastore_get_file_metadata_result::~ThriftHiveMetastore_get_file_metadata_result() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_cache_file_metadata_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_file_metadata_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -51026,11 +50859,11 @@ uint32_t ThriftHiveMetastore_cache_file_metadata_result::read(::apache::thrift::
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_cache_file_metadata_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_file_metadata_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_cache_file_metadata_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_file_metadata_result");
 
   if (this->__isset.success) {
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
@@ -51043,11 +50876,11 @@ uint32_t ThriftHiveMetastore_cache_file_metadata_result::write(::apache::thrift:
 }
 
 
-ThriftHiveMetastore_cache_file_metadata_presult::~ThriftHiveMetastore_cache_file_metadata_presult() noexcept {
+ThriftHiveMetastore_get_file_metadata_presult::~ThriftHiveMetastore_get_file_metadata_presult() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_cache_file_metadata_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_file_metadata_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -51089,11 +50922,11 @@ uint32_t ThriftHiveMetastore_cache_file_metadata_presult::read(::apache::thrift:
 }
 
 
-ThriftHiveMetastore_get_metastore_db_uuid_args::~ThriftHiveMetastore_get_metastore_db_uuid_args() noexcept {
+ThriftHiveMetastore_put_file_metadata_args::~ThriftHiveMetastore_put_file_metadata_args() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_metastore_db_uuid_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_put_file_metadata_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -51112,7 +50945,20 @@ uint32_t ThriftHiveMetastore_get_metastore_db_uuid_args::read(::apache::thrift::
     if (ftype == ::apache::thrift::protocol::T_STOP) {
       break;
     }
-    xfer += iprot->skip(ftype);
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->req.read(iprot);
+          this->__isset.req = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
     xfer += iprot->readFieldEnd();
   }
 
@@ -51121,10 +50967,14 @@ uint32_t ThriftHiveMetastore_get_metastore_db_uuid_args::read(::apache::thrift::
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_metastore_db_uuid_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_put_file_metadata_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_metastore_db_uuid_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_put_file_metadata_args");
+
+  xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += this->req.write(oprot);
+  xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
@@ -51132,14 +50982,18 @@ uint32_t ThriftHiveMetastore_get_metastore_db_uuid_args::write(::apache::thrift:
 }
 
 
-ThriftHiveMetastore_get_metastore_db_uuid_pargs::~ThriftHiveMetastore_get_metastore_db_uuid_pargs() noexcept {
+ThriftHiveMetastore_put_file_metadata_pargs::~ThriftHiveMetastore_put_file_metadata_pargs() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_metastore_db_uuid_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_put_file_metadata_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_metastore_db_uuid_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_put_file_metadata_pargs");
+
+  xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += (*(this->req)).write(oprot);
+  xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
@@ -51147,11 +51001,11 @@ uint32_t ThriftHiveMetastore_get_metastore_db_uuid_pargs::write(::apache::thrift
 }
 
 
-ThriftHiveMetastore_get_metastore_db_uuid_result::~ThriftHiveMetastore_get_metastore_db_uuid_result() noexcept {
+ThriftHiveMetastore_put_file_metadata_result::~ThriftHiveMetastore_put_file_metadata_result() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_metastore_db_uuid_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_put_file_metadata_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -51173,17 +51027,9 @@ uint32_t ThriftHiveMetastore_get_metastore_db_uuid_result::read(::apache::thrift
     switch (fid)
     {
       case 0:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->success);
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 1:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o1.read(iprot);
-          this->__isset.o1 = true;
+          xfer += this->success.read(iprot);
+          this->__isset.success = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -51200,19 +51046,15 @@ uint32_t ThriftHiveMetastore_get_metastore_db_uuid_result::read(::apache::thrift
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_metastore_db_uuid_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_put_file_metadata_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_metastore_db_uuid_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_put_file_metadata_result");
 
   if (this->__isset.success) {
-    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRING, 0);
-    xfer += oprot->writeString(this->success);
-    xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.o1) {
-    xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
-    xfer += this->o1.write(oprot);
+    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
+    xfer += this->success.write(oprot);
     xfer += oprot->writeFieldEnd();
   }
   xfer += oprot->writeFieldStop();
@@ -51221,11 +51063,11 @@ uint32_t ThriftHiveMetastore_get_metastore_db_uuid_result::write(::apache::thrif
 }
 
 
-ThriftHiveMetastore_get_metastore_db_uuid_presult::~ThriftHiveMetastore_get_metastore_db_uuid_presult() noexcept {
+ThriftHiveMetastore_put_file_metadata_presult::~ThriftHiveMetastore_put_file_metadata_presult() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_metastore_db_uuid_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_put_file_metadata_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -51247,17 +51089,9 @@ uint32_t ThriftHiveMetastore_get_metastore_db_uuid_presult::read(::apache::thrif
     switch (fid)
     {
       case 0:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString((*(this->success)));
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 1:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o1.read(iprot);
-          this->__isset.o1 = true;
+          xfer += (*(this->success)).read(iprot);
+          this->__isset.success = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -51275,11 +51109,11 @@ uint32_t ThriftHiveMetastore_get_metastore_db_uuid_presult::read(::apache::thrif
 }
 
 
-ThriftHiveMetastore_create_resource_plan_args::~ThriftHiveMetastore_create_resource_plan_args() noexcept {
+ThriftHiveMetastore_clear_file_metadata_args::~ThriftHiveMetastore_clear_file_metadata_args() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_create_resource_plan_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_clear_file_metadata_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -51302,8 +51136,8 @@ uint32_t ThriftHiveMetastore_create_resource_plan_args::read(::apache::thrift::p
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->request.read(iprot);
-          this->__isset.request = true;
+          xfer += this->req.read(iprot);
+          this->__isset.req = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -51320,13 +51154,13 @@ uint32_t ThriftHiveMetastore_create_resource_plan_args::read(::apache::thrift::p
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_create_resource_plan_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_clear_file_metadata_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_resource_plan_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_clear_file_metadata_args");
 
-  xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1);
-  xfer += this->request.write(oprot);
+  xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += this->req.write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -51335,17 +51169,17 @@ uint32_t ThriftHiveMetastore_create_resource_plan_args::write(::apache::thrift::
 }
 
 
-ThriftHiveMetastore_create_resource_plan_pargs::~ThriftHiveMetastore_create_resource_plan_pargs() noexcept {
+ThriftHiveMetastore_clear_file_metadata_pargs::~ThriftHiveMetastore_clear_file_metadata_pargs() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_create_resource_plan_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_clear_file_metadata_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_resource_plan_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_clear_file_metadata_pargs");
 
-  xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1);
-  xfer += (*(this->request)).write(oprot);
+  xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += (*(this->req)).write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -51354,11 +51188,11 @@ uint32_t ThriftHiveMetastore_create_resource_plan_pargs::write(::apache::thrift:
 }
 
 
-ThriftHiveMetastore_create_resource_plan_result::~ThriftHiveMetastore_create_resource_plan_result() noexcept {
+ThriftHiveMetastore_clear_file_metadata_result::~ThriftHiveMetastore_clear_file_metadata_result() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_create_resource_plan_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_clear_file_metadata_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -51387,30 +51221,6 @@ uint32_t ThriftHiveMetastore_create_resource_plan_result::read(::apache::thrift:
           xfer += iprot->skip(ftype);
         }
         break;
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o1.read(iprot);
-          this->__isset.o1 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 2:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o2.read(iprot);
-          this->__isset.o2 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 3:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o3.read(iprot);
-          this->__isset.o3 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -51423,28 +51233,16 @@ uint32_t ThriftHiveMetastore_create_resource_plan_result::read(::apache::thrift:
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_create_resource_plan_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_clear_file_metadata_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_resource_plan_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_clear_file_metadata_result");
 
   if (this->__isset.success) {
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
     xfer += this->success.write(oprot);
     xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.o1) {
-    xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
-    xfer += this->o1.write(oprot);
-    xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.o2) {
-    xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2);
-    xfer += this->o2.write(oprot);
-    xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.o3) {
-    xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3);
-    xfer += this->o3.write(oprot);
-    xfer += oprot->writeFieldEnd();
   }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
@@ -51452,11 +51250,11 @@ uint32_t ThriftHiveMetastore_create_resource_plan_result::write(::apache::thrift
 }
 
 
-ThriftHiveMetastore_create_resource_plan_presult::~ThriftHiveMetastore_create_resource_plan_presult() noexcept {
+ThriftHiveMetastore_clear_file_metadata_presult::~ThriftHiveMetastore_clear_file_metadata_presult() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_create_resource_plan_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_clear_file_metadata_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -51485,30 +51283,6 @@ uint32_t ThriftHiveMetastore_create_resource_plan_presult::read(::apache::thrift
           xfer += iprot->skip(ftype);
         }
         break;
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o1.read(iprot);
-          this->__isset.o1 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 2:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o2.read(iprot);
-          this->__isset.o2 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 3:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o3.read(iprot);
-          this->__isset.o3 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -51522,11 +51296,11 @@ uint32_t ThriftHiveMetastore_create_resource_plan_presult::read(::apache::thrift
 }
 
 
-ThriftHiveMetastore_get_resource_plan_args::~ThriftHiveMetastore_get_resource_plan_args() noexcept {
+ThriftHiveMetastore_cache_file_metadata_args::~ThriftHiveMetastore_cache_file_metadata_args() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_resource_plan_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_cache_file_metadata_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -51549,8 +51323,8 @@ uint32_t ThriftHiveMetastore_get_resource_plan_args::read(::apache::thrift::prot
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->request.read(iprot);
-          this->__isset.request = true;
+          xfer += this->req.read(iprot);
+          this->__isset.req = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -51567,13 +51341,13 @@ uint32_t ThriftHiveMetastore_get_resource_plan_args::read(::apache::thrift::prot
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_resource_plan_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_cache_file_metadata_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_resource_plan_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_cache_file_metadata_args");
 
-  xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1);
-  xfer += this->request.write(oprot);
+  xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += this->req.write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -51582,17 +51356,17 @@ uint32_t ThriftHiveMetastore_get_resource_plan_args::write(::apache::thrift::pro
 }
 
 
-ThriftHiveMetastore_get_resource_plan_pargs::~ThriftHiveMetastore_get_resource_plan_pargs() noexcept {
+ThriftHiveMetastore_cache_file_metadata_pargs::~ThriftHiveMetastore_cache_file_metadata_pargs() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_resource_plan_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_cache_file_metadata_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_resource_plan_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_cache_file_metadata_pargs");
 
-  xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1);
-  xfer += (*(this->request)).write(oprot);
+  xfer += oprot->writeFieldBegin("req", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += (*(this->req)).write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -51601,11 +51375,11 @@ uint32_t ThriftHiveMetastore_get_resource_plan_pargs::write(::apache::thrift::pr
 }
 
 
-ThriftHiveMetastore_get_resource_plan_result::~ThriftHiveMetastore_get_resource_plan_result() noexcept {
+ThriftHiveMetastore_cache_file_metadata_result::~ThriftHiveMetastore_cache_file_metadata_result() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_resource_plan_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_cache_file_metadata_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -51634,22 +51408,6 @@ uint32_t ThriftHiveMetastore_get_resource_plan_result::read(::apache::thrift::pr
           xfer += iprot->skip(ftype);
         }
         break;
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o1.read(iprot);
-          this->__isset.o1 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 2:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o2.read(iprot);
-          this->__isset.o2 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -51662,24 +51420,16 @@ uint32_t ThriftHiveMetastore_get_resource_plan_result::read(::apache::thrift::pr
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_resource_plan_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_cache_file_metadata_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_resource_plan_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_cache_file_metadata_result");
 
   if (this->__isset.success) {
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
     xfer += this->success.write(oprot);
     xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.o1) {
-    xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
-    xfer += this->o1.write(oprot);
-    xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.o2) {
-    xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2);
-    xfer += this->o2.write(oprot);
-    xfer += oprot->writeFieldEnd();
   }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
@@ -51687,11 +51437,11 @@ uint32_t ThriftHiveMetastore_get_resource_plan_result::write(::apache::thrift::p
 }
 
 
-ThriftHiveMetastore_get_resource_plan_presult::~ThriftHiveMetastore_get_resource_plan_presult() noexcept {
+ThriftHiveMetastore_cache_file_metadata_presult::~ThriftHiveMetastore_cache_file_metadata_presult() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_resource_plan_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_cache_file_metadata_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -51720,22 +51470,6 @@ uint32_t ThriftHiveMetastore_get_resource_plan_presult::read(::apache::thrift::p
           xfer += iprot->skip(ftype);
         }
         break;
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o1.read(iprot);
-          this->__isset.o1 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 2:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o2.read(iprot);
-          this->__isset.o2 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -51749,11 +51483,11 @@ uint32_t ThriftHiveMetastore_get_resource_plan_presult::read(::apache::thrift::p
 }
 
 
-ThriftHiveMetastore_get_active_resource_plan_args::~ThriftHiveMetastore_get_active_resource_plan_args() noexcept {
+ThriftHiveMetastore_get_metastore_db_uuid_args::~ThriftHiveMetastore_get_metastore_db_uuid_args() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_active_resource_plan_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_metastore_db_uuid_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -51772,20 +51506,7 @@ uint32_t ThriftHiveMetastore_get_active_resource_plan_args::read(::apache::thrif
     if (ftype == ::apache::thrift::protocol::T_STOP) {
       break;
     }
-    switch (fid)
-    {
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->request.read(iprot);
-          this->__isset.request = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
+    xfer += iprot->skip(ftype);
     xfer += iprot->readFieldEnd();
   }
 
@@ -51794,14 +51515,10 @@ uint32_t ThriftHiveMetastore_get_active_resource_plan_args::read(::apache::thrif
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_active_resource_plan_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_metastore_db_uuid_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_active_resource_plan_args");
-
-  xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1);
-  xfer += this->request.write(oprot);
-  xfer += oprot->writeFieldEnd();
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_metastore_db_uuid_args");
 
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
@@ -51809,18 +51526,14 @@ uint32_t ThriftHiveMetastore_get_active_resource_plan_args::write(::apache::thri
 }
 
 
-ThriftHiveMetastore_get_active_resource_plan_pargs::~ThriftHiveMetastore_get_active_resource_plan_pargs() noexcept {
+ThriftHiveMetastore_get_metastore_db_uuid_pargs::~ThriftHiveMetastore_get_metastore_db_uuid_pargs() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_active_resource_plan_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_metastore_db_uuid_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_active_resource_plan_pargs");
-
-  xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1);
-  xfer += (*(this->request)).write(oprot);
-  xfer += oprot->writeFieldEnd();
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_metastore_db_uuid_pargs");
 
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
@@ -51828,11 +51541,11 @@ uint32_t ThriftHiveMetastore_get_active_resource_plan_pargs::write(::apache::thr
 }
 
 
-ThriftHiveMetastore_get_active_resource_plan_result::~ThriftHiveMetastore_get_active_resource_plan_result() noexcept {
+ThriftHiveMetastore_get_metastore_db_uuid_result::~ThriftHiveMetastore_get_metastore_db_uuid_result() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_active_resource_plan_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_metastore_db_uuid_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -51854,8 +51567,8 @@ uint32_t ThriftHiveMetastore_get_active_resource_plan_result::read(::apache::thr
     switch (fid)
     {
       case 0:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->success.read(iprot);
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->success);
           this->__isset.success = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -51863,8 +51576,8 @@ uint32_t ThriftHiveMetastore_get_active_resource_plan_result::read(::apache::thr
         break;
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o2.read(iprot);
-          this->__isset.o2 = true;
+          xfer += this->o1.read(iprot);
+          this->__isset.o1 = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -51881,19 +51594,19 @@ uint32_t ThriftHiveMetastore_get_active_resource_plan_result::read(::apache::thr
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_active_resource_plan_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_metastore_db_uuid_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_active_resource_plan_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_metastore_db_uuid_result");
 
   if (this->__isset.success) {
-    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
-    xfer += this->success.write(oprot);
+    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRING, 0);
+    xfer += oprot->writeString(this->success);
     xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.o2) {
-    xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 1);
-    xfer += this->o2.write(oprot);
+  } else 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();
@@ -51902,11 +51615,11 @@ uint32_t ThriftHiveMetastore_get_active_resource_plan_result::write(::apache::th
 }
 
 
-ThriftHiveMetastore_get_active_resource_plan_presult::~ThriftHiveMetastore_get_active_resource_plan_presult() noexcept {
+ThriftHiveMetastore_get_metastore_db_uuid_presult::~ThriftHiveMetastore_get_metastore_db_uuid_presult() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_active_resource_plan_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_metastore_db_uuid_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -51928,8 +51641,8 @@ uint32_t ThriftHiveMetastore_get_active_resource_plan_presult::read(::apache::th
     switch (fid)
     {
       case 0:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += (*(this->success)).read(iprot);
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString((*(this->success)));
           this->__isset.success = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -51937,8 +51650,8 @@ uint32_t ThriftHiveMetastore_get_active_resource_plan_presult::read(::apache::th
         break;
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o2.read(iprot);
-          this->__isset.o2 = true;
+          xfer += this->o1.read(iprot);
+          this->__isset.o1 = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -51956,11 +51669,11 @@ uint32_t ThriftHiveMetastore_get_active_resource_plan_presult::read(::apache::th
 }
 
 
-ThriftHiveMetastore_get_all_resource_plans_args::~ThriftHiveMetastore_get_all_resource_plans_args() noexcept {
+ThriftHiveMetastore_create_resource_plan_args::~ThriftHiveMetastore_create_resource_plan_args() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_all_resource_plans_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_create_resource_plan_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -52001,10 +51714,10 @@ uint32_t ThriftHiveMetastore_get_all_resource_plans_args::read(::apache::thrift:
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_all_resource_plans_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_create_resource_plan_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_resource_plans_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_resource_plan_args");
 
   xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1);
   xfer += this->request.write(oprot);
@@ -52016,14 +51729,14 @@ uint32_t ThriftHiveMetastore_get_all_resource_plans_args::write(::apache::thrift
 }
 
 
-ThriftHiveMetastore_get_all_resource_plans_pargs::~ThriftHiveMetastore_get_all_resource_plans_pargs() noexcept {
+ThriftHiveMetastore_create_resource_plan_pargs::~ThriftHiveMetastore_create_resource_plan_pargs() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_all_resource_plans_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_create_resource_plan_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_resource_plans_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_resource_plan_pargs");
 
   xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1);
   xfer += (*(this->request)).write(oprot);
@@ -52035,11 +51748,11 @@ uint32_t ThriftHiveMetastore_get_all_resource_plans_pargs::write(::apache::thrif
 }
 
 
-ThriftHiveMetastore_get_all_resource_plans_result::~ThriftHiveMetastore_get_all_resource_plans_result() noexcept {
+ThriftHiveMetastore_create_resource_plan_result::~ThriftHiveMetastore_create_resource_plan_result() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_all_resource_plans_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_create_resource_plan_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -52076,6 +51789,22 @@ uint32_t ThriftHiveMetastore_get_all_resource_plans_result::read(::apache::thrif
           xfer += iprot->skip(ftype);
         }
         break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o2.read(iprot);
+          this->__isset.o2 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o3.read(iprot);
+          this->__isset.o3 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -52088,11 +51817,11 @@ uint32_t ThriftHiveMetastore_get_all_resource_plans_result::read(::apache::thrif
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_all_resource_plans_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_create_resource_plan_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_resource_plans_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_resource_plan_result");
 
   if (this->__isset.success) {
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
@@ -52102,6 +51831,14 @@ uint32_t ThriftHiveMetastore_get_all_resource_plans_result::write(::apache::thri
     xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
     xfer += this->o1.write(oprot);
     xfer += oprot->writeFieldEnd();
+  } else if (this->__isset.o2) {
+    xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2);
+    xfer += this->o2.write(oprot);
+    xfer += oprot->writeFieldEnd();
+  } else if (this->__isset.o3) {
+    xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3);
+    xfer += this->o3.write(oprot);
+    xfer += oprot->writeFieldEnd();
   }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
@@ -52109,11 +51846,11 @@ uint32_t ThriftHiveMetastore_get_all_resource_plans_result::write(::apache::thri
 }
 
 
-ThriftHiveMetastore_get_all_resource_plans_presult::~ThriftHiveMetastore_get_all_resource_plans_presult() noexcept {
+ThriftHiveMetastore_create_resource_plan_presult::~ThriftHiveMetastore_create_resource_plan_presult() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_get_all_resource_plans_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_create_resource_plan_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -52150,6 +51887,22 @@ uint32_t ThriftHiveMetastore_get_all_resource_plans_presult::read(::apache::thri
           xfer += iprot->skip(ftype);
         }
         break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o2.read(iprot);
+          this->__isset.o2 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o3.read(iprot);
+          this->__isset.o3 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -52163,11 +51916,11 @@ uint32_t ThriftHiveMetastore_get_all_resource_plans_presult::read(::apache::thri
 }
 
 
-ThriftHiveMetastore_alter_resource_plan_args::~ThriftHiveMetastore_alter_resource_plan_args() noexcept {
+ThriftHiveMetastore_get_resource_plan_args::~ThriftHiveMetastore_get_resource_plan_args() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_alter_resource_plan_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_resource_plan_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -52208,10 +51961,10 @@ uint32_t ThriftHiveMetastore_alter_resource_plan_args::read(::apache::thrift::pr
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_alter_resource_plan_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_resource_plan_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_resource_plan_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_resource_plan_args");
 
   xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1);
   xfer += this->request.write(oprot);
@@ -52223,14 +51976,14 @@ uint32_t ThriftHiveMetastore_alter_resource_plan_args::write(::apache::thrift::p
 }
 
 
-ThriftHiveMetastore_alter_resource_plan_pargs::~ThriftHiveMetastore_alter_resource_plan_pargs() noexcept {
+ThriftHiveMetastore_get_resource_plan_pargs::~ThriftHiveMetastore_get_resource_plan_pargs() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_alter_resource_plan_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_resource_plan_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_resource_plan_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_resource_plan_pargs");
 
   xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1);
   xfer += (*(this->request)).write(oprot);
@@ -52242,11 +51995,11 @@ uint32_t ThriftHiveMetastore_alter_resource_plan_pargs::write(::apache::thrift::
 }
 
 
-ThriftHiveMetastore_alter_resource_plan_result::~ThriftHiveMetastore_alter_resource_plan_result() noexcept {
+ThriftHiveMetastore_get_resource_plan_result::~ThriftHiveMetastore_get_resource_plan_result() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_alter_resource_plan_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_resource_plan_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -52291,14 +52044,6 @@ uint32_t ThriftHiveMetastore_alter_resource_plan_result::read(::apache::thrift::
           xfer += iprot->skip(ftype);
         }
         break;
-      case 3:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o3.read(iprot);
-          this->__isset.o3 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -52311,11 +52056,11 @@ uint32_t ThriftHiveMetastore_alter_resource_plan_result::read(::apache::thrift::
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_alter_resource_plan_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_resource_plan_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_resource_plan_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_resource_plan_result");
 
   if (this->__isset.success) {
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
@@ -52329,10 +52074,6 @@ uint32_t ThriftHiveMetastore_alter_resource_plan_result::write(::apache::thrift:
     xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2);
     xfer += this->o2.write(oprot);
     xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.o3) {
-    xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3);
-    xfer += this->o3.write(oprot);
-    xfer += oprot->writeFieldEnd();
   }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
@@ -52340,11 +52081,11 @@ uint32_t ThriftHiveMetastore_alter_resource_plan_result::write(::apache::thrift:
 }
 
 
-ThriftHiveMetastore_alter_resource_plan_presult::~ThriftHiveMetastore_alter_resource_plan_presult() noexcept {
+ThriftHiveMetastore_get_resource_plan_presult::~ThriftHiveMetastore_get_resource_plan_presult() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_alter_resource_plan_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_resource_plan_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -52389,14 +52130,6 @@ uint32_t ThriftHiveMetastore_alter_resource_plan_presult::read(::apache::thrift:
           xfer += iprot->skip(ftype);
         }
         break;
-      case 3:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o3.read(iprot);
-          this->__isset.o3 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -52410,11 +52143,11 @@ uint32_t ThriftHiveMetastore_alter_resource_plan_presult::read(::apache::thrift:
 }
 
 
-ThriftHiveMetastore_validate_resource_plan_args::~ThriftHiveMetastore_validate_resource_plan_args() noexcept {
+ThriftHiveMetastore_get_active_resource_plan_args::~ThriftHiveMetastore_get_active_resource_plan_args() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_validate_resource_plan_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_active_resource_plan_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -52455,10 +52188,10 @@ uint32_t ThriftHiveMetastore_validate_resource_plan_args::read(::apache::thrift:
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_validate_resource_plan_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_active_resource_plan_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_validate_resource_plan_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_active_resource_plan_args");
 
   xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1);
   xfer += this->request.write(oprot);
@@ -52470,14 +52203,14 @@ uint32_t ThriftHiveMetastore_validate_resource_plan_args::write(::apache::thrift
 }
 
 
-ThriftHiveMetastore_validate_resource_plan_pargs::~ThriftHiveMetastore_validate_resource_plan_pargs() noexcept {
+ThriftHiveMetastore_get_active_resource_plan_pargs::~ThriftHiveMetastore_get_active_resource_plan_pargs() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_validate_resource_plan_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_active_resource_plan_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_validate_resource_plan_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_active_resource_plan_pargs");
 
   xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1);
   xfer += (*(this->request)).write(oprot);
@@ -52489,11 +52222,11 @@ uint32_t ThriftHiveMetastore_validate_resource_plan_pargs::write(::apache::thrif
 }
 
 
-ThriftHiveMetastore_validate_resource_plan_result::~ThriftHiveMetastore_validate_resource_plan_result() noexcept {
+ThriftHiveMetastore_get_active_resource_plan_result::~ThriftHiveMetastore_get_active_resource_plan_result() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_validate_resource_plan_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_active_resource_plan_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -52524,14 +52257,6 @@ uint32_t ThriftHiveMetastore_validate_resource_plan_result::read(::apache::thrif
         break;
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o1.read(iprot);
-          this->__isset.o1 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 2:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
           xfer += this->o2.read(iprot);
           this->__isset.o2 = true;
         } else {
@@ -52550,22 +52275,18 @@ uint32_t ThriftHiveMetastore_validate_resource_plan_result::read(::apache::thrif
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_validate_resource_plan_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_active_resource_plan_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_validate_resource_plan_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_active_resource_plan_result");
 
   if (this->__isset.success) {
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
     xfer += this->success.write(oprot);
     xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.o1) {
-    xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
-    xfer += this->o1.write(oprot);
-    xfer += oprot->writeFieldEnd();
   } else if (this->__isset.o2) {
-    xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2);
+    xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 1);
     xfer += this->o2.write(oprot);
     xfer += oprot->writeFieldEnd();
   }
@@ -52575,11 +52296,11 @@ uint32_t ThriftHiveMetastore_validate_resource_plan_result::write(::apache::thri
 }
 
 
-ThriftHiveMetastore_validate_resource_plan_presult::~ThriftHiveMetastore_validate_resource_plan_presult() noexcept {
+ThriftHiveMetastore_get_active_resource_plan_presult::~ThriftHiveMetastore_get_active_resource_plan_presult() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_validate_resource_plan_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_active_resource_plan_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -52610,14 +52331,6 @@ uint32_t ThriftHiveMetastore_validate_resource_plan_presult::read(::apache::thri
         break;
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o1.read(iprot);
-          this->__isset.o1 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 2:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
           xfer += this->o2.read(iprot);
           this->__isset.o2 = true;
         } else {
@@ -52637,11 +52350,11 @@ uint32_t ThriftHiveMetastore_validate_resource_plan_presult::read(::apache::thri
 }
 
 
-ThriftHiveMetastore_drop_resource_plan_args::~ThriftHiveMetastore_drop_resource_plan_args() noexcept {
+ThriftHiveMetastore_get_all_resource_plans_args::~ThriftHiveMetastore_get_all_resource_plans_args() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_resource_plan_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_all_resource_plans_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -52682,10 +52395,10 @@ uint32_t ThriftHiveMetastore_drop_resource_plan_args::read(::apache::thrift::pro
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_drop_resource_plan_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_all_resource_plans_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_resource_plan_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_resource_plans_args");
 
   xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1);
   xfer += this->request.write(oprot);
@@ -52697,14 +52410,14 @@ uint32_t ThriftHiveMetastore_drop_resource_plan_args::write(::apache::thrift::pr
 }
 
 
-ThriftHiveMetastore_drop_resource_plan_pargs::~ThriftHiveMetastore_drop_resource_plan_pargs() noexcept {
+ThriftHiveMetastore_get_all_resource_plans_pargs::~ThriftHiveMetastore_get_all_resource_plans_pargs() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_resource_plan_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_all_resource_plans_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_resource_plan_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_resource_plans_pargs");
 
   xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1);
   xfer += (*(this->request)).write(oprot);
@@ -52716,11 +52429,11 @@ uint32_t ThriftHiveMetastore_drop_resource_plan_pargs::write(::apache::thrift::p
 }
 
 
-ThriftHiveMetastore_drop_resource_plan_result::~ThriftHiveMetastore_drop_resource_plan_result() noexcept {
+ThriftHiveMetastore_get_all_resource_plans_result::~ThriftHiveMetastore_get_all_resource_plans_result() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_resource_plan_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_all_resource_plans_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -52757,22 +52470,6 @@ uint32_t ThriftHiveMetastore_drop_resource_plan_result::read(::apache::thrift::p
           xfer += iprot->skip(ftype);
         }
         break;
-      case 2:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o2.read(iprot);
-          this->__isset.o2 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 3:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o3.read(iprot);
-          this->__isset.o3 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -52785,11 +52482,11 @@ uint32_t ThriftHiveMetastore_drop_resource_plan_result::read(::apache::thrift::p
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_drop_resource_plan_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_all_resource_plans_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_resource_plan_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_resource_plans_result");
 
   if (this->__isset.success) {
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
@@ -52799,14 +52496,6 @@ uint32_t ThriftHiveMetastore_drop_resource_plan_result::write(::apache::thrift::
     xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
     xfer += this->o1.write(oprot);
     xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.o2) {
-    xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2);
-    xfer += this->o2.write(oprot);
-    xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.o3) {
-    xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3);
-    xfer += this->o3.write(oprot);
-    xfer += oprot->writeFieldEnd();
   }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
@@ -52814,11 +52503,11 @@ uint32_t ThriftHiveMetastore_drop_resource_plan_result::write(::apache::thrift::
 }
 
 
-ThriftHiveMetastore_drop_resource_plan_presult::~ThriftHiveMetastore_drop_resource_plan_presult() noexcept {
+ThriftHiveMetastore_get_all_resource_plans_presult::~ThriftHiveMetastore_get_all_resource_plans_presult() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_resource_plan_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_all_resource_plans_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -52855,22 +52544,6 @@ uint32_t ThriftHiveMetastore_drop_resource_plan_presult::read(::apache::thrift::
           xfer += iprot->skip(ftype);
         }
         break;
-      case 2:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o2.read(iprot);
-          this->__isset.o2 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 3:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o3.read(iprot);
-          this->__isset.o3 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -52884,11 +52557,11 @@ uint32_t ThriftHiveMetastore_drop_resource_plan_presult::read(::apache::thrift::
 }
 
 
-ThriftHiveMetastore_create_wm_trigger_args::~ThriftHiveMetastore_create_wm_trigger_args() noexcept {
+ThriftHiveMetastore_alter_resource_plan_args::~ThriftHiveMetastore_alter_resource_plan_args() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_create_wm_trigger_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_alter_resource_plan_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -52929,10 +52602,10 @@ uint32_t ThriftHiveMetastore_create_wm_trigger_args::read(::apache::thrift::prot
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_create_wm_trigger_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_alter_resource_plan_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_wm_trigger_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_resource_plan_args");
 
   xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1);
   xfer += this->request.write(oprot);
@@ -52944,14 +52617,14 @@ uint32_t ThriftHiveMetastore_create_wm_trigger_args::write(::apache::thrift::pro
 }
 
 
-ThriftHiveMetastore_create_wm_trigger_pargs::~ThriftHiveMetastore_create_wm_trigger_pargs() noexcept {
+ThriftHiveMetastore_alter_resource_plan_pargs::~ThriftHiveMetastore_alter_resource_plan_pargs() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_create_wm_trigger_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_alter_resource_plan_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_wm_trigger_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_resource_plan_pargs");
 
   xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1);
   xfer += (*(this->request)).write(oprot);
@@ -52963,11 +52636,11 @@ uint32_t ThriftHiveMetastore_create_wm_trigger_pargs::write(::apache::thrift::pr
 }
 
 
-ThriftHiveMetastore_create_wm_trigger_result::~ThriftHiveMetastore_create_wm_trigger_result() noexcept {
+ThriftHiveMetastore_alter_resource_plan_result::~ThriftHiveMetastore_alter_resource_plan_result() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_create_wm_trigger_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_alter_resource_plan_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -53020,14 +52693,6 @@ uint32_t ThriftHiveMetastore_create_wm_trigger_result::read(::apache::thrift::pr
           xfer += iprot->skip(ftype);
         }
         break;
-      case 4:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o4.read(iprot);
-          this->__isset.o4 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -53040,11 +52705,11 @@ uint32_t ThriftHiveMetastore_create_wm_trigger_result::read(::apache::thrift::pr
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_create_wm_trigger_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_alter_resource_plan_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_wm_trigger_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_resource_plan_result");
 
   if (this->__isset.success) {
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
@@ -53062,10 +52727,6 @@ uint32_t ThriftHiveMetastore_create_wm_trigger_result::write(::apache::thrift::p
     xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3);
     xfer += this->o3.write(oprot);
     xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.o4) {
-    xfer += oprot->writeFieldBegin("o4", ::apache::thrift::protocol::T_STRUCT, 4);
-    xfer += this->o4.write(oprot);
-    xfer += oprot->writeFieldEnd();
   }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
@@ -53073,11 +52734,11 @@ uint32_t ThriftHiveMetastore_create_wm_trigger_result::write(::apache::thrift::p
 }
 
 
-ThriftHiveMetastore_create_wm_trigger_presult::~ThriftHiveMetastore_create_wm_trigger_presult() noexcept {
+ThriftHiveMetastore_alter_resource_plan_presult::~ThriftHiveMetastore_alter_resource_plan_presult() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_create_wm_trigger_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_alter_resource_plan_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -53130,14 +52791,6 @@ uint32_t ThriftHiveMetastore_create_wm_trigger_presult::read(::apache::thrift::p
           xfer += iprot->skip(ftype);
         }
         break;
-      case 4:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o4.read(iprot);
-          this->__isset.o4 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -53151,11 +52804,11 @@ uint32_t ThriftHiveMetastore_create_wm_trigger_presult::read(::apache::thrift::p
 }
 
 
-ThriftHiveMetastore_alter_wm_trigger_args::~ThriftHiveMetastore_alter_wm_trigger_args() noexcept {
+ThriftHiveMetastore_validate_resource_plan_args::~ThriftHiveMetastore_validate_resource_plan_args() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_alter_wm_trigger_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_validate_resource_plan_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -53196,10 +52849,10 @@ uint32_t ThriftHiveMetastore_alter_wm_trigger_args::read(::apache::thrift::proto
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_alter_wm_trigger_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_validate_resource_plan_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_wm_trigger_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_validate_resource_plan_args");
 
   xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1);
   xfer += this->request.write(oprot);
@@ -53211,14 +52864,14 @@ uint32_t ThriftHiveMetastore_alter_wm_trigger_args::write(::apache::thrift::prot
 }
 
 
-ThriftHiveMetastore_alter_wm_trigger_pargs::~ThriftHiveMetastore_alter_wm_trigger_pargs() noexcept {
+ThriftHiveMetastore_validate_resource_plan_pargs::~ThriftHiveMetastore_validate_resource_plan_pargs() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_alter_wm_trigger_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_validate_resource_plan_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_wm_trigger_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_validate_resource_plan_pargs");
 
   xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1);
   xfer += (*(this->request)).write(oprot);
@@ -53230,11 +52883,11 @@ uint32_t ThriftHiveMetastore_alter_wm_trigger_pargs::write(::apache::thrift::pro
 }
 
 
-ThriftHiveMetastore_alter_wm_trigger_result::~ThriftHiveMetastore_alter_wm_trigger_result() noexcept {
+ThriftHiveMetastore_validate_resource_plan_result::~ThriftHiveMetastore_validate_resource_plan_result() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_alter_wm_trigger_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_validate_resource_plan_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -53279,14 +52932,6 @@ uint32_t ThriftHiveMetastore_alter_wm_trigger_result::read(::apache::thrift::pro
           xfer += iprot->skip(ftype);
         }
         break;
-      case 3:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o3.read(iprot);
-          this->__isset.o3 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -53299,11 +52944,11 @@ uint32_t ThriftHiveMetastore_alter_wm_trigger_result::read(::apache::thrift::pro
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_alter_wm_trigger_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_validate_resource_plan_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_wm_trigger_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_validate_resource_plan_result");
 
   if (this->__isset.success) {
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
@@ -53317,10 +52962,6 @@ uint32_t ThriftHiveMetastore_alter_wm_trigger_result::write(::apache::thrift::pr
     xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2);
     xfer += this->o2.write(oprot);
     xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.o3) {
-    xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3);
-    xfer += this->o3.write(oprot);
-    xfer += oprot->writeFieldEnd();
   }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
@@ -53328,11 +52969,11 @@ uint32_t ThriftHiveMetastore_alter_wm_trigger_result::write(::apache::thrift::pr
 }
 
 
-ThriftHiveMetastore_alter_wm_trigger_presult::~ThriftHiveMetastore_alter_wm_trigger_presult() noexcept {
+ThriftHiveMetastore_validate_resource_plan_presult::~ThriftHiveMetastore_validate_resource_plan_presult() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_alter_wm_trigger_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_validate_resource_plan_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -53377,14 +53018,6 @@ uint32_t ThriftHiveMetastore_alter_wm_trigger_presult::read(::apache::thrift::pr
           xfer += iprot->skip(ftype);
         }
         break;
-      case 3:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o3.read(iprot);
-          this->__isset.o3 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -53398,11 +53031,11 @@ uint32_t ThriftHiveMetastore_alter_wm_trigger_presult::read(::apache::thrift::pr
 }
 
 
-ThriftHiveMetastore_drop_wm_trigger_args::~ThriftHiveMetastore_drop_wm_trigger_args() noexcept {
+ThriftHiveMetastore_drop_resource_plan_args::~ThriftHiveMetastore_drop_resource_plan_args() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_wm_trigger_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_drop_resource_plan_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -53443,10 +53076,10 @@ uint32_t ThriftHiveMetastore_drop_wm_trigger_args::read(::apache::thrift::protoc
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_drop_wm_trigger_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_drop_resource_plan_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_wm_trigger_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_resource_plan_args");
 
   xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1);
   xfer += this->request.write(oprot);
@@ -53458,14 +53091,14 @@ uint32_t ThriftHiveMetastore_drop_wm_trigger_args::write(::apache::thrift::proto
 }
 
 
-ThriftHiveMetastore_drop_wm_trigger_pargs::~ThriftHiveMetastore_drop_wm_trigger_pargs() noexcept {
+ThriftHiveMetastore_drop_resource_plan_pargs::~ThriftHiveMetastore_drop_resource_plan_pargs() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_wm_trigger_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_drop_resource_plan_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_wm_trigger_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_resource_plan_pargs");
 
   xfer += oprot->writeFieldBegin("request", ::apache::thrift::protocol::T_STRUCT, 1);
   xfer += (*(this->request)).write(oprot);
@@ -53477,11 +53110,11 @@ uint32_t ThriftHiveMetastore_drop_wm_trigger_pargs::write(::apache::thrift::prot
 }
 
 
-ThriftHiveMetastore_drop_wm_trigger_result::~ThriftHiveMetastore_drop_wm_trigger_result() noexcept {
+ThriftHiveMetastore_drop_resource_plan_result::~ThriftHiveMetastore_drop_resource_plan_result() noexcept {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_wm_trigger_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_drop_resource_plan_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -53546,11 +53179,11 @@ uint32_t ThriftHiveMetastore_drop_wm_trigger_result::read(::apache::thrift::prot
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_drop_wm_trigger_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_drop_resource_plan_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_wm_trigger_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_resource_plan_result");
 
   if (this->__isset.success) {
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
@@ -53575,11 +53208,11 @@ uint32_t ThriftHiveMetastore_drop_wm_trigger_result::write(::apache::thrift::pro
 }
 
 
-ThriftHiveMetastore_drop_wm_trigger_presult::~ThriftHiveMetastore_drop_wm_trigger_presult() noexcept {
... 37839 lines suppressed ...