You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ch...@apache.org on 2023/12/22 15:55:15 UTC

(hive) branch master updated: HIVE-27804: Implement batching in getPartition calls which returns partition list along with auth info (Vikram Ahuja, Reviewed by Chinna Rao Lalam)

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

chinnaraol 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 2d6e4a7ef4d HIVE-27804: Implement batching in getPartition calls which returns partition list along with auth info (Vikram Ahuja, Reviewed by Chinna Rao Lalam)
2d6e4a7ef4d is described below

commit 2d6e4a7ef4d9ed6ce997520e5fe3b6841ab5ee31
Author: Vikram Ahuja <vi...@users.noreply.github.com>
AuthorDate: Fri Dec 22 21:25:09 2023 +0530

    HIVE-27804: Implement batching in getPartition calls which returns partition list along with auth info (Vikram Ahuja, Reviewed by Chinna Rao Lalam)
---
 .../org/apache/hadoop/hive/ql/metadata/Hive.java   |  172 +-
 .../hadoop/hive/ql/metadata/PartitionIterable.java |   25 +-
 .../apache/hadoop/hive/ql/exec/PartitionUtil.java  |   28 +
 .../ql/exec/TestGetPartitionAuthWithBatches.java   |  300 +++
 .../hive/ql/exec/TestGetPartitionInBatches.java    |   53 +-
 .../src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp | 2560 +++++++++---------
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp    |  652 ++---
 .../src/gen/thrift/gen-cpp/hive_metastore_types.h  |   10 +-
 .../hive/metastore/api/GetOpenTxnsRequest.java     |   36 +-
 .../api/GetPartitionsPsWithAuthRequest.java        |  204 +-
 .../api/GetPartitionsPsWithAuthResponse.java       |   36 +-
 .../hive/metastore/api/ReplicationMetricList.java  |   36 +-
 .../hive/metastore/api/ThriftHiveMetastore.java    | 2780 ++++++++++----------
 .../gen-php/metastore/GetOpenTxnsRequest.php       |   18 +-
 .../metastore/GetPartitionsPsWithAuthRequest.php   |   52 +-
 .../metastore/GetPartitionsPsWithAuthResponse.php  |   20 +-
 .../gen-php/metastore/ReplicationMetricList.php    |   20 +-
 .../ThriftHiveMetastore_add_partitions_args.php    |   20 +-
 ...riftHiveMetastore_add_partitions_pspec_args.php |   20 +-
 ...Metastore_add_write_ids_to_min_history_args.php |   26 +-
 .../ThriftHiveMetastore_alter_partitions_args.php  |   20 +-
 ...er_partitions_with_environment_context_args.php |   20 +-
 .../ThriftHiveMetastore_append_partition_args.php  |   18 +-
 ...end_partition_with_environment_context_args.php |   18 +-
 ...etastore_create_table_with_constraints_args.php |  120 +-
 .../ThriftHiveMetastore_drop_partition_args.php    |   18 +-
 ...rop_partition_with_environment_context_args.php |   18 +-
 ...ThriftHiveMetastore_exchange_partition_args.php |   26 +-
 ...hriftHiveMetastore_exchange_partitions_args.php |   26 +-
 ...iftHiveMetastore_exchange_partitions_result.php |   20 +-
 ...iveMetastore_find_columns_with_stats_result.php |   18 +-
 ...hriftHiveMetastore_get_all_databases_result.php |   18 +-
 ...erialized_view_objects_for_rewriting_result.php |   20 +-
 ...ThriftHiveMetastore_get_all_packages_result.php |   18 +-
 ...eMetastore_get_all_stored_procedures_result.php |   18 +-
 .../ThriftHiveMetastore_get_all_tables_result.php  |   18 +-
 ...eMetastore_get_all_token_identifiers_result.php |   18 +-
 ...veMetastore_get_all_write_event_info_result.php |   20 +-
 .../ThriftHiveMetastore_get_databases_result.php   |   18 +-
 ...riftHiveMetastore_get_dataconnectors_result.php |   18 +-
 .../ThriftHiveMetastore_get_fields_result.php      |   20 +-
 ..._get_fields_with_environment_context_result.php |   20 +-
 .../ThriftHiveMetastore_get_functions_result.php   |   18 +-
 .../ThriftHiveMetastore_get_master_keys_result.php |   18 +-
 ...get_materialized_views_for_rewriting_result.php |   18 +-
 ...veMetastore_get_part_specs_by_filter_result.php |   20 +-
 .../ThriftHiveMetastore_get_partition_args.php     |   18 +-
 ...ftHiveMetastore_get_partition_names_ps_args.php |   18 +-
 ...HiveMetastore_get_partition_names_ps_result.php |   18 +-
 ...iveMetastore_get_partition_names_req_result.php |   18 +-
 ...iftHiveMetastore_get_partition_names_result.php |   18 +-
 ...tHiveMetastore_get_partition_with_auth_args.php |   36 +-
 ...tastore_get_partitions_by_filter_req_result.php |   20 +-
 ...veMetastore_get_partitions_by_filter_result.php |   20 +-
 ...tHiveMetastore_get_partitions_by_names_args.php |   18 +-
 ...iveMetastore_get_partitions_by_names_result.php |   20 +-
 .../ThriftHiveMetastore_get_partitions_ps_args.php |   18 +-
 ...hriftHiveMetastore_get_partitions_ps_result.php |   20 +-
 ...eMetastore_get_partitions_ps_with_auth_args.php |   36 +-
 ...etastore_get_partitions_ps_with_auth_result.php |   20 +-
 ...ftHiveMetastore_get_partitions_pspec_result.php |   20 +-
 .../ThriftHiveMetastore_get_partitions_result.php  |   20 +-
 ...HiveMetastore_get_partitions_with_auth_args.php |   18 +-
 ...veMetastore_get_partitions_with_auth_result.php |   20 +-
 .../ThriftHiveMetastore_get_privilege_set_args.php |   18 +-
 .../ThriftHiveMetastore_get_role_names_result.php  |   18 +-
 ...hriftHiveMetastore_get_runtime_stats_result.php |   20 +-
 ...iveMetastore_get_schema_all_versions_result.php |   20 +-
 .../ThriftHiveMetastore_get_schema_result.php      |   20 +-
 ..._get_schema_with_environment_context_result.php |   20 +-
 .../ThriftHiveMetastore_get_table_meta_args.php    |   18 +-
 .../ThriftHiveMetastore_get_table_meta_result.php  |   20 +-
 ...eMetastore_get_table_names_by_filter_result.php |   18 +-
 ...iveMetastore_get_table_objects_by_name_args.php |   18 +-
 ...eMetastore_get_table_objects_by_name_result.php |   20 +-
 ...riftHiveMetastore_get_tables_by_type_result.php |   18 +-
 .../ThriftHiveMetastore_get_tables_ext_result.php  |   20 +-
 .../ThriftHiveMetastore_get_tables_result.php      |   18 +-
 .../ThriftHiveMetastore_get_type_all_result.php    |   28 +-
 ...iveMetastore_isPartitionMarkedForEvent_args.php |   26 +-
 .../ThriftHiveMetastore_list_privileges_result.php |   20 +-
 .../ThriftHiveMetastore_list_roles_result.php      |   20 +-
 ...iftHiveMetastore_markPartitionForEvent_args.php |   26 +-
 ...re_partition_name_has_valid_characters_args.php |   18 +-
 ...HiveMetastore_partition_name_to_spec_result.php |   26 +-
 ...HiveMetastore_partition_name_to_vals_result.php |   18 +-
 .../ThriftHiveMetastore_rename_partition_args.php  |   18 +-
 .../metastore/ThriftHiveMetastore_set_ugi_args.php |   18 +-
 .../ThriftHiveMetastore_set_ugi_result.php         |   18 +-
 .../ThriftHiveMetastore_truncate_table_args.php    |   18 +-
 .../gen-py/hive_metastore/ThriftHiveMetastore.py   | 1056 ++++----
 .../src/gen/thrift/gen-py/hive_metastore/ttypes.py |   70 +-
 .../src/gen/thrift/gen-rb/hive_metastore_types.rb  |    4 +-
 .../src/main/thrift/hive_metastore.thrift          |    3 +-
 .../apache/hadoop/hive/metastore/HMSHandler.java   |    1 +
 .../apache/hadoop/hive/metastore/ObjectStore.java  |   21 +-
 96 files changed, 5187 insertions(+), 4534 deletions(-)

diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
index cca06126369..192fa13ffa1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
@@ -4104,49 +4104,20 @@ private void constructOneLBLocationMap(FileStatus fSta,
   }
 
   /**
-   * get all the partitions that the table has
+   * get all the partitions that the table has along with auth info
    *
    * @param tbl
    *          object for which partition is needed
-   * @return list of partition objects
+   * @return list of partition objects along with auth info
    */
   public List<Partition> getPartitions(Table tbl) throws HiveException {
     PerfLogger perfLogger = SessionState.getPerfLogger();
     perfLogger.perfLogBegin(CLASS_NAME, PerfLogger.HIVE_GET_PARTITIONS);
-
     try {
-      if (tbl.isPartitioned()) {
-        List<org.apache.hadoop.hive.metastore.api.Partition> tParts;
-        try {
-          GetPartitionsPsWithAuthRequest req = new GetPartitionsPsWithAuthRequest();
-          req.setTblName(tbl.getTableName());
-          req.setDbName(tbl.getDbName());
-          req.setUserName(getUserName());
-          req.setMaxParts((short) -1);
-          req.setGroupNames(getGroupNames());
-          if (AcidUtils.isTransactionalTable(tbl)) {
-            ValidWriteIdList validWriteIdList = getValidWriteIdList(tbl.getDbName(), tbl.getTableName());
-            req.setValidWriteIdList(validWriteIdList != null ? validWriteIdList.toString() : null);
-            req.setId(tbl.getTTable().getId());
-          }
-          GetPartitionsPsWithAuthResponse res = getMSC().listPartitionsWithAuthInfoRequest(req);
-          tParts = res.getPartitions();
-
-        } catch (NoSuchObjectException nsoe) {
-          return Lists.newArrayList();
-        } catch (Exception e) {
-          LOG.error("Failed getPartitions", e);
-          throw new HiveException(e);
-        }
-        List<Partition> parts = new ArrayList<>(tParts.size());
-        for (org.apache.hadoop.hive.metastore.api.Partition tpart : tParts) {
-          parts.add(new Partition(tbl, tpart));
-        }
-
-        return parts;
-      } else {
-        return Collections.singletonList(new Partition(tbl));
-      }
+      int batchSize= MetastoreConf.getIntVar(Hive.get().getConf(), MetastoreConf.ConfVars.BATCH_RETRIEVE_MAX);
+      return new ArrayList<>(getAllPartitionsInBatches(tbl, batchSize, DEFAULT_BATCH_DECAYING_FACTOR, MetastoreConf
+                      .getIntVar(Hive.get().getConf(), MetastoreConf.ConfVars.GETPARTITIONS_BATCH_MAX_RETRIES),
+              null, true, getUserName(), getGroupNames()));
     } finally {
       perfLogger.perfLogEnd(CLASS_NAME, PerfLogger.HIVE_GET_PARTITIONS, "HS2-cache");
     }
@@ -4186,14 +4157,33 @@ private void constructOneLBLocationMap(FileStatus fSta,
             Hive.get().getConf(), MetastoreConf.ConfVars.BATCH_RETRIEVE_MAX);
     if (batchSize > 0) {
       return getAllPartitionsInBatches(tbl, batchSize, DEFAULT_BATCH_DECAYING_FACTOR, MetastoreConf.getIntVar(
-              Hive.get().getConf(), MetastoreConf.ConfVars.GETPARTITIONS_BATCH_MAX_RETRIES));
+              Hive.get().getConf(), MetastoreConf.ConfVars.GETPARTITIONS_BATCH_MAX_RETRIES), null, false);
     } else {
       return getAllPartitions(tbl);
     }
   }
 
   public Set<Partition> getAllPartitionsInBatches(Table tbl, int batchSize, int decayingFactor,
-         int maxRetries) throws HiveException {
+       int maxRetries, Map<String, String> partialPartitionSpec, boolean isAuthRequired) throws HiveException {
+    return getAllPartitionsInBatches(tbl, batchSize, decayingFactor, maxRetries, partialPartitionSpec, isAuthRequired,
+            null, null);
+  }
+
+  /**
+   * Main method which fetches the partitions in batches
+   * @param tbl table for which partitions are needed
+   * @param batchSize Number of partitions to be fectehd in one batched call
+   * @param decayingFactor the value by which batchSize decays in the next retry in case it faces an exception
+   * @param maxRetries Number of retries allowed for this operation
+   * @param partialPartitionSpec partialPartitionSpec for the table
+   * @param isAuthRequired If auth information is required along with partitions
+   * @param userName name of the calling user
+   * @param groupNames groups the call
+   * @return list of partition objects
+   */
+  public Set<Partition> getAllPartitionsInBatches(Table tbl, int batchSize, int decayingFactor,
+       int maxRetries, Map<String, String> partialPartitionSpec, boolean isAuthRequired,
+       String userName, List<String> groupNames) throws HiveException {
     if (!tbl.isPartitioned()) {
       return Sets.newHashSet(new Partition(tbl));
     }
@@ -4202,13 +4192,11 @@ private void constructOneLBLocationMap(FileStatus fSta,
             .ExponentiallyDecayingBatchWork<Void>(batchSize, decayingFactor, maxRetries) {
       @Override
       public Void execute(int size) throws HiveException {
-        try {
-          result.clear();
-          new PartitionIterable(Hive.get(), tbl, null, size).forEach(result::add);
-          return null;
-        } catch (HiveException e) {
-          throw e;
-        }
+        result.clear();
+        PartitionIterable partitionIterable = new PartitionIterable(Hive.get(), tbl, partialPartitionSpec, size,
+            isAuthRequired, userName, groupNames);
+        partitionIterable.forEach(result::add);
+        return null;
       }
     };
     try {
@@ -4219,6 +4207,25 @@ private void constructOneLBLocationMap(FileStatus fSta,
     return result;
   }
 
+  public List<Partition> getPartitions(Table tbl, Map<String, String> partialPartSpec,
+       short limit) throws HiveException {
+    PerfLogger perfLogger = SessionState.getPerfLogger();
+    perfLogger.perfLogBegin(CLASS_NAME, PerfLogger.HIVE_GET_PARTITIONS_2);
+    try {
+      // TODO: Implement Batching when limit is >=0
+      if (limit >= 0) {
+        return getPartitionsWithAuth(tbl, partialPartSpec, limit);
+      } else {
+        int batchSize = MetastoreConf.getIntVar(Hive.get().getConf(), MetastoreConf.ConfVars.BATCH_RETRIEVE_MAX);
+        return new ArrayList<>(getAllPartitionsInBatches(tbl, batchSize, DEFAULT_BATCH_DECAYING_FACTOR,
+                MetastoreConf.getIntVar(Hive.get().getConf(), MetastoreConf.ConfVars.GETPARTITIONS_BATCH_MAX_RETRIES),
+                partialPartSpec, true, getUserName(), getGroupNames()));
+      }
+    } finally {
+      perfLogger.perfLogEnd(CLASS_NAME, PerfLogger.HIVE_GET_PARTITIONS_2, "HS2-cache");
+    }
+  }
+
   /**
    * get all the partitions of the table that matches the given partial
    * specification. partition columns whose value is can be anything should be
@@ -4230,36 +4237,30 @@ private void constructOneLBLocationMap(FileStatus fSta,
    * @return list of partition objects
    * @throws HiveException
    */
-  public List<Partition> getPartitions(Table tbl, Map<String, String> partialPartSpec,
-      short limit)
-  throws HiveException {
-    PerfLogger perfLogger = SessionState.getPerfLogger();
-    perfLogger.perfLogBegin(CLASS_NAME, PerfLogger.HIVE_GET_PARTITIONS_2);
-    try {
-      if (!tbl.isPartitioned()) {
-        throw new HiveException(ErrorMsg.TABLE_NOT_PARTITIONED, tbl.getTableName());
-      }
+  private List<Partition> getPartitionsWithAuth(Table tbl, Map<String, String> partialPartSpec,
+                                                short limit)
+          throws HiveException {
+    if (!tbl.isPartitioned()) {
+      throw new HiveException(ErrorMsg.TABLE_NOT_PARTITIONED, tbl.getTableName());
+    }
 
-      List<String> partialPvals = MetaStoreUtils.getPvals(tbl.getPartCols(), partialPartSpec);
+    List<String> partialPvals = MetaStoreUtils.getPvals(tbl.getPartCols(), partialPartSpec);
 
-      List<org.apache.hadoop.hive.metastore.api.Partition> partitions = null;
-      try {
-        String userName = getUserName();
-        partitions = getMSC().listPartitionsWithAuthInfo(tbl.getDbName(), tbl.getTableName(),
-                partialPvals, limit, userName, getGroupNames());
-      } catch (Exception e) {
-        throw new HiveException(e);
-      }
-
-      List<Partition> qlPartitions = new ArrayList<Partition>();
-      for (org.apache.hadoop.hive.metastore.api.Partition p : partitions) {
-        qlPartitions.add(new Partition(tbl, p));
-      }
+    List<org.apache.hadoop.hive.metastore.api.Partition> partitions = null;
+    try {
+      String userName = getUserName();
+      partitions = getMSC().listPartitionsWithAuthInfo(tbl.getDbName(), tbl.getTableName(),
+              partialPvals, limit, userName, getGroupNames());
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
 
-      return qlPartitions;
-    } finally {
-      perfLogger.perfLogEnd(CLASS_NAME, PerfLogger.HIVE_GET_PARTITIONS_2, "HS2-cache");
+    List<Partition> qlPartitions = new ArrayList<Partition>();
+    for (org.apache.hadoop.hive.metastore.api.Partition p : partitions) {
+      qlPartitions.add(new Partition(tbl, p));
     }
+
+    return qlPartitions;
   }
 
   /**
@@ -4385,6 +4386,37 @@ private void constructOneLBLocationMap(FileStatus fSta,
     return partitions;
   }
 
+  public List<Partition> getPartitionsAuthByNames(Table tbl, List<String> partNames, String userName,
+      List<String> groupNames) throws HiveException {
+    if (!tbl.isPartitioned()) {
+      throw new HiveException(ErrorMsg.TABLE_NOT_PARTITIONED, tbl.getTableName());
+    }
+    GetPartitionsPsWithAuthRequest req = new GetPartitionsPsWithAuthRequest();
+    req.setTblName(tbl.getTableName());
+    req.setDbName(tbl.getDbName());
+    req.setUserName(userName);
+    req.setGroupNames(groupNames);
+    req.setPartNames(partNames);
+    if (AcidUtils.isTransactionalTable(tbl)) {
+      ValidWriteIdList validWriteIdList = getValidWriteIdList(tbl.getDbName(), tbl.getTableName());
+      req.setValidWriteIdList(validWriteIdList != null ? validWriteIdList.toString() : null);
+      req.setId(tbl.getTTable().getId());
+    }
+
+    List<org.apache.hadoop.hive.metastore.api.Partition> tParts;
+    try {
+      GetPartitionsPsWithAuthResponse res = getMSC().listPartitionsWithAuthInfoRequest(req);
+      tParts = res.getPartitions();
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
+    List<Partition> parts = new ArrayList<>(tParts.size());
+    for (org.apache.hadoop.hive.metastore.api.Partition tpart : tParts) {
+      parts.add(new Partition(tbl, tpart));
+    }
+    return parts;
+  }
+
   /**
    * Get a list of Partitions by filter.
    * @param tbl The table containing the partitions.
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/PartitionIterable.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/PartitionIterable.java
index 158f31bf7f4..dabfef014ce 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/PartitionIterable.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/PartitionIterable.java
@@ -99,7 +99,11 @@ public class PartitionIterable implements Iterable<Partition> {
           batchCounter++;
         }
         try {
-          batchIter = db.getPartitionsByNames(table, nameBatch, getColStats).iterator();
+          if (isAuthRequired) {
+            batchIter = db.getPartitionsAuthByNames(table, nameBatch, userName, groupNames).iterator();
+          } else {
+            batchIter = db.getPartitionsByNames(table, nameBatch, getColStats).iterator();
+          }
         } catch (HiveException e) {
           throw new RuntimeException(e);
         }
@@ -130,6 +134,9 @@ public class PartitionIterable implements Iterable<Partition> {
   private List<String> partitionNames = null;
   private int batchSize;
   private boolean getColStats = false;
+  private boolean isAuthRequired = false;
+  private String userName;
+  private List<String> groupNames;
 
   /**
    * Dummy constructor, which simply acts as an iterator on an already-present
@@ -150,12 +157,25 @@ public class PartitionIterable implements Iterable<Partition> {
     this(db, table, partialPartitionSpec, batchSize, false);
   }
 
+  public PartitionIterable(Hive db, Table table, Map<String, String> partialPartitionSpec,
+       int batchSize, boolean isAuthRequired, String userName,
+       List<String> groupNames) throws HiveException {
+    this(db, table, partialPartitionSpec, batchSize, false, isAuthRequired, userName, groupNames);
+  }
+
+
   /**
    * Primary constructor that fetches all partitions in a given table, given
    * a Hive object and a table object, and a partial partition spec.
    */
   public PartitionIterable(Hive db, Table table, Map<String, String> partialPartitionSpec,
                            int batchSize, boolean getColStats) throws HiveException {
+    this(db, table, partialPartitionSpec, batchSize, getColStats, false, null, null);
+  }
+
+  private PartitionIterable(Hive db, Table table, Map<String, String> partialPartitionSpec,
+       int batchSize, boolean getColStats, boolean isAuthRequired, String userName,
+       List<String> groupNames) throws HiveException {
     if (batchSize < 1) {
       throw new HiveException("Invalid batch size for partition iterable. Please use a batch size greater than 0");
     }
@@ -165,6 +185,9 @@ public class PartitionIterable implements Iterable<Partition> {
     this.partialPartitionSpec = partialPartitionSpec;
     this.batchSize = batchSize;
     this.getColStats = getColStats;
+    this.isAuthRequired = isAuthRequired;
+    this.userName = userName;
+    this.groupNames = groupNames;
 
     if (this.partialPartitionSpec == null){
       partitionNames = db.getPartitionNames(
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/PartitionUtil.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/PartitionUtil.java
index f449c79760f..59179ebb51c 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/PartitionUtil.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/PartitionUtil.java
@@ -18,19 +18,25 @@
 
 package org.apache.hadoop.hive.ql.exec;
 
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.CheckResult;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder;
 import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat;
 import org.apache.hadoop.hive.ql.stats.StatsUtils;
 import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.hadoop.util.StringUtils;
 
 import java.util.Arrays;
+import java.util.ArrayList;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Set;
 
 import static org.junit.Assert.fail;
@@ -82,4 +88,26 @@ public class PartitionUtil {
         }
         return partsNotInMs;
     }
+
+
+    public static void addPartitions(IMetaStoreClient db, String dbName, String tableName, String location,
+         HiveConf hiveConf, int numPartitions) throws Exception {
+        List<Partition> partitions = new ArrayList<>();
+        for (int i = 0; i < numPartitions; i++) {
+            partitions.add(buildPartition(dbName, tableName, String.valueOf(i), location + "/city=" + i, hiveConf));
+        }
+        db.add_partitions(partitions, true, true);
+    }
+
+    protected static Partition buildPartition(String dbName, String tableName, String value,
+         String location, HiveConf hiveConf) throws MetaException {
+        return new PartitionBuilder()
+                .setDbName(dbName)
+                .setTableName(tableName)
+                .addValue(value)
+                .addCol("test_id", "int", "test col id")
+                .addCol("test_value", "string", "test col value")
+                .setLocation(location)
+                .build(hiveConf);
+    }
 }
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestGetPartitionAuthWithBatches.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestGetPartitionAuthWithBatches.java
new file mode 100644
index 00000000000..191d211d4b7
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestGetPartitionAuthWithBatches.java
@@ -0,0 +1,300 @@
+/*
+ * 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.exec;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+
+import org.apache.hadoop.hive.metastore.api.GetPartitionsPsWithAuthRequest;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.Partition;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.junit.Assert;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+public class TestGetPartitionAuthWithBatches {
+
+    private final String catName = "hive";
+    private final String dbName = "default";
+    private final String tableName = "test_partition_batch_with_auth";
+    private static HiveConf hiveConf;
+    private static HiveMetaStoreClient msc;
+    private static Hive hive;
+    private Table table;
+    private final static int NUM_PARTITIONS = 30;
+    private final static int DECAYING_FACTOR = 2;
+    private final static int MAX_RETRIES = 0;
+    private final static boolean IS_AUTH_REQUIRED = true;
+    private final static String USER_NAME = "username";
+    private final static List<String> GROUP_NAMES = Arrays.asList("Grp1", "Grp2");
+    private final static Map<String, String> PARTIAL_PARTITION_SPEC = null;
+
+    @BeforeClass
+    public static void setupClass() throws HiveException {
+        hiveConf = new HiveConf(TestGetPartitionAuthWithBatches.class);
+        hiveConf.set("hive.security.authorization.enabled", "true");
+        hiveConf.set("hive.security.authorization.manager","org.apache.hadoop.hive.ql.security.authorization.DefaultHiveAuthorizationProvider");
+        hive = Hive.get();
+        SessionState.start(hiveConf);
+        try {
+            msc = new HiveMetaStoreClient(hiveConf);
+        } catch (MetaException e) {
+            throw new HiveException(e);
+        }
+    }
+
+    @Before
+    public void before() throws Exception {
+        PartitionUtil.createPartitionedTable(msc, catName, dbName, tableName);
+        table = msc.getTable(catName, dbName, tableName);
+        PartitionUtil.addPartitions(msc, dbName, tableName, table.getSd().getLocation(), hiveConf, NUM_PARTITIONS);
+    }
+
+    @After
+    public void after() throws Exception {
+        PartitionUtil.cleanUpTableQuietly(msc, catName, dbName, tableName);
+    }
+
+    @Test
+    public void testNumberOfPartitionsRetrieved() throws HiveException {
+        List<String> numParts = hive.getPartitionNames(dbName, tableName, (short)-1);
+        Assert.assertEquals(numParts.size(), NUM_PARTITIONS);
+        List<Partition> partitions = hive.getPartitionsAuthByNames(new org.apache.hadoop.hive.ql.metadata.Table(table),
+                numParts.subList(0,5), USER_NAME, GROUP_NAMES);
+        Assert.assertEquals(partitions.size(), 5);
+    }
+
+    /**
+     * Tests the number of partitions recieved from the HMS
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testGetPartitionsAPI() throws Exception {
+        List<org.apache.hadoop.hive.ql.metadata.Partition> part = hive.getPartitions(hive.getTable(dbName, tableName));
+        Assert.assertEquals(part.size(), NUM_PARTITIONS);
+    }
+
+    @Test
+    public void testGetPartitionsAPI2() throws Exception {
+        List<org.apache.hadoop.hive.ql.metadata.Partition> part = hive.getPartitions(hive.getTable(dbName, tableName),
+                new HashMap() , (short) -1);
+        Assert.assertEquals(part.size(), NUM_PARTITIONS);
+    }
+
+    @Test
+    public void testGetPartitionsAPI2limit() throws Exception {
+        List<org.apache.hadoop.hive.ql.metadata.Partition> part = hive.getPartitions(hive.getTable(dbName, tableName),
+                new HashMap() , (short) 1);
+        Assert.assertEquals(part.size(), 1);
+
+        List<org.apache.hadoop.hive.ql.metadata.Partition> part1 = hive.getPartitions(hive.getTable(dbName, tableName),
+                new HashMap() , (short) 10);
+        Assert.assertEquals(part1.size(), 10);
+    }
+
+    /**
+     * Tests the number of times Hive.getPartitions calls are executed with total number of
+     * partitions to be added are equally divisible by batch size
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testNumberOfGetPartitionCalls() throws Exception {
+        HiveMetaStoreClient spyMSC = spy(msc);
+        hive.setMSC(spyMSC);
+        // test with a batch size of 10 and decaying factor of 2
+        int batchSize = 10;
+        hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName), batchSize, DECAYING_FACTOR, MAX_RETRIES,
+                PARTIAL_PARTITION_SPEC, IS_AUTH_REQUIRED, USER_NAME, GROUP_NAMES);
+        ArgumentCaptor<GetPartitionsPsWithAuthRequest> req = ArgumentCaptor.forClass(GetPartitionsPsWithAuthRequest.class);
+        // there should be 3 calls to get partitions
+        verify(spyMSC, times(3)).listPartitionsWithAuthInfoRequest(req.capture());
+        req.getAllValues().forEach(part-> Assert.assertEquals(part.getPartNames().size(),10));
+    }
+
+    @Test
+    public void testNumberOfGetPartitionCalls2() throws Exception {
+        HiveMetaStoreClient spyMSC = spy(msc);
+        hive.setMSC(spyMSC);
+        // test with a batch size of 10 and decaying factor of 2
+        int batchSize = 10;
+        hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName), batchSize, DECAYING_FACTOR, MAX_RETRIES,
+                new HashMap(), IS_AUTH_REQUIRED, USER_NAME, GROUP_NAMES);
+        ArgumentCaptor<GetPartitionsPsWithAuthRequest> req = ArgumentCaptor.forClass(GetPartitionsPsWithAuthRequest.class);
+        // there should be 3 calls to get partitions
+        verify(spyMSC, times(3)).listPartitionsWithAuthInfoRequest(req.capture());
+        req.getAllValues().forEach(part-> Assert.assertEquals(part.getPartNames().size(), 10));
+    }
+
+    /**
+     * Tests the number of times Hive.getAllPartitionsOf calls are executed with total number of
+     * partitions to be added are not exactly divisible by batch size
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testUnevenNumberOfGetPartitionCalls() throws Exception {
+        HiveMetaStoreClient spyMSC = spy(msc);
+        hive.setMSC(spyMSC);
+        // there should be 2 calls to get partitions with batch sizes of 19, 11
+        int batchSize = 19;
+        hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName),batchSize, DECAYING_FACTOR, MAX_RETRIES,
+                PARTIAL_PARTITION_SPEC, IS_AUTH_REQUIRED, USER_NAME, GROUP_NAMES);
+        ArgumentCaptor<GetPartitionsPsWithAuthRequest> req = ArgumentCaptor.forClass(GetPartitionsPsWithAuthRequest.class);
+        // there should be 2 calls to get partitions
+        verify(spyMSC, times(2)).listPartitionsWithAuthInfoRequest(req.capture());
+        // confirm the batch sizes were 19, 11 in the two calls to get partitions
+        List<GetPartitionsPsWithAuthRequest> apds = req.getAllValues();
+        Assert.assertEquals(19, apds.get(0).getPartNames().size());
+        Assert.assertEquals(11, apds.get(1).getPartNames().size());
+    }
+
+    /**
+     * Tests the number of times Hive.getAllPartitionsOf calls are executed with total number of
+     * partitions to is less than batch size
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testSmallNumberOfPartitions() throws Exception {
+        HiveMetaStoreClient spyMSC = spy(msc);
+        hive.setMSC(spyMSC);
+        int batchSize = 100;
+        hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName),batchSize, DECAYING_FACTOR, MAX_RETRIES,
+                PARTIAL_PARTITION_SPEC, IS_AUTH_REQUIRED, USER_NAME, GROUP_NAMES);
+        ArgumentCaptor<GetPartitionsPsWithAuthRequest> req = ArgumentCaptor.forClass(GetPartitionsPsWithAuthRequest.class);
+        // there should be 1 call to get partitions
+        verify(spyMSC, times(1)).listPartitionsWithAuthInfoRequest(req.capture());
+        Assert.assertEquals(NUM_PARTITIONS, req.getValue().getPartNames().size());
+    }
+
+    /**
+     * Tests the retries exhausted case when getAllPartitionsOf method call always keep throwing
+     * HiveException. The batch sizes should exponentially decreased based on the decaying factor and
+     * ultimately give up when it reaches 0
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testRetriesExhaustedBatchSize() throws Exception {
+        HiveMetaStoreClient spyMSC = spy(msc);
+        hive.setMSC(spyMSC);
+        doThrow(MetaException.class).when(spyMSC).listPartitionsWithAuthInfoRequest(any());
+        try {
+            hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName), NUM_PARTITIONS, DECAYING_FACTOR, MAX_RETRIES,
+                    PARTIAL_PARTITION_SPEC, IS_AUTH_REQUIRED, USER_NAME, GROUP_NAMES);
+        } catch (HiveException ignored) {}
+        ArgumentCaptor<GetPartitionsPsWithAuthRequest> req = ArgumentCaptor.forClass(GetPartitionsPsWithAuthRequest.class);
+        // there should be 5 call to get partitions with batch sizes as 30, 15, 7, 3, 1
+        verify(spyMSC, times(5)).listPartitionsWithAuthInfoRequest(req.capture());
+        List<GetPartitionsPsWithAuthRequest> apds = req.getAllValues();
+        Assert.assertEquals(5, apds.size());
+
+        Assert.assertEquals(30, apds.get(0).getPartNamesSize());
+        Assert.assertEquals(15, apds.get(1).getPartNamesSize());
+        Assert.assertEquals(7, apds.get(2).getPartNamesSize());
+        Assert.assertEquals(3, apds.get(3).getPartNamesSize());
+        Assert.assertEquals(1, apds.get(4).getPartNamesSize());
+    }
+
+    /**
+     * Tests the maximum retry attempts provided by configuration
+     * @throws Exception
+     */
+    @Test
+    public void testMaxRetriesReached() throws Exception {
+        HiveMetaStoreClient spyMSC = spy(msc);
+        hive.setMSC(spyMSC);
+        doThrow(MetaException.class).when(spyMSC).listPartitionsWithAuthInfoRequest(any());
+        int maxRetries = 2;
+        try {
+            hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName), NUM_PARTITIONS, DECAYING_FACTOR, maxRetries,
+                    PARTIAL_PARTITION_SPEC, IS_AUTH_REQUIRED, USER_NAME, GROUP_NAMES);
+        } catch (HiveException ignored) {}
+        ArgumentCaptor<GetPartitionsPsWithAuthRequest> req = ArgumentCaptor.forClass(GetPartitionsPsWithAuthRequest.class);
+        // there should be 2 call to get partitions with batch sizes as 30, 15
+        verify(spyMSC, times(2)).listPartitionsWithAuthInfoRequest(req.capture());
+        List<GetPartitionsPsWithAuthRequest> apds = req.getAllValues();
+        Assert.assertEquals(2, apds.size());
+
+        Assert.assertEquals(30, apds.get(0).getPartNamesSize());
+        Assert.assertEquals(15, apds.get(1).getPartNamesSize());
+    }
+
+    /**
+     * Tests the number of calls to getPartitions and the respective batch sizes when first call to
+     * getPartitions throws HiveException. The batch size should be reduced by the decayingFactor
+     * and the second call should fetch all the results
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testBatchingWhenException() throws Exception {
+        HiveMetaStoreClient spyMSC = spy(msc);
+        hive.setMSC(spyMSC);
+        // This will throw exception only the first time.
+        doThrow(new MetaException()).doCallRealMethod()
+                .when(spyMSC).listPartitionsWithAuthInfoRequest(any());
+
+        int maxRetries = 5;
+        hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName), NUM_PARTITIONS, DECAYING_FACTOR, maxRetries,
+                PARTIAL_PARTITION_SPEC, IS_AUTH_REQUIRED, USER_NAME, GROUP_NAMES);
+        ArgumentCaptor<GetPartitionsPsWithAuthRequest> req = ArgumentCaptor.forClass(GetPartitionsPsWithAuthRequest.class);
+        // The first call with batch size of 30 will fail, the rest two call will be of size 15 each. Total 3 calls
+        verify(spyMSC, times(3)).listPartitionsWithAuthInfoRequest(req.capture());
+        List<GetPartitionsPsWithAuthRequest> apds = req.getAllValues();
+        Assert.assertEquals(3, apds.size());
+
+        Assert.assertEquals(30, apds.get(0).getPartNamesSize());
+        Assert.assertEquals(15, apds.get(1).getPartNamesSize());
+        Assert.assertEquals(15, apds.get(2).getPartNamesSize());
+
+        Set<String> partNames = new HashSet<>(apds.get(1).getPartNames());
+        partNames.addAll(apds.get(2).getPartNames());
+        assert(partNames.size() == NUM_PARTITIONS);
+
+        List<String> partitionNames = hive.getPartitionNames(table.getDbName(),table.getTableName(), (short) -1);
+        assert(partitionNames.size() == NUM_PARTITIONS);
+        partitionNames.forEach(partNames::remove);
+        assert(partitionNames.size() == NUM_PARTITIONS);
+        // In case any duplicate/incomplete list is given by hive.getAllPartitionsInBatches, the below assertion will fail
+        assert(partNames.size() == 0);
+    }
+}
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestGetPartitionInBatches.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestGetPartitionInBatches.java
index f11eee0500d..d4afff716bf 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestGetPartitionInBatches.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestGetPartitionInBatches.java
@@ -24,12 +24,11 @@ import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
 import org.apache.hadoop.hive.metastore.api.GetPartitionsByNamesRequest;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.MetastoreException;
-import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.PartitionIterable;
+import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.junit.Assert;
 import org.junit.After;
@@ -38,10 +37,10 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 
-
-import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
 import static org.mockito.ArgumentMatchers.any;
@@ -59,6 +58,9 @@ public class TestGetPartitionInBatches {
     private static HiveMetaStoreClient msc;
     private static Hive hive;
     private Table table;
+    private final static int NUM_PARTITIONS = 30;
+    private final static boolean IS_AUTH_REQUIRED = false;
+    private final static Map<String, String> PARTIAL_PARTITION_SPEC = null;
 
     @BeforeClass
     public static void setupClass() throws HiveException {
@@ -76,7 +78,7 @@ public class TestGetPartitionInBatches {
     public void before() throws Exception {
         PartitionUtil.createPartitionedTable(msc, catName, dbName, tableName);
         table = msc.getTable(catName, dbName, tableName);
-        addPartitions(dbName, tableName);
+        PartitionUtil.addPartitions(msc, dbName, tableName, table.getSd().getLocation(), hiveConf, NUM_PARTITIONS);
     }
 
     @After
@@ -84,24 +86,13 @@ public class TestGetPartitionInBatches {
         PartitionUtil.cleanUpTableQuietly(msc, catName, dbName, tableName);
     }
 
-    private void addPartitions(String dbName, String tableName) throws Exception {
-        List<Partition> partitions = new ArrayList<>();
-        for (int i = 0; i < 30; i++) {
-            partitions.add(buildPartition(dbName, tableName, String.valueOf(i), table.getSd().getLocation() + "/city=" + i));
-        }
-        msc.add_partitions(partitions, true, true);
-    }
-
-    protected Partition buildPartition(String dbName, String tableName, String value,
-        String location) throws MetaException {
-        return new PartitionBuilder()
-                .setDbName(dbName)
-                .setTableName(tableName)
-                .addValue(value)
-                .addCol("test_id", "int", "test col id")
-                .addCol("test_value", "string", "test col value")
-                .setLocation(location)
-                .build(hiveConf);
+    @Test
+    public void TestNumberOfPartitionsRetrieved() throws HiveException {
+        List<String> numParts = hive.getPartitionNames(dbName, tableName, (short)-1);
+        Assert.assertEquals(numParts.size(), NUM_PARTITIONS);
+        List<Partition> partitions = hive.getPartitionsByNames(new org.apache.hadoop.hive.ql.metadata.Table(table),
+                numParts.subList(0,5), false);
+        Assert.assertEquals(partitions.size(), 5);
     }
 
     /**
@@ -110,9 +101,9 @@ public class TestGetPartitionInBatches {
      * @throws Exception
      */
     @Test
-    public void testgetAllPartitionsOf() throws Exception {
+    public void testGetAllPartitionsOf() throws Exception {
         Set<org.apache.hadoop.hive.ql.metadata.Partition> part = hive.getAllPartitionsOf(hive.getTable(dbName, tableName));
-        Assert.assertEquals(part.size(), 30);
+        Assert.assertEquals(part.size(), NUM_PARTITIONS);
     }
 
     /**
@@ -126,7 +117,7 @@ public class TestGetPartitionInBatches {
         HiveMetaStoreClient spyMSC = spy(msc);
         hive.setMSC(spyMSC);
         // test with a batch size of 10 and decaying factor of 2
-        hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName),10, 2, 0);
+        hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName),10, 2, 0, PARTIAL_PARTITION_SPEC, IS_AUTH_REQUIRED);
         ArgumentCaptor<GetPartitionsByNamesRequest> req = ArgumentCaptor.forClass(GetPartitionsByNamesRequest.class);
         // there should be 3 calls to get partitions
         verify(spyMSC, times(3)).getPartitionsByNames(req.capture());
@@ -144,7 +135,7 @@ public class TestGetPartitionInBatches {
         HiveMetaStoreClient spyMSC = spy(msc);
         hive.setMSC(spyMSC);
         // there should be 2 calls to get partitions with batch sizes of 19, 11
-        hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName),19, 2, 0);
+        hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName),19, 2, 0, PARTIAL_PARTITION_SPEC, IS_AUTH_REQUIRED);
         ArgumentCaptor<GetPartitionsByNamesRequest> req = ArgumentCaptor.forClass(GetPartitionsByNamesRequest.class);
         // there should be 2 calls to get partitions
         verify(spyMSC, times(2)).getPartitionsByNames(req.capture());
@@ -164,7 +155,7 @@ public class TestGetPartitionInBatches {
     public void testSmallNumberOfPartitions() throws Exception {
         HiveMetaStoreClient spyMSC = spy(msc);
         hive.setMSC(spyMSC);
-        hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName),100, 2, 0);
+        hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName),100, 2, 0, PARTIAL_PARTITION_SPEC, IS_AUTH_REQUIRED);
         ArgumentCaptor<GetPartitionsByNamesRequest> req = ArgumentCaptor.forClass(GetPartitionsByNamesRequest.class);
         // there should be 1 call to get partitions
         verify(spyMSC, times(1)).getPartitionsByNames(req.capture());
@@ -184,7 +175,7 @@ public class TestGetPartitionInBatches {
         hive.setMSC(spyMSC);
         doThrow(MetaException.class).when(spyMSC).getPartitionsByNames(any());
         try {
-            hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName), 30, 2, 0);
+            hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName), 30, 2, 0, PARTIAL_PARTITION_SPEC, IS_AUTH_REQUIRED);
         } catch (Exception ignored) {}
         ArgumentCaptor<GetPartitionsByNamesRequest> req = ArgumentCaptor.forClass(GetPartitionsByNamesRequest.class);
         // there should be 5 call to get partitions with batch sizes as 30, 15, 7, 3, 1
@@ -209,7 +200,7 @@ public class TestGetPartitionInBatches {
         hive.setMSC(spyMSC);
         doThrow(MetaException.class).when(spyMSC).getPartitionsByNames(any());
         try {
-            hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName), 30, 2, 2);
+            hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName), 30, 2, 2, PARTIAL_PARTITION_SPEC, IS_AUTH_REQUIRED);
         } catch (Exception ignored) {}
         ArgumentCaptor<GetPartitionsByNamesRequest> req = ArgumentCaptor.forClass(GetPartitionsByNamesRequest.class);
         // there should be 2 call to get partitions with batch sizes as 30, 15
@@ -236,7 +227,7 @@ public class TestGetPartitionInBatches {
         doThrow(new MetaException()).doCallRealMethod()
                 .when(spyMSC).getPartitionsByNames(any());
 
-        hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName), 30, 2, 5);
+        hive.getAllPartitionsInBatches(hive.getTable(dbName, tableName), 30, 2, 5, PARTIAL_PARTITION_SPEC, IS_AUTH_REQUIRED);
         ArgumentCaptor<GetPartitionsByNamesRequest> req = ArgumentCaptor.forClass(GetPartitionsByNamesRequest.class);
         // The first call with batch size of 30 will fail, the rest two call will be of size 15 each. Total 3 calls
         verify(spyMSC, times(3)).getPartitionsByNames(req.capture());
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 9afe386894e..5ab6a4c8173 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
@@ -2975,14 +2975,14 @@ uint32_t ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1856;
-            ::apache::thrift::protocol::TType _etype1859;
-            xfer += iprot->readListBegin(_etype1859, _size1856);
-            this->success.resize(_size1856);
-            uint32_t _i1860;
-            for (_i1860 = 0; _i1860 < _size1856; ++_i1860)
+            uint32_t _size1862;
+            ::apache::thrift::protocol::TType _etype1865;
+            xfer += iprot->readListBegin(_etype1865, _size1862);
+            this->success.resize(_size1862);
+            uint32_t _i1866;
+            for (_i1866 = 0; _i1866 < _size1862; ++_i1866)
             {
-              xfer += iprot->readString(this->success[_i1860]);
+              xfer += iprot->readString(this->success[_i1866]);
             }
             xfer += iprot->readListEnd();
           }
@@ -3021,10 +3021,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 _iter1861;
-      for (_iter1861 = this->success.begin(); _iter1861 != this->success.end(); ++_iter1861)
+      std::vector<std::string> ::const_iterator _iter1867;
+      for (_iter1867 = this->success.begin(); _iter1867 != this->success.end(); ++_iter1867)
       {
-        xfer += oprot->writeString((*_iter1861));
+        xfer += oprot->writeString((*_iter1867));
       }
       xfer += oprot->writeListEnd();
     }
@@ -3069,14 +3069,14 @@ uint32_t ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1862;
-            ::apache::thrift::protocol::TType _etype1865;
-            xfer += iprot->readListBegin(_etype1865, _size1862);
-            (*(this->success)).resize(_size1862);
-            uint32_t _i1866;
-            for (_i1866 = 0; _i1866 < _size1862; ++_i1866)
+            uint32_t _size1868;
+            ::apache::thrift::protocol::TType _etype1871;
+            xfer += iprot->readListBegin(_etype1871, _size1868);
+            (*(this->success)).resize(_size1868);
+            uint32_t _i1872;
+            for (_i1872 = 0; _i1872 < _size1868; ++_i1872)
             {
-              xfer += iprot->readString((*(this->success))[_i1866]);
+              xfer += iprot->readString((*(this->success))[_i1872]);
             }
             xfer += iprot->readListEnd();
           }
@@ -3193,14 +3193,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1867;
-            ::apache::thrift::protocol::TType _etype1870;
-            xfer += iprot->readListBegin(_etype1870, _size1867);
-            this->success.resize(_size1867);
-            uint32_t _i1871;
-            for (_i1871 = 0; _i1871 < _size1867; ++_i1871)
+            uint32_t _size1873;
+            ::apache::thrift::protocol::TType _etype1876;
+            xfer += iprot->readListBegin(_etype1876, _size1873);
+            this->success.resize(_size1873);
+            uint32_t _i1877;
+            for (_i1877 = 0; _i1877 < _size1873; ++_i1877)
             {
-              xfer += iprot->readString(this->success[_i1871]);
+              xfer += iprot->readString(this->success[_i1877]);
             }
             xfer += iprot->readListEnd();
           }
@@ -3239,10 +3239,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 _iter1872;
-      for (_iter1872 = this->success.begin(); _iter1872 != this->success.end(); ++_iter1872)
+      std::vector<std::string> ::const_iterator _iter1878;
+      for (_iter1878 = this->success.begin(); _iter1878 != this->success.end(); ++_iter1878)
       {
-        xfer += oprot->writeString((*_iter1872));
+        xfer += oprot->writeString((*_iter1878));
       }
       xfer += oprot->writeListEnd();
     }
@@ -3287,14 +3287,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1873;
-            ::apache::thrift::protocol::TType _etype1876;
-            xfer += iprot->readListBegin(_etype1876, _size1873);
-            (*(this->success)).resize(_size1873);
-            uint32_t _i1877;
-            for (_i1877 = 0; _i1877 < _size1873; ++_i1877)
+            uint32_t _size1879;
+            ::apache::thrift::protocol::TType _etype1882;
+            xfer += iprot->readListBegin(_etype1882, _size1879);
+            (*(this->success)).resize(_size1879);
+            uint32_t _i1883;
+            for (_i1883 = 0; _i1883 < _size1879; ++_i1883)
             {
-              xfer += iprot->readString((*(this->success))[_i1877]);
+              xfer += iprot->readString((*(this->success))[_i1883]);
             }
             xfer += iprot->readListEnd();
           }
@@ -4347,14 +4347,14 @@ uint32_t ThriftHiveMetastore_get_dataconnectors_result::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1878;
-            ::apache::thrift::protocol::TType _etype1881;
-            xfer += iprot->readListBegin(_etype1881, _size1878);
-            this->success.resize(_size1878);
-            uint32_t _i1882;
-            for (_i1882 = 0; _i1882 < _size1878; ++_i1882)
+            uint32_t _size1884;
+            ::apache::thrift::protocol::TType _etype1887;
+            xfer += iprot->readListBegin(_etype1887, _size1884);
+            this->success.resize(_size1884);
+            uint32_t _i1888;
+            for (_i1888 = 0; _i1888 < _size1884; ++_i1888)
             {
-              xfer += iprot->readString(this->success[_i1882]);
+              xfer += iprot->readString(this->success[_i1888]);
             }
             xfer += iprot->readListEnd();
           }
@@ -4393,10 +4393,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 _iter1883;
-      for (_iter1883 = this->success.begin(); _iter1883 != this->success.end(); ++_iter1883)
+      std::vector<std::string> ::const_iterator _iter1889;
+      for (_iter1889 = this->success.begin(); _iter1889 != this->success.end(); ++_iter1889)
       {
-        xfer += oprot->writeString((*_iter1883));
+        xfer += oprot->writeString((*_iter1889));
       }
       xfer += oprot->writeListEnd();
     }
@@ -4441,14 +4441,14 @@ uint32_t ThriftHiveMetastore_get_dataconnectors_presult::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1884;
-            ::apache::thrift::protocol::TType _etype1887;
-            xfer += iprot->readListBegin(_etype1887, _size1884);
-            (*(this->success)).resize(_size1884);
-            uint32_t _i1888;
-            for (_i1888 = 0; _i1888 < _size1884; ++_i1888)
+            uint32_t _size1890;
+            ::apache::thrift::protocol::TType _etype1893;
+            xfer += iprot->readListBegin(_etype1893, _size1890);
+            (*(this->success)).resize(_size1890);
+            uint32_t _i1894;
+            for (_i1894 = 0; _i1894 < _size1890; ++_i1894)
             {
-              xfer += iprot->readString((*(this->success))[_i1888]);
+              xfer += iprot->readString((*(this->success))[_i1894]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5510,17 +5510,17 @@ uint32_t ThriftHiveMetastore_get_type_all_result::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size1889;
-            ::apache::thrift::protocol::TType _ktype1890;
-            ::apache::thrift::protocol::TType _vtype1891;
-            xfer += iprot->readMapBegin(_ktype1890, _vtype1891, _size1889);
-            uint32_t _i1893;
-            for (_i1893 = 0; _i1893 < _size1889; ++_i1893)
+            uint32_t _size1895;
+            ::apache::thrift::protocol::TType _ktype1896;
+            ::apache::thrift::protocol::TType _vtype1897;
+            xfer += iprot->readMapBegin(_ktype1896, _vtype1897, _size1895);
+            uint32_t _i1899;
+            for (_i1899 = 0; _i1899 < _size1895; ++_i1899)
             {
-              std::string _key1894;
-              xfer += iprot->readString(_key1894);
-              Type& _val1895 = this->success[_key1894];
-              xfer += _val1895.read(iprot);
+              std::string _key1900;
+              xfer += iprot->readString(_key1900);
+              Type& _val1901 = this->success[_key1900];
+              xfer += _val1901.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -5559,11 +5559,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 _iter1896;
-      for (_iter1896 = this->success.begin(); _iter1896 != this->success.end(); ++_iter1896)
+      std::map<std::string, Type> ::const_iterator _iter1902;
+      for (_iter1902 = this->success.begin(); _iter1902 != this->success.end(); ++_iter1902)
       {
-        xfer += oprot->writeString(_iter1896->first);
-        xfer += _iter1896->second.write(oprot);
+        xfer += oprot->writeString(_iter1902->first);
+        xfer += _iter1902->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -5608,17 +5608,17 @@ uint32_t ThriftHiveMetastore_get_type_all_presult::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size1897;
-            ::apache::thrift::protocol::TType _ktype1898;
-            ::apache::thrift::protocol::TType _vtype1899;
-            xfer += iprot->readMapBegin(_ktype1898, _vtype1899, _size1897);
-            uint32_t _i1901;
-            for (_i1901 = 0; _i1901 < _size1897; ++_i1901)
+            uint32_t _size1903;
+            ::apache::thrift::protocol::TType _ktype1904;
+            ::apache::thrift::protocol::TType _vtype1905;
+            xfer += iprot->readMapBegin(_ktype1904, _vtype1905, _size1903);
+            uint32_t _i1907;
+            for (_i1907 = 0; _i1907 < _size1903; ++_i1907)
             {
-              std::string _key1902;
-              xfer += iprot->readString(_key1902);
-              Type& _val1903 = (*(this->success))[_key1902];
-              xfer += _val1903.read(iprot);
+              std::string _key1908;
+              xfer += iprot->readString(_key1908);
+              Type& _val1909 = (*(this->success))[_key1908];
+              xfer += _val1909.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -5772,14 +5772,14 @@ uint32_t ThriftHiveMetastore_get_fields_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1904;
-            ::apache::thrift::protocol::TType _etype1907;
-            xfer += iprot->readListBegin(_etype1907, _size1904);
-            this->success.resize(_size1904);
-            uint32_t _i1908;
-            for (_i1908 = 0; _i1908 < _size1904; ++_i1908)
+            uint32_t _size1910;
+            ::apache::thrift::protocol::TType _etype1913;
+            xfer += iprot->readListBegin(_etype1913, _size1910);
+            this->success.resize(_size1910);
+            uint32_t _i1914;
+            for (_i1914 = 0; _i1914 < _size1910; ++_i1914)
             {
-              xfer += this->success[_i1908].read(iprot);
+              xfer += this->success[_i1914].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5834,10 +5834,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 _iter1909;
-      for (_iter1909 = this->success.begin(); _iter1909 != this->success.end(); ++_iter1909)
+      std::vector<FieldSchema> ::const_iterator _iter1915;
+      for (_iter1915 = this->success.begin(); _iter1915 != this->success.end(); ++_iter1915)
       {
-        xfer += (*_iter1909).write(oprot);
+        xfer += (*_iter1915).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -5890,14 +5890,14 @@ uint32_t ThriftHiveMetastore_get_fields_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1910;
-            ::apache::thrift::protocol::TType _etype1913;
-            xfer += iprot->readListBegin(_etype1913, _size1910);
-            (*(this->success)).resize(_size1910);
-            uint32_t _i1914;
-            for (_i1914 = 0; _i1914 < _size1910; ++_i1914)
+            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))[_i1914].read(iprot);
+              xfer += (*(this->success))[_i1920].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6083,14 +6083,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1915;
-            ::apache::thrift::protocol::TType _etype1918;
-            xfer += iprot->readListBegin(_etype1918, _size1915);
-            this->success.resize(_size1915);
-            uint32_t _i1919;
-            for (_i1919 = 0; _i1919 < _size1915; ++_i1919)
+            uint32_t _size1921;
+            ::apache::thrift::protocol::TType _etype1924;
+            xfer += iprot->readListBegin(_etype1924, _size1921);
+            this->success.resize(_size1921);
+            uint32_t _i1925;
+            for (_i1925 = 0; _i1925 < _size1921; ++_i1925)
             {
-              xfer += this->success[_i1919].read(iprot);
+              xfer += this->success[_i1925].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6145,10 +6145,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 _iter1920;
-      for (_iter1920 = this->success.begin(); _iter1920 != this->success.end(); ++_iter1920)
+      std::vector<FieldSchema> ::const_iterator _iter1926;
+      for (_iter1926 = this->success.begin(); _iter1926 != this->success.end(); ++_iter1926)
       {
-        xfer += (*_iter1920).write(oprot);
+        xfer += (*_iter1926).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -6201,14 +6201,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1921;
-            ::apache::thrift::protocol::TType _etype1924;
-            xfer += iprot->readListBegin(_etype1924, _size1921);
-            (*(this->success)).resize(_size1921);
-            uint32_t _i1925;
-            for (_i1925 = 0; _i1925 < _size1921; ++_i1925)
+            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 += (*(this->success))[_i1925].read(iprot);
+              xfer += (*(this->success))[_i1931].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6625,14 +6625,14 @@ uint32_t ThriftHiveMetastore_get_schema_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1926;
-            ::apache::thrift::protocol::TType _etype1929;
-            xfer += iprot->readListBegin(_etype1929, _size1926);
-            this->success.resize(_size1926);
-            uint32_t _i1930;
-            for (_i1930 = 0; _i1930 < _size1926; ++_i1930)
+            uint32_t _size1932;
+            ::apache::thrift::protocol::TType _etype1935;
+            xfer += iprot->readListBegin(_etype1935, _size1932);
+            this->success.resize(_size1932);
+            uint32_t _i1936;
+            for (_i1936 = 0; _i1936 < _size1932; ++_i1936)
             {
-              xfer += this->success[_i1930].read(iprot);
+              xfer += this->success[_i1936].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6687,10 +6687,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 _iter1931;
-      for (_iter1931 = this->success.begin(); _iter1931 != this->success.end(); ++_iter1931)
+      std::vector<FieldSchema> ::const_iterator _iter1937;
+      for (_iter1937 = this->success.begin(); _iter1937 != this->success.end(); ++_iter1937)
       {
-        xfer += (*_iter1931).write(oprot);
+        xfer += (*_iter1937).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -6743,14 +6743,14 @@ uint32_t ThriftHiveMetastore_get_schema_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1932;
-            ::apache::thrift::protocol::TType _etype1935;
-            xfer += iprot->readListBegin(_etype1935, _size1932);
-            (*(this->success)).resize(_size1932);
-            uint32_t _i1936;
-            for (_i1936 = 0; _i1936 < _size1932; ++_i1936)
+            uint32_t _size1938;
+            ::apache::thrift::protocol::TType _etype1941;
+            xfer += iprot->readListBegin(_etype1941, _size1938);
+            (*(this->success)).resize(_size1938);
+            uint32_t _i1942;
+            for (_i1942 = 0; _i1942 < _size1938; ++_i1942)
             {
-              xfer += (*(this->success))[_i1936].read(iprot);
+              xfer += (*(this->success))[_i1942].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6936,14 +6936,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::read(::
         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 _size1943;
+            ::apache::thrift::protocol::TType _etype1946;
+            xfer += iprot->readListBegin(_etype1946, _size1943);
+            this->success.resize(_size1943);
+            uint32_t _i1947;
+            for (_i1947 = 0; _i1947 < _size1943; ++_i1947)
             {
-              xfer += this->success[_i1941].read(iprot);
+              xfer += this->success[_i1947].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6998,10 +6998,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 _iter1942;
-      for (_iter1942 = this->success.begin(); _iter1942 != this->success.end(); ++_iter1942)
+      std::vector<FieldSchema> ::const_iterator _iter1948;
+      for (_iter1948 = this->success.begin(); _iter1948 != this->success.end(); ++_iter1948)
       {
-        xfer += (*_iter1942).write(oprot);
+        xfer += (*_iter1948).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -7054,14 +7054,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_presult::read(:
         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 _size1949;
+            ::apache::thrift::protocol::TType _etype1952;
+            xfer += iprot->readListBegin(_etype1952, _size1949);
+            (*(this->success)).resize(_size1949);
+            uint32_t _i1953;
+            for (_i1953 = 0; _i1953 < _size1949; ++_i1953)
             {
-              xfer += (*(this->success))[_i1947].read(iprot);
+              xfer += (*(this->success))[_i1953].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7901,14 +7901,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->primaryKeys.clear();
-            uint32_t _size1948;
-            ::apache::thrift::protocol::TType _etype1951;
-            xfer += iprot->readListBegin(_etype1951, _size1948);
-            this->primaryKeys.resize(_size1948);
-            uint32_t _i1952;
-            for (_i1952 = 0; _i1952 < _size1948; ++_i1952)
+            uint32_t _size1954;
+            ::apache::thrift::protocol::TType _etype1957;
+            xfer += iprot->readListBegin(_etype1957, _size1954);
+            this->primaryKeys.resize(_size1954);
+            uint32_t _i1958;
+            for (_i1958 = 0; _i1958 < _size1954; ++_i1958)
             {
-              xfer += this->primaryKeys[_i1952].read(iprot);
+              xfer += this->primaryKeys[_i1958].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7921,14 +7921,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->foreignKeys.clear();
-            uint32_t _size1953;
-            ::apache::thrift::protocol::TType _etype1956;
-            xfer += iprot->readListBegin(_etype1956, _size1953);
-            this->foreignKeys.resize(_size1953);
-            uint32_t _i1957;
-            for (_i1957 = 0; _i1957 < _size1953; ++_i1957)
+            uint32_t _size1959;
+            ::apache::thrift::protocol::TType _etype1962;
+            xfer += iprot->readListBegin(_etype1962, _size1959);
+            this->foreignKeys.resize(_size1959);
+            uint32_t _i1963;
+            for (_i1963 = 0; _i1963 < _size1959; ++_i1963)
             {
-              xfer += this->foreignKeys[_i1957].read(iprot);
+              xfer += this->foreignKeys[_i1963].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7941,14 +7941,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->uniqueConstraints.clear();
-            uint32_t _size1958;
-            ::apache::thrift::protocol::TType _etype1961;
-            xfer += iprot->readListBegin(_etype1961, _size1958);
-            this->uniqueConstraints.resize(_size1958);
-            uint32_t _i1962;
-            for (_i1962 = 0; _i1962 < _size1958; ++_i1962)
+            uint32_t _size1964;
+            ::apache::thrift::protocol::TType _etype1967;
+            xfer += iprot->readListBegin(_etype1967, _size1964);
+            this->uniqueConstraints.resize(_size1964);
+            uint32_t _i1968;
+            for (_i1968 = 0; _i1968 < _size1964; ++_i1968)
             {
-              xfer += this->uniqueConstraints[_i1962].read(iprot);
+              xfer += this->uniqueConstraints[_i1968].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7961,14 +7961,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->notNullConstraints.clear();
-            uint32_t _size1963;
-            ::apache::thrift::protocol::TType _etype1966;
-            xfer += iprot->readListBegin(_etype1966, _size1963);
-            this->notNullConstraints.resize(_size1963);
-            uint32_t _i1967;
-            for (_i1967 = 0; _i1967 < _size1963; ++_i1967)
+            uint32_t _size1969;
+            ::apache::thrift::protocol::TType _etype1972;
+            xfer += iprot->readListBegin(_etype1972, _size1969);
+            this->notNullConstraints.resize(_size1969);
+            uint32_t _i1973;
+            for (_i1973 = 0; _i1973 < _size1969; ++_i1973)
             {
-              xfer += this->notNullConstraints[_i1967].read(iprot);
+              xfer += this->notNullConstraints[_i1973].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7981,14 +7981,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->defaultConstraints.clear();
-            uint32_t _size1968;
-            ::apache::thrift::protocol::TType _etype1971;
-            xfer += iprot->readListBegin(_etype1971, _size1968);
-            this->defaultConstraints.resize(_size1968);
-            uint32_t _i1972;
-            for (_i1972 = 0; _i1972 < _size1968; ++_i1972)
+            uint32_t _size1974;
+            ::apache::thrift::protocol::TType _etype1977;
+            xfer += iprot->readListBegin(_etype1977, _size1974);
+            this->defaultConstraints.resize(_size1974);
+            uint32_t _i1978;
+            for (_i1978 = 0; _i1978 < _size1974; ++_i1978)
             {
-              xfer += this->defaultConstraints[_i1972].read(iprot);
+              xfer += this->defaultConstraints[_i1978].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8001,14 +8001,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->checkConstraints.clear();
-            uint32_t _size1973;
-            ::apache::thrift::protocol::TType _etype1976;
-            xfer += iprot->readListBegin(_etype1976, _size1973);
-            this->checkConstraints.resize(_size1973);
-            uint32_t _i1977;
-            for (_i1977 = 0; _i1977 < _size1973; ++_i1977)
+            uint32_t _size1979;
+            ::apache::thrift::protocol::TType _etype1982;
+            xfer += iprot->readListBegin(_etype1982, _size1979);
+            this->checkConstraints.resize(_size1979);
+            uint32_t _i1983;
+            for (_i1983 = 0; _i1983 < _size1979; ++_i1983)
             {
-              xfer += this->checkConstraints[_i1977].read(iprot);
+              xfer += this->checkConstraints[_i1983].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8041,10 +8041,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 _iter1978;
-    for (_iter1978 = this->primaryKeys.begin(); _iter1978 != this->primaryKeys.end(); ++_iter1978)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter1984;
+    for (_iter1984 = this->primaryKeys.begin(); _iter1984 != this->primaryKeys.end(); ++_iter1984)
     {
-      xfer += (*_iter1978).write(oprot);
+      xfer += (*_iter1984).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8053,10 +8053,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 _iter1979;
-    for (_iter1979 = this->foreignKeys.begin(); _iter1979 != this->foreignKeys.end(); ++_iter1979)
+    std::vector<SQLForeignKey> ::const_iterator _iter1985;
+    for (_iter1985 = this->foreignKeys.begin(); _iter1985 != this->foreignKeys.end(); ++_iter1985)
     {
-      xfer += (*_iter1979).write(oprot);
+      xfer += (*_iter1985).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8065,10 +8065,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 _iter1980;
-    for (_iter1980 = this->uniqueConstraints.begin(); _iter1980 != this->uniqueConstraints.end(); ++_iter1980)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter1986;
+    for (_iter1986 = this->uniqueConstraints.begin(); _iter1986 != this->uniqueConstraints.end(); ++_iter1986)
     {
-      xfer += (*_iter1980).write(oprot);
+      xfer += (*_iter1986).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8077,10 +8077,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 _iter1981;
-    for (_iter1981 = this->notNullConstraints.begin(); _iter1981 != this->notNullConstraints.end(); ++_iter1981)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter1987;
+    for (_iter1987 = this->notNullConstraints.begin(); _iter1987 != this->notNullConstraints.end(); ++_iter1987)
     {
-      xfer += (*_iter1981).write(oprot);
+      xfer += (*_iter1987).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8089,10 +8089,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 _iter1982;
-    for (_iter1982 = this->defaultConstraints.begin(); _iter1982 != this->defaultConstraints.end(); ++_iter1982)
+    std::vector<SQLDefaultConstraint> ::const_iterator _iter1988;
+    for (_iter1988 = this->defaultConstraints.begin(); _iter1988 != this->defaultConstraints.end(); ++_iter1988)
     {
-      xfer += (*_iter1982).write(oprot);
+      xfer += (*_iter1988).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8101,10 +8101,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 _iter1983;
-    for (_iter1983 = this->checkConstraints.begin(); _iter1983 != this->checkConstraints.end(); ++_iter1983)
+    std::vector<SQLCheckConstraint> ::const_iterator _iter1989;
+    for (_iter1989 = this->checkConstraints.begin(); _iter1989 != this->checkConstraints.end(); ++_iter1989)
     {
-      xfer += (*_iter1983).write(oprot);
+      xfer += (*_iter1989).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8132,10 +8132,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 _iter1984;
-    for (_iter1984 = (*(this->primaryKeys)).begin(); _iter1984 != (*(this->primaryKeys)).end(); ++_iter1984)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter1990;
+    for (_iter1990 = (*(this->primaryKeys)).begin(); _iter1990 != (*(this->primaryKeys)).end(); ++_iter1990)
     {
-      xfer += (*_iter1984).write(oprot);
+      xfer += (*_iter1990).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8144,10 +8144,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 _iter1985;
-    for (_iter1985 = (*(this->foreignKeys)).begin(); _iter1985 != (*(this->foreignKeys)).end(); ++_iter1985)
+    std::vector<SQLForeignKey> ::const_iterator _iter1991;
+    for (_iter1991 = (*(this->foreignKeys)).begin(); _iter1991 != (*(this->foreignKeys)).end(); ++_iter1991)
     {
-      xfer += (*_iter1985).write(oprot);
+      xfer += (*_iter1991).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8156,10 +8156,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 _iter1986;
-    for (_iter1986 = (*(this->uniqueConstraints)).begin(); _iter1986 != (*(this->uniqueConstraints)).end(); ++_iter1986)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter1992;
+    for (_iter1992 = (*(this->uniqueConstraints)).begin(); _iter1992 != (*(this->uniqueConstraints)).end(); ++_iter1992)
     {
-      xfer += (*_iter1986).write(oprot);
+      xfer += (*_iter1992).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8168,10 +8168,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 _iter1987;
-    for (_iter1987 = (*(this->notNullConstraints)).begin(); _iter1987 != (*(this->notNullConstraints)).end(); ++_iter1987)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter1993;
+    for (_iter1993 = (*(this->notNullConstraints)).begin(); _iter1993 != (*(this->notNullConstraints)).end(); ++_iter1993)
     {
-      xfer += (*_iter1987).write(oprot);
+      xfer += (*_iter1993).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8180,10 +8180,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 _iter1988;
-    for (_iter1988 = (*(this->defaultConstraints)).begin(); _iter1988 != (*(this->defaultConstraints)).end(); ++_iter1988)
+    std::vector<SQLDefaultConstraint> ::const_iterator _iter1994;
+    for (_iter1994 = (*(this->defaultConstraints)).begin(); _iter1994 != (*(this->defaultConstraints)).end(); ++_iter1994)
     {
-      xfer += (*_iter1988).write(oprot);
+      xfer += (*_iter1994).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8192,10 +8192,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 _iter1989;
-    for (_iter1989 = (*(this->checkConstraints)).begin(); _iter1989 != (*(this->checkConstraints)).end(); ++_iter1989)
+    std::vector<SQLCheckConstraint> ::const_iterator _iter1995;
+    for (_iter1995 = (*(this->checkConstraints)).begin(); _iter1995 != (*(this->checkConstraints)).end(); ++_iter1995)
     {
-      xfer += (*_iter1989).write(oprot);
+      xfer += (*_iter1995).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -10877,14 +10877,14 @@ uint32_t ThriftHiveMetastore_truncate_table_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partNames.clear();
-            uint32_t _size1990;
-            ::apache::thrift::protocol::TType _etype1993;
-            xfer += iprot->readListBegin(_etype1993, _size1990);
-            this->partNames.resize(_size1990);
-            uint32_t _i1994;
-            for (_i1994 = 0; _i1994 < _size1990; ++_i1994)
+            uint32_t _size1996;
+            ::apache::thrift::protocol::TType _etype1999;
+            xfer += iprot->readListBegin(_etype1999, _size1996);
+            this->partNames.resize(_size1996);
+            uint32_t _i2000;
+            for (_i2000 = 0; _i2000 < _size1996; ++_i2000)
             {
-              xfer += iprot->readString(this->partNames[_i1994]);
+              xfer += iprot->readString(this->partNames[_i2000]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10921,10 +10921,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 _iter1995;
-    for (_iter1995 = this->partNames.begin(); _iter1995 != this->partNames.end(); ++_iter1995)
+    std::vector<std::string> ::const_iterator _iter2001;
+    for (_iter2001 = this->partNames.begin(); _iter2001 != this->partNames.end(); ++_iter2001)
     {
-      xfer += oprot->writeString((*_iter1995));
+      xfer += oprot->writeString((*_iter2001));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10956,10 +10956,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 _iter1996;
-    for (_iter1996 = (*(this->partNames)).begin(); _iter1996 != (*(this->partNames)).end(); ++_iter1996)
+    std::vector<std::string> ::const_iterator _iter2002;
+    for (_iter2002 = (*(this->partNames)).begin(); _iter2002 != (*(this->partNames)).end(); ++_iter2002)
     {
-      xfer += oprot->writeString((*_iter1996));
+      xfer += oprot->writeString((*_iter2002));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11410,14 +11410,14 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1997;
-            ::apache::thrift::protocol::TType _etype2000;
-            xfer += iprot->readListBegin(_etype2000, _size1997);
-            this->success.resize(_size1997);
-            uint32_t _i2001;
-            for (_i2001 = 0; _i2001 < _size1997; ++_i2001)
+            uint32_t _size2003;
+            ::apache::thrift::protocol::TType _etype2006;
+            xfer += iprot->readListBegin(_etype2006, _size2003);
+            this->success.resize(_size2003);
+            uint32_t _i2007;
+            for (_i2007 = 0; _i2007 < _size2003; ++_i2007)
             {
-              xfer += iprot->readString(this->success[_i2001]);
+              xfer += iprot->readString(this->success[_i2007]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11456,10 +11456,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 _iter2002;
-      for (_iter2002 = this->success.begin(); _iter2002 != this->success.end(); ++_iter2002)
+      std::vector<std::string> ::const_iterator _iter2008;
+      for (_iter2008 = this->success.begin(); _iter2008 != this->success.end(); ++_iter2008)
       {
-        xfer += oprot->writeString((*_iter2002));
+        xfer += oprot->writeString((*_iter2008));
       }
       xfer += oprot->writeListEnd();
     }
@@ -11504,14 +11504,14 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2003;
-            ::apache::thrift::protocol::TType _etype2006;
-            xfer += iprot->readListBegin(_etype2006, _size2003);
-            (*(this->success)).resize(_size2003);
-            uint32_t _i2007;
-            for (_i2007 = 0; _i2007 < _size2003; ++_i2007)
+            uint32_t _size2009;
+            ::apache::thrift::protocol::TType _etype2012;
+            xfer += iprot->readListBegin(_etype2012, _size2009);
+            (*(this->success)).resize(_size2009);
+            uint32_t _i2013;
+            for (_i2013 = 0; _i2013 < _size2009; ++_i2013)
             {
-              xfer += iprot->readString((*(this->success))[_i2007]);
+              xfer += iprot->readString((*(this->success))[_i2013]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11681,14 +11681,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 _size2008;
-            ::apache::thrift::protocol::TType _etype2011;
-            xfer += iprot->readListBegin(_etype2011, _size2008);
-            this->success.resize(_size2008);
-            uint32_t _i2012;
-            for (_i2012 = 0; _i2012 < _size2008; ++_i2012)
+            uint32_t _size2014;
+            ::apache::thrift::protocol::TType _etype2017;
+            xfer += iprot->readListBegin(_etype2017, _size2014);
+            this->success.resize(_size2014);
+            uint32_t _i2018;
+            for (_i2018 = 0; _i2018 < _size2014; ++_i2018)
             {
-              xfer += iprot->readString(this->success[_i2012]);
+              xfer += iprot->readString(this->success[_i2018]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11727,10 +11727,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 _iter2013;
-      for (_iter2013 = this->success.begin(); _iter2013 != this->success.end(); ++_iter2013)
+      std::vector<std::string> ::const_iterator _iter2019;
+      for (_iter2019 = this->success.begin(); _iter2019 != this->success.end(); ++_iter2019)
       {
-        xfer += oprot->writeString((*_iter2013));
+        xfer += oprot->writeString((*_iter2019));
       }
       xfer += oprot->writeListEnd();
     }
@@ -11775,14 +11775,14 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_presult::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2014;
-            ::apache::thrift::protocol::TType _etype2017;
-            xfer += iprot->readListBegin(_etype2017, _size2014);
-            (*(this->success)).resize(_size2014);
-            uint32_t _i2018;
-            for (_i2018 = 0; _i2018 < _size2014; ++_i2018)
+            uint32_t _size2020;
+            ::apache::thrift::protocol::TType _etype2023;
+            xfer += iprot->readListBegin(_etype2023, _size2020);
+            (*(this->success)).resize(_size2020);
+            uint32_t _i2024;
+            for (_i2024 = 0; _i2024 < _size2020; ++_i2024)
             {
-              xfer += iprot->readString((*(this->success))[_i2018]);
+              xfer += iprot->readString((*(this->success))[_i2024]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11899,14 +11899,14 @@ uint32_t ThriftHiveMetastore_get_all_materialized_view_objects_for_rewriting_res
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2019;
-            ::apache::thrift::protocol::TType _etype2022;
-            xfer += iprot->readListBegin(_etype2022, _size2019);
-            this->success.resize(_size2019);
-            uint32_t _i2023;
-            for (_i2023 = 0; _i2023 < _size2019; ++_i2023)
+            uint32_t _size2025;
+            ::apache::thrift::protocol::TType _etype2028;
+            xfer += iprot->readListBegin(_etype2028, _size2025);
+            this->success.resize(_size2025);
+            uint32_t _i2029;
+            for (_i2029 = 0; _i2029 < _size2025; ++_i2029)
             {
-              xfer += this->success[_i2023].read(iprot);
+              xfer += this->success[_i2029].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -11945,10 +11945,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 _iter2024;
-      for (_iter2024 = this->success.begin(); _iter2024 != this->success.end(); ++_iter2024)
+      std::vector<Table> ::const_iterator _iter2030;
+      for (_iter2030 = this->success.begin(); _iter2030 != this->success.end(); ++_iter2030)
       {
-        xfer += (*_iter2024).write(oprot);
+        xfer += (*_iter2030).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -11993,14 +11993,14 @@ uint32_t ThriftHiveMetastore_get_all_materialized_view_objects_for_rewriting_pre
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2025;
-            ::apache::thrift::protocol::TType _etype2028;
-            xfer += iprot->readListBegin(_etype2028, _size2025);
-            (*(this->success)).resize(_size2025);
-            uint32_t _i2029;
-            for (_i2029 = 0; _i2029 < _size2025; ++_i2029)
+            uint32_t _size2031;
+            ::apache::thrift::protocol::TType _etype2034;
+            xfer += iprot->readListBegin(_etype2034, _size2031);
+            (*(this->success)).resize(_size2031);
+            uint32_t _i2035;
+            for (_i2035 = 0; _i2035 < _size2031; ++_i2035)
             {
-              xfer += (*(this->success))[_i2029].read(iprot);
+              xfer += (*(this->success))[_i2035].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12138,14 +12138,14 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_result::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2030;
-            ::apache::thrift::protocol::TType _etype2033;
-            xfer += iprot->readListBegin(_etype2033, _size2030);
-            this->success.resize(_size2030);
-            uint32_t _i2034;
-            for (_i2034 = 0; _i2034 < _size2030; ++_i2034)
+            uint32_t _size2036;
+            ::apache::thrift::protocol::TType _etype2039;
+            xfer += iprot->readListBegin(_etype2039, _size2036);
+            this->success.resize(_size2036);
+            uint32_t _i2040;
+            for (_i2040 = 0; _i2040 < _size2036; ++_i2040)
             {
-              xfer += iprot->readString(this->success[_i2034]);
+              xfer += iprot->readString(this->success[_i2040]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12184,10 +12184,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 _iter2035;
-      for (_iter2035 = this->success.begin(); _iter2035 != this->success.end(); ++_iter2035)
+      std::vector<std::string> ::const_iterator _iter2041;
+      for (_iter2041 = this->success.begin(); _iter2041 != this->success.end(); ++_iter2041)
       {
-        xfer += oprot->writeString((*_iter2035));
+        xfer += oprot->writeString((*_iter2041));
       }
       xfer += oprot->writeListEnd();
     }
@@ -12232,14 +12232,14 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_presult::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2036;
-            ::apache::thrift::protocol::TType _etype2039;
-            xfer += iprot->readListBegin(_etype2039, _size2036);
-            (*(this->success)).resize(_size2036);
-            uint32_t _i2040;
-            for (_i2040 = 0; _i2040 < _size2036; ++_i2040)
+            uint32_t _size2042;
+            ::apache::thrift::protocol::TType _etype2045;
+            xfer += iprot->readListBegin(_etype2045, _size2042);
+            (*(this->success)).resize(_size2042);
+            uint32_t _i2046;
+            for (_i2046 = 0; _i2046 < _size2042; ++_i2046)
             {
-              xfer += iprot->readString((*(this->success))[_i2040]);
+              xfer += iprot->readString((*(this->success))[_i2046]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12314,14 +12314,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 _size2041;
-            ::apache::thrift::protocol::TType _etype2044;
-            xfer += iprot->readListBegin(_etype2044, _size2041);
-            this->tbl_types.resize(_size2041);
-            uint32_t _i2045;
-            for (_i2045 = 0; _i2045 < _size2041; ++_i2045)
+            uint32_t _size2047;
+            ::apache::thrift::protocol::TType _etype2050;
+            xfer += iprot->readListBegin(_etype2050, _size2047);
+            this->tbl_types.resize(_size2047);
+            uint32_t _i2051;
+            for (_i2051 = 0; _i2051 < _size2047; ++_i2051)
             {
-              xfer += iprot->readString(this->tbl_types[_i2045]);
+              xfer += iprot->readString(this->tbl_types[_i2051]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12358,10 +12358,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 _iter2046;
-    for (_iter2046 = this->tbl_types.begin(); _iter2046 != this->tbl_types.end(); ++_iter2046)
+    std::vector<std::string> ::const_iterator _iter2052;
+    for (_iter2052 = this->tbl_types.begin(); _iter2052 != this->tbl_types.end(); ++_iter2052)
     {
-      xfer += oprot->writeString((*_iter2046));
+      xfer += oprot->writeString((*_iter2052));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12393,10 +12393,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 _iter2047;
-    for (_iter2047 = (*(this->tbl_types)).begin(); _iter2047 != (*(this->tbl_types)).end(); ++_iter2047)
+    std::vector<std::string> ::const_iterator _iter2053;
+    for (_iter2053 = (*(this->tbl_types)).begin(); _iter2053 != (*(this->tbl_types)).end(); ++_iter2053)
     {
-      xfer += oprot->writeString((*_iter2047));
+      xfer += oprot->writeString((*_iter2053));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12437,14 +12437,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2048;
-            ::apache::thrift::protocol::TType _etype2051;
-            xfer += iprot->readListBegin(_etype2051, _size2048);
-            this->success.resize(_size2048);
-            uint32_t _i2052;
-            for (_i2052 = 0; _i2052 < _size2048; ++_i2052)
+            uint32_t _size2054;
+            ::apache::thrift::protocol::TType _etype2057;
+            xfer += iprot->readListBegin(_etype2057, _size2054);
+            this->success.resize(_size2054);
+            uint32_t _i2058;
+            for (_i2058 = 0; _i2058 < _size2054; ++_i2058)
             {
-              xfer += this->success[_i2052].read(iprot);
+              xfer += this->success[_i2058].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12483,10 +12483,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 _iter2053;
-      for (_iter2053 = this->success.begin(); _iter2053 != this->success.end(); ++_iter2053)
+      std::vector<TableMeta> ::const_iterator _iter2059;
+      for (_iter2059 = this->success.begin(); _iter2059 != this->success.end(); ++_iter2059)
       {
-        xfer += (*_iter2053).write(oprot);
+        xfer += (*_iter2059).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -12531,14 +12531,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2054;
-            ::apache::thrift::protocol::TType _etype2057;
-            xfer += iprot->readListBegin(_etype2057, _size2054);
-            (*(this->success)).resize(_size2054);
-            uint32_t _i2058;
-            for (_i2058 = 0; _i2058 < _size2054; ++_i2058)
+            uint32_t _size2060;
+            ::apache::thrift::protocol::TType _etype2063;
+            xfer += iprot->readListBegin(_etype2063, _size2060);
+            (*(this->success)).resize(_size2060);
+            uint32_t _i2064;
+            for (_i2064 = 0; _i2064 < _size2060; ++_i2064)
             {
-              xfer += (*(this->success))[_i2058].read(iprot);
+              xfer += (*(this->success))[_i2064].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12676,14 +12676,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2059;
-            ::apache::thrift::protocol::TType _etype2062;
-            xfer += iprot->readListBegin(_etype2062, _size2059);
-            this->success.resize(_size2059);
-            uint32_t _i2063;
-            for (_i2063 = 0; _i2063 < _size2059; ++_i2063)
+            uint32_t _size2065;
+            ::apache::thrift::protocol::TType _etype2068;
+            xfer += iprot->readListBegin(_etype2068, _size2065);
+            this->success.resize(_size2065);
+            uint32_t _i2069;
+            for (_i2069 = 0; _i2069 < _size2065; ++_i2069)
             {
-              xfer += iprot->readString(this->success[_i2063]);
+              xfer += iprot->readString(this->success[_i2069]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12722,10 +12722,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 _iter2064;
-      for (_iter2064 = this->success.begin(); _iter2064 != this->success.end(); ++_iter2064)
+      std::vector<std::string> ::const_iterator _iter2070;
+      for (_iter2070 = this->success.begin(); _iter2070 != this->success.end(); ++_iter2070)
       {
-        xfer += oprot->writeString((*_iter2064));
+        xfer += oprot->writeString((*_iter2070));
       }
       xfer += oprot->writeListEnd();
     }
@@ -12770,14 +12770,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2065;
-            ::apache::thrift::protocol::TType _etype2068;
-            xfer += iprot->readListBegin(_etype2068, _size2065);
-            (*(this->success)).resize(_size2065);
-            uint32_t _i2069;
-            for (_i2069 = 0; _i2069 < _size2065; ++_i2069)
+            uint32_t _size2071;
+            ::apache::thrift::protocol::TType _etype2074;
+            xfer += iprot->readListBegin(_etype2074, _size2071);
+            (*(this->success)).resize(_size2071);
+            uint32_t _i2075;
+            for (_i2075 = 0; _i2075 < _size2071; ++_i2075)
             {
-              xfer += iprot->readString((*(this->success))[_i2069]);
+              xfer += iprot->readString((*(this->success))[_i2075]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13087,14 +13087,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 _size2070;
-            ::apache::thrift::protocol::TType _etype2073;
-            xfer += iprot->readListBegin(_etype2073, _size2070);
-            this->tbl_names.resize(_size2070);
-            uint32_t _i2074;
-            for (_i2074 = 0; _i2074 < _size2070; ++_i2074)
+            uint32_t _size2076;
+            ::apache::thrift::protocol::TType _etype2079;
+            xfer += iprot->readListBegin(_etype2079, _size2076);
+            this->tbl_names.resize(_size2076);
+            uint32_t _i2080;
+            for (_i2080 = 0; _i2080 < _size2076; ++_i2080)
             {
-              xfer += iprot->readString(this->tbl_names[_i2074]);
+              xfer += iprot->readString(this->tbl_names[_i2080]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13127,10 +13127,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 _iter2075;
-    for (_iter2075 = this->tbl_names.begin(); _iter2075 != this->tbl_names.end(); ++_iter2075)
+    std::vector<std::string> ::const_iterator _iter2081;
+    for (_iter2081 = this->tbl_names.begin(); _iter2081 != this->tbl_names.end(); ++_iter2081)
     {
-      xfer += oprot->writeString((*_iter2075));
+      xfer += oprot->writeString((*_iter2081));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13158,10 +13158,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 _iter2076;
-    for (_iter2076 = (*(this->tbl_names)).begin(); _iter2076 != (*(this->tbl_names)).end(); ++_iter2076)
+    std::vector<std::string> ::const_iterator _iter2082;
+    for (_iter2082 = (*(this->tbl_names)).begin(); _iter2082 != (*(this->tbl_names)).end(); ++_iter2082)
     {
-      xfer += oprot->writeString((*_iter2076));
+      xfer += oprot->writeString((*_iter2082));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13202,14 +13202,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 _size2077;
-            ::apache::thrift::protocol::TType _etype2080;
-            xfer += iprot->readListBegin(_etype2080, _size2077);
-            this->success.resize(_size2077);
-            uint32_t _i2081;
-            for (_i2081 = 0; _i2081 < _size2077; ++_i2081)
+            uint32_t _size2083;
+            ::apache::thrift::protocol::TType _etype2086;
+            xfer += iprot->readListBegin(_etype2086, _size2083);
+            this->success.resize(_size2083);
+            uint32_t _i2087;
+            for (_i2087 = 0; _i2087 < _size2083; ++_i2087)
             {
-              xfer += this->success[_i2081].read(iprot);
+              xfer += this->success[_i2087].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13240,10 +13240,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 _iter2082;
-      for (_iter2082 = this->success.begin(); _iter2082 != this->success.end(); ++_iter2082)
+      std::vector<Table> ::const_iterator _iter2088;
+      for (_iter2088 = this->success.begin(); _iter2088 != this->success.end(); ++_iter2088)
       {
-        xfer += (*_iter2082).write(oprot);
+        xfer += (*_iter2088).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -13284,14 +13284,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 _size2083;
-            ::apache::thrift::protocol::TType _etype2086;
-            xfer += iprot->readListBegin(_etype2086, _size2083);
-            (*(this->success)).resize(_size2083);
-            uint32_t _i2087;
-            for (_i2087 = 0; _i2087 < _size2083; ++_i2087)
+            uint32_t _size2089;
+            ::apache::thrift::protocol::TType _etype2092;
+            xfer += iprot->readListBegin(_etype2092, _size2089);
+            (*(this->success)).resize(_size2089);
+            uint32_t _i2093;
+            for (_i2093 = 0; _i2093 < _size2089; ++_i2093)
             {
-              xfer += (*(this->success))[_i2087].read(iprot);
+              xfer += (*(this->success))[_i2093].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13421,14 +13421,14 @@ uint32_t ThriftHiveMetastore_get_tables_ext_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2088;
-            ::apache::thrift::protocol::TType _etype2091;
-            xfer += iprot->readListBegin(_etype2091, _size2088);
-            this->success.resize(_size2088);
-            uint32_t _i2092;
-            for (_i2092 = 0; _i2092 < _size2088; ++_i2092)
+            uint32_t _size2094;
+            ::apache::thrift::protocol::TType _etype2097;
+            xfer += iprot->readListBegin(_etype2097, _size2094);
+            this->success.resize(_size2094);
+            uint32_t _i2098;
+            for (_i2098 = 0; _i2098 < _size2094; ++_i2098)
             {
-              xfer += this->success[_i2092].read(iprot);
+              xfer += this->success[_i2098].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13467,10 +13467,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 _iter2093;
-      for (_iter2093 = this->success.begin(); _iter2093 != this->success.end(); ++_iter2093)
+      std::vector<ExtendedTableInfo> ::const_iterator _iter2099;
+      for (_iter2099 = this->success.begin(); _iter2099 != this->success.end(); ++_iter2099)
       {
-        xfer += (*_iter2093).write(oprot);
+        xfer += (*_iter2099).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -13515,14 +13515,14 @@ uint32_t ThriftHiveMetastore_get_tables_ext_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2094;
-            ::apache::thrift::protocol::TType _etype2097;
-            xfer += iprot->readListBegin(_etype2097, _size2094);
-            (*(this->success)).resize(_size2094);
-            uint32_t _i2098;
-            for (_i2098 = 0; _i2098 < _size2094; ++_i2098)
+            uint32_t _size2100;
+            ::apache::thrift::protocol::TType _etype2103;
+            xfer += iprot->readListBegin(_etype2103, _size2100);
+            (*(this->success)).resize(_size2100);
+            uint32_t _i2104;
+            for (_i2104 = 0; _i2104 < _size2100; ++_i2104)
             {
-              xfer += (*(this->success))[_i2098].read(iprot);
+              xfer += (*(this->success))[_i2104].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14704,14 +14704,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 _size2099;
-            ::apache::thrift::protocol::TType _etype2102;
-            xfer += iprot->readListBegin(_etype2102, _size2099);
-            this->success.resize(_size2099);
-            uint32_t _i2103;
-            for (_i2103 = 0; _i2103 < _size2099; ++_i2103)
+            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 += iprot->readString(this->success[_i2103]);
+              xfer += iprot->readString(this->success[_i2109]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14766,10 +14766,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 _iter2104;
-      for (_iter2104 = this->success.begin(); _iter2104 != this->success.end(); ++_iter2104)
+      std::vector<std::string> ::const_iterator _iter2110;
+      for (_iter2110 = this->success.begin(); _iter2110 != this->success.end(); ++_iter2110)
       {
-        xfer += oprot->writeString((*_iter2104));
+        xfer += oprot->writeString((*_iter2110));
       }
       xfer += oprot->writeListEnd();
     }
@@ -14822,14 +14822,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 _size2105;
-            ::apache::thrift::protocol::TType _etype2108;
-            xfer += iprot->readListBegin(_etype2108, _size2105);
-            (*(this->success)).resize(_size2105);
-            uint32_t _i2109;
-            for (_i2109 = 0; _i2109 < _size2105; ++_i2109)
+            uint32_t _size2111;
+            ::apache::thrift::protocol::TType _etype2114;
+            xfer += iprot->readListBegin(_etype2114, _size2111);
+            (*(this->success)).resize(_size2111);
+            uint32_t _i2115;
+            for (_i2115 = 0; _i2115 < _size2111; ++_i2115)
             {
-              xfer += iprot->readString((*(this->success))[_i2109]);
+              xfer += iprot->readString((*(this->success))[_i2115]);
             }
             xfer += iprot->readListEnd();
           }
@@ -16390,14 +16390,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size2110;
-            ::apache::thrift::protocol::TType _etype2113;
-            xfer += iprot->readListBegin(_etype2113, _size2110);
-            this->new_parts.resize(_size2110);
-            uint32_t _i2114;
-            for (_i2114 = 0; _i2114 < _size2110; ++_i2114)
+            uint32_t _size2116;
+            ::apache::thrift::protocol::TType _etype2119;
+            xfer += iprot->readListBegin(_etype2119, _size2116);
+            this->new_parts.resize(_size2116);
+            uint32_t _i2120;
+            for (_i2120 = 0; _i2120 < _size2116; ++_i2120)
             {
-              xfer += this->new_parts[_i2114].read(iprot);
+              xfer += this->new_parts[_i2120].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -16426,10 +16426,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 _iter2115;
-    for (_iter2115 = this->new_parts.begin(); _iter2115 != this->new_parts.end(); ++_iter2115)
+    std::vector<Partition> ::const_iterator _iter2121;
+    for (_iter2121 = this->new_parts.begin(); _iter2121 != this->new_parts.end(); ++_iter2121)
     {
-      xfer += (*_iter2115).write(oprot);
+      xfer += (*_iter2121).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -16453,10 +16453,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 _iter2116;
-    for (_iter2116 = (*(this->new_parts)).begin(); _iter2116 != (*(this->new_parts)).end(); ++_iter2116)
+    std::vector<Partition> ::const_iterator _iter2122;
+    for (_iter2122 = (*(this->new_parts)).begin(); _iter2122 != (*(this->new_parts)).end(); ++_iter2122)
     {
-      xfer += (*_iter2116).write(oprot);
+      xfer += (*_iter2122).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -16665,14 +16665,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 _size2117;
-            ::apache::thrift::protocol::TType _etype2120;
-            xfer += iprot->readListBegin(_etype2120, _size2117);
-            this->new_parts.resize(_size2117);
-            uint32_t _i2121;
-            for (_i2121 = 0; _i2121 < _size2117; ++_i2121)
+            uint32_t _size2123;
+            ::apache::thrift::protocol::TType _etype2126;
+            xfer += iprot->readListBegin(_etype2126, _size2123);
+            this->new_parts.resize(_size2123);
+            uint32_t _i2127;
+            for (_i2127 = 0; _i2127 < _size2123; ++_i2127)
             {
-              xfer += this->new_parts[_i2121].read(iprot);
+              xfer += this->new_parts[_i2127].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -16701,10 +16701,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 _iter2122;
-    for (_iter2122 = this->new_parts.begin(); _iter2122 != this->new_parts.end(); ++_iter2122)
+    std::vector<PartitionSpec> ::const_iterator _iter2128;
+    for (_iter2128 = this->new_parts.begin(); _iter2128 != this->new_parts.end(); ++_iter2128)
     {
-      xfer += (*_iter2122).write(oprot);
+      xfer += (*_iter2128).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -16728,10 +16728,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 _iter2123;
-    for (_iter2123 = (*(this->new_parts)).begin(); _iter2123 != (*(this->new_parts)).end(); ++_iter2123)
+    std::vector<PartitionSpec> ::const_iterator _iter2129;
+    for (_iter2129 = (*(this->new_parts)).begin(); _iter2129 != (*(this->new_parts)).end(); ++_iter2129)
     {
-      xfer += (*_iter2123).write(oprot);
+      xfer += (*_iter2129).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -16956,14 +16956,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size2124;
-            ::apache::thrift::protocol::TType _etype2127;
-            xfer += iprot->readListBegin(_etype2127, _size2124);
-            this->part_vals.resize(_size2124);
-            uint32_t _i2128;
-            for (_i2128 = 0; _i2128 < _size2124; ++_i2128)
+            uint32_t _size2130;
+            ::apache::thrift::protocol::TType _etype2133;
+            xfer += iprot->readListBegin(_etype2133, _size2130);
+            this->part_vals.resize(_size2130);
+            uint32_t _i2134;
+            for (_i2134 = 0; _i2134 < _size2130; ++_i2134)
             {
-              xfer += iprot->readString(this->part_vals[_i2128]);
+              xfer += iprot->readString(this->part_vals[_i2134]);
             }
             xfer += iprot->readListEnd();
           }
@@ -17000,10 +17000,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 _iter2129;
-    for (_iter2129 = this->part_vals.begin(); _iter2129 != this->part_vals.end(); ++_iter2129)
+    std::vector<std::string> ::const_iterator _iter2135;
+    for (_iter2135 = this->part_vals.begin(); _iter2135 != this->part_vals.end(); ++_iter2135)
     {
-      xfer += oprot->writeString((*_iter2129));
+      xfer += oprot->writeString((*_iter2135));
     }
     xfer += oprot->writeListEnd();
   }
@@ -17035,10 +17035,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 _iter2130;
-    for (_iter2130 = (*(this->part_vals)).begin(); _iter2130 != (*(this->part_vals)).end(); ++_iter2130)
+    std::vector<std::string> ::const_iterator _iter2136;
+    for (_iter2136 = (*(this->part_vals)).begin(); _iter2136 != (*(this->part_vals)).end(); ++_iter2136)
     {
-      xfer += oprot->writeString((*_iter2130));
+      xfer += oprot->writeString((*_iter2136));
     }
     xfer += oprot->writeListEnd();
   }
@@ -17510,14 +17510,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size2131;
-            ::apache::thrift::protocol::TType _etype2134;
-            xfer += iprot->readListBegin(_etype2134, _size2131);
-            this->part_vals.resize(_size2131);
-            uint32_t _i2135;
-            for (_i2135 = 0; _i2135 < _size2131; ++_i2135)
+            uint32_t _size2137;
+            ::apache::thrift::protocol::TType _etype2140;
+            xfer += iprot->readListBegin(_etype2140, _size2137);
+            this->part_vals.resize(_size2137);
+            uint32_t _i2141;
+            for (_i2141 = 0; _i2141 < _size2137; ++_i2141)
             {
-              xfer += iprot->readString(this->part_vals[_i2135]);
+              xfer += iprot->readString(this->part_vals[_i2141]);
             }
             xfer += iprot->readListEnd();
           }
@@ -17562,10 +17562,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 _iter2136;
-    for (_iter2136 = this->part_vals.begin(); _iter2136 != this->part_vals.end(); ++_iter2136)
+    std::vector<std::string> ::const_iterator _iter2142;
+    for (_iter2142 = this->part_vals.begin(); _iter2142 != this->part_vals.end(); ++_iter2142)
     {
-      xfer += oprot->writeString((*_iter2136));
+      xfer += oprot->writeString((*_iter2142));
     }
     xfer += oprot->writeListEnd();
   }
@@ -17601,10 +17601,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 _iter2137;
-    for (_iter2137 = (*(this->part_vals)).begin(); _iter2137 != (*(this->part_vals)).end(); ++_iter2137)
+    std::vector<std::string> ::const_iterator _iter2143;
+    for (_iter2143 = (*(this->part_vals)).begin(); _iter2143 != (*(this->part_vals)).end(); ++_iter2143)
     {
-      xfer += oprot->writeString((*_iter2137));
+      xfer += oprot->writeString((*_iter2143));
     }
     xfer += oprot->writeListEnd();
   }
@@ -18407,14 +18407,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size2138;
-            ::apache::thrift::protocol::TType _etype2141;
-            xfer += iprot->readListBegin(_etype2141, _size2138);
-            this->part_vals.resize(_size2138);
-            uint32_t _i2142;
-            for (_i2142 = 0; _i2142 < _size2138; ++_i2142)
+            uint32_t _size2144;
+            ::apache::thrift::protocol::TType _etype2147;
+            xfer += iprot->readListBegin(_etype2147, _size2144);
+            this->part_vals.resize(_size2144);
+            uint32_t _i2148;
+            for (_i2148 = 0; _i2148 < _size2144; ++_i2148)
             {
-              xfer += iprot->readString(this->part_vals[_i2142]);
+              xfer += iprot->readString(this->part_vals[_i2148]);
             }
             xfer += iprot->readListEnd();
           }
@@ -18459,10 +18459,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 _iter2143;
-    for (_iter2143 = this->part_vals.begin(); _iter2143 != this->part_vals.end(); ++_iter2143)
+    std::vector<std::string> ::const_iterator _iter2149;
+    for (_iter2149 = this->part_vals.begin(); _iter2149 != this->part_vals.end(); ++_iter2149)
     {
-      xfer += oprot->writeString((*_iter2143));
+      xfer += oprot->writeString((*_iter2149));
     }
     xfer += oprot->writeListEnd();
   }
@@ -18498,10 +18498,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 _iter2144;
-    for (_iter2144 = (*(this->part_vals)).begin(); _iter2144 != (*(this->part_vals)).end(); ++_iter2144)
+    std::vector<std::string> ::const_iterator _iter2150;
+    for (_iter2150 = (*(this->part_vals)).begin(); _iter2150 != (*(this->part_vals)).end(); ++_iter2150)
     {
-      xfer += oprot->writeString((*_iter2144));
+      xfer += oprot->writeString((*_iter2150));
     }
     xfer += oprot->writeListEnd();
   }
@@ -18710,14 +18710,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size2145;
-            ::apache::thrift::protocol::TType _etype2148;
-            xfer += iprot->readListBegin(_etype2148, _size2145);
-            this->part_vals.resize(_size2145);
-            uint32_t _i2149;
-            for (_i2149 = 0; _i2149 < _size2145; ++_i2149)
+            uint32_t _size2151;
+            ::apache::thrift::protocol::TType _etype2154;
+            xfer += iprot->readListBegin(_etype2154, _size2151);
+            this->part_vals.resize(_size2151);
+            uint32_t _i2155;
+            for (_i2155 = 0; _i2155 < _size2151; ++_i2155)
             {
-              xfer += iprot->readString(this->part_vals[_i2149]);
+              xfer += iprot->readString(this->part_vals[_i2155]);
             }
             xfer += iprot->readListEnd();
           }
@@ -18770,10 +18770,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 _iter2150;
-    for (_iter2150 = this->part_vals.begin(); _iter2150 != this->part_vals.end(); ++_iter2150)
+    std::vector<std::string> ::const_iterator _iter2156;
+    for (_iter2156 = this->part_vals.begin(); _iter2156 != this->part_vals.end(); ++_iter2156)
     {
-      xfer += oprot->writeString((*_iter2150));
+      xfer += oprot->writeString((*_iter2156));
     }
     xfer += oprot->writeListEnd();
   }
@@ -18813,10 +18813,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 _iter2151;
-    for (_iter2151 = (*(this->part_vals)).begin(); _iter2151 != (*(this->part_vals)).end(); ++_iter2151)
+    std::vector<std::string> ::const_iterator _iter2157;
+    for (_iter2157 = (*(this->part_vals)).begin(); _iter2157 != (*(this->part_vals)).end(); ++_iter2157)
     {
-      xfer += oprot->writeString((*_iter2151));
+      xfer += oprot->writeString((*_iter2157));
     }
     xfer += oprot->writeListEnd();
   }
@@ -19822,14 +19822,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size2152;
-            ::apache::thrift::protocol::TType _etype2155;
-            xfer += iprot->readListBegin(_etype2155, _size2152);
-            this->part_vals.resize(_size2152);
-            uint32_t _i2156;
-            for (_i2156 = 0; _i2156 < _size2152; ++_i2156)
+            uint32_t _size2158;
+            ::apache::thrift::protocol::TType _etype2161;
+            xfer += iprot->readListBegin(_etype2161, _size2158);
+            this->part_vals.resize(_size2158);
+            uint32_t _i2162;
+            for (_i2162 = 0; _i2162 < _size2158; ++_i2162)
             {
-              xfer += iprot->readString(this->part_vals[_i2156]);
+              xfer += iprot->readString(this->part_vals[_i2162]);
             }
             xfer += iprot->readListEnd();
           }
@@ -19866,10 +19866,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 _iter2157;
-    for (_iter2157 = this->part_vals.begin(); _iter2157 != this->part_vals.end(); ++_iter2157)
+    std::vector<std::string> ::const_iterator _iter2163;
+    for (_iter2163 = this->part_vals.begin(); _iter2163 != this->part_vals.end(); ++_iter2163)
     {
-      xfer += oprot->writeString((*_iter2157));
+      xfer += oprot->writeString((*_iter2163));
     }
     xfer += oprot->writeListEnd();
   }
@@ -19901,10 +19901,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 _iter2158;
-    for (_iter2158 = (*(this->part_vals)).begin(); _iter2158 != (*(this->part_vals)).end(); ++_iter2158)
+    std::vector<std::string> ::const_iterator _iter2164;
+    for (_iter2164 = (*(this->part_vals)).begin(); _iter2164 != (*(this->part_vals)).end(); ++_iter2164)
     {
-      xfer += oprot->writeString((*_iter2158));
+      xfer += oprot->writeString((*_iter2164));
     }
     xfer += oprot->writeListEnd();
   }
@@ -20320,17 +20320,17 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size2159;
-            ::apache::thrift::protocol::TType _ktype2160;
-            ::apache::thrift::protocol::TType _vtype2161;
-            xfer += iprot->readMapBegin(_ktype2160, _vtype2161, _size2159);
-            uint32_t _i2163;
-            for (_i2163 = 0; _i2163 < _size2159; ++_i2163)
+            uint32_t _size2165;
+            ::apache::thrift::protocol::TType _ktype2166;
+            ::apache::thrift::protocol::TType _vtype2167;
+            xfer += iprot->readMapBegin(_ktype2166, _vtype2167, _size2165);
+            uint32_t _i2169;
+            for (_i2169 = 0; _i2169 < _size2165; ++_i2169)
             {
-              std::string _key2164;
-              xfer += iprot->readString(_key2164);
-              std::string& _val2165 = this->partitionSpecs[_key2164];
-              xfer += iprot->readString(_val2165);
+              std::string _key2170;
+              xfer += iprot->readString(_key2170);
+              std::string& _val2171 = this->partitionSpecs[_key2170];
+              xfer += iprot->readString(_val2171);
             }
             xfer += iprot->readMapEnd();
           }
@@ -20391,11 +20391,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 _iter2166;
-    for (_iter2166 = this->partitionSpecs.begin(); _iter2166 != this->partitionSpecs.end(); ++_iter2166)
+    std::map<std::string, std::string> ::const_iterator _iter2172;
+    for (_iter2172 = this->partitionSpecs.begin(); _iter2172 != this->partitionSpecs.end(); ++_iter2172)
     {
-      xfer += oprot->writeString(_iter2166->first);
-      xfer += oprot->writeString(_iter2166->second);
+      xfer += oprot->writeString(_iter2172->first);
+      xfer += oprot->writeString(_iter2172->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -20435,11 +20435,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 _iter2167;
-    for (_iter2167 = (*(this->partitionSpecs)).begin(); _iter2167 != (*(this->partitionSpecs)).end(); ++_iter2167)
+    std::map<std::string, std::string> ::const_iterator _iter2173;
+    for (_iter2173 = (*(this->partitionSpecs)).begin(); _iter2173 != (*(this->partitionSpecs)).end(); ++_iter2173)
     {
-      xfer += oprot->writeString(_iter2167->first);
-      xfer += oprot->writeString(_iter2167->second);
+      xfer += oprot->writeString(_iter2173->first);
+      xfer += oprot->writeString(_iter2173->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -20684,17 +20684,17 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size2168;
-            ::apache::thrift::protocol::TType _ktype2169;
-            ::apache::thrift::protocol::TType _vtype2170;
-            xfer += iprot->readMapBegin(_ktype2169, _vtype2170, _size2168);
-            uint32_t _i2172;
-            for (_i2172 = 0; _i2172 < _size2168; ++_i2172)
+            uint32_t _size2174;
+            ::apache::thrift::protocol::TType _ktype2175;
+            ::apache::thrift::protocol::TType _vtype2176;
+            xfer += iprot->readMapBegin(_ktype2175, _vtype2176, _size2174);
+            uint32_t _i2178;
+            for (_i2178 = 0; _i2178 < _size2174; ++_i2178)
             {
-              std::string _key2173;
-              xfer += iprot->readString(_key2173);
-              std::string& _val2174 = this->partitionSpecs[_key2173];
-              xfer += iprot->readString(_val2174);
+              std::string _key2179;
+              xfer += iprot->readString(_key2179);
+              std::string& _val2180 = this->partitionSpecs[_key2179];
+              xfer += iprot->readString(_val2180);
             }
             xfer += iprot->readMapEnd();
           }
@@ -20755,11 +20755,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 _iter2175;
-    for (_iter2175 = this->partitionSpecs.begin(); _iter2175 != this->partitionSpecs.end(); ++_iter2175)
+    std::map<std::string, std::string> ::const_iterator _iter2181;
+    for (_iter2181 = this->partitionSpecs.begin(); _iter2181 != this->partitionSpecs.end(); ++_iter2181)
     {
-      xfer += oprot->writeString(_iter2175->first);
-      xfer += oprot->writeString(_iter2175->second);
+      xfer += oprot->writeString(_iter2181->first);
+      xfer += oprot->writeString(_iter2181->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -20799,11 +20799,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 _iter2176;
-    for (_iter2176 = (*(this->partitionSpecs)).begin(); _iter2176 != (*(this->partitionSpecs)).end(); ++_iter2176)
+    std::map<std::string, std::string> ::const_iterator _iter2182;
+    for (_iter2182 = (*(this->partitionSpecs)).begin(); _iter2182 != (*(this->partitionSpecs)).end(); ++_iter2182)
     {
-      xfer += oprot->writeString(_iter2176->first);
-      xfer += oprot->writeString(_iter2176->second);
+      xfer += oprot->writeString(_iter2182->first);
+      xfer += oprot->writeString(_iter2182->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -20860,14 +20860,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2177;
-            ::apache::thrift::protocol::TType _etype2180;
-            xfer += iprot->readListBegin(_etype2180, _size2177);
-            this->success.resize(_size2177);
-            uint32_t _i2181;
-            for (_i2181 = 0; _i2181 < _size2177; ++_i2181)
+            uint32_t _size2183;
+            ::apache::thrift::protocol::TType _etype2186;
+            xfer += iprot->readListBegin(_etype2186, _size2183);
+            this->success.resize(_size2183);
+            uint32_t _i2187;
+            for (_i2187 = 0; _i2187 < _size2183; ++_i2187)
             {
-              xfer += this->success[_i2181].read(iprot);
+              xfer += this->success[_i2187].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -20930,10 +20930,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 _iter2182;
-      for (_iter2182 = this->success.begin(); _iter2182 != this->success.end(); ++_iter2182)
+      std::vector<Partition> ::const_iterator _iter2188;
+      for (_iter2188 = this->success.begin(); _iter2188 != this->success.end(); ++_iter2188)
       {
-        xfer += (*_iter2182).write(oprot);
+        xfer += (*_iter2188).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -20990,14 +20990,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_presult::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2183;
-            ::apache::thrift::protocol::TType _etype2186;
-            xfer += iprot->readListBegin(_etype2186, _size2183);
-            (*(this->success)).resize(_size2183);
-            uint32_t _i2187;
-            for (_i2187 = 0; _i2187 < _size2183; ++_i2187)
+            uint32_t _size2189;
+            ::apache::thrift::protocol::TType _etype2192;
+            xfer += iprot->readListBegin(_etype2192, _size2189);
+            (*(this->success)).resize(_size2189);
+            uint32_t _i2193;
+            for (_i2193 = 0; _i2193 < _size2189; ++_i2193)
             {
-              xfer += (*(this->success))[_i2187].read(iprot);
+              xfer += (*(this->success))[_i2193].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -21096,14 +21096,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 _size2188;
-            ::apache::thrift::protocol::TType _etype2191;
-            xfer += iprot->readListBegin(_etype2191, _size2188);
-            this->part_vals.resize(_size2188);
-            uint32_t _i2192;
-            for (_i2192 = 0; _i2192 < _size2188; ++_i2192)
+            uint32_t _size2194;
+            ::apache::thrift::protocol::TType _etype2197;
+            xfer += iprot->readListBegin(_etype2197, _size2194);
+            this->part_vals.resize(_size2194);
+            uint32_t _i2198;
+            for (_i2198 = 0; _i2198 < _size2194; ++_i2198)
             {
-              xfer += iprot->readString(this->part_vals[_i2192]);
+              xfer += iprot->readString(this->part_vals[_i2198]);
             }
             xfer += iprot->readListEnd();
           }
@@ -21124,14 +21124,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 _size2193;
-            ::apache::thrift::protocol::TType _etype2196;
-            xfer += iprot->readListBegin(_etype2196, _size2193);
-            this->group_names.resize(_size2193);
-            uint32_t _i2197;
-            for (_i2197 = 0; _i2197 < _size2193; ++_i2197)
+            uint32_t _size2199;
+            ::apache::thrift::protocol::TType _etype2202;
+            xfer += iprot->readListBegin(_etype2202, _size2199);
+            this->group_names.resize(_size2199);
+            uint32_t _i2203;
+            for (_i2203 = 0; _i2203 < _size2199; ++_i2203)
             {
-              xfer += iprot->readString(this->group_names[_i2197]);
+              xfer += iprot->readString(this->group_names[_i2203]);
             }
             xfer += iprot->readListEnd();
           }
@@ -21168,10 +21168,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 _iter2198;
-    for (_iter2198 = this->part_vals.begin(); _iter2198 != this->part_vals.end(); ++_iter2198)
+    std::vector<std::string> ::const_iterator _iter2204;
+    for (_iter2204 = this->part_vals.begin(); _iter2204 != this->part_vals.end(); ++_iter2204)
     {
-      xfer += oprot->writeString((*_iter2198));
+      xfer += oprot->writeString((*_iter2204));
     }
     xfer += oprot->writeListEnd();
   }
@@ -21184,10 +21184,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 _iter2199;
-    for (_iter2199 = this->group_names.begin(); _iter2199 != this->group_names.end(); ++_iter2199)
+    std::vector<std::string> ::const_iterator _iter2205;
+    for (_iter2205 = this->group_names.begin(); _iter2205 != this->group_names.end(); ++_iter2205)
     {
-      xfer += oprot->writeString((*_iter2199));
+      xfer += oprot->writeString((*_iter2205));
     }
     xfer += oprot->writeListEnd();
   }
@@ -21219,10 +21219,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 _iter2200;
-    for (_iter2200 = (*(this->part_vals)).begin(); _iter2200 != (*(this->part_vals)).end(); ++_iter2200)
+    std::vector<std::string> ::const_iterator _iter2206;
+    for (_iter2206 = (*(this->part_vals)).begin(); _iter2206 != (*(this->part_vals)).end(); ++_iter2206)
     {
-      xfer += oprot->writeString((*_iter2200));
+      xfer += oprot->writeString((*_iter2206));
     }
     xfer += oprot->writeListEnd();
   }
@@ -21235,10 +21235,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 _iter2201;
-    for (_iter2201 = (*(this->group_names)).begin(); _iter2201 != (*(this->group_names)).end(); ++_iter2201)
+    std::vector<std::string> ::const_iterator _iter2207;
+    for (_iter2207 = (*(this->group_names)).begin(); _iter2207 != (*(this->group_names)).end(); ++_iter2207)
     {
-      xfer += oprot->writeString((*_iter2201));
+      xfer += oprot->writeString((*_iter2207));
     }
     xfer += oprot->writeListEnd();
   }
@@ -21797,14 +21797,14 @@ uint32_t ThriftHiveMetastore_get_partitions_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2202;
-            ::apache::thrift::protocol::TType _etype2205;
-            xfer += iprot->readListBegin(_etype2205, _size2202);
-            this->success.resize(_size2202);
-            uint32_t _i2206;
-            for (_i2206 = 0; _i2206 < _size2202; ++_i2206)
+            uint32_t _size2208;
+            ::apache::thrift::protocol::TType _etype2211;
+            xfer += iprot->readListBegin(_etype2211, _size2208);
+            this->success.resize(_size2208);
+            uint32_t _i2212;
+            for (_i2212 = 0; _i2212 < _size2208; ++_i2212)
             {
-              xfer += this->success[_i2206].read(iprot);
+              xfer += this->success[_i2212].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -21851,10 +21851,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 _iter2207;
-      for (_iter2207 = this->success.begin(); _iter2207 != this->success.end(); ++_iter2207)
+      std::vector<Partition> ::const_iterator _iter2213;
+      for (_iter2213 = this->success.begin(); _iter2213 != this->success.end(); ++_iter2213)
       {
-        xfer += (*_iter2207).write(oprot);
+        xfer += (*_iter2213).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -21903,14 +21903,14 @@ uint32_t ThriftHiveMetastore_get_partitions_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2208;
-            ::apache::thrift::protocol::TType _etype2211;
-            xfer += iprot->readListBegin(_etype2211, _size2208);
-            (*(this->success)).resize(_size2208);
-            uint32_t _i2212;
-            for (_i2212 = 0; _i2212 < _size2208; ++_i2212)
+            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)
             {
-              xfer += (*(this->success))[_i2212].read(iprot);
+              xfer += (*(this->success))[_i2218].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -22236,14 +22236,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 _size2213;
-            ::apache::thrift::protocol::TType _etype2216;
-            xfer += iprot->readListBegin(_etype2216, _size2213);
-            this->group_names.resize(_size2213);
-            uint32_t _i2217;
-            for (_i2217 = 0; _i2217 < _size2213; ++_i2217)
+            uint32_t _size2219;
+            ::apache::thrift::protocol::TType _etype2222;
+            xfer += iprot->readListBegin(_etype2222, _size2219);
+            this->group_names.resize(_size2219);
+            uint32_t _i2223;
+            for (_i2223 = 0; _i2223 < _size2219; ++_i2223)
             {
-              xfer += iprot->readString(this->group_names[_i2217]);
+              xfer += iprot->readString(this->group_names[_i2223]);
             }
             xfer += iprot->readListEnd();
           }
@@ -22288,10 +22288,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 _iter2218;
-    for (_iter2218 = this->group_names.begin(); _iter2218 != this->group_names.end(); ++_iter2218)
+    std::vector<std::string> ::const_iterator _iter2224;
+    for (_iter2224 = this->group_names.begin(); _iter2224 != this->group_names.end(); ++_iter2224)
     {
-      xfer += oprot->writeString((*_iter2218));
+      xfer += oprot->writeString((*_iter2224));
     }
     xfer += oprot->writeListEnd();
   }
@@ -22331,10 +22331,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 _iter2219;
-    for (_iter2219 = (*(this->group_names)).begin(); _iter2219 != (*(this->group_names)).end(); ++_iter2219)
+    std::vector<std::string> ::const_iterator _iter2225;
+    for (_iter2225 = (*(this->group_names)).begin(); _iter2225 != (*(this->group_names)).end(); ++_iter2225)
     {
-      xfer += oprot->writeString((*_iter2219));
+      xfer += oprot->writeString((*_iter2225));
     }
     xfer += oprot->writeListEnd();
   }
@@ -22375,14 +22375,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::read(::apache::thr
         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 _size2226;
+            ::apache::thrift::protocol::TType _etype2229;
+            xfer += iprot->readListBegin(_etype2229, _size2226);
+            this->success.resize(_size2226);
+            uint32_t _i2230;
+            for (_i2230 = 0; _i2230 < _size2226; ++_i2230)
             {
-              xfer += this->success[_i2224].read(iprot);
+              xfer += this->success[_i2230].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -22429,10 +22429,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 _iter2225;
-      for (_iter2225 = this->success.begin(); _iter2225 != this->success.end(); ++_iter2225)
+      std::vector<Partition> ::const_iterator _iter2231;
+      for (_iter2231 = this->success.begin(); _iter2231 != this->success.end(); ++_iter2231)
       {
-        xfer += (*_iter2225).write(oprot);
+        xfer += (*_iter2231).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -22481,14 +22481,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_presult::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2226;
-            ::apache::thrift::protocol::TType _etype2229;
-            xfer += iprot->readListBegin(_etype2229, _size2226);
-            (*(this->success)).resize(_size2226);
-            uint32_t _i2230;
-            for (_i2230 = 0; _i2230 < _size2226; ++_i2230)
+            uint32_t _size2232;
+            ::apache::thrift::protocol::TType _etype2235;
+            xfer += iprot->readListBegin(_etype2235, _size2232);
+            (*(this->success)).resize(_size2232);
+            uint32_t _i2236;
+            for (_i2236 = 0; _i2236 < _size2232; ++_i2236)
             {
-              xfer += (*(this->success))[_i2230].read(iprot);
+              xfer += (*(this->success))[_i2236].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -22666,14 +22666,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::read(::apache::thrift:
         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 _size2237;
+            ::apache::thrift::protocol::TType _etype2240;
+            xfer += iprot->readListBegin(_etype2240, _size2237);
+            this->success.resize(_size2237);
+            uint32_t _i2241;
+            for (_i2241 = 0; _i2241 < _size2237; ++_i2241)
             {
-              xfer += this->success[_i2235].read(iprot);
+              xfer += this->success[_i2241].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -22720,10 +22720,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 _iter2236;
-      for (_iter2236 = this->success.begin(); _iter2236 != this->success.end(); ++_iter2236)
+      std::vector<PartitionSpec> ::const_iterator _iter2242;
+      for (_iter2242 = this->success.begin(); _iter2242 != this->success.end(); ++_iter2242)
       {
-        xfer += (*_iter2236).write(oprot);
+        xfer += (*_iter2242).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -22772,14 +22772,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_presult::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2237;
-            ::apache::thrift::protocol::TType _etype2240;
-            xfer += iprot->readListBegin(_etype2240, _size2237);
-            (*(this->success)).resize(_size2237);
-            uint32_t _i2241;
-            for (_i2241 = 0; _i2241 < _size2237; ++_i2241)
+            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)
             {
-              xfer += (*(this->success))[_i2241].read(iprot);
+              xfer += (*(this->success))[_i2247].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -22957,14 +22957,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2242;
-            ::apache::thrift::protocol::TType _etype2245;
-            xfer += iprot->readListBegin(_etype2245, _size2242);
-            this->success.resize(_size2242);
-            uint32_t _i2246;
-            for (_i2246 = 0; _i2246 < _size2242; ++_i2246)
+            uint32_t _size2248;
+            ::apache::thrift::protocol::TType _etype2251;
+            xfer += iprot->readListBegin(_etype2251, _size2248);
+            this->success.resize(_size2248);
+            uint32_t _i2252;
+            for (_i2252 = 0; _i2252 < _size2248; ++_i2252)
             {
-              xfer += iprot->readString(this->success[_i2246]);
+              xfer += iprot->readString(this->success[_i2252]);
             }
             xfer += iprot->readListEnd();
           }
@@ -23011,10 +23011,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 _iter2247;
-      for (_iter2247 = this->success.begin(); _iter2247 != this->success.end(); ++_iter2247)
+      std::vector<std::string> ::const_iterator _iter2253;
+      for (_iter2253 = this->success.begin(); _iter2253 != this->success.end(); ++_iter2253)
       {
-        xfer += oprot->writeString((*_iter2247));
+        xfer += oprot->writeString((*_iter2253));
       }
       xfer += oprot->writeListEnd();
     }
@@ -23063,14 +23063,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_presult::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2248;
-            ::apache::thrift::protocol::TType _etype2251;
-            xfer += iprot->readListBegin(_etype2251, _size2248);
-            (*(this->success)).resize(_size2248);
-            uint32_t _i2252;
-            for (_i2252 = 0; _i2252 < _size2248; ++_i2252)
+            uint32_t _size2254;
+            ::apache::thrift::protocol::TType _etype2257;
+            xfer += iprot->readListBegin(_etype2257, _size2254);
+            (*(this->success)).resize(_size2254);
+            uint32_t _i2258;
+            for (_i2258 = 0; _i2258 < _size2254; ++_i2258)
             {
-              xfer += iprot->readString((*(this->success))[_i2252]);
+              xfer += iprot->readString((*(this->success))[_i2258]);
             }
             xfer += iprot->readListEnd();
           }
@@ -23380,14 +23380,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 _size2253;
-            ::apache::thrift::protocol::TType _etype2256;
-            xfer += iprot->readListBegin(_etype2256, _size2253);
-            this->part_vals.resize(_size2253);
-            uint32_t _i2257;
-            for (_i2257 = 0; _i2257 < _size2253; ++_i2257)
+            uint32_t _size2259;
+            ::apache::thrift::protocol::TType _etype2262;
+            xfer += iprot->readListBegin(_etype2262, _size2259);
+            this->part_vals.resize(_size2259);
+            uint32_t _i2263;
+            for (_i2263 = 0; _i2263 < _size2259; ++_i2263)
             {
-              xfer += iprot->readString(this->part_vals[_i2257]);
+              xfer += iprot->readString(this->part_vals[_i2263]);
             }
             xfer += iprot->readListEnd();
           }
@@ -23432,10 +23432,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 _iter2258;
-    for (_iter2258 = this->part_vals.begin(); _iter2258 != this->part_vals.end(); ++_iter2258)
+    std::vector<std::string> ::const_iterator _iter2264;
+    for (_iter2264 = this->part_vals.begin(); _iter2264 != this->part_vals.end(); ++_iter2264)
     {
-      xfer += oprot->writeString((*_iter2258));
+      xfer += oprot->writeString((*_iter2264));
     }
     xfer += oprot->writeListEnd();
   }
@@ -23471,10 +23471,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 _iter2259;
-    for (_iter2259 = (*(this->part_vals)).begin(); _iter2259 != (*(this->part_vals)).end(); ++_iter2259)
+    std::vector<std::string> ::const_iterator _iter2265;
+    for (_iter2265 = (*(this->part_vals)).begin(); _iter2265 != (*(this->part_vals)).end(); ++_iter2265)
     {
-      xfer += oprot->writeString((*_iter2259));
+      xfer += oprot->writeString((*_iter2265));
     }
     xfer += oprot->writeListEnd();
   }
@@ -23519,14 +23519,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2260;
-            ::apache::thrift::protocol::TType _etype2263;
-            xfer += iprot->readListBegin(_etype2263, _size2260);
-            this->success.resize(_size2260);
-            uint32_t _i2264;
-            for (_i2264 = 0; _i2264 < _size2260; ++_i2264)
+            uint32_t _size2266;
+            ::apache::thrift::protocol::TType _etype2269;
+            xfer += iprot->readListBegin(_etype2269, _size2266);
+            this->success.resize(_size2266);
+            uint32_t _i2270;
+            for (_i2270 = 0; _i2270 < _size2266; ++_i2270)
             {
-              xfer += this->success[_i2264].read(iprot);
+              xfer += this->success[_i2270].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -23573,10 +23573,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 _iter2265;
-      for (_iter2265 = this->success.begin(); _iter2265 != this->success.end(); ++_iter2265)
+      std::vector<Partition> ::const_iterator _iter2271;
+      for (_iter2271 = this->success.begin(); _iter2271 != this->success.end(); ++_iter2271)
       {
-        xfer += (*_iter2265).write(oprot);
+        xfer += (*_iter2271).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -23625,14 +23625,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2266;
-            ::apache::thrift::protocol::TType _etype2269;
-            xfer += iprot->readListBegin(_etype2269, _size2266);
-            (*(this->success)).resize(_size2266);
-            uint32_t _i2270;
-            for (_i2270 = 0; _i2270 < _size2266; ++_i2270)
+            uint32_t _size2272;
+            ::apache::thrift::protocol::TType _etype2275;
+            xfer += iprot->readListBegin(_etype2275, _size2272);
+            (*(this->success)).resize(_size2272);
+            uint32_t _i2276;
+            for (_i2276 = 0; _i2276 < _size2272; ++_i2276)
             {
-              xfer += (*(this->success))[_i2270].read(iprot);
+              xfer += (*(this->success))[_i2276].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -23715,14 +23715,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 _size2271;
-            ::apache::thrift::protocol::TType _etype2274;
-            xfer += iprot->readListBegin(_etype2274, _size2271);
-            this->part_vals.resize(_size2271);
-            uint32_t _i2275;
-            for (_i2275 = 0; _i2275 < _size2271; ++_i2275)
+            uint32_t _size2277;
+            ::apache::thrift::protocol::TType _etype2280;
+            xfer += iprot->readListBegin(_etype2280, _size2277);
+            this->part_vals.resize(_size2277);
+            uint32_t _i2281;
+            for (_i2281 = 0; _i2281 < _size2277; ++_i2281)
             {
-              xfer += iprot->readString(this->part_vals[_i2275]);
+              xfer += iprot->readString(this->part_vals[_i2281]);
             }
             xfer += iprot->readListEnd();
           }
@@ -23751,14 +23751,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 _size2276;
-            ::apache::thrift::protocol::TType _etype2279;
-            xfer += iprot->readListBegin(_etype2279, _size2276);
-            this->group_names.resize(_size2276);
-            uint32_t _i2280;
-            for (_i2280 = 0; _i2280 < _size2276; ++_i2280)
+            uint32_t _size2282;
+            ::apache::thrift::protocol::TType _etype2285;
+            xfer += iprot->readListBegin(_etype2285, _size2282);
+            this->group_names.resize(_size2282);
+            uint32_t _i2286;
+            for (_i2286 = 0; _i2286 < _size2282; ++_i2286)
             {
-              xfer += iprot->readString(this->group_names[_i2280]);
+              xfer += iprot->readString(this->group_names[_i2286]);
             }
             xfer += iprot->readListEnd();
           }
@@ -23795,10 +23795,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 _iter2281;
-    for (_iter2281 = this->part_vals.begin(); _iter2281 != this->part_vals.end(); ++_iter2281)
+    std::vector<std::string> ::const_iterator _iter2287;
+    for (_iter2287 = this->part_vals.begin(); _iter2287 != this->part_vals.end(); ++_iter2287)
     {
-      xfer += oprot->writeString((*_iter2281));
+      xfer += oprot->writeString((*_iter2287));
     }
     xfer += oprot->writeListEnd();
   }
@@ -23815,10 +23815,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 _iter2282;
-    for (_iter2282 = this->group_names.begin(); _iter2282 != this->group_names.end(); ++_iter2282)
+    std::vector<std::string> ::const_iterator _iter2288;
+    for (_iter2288 = this->group_names.begin(); _iter2288 != this->group_names.end(); ++_iter2288)
     {
-      xfer += oprot->writeString((*_iter2282));
+      xfer += oprot->writeString((*_iter2288));
     }
     xfer += oprot->writeListEnd();
   }
@@ -23850,10 +23850,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 _iter2283;
-    for (_iter2283 = (*(this->part_vals)).begin(); _iter2283 != (*(this->part_vals)).end(); ++_iter2283)
+    std::vector<std::string> ::const_iterator _iter2289;
+    for (_iter2289 = (*(this->part_vals)).begin(); _iter2289 != (*(this->part_vals)).end(); ++_iter2289)
     {
-      xfer += oprot->writeString((*_iter2283));
+      xfer += oprot->writeString((*_iter2289));
     }
     xfer += oprot->writeListEnd();
   }
@@ -23870,10 +23870,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 _iter2284;
-    for (_iter2284 = (*(this->group_names)).begin(); _iter2284 != (*(this->group_names)).end(); ++_iter2284)
+    std::vector<std::string> ::const_iterator _iter2290;
+    for (_iter2290 = (*(this->group_names)).begin(); _iter2290 != (*(this->group_names)).end(); ++_iter2290)
     {
-      xfer += oprot->writeString((*_iter2284));
+      xfer += oprot->writeString((*_iter2290));
     }
     xfer += oprot->writeListEnd();
   }
@@ -23914,14 +23914,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_result::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2285;
-            ::apache::thrift::protocol::TType _etype2288;
-            xfer += iprot->readListBegin(_etype2288, _size2285);
-            this->success.resize(_size2285);
-            uint32_t _i2289;
-            for (_i2289 = 0; _i2289 < _size2285; ++_i2289)
+            uint32_t _size2291;
+            ::apache::thrift::protocol::TType _etype2294;
+            xfer += iprot->readListBegin(_etype2294, _size2291);
+            this->success.resize(_size2291);
+            uint32_t _i2295;
+            for (_i2295 = 0; _i2295 < _size2291; ++_i2295)
             {
-              xfer += this->success[_i2289].read(iprot);
+              xfer += this->success[_i2295].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -23968,10 +23968,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 _iter2290;
-      for (_iter2290 = this->success.begin(); _iter2290 != this->success.end(); ++_iter2290)
+      std::vector<Partition> ::const_iterator _iter2296;
+      for (_iter2296 = this->success.begin(); _iter2296 != this->success.end(); ++_iter2296)
       {
-        xfer += (*_iter2290).write(oprot);
+        xfer += (*_iter2296).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -24020,14 +24020,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_presult::read(::apache:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2291;
-            ::apache::thrift::protocol::TType _etype2294;
-            xfer += iprot->readListBegin(_etype2294, _size2291);
-            (*(this->success)).resize(_size2291);
-            uint32_t _i2295;
-            for (_i2295 = 0; _i2295 < _size2291; ++_i2295)
+            uint32_t _size2297;
+            ::apache::thrift::protocol::TType _etype2300;
+            xfer += iprot->readListBegin(_etype2300, _size2297);
+            (*(this->success)).resize(_size2297);
+            uint32_t _i2301;
+            for (_i2301 = 0; _i2301 < _size2297; ++_i2301)
             {
-              xfer += (*(this->success))[_i2295].read(iprot);
+              xfer += (*(this->success))[_i2301].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -24337,14 +24337,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 _size2296;
-            ::apache::thrift::protocol::TType _etype2299;
-            xfer += iprot->readListBegin(_etype2299, _size2296);
-            this->part_vals.resize(_size2296);
-            uint32_t _i2300;
-            for (_i2300 = 0; _i2300 < _size2296; ++_i2300)
+            uint32_t _size2302;
+            ::apache::thrift::protocol::TType _etype2305;
+            xfer += iprot->readListBegin(_etype2305, _size2302);
+            this->part_vals.resize(_size2302);
+            uint32_t _i2306;
+            for (_i2306 = 0; _i2306 < _size2302; ++_i2306)
             {
-              xfer += iprot->readString(this->part_vals[_i2300]);
+              xfer += iprot->readString(this->part_vals[_i2306]);
             }
             xfer += iprot->readListEnd();
           }
@@ -24389,10 +24389,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 _iter2301;
-    for (_iter2301 = this->part_vals.begin(); _iter2301 != this->part_vals.end(); ++_iter2301)
+    std::vector<std::string> ::const_iterator _iter2307;
+    for (_iter2307 = this->part_vals.begin(); _iter2307 != this->part_vals.end(); ++_iter2307)
     {
-      xfer += oprot->writeString((*_iter2301));
+      xfer += oprot->writeString((*_iter2307));
     }
     xfer += oprot->writeListEnd();
   }
@@ -24428,10 +24428,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 _iter2302;
-    for (_iter2302 = (*(this->part_vals)).begin(); _iter2302 != (*(this->part_vals)).end(); ++_iter2302)
+    std::vector<std::string> ::const_iterator _iter2308;
+    for (_iter2308 = (*(this->part_vals)).begin(); _iter2308 != (*(this->part_vals)).end(); ++_iter2308)
     {
-      xfer += oprot->writeString((*_iter2302));
+      xfer += oprot->writeString((*_iter2308));
     }
     xfer += oprot->writeListEnd();
   }
@@ -24476,14 +24476,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_result::read(::apache::thrif
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2303;
-            ::apache::thrift::protocol::TType _etype2306;
-            xfer += iprot->readListBegin(_etype2306, _size2303);
-            this->success.resize(_size2303);
-            uint32_t _i2307;
-            for (_i2307 = 0; _i2307 < _size2303; ++_i2307)
+            uint32_t _size2309;
+            ::apache::thrift::protocol::TType _etype2312;
+            xfer += iprot->readListBegin(_etype2312, _size2309);
+            this->success.resize(_size2309);
+            uint32_t _i2313;
+            for (_i2313 = 0; _i2313 < _size2309; ++_i2313)
             {
-              xfer += iprot->readString(this->success[_i2307]);
+              xfer += iprot->readString(this->success[_i2313]);
             }
             xfer += iprot->readListEnd();
           }
@@ -24530,10 +24530,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 _iter2308;
-      for (_iter2308 = this->success.begin(); _iter2308 != this->success.end(); ++_iter2308)
+      std::vector<std::string> ::const_iterator _iter2314;
+      for (_iter2314 = this->success.begin(); _iter2314 != this->success.end(); ++_iter2314)
       {
-        xfer += oprot->writeString((*_iter2308));
+        xfer += oprot->writeString((*_iter2314));
       }
       xfer += oprot->writeListEnd();
     }
@@ -24582,14 +24582,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_presult::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2309;
-            ::apache::thrift::protocol::TType _etype2312;
-            xfer += iprot->readListBegin(_etype2312, _size2309);
-            (*(this->success)).resize(_size2309);
-            uint32_t _i2313;
-            for (_i2313 = 0; _i2313 < _size2309; ++_i2313)
+            uint32_t _size2315;
+            ::apache::thrift::protocol::TType _etype2318;
+            xfer += iprot->readListBegin(_etype2318, _size2315);
+            (*(this->success)).resize(_size2315);
+            uint32_t _i2319;
+            for (_i2319 = 0; _i2319 < _size2315; ++_i2319)
             {
-              xfer += iprot->readString((*(this->success))[_i2313]);
+              xfer += iprot->readString((*(this->success))[_i2319]);
             }
             xfer += iprot->readListEnd();
           }
@@ -24962,14 +24962,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_req_result::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2314;
-            ::apache::thrift::protocol::TType _etype2317;
-            xfer += iprot->readListBegin(_etype2317, _size2314);
-            this->success.resize(_size2314);
-            uint32_t _i2318;
-            for (_i2318 = 0; _i2318 < _size2314; ++_i2318)
+            uint32_t _size2320;
+            ::apache::thrift::protocol::TType _etype2323;
+            xfer += iprot->readListBegin(_etype2323, _size2320);
+            this->success.resize(_size2320);
+            uint32_t _i2324;
+            for (_i2324 = 0; _i2324 < _size2320; ++_i2324)
             {
-              xfer += iprot->readString(this->success[_i2318]);
+              xfer += iprot->readString(this->success[_i2324]);
             }
             xfer += iprot->readListEnd();
           }
@@ -25016,10 +25016,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 _iter2319;
-      for (_iter2319 = this->success.begin(); _iter2319 != this->success.end(); ++_iter2319)
+      std::vector<std::string> ::const_iterator _iter2325;
+      for (_iter2325 = this->success.begin(); _iter2325 != this->success.end(); ++_iter2325)
       {
-        xfer += oprot->writeString((*_iter2319));
+        xfer += oprot->writeString((*_iter2325));
       }
       xfer += oprot->writeListEnd();
     }
@@ -25068,14 +25068,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_req_presult::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2320;
-            ::apache::thrift::protocol::TType _etype2323;
-            xfer += iprot->readListBegin(_etype2323, _size2320);
-            (*(this->success)).resize(_size2320);
-            uint32_t _i2324;
-            for (_i2324 = 0; _i2324 < _size2320; ++_i2324)
+            uint32_t _size2326;
+            ::apache::thrift::protocol::TType _etype2329;
+            xfer += iprot->readListBegin(_etype2329, _size2326);
+            (*(this->success)).resize(_size2326);
+            uint32_t _i2330;
+            for (_i2330 = 0; _i2330 < _size2326; ++_i2330)
             {
-              xfer += iprot->readString((*(this->success))[_i2324]);
+              xfer += iprot->readString((*(this->success))[_i2330]);
             }
             xfer += iprot->readListEnd();
           }
@@ -25269,14 +25269,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_filter_result::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2325;
-            ::apache::thrift::protocol::TType _etype2328;
-            xfer += iprot->readListBegin(_etype2328, _size2325);
-            this->success.resize(_size2325);
-            uint32_t _i2329;
-            for (_i2329 = 0; _i2329 < _size2325; ++_i2329)
+            uint32_t _size2331;
+            ::apache::thrift::protocol::TType _etype2334;
+            xfer += iprot->readListBegin(_etype2334, _size2331);
+            this->success.resize(_size2331);
+            uint32_t _i2335;
+            for (_i2335 = 0; _i2335 < _size2331; ++_i2335)
             {
-              xfer += this->success[_i2329].read(iprot);
+              xfer += this->success[_i2335].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -25323,10 +25323,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 _iter2330;
-      for (_iter2330 = this->success.begin(); _iter2330 != this->success.end(); ++_iter2330)
+      std::vector<Partition> ::const_iterator _iter2336;
+      for (_iter2336 = this->success.begin(); _iter2336 != this->success.end(); ++_iter2336)
       {
-        xfer += (*_iter2330).write(oprot);
+        xfer += (*_iter2336).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -25375,14 +25375,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_filter_presult::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2331;
-            ::apache::thrift::protocol::TType _etype2334;
-            xfer += iprot->readListBegin(_etype2334, _size2331);
-            (*(this->success)).resize(_size2331);
-            uint32_t _i2335;
-            for (_i2335 = 0; _i2335 < _size2331; ++_i2335)
+            uint32_t _size2337;
+            ::apache::thrift::protocol::TType _etype2340;
+            xfer += iprot->readListBegin(_etype2340, _size2337);
+            (*(this->success)).resize(_size2337);
+            uint32_t _i2341;
+            for (_i2341 = 0; _i2341 < _size2337; ++_i2341)
             {
-              xfer += (*(this->success))[_i2335].read(iprot);
+              xfer += (*(this->success))[_i2341].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -25528,14 +25528,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_filter_req_result::read(::apache:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            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)
+            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 += this->success[_i2340].read(iprot);
+              xfer += this->success[_i2346].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -25582,10 +25582,10 @@ uint32_t ThriftHiveMetastore_get_partitions_by_filter_req_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 _iter2341;
-      for (_iter2341 = this->success.begin(); _iter2341 != this->success.end(); ++_iter2341)
+      std::vector<Partition> ::const_iterator _iter2347;
+      for (_iter2347 = this->success.begin(); _iter2347 != this->success.end(); ++_iter2347)
       {
-        xfer += (*_iter2341).write(oprot);
+        xfer += (*_iter2347).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -25634,14 +25634,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_filter_req_presult::read(::apache
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            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)
+            uint32_t _size2348;
+            ::apache::thrift::protocol::TType _etype2351;
+            xfer += iprot->readListBegin(_etype2351, _size2348);
+            (*(this->success)).resize(_size2348);
+            uint32_t _i2352;
+            for (_i2352 = 0; _i2352 < _size2348; ++_i2352)
             {
-              xfer += (*(this->success))[_i2346].read(iprot);
+              xfer += (*(this->success))[_i2352].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -25835,14 +25835,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 _size2347;
-            ::apache::thrift::protocol::TType _etype2350;
-            xfer += iprot->readListBegin(_etype2350, _size2347);
-            this->success.resize(_size2347);
-            uint32_t _i2351;
-            for (_i2351 = 0; _i2351 < _size2347; ++_i2351)
+            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 += this->success[_i2351].read(iprot);
+              xfer += this->success[_i2357].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -25889,10 +25889,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 _iter2352;
-      for (_iter2352 = this->success.begin(); _iter2352 != this->success.end(); ++_iter2352)
+      std::vector<PartitionSpec> ::const_iterator _iter2358;
+      for (_iter2358 = this->success.begin(); _iter2358 != this->success.end(); ++_iter2358)
       {
-        xfer += (*_iter2352).write(oprot);
+        xfer += (*_iter2358).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -25941,14 +25941,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 _size2353;
-            ::apache::thrift::protocol::TType _etype2356;
-            xfer += iprot->readListBegin(_etype2356, _size2353);
-            (*(this->success)).resize(_size2353);
-            uint32_t _i2357;
-            for (_i2357 = 0; _i2357 < _size2353; ++_i2357)
+            uint32_t _size2359;
+            ::apache::thrift::protocol::TType _etype2362;
+            xfer += iprot->readListBegin(_etype2362, _size2359);
+            (*(this->success)).resize(_size2359);
+            uint32_t _i2363;
+            for (_i2363 = 0; _i2363 < _size2359; ++_i2363)
             {
-              xfer += (*(this->success))[_i2357].read(iprot);
+              xfer += (*(this->success))[_i2363].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -26744,14 +26744,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_args::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->names.clear();
-            uint32_t _size2358;
-            ::apache::thrift::protocol::TType _etype2361;
-            xfer += iprot->readListBegin(_etype2361, _size2358);
-            this->names.resize(_size2358);
-            uint32_t _i2362;
-            for (_i2362 = 0; _i2362 < _size2358; ++_i2362)
+            uint32_t _size2364;
+            ::apache::thrift::protocol::TType _etype2367;
+            xfer += iprot->readListBegin(_etype2367, _size2364);
+            this->names.resize(_size2364);
+            uint32_t _i2368;
+            for (_i2368 = 0; _i2368 < _size2364; ++_i2368)
             {
-              xfer += iprot->readString(this->names[_i2362]);
+              xfer += iprot->readString(this->names[_i2368]);
             }
             xfer += iprot->readListEnd();
           }
@@ -26788,10 +26788,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 _iter2363;
-    for (_iter2363 = this->names.begin(); _iter2363 != this->names.end(); ++_iter2363)
+    std::vector<std::string> ::const_iterator _iter2369;
+    for (_iter2369 = this->names.begin(); _iter2369 != this->names.end(); ++_iter2369)
     {
-      xfer += oprot->writeString((*_iter2363));
+      xfer += oprot->writeString((*_iter2369));
     }
     xfer += oprot->writeListEnd();
   }
@@ -26823,10 +26823,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 _iter2364;
-    for (_iter2364 = (*(this->names)).begin(); _iter2364 != (*(this->names)).end(); ++_iter2364)
+    std::vector<std::string> ::const_iterator _iter2370;
+    for (_iter2370 = (*(this->names)).begin(); _iter2370 != (*(this->names)).end(); ++_iter2370)
     {
-      xfer += oprot->writeString((*_iter2364));
+      xfer += oprot->writeString((*_iter2370));
     }
     xfer += oprot->writeListEnd();
   }
@@ -26867,14 +26867,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_result::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2365;
-            ::apache::thrift::protocol::TType _etype2368;
-            xfer += iprot->readListBegin(_etype2368, _size2365);
-            this->success.resize(_size2365);
-            uint32_t _i2369;
-            for (_i2369 = 0; _i2369 < _size2365; ++_i2369)
+            uint32_t _size2371;
+            ::apache::thrift::protocol::TType _etype2374;
+            xfer += iprot->readListBegin(_etype2374, _size2371);
+            this->success.resize(_size2371);
+            uint32_t _i2375;
+            for (_i2375 = 0; _i2375 < _size2371; ++_i2375)
             {
-              xfer += this->success[_i2369].read(iprot);
+              xfer += this->success[_i2375].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -26929,10 +26929,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 _iter2370;
-      for (_iter2370 = this->success.begin(); _iter2370 != this->success.end(); ++_iter2370)
+      std::vector<Partition> ::const_iterator _iter2376;
+      for (_iter2376 = this->success.begin(); _iter2376 != this->success.end(); ++_iter2376)
       {
-        xfer += (*_iter2370).write(oprot);
+        xfer += (*_iter2376).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -26985,14 +26985,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_presult::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2371;
-            ::apache::thrift::protocol::TType _etype2374;
-            xfer += iprot->readListBegin(_etype2374, _size2371);
-            (*(this->success)).resize(_size2371);
-            uint32_t _i2375;
-            for (_i2375 = 0; _i2375 < _size2371; ++_i2375)
+            uint32_t _size2377;
+            ::apache::thrift::protocol::TType _etype2380;
+            xfer += iprot->readListBegin(_etype2380, _size2377);
+            (*(this->success)).resize(_size2377);
+            uint32_t _i2381;
+            for (_i2381 = 0; _i2381 < _size2377; ++_i2381)
             {
-              xfer += (*(this->success))[_i2375].read(iprot);
+              xfer += (*(this->success))[_i2381].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -28023,14 +28023,14 @@ uint32_t ThriftHiveMetastore_alter_partitions_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size2376;
-            ::apache::thrift::protocol::TType _etype2379;
-            xfer += iprot->readListBegin(_etype2379, _size2376);
-            this->new_parts.resize(_size2376);
-            uint32_t _i2380;
-            for (_i2380 = 0; _i2380 < _size2376; ++_i2380)
+            uint32_t _size2382;
+            ::apache::thrift::protocol::TType _etype2385;
+            xfer += iprot->readListBegin(_etype2385, _size2382);
+            this->new_parts.resize(_size2382);
+            uint32_t _i2386;
+            for (_i2386 = 0; _i2386 < _size2382; ++_i2386)
             {
-              xfer += this->new_parts[_i2380].read(iprot);
+              xfer += this->new_parts[_i2386].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -28067,10 +28067,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 _iter2381;
-    for (_iter2381 = this->new_parts.begin(); _iter2381 != this->new_parts.end(); ++_iter2381)
+    std::vector<Partition> ::const_iterator _iter2387;
+    for (_iter2387 = this->new_parts.begin(); _iter2387 != this->new_parts.end(); ++_iter2387)
     {
-      xfer += (*_iter2381).write(oprot);
+      xfer += (*_iter2387).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -28102,10 +28102,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 _iter2382;
-    for (_iter2382 = (*(this->new_parts)).begin(); _iter2382 != (*(this->new_parts)).end(); ++_iter2382)
+    std::vector<Partition> ::const_iterator _iter2388;
+    for (_iter2388 = (*(this->new_parts)).begin(); _iter2388 != (*(this->new_parts)).end(); ++_iter2388)
     {
-      xfer += (*_iter2382).write(oprot);
+      xfer += (*_iter2388).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -28290,14 +28290,14 @@ uint32_t ThriftHiveMetastore_alter_partitions_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size2383;
-            ::apache::thrift::protocol::TType _etype2386;
-            xfer += iprot->readListBegin(_etype2386, _size2383);
-            this->new_parts.resize(_size2383);
-            uint32_t _i2387;
-            for (_i2387 = 0; _i2387 < _size2383; ++_i2387)
+            uint32_t _size2389;
+            ::apache::thrift::protocol::TType _etype2392;
+            xfer += iprot->readListBegin(_etype2392, _size2389);
+            this->new_parts.resize(_size2389);
+            uint32_t _i2393;
+            for (_i2393 = 0; _i2393 < _size2389; ++_i2393)
             {
-              xfer += this->new_parts[_i2387].read(iprot);
+              xfer += this->new_parts[_i2393].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -28342,10 +28342,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 _iter2388;
-    for (_iter2388 = this->new_parts.begin(); _iter2388 != this->new_parts.end(); ++_iter2388)
+    std::vector<Partition> ::const_iterator _iter2394;
+    for (_iter2394 = this->new_parts.begin(); _iter2394 != this->new_parts.end(); ++_iter2394)
     {
-      xfer += (*_iter2388).write(oprot);
+      xfer += (*_iter2394).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -28381,10 +28381,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 _iter2389;
-    for (_iter2389 = (*(this->new_parts)).begin(); _iter2389 != (*(this->new_parts)).end(); ++_iter2389)
+    std::vector<Partition> ::const_iterator _iter2395;
+    for (_iter2395 = (*(this->new_parts)).begin(); _iter2395 != (*(this->new_parts)).end(); ++_iter2395)
     {
-      xfer += (*_iter2389).write(oprot);
+      xfer += (*_iter2395).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -29055,14 +29055,14 @@ uint32_t ThriftHiveMetastore_rename_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size2390;
-            ::apache::thrift::protocol::TType _etype2393;
-            xfer += iprot->readListBegin(_etype2393, _size2390);
-            this->part_vals.resize(_size2390);
-            uint32_t _i2394;
-            for (_i2394 = 0; _i2394 < _size2390; ++_i2394)
+            uint32_t _size2396;
+            ::apache::thrift::protocol::TType _etype2399;
+            xfer += iprot->readListBegin(_etype2399, _size2396);
+            this->part_vals.resize(_size2396);
+            uint32_t _i2400;
+            for (_i2400 = 0; _i2400 < _size2396; ++_i2400)
             {
-              xfer += iprot->readString(this->part_vals[_i2394]);
+              xfer += iprot->readString(this->part_vals[_i2400]);
             }
             xfer += iprot->readListEnd();
           }
@@ -29107,10 +29107,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 _iter2395;
-    for (_iter2395 = this->part_vals.begin(); _iter2395 != this->part_vals.end(); ++_iter2395)
+    std::vector<std::string> ::const_iterator _iter2401;
+    for (_iter2401 = this->part_vals.begin(); _iter2401 != this->part_vals.end(); ++_iter2401)
     {
-      xfer += oprot->writeString((*_iter2395));
+      xfer += oprot->writeString((*_iter2401));
     }
     xfer += oprot->writeListEnd();
   }
@@ -29146,10 +29146,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 _iter2396;
-    for (_iter2396 = (*(this->part_vals)).begin(); _iter2396 != (*(this->part_vals)).end(); ++_iter2396)
+    std::vector<std::string> ::const_iterator _iter2402;
+    for (_iter2402 = (*(this->part_vals)).begin(); _iter2402 != (*(this->part_vals)).end(); ++_iter2402)
     {
-      xfer += oprot->writeString((*_iter2396));
+      xfer += oprot->writeString((*_iter2402));
     }
     xfer += oprot->writeListEnd();
   }
@@ -29549,14 +29549,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 _size2397;
-            ::apache::thrift::protocol::TType _etype2400;
-            xfer += iprot->readListBegin(_etype2400, _size2397);
-            this->part_vals.resize(_size2397);
-            uint32_t _i2401;
-            for (_i2401 = 0; _i2401 < _size2397; ++_i2401)
+            uint32_t _size2403;
+            ::apache::thrift::protocol::TType _etype2406;
+            xfer += iprot->readListBegin(_etype2406, _size2403);
+            this->part_vals.resize(_size2403);
+            uint32_t _i2407;
+            for (_i2407 = 0; _i2407 < _size2403; ++_i2407)
             {
-              xfer += iprot->readString(this->part_vals[_i2401]);
+              xfer += iprot->readString(this->part_vals[_i2407]);
             }
             xfer += iprot->readListEnd();
           }
@@ -29593,10 +29593,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 _iter2402;
-    for (_iter2402 = this->part_vals.begin(); _iter2402 != this->part_vals.end(); ++_iter2402)
+    std::vector<std::string> ::const_iterator _iter2408;
+    for (_iter2408 = this->part_vals.begin(); _iter2408 != this->part_vals.end(); ++_iter2408)
     {
-      xfer += oprot->writeString((*_iter2402));
+      xfer += oprot->writeString((*_iter2408));
     }
     xfer += oprot->writeListEnd();
   }
@@ -29624,10 +29624,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 _iter2403;
-    for (_iter2403 = (*(this->part_vals)).begin(); _iter2403 != (*(this->part_vals)).end(); ++_iter2403)
+    std::vector<std::string> ::const_iterator _iter2409;
+    for (_iter2409 = (*(this->part_vals)).begin(); _iter2409 != (*(this->part_vals)).end(); ++_iter2409)
     {
-      xfer += oprot->writeString((*_iter2403));
+      xfer += oprot->writeString((*_iter2409));
     }
     xfer += oprot->writeListEnd();
   }
@@ -30102,14 +30102,14 @@ uint32_t ThriftHiveMetastore_partition_name_to_vals_result::read(::apache::thrif
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2404;
-            ::apache::thrift::protocol::TType _etype2407;
-            xfer += iprot->readListBegin(_etype2407, _size2404);
-            this->success.resize(_size2404);
-            uint32_t _i2408;
-            for (_i2408 = 0; _i2408 < _size2404; ++_i2408)
+            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[_i2408]);
+              xfer += iprot->readString(this->success[_i2414]);
             }
             xfer += iprot->readListEnd();
           }
@@ -30148,10 +30148,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 _iter2409;
-      for (_iter2409 = this->success.begin(); _iter2409 != this->success.end(); ++_iter2409)
+      std::vector<std::string> ::const_iterator _iter2415;
+      for (_iter2415 = this->success.begin(); _iter2415 != this->success.end(); ++_iter2415)
       {
-        xfer += oprot->writeString((*_iter2409));
+        xfer += oprot->writeString((*_iter2415));
       }
       xfer += oprot->writeListEnd();
     }
@@ -30196,14 +30196,14 @@ uint32_t ThriftHiveMetastore_partition_name_to_vals_presult::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            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)
+            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))[_i2414]);
+              xfer += iprot->readString((*(this->success))[_i2420]);
             }
             xfer += iprot->readListEnd();
           }
@@ -30341,17 +30341,17 @@ uint32_t ThriftHiveMetastore_partition_name_to_spec_result::read(::apache::thrif
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size2415;
-            ::apache::thrift::protocol::TType _ktype2416;
-            ::apache::thrift::protocol::TType _vtype2417;
-            xfer += iprot->readMapBegin(_ktype2416, _vtype2417, _size2415);
-            uint32_t _i2419;
-            for (_i2419 = 0; _i2419 < _size2415; ++_i2419)
+            uint32_t _size2421;
+            ::apache::thrift::protocol::TType _ktype2422;
+            ::apache::thrift::protocol::TType _vtype2423;
+            xfer += iprot->readMapBegin(_ktype2422, _vtype2423, _size2421);
+            uint32_t _i2425;
+            for (_i2425 = 0; _i2425 < _size2421; ++_i2425)
             {
-              std::string _key2420;
-              xfer += iprot->readString(_key2420);
-              std::string& _val2421 = this->success[_key2420];
-              xfer += iprot->readString(_val2421);
+              std::string _key2426;
+              xfer += iprot->readString(_key2426);
+              std::string& _val2427 = this->success[_key2426];
+              xfer += iprot->readString(_val2427);
             }
             xfer += iprot->readMapEnd();
           }
@@ -30390,11 +30390,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 _iter2422;
-      for (_iter2422 = this->success.begin(); _iter2422 != this->success.end(); ++_iter2422)
+      std::map<std::string, std::string> ::const_iterator _iter2428;
+      for (_iter2428 = this->success.begin(); _iter2428 != this->success.end(); ++_iter2428)
       {
-        xfer += oprot->writeString(_iter2422->first);
-        xfer += oprot->writeString(_iter2422->second);
+        xfer += oprot->writeString(_iter2428->first);
+        xfer += oprot->writeString(_iter2428->second);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -30439,17 +30439,17 @@ uint32_t ThriftHiveMetastore_partition_name_to_spec_presult::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size2423;
-            ::apache::thrift::protocol::TType _ktype2424;
-            ::apache::thrift::protocol::TType _vtype2425;
-            xfer += iprot->readMapBegin(_ktype2424, _vtype2425, _size2423);
-            uint32_t _i2427;
-            for (_i2427 = 0; _i2427 < _size2423; ++_i2427)
+            uint32_t _size2429;
+            ::apache::thrift::protocol::TType _ktype2430;
+            ::apache::thrift::protocol::TType _vtype2431;
+            xfer += iprot->readMapBegin(_ktype2430, _vtype2431, _size2429);
+            uint32_t _i2433;
+            for (_i2433 = 0; _i2433 < _size2429; ++_i2433)
             {
-              std::string _key2428;
-              xfer += iprot->readString(_key2428);
-              std::string& _val2429 = (*(this->success))[_key2428];
-              xfer += iprot->readString(_val2429);
+              std::string _key2434;
+              xfer += iprot->readString(_key2434);
+              std::string& _val2435 = (*(this->success))[_key2434];
+              xfer += iprot->readString(_val2435);
             }
             xfer += iprot->readMapEnd();
           }
@@ -30524,17 +30524,17 @@ uint32_t ThriftHiveMetastore_markPartitionForEvent_args::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->part_vals.clear();
-            uint32_t _size2430;
-            ::apache::thrift::protocol::TType _ktype2431;
-            ::apache::thrift::protocol::TType _vtype2432;
-            xfer += iprot->readMapBegin(_ktype2431, _vtype2432, _size2430);
-            uint32_t _i2434;
-            for (_i2434 = 0; _i2434 < _size2430; ++_i2434)
+            uint32_t _size2436;
+            ::apache::thrift::protocol::TType _ktype2437;
+            ::apache::thrift::protocol::TType _vtype2438;
+            xfer += iprot->readMapBegin(_ktype2437, _vtype2438, _size2436);
+            uint32_t _i2440;
+            for (_i2440 = 0; _i2440 < _size2436; ++_i2440)
             {
-              std::string _key2435;
-              xfer += iprot->readString(_key2435);
-              std::string& _val2436 = this->part_vals[_key2435];
-              xfer += iprot->readString(_val2436);
+              std::string _key2441;
+              xfer += iprot->readString(_key2441);
+              std::string& _val2442 = this->part_vals[_key2441];
+              xfer += iprot->readString(_val2442);
             }
             xfer += iprot->readMapEnd();
           }
@@ -30545,9 +30545,9 @@ uint32_t ThriftHiveMetastore_markPartitionForEvent_args::read(::apache::thrift::
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast2437;
-          xfer += iprot->readI32(ecast2437);
-          this->eventType = static_cast<PartitionEventType::type>(ecast2437);
+          int32_t ecast2443;
+          xfer += iprot->readI32(ecast2443);
+          this->eventType = static_cast<PartitionEventType::type>(ecast2443);
           this->__isset.eventType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -30581,11 +30581,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 _iter2438;
-    for (_iter2438 = this->part_vals.begin(); _iter2438 != this->part_vals.end(); ++_iter2438)
+    std::map<std::string, std::string> ::const_iterator _iter2444;
+    for (_iter2444 = this->part_vals.begin(); _iter2444 != this->part_vals.end(); ++_iter2444)
     {
-      xfer += oprot->writeString(_iter2438->first);
-      xfer += oprot->writeString(_iter2438->second);
+      xfer += oprot->writeString(_iter2444->first);
+      xfer += oprot->writeString(_iter2444->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -30621,11 +30621,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 _iter2439;
-    for (_iter2439 = (*(this->part_vals)).begin(); _iter2439 != (*(this->part_vals)).end(); ++_iter2439)
+    std::map<std::string, std::string> ::const_iterator _iter2445;
+    for (_iter2445 = (*(this->part_vals)).begin(); _iter2445 != (*(this->part_vals)).end(); ++_iter2445)
     {
-      xfer += oprot->writeString(_iter2439->first);
-      xfer += oprot->writeString(_iter2439->second);
+      xfer += oprot->writeString(_iter2445->first);
+      xfer += oprot->writeString(_iter2445->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -30894,17 +30894,17 @@ uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_args::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->part_vals.clear();
-            uint32_t _size2440;
-            ::apache::thrift::protocol::TType _ktype2441;
-            ::apache::thrift::protocol::TType _vtype2442;
-            xfer += iprot->readMapBegin(_ktype2441, _vtype2442, _size2440);
-            uint32_t _i2444;
-            for (_i2444 = 0; _i2444 < _size2440; ++_i2444)
+            uint32_t _size2446;
+            ::apache::thrift::protocol::TType _ktype2447;
+            ::apache::thrift::protocol::TType _vtype2448;
+            xfer += iprot->readMapBegin(_ktype2447, _vtype2448, _size2446);
+            uint32_t _i2450;
+            for (_i2450 = 0; _i2450 < _size2446; ++_i2450)
             {
-              std::string _key2445;
-              xfer += iprot->readString(_key2445);
-              std::string& _val2446 = this->part_vals[_key2445];
-              xfer += iprot->readString(_val2446);
+              std::string _key2451;
+              xfer += iprot->readString(_key2451);
+              std::string& _val2452 = this->part_vals[_key2451];
+              xfer += iprot->readString(_val2452);
             }
             xfer += iprot->readMapEnd();
           }
@@ -30915,9 +30915,9 @@ uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_args::read(::apache::thri
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast2447;
-          xfer += iprot->readI32(ecast2447);
-          this->eventType = static_cast<PartitionEventType::type>(ecast2447);
+          int32_t ecast2453;
+          xfer += iprot->readI32(ecast2453);
+          this->eventType = static_cast<PartitionEventType::type>(ecast2453);
           this->__isset.eventType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -30951,11 +30951,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 _iter2448;
-    for (_iter2448 = this->part_vals.begin(); _iter2448 != this->part_vals.end(); ++_iter2448)
+    std::map<std::string, std::string> ::const_iterator _iter2454;
+    for (_iter2454 = this->part_vals.begin(); _iter2454 != this->part_vals.end(); ++_iter2454)
     {
-      xfer += oprot->writeString(_iter2448->first);
-      xfer += oprot->writeString(_iter2448->second);
+      xfer += oprot->writeString(_iter2454->first);
+      xfer += oprot->writeString(_iter2454->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -30991,11 +30991,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 _iter2449;
-    for (_iter2449 = (*(this->part_vals)).begin(); _iter2449 != (*(this->part_vals)).end(); ++_iter2449)
+    std::map<std::string, std::string> ::const_iterator _iter2455;
+    for (_iter2455 = (*(this->part_vals)).begin(); _iter2455 != (*(this->part_vals)).end(); ++_iter2455)
     {
-      xfer += oprot->writeString(_iter2449->first);
-      xfer += oprot->writeString(_iter2449->second);
+      xfer += oprot->writeString(_iter2455->first);
+      xfer += oprot->writeString(_iter2455->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -37124,14 +37124,14 @@ uint32_t ThriftHiveMetastore_get_functions_result::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2450;
-            ::apache::thrift::protocol::TType _etype2453;
-            xfer += iprot->readListBegin(_etype2453, _size2450);
-            this->success.resize(_size2450);
-            uint32_t _i2454;
-            for (_i2454 = 0; _i2454 < _size2450; ++_i2454)
+            uint32_t _size2456;
+            ::apache::thrift::protocol::TType _etype2459;
+            xfer += iprot->readListBegin(_etype2459, _size2456);
+            this->success.resize(_size2456);
+            uint32_t _i2460;
+            for (_i2460 = 0; _i2460 < _size2456; ++_i2460)
             {
-              xfer += iprot->readString(this->success[_i2454]);
+              xfer += iprot->readString(this->success[_i2460]);
             }
             xfer += iprot->readListEnd();
           }
@@ -37170,10 +37170,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 _iter2455;
-      for (_iter2455 = this->success.begin(); _iter2455 != this->success.end(); ++_iter2455)
+      std::vector<std::string> ::const_iterator _iter2461;
+      for (_iter2461 = this->success.begin(); _iter2461 != this->success.end(); ++_iter2461)
       {
-        xfer += oprot->writeString((*_iter2455));
+        xfer += oprot->writeString((*_iter2461));
       }
       xfer += oprot->writeListEnd();
     }
@@ -37218,14 +37218,14 @@ uint32_t ThriftHiveMetastore_get_functions_presult::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2456;
-            ::apache::thrift::protocol::TType _etype2459;
-            xfer += iprot->readListBegin(_etype2459, _size2456);
-            (*(this->success)).resize(_size2456);
-            uint32_t _i2460;
-            for (_i2460 = 0; _i2460 < _size2456; ++_i2460)
+            uint32_t _size2462;
+            ::apache::thrift::protocol::TType _etype2465;
+            xfer += iprot->readListBegin(_etype2465, _size2462);
+            (*(this->success)).resize(_size2462);
+            uint32_t _i2466;
+            for (_i2466 = 0; _i2466 < _size2462; ++_i2466)
             {
-              xfer += iprot->readString((*(this->success))[_i2460]);
+              xfer += iprot->readString((*(this->success))[_i2466]);
             }
             xfer += iprot->readListEnd();
           }
@@ -38185,14 +38185,14 @@ uint32_t ThriftHiveMetastore_get_role_names_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2461;
-            ::apache::thrift::protocol::TType _etype2464;
-            xfer += iprot->readListBegin(_etype2464, _size2461);
-            this->success.resize(_size2461);
-            uint32_t _i2465;
-            for (_i2465 = 0; _i2465 < _size2461; ++_i2465)
+            uint32_t _size2467;
+            ::apache::thrift::protocol::TType _etype2470;
+            xfer += iprot->readListBegin(_etype2470, _size2467);
+            this->success.resize(_size2467);
+            uint32_t _i2471;
+            for (_i2471 = 0; _i2471 < _size2467; ++_i2471)
             {
-              xfer += iprot->readString(this->success[_i2465]);
+              xfer += iprot->readString(this->success[_i2471]);
             }
             xfer += iprot->readListEnd();
           }
@@ -38231,10 +38231,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 _iter2466;
-      for (_iter2466 = this->success.begin(); _iter2466 != this->success.end(); ++_iter2466)
+      std::vector<std::string> ::const_iterator _iter2472;
+      for (_iter2472 = this->success.begin(); _iter2472 != this->success.end(); ++_iter2472)
       {
-        xfer += oprot->writeString((*_iter2466));
+        xfer += oprot->writeString((*_iter2472));
       }
       xfer += oprot->writeListEnd();
     }
@@ -38279,14 +38279,14 @@ uint32_t ThriftHiveMetastore_get_role_names_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2467;
-            ::apache::thrift::protocol::TType _etype2470;
-            xfer += iprot->readListBegin(_etype2470, _size2467);
-            (*(this->success)).resize(_size2467);
-            uint32_t _i2471;
-            for (_i2471 = 0; _i2471 < _size2467; ++_i2471)
+            uint32_t _size2473;
+            ::apache::thrift::protocol::TType _etype2476;
+            xfer += iprot->readListBegin(_etype2476, _size2473);
+            (*(this->success)).resize(_size2473);
+            uint32_t _i2477;
+            for (_i2477 = 0; _i2477 < _size2473; ++_i2477)
             {
-              xfer += iprot->readString((*(this->success))[_i2471]);
+              xfer += iprot->readString((*(this->success))[_i2477]);
             }
             xfer += iprot->readListEnd();
           }
@@ -38359,9 +38359,9 @@ uint32_t ThriftHiveMetastore_grant_role_args::read(::apache::thrift::protocol::T
         break;
       case 3:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast2472;
-          xfer += iprot->readI32(ecast2472);
-          this->principal_type = static_cast<PrincipalType::type>(ecast2472);
+          int32_t ecast2478;
+          xfer += iprot->readI32(ecast2478);
+          this->principal_type = static_cast<PrincipalType::type>(ecast2478);
           this->__isset.principal_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -38377,9 +38377,9 @@ uint32_t ThriftHiveMetastore_grant_role_args::read(::apache::thrift::protocol::T
         break;
       case 5:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast2473;
-          xfer += iprot->readI32(ecast2473);
-          this->grantorType = static_cast<PrincipalType::type>(ecast2473);
+          int32_t ecast2479;
+          xfer += iprot->readI32(ecast2479);
+          this->grantorType = static_cast<PrincipalType::type>(ecast2479);
           this->__isset.grantorType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -38650,9 +38650,9 @@ uint32_t ThriftHiveMetastore_revoke_role_args::read(::apache::thrift::protocol::
         break;
       case 3:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast2474;
-          xfer += iprot->readI32(ecast2474);
-          this->principal_type = static_cast<PrincipalType::type>(ecast2474);
+          int32_t ecast2480;
+          xfer += iprot->readI32(ecast2480);
+          this->principal_type = static_cast<PrincipalType::type>(ecast2480);
           this->__isset.principal_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -38883,9 +38883,9 @@ uint32_t ThriftHiveMetastore_list_roles_args::read(::apache::thrift::protocol::T
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast2475;
-          xfer += iprot->readI32(ecast2475);
-          this->principal_type = static_cast<PrincipalType::type>(ecast2475);
+          int32_t ecast2481;
+          xfer += iprot->readI32(ecast2481);
+          this->principal_type = static_cast<PrincipalType::type>(ecast2481);
           this->__isset.principal_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -38974,14 +38974,14 @@ uint32_t ThriftHiveMetastore_list_roles_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2476;
-            ::apache::thrift::protocol::TType _etype2479;
-            xfer += iprot->readListBegin(_etype2479, _size2476);
-            this->success.resize(_size2476);
-            uint32_t _i2480;
-            for (_i2480 = 0; _i2480 < _size2476; ++_i2480)
+            uint32_t _size2482;
+            ::apache::thrift::protocol::TType _etype2485;
+            xfer += iprot->readListBegin(_etype2485, _size2482);
+            this->success.resize(_size2482);
+            uint32_t _i2486;
+            for (_i2486 = 0; _i2486 < _size2482; ++_i2486)
             {
-              xfer += this->success[_i2480].read(iprot);
+              xfer += this->success[_i2486].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -39020,10 +39020,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 _iter2481;
-      for (_iter2481 = this->success.begin(); _iter2481 != this->success.end(); ++_iter2481)
+      std::vector<Role> ::const_iterator _iter2487;
+      for (_iter2487 = this->success.begin(); _iter2487 != this->success.end(); ++_iter2487)
       {
-        xfer += (*_iter2481).write(oprot);
+        xfer += (*_iter2487).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -39068,14 +39068,14 @@ uint32_t ThriftHiveMetastore_list_roles_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2482;
-            ::apache::thrift::protocol::TType _etype2485;
-            xfer += iprot->readListBegin(_etype2485, _size2482);
-            (*(this->success)).resize(_size2482);
-            uint32_t _i2486;
-            for (_i2486 = 0; _i2486 < _size2482; ++_i2486)
+            uint32_t _size2488;
+            ::apache::thrift::protocol::TType _etype2491;
+            xfer += iprot->readListBegin(_etype2491, _size2488);
+            (*(this->success)).resize(_size2488);
+            uint32_t _i2492;
+            for (_i2492 = 0; _i2492 < _size2488; ++_i2492)
             {
-              xfer += (*(this->success))[_i2486].read(iprot);
+              xfer += (*(this->success))[_i2492].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -39771,14 +39771,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 _size2487;
-            ::apache::thrift::protocol::TType _etype2490;
-            xfer += iprot->readListBegin(_etype2490, _size2487);
-            this->group_names.resize(_size2487);
-            uint32_t _i2491;
-            for (_i2491 = 0; _i2491 < _size2487; ++_i2491)
+            uint32_t _size2493;
+            ::apache::thrift::protocol::TType _etype2496;
+            xfer += iprot->readListBegin(_etype2496, _size2493);
+            this->group_names.resize(_size2493);
+            uint32_t _i2497;
+            for (_i2497 = 0; _i2497 < _size2493; ++_i2497)
             {
-              xfer += iprot->readString(this->group_names[_i2491]);
+              xfer += iprot->readString(this->group_names[_i2497]);
             }
             xfer += iprot->readListEnd();
           }
@@ -39815,10 +39815,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 _iter2492;
-    for (_iter2492 = this->group_names.begin(); _iter2492 != this->group_names.end(); ++_iter2492)
+    std::vector<std::string> ::const_iterator _iter2498;
+    for (_iter2498 = this->group_names.begin(); _iter2498 != this->group_names.end(); ++_iter2498)
     {
-      xfer += oprot->writeString((*_iter2492));
+      xfer += oprot->writeString((*_iter2498));
     }
     xfer += oprot->writeListEnd();
   }
@@ -39850,10 +39850,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 _iter2493;
-    for (_iter2493 = (*(this->group_names)).begin(); _iter2493 != (*(this->group_names)).end(); ++_iter2493)
+    std::vector<std::string> ::const_iterator _iter2499;
+    for (_iter2499 = (*(this->group_names)).begin(); _iter2499 != (*(this->group_names)).end(); ++_iter2499)
     {
-      xfer += oprot->writeString((*_iter2493));
+      xfer += oprot->writeString((*_iter2499));
     }
     xfer += oprot->writeListEnd();
   }
@@ -40028,9 +40028,9 @@ uint32_t ThriftHiveMetastore_list_privileges_args::read(::apache::thrift::protoc
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast2494;
-          xfer += iprot->readI32(ecast2494);
-          this->principal_type = static_cast<PrincipalType::type>(ecast2494);
+          int32_t ecast2500;
+          xfer += iprot->readI32(ecast2500);
+          this->principal_type = static_cast<PrincipalType::type>(ecast2500);
           this->__isset.principal_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -40135,14 +40135,14 @@ uint32_t ThriftHiveMetastore_list_privileges_result::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2495;
-            ::apache::thrift::protocol::TType _etype2498;
-            xfer += iprot->readListBegin(_etype2498, _size2495);
-            this->success.resize(_size2495);
-            uint32_t _i2499;
-            for (_i2499 = 0; _i2499 < _size2495; ++_i2499)
+            uint32_t _size2501;
+            ::apache::thrift::protocol::TType _etype2504;
+            xfer += iprot->readListBegin(_etype2504, _size2501);
+            this->success.resize(_size2501);
+            uint32_t _i2505;
+            for (_i2505 = 0; _i2505 < _size2501; ++_i2505)
             {
-              xfer += this->success[_i2499].read(iprot);
+              xfer += this->success[_i2505].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -40181,10 +40181,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 _iter2500;
-      for (_iter2500 = this->success.begin(); _iter2500 != this->success.end(); ++_iter2500)
+      std::vector<HiveObjectPrivilege> ::const_iterator _iter2506;
+      for (_iter2506 = this->success.begin(); _iter2506 != this->success.end(); ++_iter2506)
       {
-        xfer += (*_iter2500).write(oprot);
+        xfer += (*_iter2506).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -40229,14 +40229,14 @@ uint32_t ThriftHiveMetastore_list_privileges_presult::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2501;
-            ::apache::thrift::protocol::TType _etype2504;
-            xfer += iprot->readListBegin(_etype2504, _size2501);
-            (*(this->success)).resize(_size2501);
-            uint32_t _i2505;
-            for (_i2505 = 0; _i2505 < _size2501; ++_i2505)
+            uint32_t _size2507;
+            ::apache::thrift::protocol::TType _etype2510;
+            xfer += iprot->readListBegin(_etype2510, _size2507);
+            (*(this->success)).resize(_size2507);
+            uint32_t _i2511;
+            for (_i2511 = 0; _i2511 < _size2507; ++_i2511)
             {
-              xfer += (*(this->success))[_i2505].read(iprot);
+              xfer += (*(this->success))[_i2511].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -41163,14 +41163,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 _size2506;
-            ::apache::thrift::protocol::TType _etype2509;
-            xfer += iprot->readListBegin(_etype2509, _size2506);
-            this->group_names.resize(_size2506);
-            uint32_t _i2510;
-            for (_i2510 = 0; _i2510 < _size2506; ++_i2510)
+            uint32_t _size2512;
+            ::apache::thrift::protocol::TType _etype2515;
+            xfer += iprot->readListBegin(_etype2515, _size2512);
+            this->group_names.resize(_size2512);
+            uint32_t _i2516;
+            for (_i2516 = 0; _i2516 < _size2512; ++_i2516)
             {
-              xfer += iprot->readString(this->group_names[_i2510]);
+              xfer += iprot->readString(this->group_names[_i2516]);
             }
             xfer += iprot->readListEnd();
           }
@@ -41203,10 +41203,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 _iter2511;
-    for (_iter2511 = this->group_names.begin(); _iter2511 != this->group_names.end(); ++_iter2511)
+    std::vector<std::string> ::const_iterator _iter2517;
+    for (_iter2517 = this->group_names.begin(); _iter2517 != this->group_names.end(); ++_iter2517)
     {
-      xfer += oprot->writeString((*_iter2511));
+      xfer += oprot->writeString((*_iter2517));
     }
     xfer += oprot->writeListEnd();
   }
@@ -41234,10 +41234,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 _iter2512;
-    for (_iter2512 = (*(this->group_names)).begin(); _iter2512 != (*(this->group_names)).end(); ++_iter2512)
+    std::vector<std::string> ::const_iterator _iter2518;
+    for (_iter2518 = (*(this->group_names)).begin(); _iter2518 != (*(this->group_names)).end(); ++_iter2518)
     {
-      xfer += oprot->writeString((*_iter2512));
+      xfer += oprot->writeString((*_iter2518));
     }
     xfer += oprot->writeListEnd();
   }
@@ -41278,14 +41278,14 @@ uint32_t ThriftHiveMetastore_set_ugi_result::read(::apache::thrift::protocol::TP
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2513;
-            ::apache::thrift::protocol::TType _etype2516;
-            xfer += iprot->readListBegin(_etype2516, _size2513);
-            this->success.resize(_size2513);
-            uint32_t _i2517;
-            for (_i2517 = 0; _i2517 < _size2513; ++_i2517)
+            uint32_t _size2519;
+            ::apache::thrift::protocol::TType _etype2522;
+            xfer += iprot->readListBegin(_etype2522, _size2519);
+            this->success.resize(_size2519);
+            uint32_t _i2523;
+            for (_i2523 = 0; _i2523 < _size2519; ++_i2523)
             {
-              xfer += iprot->readString(this->success[_i2517]);
+              xfer += iprot->readString(this->success[_i2523]);
             }
             xfer += iprot->readListEnd();
           }
@@ -41324,10 +41324,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 _iter2518;
-      for (_iter2518 = this->success.begin(); _iter2518 != this->success.end(); ++_iter2518)
+      std::vector<std::string> ::const_iterator _iter2524;
+      for (_iter2524 = this->success.begin(); _iter2524 != this->success.end(); ++_iter2524)
       {
-        xfer += oprot->writeString((*_iter2518));
+        xfer += oprot->writeString((*_iter2524));
       }
       xfer += oprot->writeListEnd();
     }
@@ -41372,14 +41372,14 @@ uint32_t ThriftHiveMetastore_set_ugi_presult::read(::apache::thrift::protocol::T
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2519;
-            ::apache::thrift::protocol::TType _etype2522;
-            xfer += iprot->readListBegin(_etype2522, _size2519);
-            (*(this->success)).resize(_size2519);
-            uint32_t _i2523;
-            for (_i2523 = 0; _i2523 < _size2519; ++_i2523)
+            uint32_t _size2525;
+            ::apache::thrift::protocol::TType _etype2528;
+            xfer += iprot->readListBegin(_etype2528, _size2525);
+            (*(this->success)).resize(_size2525);
+            uint32_t _i2529;
+            for (_i2529 = 0; _i2529 < _size2525; ++_i2529)
             {
-              xfer += iprot->readString((*(this->success))[_i2523]);
+              xfer += iprot->readString((*(this->success))[_i2529]);
             }
             xfer += iprot->readListEnd();
           }
@@ -42690,14 +42690,14 @@ uint32_t ThriftHiveMetastore_get_all_token_identifiers_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2524;
-            ::apache::thrift::protocol::TType _etype2527;
-            xfer += iprot->readListBegin(_etype2527, _size2524);
-            this->success.resize(_size2524);
-            uint32_t _i2528;
-            for (_i2528 = 0; _i2528 < _size2524; ++_i2528)
+            uint32_t _size2530;
+            ::apache::thrift::protocol::TType _etype2533;
+            xfer += iprot->readListBegin(_etype2533, _size2530);
+            this->success.resize(_size2530);
+            uint32_t _i2534;
+            for (_i2534 = 0; _i2534 < _size2530; ++_i2534)
             {
-              xfer += iprot->readString(this->success[_i2528]);
+              xfer += iprot->readString(this->success[_i2534]);
             }
             xfer += iprot->readListEnd();
           }
@@ -42728,10 +42728,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 _iter2529;
-      for (_iter2529 = this->success.begin(); _iter2529 != this->success.end(); ++_iter2529)
+      std::vector<std::string> ::const_iterator _iter2535;
+      for (_iter2535 = this->success.begin(); _iter2535 != this->success.end(); ++_iter2535)
       {
-        xfer += oprot->writeString((*_iter2529));
+        xfer += oprot->writeString((*_iter2535));
       }
       xfer += oprot->writeListEnd();
     }
@@ -42772,14 +42772,14 @@ uint32_t ThriftHiveMetastore_get_all_token_identifiers_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2530;
-            ::apache::thrift::protocol::TType _etype2533;
-            xfer += iprot->readListBegin(_etype2533, _size2530);
-            (*(this->success)).resize(_size2530);
-            uint32_t _i2534;
-            for (_i2534 = 0; _i2534 < _size2530; ++_i2534)
+            uint32_t _size2536;
+            ::apache::thrift::protocol::TType _etype2539;
+            xfer += iprot->readListBegin(_etype2539, _size2536);
+            (*(this->success)).resize(_size2536);
+            uint32_t _i2540;
+            for (_i2540 = 0; _i2540 < _size2536; ++_i2540)
             {
-              xfer += iprot->readString((*(this->success))[_i2534]);
+              xfer += iprot->readString((*(this->success))[_i2540]);
             }
             xfer += iprot->readListEnd();
           }
@@ -43505,14 +43505,14 @@ uint32_t ThriftHiveMetastore_get_master_keys_result::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2535;
-            ::apache::thrift::protocol::TType _etype2538;
-            xfer += iprot->readListBegin(_etype2538, _size2535);
-            this->success.resize(_size2535);
-            uint32_t _i2539;
-            for (_i2539 = 0; _i2539 < _size2535; ++_i2539)
+            uint32_t _size2541;
+            ::apache::thrift::protocol::TType _etype2544;
+            xfer += iprot->readListBegin(_etype2544, _size2541);
+            this->success.resize(_size2541);
+            uint32_t _i2545;
+            for (_i2545 = 0; _i2545 < _size2541; ++_i2545)
             {
-              xfer += iprot->readString(this->success[_i2539]);
+              xfer += iprot->readString(this->success[_i2545]);
             }
             xfer += iprot->readListEnd();
           }
@@ -43543,10 +43543,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 _iter2540;
-      for (_iter2540 = this->success.begin(); _iter2540 != this->success.end(); ++_iter2540)
+      std::vector<std::string> ::const_iterator _iter2546;
+      for (_iter2546 = this->success.begin(); _iter2546 != this->success.end(); ++_iter2546)
       {
-        xfer += oprot->writeString((*_iter2540));
+        xfer += oprot->writeString((*_iter2546));
       }
       xfer += oprot->writeListEnd();
     }
@@ -43587,14 +43587,14 @@ uint32_t ThriftHiveMetastore_get_master_keys_presult::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2541;
-            ::apache::thrift::protocol::TType _etype2544;
-            xfer += iprot->readListBegin(_etype2544, _size2541);
-            (*(this->success)).resize(_size2541);
-            uint32_t _i2545;
-            for (_i2545 = 0; _i2545 < _size2541; ++_i2545)
+            uint32_t _size2547;
+            ::apache::thrift::protocol::TType _etype2550;
+            xfer += iprot->readListBegin(_etype2550, _size2547);
+            (*(this->success)).resize(_size2547);
+            uint32_t _i2551;
+            for (_i2551 = 0; _i2551 < _size2547; ++_i2551)
             {
-              xfer += iprot->readString((*(this->success))[_i2545]);
+              xfer += iprot->readString((*(this->success))[_i2551]);
             }
             xfer += iprot->readListEnd();
           }
@@ -45343,17 +45343,17 @@ uint32_t ThriftHiveMetastore_add_write_ids_to_min_history_args::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->writeIds.clear();
-            uint32_t _size2546;
-            ::apache::thrift::protocol::TType _ktype2547;
-            ::apache::thrift::protocol::TType _vtype2548;
-            xfer += iprot->readMapBegin(_ktype2547, _vtype2548, _size2546);
-            uint32_t _i2550;
-            for (_i2550 = 0; _i2550 < _size2546; ++_i2550)
+            uint32_t _size2552;
+            ::apache::thrift::protocol::TType _ktype2553;
+            ::apache::thrift::protocol::TType _vtype2554;
+            xfer += iprot->readMapBegin(_ktype2553, _vtype2554, _size2552);
+            uint32_t _i2556;
+            for (_i2556 = 0; _i2556 < _size2552; ++_i2556)
             {
-              std::string _key2551;
-              xfer += iprot->readString(_key2551);
-              int64_t& _val2552 = this->writeIds[_key2551];
-              xfer += iprot->readI64(_val2552);
+              std::string _key2557;
+              xfer += iprot->readString(_key2557);
+              int64_t& _val2558 = this->writeIds[_key2557];
+              xfer += iprot->readI64(_val2558);
             }
             xfer += iprot->readMapEnd();
           }
@@ -45386,11 +45386,11 @@ uint32_t ThriftHiveMetastore_add_write_ids_to_min_history_args::write(::apache::
   xfer += oprot->writeFieldBegin("writeIds", ::apache::thrift::protocol::T_MAP, 2);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->writeIds.size()));
-    std::map<std::string, int64_t> ::const_iterator _iter2553;
-    for (_iter2553 = this->writeIds.begin(); _iter2553 != this->writeIds.end(); ++_iter2553)
+    std::map<std::string, int64_t> ::const_iterator _iter2559;
+    for (_iter2559 = this->writeIds.begin(); _iter2559 != this->writeIds.end(); ++_iter2559)
     {
-      xfer += oprot->writeString(_iter2553->first);
-      xfer += oprot->writeI64(_iter2553->second);
+      xfer += oprot->writeString(_iter2559->first);
+      xfer += oprot->writeI64(_iter2559->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -45418,11 +45418,11 @@ uint32_t ThriftHiveMetastore_add_write_ids_to_min_history_pargs::write(::apache:
   xfer += oprot->writeFieldBegin("writeIds", ::apache::thrift::protocol::T_MAP, 2);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_I64, static_cast<uint32_t>((*(this->writeIds)).size()));
-    std::map<std::string, int64_t> ::const_iterator _iter2554;
-    for (_iter2554 = (*(this->writeIds)).begin(); _iter2554 != (*(this->writeIds)).end(); ++_iter2554)
+    std::map<std::string, int64_t> ::const_iterator _iter2560;
+    for (_iter2560 = (*(this->writeIds)).begin(); _iter2560 != (*(this->writeIds)).end(); ++_iter2560)
     {
-      xfer += oprot->writeString(_iter2554->first);
-      xfer += oprot->writeI64(_iter2554->second);
+      xfer += oprot->writeString(_iter2560->first);
+      xfer += oprot->writeI64(_iter2560->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -49322,14 +49322,14 @@ uint32_t ThriftHiveMetastore_find_columns_with_stats_result::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2555;
-            ::apache::thrift::protocol::TType _etype2558;
-            xfer += iprot->readListBegin(_etype2558, _size2555);
-            this->success.resize(_size2555);
-            uint32_t _i2559;
-            for (_i2559 = 0; _i2559 < _size2555; ++_i2559)
+            uint32_t _size2561;
+            ::apache::thrift::protocol::TType _etype2564;
+            xfer += iprot->readListBegin(_etype2564, _size2561);
+            this->success.resize(_size2561);
+            uint32_t _i2565;
+            for (_i2565 = 0; _i2565 < _size2561; ++_i2565)
             {
-              xfer += iprot->readString(this->success[_i2559]);
+              xfer += iprot->readString(this->success[_i2565]);
             }
             xfer += iprot->readListEnd();
           }
@@ -49360,10 +49360,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 _iter2560;
-      for (_iter2560 = this->success.begin(); _iter2560 != this->success.end(); ++_iter2560)
+      std::vector<std::string> ::const_iterator _iter2566;
+      for (_iter2566 = this->success.begin(); _iter2566 != this->success.end(); ++_iter2566)
       {
-        xfer += oprot->writeString((*_iter2560));
+        xfer += oprot->writeString((*_iter2566));
       }
       xfer += oprot->writeListEnd();
     }
@@ -49404,14 +49404,14 @@ uint32_t ThriftHiveMetastore_find_columns_with_stats_presult::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2561;
-            ::apache::thrift::protocol::TType _etype2564;
-            xfer += iprot->readListBegin(_etype2564, _size2561);
-            (*(this->success)).resize(_size2561);
-            uint32_t _i2565;
-            for (_i2565 = 0; _i2565 < _size2561; ++_i2565)
+            uint32_t _size2567;
+            ::apache::thrift::protocol::TType _etype2570;
+            xfer += iprot->readListBegin(_etype2570, _size2567);
+            (*(this->success)).resize(_size2567);
+            uint32_t _i2571;
+            for (_i2571 = 0; _i2571 < _size2567; ++_i2571)
             {
-              xfer += iprot->readString((*(this->success))[_i2565]);
+              xfer += iprot->readString((*(this->success))[_i2571]);
             }
             xfer += iprot->readListEnd();
           }
@@ -59334,14 +59334,14 @@ uint32_t ThriftHiveMetastore_get_schema_all_versions_result::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2566;
-            ::apache::thrift::protocol::TType _etype2569;
-            xfer += iprot->readListBegin(_etype2569, _size2566);
-            this->success.resize(_size2566);
-            uint32_t _i2570;
-            for (_i2570 = 0; _i2570 < _size2566; ++_i2570)
+            uint32_t _size2572;
+            ::apache::thrift::protocol::TType _etype2575;
+            xfer += iprot->readListBegin(_etype2575, _size2572);
+            this->success.resize(_size2572);
+            uint32_t _i2576;
+            for (_i2576 = 0; _i2576 < _size2572; ++_i2576)
             {
-              xfer += this->success[_i2570].read(iprot);
+              xfer += this->success[_i2576].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -59388,10 +59388,10 @@ uint32_t ThriftHiveMetastore_get_schema_all_versions_result::write(::apache::thr
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<SchemaVersion> ::const_iterator _iter2571;
-      for (_iter2571 = this->success.begin(); _iter2571 != this->success.end(); ++_iter2571)
+      std::vector<SchemaVersion> ::const_iterator _iter2577;
+      for (_iter2577 = this->success.begin(); _iter2577 != this->success.end(); ++_iter2577)
       {
-        xfer += (*_iter2571).write(oprot);
+        xfer += (*_iter2577).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -59440,14 +59440,14 @@ uint32_t ThriftHiveMetastore_get_schema_all_versions_presult::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2572;
-            ::apache::thrift::protocol::TType _etype2575;
-            xfer += iprot->readListBegin(_etype2575, _size2572);
-            (*(this->success)).resize(_size2572);
-            uint32_t _i2576;
-            for (_i2576 = 0; _i2576 < _size2572; ++_i2576)
+            uint32_t _size2578;
+            ::apache::thrift::protocol::TType _etype2581;
+            xfer += iprot->readListBegin(_etype2581, _size2578);
+            (*(this->success)).resize(_size2578);
+            uint32_t _i2582;
+            for (_i2582 = 0; _i2582 < _size2578; ++_i2582)
             {
-              xfer += (*(this->success))[_i2576].read(iprot);
+              xfer += (*(this->success))[_i2582].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -61500,14 +61500,14 @@ uint32_t ThriftHiveMetastore_get_runtime_stats_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2577;
-            ::apache::thrift::protocol::TType _etype2580;
-            xfer += iprot->readListBegin(_etype2580, _size2577);
-            this->success.resize(_size2577);
-            uint32_t _i2581;
-            for (_i2581 = 0; _i2581 < _size2577; ++_i2581)
+            uint32_t _size2583;
+            ::apache::thrift::protocol::TType _etype2586;
+            xfer += iprot->readListBegin(_etype2586, _size2583);
+            this->success.resize(_size2583);
+            uint32_t _i2587;
+            for (_i2587 = 0; _i2587 < _size2583; ++_i2587)
             {
-              xfer += this->success[_i2581].read(iprot);
+              xfer += this->success[_i2587].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -61546,10 +61546,10 @@ uint32_t ThriftHiveMetastore_get_runtime_stats_result::write(::apache::thrift::p
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<RuntimeStat> ::const_iterator _iter2582;
-      for (_iter2582 = this->success.begin(); _iter2582 != this->success.end(); ++_iter2582)
+      std::vector<RuntimeStat> ::const_iterator _iter2588;
+      for (_iter2588 = this->success.begin(); _iter2588 != this->success.end(); ++_iter2588)
       {
-        xfer += (*_iter2582).write(oprot);
+        xfer += (*_iter2588).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -61594,14 +61594,14 @@ uint32_t ThriftHiveMetastore_get_runtime_stats_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2583;
-            ::apache::thrift::protocol::TType _etype2586;
-            xfer += iprot->readListBegin(_etype2586, _size2583);
-            (*(this->success)).resize(_size2583);
-            uint32_t _i2587;
-            for (_i2587 = 0; _i2587 < _size2583; ++_i2587)
+            uint32_t _size2589;
+            ::apache::thrift::protocol::TType _etype2592;
+            xfer += iprot->readListBegin(_etype2592, _size2589);
+            (*(this->success)).resize(_size2589);
+            uint32_t _i2593;
+            for (_i2593 = 0; _i2593 < _size2589; ++_i2593)
             {
-              xfer += (*(this->success))[_i2587].read(iprot);
+              xfer += (*(this->success))[_i2593].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -64036,14 +64036,14 @@ uint32_t ThriftHiveMetastore_get_all_stored_procedures_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2588;
-            ::apache::thrift::protocol::TType _etype2591;
-            xfer += iprot->readListBegin(_etype2591, _size2588);
-            this->success.resize(_size2588);
-            uint32_t _i2592;
-            for (_i2592 = 0; _i2592 < _size2588; ++_i2592)
+            uint32_t _size2594;
+            ::apache::thrift::protocol::TType _etype2597;
+            xfer += iprot->readListBegin(_etype2597, _size2594);
+            this->success.resize(_size2594);
+            uint32_t _i2598;
+            for (_i2598 = 0; _i2598 < _size2594; ++_i2598)
             {
-              xfer += iprot->readString(this->success[_i2592]);
+              xfer += iprot->readString(this->success[_i2598]);
             }
             xfer += iprot->readListEnd();
           }
@@ -64082,10 +64082,10 @@ uint32_t ThriftHiveMetastore_get_all_stored_procedures_result::write(::apache::t
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter2593;
-      for (_iter2593 = this->success.begin(); _iter2593 != this->success.end(); ++_iter2593)
+      std::vector<std::string> ::const_iterator _iter2599;
+      for (_iter2599 = this->success.begin(); _iter2599 != this->success.end(); ++_iter2599)
       {
-        xfer += oprot->writeString((*_iter2593));
+        xfer += oprot->writeString((*_iter2599));
       }
       xfer += oprot->writeListEnd();
     }
@@ -64130,14 +64130,14 @@ uint32_t ThriftHiveMetastore_get_all_stored_procedures_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2594;
-            ::apache::thrift::protocol::TType _etype2597;
-            xfer += iprot->readListBegin(_etype2597, _size2594);
-            (*(this->success)).resize(_size2594);
-            uint32_t _i2598;
-            for (_i2598 = 0; _i2598 < _size2594; ++_i2598)
+            uint32_t _size2600;
+            ::apache::thrift::protocol::TType _etype2603;
+            xfer += iprot->readListBegin(_etype2603, _size2600);
+            (*(this->success)).resize(_size2600);
+            uint32_t _i2604;
+            for (_i2604 = 0; _i2604 < _size2600; ++_i2604)
             {
-              xfer += iprot->readString((*(this->success))[_i2598]);
+              xfer += iprot->readString((*(this->success))[_i2604]);
             }
             xfer += iprot->readListEnd();
           }
@@ -64689,14 +64689,14 @@ uint32_t ThriftHiveMetastore_get_all_packages_result::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2599;
-            ::apache::thrift::protocol::TType _etype2602;
-            xfer += iprot->readListBegin(_etype2602, _size2599);
-            this->success.resize(_size2599);
-            uint32_t _i2603;
-            for (_i2603 = 0; _i2603 < _size2599; ++_i2603)
+            uint32_t _size2605;
+            ::apache::thrift::protocol::TType _etype2608;
+            xfer += iprot->readListBegin(_etype2608, _size2605);
+            this->success.resize(_size2605);
+            uint32_t _i2609;
+            for (_i2609 = 0; _i2609 < _size2605; ++_i2609)
             {
-              xfer += iprot->readString(this->success[_i2603]);
+              xfer += iprot->readString(this->success[_i2609]);
             }
             xfer += iprot->readListEnd();
           }
@@ -64735,10 +64735,10 @@ uint32_t ThriftHiveMetastore_get_all_packages_result::write(::apache::thrift::pr
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter2604;
-      for (_iter2604 = this->success.begin(); _iter2604 != this->success.end(); ++_iter2604)
+      std::vector<std::string> ::const_iterator _iter2610;
+      for (_iter2610 = this->success.begin(); _iter2610 != this->success.end(); ++_iter2610)
       {
-        xfer += oprot->writeString((*_iter2604));
+        xfer += oprot->writeString((*_iter2610));
       }
       xfer += oprot->writeListEnd();
     }
@@ -64783,14 +64783,14 @@ uint32_t ThriftHiveMetastore_get_all_packages_presult::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2605;
-            ::apache::thrift::protocol::TType _etype2608;
-            xfer += iprot->readListBegin(_etype2608, _size2605);
-            (*(this->success)).resize(_size2605);
-            uint32_t _i2609;
-            for (_i2609 = 0; _i2609 < _size2605; ++_i2609)
+            uint32_t _size2611;
+            ::apache::thrift::protocol::TType _etype2614;
+            xfer += iprot->readListBegin(_etype2614, _size2611);
+            (*(this->success)).resize(_size2611);
+            uint32_t _i2615;
+            for (_i2615 = 0; _i2615 < _size2611; ++_i2615)
             {
-              xfer += iprot->readString((*(this->success))[_i2609]);
+              xfer += iprot->readString((*(this->success))[_i2615]);
             }
             xfer += iprot->readListEnd();
           }
@@ -65115,14 +65115,14 @@ uint32_t ThriftHiveMetastore_get_all_write_event_info_result::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2610;
-            ::apache::thrift::protocol::TType _etype2613;
-            xfer += iprot->readListBegin(_etype2613, _size2610);
-            this->success.resize(_size2610);
-            uint32_t _i2614;
-            for (_i2614 = 0; _i2614 < _size2610; ++_i2614)
+            uint32_t _size2616;
+            ::apache::thrift::protocol::TType _etype2619;
+            xfer += iprot->readListBegin(_etype2619, _size2616);
+            this->success.resize(_size2616);
+            uint32_t _i2620;
+            for (_i2620 = 0; _i2620 < _size2616; ++_i2620)
             {
-              xfer += this->success[_i2614].read(iprot);
+              xfer += this->success[_i2620].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -65161,10 +65161,10 @@ uint32_t ThriftHiveMetastore_get_all_write_event_info_result::write(::apache::th
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<WriteEventInfo> ::const_iterator _iter2615;
-      for (_iter2615 = this->success.begin(); _iter2615 != this->success.end(); ++_iter2615)
+      std::vector<WriteEventInfo> ::const_iterator _iter2621;
+      for (_iter2621 = this->success.begin(); _iter2621 != this->success.end(); ++_iter2621)
       {
-        xfer += (*_iter2615).write(oprot);
+        xfer += (*_iter2621).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -65209,14 +65209,14 @@ uint32_t ThriftHiveMetastore_get_all_write_event_info_presult::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2616;
-            ::apache::thrift::protocol::TType _etype2619;
-            xfer += iprot->readListBegin(_etype2619, _size2616);
-            (*(this->success)).resize(_size2616);
-            uint32_t _i2620;
-            for (_i2620 = 0; _i2620 < _size2616; ++_i2620)
+            uint32_t _size2622;
+            ::apache::thrift::protocol::TType _etype2625;
+            xfer += iprot->readListBegin(_etype2625, _size2622);
+            (*(this->success)).resize(_size2622);
+            uint32_t _i2626;
+            for (_i2626 = 0; _i2626 < _size2622; ++_i2626)
             {
-              xfer += (*(this->success))[_i2620].read(iprot);
+              xfer += (*(this->success))[_i2626].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
index 401d63df93f..2a7a8d65b5e 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -50167,6 +50167,11 @@ void GetPartitionsPsWithAuthRequest::__set_excludeParamKeyPattern(const std::str
   this->excludeParamKeyPattern = val;
 __isset.excludeParamKeyPattern = true;
 }
+
+void GetPartitionsPsWithAuthRequest::__set_partNames(const std::vector<std::string> & val) {
+  this->partNames = val;
+__isset.partNames = true;
+}
 std::ostream& operator<<(std::ostream& out, const GetPartitionsPsWithAuthRequest& obj)
 {
   obj.printTo(out);
@@ -50317,6 +50322,26 @@ uint32_t GetPartitionsPsWithAuthRequest::read(::apache::thrift::protocol::TProto
           xfer += iprot->skip(ftype);
         }
         break;
+      case 13:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->partNames.clear();
+            uint32_t _size1771;
+            ::apache::thrift::protocol::TType _etype1774;
+            xfer += iprot->readListBegin(_etype1774, _size1771);
+            this->partNames.resize(_size1771);
+            uint32_t _i1775;
+            for (_i1775 = 0; _i1775 < _size1771; ++_i1775)
+            {
+              xfer += iprot->readString(this->partNames[_i1775]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.partNames = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -50355,10 +50380,10 @@ uint32_t GetPartitionsPsWithAuthRequest::write(::apache::thrift::protocol::TProt
     xfer += oprot->writeFieldBegin("partVals", ::apache::thrift::protocol::T_LIST, 4);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partVals.size()));
-      std::vector<std::string> ::const_iterator _iter1771;
-      for (_iter1771 = this->partVals.begin(); _iter1771 != this->partVals.end(); ++_iter1771)
+      std::vector<std::string> ::const_iterator _iter1776;
+      for (_iter1776 = this->partVals.begin(); _iter1776 != this->partVals.end(); ++_iter1776)
       {
-        xfer += oprot->writeString((*_iter1771));
+        xfer += oprot->writeString((*_iter1776));
       }
       xfer += oprot->writeListEnd();
     }
@@ -50378,10 +50403,10 @@ uint32_t GetPartitionsPsWithAuthRequest::write(::apache::thrift::protocol::TProt
     xfer += oprot->writeFieldBegin("groupNames", ::apache::thrift::protocol::T_LIST, 7);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->groupNames.size()));
-      std::vector<std::string> ::const_iterator _iter1772;
-      for (_iter1772 = this->groupNames.begin(); _iter1772 != this->groupNames.end(); ++_iter1772)
+      std::vector<std::string> ::const_iterator _iter1777;
+      for (_iter1777 = this->groupNames.begin(); _iter1777 != this->groupNames.end(); ++_iter1777)
       {
-        xfer += oprot->writeString((*_iter1772));
+        xfer += oprot->writeString((*_iter1777));
       }
       xfer += oprot->writeListEnd();
     }
@@ -50412,6 +50437,19 @@ uint32_t GetPartitionsPsWithAuthRequest::write(::apache::thrift::protocol::TProt
     xfer += oprot->writeString(this->excludeParamKeyPattern);
     xfer += oprot->writeFieldEnd();
   }
+  if (this->__isset.partNames) {
+    xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 13);
+    {
+      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partNames.size()));
+      std::vector<std::string> ::const_iterator _iter1778;
+      for (_iter1778 = this->partNames.begin(); _iter1778 != this->partNames.end(); ++_iter1778)
+      {
+        xfer += oprot->writeString((*_iter1778));
+      }
+      xfer += oprot->writeListEnd();
+    }
+    xfer += oprot->writeFieldEnd();
+  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
@@ -50431,38 +50469,41 @@ void swap(GetPartitionsPsWithAuthRequest &a, GetPartitionsPsWithAuthRequest &b)
   swap(a.skipColumnSchemaForPartition, b.skipColumnSchemaForPartition);
   swap(a.includeParamKeyPattern, b.includeParamKeyPattern);
   swap(a.excludeParamKeyPattern, b.excludeParamKeyPattern);
+  swap(a.partNames, b.partNames);
   swap(a.__isset, b.__isset);
 }
 
-GetPartitionsPsWithAuthRequest::GetPartitionsPsWithAuthRequest(const GetPartitionsPsWithAuthRequest& other1773) {
-  catName = other1773.catName;
-  dbName = other1773.dbName;
-  tblName = other1773.tblName;
-  partVals = other1773.partVals;
-  maxParts = other1773.maxParts;
-  userName = other1773.userName;
-  groupNames = other1773.groupNames;
-  validWriteIdList = other1773.validWriteIdList;
-  id = other1773.id;
-  skipColumnSchemaForPartition = other1773.skipColumnSchemaForPartition;
-  includeParamKeyPattern = other1773.includeParamKeyPattern;
-  excludeParamKeyPattern = other1773.excludeParamKeyPattern;
-  __isset = other1773.__isset;
-}
-GetPartitionsPsWithAuthRequest& GetPartitionsPsWithAuthRequest::operator=(const GetPartitionsPsWithAuthRequest& other1774) {
-  catName = other1774.catName;
-  dbName = other1774.dbName;
-  tblName = other1774.tblName;
-  partVals = other1774.partVals;
-  maxParts = other1774.maxParts;
-  userName = other1774.userName;
-  groupNames = other1774.groupNames;
-  validWriteIdList = other1774.validWriteIdList;
-  id = other1774.id;
-  skipColumnSchemaForPartition = other1774.skipColumnSchemaForPartition;
-  includeParamKeyPattern = other1774.includeParamKeyPattern;
-  excludeParamKeyPattern = other1774.excludeParamKeyPattern;
-  __isset = other1774.__isset;
+GetPartitionsPsWithAuthRequest::GetPartitionsPsWithAuthRequest(const GetPartitionsPsWithAuthRequest& other1779) {
+  catName = other1779.catName;
+  dbName = other1779.dbName;
+  tblName = other1779.tblName;
+  partVals = other1779.partVals;
+  maxParts = other1779.maxParts;
+  userName = other1779.userName;
+  groupNames = other1779.groupNames;
+  validWriteIdList = other1779.validWriteIdList;
+  id = other1779.id;
+  skipColumnSchemaForPartition = other1779.skipColumnSchemaForPartition;
+  includeParamKeyPattern = other1779.includeParamKeyPattern;
+  excludeParamKeyPattern = other1779.excludeParamKeyPattern;
+  partNames = other1779.partNames;
+  __isset = other1779.__isset;
+}
+GetPartitionsPsWithAuthRequest& GetPartitionsPsWithAuthRequest::operator=(const GetPartitionsPsWithAuthRequest& other1780) {
+  catName = other1780.catName;
+  dbName = other1780.dbName;
+  tblName = other1780.tblName;
+  partVals = other1780.partVals;
+  maxParts = other1780.maxParts;
+  userName = other1780.userName;
+  groupNames = other1780.groupNames;
+  validWriteIdList = other1780.validWriteIdList;
+  id = other1780.id;
+  skipColumnSchemaForPartition = other1780.skipColumnSchemaForPartition;
+  includeParamKeyPattern = other1780.includeParamKeyPattern;
+  excludeParamKeyPattern = other1780.excludeParamKeyPattern;
+  partNames = other1780.partNames;
+  __isset = other1780.__isset;
   return *this;
 }
 void GetPartitionsPsWithAuthRequest::printTo(std::ostream& out) const {
@@ -50480,6 +50521,7 @@ void GetPartitionsPsWithAuthRequest::printTo(std::ostream& out) const {
   out << ", " << "skipColumnSchemaForPartition="; (__isset.skipColumnSchemaForPartition ? (out << to_string(skipColumnSchemaForPartition)) : (out << "<null>"));
   out << ", " << "includeParamKeyPattern="; (__isset.includeParamKeyPattern ? (out << to_string(includeParamKeyPattern)) : (out << "<null>"));
   out << ", " << "excludeParamKeyPattern="; (__isset.excludeParamKeyPattern ? (out << to_string(excludeParamKeyPattern)) : (out << "<null>"));
+  out << ", " << "partNames="; (__isset.partNames ? (out << to_string(partNames)) : (out << "<null>"));
   out << ")";
 }
 
@@ -50524,14 +50566,14 @@ uint32_t GetPartitionsPsWithAuthResponse::read(::apache::thrift::protocol::TProt
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitions.clear();
-            uint32_t _size1775;
-            ::apache::thrift::protocol::TType _etype1778;
-            xfer += iprot->readListBegin(_etype1778, _size1775);
-            this->partitions.resize(_size1775);
-            uint32_t _i1779;
-            for (_i1779 = 0; _i1779 < _size1775; ++_i1779)
+            uint32_t _size1781;
+            ::apache::thrift::protocol::TType _etype1784;
+            xfer += iprot->readListBegin(_etype1784, _size1781);
+            this->partitions.resize(_size1781);
+            uint32_t _i1785;
+            for (_i1785 = 0; _i1785 < _size1781; ++_i1785)
             {
-              xfer += this->partitions[_i1779].read(iprot);
+              xfer += this->partitions[_i1785].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -50562,10 +50604,10 @@ uint32_t GetPartitionsPsWithAuthResponse::write(::apache::thrift::protocol::TPro
   xfer += oprot->writeFieldBegin("partitions", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->partitions.size()));
-    std::vector<Partition> ::const_iterator _iter1780;
-    for (_iter1780 = this->partitions.begin(); _iter1780 != this->partitions.end(); ++_iter1780)
+    std::vector<Partition> ::const_iterator _iter1786;
+    for (_iter1786 = this->partitions.begin(); _iter1786 != this->partitions.end(); ++_iter1786)
     {
-      xfer += (*_iter1780).write(oprot);
+      xfer += (*_iter1786).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -50581,11 +50623,11 @@ void swap(GetPartitionsPsWithAuthResponse &a, GetPartitionsPsWithAuthResponse &b
   swap(a.partitions, b.partitions);
 }
 
-GetPartitionsPsWithAuthResponse::GetPartitionsPsWithAuthResponse(const GetPartitionsPsWithAuthResponse& other1781) {
-  partitions = other1781.partitions;
+GetPartitionsPsWithAuthResponse::GetPartitionsPsWithAuthResponse(const GetPartitionsPsWithAuthResponse& other1787) {
+  partitions = other1787.partitions;
 }
-GetPartitionsPsWithAuthResponse& GetPartitionsPsWithAuthResponse::operator=(const GetPartitionsPsWithAuthResponse& other1782) {
-  partitions = other1782.partitions;
+GetPartitionsPsWithAuthResponse& GetPartitionsPsWithAuthResponse::operator=(const GetPartitionsPsWithAuthResponse& other1788) {
+  partitions = other1788.partitions;
   return *this;
 }
 void GetPartitionsPsWithAuthResponse::printTo(std::ostream& out) const {
@@ -50771,23 +50813,23 @@ void swap(ReplicationMetrics &a, ReplicationMetrics &b) {
   swap(a.__isset, b.__isset);
 }
 
-ReplicationMetrics::ReplicationMetrics(const ReplicationMetrics& other1783) {
-  scheduledExecutionId = other1783.scheduledExecutionId;
-  policy = other1783.policy;
-  dumpExecutionId = other1783.dumpExecutionId;
-  metadata = other1783.metadata;
-  progress = other1783.progress;
-  messageFormat = other1783.messageFormat;
-  __isset = other1783.__isset;
+ReplicationMetrics::ReplicationMetrics(const ReplicationMetrics& other1789) {
+  scheduledExecutionId = other1789.scheduledExecutionId;
+  policy = other1789.policy;
+  dumpExecutionId = other1789.dumpExecutionId;
+  metadata = other1789.metadata;
+  progress = other1789.progress;
+  messageFormat = other1789.messageFormat;
+  __isset = other1789.__isset;
 }
-ReplicationMetrics& ReplicationMetrics::operator=(const ReplicationMetrics& other1784) {
-  scheduledExecutionId = other1784.scheduledExecutionId;
-  policy = other1784.policy;
-  dumpExecutionId = other1784.dumpExecutionId;
-  metadata = other1784.metadata;
-  progress = other1784.progress;
-  messageFormat = other1784.messageFormat;
-  __isset = other1784.__isset;
+ReplicationMetrics& ReplicationMetrics::operator=(const ReplicationMetrics& other1790) {
+  scheduledExecutionId = other1790.scheduledExecutionId;
+  policy = other1790.policy;
+  dumpExecutionId = other1790.dumpExecutionId;
+  metadata = other1790.metadata;
+  progress = other1790.progress;
+  messageFormat = other1790.messageFormat;
+  __isset = other1790.__isset;
   return *this;
 }
 void ReplicationMetrics::printTo(std::ostream& out) const {
@@ -50843,14 +50885,14 @@ uint32_t ReplicationMetricList::read(::apache::thrift::protocol::TProtocol* ipro
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->replicationMetricList.clear();
-            uint32_t _size1785;
-            ::apache::thrift::protocol::TType _etype1788;
-            xfer += iprot->readListBegin(_etype1788, _size1785);
-            this->replicationMetricList.resize(_size1785);
-            uint32_t _i1789;
-            for (_i1789 = 0; _i1789 < _size1785; ++_i1789)
+            uint32_t _size1791;
+            ::apache::thrift::protocol::TType _etype1794;
+            xfer += iprot->readListBegin(_etype1794, _size1791);
+            this->replicationMetricList.resize(_size1791);
+            uint32_t _i1795;
+            for (_i1795 = 0; _i1795 < _size1791; ++_i1795)
             {
-              xfer += this->replicationMetricList[_i1789].read(iprot);
+              xfer += this->replicationMetricList[_i1795].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -50881,10 +50923,10 @@ uint32_t ReplicationMetricList::write(::apache::thrift::protocol::TProtocol* opr
   xfer += oprot->writeFieldBegin("replicationMetricList", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->replicationMetricList.size()));
-    std::vector<ReplicationMetrics> ::const_iterator _iter1790;
-    for (_iter1790 = this->replicationMetricList.begin(); _iter1790 != this->replicationMetricList.end(); ++_iter1790)
+    std::vector<ReplicationMetrics> ::const_iterator _iter1796;
+    for (_iter1796 = this->replicationMetricList.begin(); _iter1796 != this->replicationMetricList.end(); ++_iter1796)
     {
-      xfer += (*_iter1790).write(oprot);
+      xfer += (*_iter1796).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -50900,11 +50942,11 @@ void swap(ReplicationMetricList &a, ReplicationMetricList &b) {
   swap(a.replicationMetricList, b.replicationMetricList);
 }
 
-ReplicationMetricList::ReplicationMetricList(const ReplicationMetricList& other1791) {
-  replicationMetricList = other1791.replicationMetricList;
+ReplicationMetricList::ReplicationMetricList(const ReplicationMetricList& other1797) {
+  replicationMetricList = other1797.replicationMetricList;
 }
-ReplicationMetricList& ReplicationMetricList::operator=(const ReplicationMetricList& other1792) {
-  replicationMetricList = other1792.replicationMetricList;
+ReplicationMetricList& ReplicationMetricList::operator=(const ReplicationMetricList& other1798) {
+  replicationMetricList = other1798.replicationMetricList;
   return *this;
 }
 void ReplicationMetricList::printTo(std::ostream& out) const {
@@ -51030,17 +51072,17 @@ void swap(GetReplicationMetricsRequest &a, GetReplicationMetricsRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-GetReplicationMetricsRequest::GetReplicationMetricsRequest(const GetReplicationMetricsRequest& other1793) {
-  scheduledExecutionId = other1793.scheduledExecutionId;
-  policy = other1793.policy;
-  dumpExecutionId = other1793.dumpExecutionId;
-  __isset = other1793.__isset;
+GetReplicationMetricsRequest::GetReplicationMetricsRequest(const GetReplicationMetricsRequest& other1799) {
+  scheduledExecutionId = other1799.scheduledExecutionId;
+  policy = other1799.policy;
+  dumpExecutionId = other1799.dumpExecutionId;
+  __isset = other1799.__isset;
 }
-GetReplicationMetricsRequest& GetReplicationMetricsRequest::operator=(const GetReplicationMetricsRequest& other1794) {
-  scheduledExecutionId = other1794.scheduledExecutionId;
-  policy = other1794.policy;
-  dumpExecutionId = other1794.dumpExecutionId;
-  __isset = other1794.__isset;
+GetReplicationMetricsRequest& GetReplicationMetricsRequest::operator=(const GetReplicationMetricsRequest& other1800) {
+  scheduledExecutionId = other1800.scheduledExecutionId;
+  policy = other1800.policy;
+  dumpExecutionId = other1800.dumpExecutionId;
+  __isset = other1800.__isset;
   return *this;
 }
 void GetReplicationMetricsRequest::printTo(std::ostream& out) const {
@@ -51093,16 +51135,16 @@ uint32_t GetOpenTxnsRequest::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->excludeTxnTypes.clear();
-            uint32_t _size1795;
-            ::apache::thrift::protocol::TType _etype1798;
-            xfer += iprot->readListBegin(_etype1798, _size1795);
-            this->excludeTxnTypes.resize(_size1795);
-            uint32_t _i1799;
-            for (_i1799 = 0; _i1799 < _size1795; ++_i1799)
+            uint32_t _size1801;
+            ::apache::thrift::protocol::TType _etype1804;
+            xfer += iprot->readListBegin(_etype1804, _size1801);
+            this->excludeTxnTypes.resize(_size1801);
+            uint32_t _i1805;
+            for (_i1805 = 0; _i1805 < _size1801; ++_i1805)
             {
-              int32_t ecast1800;
-              xfer += iprot->readI32(ecast1800);
-              this->excludeTxnTypes[_i1799] = static_cast<TxnType::type>(ecast1800);
+              int32_t ecast1806;
+              xfer += iprot->readI32(ecast1806);
+              this->excludeTxnTypes[_i1805] = static_cast<TxnType::type>(ecast1806);
             }
             xfer += iprot->readListEnd();
           }
@@ -51132,10 +51174,10 @@ uint32_t GetOpenTxnsRequest::write(::apache::thrift::protocol::TProtocol* oprot)
     xfer += oprot->writeFieldBegin("excludeTxnTypes", ::apache::thrift::protocol::T_LIST, 1);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I32, static_cast<uint32_t>(this->excludeTxnTypes.size()));
-      std::vector<TxnType::type> ::const_iterator _iter1801;
-      for (_iter1801 = this->excludeTxnTypes.begin(); _iter1801 != this->excludeTxnTypes.end(); ++_iter1801)
+      std::vector<TxnType::type> ::const_iterator _iter1807;
+      for (_iter1807 = this->excludeTxnTypes.begin(); _iter1807 != this->excludeTxnTypes.end(); ++_iter1807)
       {
-        xfer += oprot->writeI32(static_cast<int32_t>((*_iter1801)));
+        xfer += oprot->writeI32(static_cast<int32_t>((*_iter1807)));
       }
       xfer += oprot->writeListEnd();
     }
@@ -51152,13 +51194,13 @@ void swap(GetOpenTxnsRequest &a, GetOpenTxnsRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-GetOpenTxnsRequest::GetOpenTxnsRequest(const GetOpenTxnsRequest& other1802) {
-  excludeTxnTypes = other1802.excludeTxnTypes;
-  __isset = other1802.__isset;
+GetOpenTxnsRequest::GetOpenTxnsRequest(const GetOpenTxnsRequest& other1808) {
+  excludeTxnTypes = other1808.excludeTxnTypes;
+  __isset = other1808.__isset;
 }
-GetOpenTxnsRequest& GetOpenTxnsRequest::operator=(const GetOpenTxnsRequest& other1803) {
-  excludeTxnTypes = other1803.excludeTxnTypes;
-  __isset = other1803.__isset;
+GetOpenTxnsRequest& GetOpenTxnsRequest::operator=(const GetOpenTxnsRequest& other1809) {
+  excludeTxnTypes = other1809.excludeTxnTypes;
+  __isset = other1809.__isset;
   return *this;
 }
 void GetOpenTxnsRequest::printTo(std::ostream& out) const {
@@ -51286,15 +51328,15 @@ void swap(StoredProcedureRequest &a, StoredProcedureRequest &b) {
   swap(a.procName, b.procName);
 }
 
-StoredProcedureRequest::StoredProcedureRequest(const StoredProcedureRequest& other1804) {
-  catName = other1804.catName;
-  dbName = other1804.dbName;
-  procName = other1804.procName;
+StoredProcedureRequest::StoredProcedureRequest(const StoredProcedureRequest& other1810) {
+  catName = other1810.catName;
+  dbName = other1810.dbName;
+  procName = other1810.procName;
 }
-StoredProcedureRequest& StoredProcedureRequest::operator=(const StoredProcedureRequest& other1805) {
-  catName = other1805.catName;
-  dbName = other1805.dbName;
-  procName = other1805.procName;
+StoredProcedureRequest& StoredProcedureRequest::operator=(const StoredProcedureRequest& other1811) {
+  catName = other1811.catName;
+  dbName = other1811.dbName;
+  procName = other1811.procName;
   return *this;
 }
 void StoredProcedureRequest::printTo(std::ostream& out) const {
@@ -51404,15 +51446,15 @@ void swap(ListStoredProcedureRequest &a, ListStoredProcedureRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-ListStoredProcedureRequest::ListStoredProcedureRequest(const ListStoredProcedureRequest& other1806) {
-  catName = other1806.catName;
-  dbName = other1806.dbName;
-  __isset = other1806.__isset;
+ListStoredProcedureRequest::ListStoredProcedureRequest(const ListStoredProcedureRequest& other1812) {
+  catName = other1812.catName;
+  dbName = other1812.dbName;
+  __isset = other1812.__isset;
 }
-ListStoredProcedureRequest& ListStoredProcedureRequest::operator=(const ListStoredProcedureRequest& other1807) {
-  catName = other1807.catName;
-  dbName = other1807.dbName;
-  __isset = other1807.__isset;
+ListStoredProcedureRequest& ListStoredProcedureRequest::operator=(const ListStoredProcedureRequest& other1813) {
+  catName = other1813.catName;
+  dbName = other1813.dbName;
+  __isset = other1813.__isset;
   return *this;
 }
 void ListStoredProcedureRequest::printTo(std::ostream& out) const {
@@ -51567,21 +51609,21 @@ void swap(StoredProcedure &a, StoredProcedure &b) {
   swap(a.__isset, b.__isset);
 }
 
-StoredProcedure::StoredProcedure(const StoredProcedure& other1808) {
-  name = other1808.name;
-  dbName = other1808.dbName;
-  catName = other1808.catName;
-  ownerName = other1808.ownerName;
-  source = other1808.source;
-  __isset = other1808.__isset;
+StoredProcedure::StoredProcedure(const StoredProcedure& other1814) {
+  name = other1814.name;
+  dbName = other1814.dbName;
+  catName = other1814.catName;
+  ownerName = other1814.ownerName;
+  source = other1814.source;
+  __isset = other1814.__isset;
 }
-StoredProcedure& StoredProcedure::operator=(const StoredProcedure& other1809) {
-  name = other1809.name;
-  dbName = other1809.dbName;
-  catName = other1809.catName;
-  ownerName = other1809.ownerName;
-  source = other1809.source;
-  __isset = other1809.__isset;
+StoredProcedure& StoredProcedure::operator=(const StoredProcedure& other1815) {
+  name = other1815.name;
+  dbName = other1815.dbName;
+  catName = other1815.catName;
+  ownerName = other1815.ownerName;
+  source = other1815.source;
+  __isset = other1815.__isset;
   return *this;
 }
 void StoredProcedure::printTo(std::ostream& out) const {
@@ -51756,23 +51798,23 @@ void swap(AddPackageRequest &a, AddPackageRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-AddPackageRequest::AddPackageRequest(const AddPackageRequest& other1810) {
-  catName = other1810.catName;
-  dbName = other1810.dbName;
-  packageName = other1810.packageName;
-  ownerName = other1810.ownerName;
-  header = other1810.header;
-  body = other1810.body;
-  __isset = other1810.__isset;
+AddPackageRequest::AddPackageRequest(const AddPackageRequest& other1816) {
+  catName = other1816.catName;
+  dbName = other1816.dbName;
+  packageName = other1816.packageName;
+  ownerName = other1816.ownerName;
+  header = other1816.header;
+  body = other1816.body;
+  __isset = other1816.__isset;
 }
-AddPackageRequest& AddPackageRequest::operator=(const AddPackageRequest& other1811) {
-  catName = other1811.catName;
-  dbName = other1811.dbName;
-  packageName = other1811.packageName;
-  ownerName = other1811.ownerName;
-  header = other1811.header;
-  body = other1811.body;
-  __isset = other1811.__isset;
+AddPackageRequest& AddPackageRequest::operator=(const AddPackageRequest& other1817) {
+  catName = other1817.catName;
+  dbName = other1817.dbName;
+  packageName = other1817.packageName;
+  ownerName = other1817.ownerName;
+  header = other1817.header;
+  body = other1817.body;
+  __isset = other1817.__isset;
   return *this;
 }
 void AddPackageRequest::printTo(std::ostream& out) const {
@@ -51905,15 +51947,15 @@ void swap(GetPackageRequest &a, GetPackageRequest &b) {
   swap(a.packageName, b.packageName);
 }
 
-GetPackageRequest::GetPackageRequest(const GetPackageRequest& other1812) {
-  catName = other1812.catName;
-  dbName = other1812.dbName;
-  packageName = other1812.packageName;
+GetPackageRequest::GetPackageRequest(const GetPackageRequest& other1818) {
+  catName = other1818.catName;
+  dbName = other1818.dbName;
+  packageName = other1818.packageName;
 }
-GetPackageRequest& GetPackageRequest::operator=(const GetPackageRequest& other1813) {
-  catName = other1813.catName;
-  dbName = other1813.dbName;
-  packageName = other1813.packageName;
+GetPackageRequest& GetPackageRequest::operator=(const GetPackageRequest& other1819) {
+  catName = other1819.catName;
+  dbName = other1819.dbName;
+  packageName = other1819.packageName;
   return *this;
 }
 void GetPackageRequest::printTo(std::ostream& out) const {
@@ -52043,15 +52085,15 @@ void swap(DropPackageRequest &a, DropPackageRequest &b) {
   swap(a.packageName, b.packageName);
 }
 
-DropPackageRequest::DropPackageRequest(const DropPackageRequest& other1814) {
-  catName = other1814.catName;
-  dbName = other1814.dbName;
-  packageName = other1814.packageName;
+DropPackageRequest::DropPackageRequest(const DropPackageRequest& other1820) {
+  catName = other1820.catName;
+  dbName = other1820.dbName;
+  packageName = other1820.packageName;
 }
-DropPackageRequest& DropPackageRequest::operator=(const DropPackageRequest& other1815) {
-  catName = other1815.catName;
-  dbName = other1815.dbName;
-  packageName = other1815.packageName;
+DropPackageRequest& DropPackageRequest::operator=(const DropPackageRequest& other1821) {
+  catName = other1821.catName;
+  dbName = other1821.dbName;
+  packageName = other1821.packageName;
   return *this;
 }
 void DropPackageRequest::printTo(std::ostream& out) const {
@@ -52161,15 +52203,15 @@ void swap(ListPackageRequest &a, ListPackageRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-ListPackageRequest::ListPackageRequest(const ListPackageRequest& other1816) {
-  catName = other1816.catName;
-  dbName = other1816.dbName;
-  __isset = other1816.__isset;
+ListPackageRequest::ListPackageRequest(const ListPackageRequest& other1822) {
+  catName = other1822.catName;
+  dbName = other1822.dbName;
+  __isset = other1822.__isset;
 }
-ListPackageRequest& ListPackageRequest::operator=(const ListPackageRequest& other1817) {
-  catName = other1817.catName;
-  dbName = other1817.dbName;
-  __isset = other1817.__isset;
+ListPackageRequest& ListPackageRequest::operator=(const ListPackageRequest& other1823) {
+  catName = other1823.catName;
+  dbName = other1823.dbName;
+  __isset = other1823.__isset;
   return *this;
 }
 void ListPackageRequest::printTo(std::ostream& out) const {
@@ -52341,23 +52383,23 @@ void swap(Package &a, Package &b) {
   swap(a.__isset, b.__isset);
 }
 
-Package::Package(const Package& other1818) {
-  catName = other1818.catName;
-  dbName = other1818.dbName;
-  packageName = other1818.packageName;
-  ownerName = other1818.ownerName;
-  header = other1818.header;
-  body = other1818.body;
-  __isset = other1818.__isset;
+Package::Package(const Package& other1824) {
+  catName = other1824.catName;
+  dbName = other1824.dbName;
+  packageName = other1824.packageName;
+  ownerName = other1824.ownerName;
+  header = other1824.header;
+  body = other1824.body;
+  __isset = other1824.__isset;
 }
-Package& Package::operator=(const Package& other1819) {
-  catName = other1819.catName;
-  dbName = other1819.dbName;
-  packageName = other1819.packageName;
-  ownerName = other1819.ownerName;
-  header = other1819.header;
-  body = other1819.body;
-  __isset = other1819.__isset;
+Package& Package::operator=(const Package& other1825) {
+  catName = other1825.catName;
+  dbName = other1825.dbName;
+  packageName = other1825.packageName;
+  ownerName = other1825.ownerName;
+  header = other1825.header;
+  body = other1825.body;
+  __isset = other1825.__isset;
   return *this;
 }
 void Package::printTo(std::ostream& out) const {
@@ -52489,17 +52531,17 @@ void swap(GetAllWriteEventInfoRequest &a, GetAllWriteEventInfoRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-GetAllWriteEventInfoRequest::GetAllWriteEventInfoRequest(const GetAllWriteEventInfoRequest& other1820) {
-  txnId = other1820.txnId;
-  dbName = other1820.dbName;
-  tableName = other1820.tableName;
-  __isset = other1820.__isset;
+GetAllWriteEventInfoRequest::GetAllWriteEventInfoRequest(const GetAllWriteEventInfoRequest& other1826) {
+  txnId = other1826.txnId;
+  dbName = other1826.dbName;
+  tableName = other1826.tableName;
+  __isset = other1826.__isset;
 }
-GetAllWriteEventInfoRequest& GetAllWriteEventInfoRequest::operator=(const GetAllWriteEventInfoRequest& other1821) {
-  txnId = other1821.txnId;
-  dbName = other1821.dbName;
-  tableName = other1821.tableName;
-  __isset = other1821.__isset;
+GetAllWriteEventInfoRequest& GetAllWriteEventInfoRequest::operator=(const GetAllWriteEventInfoRequest& other1827) {
+  txnId = other1827.txnId;
+  dbName = other1827.dbName;
+  tableName = other1827.tableName;
+  __isset = other1827.__isset;
   return *this;
 }
 void GetAllWriteEventInfoRequest::printTo(std::ostream& out) const {
@@ -52587,13 +52629,13 @@ void swap(MetaException &a, MetaException &b) {
   swap(a.__isset, b.__isset);
 }
 
-MetaException::MetaException(const MetaException& other1822) : TException() {
-  message = other1822.message;
-  __isset = other1822.__isset;
+MetaException::MetaException(const MetaException& other1828) : TException() {
+  message = other1828.message;
+  __isset = other1828.__isset;
 }
-MetaException& MetaException::operator=(const MetaException& other1823) {
-  message = other1823.message;
-  __isset = other1823.__isset;
+MetaException& MetaException::operator=(const MetaException& other1829) {
+  message = other1829.message;
+  __isset = other1829.__isset;
   return *this;
 }
 void MetaException::printTo(std::ostream& out) const {
@@ -52690,13 +52732,13 @@ void swap(UnknownTableException &a, UnknownTableException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownTableException::UnknownTableException(const UnknownTableException& other1824) : TException() {
-  message = other1824.message;
-  __isset = other1824.__isset;
+UnknownTableException::UnknownTableException(const UnknownTableException& other1830) : TException() {
+  message = other1830.message;
+  __isset = other1830.__isset;
 }
-UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other1825) {
-  message = other1825.message;
-  __isset = other1825.__isset;
+UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other1831) {
+  message = other1831.message;
+  __isset = other1831.__isset;
   return *this;
 }
 void UnknownTableException::printTo(std::ostream& out) const {
@@ -52793,13 +52835,13 @@ void swap(UnknownDBException &a, UnknownDBException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownDBException::UnknownDBException(const UnknownDBException& other1826) : TException() {
-  message = other1826.message;
-  __isset = other1826.__isset;
+UnknownDBException::UnknownDBException(const UnknownDBException& other1832) : TException() {
+  message = other1832.message;
+  __isset = other1832.__isset;
 }
-UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other1827) {
-  message = other1827.message;
-  __isset = other1827.__isset;
+UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other1833) {
+  message = other1833.message;
+  __isset = other1833.__isset;
   return *this;
 }
 void UnknownDBException::printTo(std::ostream& out) const {
@@ -52896,13 +52938,13 @@ void swap(AlreadyExistsException &a, AlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other1828) : TException() {
-  message = other1828.message;
-  __isset = other1828.__isset;
+AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other1834) : TException() {
+  message = other1834.message;
+  __isset = other1834.__isset;
 }
-AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other1829) {
-  message = other1829.message;
-  __isset = other1829.__isset;
+AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other1835) {
+  message = other1835.message;
+  __isset = other1835.__isset;
   return *this;
 }
 void AlreadyExistsException::printTo(std::ostream& out) const {
@@ -52999,13 +53041,13 @@ void swap(InvalidPartitionException &a, InvalidPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other1830) : TException() {
-  message = other1830.message;
-  __isset = other1830.__isset;
+InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other1836) : TException() {
+  message = other1836.message;
+  __isset = other1836.__isset;
 }
-InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other1831) {
-  message = other1831.message;
-  __isset = other1831.__isset;
+InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other1837) {
+  message = other1837.message;
+  __isset = other1837.__isset;
   return *this;
 }
 void InvalidPartitionException::printTo(std::ostream& out) const {
@@ -53102,13 +53144,13 @@ void swap(UnknownPartitionException &a, UnknownPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other1832) : TException() {
-  message = other1832.message;
-  __isset = other1832.__isset;
+UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other1838) : TException() {
+  message = other1838.message;
+  __isset = other1838.__isset;
 }
-UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other1833) {
-  message = other1833.message;
-  __isset = other1833.__isset;
+UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other1839) {
+  message = other1839.message;
+  __isset = other1839.__isset;
   return *this;
 }
 void UnknownPartitionException::printTo(std::ostream& out) const {
@@ -53205,13 +53247,13 @@ void swap(InvalidObjectException &a, InvalidObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidObjectException::InvalidObjectException(const InvalidObjectException& other1834) : TException() {
-  message = other1834.message;
-  __isset = other1834.__isset;
+InvalidObjectException::InvalidObjectException(const InvalidObjectException& other1840) : TException() {
+  message = other1840.message;
+  __isset = other1840.__isset;
 }
-InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other1835) {
-  message = other1835.message;
-  __isset = other1835.__isset;
+InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other1841) {
+  message = other1841.message;
+  __isset = other1841.__isset;
   return *this;
 }
 void InvalidObjectException::printTo(std::ostream& out) const {
@@ -53308,13 +53350,13 @@ void swap(NoSuchObjectException &a, NoSuchObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other1836) : TException() {
-  message = other1836.message;
-  __isset = other1836.__isset;
+NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other1842) : TException() {
+  message = other1842.message;
+  __isset = other1842.__isset;
 }
-NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other1837) {
-  message = other1837.message;
-  __isset = other1837.__isset;
+NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other1843) {
+  message = other1843.message;
+  __isset = other1843.__isset;
   return *this;
 }
 void NoSuchObjectException::printTo(std::ostream& out) const {
@@ -53411,13 +53453,13 @@ void swap(InvalidOperationException &a, InvalidOperationException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidOperationException::InvalidOperationException(const InvalidOperationException& other1838) : TException() {
-  message = other1838.message;
-  __isset = other1838.__isset;
+InvalidOperationException::InvalidOperationException(const InvalidOperationException& other1844) : TException() {
+  message = other1844.message;
+  __isset = other1844.__isset;
 }
-InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other1839) {
-  message = other1839.message;
-  __isset = other1839.__isset;
+InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other1845) {
+  message = other1845.message;
+  __isset = other1845.__isset;
   return *this;
 }
 void InvalidOperationException::printTo(std::ostream& out) const {
@@ -53514,13 +53556,13 @@ void swap(ConfigValSecurityException &a, ConfigValSecurityException &b) {
   swap(a.__isset, b.__isset);
 }
 
-ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other1840) : TException() {
-  message = other1840.message;
-  __isset = other1840.__isset;
+ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other1846) : TException() {
+  message = other1846.message;
+  __isset = other1846.__isset;
 }
-ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other1841) {
-  message = other1841.message;
-  __isset = other1841.__isset;
+ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other1847) {
+  message = other1847.message;
+  __isset = other1847.__isset;
   return *this;
 }
 void ConfigValSecurityException::printTo(std::ostream& out) const {
@@ -53617,13 +53659,13 @@ void swap(InvalidInputException &a, InvalidInputException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidInputException::InvalidInputException(const InvalidInputException& other1842) : TException() {
-  message = other1842.message;
-  __isset = other1842.__isset;
+InvalidInputException::InvalidInputException(const InvalidInputException& other1848) : TException() {
+  message = other1848.message;
+  __isset = other1848.__isset;
 }
-InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other1843) {
-  message = other1843.message;
-  __isset = other1843.__isset;
+InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other1849) {
+  message = other1849.message;
+  __isset = other1849.__isset;
   return *this;
 }
 void InvalidInputException::printTo(std::ostream& out) const {
@@ -53720,13 +53762,13 @@ void swap(NoSuchTxnException &a, NoSuchTxnException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other1844) : TException() {
-  message = other1844.message;
-  __isset = other1844.__isset;
+NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other1850) : TException() {
+  message = other1850.message;
+  __isset = other1850.__isset;
 }
-NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other1845) {
-  message = other1845.message;
-  __isset = other1845.__isset;
+NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other1851) {
+  message = other1851.message;
+  __isset = other1851.__isset;
   return *this;
 }
 void NoSuchTxnException::printTo(std::ostream& out) const {
@@ -53823,13 +53865,13 @@ void swap(TxnAbortedException &a, TxnAbortedException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnAbortedException::TxnAbortedException(const TxnAbortedException& other1846) : TException() {
-  message = other1846.message;
-  __isset = other1846.__isset;
+TxnAbortedException::TxnAbortedException(const TxnAbortedException& other1852) : TException() {
+  message = other1852.message;
+  __isset = other1852.__isset;
 }
-TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other1847) {
-  message = other1847.message;
-  __isset = other1847.__isset;
+TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other1853) {
+  message = other1853.message;
+  __isset = other1853.__isset;
   return *this;
 }
 void TxnAbortedException::printTo(std::ostream& out) const {
@@ -53926,13 +53968,13 @@ void swap(TxnOpenException &a, TxnOpenException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnOpenException::TxnOpenException(const TxnOpenException& other1848) : TException() {
-  message = other1848.message;
-  __isset = other1848.__isset;
+TxnOpenException::TxnOpenException(const TxnOpenException& other1854) : TException() {
+  message = other1854.message;
+  __isset = other1854.__isset;
 }
-TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other1849) {
-  message = other1849.message;
-  __isset = other1849.__isset;
+TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other1855) {
+  message = other1855.message;
+  __isset = other1855.__isset;
   return *this;
 }
 void TxnOpenException::printTo(std::ostream& out) const {
@@ -54029,13 +54071,13 @@ void swap(NoSuchLockException &a, NoSuchLockException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchLockException::NoSuchLockException(const NoSuchLockException& other1850) : TException() {
-  message = other1850.message;
-  __isset = other1850.__isset;
+NoSuchLockException::NoSuchLockException(const NoSuchLockException& other1856) : TException() {
+  message = other1856.message;
+  __isset = other1856.__isset;
 }
-NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other1851) {
-  message = other1851.message;
-  __isset = other1851.__isset;
+NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other1857) {
+  message = other1857.message;
+  __isset = other1857.__isset;
   return *this;
 }
 void NoSuchLockException::printTo(std::ostream& out) const {
@@ -54132,13 +54174,13 @@ void swap(CompactionAbortedException &a, CompactionAbortedException &b) {
   swap(a.__isset, b.__isset);
 }
 
-CompactionAbortedException::CompactionAbortedException(const CompactionAbortedException& other1852) : TException() {
-  message = other1852.message;
-  __isset = other1852.__isset;
+CompactionAbortedException::CompactionAbortedException(const CompactionAbortedException& other1858) : TException() {
+  message = other1858.message;
+  __isset = other1858.__isset;
 }
-CompactionAbortedException& CompactionAbortedException::operator=(const CompactionAbortedException& other1853) {
-  message = other1853.message;
-  __isset = other1853.__isset;
+CompactionAbortedException& CompactionAbortedException::operator=(const CompactionAbortedException& other1859) {
+  message = other1859.message;
+  __isset = other1859.__isset;
   return *this;
 }
 void CompactionAbortedException::printTo(std::ostream& out) const {
@@ -54235,13 +54277,13 @@ void swap(NoSuchCompactionException &a, NoSuchCompactionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchCompactionException::NoSuchCompactionException(const NoSuchCompactionException& other1854) : TException() {
-  message = other1854.message;
-  __isset = other1854.__isset;
+NoSuchCompactionException::NoSuchCompactionException(const NoSuchCompactionException& other1860) : TException() {
+  message = other1860.message;
+  __isset = other1860.__isset;
 }
-NoSuchCompactionException& NoSuchCompactionException::operator=(const NoSuchCompactionException& other1855) {
-  message = other1855.message;
-  __isset = other1855.__isset;
+NoSuchCompactionException& NoSuchCompactionException::operator=(const NoSuchCompactionException& other1861) {
+  message = other1861.message;
+  __isset = other1861.__isset;
   return *this;
 }
 void NoSuchCompactionException::printTo(std::ostream& out) const {
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h
index 47d79d6ae08..4e0a567345b 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -19720,7 +19720,7 @@ void swap(GetPartitionNamesPsResponse &a, GetPartitionNamesPsResponse &b);
 std::ostream& operator<<(std::ostream& out, const GetPartitionNamesPsResponse& obj);
 
 typedef struct _GetPartitionsPsWithAuthRequest__isset {
-  _GetPartitionsPsWithAuthRequest__isset() : catName(false), partVals(false), maxParts(true), userName(false), groupNames(false), validWriteIdList(false), id(true), skipColumnSchemaForPartition(false), includeParamKeyPattern(false), excludeParamKeyPattern(false) {}
+  _GetPartitionsPsWithAuthRequest__isset() : catName(false), partVals(false), maxParts(true), userName(false), groupNames(false), validWriteIdList(false), id(true), skipColumnSchemaForPartition(false), includeParamKeyPattern(false), excludeParamKeyPattern(false), partNames(false) {}
   bool catName :1;
   bool partVals :1;
   bool maxParts :1;
@@ -19731,6 +19731,7 @@ typedef struct _GetPartitionsPsWithAuthRequest__isset {
   bool skipColumnSchemaForPartition :1;
   bool includeParamKeyPattern :1;
   bool excludeParamKeyPattern :1;
+  bool partNames :1;
 } _GetPartitionsPsWithAuthRequest__isset;
 
 class GetPartitionsPsWithAuthRequest : public virtual ::apache::thrift::TBase {
@@ -19764,6 +19765,7 @@ class GetPartitionsPsWithAuthRequest : public virtual ::apache::thrift::TBase {
   bool skipColumnSchemaForPartition;
   std::string includeParamKeyPattern;
   std::string excludeParamKeyPattern;
+  std::vector<std::string>  partNames;
 
   _GetPartitionsPsWithAuthRequest__isset __isset;
 
@@ -19791,6 +19793,8 @@ class GetPartitionsPsWithAuthRequest : public virtual ::apache::thrift::TBase {
 
   void __set_excludeParamKeyPattern(const std::string& val);
 
+  void __set_partNames(const std::vector<std::string> & val);
+
   bool operator == (const GetPartitionsPsWithAuthRequest & rhs) const
   {
     if (__isset.catName != rhs.__isset.catName)
@@ -19837,6 +19841,10 @@ class GetPartitionsPsWithAuthRequest : public virtual ::apache::thrift::TBase {
       return false;
     else if (__isset.excludeParamKeyPattern && !(excludeParamKeyPattern == rhs.excludeParamKeyPattern))
       return false;
+    if (__isset.partNames != rhs.__isset.partNames)
+      return false;
+    else if (__isset.partNames && !(partNames == rhs.partNames))
+      return false;
     return true;
   }
   bool operator != (const GetPartitionsPsWithAuthRequest &rhs) const {
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsRequest.java
index 9e3e03c5f30..9a77c9b3a30 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsRequest.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsRequest.java
@@ -321,15 +321,15 @@ package org.apache.hadoop.hive.metastore.api;
           case 1: // EXCLUDE_TXN_TYPES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list1520 = iprot.readListBegin();
-                struct.excludeTxnTypes = new java.util.ArrayList<TxnType>(_list1520.size);
-                @org.apache.thrift.annotation.Nullable TxnType _elem1521;
-                for (int _i1522 = 0; _i1522 < _list1520.size; ++_i1522)
+                org.apache.thrift.protocol.TList _list1528 = iprot.readListBegin();
+                struct.excludeTxnTypes = new java.util.ArrayList<TxnType>(_list1528.size);
+                @org.apache.thrift.annotation.Nullable TxnType _elem1529;
+                for (int _i1530 = 0; _i1530 < _list1528.size; ++_i1530)
                 {
-                  _elem1521 = org.apache.hadoop.hive.metastore.api.TxnType.findByValue(iprot.readI32());
-                  if (_elem1521 != null)
+                  _elem1529 = org.apache.hadoop.hive.metastore.api.TxnType.findByValue(iprot.readI32());
+                  if (_elem1529 != null)
                   {
-                    struct.excludeTxnTypes.add(_elem1521);
+                    struct.excludeTxnTypes.add(_elem1529);
                   }
                 }
                 iprot.readListEnd();
@@ -357,9 +357,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(EXCLUDE_TXN_TYPES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, struct.excludeTxnTypes.size()));
-            for (TxnType _iter1523 : struct.excludeTxnTypes)
+            for (TxnType _iter1531 : struct.excludeTxnTypes)
             {
-              oprot.writeI32(_iter1523.getValue());
+              oprot.writeI32(_iter1531.getValue());
             }
             oprot.writeListEnd();
           }
@@ -391,9 +391,9 @@ package org.apache.hadoop.hive.metastore.api;
       if (struct.isSetExcludeTxnTypes()) {
         {
           oprot.writeI32(struct.excludeTxnTypes.size());
-          for (TxnType _iter1524 : struct.excludeTxnTypes)
+          for (TxnType _iter1532 : struct.excludeTxnTypes)
           {
-            oprot.writeI32(_iter1524.getValue());
+            oprot.writeI32(_iter1532.getValue());
           }
         }
       }
@@ -405,15 +405,15 @@ package org.apache.hadoop.hive.metastore.api;
       java.util.BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list1525 = iprot.readListBegin(org.apache.thrift.protocol.TType.I32);
-          struct.excludeTxnTypes = new java.util.ArrayList<TxnType>(_list1525.size);
-          @org.apache.thrift.annotation.Nullable TxnType _elem1526;
-          for (int _i1527 = 0; _i1527 < _list1525.size; ++_i1527)
+          org.apache.thrift.protocol.TList _list1533 = iprot.readListBegin(org.apache.thrift.protocol.TType.I32);
+          struct.excludeTxnTypes = new java.util.ArrayList<TxnType>(_list1533.size);
+          @org.apache.thrift.annotation.Nullable TxnType _elem1534;
+          for (int _i1535 = 0; _i1535 < _list1533.size; ++_i1535)
           {
-            _elem1526 = org.apache.hadoop.hive.metastore.api.TxnType.findByValue(iprot.readI32());
-            if (_elem1526 != null)
+            _elem1534 = org.apache.hadoop.hive.metastore.api.TxnType.findByValue(iprot.readI32());
+            if (_elem1534 != null)
             {
-              struct.excludeTxnTypes.add(_elem1526);
+              struct.excludeTxnTypes.add(_elem1534);
             }
           }
         }
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsPsWithAuthRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsPsWithAuthRequest.java
index 8cd28971b2a..d490a6810f0 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsPsWithAuthRequest.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsPsWithAuthRequest.java
@@ -23,6 +23,7 @@ package org.apache.hadoop.hive.metastore.api;
   private static final org.apache.thrift.protocol.TField SKIP_COLUMN_SCHEMA_FOR_PARTITION_FIELD_DESC = new org.apache.thrift.protocol.TField("skipColumnSchemaForPartition", org.apache.thrift.protocol.TType.BOOL, (short)10);
   private static final org.apache.thrift.protocol.TField INCLUDE_PARAM_KEY_PATTERN_FIELD_DESC = new org.apache.thrift.protocol.TField("includeParamKeyPattern", org.apache.thrift.protocol.TType.STRING, (short)11);
   private static final org.apache.thrift.protocol.TField EXCLUDE_PARAM_KEY_PATTERN_FIELD_DESC = new org.apache.thrift.protocol.TField("excludeParamKeyPattern", org.apache.thrift.protocol.TType.STRING, (short)12);
+  private static final org.apache.thrift.protocol.TField PART_NAMES_FIELD_DESC = new org.apache.thrift.protocol.TField("partNames", org.apache.thrift.protocol.TType.LIST, (short)13);
 
   private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new GetPartitionsPsWithAuthRequestStandardSchemeFactory();
   private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new GetPartitionsPsWithAuthRequestTupleSchemeFactory();
@@ -39,6 +40,7 @@ package org.apache.hadoop.hive.metastore.api;
   private boolean skipColumnSchemaForPartition; // optional
   private @org.apache.thrift.annotation.Nullable java.lang.String includeParamKeyPattern; // optional
   private @org.apache.thrift.annotation.Nullable java.lang.String excludeParamKeyPattern; // optional
+  private @org.apache.thrift.annotation.Nullable java.util.List<java.lang.String> partNames; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -53,7 +55,8 @@ package org.apache.hadoop.hive.metastore.api;
     ID((short)9, "id"),
     SKIP_COLUMN_SCHEMA_FOR_PARTITION((short)10, "skipColumnSchemaForPartition"),
     INCLUDE_PARAM_KEY_PATTERN((short)11, "includeParamKeyPattern"),
-    EXCLUDE_PARAM_KEY_PATTERN((short)12, "excludeParamKeyPattern");
+    EXCLUDE_PARAM_KEY_PATTERN((short)12, "excludeParamKeyPattern"),
+    PART_NAMES((short)13, "partNames");
 
     private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
 
@@ -93,6 +96,8 @@ package org.apache.hadoop.hive.metastore.api;
           return INCLUDE_PARAM_KEY_PATTERN;
         case 12: // EXCLUDE_PARAM_KEY_PATTERN
           return EXCLUDE_PARAM_KEY_PATTERN;
+        case 13: // PART_NAMES
+          return PART_NAMES;
         default:
           return null;
       }
@@ -138,7 +143,7 @@ package org.apache.hadoop.hive.metastore.api;
   private static final int __ID_ISSET_ID = 1;
   private static final int __SKIPCOLUMNSCHEMAFORPARTITION_ISSET_ID = 2;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.CAT_NAME,_Fields.PART_VALS,_Fields.MAX_PARTS,_Fields.USER_NAME,_Fields.GROUP_NAMES,_Fields.VALID_WRITE_ID_LIST,_Fields.ID,_Fields.SKIP_COLUMN_SCHEMA_FOR_PARTITION,_Fields.INCLUDE_PARAM_KEY_PATTERN,_Fields.EXCLUDE_PARAM_KEY_PATTERN};
+  private static final _Fields optionals[] = {_Fields.CAT_NAME,_Fields.PART_VALS,_Fields.MAX_PARTS,_Fields.USER_NAME,_Fields.GROUP_NAMES,_Fields.VALID_WRITE_ID_LIST,_Fields.ID,_Fields.SKIP_COLUMN_SCHEMA_FOR_PARTITION,_Fields.INCLUDE_PARAM_KEY_PATTERN,_Fields.EXCLUDE_PARAM_KEY_PATTERN,_Fields.PART_NAMES};
   public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -168,6 +173,9 @@ package org.apache.hadoop.hive.metastore.api;
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.EXCLUDE_PARAM_KEY_PATTERN, new org.apache.thrift.meta_data.FieldMetaData("excludeParamKeyPattern", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.PART_NAMES, new org.apache.thrift.meta_data.FieldMetaData("partNames", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
     metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetPartitionsPsWithAuthRequest.class, metaDataMap);
   }
@@ -225,6 +233,10 @@ package org.apache.hadoop.hive.metastore.api;
     if (other.isSetExcludeParamKeyPattern()) {
       this.excludeParamKeyPattern = other.excludeParamKeyPattern;
     }
+    if (other.isSetPartNames()) {
+      java.util.List<java.lang.String> __this__partNames = new java.util.ArrayList<java.lang.String>(other.partNames);
+      this.partNames = __this__partNames;
+    }
   }
 
   public GetPartitionsPsWithAuthRequest deepCopy() {
@@ -248,6 +260,7 @@ package org.apache.hadoop.hive.metastore.api;
     this.skipColumnSchemaForPartition = false;
     this.includeParamKeyPattern = null;
     this.excludeParamKeyPattern = null;
+    this.partNames = null;
   }
 
   @org.apache.thrift.annotation.Nullable
@@ -564,6 +577,46 @@ package org.apache.hadoop.hive.metastore.api;
     }
   }
 
+  public int getPartNamesSize() {
+    return (this.partNames == null) ? 0 : this.partNames.size();
+  }
+
+  @org.apache.thrift.annotation.Nullable
+  public java.util.Iterator<java.lang.String> getPartNamesIterator() {
+    return (this.partNames == null) ? null : this.partNames.iterator();
+  }
+
+  public void addToPartNames(java.lang.String elem) {
+    if (this.partNames == null) {
+      this.partNames = new java.util.ArrayList<java.lang.String>();
+    }
+    this.partNames.add(elem);
+  }
+
+  @org.apache.thrift.annotation.Nullable
+  public java.util.List<java.lang.String> getPartNames() {
+    return this.partNames;
+  }
+
+  public void setPartNames(@org.apache.thrift.annotation.Nullable java.util.List<java.lang.String> partNames) {
+    this.partNames = partNames;
+  }
+
+  public void unsetPartNames() {
+    this.partNames = null;
+  }
+
+  /** Returns true if field partNames is set (has been assigned a value) and false otherwise */
+  public boolean isSetPartNames() {
+    return this.partNames != null;
+  }
+
+  public void setPartNamesIsSet(boolean value) {
+    if (!value) {
+      this.partNames = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
     switch (field) {
     case CAT_NAME:
@@ -662,6 +715,14 @@ package org.apache.hadoop.hive.metastore.api;
       }
       break;
 
+    case PART_NAMES:
+      if (value == null) {
+        unsetPartNames();
+      } else {
+        setPartNames((java.util.List<java.lang.String>)value);
+      }
+      break;
+
     }
   }
 
@@ -704,6 +765,9 @@ package org.apache.hadoop.hive.metastore.api;
     case EXCLUDE_PARAM_KEY_PATTERN:
       return getExcludeParamKeyPattern();
 
+    case PART_NAMES:
+      return getPartNames();
+
     }
     throw new java.lang.IllegalStateException();
   }
@@ -739,6 +803,8 @@ package org.apache.hadoop.hive.metastore.api;
       return isSetIncludeParamKeyPattern();
     case EXCLUDE_PARAM_KEY_PATTERN:
       return isSetExcludeParamKeyPattern();
+    case PART_NAMES:
+      return isSetPartNames();
     }
     throw new java.lang.IllegalStateException();
   }
@@ -864,6 +930,15 @@ package org.apache.hadoop.hive.metastore.api;
         return false;
     }
 
+    boolean this_present_partNames = true && this.isSetPartNames();
+    boolean that_present_partNames = true && that.isSetPartNames();
+    if (this_present_partNames || that_present_partNames) {
+      if (!(this_present_partNames && that_present_partNames))
+        return false;
+      if (!this.partNames.equals(that.partNames))
+        return false;
+    }
+
     return true;
   }
 
@@ -919,6 +994,10 @@ package org.apache.hadoop.hive.metastore.api;
     if (isSetExcludeParamKeyPattern())
       hashCode = hashCode * 8191 + excludeParamKeyPattern.hashCode();
 
+    hashCode = hashCode * 8191 + ((isSetPartNames()) ? 131071 : 524287);
+    if (isSetPartNames())
+      hashCode = hashCode * 8191 + partNames.hashCode();
+
     return hashCode;
   }
 
@@ -1050,6 +1129,16 @@ package org.apache.hadoop.hive.metastore.api;
         return lastComparison;
       }
     }
+    lastComparison = java.lang.Boolean.compare(isSetPartNames(), other.isSetPartNames());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetPartNames()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.partNames, other.partNames);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -1174,6 +1263,16 @@ package org.apache.hadoop.hive.metastore.api;
       }
       first = false;
     }
+    if (isSetPartNames()) {
+      if (!first) sb.append(", ");
+      sb.append("partNames:");
+      if (this.partNames == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.partNames);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -1343,6 +1442,24 @@ package org.apache.hadoop.hive.metastore.api;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 13: // PART_NAMES
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list1494 = iprot.readListBegin();
+                struct.partNames = new java.util.ArrayList<java.lang.String>(_list1494.size);
+                @org.apache.thrift.annotation.Nullable java.lang.String _elem1495;
+                for (int _i1496 = 0; _i1496 < _list1494.size; ++_i1496)
+                {
+                  _elem1495 = iprot.readString();
+                  struct.partNames.add(_elem1495);
+                }
+                iprot.readListEnd();
+              }
+              struct.setPartNamesIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -1378,9 +1495,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partVals.size()));
-            for (java.lang.String _iter1494 : struct.partVals)
+            for (java.lang.String _iter1497 : struct.partVals)
             {
-              oprot.writeString(_iter1494);
+              oprot.writeString(_iter1497);
             }
             oprot.writeListEnd();
           }
@@ -1404,9 +1521,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.groupNames.size()));
-            for (java.lang.String _iter1495 : struct.groupNames)
+            for (java.lang.String _iter1498 : struct.groupNames)
             {
-              oprot.writeString(_iter1495);
+              oprot.writeString(_iter1498);
             }
             oprot.writeListEnd();
           }
@@ -1444,6 +1561,20 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldEnd();
         }
       }
+      if (struct.partNames != null) {
+        if (struct.isSetPartNames()) {
+          oprot.writeFieldBegin(PART_NAMES_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partNames.size()));
+            for (java.lang.String _iter1499 : struct.partNames)
+            {
+              oprot.writeString(_iter1499);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -1494,16 +1625,19 @@ package org.apache.hadoop.hive.metastore.api;
       if (struct.isSetExcludeParamKeyPattern()) {
         optionals.set(9);
       }
-      oprot.writeBitSet(optionals, 10);
+      if (struct.isSetPartNames()) {
+        optionals.set(10);
+      }
+      oprot.writeBitSet(optionals, 11);
       if (struct.isSetCatName()) {
         oprot.writeString(struct.catName);
       }
       if (struct.isSetPartVals()) {
         {
           oprot.writeI32(struct.partVals.size());
-          for (java.lang.String _iter1496 : struct.partVals)
+          for (java.lang.String _iter1500 : struct.partVals)
           {
-            oprot.writeString(_iter1496);
+            oprot.writeString(_iter1500);
           }
         }
       }
@@ -1516,9 +1650,9 @@ package org.apache.hadoop.hive.metastore.api;
       if (struct.isSetGroupNames()) {
         {
           oprot.writeI32(struct.groupNames.size());
-          for (java.lang.String _iter1497 : struct.groupNames)
+          for (java.lang.String _iter1501 : struct.groupNames)
           {
-            oprot.writeString(_iter1497);
+            oprot.writeString(_iter1501);
           }
         }
       }
@@ -1537,6 +1671,15 @@ package org.apache.hadoop.hive.metastore.api;
       if (struct.isSetExcludeParamKeyPattern()) {
         oprot.writeString(struct.excludeParamKeyPattern);
       }
+      if (struct.isSetPartNames()) {
+        {
+          oprot.writeI32(struct.partNames.size());
+          for (java.lang.String _iter1502 : struct.partNames)
+          {
+            oprot.writeString(_iter1502);
+          }
+        }
+      }
     }
 
     @Override
@@ -1546,20 +1689,20 @@ package org.apache.hadoop.hive.metastore.api;
       struct.setDbNameIsSet(true);
       struct.tblName = iprot.readString();
       struct.setTblNameIsSet(true);
-      java.util.BitSet incoming = iprot.readBitSet(10);
+      java.util.BitSet incoming = iprot.readBitSet(11);
       if (incoming.get(0)) {
         struct.catName = iprot.readString();
         struct.setCatNameIsSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list1498 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-          struct.partVals = new java.util.ArrayList<java.lang.String>(_list1498.size);
-          @org.apache.thrift.annotation.Nullable java.lang.String _elem1499;
-          for (int _i1500 = 0; _i1500 < _list1498.size; ++_i1500)
+          org.apache.thrift.protocol.TList _list1503 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+          struct.partVals = new java.util.ArrayList<java.lang.String>(_list1503.size);
+          @org.apache.thrift.annotation.Nullable java.lang.String _elem1504;
+          for (int _i1505 = 0; _i1505 < _list1503.size; ++_i1505)
           {
-            _elem1499 = iprot.readString();
-            struct.partVals.add(_elem1499);
+            _elem1504 = iprot.readString();
+            struct.partVals.add(_elem1504);
           }
         }
         struct.setPartValsIsSet(true);
@@ -1574,13 +1717,13 @@ package org.apache.hadoop.hive.metastore.api;
       }
       if (incoming.get(4)) {
         {
-          org.apache.thrift.protocol.TList _list1501 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-          struct.groupNames = new java.util.ArrayList<java.lang.String>(_list1501.size);
-          @org.apache.thrift.annotation.Nullable java.lang.String _elem1502;
-          for (int _i1503 = 0; _i1503 < _list1501.size; ++_i1503)
+          org.apache.thrift.protocol.TList _list1506 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+          struct.groupNames = new java.util.ArrayList<java.lang.String>(_list1506.size);
+          @org.apache.thrift.annotation.Nullable java.lang.String _elem1507;
+          for (int _i1508 = 0; _i1508 < _list1506.size; ++_i1508)
           {
-            _elem1502 = iprot.readString();
-            struct.groupNames.add(_elem1502);
+            _elem1507 = iprot.readString();
+            struct.groupNames.add(_elem1507);
           }
         }
         struct.setGroupNamesIsSet(true);
@@ -1605,6 +1748,19 @@ package org.apache.hadoop.hive.metastore.api;
         struct.excludeParamKeyPattern = iprot.readString();
         struct.setExcludeParamKeyPatternIsSet(true);
       }
+      if (incoming.get(10)) {
+        {
+          org.apache.thrift.protocol.TList _list1509 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+          struct.partNames = new java.util.ArrayList<java.lang.String>(_list1509.size);
+          @org.apache.thrift.annotation.Nullable java.lang.String _elem1510;
+          for (int _i1511 = 0; _i1511 < _list1509.size; ++_i1511)
+          {
+            _elem1510 = iprot.readString();
+            struct.partNames.add(_elem1510);
+          }
+        }
+        struct.setPartNamesIsSet(true);
+      }
     }
   }
 
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsPsWithAuthResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsPsWithAuthResponse.java
index fa839ad0470..6984966a98d 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsPsWithAuthResponse.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsPsWithAuthResponse.java
@@ -329,14 +329,14 @@ package org.apache.hadoop.hive.metastore.api;
           case 1: // PARTITIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list1504 = iprot.readListBegin();
-                struct.partitions = new java.util.ArrayList<Partition>(_list1504.size);
-                @org.apache.thrift.annotation.Nullable Partition _elem1505;
-                for (int _i1506 = 0; _i1506 < _list1504.size; ++_i1506)
+                org.apache.thrift.protocol.TList _list1512 = iprot.readListBegin();
+                struct.partitions = new java.util.ArrayList<Partition>(_list1512.size);
+                @org.apache.thrift.annotation.Nullable Partition _elem1513;
+                for (int _i1514 = 0; _i1514 < _list1512.size; ++_i1514)
                 {
-                  _elem1505 = new Partition();
-                  _elem1505.read(iprot);
-                  struct.partitions.add(_elem1505);
+                  _elem1513 = new Partition();
+                  _elem1513.read(iprot);
+                  struct.partitions.add(_elem1513);
                 }
                 iprot.readListEnd();
               }
@@ -362,9 +362,9 @@ package org.apache.hadoop.hive.metastore.api;
         oprot.writeFieldBegin(PARTITIONS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitions.size()));
-          for (Partition _iter1507 : struct.partitions)
+          for (Partition _iter1515 : struct.partitions)
           {
-            _iter1507.write(oprot);
+            _iter1515.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -389,9 +389,9 @@ package org.apache.hadoop.hive.metastore.api;
       org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
       {
         oprot.writeI32(struct.partitions.size());
-        for (Partition _iter1508 : struct.partitions)
+        for (Partition _iter1516 : struct.partitions)
         {
-          _iter1508.write(oprot);
+          _iter1516.write(oprot);
         }
       }
     }
@@ -400,14 +400,14 @@ package org.apache.hadoop.hive.metastore.api;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetPartitionsPsWithAuthResponse struct) throws org.apache.thrift.TException {
       org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list1509 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-        struct.partitions = new java.util.ArrayList<Partition>(_list1509.size);
-        @org.apache.thrift.annotation.Nullable Partition _elem1510;
-        for (int _i1511 = 0; _i1511 < _list1509.size; ++_i1511)
+        org.apache.thrift.protocol.TList _list1517 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+        struct.partitions = new java.util.ArrayList<Partition>(_list1517.size);
+        @org.apache.thrift.annotation.Nullable Partition _elem1518;
+        for (int _i1519 = 0; _i1519 < _list1517.size; ++_i1519)
         {
-          _elem1510 = new Partition();
-          _elem1510.read(iprot);
-          struct.partitions.add(_elem1510);
+          _elem1518 = new Partition();
+          _elem1518.read(iprot);
+          struct.partitions.add(_elem1518);
         }
       }
       struct.setPartitionsIsSet(true);
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplicationMetricList.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplicationMetricList.java
index 891fa2969a9..cbfdcbc01c4 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplicationMetricList.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplicationMetricList.java
@@ -329,14 +329,14 @@ package org.apache.hadoop.hive.metastore.api;
           case 1: // REPLICATION_METRIC_LIST
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list1512 = iprot.readListBegin();
-                struct.replicationMetricList = new java.util.ArrayList<ReplicationMetrics>(_list1512.size);
-                @org.apache.thrift.annotation.Nullable ReplicationMetrics _elem1513;
-                for (int _i1514 = 0; _i1514 < _list1512.size; ++_i1514)
+                org.apache.thrift.protocol.TList _list1520 = iprot.readListBegin();
+                struct.replicationMetricList = new java.util.ArrayList<ReplicationMetrics>(_list1520.size);
+                @org.apache.thrift.annotation.Nullable ReplicationMetrics _elem1521;
+                for (int _i1522 = 0; _i1522 < _list1520.size; ++_i1522)
                 {
-                  _elem1513 = new ReplicationMetrics();
-                  _elem1513.read(iprot);
-                  struct.replicationMetricList.add(_elem1513);
+                  _elem1521 = new ReplicationMetrics();
+                  _elem1521.read(iprot);
+                  struct.replicationMetricList.add(_elem1521);
                 }
                 iprot.readListEnd();
               }
@@ -362,9 +362,9 @@ package org.apache.hadoop.hive.metastore.api;
         oprot.writeFieldBegin(REPLICATION_METRIC_LIST_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.replicationMetricList.size()));
-          for (ReplicationMetrics _iter1515 : struct.replicationMetricList)
+          for (ReplicationMetrics _iter1523 : struct.replicationMetricList)
           {
-            _iter1515.write(oprot);
+            _iter1523.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -389,9 +389,9 @@ package org.apache.hadoop.hive.metastore.api;
       org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
       {
         oprot.writeI32(struct.replicationMetricList.size());
-        for (ReplicationMetrics _iter1516 : struct.replicationMetricList)
+        for (ReplicationMetrics _iter1524 : struct.replicationMetricList)
         {
-          _iter1516.write(oprot);
+          _iter1524.write(oprot);
         }
       }
     }
@@ -400,14 +400,14 @@ package org.apache.hadoop.hive.metastore.api;
     public void read(org.apache.thrift.protocol.TProtocol prot, ReplicationMetricList struct) throws org.apache.thrift.TException {
       org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list1517 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-        struct.replicationMetricList = new java.util.ArrayList<ReplicationMetrics>(_list1517.size);
-        @org.apache.thrift.annotation.Nullable ReplicationMetrics _elem1518;
-        for (int _i1519 = 0; _i1519 < _list1517.size; ++_i1519)
+        org.apache.thrift.protocol.TList _list1525 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+        struct.replicationMetricList = new java.util.ArrayList<ReplicationMetrics>(_list1525.size);
+        @org.apache.thrift.annotation.Nullable ReplicationMetrics _elem1526;
+        for (int _i1527 = 0; _i1527 < _list1525.size; ++_i1527)
         {
-          _elem1518 = new ReplicationMetrics();
-          _elem1518.read(iprot);
-          struct.replicationMetricList.add(_elem1518);
+          _elem1526 = new ReplicationMetrics();
+          _elem1526.read(iprot);
+          struct.replicationMetricList.add(_elem1526);
         }
       }
       struct.setReplicationMetricListIsSet(true);
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index eb9b6dc553d..57799851969 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@ -58665,13 +58665,13 @@ package org.apache.hadoop.hive.metastore.api;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1528 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<java.lang.String>(_list1528.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1529;
-                  for (int _i1530 = 0; _i1530 < _list1528.size; ++_i1530)
+                  org.apache.thrift.protocol.TList _list1536 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<java.lang.String>(_list1536.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1537;
+                  for (int _i1538 = 0; _i1538 < _list1536.size; ++_i1538)
                   {
-                    _elem1529 = iprot.readString();
-                    struct.success.add(_elem1529);
+                    _elem1537 = iprot.readString();
+                    struct.success.add(_elem1537);
                   }
                   iprot.readListEnd();
                 }
@@ -58706,9 +58706,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (java.lang.String _iter1531 : struct.success)
+            for (java.lang.String _iter1539 : struct.success)
             {
-              oprot.writeString(_iter1531);
+              oprot.writeString(_iter1539);
             }
             oprot.writeListEnd();
           }
@@ -58747,9 +58747,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (java.lang.String _iter1532 : struct.success)
+            for (java.lang.String _iter1540 : struct.success)
             {
-              oprot.writeString(_iter1532);
+              oprot.writeString(_iter1540);
             }
           }
         }
@@ -58764,13 +58764,13 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1533 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.success = new java.util.ArrayList<java.lang.String>(_list1533.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem1534;
-            for (int _i1535 = 0; _i1535 < _list1533.size; ++_i1535)
+            org.apache.thrift.protocol.TList _list1541 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.success = new java.util.ArrayList<java.lang.String>(_list1541.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem1542;
+            for (int _i1543 = 0; _i1543 < _list1541.size; ++_i1543)
             {
-              _elem1534 = iprot.readString();
-              struct.success.add(_elem1534);
+              _elem1542 = iprot.readString();
+              struct.success.add(_elem1542);
             }
           }
           struct.setSuccessIsSet(true);
@@ -59433,13 +59433,13 @@ package org.apache.hadoop.hive.metastore.api;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1536 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<java.lang.String>(_list1536.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1537;
-                  for (int _i1538 = 0; _i1538 < _list1536.size; ++_i1538)
+                  org.apache.thrift.protocol.TList _list1544 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<java.lang.String>(_list1544.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1545;
+                  for (int _i1546 = 0; _i1546 < _list1544.size; ++_i1546)
                   {
-                    _elem1537 = iprot.readString();
-                    struct.success.add(_elem1537);
+                    _elem1545 = iprot.readString();
+                    struct.success.add(_elem1545);
                   }
                   iprot.readListEnd();
                 }
@@ -59474,9 +59474,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (java.lang.String _iter1539 : struct.success)
+            for (java.lang.String _iter1547 : struct.success)
             {
-              oprot.writeString(_iter1539);
+              oprot.writeString(_iter1547);
             }
             oprot.writeListEnd();
           }
@@ -59515,9 +59515,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (java.lang.String _iter1540 : struct.success)
+            for (java.lang.String _iter1548 : struct.success)
             {
-              oprot.writeString(_iter1540);
+              oprot.writeString(_iter1548);
             }
           }
         }
@@ -59532,13 +59532,13 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1541 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.success = new java.util.ArrayList<java.lang.String>(_list1541.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem1542;
-            for (int _i1543 = 0; _i1543 < _list1541.size; ++_i1543)
+            org.apache.thrift.protocol.TList _list1549 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.success = new java.util.ArrayList<java.lang.String>(_list1549.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem1550;
+            for (int _i1551 = 0; _i1551 < _list1549.size; ++_i1551)
             {
-              _elem1542 = iprot.readString();
-              struct.success.add(_elem1542);
+              _elem1550 = iprot.readString();
+              struct.success.add(_elem1550);
             }
           }
           struct.setSuccessIsSet(true);
@@ -64163,13 +64163,13 @@ package org.apache.hadoop.hive.metastore.api;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1544 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<java.lang.String>(_list1544.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1545;
-                  for (int _i1546 = 0; _i1546 < _list1544.size; ++_i1546)
+                  org.apache.thrift.protocol.TList _list1552 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<java.lang.String>(_list1552.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1553;
+                  for (int _i1554 = 0; _i1554 < _list1552.size; ++_i1554)
                   {
-                    _elem1545 = iprot.readString();
-                    struct.success.add(_elem1545);
+                    _elem1553 = iprot.readString();
+                    struct.success.add(_elem1553);
                   }
                   iprot.readListEnd();
                 }
@@ -64204,9 +64204,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (java.lang.String _iter1547 : struct.success)
+            for (java.lang.String _iter1555 : struct.success)
             {
-              oprot.writeString(_iter1547);
+              oprot.writeString(_iter1555);
             }
             oprot.writeListEnd();
           }
@@ -64245,9 +64245,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (java.lang.String _iter1548 : struct.success)
+            for (java.lang.String _iter1556 : struct.success)
             {
-              oprot.writeString(_iter1548);
+              oprot.writeString(_iter1556);
             }
           }
         }
@@ -64262,13 +64262,13 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1549 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.success = new java.util.ArrayList<java.lang.String>(_list1549.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem1550;
-            for (int _i1551 = 0; _i1551 < _list1549.size; ++_i1551)
+            org.apache.thrift.protocol.TList _list1557 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.success = new java.util.ArrayList<java.lang.String>(_list1557.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem1558;
+            for (int _i1559 = 0; _i1559 < _list1557.size; ++_i1559)
             {
-              _elem1550 = iprot.readString();
-              struct.success.add(_elem1550);
+              _elem1558 = iprot.readString();
+              struct.success.add(_elem1558);
             }
           }
           struct.setSuccessIsSet(true);
@@ -68909,16 +68909,16 @@ package org.apache.hadoop.hive.metastore.api;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map1552 = iprot.readMapBegin();
-                  struct.success = new java.util.HashMap<java.lang.String,Type>(2*_map1552.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _key1553;
-                  @org.apache.thrift.annotation.Nullable Type _val1554;
-                  for (int _i1555 = 0; _i1555 < _map1552.size; ++_i1555)
+                  org.apache.thrift.protocol.TMap _map1560 = iprot.readMapBegin();
+                  struct.success = new java.util.HashMap<java.lang.String,Type>(2*_map1560.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _key1561;
+                  @org.apache.thrift.annotation.Nullable Type _val1562;
+                  for (int _i1563 = 0; _i1563 < _map1560.size; ++_i1563)
                   {
-                    _key1553 = iprot.readString();
-                    _val1554 = new Type();
-                    _val1554.read(iprot);
-                    struct.success.put(_key1553, _val1554);
+                    _key1561 = iprot.readString();
+                    _val1562 = new Type();
+                    _val1562.read(iprot);
+                    struct.success.put(_key1561, _val1562);
                   }
                   iprot.readMapEnd();
                 }
@@ -68953,10 +68953,10 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (java.util.Map.Entry<java.lang.String, Type> _iter1556 : struct.success.entrySet())
+            for (java.util.Map.Entry<java.lang.String, Type> _iter1564 : struct.success.entrySet())
             {
-              oprot.writeString(_iter1556.getKey());
-              _iter1556.getValue().write(oprot);
+              oprot.writeString(_iter1564.getKey());
+              _iter1564.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -68995,10 +68995,10 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (java.util.Map.Entry<java.lang.String, Type> _iter1557 : struct.success.entrySet())
+            for (java.util.Map.Entry<java.lang.String, Type> _iter1565 : struct.success.entrySet())
             {
-              oprot.writeString(_iter1557.getKey());
-              _iter1557.getValue().write(oprot);
+              oprot.writeString(_iter1565.getKey());
+              _iter1565.getValue().write(oprot);
             }
           }
         }
@@ -69013,16 +69013,16 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map1558 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT); 
-            struct.success = new java.util.HashMap<java.lang.String,Type>(2*_map1558.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _key1559;
-            @org.apache.thrift.annotation.Nullable Type _val1560;
-            for (int _i1561 = 0; _i1561 < _map1558.size; ++_i1561)
+            org.apache.thrift.protocol.TMap _map1566 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT); 
+            struct.success = new java.util.HashMap<java.lang.String,Type>(2*_map1566.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _key1567;
+            @org.apache.thrift.annotation.Nullable Type _val1568;
+            for (int _i1569 = 0; _i1569 < _map1566.size; ++_i1569)
             {
-              _key1559 = iprot.readString();
-              _val1560 = new Type();
-              _val1560.read(iprot);
-              struct.success.put(_key1559, _val1560);
+              _key1567 = iprot.readString();
+              _val1568 = new Type();
+              _val1568.read(iprot);
+              struct.success.put(_key1567, _val1568);
             }
           }
           struct.setSuccessIsSet(true);
@@ -70066,14 +70066,14 @@ package org.apache.hadoop.hive.metastore.api;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1562 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<FieldSchema>(_list1562.size);
-                  @org.apache.thrift.annotation.Nullable FieldSchema _elem1563;
-                  for (int _i1564 = 0; _i1564 < _list1562.size; ++_i1564)
+                  org.apache.thrift.protocol.TList _list1570 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<FieldSchema>(_list1570.size);
+                  @org.apache.thrift.annotation.Nullable FieldSchema _elem1571;
+                  for (int _i1572 = 0; _i1572 < _list1570.size; ++_i1572)
                   {
-                    _elem1563 = new FieldSchema();
-                    _elem1563.read(iprot);
-                    struct.success.add(_elem1563);
+                    _elem1571 = new FieldSchema();
+                    _elem1571.read(iprot);
+                    struct.success.add(_elem1571);
                   }
                   iprot.readListEnd();
                 }
@@ -70126,9 +70126,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter1565 : struct.success)
+            for (FieldSchema _iter1573 : struct.success)
             {
-              _iter1565.write(oprot);
+              _iter1573.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -70183,9 +70183,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter1566 : struct.success)
+            for (FieldSchema _iter1574 : struct.success)
             {
-              _iter1566.write(oprot);
+              _iter1574.write(oprot);
             }
           }
         }
@@ -70206,14 +70206,14 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1567 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.success = new java.util.ArrayList<FieldSchema>(_list1567.size);
-            @org.apache.thrift.annotation.Nullable FieldSchema _elem1568;
-            for (int _i1569 = 0; _i1569 < _list1567.size; ++_i1569)
+            org.apache.thrift.protocol.TList _list1575 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.success = new java.util.ArrayList<FieldSchema>(_list1575.size);
+            @org.apache.thrift.annotation.Nullable FieldSchema _elem1576;
+            for (int _i1577 = 0; _i1577 < _list1575.size; ++_i1577)
             {
-              _elem1568 = new FieldSchema();
-              _elem1568.read(iprot);
-              struct.success.add(_elem1568);
+              _elem1576 = new FieldSchema();
+              _elem1576.read(iprot);
+              struct.success.add(_elem1576);
             }
           }
           struct.setSuccessIsSet(true);
@@ -71376,14 +71376,14 @@ package org.apache.hadoop.hive.metastore.api;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1570 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<FieldSchema>(_list1570.size);
-                  @org.apache.thrift.annotation.Nullable FieldSchema _elem1571;
-                  for (int _i1572 = 0; _i1572 < _list1570.size; ++_i1572)
+                  org.apache.thrift.protocol.TList _list1578 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<FieldSchema>(_list1578.size);
+                  @org.apache.thrift.annotation.Nullable FieldSchema _elem1579;
+                  for (int _i1580 = 0; _i1580 < _list1578.size; ++_i1580)
                   {
-                    _elem1571 = new FieldSchema();
-                    _elem1571.read(iprot);
-                    struct.success.add(_elem1571);
+                    _elem1579 = new FieldSchema();
+                    _elem1579.read(iprot);
+                    struct.success.add(_elem1579);
                   }
                   iprot.readListEnd();
                 }
@@ -71436,9 +71436,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter1573 : struct.success)
+            for (FieldSchema _iter1581 : struct.success)
             {
-              _iter1573.write(oprot);
+              _iter1581.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -71493,9 +71493,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter1574 : struct.success)
+            for (FieldSchema _iter1582 : struct.success)
             {
-              _iter1574.write(oprot);
+              _iter1582.write(oprot);
             }
           }
         }
@@ -71516,14 +71516,14 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1575 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.success = new java.util.ArrayList<FieldSchema>(_list1575.size);
-            @org.apache.thrift.annotation.Nullable FieldSchema _elem1576;
-            for (int _i1577 = 0; _i1577 < _list1575.size; ++_i1577)
+            org.apache.thrift.protocol.TList _list1583 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.success = new java.util.ArrayList<FieldSchema>(_list1583.size);
+            @org.apache.thrift.annotation.Nullable FieldSchema _elem1584;
+            for (int _i1585 = 0; _i1585 < _list1583.size; ++_i1585)
             {
-              _elem1576 = new FieldSchema();
-              _elem1576.read(iprot);
-              struct.success.add(_elem1576);
+              _elem1584 = new FieldSchema();
+              _elem1584.read(iprot);
+              struct.success.add(_elem1584);
             }
           }
           struct.setSuccessIsSet(true);
@@ -73629,14 +73629,14 @@ package org.apache.hadoop.hive.metastore.api;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1578 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<FieldSchema>(_list1578.size);
-                  @org.apache.thrift.annotation.Nullable FieldSchema _elem1579;
-                  for (int _i1580 = 0; _i1580 < _list1578.size; ++_i1580)
+                  org.apache.thrift.protocol.TList _list1586 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<FieldSchema>(_list1586.size);
+                  @org.apache.thrift.annotation.Nullable FieldSchema _elem1587;
+                  for (int _i1588 = 0; _i1588 < _list1586.size; ++_i1588)
                   {
-                    _elem1579 = new FieldSchema();
-                    _elem1579.read(iprot);
-                    struct.success.add(_elem1579);
+                    _elem1587 = new FieldSchema();
+                    _elem1587.read(iprot);
+                    struct.success.add(_elem1587);
                   }
                   iprot.readListEnd();
                 }
@@ -73689,9 +73689,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter1581 : struct.success)
+            for (FieldSchema _iter1589 : struct.success)
             {
-              _iter1581.write(oprot);
+              _iter1589.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -73746,9 +73746,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter1582 : struct.success)
+            for (FieldSchema _iter1590 : struct.success)
             {
-              _iter1582.write(oprot);
+              _iter1590.write(oprot);
             }
           }
         }
@@ -73769,14 +73769,14 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1583 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.success = new java.util.ArrayList<FieldSchema>(_list1583.size);
-            @org.apache.thrift.annotation.Nullable FieldSchema _elem1584;
-            for (int _i1585 = 0; _i1585 < _list1583.size; ++_i1585)
+            org.apache.thrift.protocol.TList _list1591 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.success = new java.util.ArrayList<FieldSchema>(_list1591.size);
+            @org.apache.thrift.annotation.Nullable FieldSchema _elem1592;
+            for (int _i1593 = 0; _i1593 < _list1591.size; ++_i1593)
             {
-              _elem1584 = new FieldSchema();
-              _elem1584.read(iprot);
-              struct.success.add(_elem1584);
+              _elem1592 = new FieldSchema();
+              _elem1592.read(iprot);
+              struct.success.add(_elem1592);
             }
           }
           struct.setSuccessIsSet(true);
@@ -74939,14 +74939,14 @@ package org.apache.hadoop.hive.metastore.api;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1586 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<FieldSchema>(_list1586.size);
-                  @org.apache.thrift.annotation.Nullable FieldSchema _elem1587;
-                  for (int _i1588 = 0; _i1588 < _list1586.size; ++_i1588)
+                  org.apache.thrift.protocol.TList _list1594 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<FieldSchema>(_list1594.size);
+                  @org.apache.thrift.annotation.Nullable FieldSchema _elem1595;
+                  for (int _i1596 = 0; _i1596 < _list1594.size; ++_i1596)
                   {
-                    _elem1587 = new FieldSchema();
-                    _elem1587.read(iprot);
-                    struct.success.add(_elem1587);
+                    _elem1595 = new FieldSchema();
+                    _elem1595.read(iprot);
+                    struct.success.add(_elem1595);
                   }
                   iprot.readListEnd();
                 }
@@ -74999,9 +74999,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter1589 : struct.success)
+            for (FieldSchema _iter1597 : struct.success)
             {
-              _iter1589.write(oprot);
+              _iter1597.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -75056,9 +75056,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter1590 : struct.success)
+            for (FieldSchema _iter1598 : struct.success)
             {
-              _iter1590.write(oprot);
+              _iter1598.write(oprot);
             }
           }
         }
@@ -75079,14 +75079,14 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1591 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.success = new java.util.ArrayList<FieldSchema>(_list1591.size);
-            @org.apache.thrift.annotation.Nullable FieldSchema _elem1592;
-            for (int _i1593 = 0; _i1593 < _list1591.size; ++_i1593)
+            org.apache.thrift.protocol.TList _list1599 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.success = new java.util.ArrayList<FieldSchema>(_list1599.size);
+            @org.apache.thrift.annotation.Nullable FieldSchema _elem1600;
+            for (int _i1601 = 0; _i1601 < _list1599.size; ++_i1601)
             {
-              _elem1592 = new FieldSchema();
-              _elem1592.read(iprot);
-              struct.success.add(_elem1592);
+              _elem1600 = new FieldSchema();
+              _elem1600.read(iprot);
+              struct.success.add(_elem1600);
             }
           }
           struct.setSuccessIsSet(true);
@@ -79293,14 +79293,14 @@ package org.apache.hadoop.hive.metastore.api;
             case 2: // PRIMARY_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1594 = iprot.readListBegin();
-                  struct.primaryKeys = new java.util.ArrayList<SQLPrimaryKey>(_list1594.size);
-                  @org.apache.thrift.annotation.Nullable SQLPrimaryKey _elem1595;
-                  for (int _i1596 = 0; _i1596 < _list1594.size; ++_i1596)
+                  org.apache.thrift.protocol.TList _list1602 = iprot.readListBegin();
+                  struct.primaryKeys = new java.util.ArrayList<SQLPrimaryKey>(_list1602.size);
+                  @org.apache.thrift.annotation.Nullable SQLPrimaryKey _elem1603;
+                  for (int _i1604 = 0; _i1604 < _list1602.size; ++_i1604)
                   {
-                    _elem1595 = new SQLPrimaryKey();
-                    _elem1595.read(iprot);
-                    struct.primaryKeys.add(_elem1595);
+                    _elem1603 = new SQLPrimaryKey();
+                    _elem1603.read(iprot);
+                    struct.primaryKeys.add(_elem1603);
                   }
                   iprot.readListEnd();
                 }
@@ -79312,14 +79312,14 @@ package org.apache.hadoop.hive.metastore.api;
             case 3: // FOREIGN_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1597 = iprot.readListBegin();
-                  struct.foreignKeys = new java.util.ArrayList<SQLForeignKey>(_list1597.size);
-                  @org.apache.thrift.annotation.Nullable SQLForeignKey _elem1598;
-                  for (int _i1599 = 0; _i1599 < _list1597.size; ++_i1599)
+                  org.apache.thrift.protocol.TList _list1605 = iprot.readListBegin();
+                  struct.foreignKeys = new java.util.ArrayList<SQLForeignKey>(_list1605.size);
+                  @org.apache.thrift.annotation.Nullable SQLForeignKey _elem1606;
+                  for (int _i1607 = 0; _i1607 < _list1605.size; ++_i1607)
                   {
-                    _elem1598 = new SQLForeignKey();
-                    _elem1598.read(iprot);
-                    struct.foreignKeys.add(_elem1598);
+                    _elem1606 = new SQLForeignKey();
+                    _elem1606.read(iprot);
+                    struct.foreignKeys.add(_elem1606);
                   }
                   iprot.readListEnd();
                 }
@@ -79331,14 +79331,14 @@ package org.apache.hadoop.hive.metastore.api;
             case 4: // UNIQUE_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1600 = iprot.readListBegin();
-                  struct.uniqueConstraints = new java.util.ArrayList<SQLUniqueConstraint>(_list1600.size);
-                  @org.apache.thrift.annotation.Nullable SQLUniqueConstraint _elem1601;
-                  for (int _i1602 = 0; _i1602 < _list1600.size; ++_i1602)
+                  org.apache.thrift.protocol.TList _list1608 = iprot.readListBegin();
+                  struct.uniqueConstraints = new java.util.ArrayList<SQLUniqueConstraint>(_list1608.size);
+                  @org.apache.thrift.annotation.Nullable SQLUniqueConstraint _elem1609;
+                  for (int _i1610 = 0; _i1610 < _list1608.size; ++_i1610)
                   {
-                    _elem1601 = new SQLUniqueConstraint();
-                    _elem1601.read(iprot);
-                    struct.uniqueConstraints.add(_elem1601);
+                    _elem1609 = new SQLUniqueConstraint();
+                    _elem1609.read(iprot);
+                    struct.uniqueConstraints.add(_elem1609);
                   }
                   iprot.readListEnd();
                 }
@@ -79350,14 +79350,14 @@ package org.apache.hadoop.hive.metastore.api;
             case 5: // NOT_NULL_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1603 = iprot.readListBegin();
-                  struct.notNullConstraints = new java.util.ArrayList<SQLNotNullConstraint>(_list1603.size);
-                  @org.apache.thrift.annotation.Nullable SQLNotNullConstraint _elem1604;
-                  for (int _i1605 = 0; _i1605 < _list1603.size; ++_i1605)
+                  org.apache.thrift.protocol.TList _list1611 = iprot.readListBegin();
+                  struct.notNullConstraints = new java.util.ArrayList<SQLNotNullConstraint>(_list1611.size);
+                  @org.apache.thrift.annotation.Nullable SQLNotNullConstraint _elem1612;
+                  for (int _i1613 = 0; _i1613 < _list1611.size; ++_i1613)
                   {
-                    _elem1604 = new SQLNotNullConstraint();
-                    _elem1604.read(iprot);
-                    struct.notNullConstraints.add(_elem1604);
+                    _elem1612 = new SQLNotNullConstraint();
+                    _elem1612.read(iprot);
+                    struct.notNullConstraints.add(_elem1612);
                   }
                   iprot.readListEnd();
                 }
@@ -79369,14 +79369,14 @@ package org.apache.hadoop.hive.metastore.api;
             case 6: // DEFAULT_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1606 = iprot.readListBegin();
-                  struct.defaultConstraints = new java.util.ArrayList<SQLDefaultConstraint>(_list1606.size);
-                  @org.apache.thrift.annotation.Nullable SQLDefaultConstraint _elem1607;
-                  for (int _i1608 = 0; _i1608 < _list1606.size; ++_i1608)
+                  org.apache.thrift.protocol.TList _list1614 = iprot.readListBegin();
+                  struct.defaultConstraints = new java.util.ArrayList<SQLDefaultConstraint>(_list1614.size);
+                  @org.apache.thrift.annotation.Nullable SQLDefaultConstraint _elem1615;
+                  for (int _i1616 = 0; _i1616 < _list1614.size; ++_i1616)
                   {
-                    _elem1607 = new SQLDefaultConstraint();
-                    _elem1607.read(iprot);
-                    struct.defaultConstraints.add(_elem1607);
+                    _elem1615 = new SQLDefaultConstraint();
+                    _elem1615.read(iprot);
+                    struct.defaultConstraints.add(_elem1615);
                   }
                   iprot.readListEnd();
                 }
@@ -79388,14 +79388,14 @@ package org.apache.hadoop.hive.metastore.api;
             case 7: // CHECK_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1609 = iprot.readListBegin();
-                  struct.checkConstraints = new java.util.ArrayList<SQLCheckConstraint>(_list1609.size);
-                  @org.apache.thrift.annotation.Nullable SQLCheckConstraint _elem1610;
-                  for (int _i1611 = 0; _i1611 < _list1609.size; ++_i1611)
+                  org.apache.thrift.protocol.TList _list1617 = iprot.readListBegin();
+                  struct.checkConstraints = new java.util.ArrayList<SQLCheckConstraint>(_list1617.size);
+                  @org.apache.thrift.annotation.Nullable SQLCheckConstraint _elem1618;
+                  for (int _i1619 = 0; _i1619 < _list1617.size; ++_i1619)
                   {
-                    _elem1610 = new SQLCheckConstraint();
-                    _elem1610.read(iprot);
-                    struct.checkConstraints.add(_elem1610);
+                    _elem1618 = new SQLCheckConstraint();
+                    _elem1618.read(iprot);
+                    struct.checkConstraints.add(_elem1618);
                   }
                   iprot.readListEnd();
                 }
@@ -79426,9 +79426,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(PRIMARY_KEYS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.primaryKeys.size()));
-            for (SQLPrimaryKey _iter1612 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter1620 : struct.primaryKeys)
             {
-              _iter1612.write(oprot);
+              _iter1620.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -79438,9 +79438,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(FOREIGN_KEYS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.foreignKeys.size()));
-            for (SQLForeignKey _iter1613 : struct.foreignKeys)
+            for (SQLForeignKey _iter1621 : struct.foreignKeys)
             {
-              _iter1613.write(oprot);
+              _iter1621.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -79450,9 +79450,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(UNIQUE_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.uniqueConstraints.size()));
-            for (SQLUniqueConstraint _iter1614 : struct.uniqueConstraints)
+            for (SQLUniqueConstraint _iter1622 : struct.uniqueConstraints)
             {
-              _iter1614.write(oprot);
+              _iter1622.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -79462,9 +79462,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(NOT_NULL_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.notNullConstraints.size()));
-            for (SQLNotNullConstraint _iter1615 : struct.notNullConstraints)
+            for (SQLNotNullConstraint _iter1623 : struct.notNullConstraints)
             {
-              _iter1615.write(oprot);
+              _iter1623.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -79474,9 +79474,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(DEFAULT_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.defaultConstraints.size()));
-            for (SQLDefaultConstraint _iter1616 : struct.defaultConstraints)
+            for (SQLDefaultConstraint _iter1624 : struct.defaultConstraints)
             {
-              _iter1616.write(oprot);
+              _iter1624.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -79486,9 +79486,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(CHECK_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.checkConstraints.size()));
-            for (SQLCheckConstraint _iter1617 : struct.checkConstraints)
+            for (SQLCheckConstraint _iter1625 : struct.checkConstraints)
             {
-              _iter1617.write(oprot);
+              _iter1625.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -79540,54 +79540,54 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetPrimaryKeys()) {
           {
             oprot.writeI32(struct.primaryKeys.size());
-            for (SQLPrimaryKey _iter1618 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter1626 : struct.primaryKeys)
             {
-              _iter1618.write(oprot);
+              _iter1626.write(oprot);
             }
           }
         }
         if (struct.isSetForeignKeys()) {
           {
             oprot.writeI32(struct.foreignKeys.size());
-            for (SQLForeignKey _iter1619 : struct.foreignKeys)
+            for (SQLForeignKey _iter1627 : struct.foreignKeys)
             {
-              _iter1619.write(oprot);
+              _iter1627.write(oprot);
             }
           }
         }
         if (struct.isSetUniqueConstraints()) {
           {
             oprot.writeI32(struct.uniqueConstraints.size());
-            for (SQLUniqueConstraint _iter1620 : struct.uniqueConstraints)
+            for (SQLUniqueConstraint _iter1628 : struct.uniqueConstraints)
             {
-              _iter1620.write(oprot);
+              _iter1628.write(oprot);
             }
           }
         }
         if (struct.isSetNotNullConstraints()) {
           {
             oprot.writeI32(struct.notNullConstraints.size());
-            for (SQLNotNullConstraint _iter1621 : struct.notNullConstraints)
+            for (SQLNotNullConstraint _iter1629 : struct.notNullConstraints)
             {
-              _iter1621.write(oprot);
+              _iter1629.write(oprot);
             }
           }
         }
         if (struct.isSetDefaultConstraints()) {
           {
             oprot.writeI32(struct.defaultConstraints.size());
-            for (SQLDefaultConstraint _iter1622 : struct.defaultConstraints)
+            for (SQLDefaultConstraint _iter1630 : struct.defaultConstraints)
             {
-              _iter1622.write(oprot);
+              _iter1630.write(oprot);
             }
           }
         }
         if (struct.isSetCheckConstraints()) {
           {
             oprot.writeI32(struct.checkConstraints.size());
-            for (SQLCheckConstraint _iter1623 : struct.checkConstraints)
+            for (SQLCheckConstraint _iter1631 : struct.checkConstraints)
             {
-              _iter1623.write(oprot);
+              _iter1631.write(oprot);
             }
           }
         }
@@ -79604,84 +79604,84 @@ package org.apache.hadoop.hive.metastore.api;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list1624 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.primaryKeys = new java.util.ArrayList<SQLPrimaryKey>(_list1624.size);
-            @org.apache.thrift.annotation.Nullable SQLPrimaryKey _elem1625;
-            for (int _i1626 = 0; _i1626 < _list1624.size; ++_i1626)
+            org.apache.thrift.protocol.TList _list1632 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.primaryKeys = new java.util.ArrayList<SQLPrimaryKey>(_list1632.size);
+            @org.apache.thrift.annotation.Nullable SQLPrimaryKey _elem1633;
+            for (int _i1634 = 0; _i1634 < _list1632.size; ++_i1634)
             {
-              _elem1625 = new SQLPrimaryKey();
-              _elem1625.read(iprot);
-              struct.primaryKeys.add(_elem1625);
+              _elem1633 = new SQLPrimaryKey();
+              _elem1633.read(iprot);
+              struct.primaryKeys.add(_elem1633);
             }
           }
           struct.setPrimaryKeysIsSet(true);
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1627 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.foreignKeys = new java.util.ArrayList<SQLForeignKey>(_list1627.size);
-            @org.apache.thrift.annotation.Nullable SQLForeignKey _elem1628;
-            for (int _i1629 = 0; _i1629 < _list1627.size; ++_i1629)
+            org.apache.thrift.protocol.TList _list1635 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.foreignKeys = new java.util.ArrayList<SQLForeignKey>(_list1635.size);
+            @org.apache.thrift.annotation.Nullable SQLForeignKey _elem1636;
+            for (int _i1637 = 0; _i1637 < _list1635.size; ++_i1637)
             {
-              _elem1628 = new SQLForeignKey();
-              _elem1628.read(iprot);
-              struct.foreignKeys.add(_elem1628);
+              _elem1636 = new SQLForeignKey();
+              _elem1636.read(iprot);
+              struct.foreignKeys.add(_elem1636);
             }
           }
           struct.setForeignKeysIsSet(true);
         }
         if (incoming.get(3)) {
           {
-            org.apache.thrift.protocol.TList _list1630 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.uniqueConstraints = new java.util.ArrayList<SQLUniqueConstraint>(_list1630.size);
-            @org.apache.thrift.annotation.Nullable SQLUniqueConstraint _elem1631;
-            for (int _i1632 = 0; _i1632 < _list1630.size; ++_i1632)
+            org.apache.thrift.protocol.TList _list1638 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.uniqueConstraints = new java.util.ArrayList<SQLUniqueConstraint>(_list1638.size);
+            @org.apache.thrift.annotation.Nullable SQLUniqueConstraint _elem1639;
+            for (int _i1640 = 0; _i1640 < _list1638.size; ++_i1640)
             {
-              _elem1631 = new SQLUniqueConstraint();
-              _elem1631.read(iprot);
-              struct.uniqueConstraints.add(_elem1631);
+              _elem1639 = new SQLUniqueConstraint();
+              _elem1639.read(iprot);
+              struct.uniqueConstraints.add(_elem1639);
             }
           }
           struct.setUniqueConstraintsIsSet(true);
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol.TList _list1633 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.notNullConstraints = new java.util.ArrayList<SQLNotNullConstraint>(_list1633.size);
-            @org.apache.thrift.annotation.Nullable SQLNotNullConstraint _elem1634;
-            for (int _i1635 = 0; _i1635 < _list1633.size; ++_i1635)
+            org.apache.thrift.protocol.TList _list1641 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.notNullConstraints = new java.util.ArrayList<SQLNotNullConstraint>(_list1641.size);
+            @org.apache.thrift.annotation.Nullable SQLNotNullConstraint _elem1642;
+            for (int _i1643 = 0; _i1643 < _list1641.size; ++_i1643)
             {
-              _elem1634 = new SQLNotNullConstraint();
-              _elem1634.read(iprot);
-              struct.notNullConstraints.add(_elem1634);
+              _elem1642 = new SQLNotNullConstraint();
+              _elem1642.read(iprot);
+              struct.notNullConstraints.add(_elem1642);
             }
           }
           struct.setNotNullConstraintsIsSet(true);
         }
         if (incoming.get(5)) {
           {
-            org.apache.thrift.protocol.TList _list1636 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.defaultConstraints = new java.util.ArrayList<SQLDefaultConstraint>(_list1636.size);
-            @org.apache.thrift.annotation.Nullable SQLDefaultConstraint _elem1637;
-            for (int _i1638 = 0; _i1638 < _list1636.size; ++_i1638)
+            org.apache.thrift.protocol.TList _list1644 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.defaultConstraints = new java.util.ArrayList<SQLDefaultConstraint>(_list1644.size);
+            @org.apache.thrift.annotation.Nullable SQLDefaultConstraint _elem1645;
+            for (int _i1646 = 0; _i1646 < _list1644.size; ++_i1646)
             {
-              _elem1637 = new SQLDefaultConstraint();
-              _elem1637.read(iprot);
-              struct.defaultConstraints.add(_elem1637);
+              _elem1645 = new SQLDefaultConstraint();
+              _elem1645.read(iprot);
+              struct.defaultConstraints.add(_elem1645);
             }
           }
           struct.setDefaultConstraintsIsSet(true);
         }
         if (incoming.get(6)) {
           {
-            org.apache.thrift.protocol.TList _list1639 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.checkConstraints = new java.util.ArrayList<SQLCheckConstraint>(_list1639.size);
-            @org.apache.thrift.annotation.Nullable SQLCheckConstraint _elem1640;
-            for (int _i1641 = 0; _i1641 < _list1639.size; ++_i1641)
+            org.apache.thrift.protocol.TList _list1647 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.checkConstraints = new java.util.ArrayList<SQLCheckConstraint>(_list1647.size);
+            @org.apache.thrift.annotation.Nullable SQLCheckConstraint _elem1648;
+            for (int _i1649 = 0; _i1649 < _list1647.size; ++_i1649)
             {
-              _elem1640 = new SQLCheckConstraint();
-              _elem1640.read(iprot);
-              struct.checkConstraints.add(_elem1640);
+              _elem1648 = new SQLCheckConstraint();
+              _elem1648.read(iprot);
+              struct.checkConstraints.add(_elem1648);
             }
           }
           struct.setCheckConstraintsIsSet(true);
@@ -91113,13 +91113,13 @@ package org.apache.hadoop.hive.metastore.api;
             case 3: // PART_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1642 = iprot.readListBegin();
-                  struct.partNames = new java.util.ArrayList<java.lang.String>(_list1642.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1643;
-                  for (int _i1644 = 0; _i1644 < _list1642.size; ++_i1644)
+                  org.apache.thrift.protocol.TList _list1650 = iprot.readListBegin();
+                  struct.partNames = new java.util.ArrayList<java.lang.String>(_list1650.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1651;
+                  for (int _i1652 = 0; _i1652 < _list1650.size; ++_i1652)
                   {
-                    _elem1643 = iprot.readString();
-                    struct.partNames.add(_elem1643);
+                    _elem1651 = iprot.readString();
+                    struct.partNames.add(_elem1651);
                   }
                   iprot.readListEnd();
                 }
@@ -91155,9 +91155,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(PART_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partNames.size()));
-            for (java.lang.String _iter1645 : struct.partNames)
+            for (java.lang.String _iter1653 : struct.partNames)
             {
-              oprot.writeString(_iter1645);
+              oprot.writeString(_iter1653);
             }
             oprot.writeListEnd();
           }
@@ -91200,9 +91200,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetPartNames()) {
           {
             oprot.writeI32(struct.partNames.size());
-            for (java.lang.String _iter1646 : struct.partNames)
+            for (java.lang.String _iter1654 : struct.partNames)
             {
-              oprot.writeString(_iter1646);
+              oprot.writeString(_iter1654);
             }
           }
         }
@@ -91222,13 +91222,13 @@ package org.apache.hadoop.hive.metastore.api;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1647 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.partNames = new java.util.ArrayList<java.lang.String>(_list1647.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem1648;
-            for (int _i1649 = 0; _i1649 < _list1647.size; ++_i1649)
+            org.apache.thrift.protocol.TList _list1655 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.partNames = new java.util.ArrayList<java.lang.String>(_list1655.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem1656;
+            for (int _i1657 = 0; _i1657 < _list1655.size; ++_i1657)
             {
-              _elem1648 = iprot.readString();
-              struct.partNames.add(_elem1648);
+              _elem1656 = iprot.readString();
+              struct.partNames.add(_elem1656);
             }
           }
           struct.setPartNamesIsSet(true);
@@ -93306,13 +93306,13 @@ package org.apache.hadoop.hive.metastore.api;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1650 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<java.lang.String>(_list1650.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1651;
-                  for (int _i1652 = 0; _i1652 < _list1650.size; ++_i1652)
+                  org.apache.thrift.protocol.TList _list1658 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<java.lang.String>(_list1658.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1659;
+                  for (int _i1660 = 0; _i1660 < _list1658.size; ++_i1660)
                   {
-                    _elem1651 = iprot.readString();
-                    struct.success.add(_elem1651);
+                    _elem1659 = iprot.readString();
+                    struct.success.add(_elem1659);
                   }
                   iprot.readListEnd();
                 }
@@ -93347,9 +93347,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (java.lang.String _iter1653 : struct.success)
+            for (java.lang.String _iter1661 : struct.success)
             {
-              oprot.writeString(_iter1653);
+              oprot.writeString(_iter1661);
             }
             oprot.writeListEnd();
           }
@@ -93388,9 +93388,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (java.lang.String _iter1654 : struct.success)
+            for (java.lang.String _iter1662 : struct.success)
             {
-              oprot.writeString(_iter1654);
+              oprot.writeString(_iter1662);
             }
           }
         }
@@ -93405,13 +93405,13 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1655 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.success = new java.util.ArrayList<java.lang.String>(_list1655.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem1656;
-            for (int _i1657 = 0; _i1657 < _list1655.size; ++_i1657)
+            org.apache.thrift.protocol.TList _list1663 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.success = new java.util.ArrayList<java.lang.String>(_list1663.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem1664;
+            for (int _i1665 = 0; _i1665 < _list1663.size; ++_i1665)
             {
-              _elem1656 = iprot.readString();
-              struct.success.add(_elem1656);
+              _elem1664 = iprot.readString();
+              struct.success.add(_elem1664);
             }
           }
           struct.setSuccessIsSet(true);
@@ -94394,13 +94394,13 @@ package org.apache.hadoop.hive.metastore.api;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1658 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<java.lang.String>(_list1658.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1659;
-                  for (int _i1660 = 0; _i1660 < _list1658.size; ++_i1660)
+                  org.apache.thrift.protocol.TList _list1666 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<java.lang.String>(_list1666.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1667;
+                  for (int _i1668 = 0; _i1668 < _list1666.size; ++_i1668)
                   {
-                    _elem1659 = iprot.readString();
-                    struct.success.add(_elem1659);
+                    _elem1667 = iprot.readString();
+                    struct.success.add(_elem1667);
                   }
                   iprot.readListEnd();
                 }
@@ -94435,9 +94435,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (java.lang.String _iter1661 : struct.success)
+            for (java.lang.String _iter1669 : struct.success)
             {
-              oprot.writeString(_iter1661);
+              oprot.writeString(_iter1669);
             }
             oprot.writeListEnd();
           }
@@ -94476,9 +94476,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (java.lang.String _iter1662 : struct.success)
+            for (java.lang.String _iter1670 : struct.success)
             {
-              oprot.writeString(_iter1662);
+              oprot.writeString(_iter1670);
             }
           }
         }
@@ -94493,13 +94493,13 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1663 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.success = new java.util.ArrayList<java.lang.String>(_list1663.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem1664;
-            for (int _i1665 = 0; _i1665 < _list1663.size; ++_i1665)
+            org.apache.thrift.protocol.TList _list1671 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.success = new java.util.ArrayList<java.lang.String>(_list1671.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem1672;
+            for (int _i1673 = 0; _i1673 < _list1671.size; ++_i1673)
             {
-              _elem1664 = iprot.readString();
-              struct.success.add(_elem1664);
+              _elem1672 = iprot.readString();
+              struct.success.add(_elem1672);
             }
           }
           struct.setSuccessIsSet(true);
@@ -95165,14 +95165,14 @@ package org.apache.hadoop.hive.metastore.api;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1666 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<Table>(_list1666.size);
-                  @org.apache.thrift.annotation.Nullable Table _elem1667;
-                  for (int _i1668 = 0; _i1668 < _list1666.size; ++_i1668)
+                  org.apache.thrift.protocol.TList _list1674 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<Table>(_list1674.size);
+                  @org.apache.thrift.annotation.Nullable Table _elem1675;
+                  for (int _i1676 = 0; _i1676 < _list1674.size; ++_i1676)
                   {
-                    _elem1667 = new Table();
-                    _elem1667.read(iprot);
-                    struct.success.add(_elem1667);
+                    _elem1675 = new Table();
+                    _elem1675.read(iprot);
+                    struct.success.add(_elem1675);
                   }
                   iprot.readListEnd();
                 }
@@ -95207,9 +95207,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Table _iter1669 : struct.success)
+            for (Table _iter1677 : struct.success)
             {
-              _iter1669.write(oprot);
+              _iter1677.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -95248,9 +95248,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Table _iter1670 : struct.success)
+            for (Table _iter1678 : struct.success)
             {
-              _iter1670.write(oprot);
+              _iter1678.write(oprot);
             }
           }
         }
@@ -95265,14 +95265,14 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1671 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.success = new java.util.ArrayList<Table>(_list1671.size);
-            @org.apache.thrift.annotation.Nullable Table _elem1672;
-            for (int _i1673 = 0; _i1673 < _list1671.size; ++_i1673)
+            org.apache.thrift.protocol.TList _list1679 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.success = new java.util.ArrayList<Table>(_list1679.size);
+            @org.apache.thrift.annotation.Nullable Table _elem1680;
+            for (int _i1681 = 0; _i1681 < _list1679.size; ++_i1681)
             {
-              _elem1672 = new Table();
-              _elem1672.read(iprot);
-              struct.success.add(_elem1672);
+              _elem1680 = new Table();
+              _elem1680.read(iprot);
+              struct.success.add(_elem1680);
             }
           }
           struct.setSuccessIsSet(true);
@@ -96047,13 +96047,13 @@ package org.apache.hadoop.hive.metastore.api;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1674 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<java.lang.String>(_list1674.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1675;
-                  for (int _i1676 = 0; _i1676 < _list1674.size; ++_i1676)
+                  org.apache.thrift.protocol.TList _list1682 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<java.lang.String>(_list1682.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1683;
+                  for (int _i1684 = 0; _i1684 < _list1682.size; ++_i1684)
                   {
-                    _elem1675 = iprot.readString();
-                    struct.success.add(_elem1675);
+                    _elem1683 = iprot.readString();
+                    struct.success.add(_elem1683);
                   }
                   iprot.readListEnd();
                 }
@@ -96088,9 +96088,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (java.lang.String _iter1677 : struct.success)
+            for (java.lang.String _iter1685 : struct.success)
             {
-              oprot.writeString(_iter1677);
+              oprot.writeString(_iter1685);
             }
             oprot.writeListEnd();
           }
@@ -96129,9 +96129,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (java.lang.String _iter1678 : struct.success)
+            for (java.lang.String _iter1686 : struct.success)
             {
-              oprot.writeString(_iter1678);
+              oprot.writeString(_iter1686);
             }
           }
         }
@@ -96146,13 +96146,13 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1679 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.success = new java.util.ArrayList<java.lang.String>(_list1679.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem1680;
-            for (int _i1681 = 0; _i1681 < _list1679.size; ++_i1681)
+            org.apache.thrift.protocol.TList _list1687 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.success = new java.util.ArrayList<java.lang.String>(_list1687.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem1688;
+            for (int _i1689 = 0; _i1689 < _list1687.size; ++_i1689)
             {
-              _elem1680 = iprot.readString();
-              struct.success.add(_elem1680);
+              _elem1688 = iprot.readString();
+              struct.success.add(_elem1688);
             }
           }
           struct.setSuccessIsSet(true);
@@ -96662,13 +96662,13 @@ package org.apache.hadoop.hive.metastore.api;
             case 3: // TBL_TYPES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1682 = iprot.readListBegin();
-                  struct.tbl_types = new java.util.ArrayList<java.lang.String>(_list1682.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1683;
-                  for (int _i1684 = 0; _i1684 < _list1682.size; ++_i1684)
+                  org.apache.thrift.protocol.TList _list1690 = iprot.readListBegin();
+                  struct.tbl_types = new java.util.ArrayList<java.lang.String>(_list1690.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1691;
+                  for (int _i1692 = 0; _i1692 < _list1690.size; ++_i1692)
                   {
-                    _elem1683 = iprot.readString();
-                    struct.tbl_types.add(_elem1683);
+                    _elem1691 = iprot.readString();
+                    struct.tbl_types.add(_elem1691);
                   }
                   iprot.readListEnd();
                 }
@@ -96704,9 +96704,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(TBL_TYPES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_types.size()));
-            for (java.lang.String _iter1685 : struct.tbl_types)
+            for (java.lang.String _iter1693 : struct.tbl_types)
             {
-              oprot.writeString(_iter1685);
+              oprot.writeString(_iter1693);
             }
             oprot.writeListEnd();
           }
@@ -96749,9 +96749,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetTbl_types()) {
           {
             oprot.writeI32(struct.tbl_types.size());
-            for (java.lang.String _iter1686 : struct.tbl_types)
+            for (java.lang.String _iter1694 : struct.tbl_types)
             {
-              oprot.writeString(_iter1686);
+              oprot.writeString(_iter1694);
             }
           }
         }
@@ -96771,13 +96771,13 @@ package org.apache.hadoop.hive.metastore.api;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1687 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.tbl_types = new java.util.ArrayList<java.lang.String>(_list1687.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem1688;
-            for (int _i1689 = 0; _i1689 < _list1687.size; ++_i1689)
+            org.apache.thrift.protocol.TList _list1695 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.tbl_types = new java.util.ArrayList<java.lang.String>(_list1695.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem1696;
+            for (int _i1697 = 0; _i1697 < _list1695.size; ++_i1697)
             {
-              _elem1688 = iprot.readString();
-              struct.tbl_types.add(_elem1688);
+              _elem1696 = iprot.readString();
+              struct.tbl_types.add(_elem1696);
             }
           }
           struct.setTbl_typesIsSet(true);
@@ -97188,14 +97188,14 @@ package org.apache.hadoop.hive.metastore.api;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1690 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<TableMeta>(_list1690.size);
-                  @org.apache.thrift.annotation.Nullable TableMeta _elem1691;
-                  for (int _i1692 = 0; _i1692 < _list1690.size; ++_i1692)
+                  org.apache.thrift.protocol.TList _list1698 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<TableMeta>(_list1698.size);
+                  @org.apache.thrift.annotation.Nullable TableMeta _elem1699;
+                  for (int _i1700 = 0; _i1700 < _list1698.size; ++_i1700)
                   {
-                    _elem1691 = new TableMeta();
-                    _elem1691.read(iprot);
-                    struct.success.add(_elem1691);
+                    _elem1699 = new TableMeta();
+                    _elem1699.read(iprot);
+                    struct.success.add(_elem1699);
                   }
                   iprot.readListEnd();
                 }
@@ -97230,9 +97230,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (TableMeta _iter1693 : struct.success)
+            for (TableMeta _iter1701 : struct.success)
             {
-              _iter1693.write(oprot);
+              _iter1701.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -97271,9 +97271,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (TableMeta _iter1694 : struct.success)
+            for (TableMeta _iter1702 : struct.success)
             {
-              _iter1694.write(oprot);
+              _iter1702.write(oprot);
             }
           }
         }
@@ -97288,14 +97288,14 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1695 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.success = new java.util.ArrayList<TableMeta>(_list1695.size);
-            @org.apache.thrift.annotation.Nullable TableMeta _elem1696;
-            for (int _i1697 = 0; _i1697 < _list1695.size; ++_i1697)
+            org.apache.thrift.protocol.TList _list1703 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.success = new java.util.ArrayList<TableMeta>(_list1703.size);
+            @org.apache.thrift.annotation.Nullable TableMeta _elem1704;
+            for (int _i1705 = 0; _i1705 < _list1703.size; ++_i1705)
             {
-              _elem1696 = new TableMeta();
-              _elem1696.read(iprot);
-              struct.success.add(_elem1696);
+              _elem1704 = new TableMeta();
+              _elem1704.read(iprot);
+              struct.success.add(_elem1704);
             }
           }
           struct.setSuccessIsSet(true);
@@ -98070,13 +98070,13 @@ package org.apache.hadoop.hive.metastore.api;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1698 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<java.lang.String>(_list1698.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1699;
-                  for (int _i1700 = 0; _i1700 < _list1698.size; ++_i1700)
+                  org.apache.thrift.protocol.TList _list1706 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<java.lang.String>(_list1706.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1707;
+                  for (int _i1708 = 0; _i1708 < _list1706.size; ++_i1708)
                   {
-                    _elem1699 = iprot.readString();
-                    struct.success.add(_elem1699);
+                    _elem1707 = iprot.readString();
+                    struct.success.add(_elem1707);
                   }
                   iprot.readListEnd();
                 }
@@ -98111,9 +98111,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (java.lang.String _iter1701 : struct.success)
+            for (java.lang.String _iter1709 : struct.success)
             {
-              oprot.writeString(_iter1701);
+              oprot.writeString(_iter1709);
             }
             oprot.writeListEnd();
           }
@@ -98152,9 +98152,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (java.lang.String _iter1702 : struct.success)
+            for (java.lang.String _iter1710 : struct.success)
             {
-              oprot.writeString(_iter1702);
+              oprot.writeString(_iter1710);
             }
           }
         }
@@ -98169,13 +98169,13 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1703 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.success = new java.util.ArrayList<java.lang.String>(_list1703.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem1704;
-            for (int _i1705 = 0; _i1705 < _list1703.size; ++_i1705)
+            org.apache.thrift.protocol.TList _list1711 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.success = new java.util.ArrayList<java.lang.String>(_list1711.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem1712;
+            for (int _i1713 = 0; _i1713 < _list1711.size; ++_i1713)
             {
-              _elem1704 = iprot.readString();
-              struct.success.add(_elem1704);
+              _elem1712 = iprot.readString();
+              struct.success.add(_elem1712);
             }
           }
           struct.setSuccessIsSet(true);
@@ -99641,13 +99641,13 @@ package org.apache.hadoop.hive.metastore.api;
             case 2: // TBL_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1706 = iprot.readListBegin();
-                  struct.tbl_names = new java.util.ArrayList<java.lang.String>(_list1706.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1707;
-                  for (int _i1708 = 0; _i1708 < _list1706.size; ++_i1708)
+                  org.apache.thrift.protocol.TList _list1714 = iprot.readListBegin();
+                  struct.tbl_names = new java.util.ArrayList<java.lang.String>(_list1714.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1715;
+                  for (int _i1716 = 0; _i1716 < _list1714.size; ++_i1716)
                   {
-                    _elem1707 = iprot.readString();
-                    struct.tbl_names.add(_elem1707);
+                    _elem1715 = iprot.readString();
+                    struct.tbl_names.add(_elem1715);
                   }
                   iprot.readListEnd();
                 }
@@ -99678,9 +99678,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_names.size()));
-            for (java.lang.String _iter1709 : struct.tbl_names)
+            for (java.lang.String _iter1717 : struct.tbl_names)
             {
-              oprot.writeString(_iter1709);
+              oprot.writeString(_iter1717);
             }
             oprot.writeListEnd();
           }
@@ -99717,9 +99717,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetTbl_names()) {
           {
             oprot.writeI32(struct.tbl_names.size());
-            for (java.lang.String _iter1710 : struct.tbl_names)
+            for (java.lang.String _iter1718 : struct.tbl_names)
             {
-              oprot.writeString(_iter1710);
+              oprot.writeString(_iter1718);
             }
           }
         }
@@ -99735,13 +99735,13 @@ package org.apache.hadoop.hive.metastore.api;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list1711 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.tbl_names = new java.util.ArrayList<java.lang.String>(_list1711.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem1712;
-            for (int _i1713 = 0; _i1713 < _list1711.size; ++_i1713)
+            org.apache.thrift.protocol.TList _list1719 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.tbl_names = new java.util.ArrayList<java.lang.String>(_list1719.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem1720;
+            for (int _i1721 = 0; _i1721 < _list1719.size; ++_i1721)
             {
-              _elem1712 = iprot.readString();
-              struct.tbl_names.add(_elem1712);
+              _elem1720 = iprot.readString();
+              struct.tbl_names.add(_elem1720);
             }
           }
           struct.setTbl_namesIsSet(true);
@@ -100071,14 +100071,14 @@ package org.apache.hadoop.hive.metastore.api;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1714 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<Table>(_list1714.size);
-                  @org.apache.thrift.annotation.Nullable Table _elem1715;
-                  for (int _i1716 = 0; _i1716 < _list1714.size; ++_i1716)
+                  org.apache.thrift.protocol.TList _list1722 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<Table>(_list1722.size);
+                  @org.apache.thrift.annotation.Nullable Table _elem1723;
+                  for (int _i1724 = 0; _i1724 < _list1722.size; ++_i1724)
                   {
-                    _elem1715 = new Table();
-                    _elem1715.read(iprot);
-                    struct.success.add(_elem1715);
+                    _elem1723 = new Table();
+                    _elem1723.read(iprot);
+                    struct.success.add(_elem1723);
                   }
                   iprot.readListEnd();
                 }
@@ -100104,9 +100104,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Table _iter1717 : struct.success)
+            for (Table _iter1725 : struct.success)
             {
-              _iter1717.write(oprot);
+              _iter1725.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -100137,9 +100137,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Table _iter1718 : struct.success)
+            for (Table _iter1726 : struct.success)
             {
-              _iter1718.write(oprot);
+              _iter1726.write(oprot);
             }
           }
         }
@@ -100151,14 +100151,14 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1719 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.success = new java.util.ArrayList<Table>(_list1719.size);
-            @org.apache.thrift.annotation.Nullable Table _elem1720;
-            for (int _i1721 = 0; _i1721 < _list1719.size; ++_i1721)
+            org.apache.thrift.protocol.TList _list1727 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.success = new java.util.ArrayList<Table>(_list1727.size);
+            @org.apache.thrift.annotation.Nullable Table _elem1728;
+            for (int _i1729 = 0; _i1729 < _list1727.size; ++_i1729)
             {
-              _elem1720 = new Table();
-              _elem1720.read(iprot);
-              struct.success.add(_elem1720);
+              _elem1728 = new Table();
+              _elem1728.read(iprot);
+              struct.success.add(_elem1728);
             }
           }
           struct.setSuccessIsSet(true);
@@ -100936,14 +100936,14 @@ package org.apache.hadoop.hive.metastore.api;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1722 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<ExtendedTableInfo>(_list1722.size);
-                  @org.apache.thrift.annotation.Nullable ExtendedTableInfo _elem1723;
-                  for (int _i1724 = 0; _i1724 < _list1722.size; ++_i1724)
+                  org.apache.thrift.protocol.TList _list1730 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<ExtendedTableInfo>(_list1730.size);
+                  @org.apache.thrift.annotation.Nullable ExtendedTableInfo _elem1731;
+                  for (int _i1732 = 0; _i1732 < _list1730.size; ++_i1732)
                   {
-                    _elem1723 = new ExtendedTableInfo();
-                    _elem1723.read(iprot);
-                    struct.success.add(_elem1723);
+                    _elem1731 = new ExtendedTableInfo();
+                    _elem1731.read(iprot);
+                    struct.success.add(_elem1731);
                   }
                   iprot.readListEnd();
                 }
@@ -100978,9 +100978,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (ExtendedTableInfo _iter1725 : struct.success)
+            for (ExtendedTableInfo _iter1733 : struct.success)
             {
-              _iter1725.write(oprot);
+              _iter1733.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -101019,9 +101019,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (ExtendedTableInfo _iter1726 : struct.success)
+            for (ExtendedTableInfo _iter1734 : struct.success)
             {
-              _iter1726.write(oprot);
+              _iter1734.write(oprot);
             }
           }
         }
@@ -101036,14 +101036,14 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1727 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.success = new java.util.ArrayList<ExtendedTableInfo>(_list1727.size);
-            @org.apache.thrift.annotation.Nullable ExtendedTableInfo _elem1728;
-            for (int _i1729 = 0; _i1729 < _list1727.size; ++_i1729)
+            org.apache.thrift.protocol.TList _list1735 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.success = new java.util.ArrayList<ExtendedTableInfo>(_list1735.size);
+            @org.apache.thrift.annotation.Nullable ExtendedTableInfo _elem1736;
+            for (int _i1737 = 0; _i1737 < _list1735.size; ++_i1737)
             {
-              _elem1728 = new ExtendedTableInfo();
-              _elem1728.read(iprot);
-              struct.success.add(_elem1728);
+              _elem1736 = new ExtendedTableInfo();
+              _elem1736.read(iprot);
+              struct.success.add(_elem1736);
             }
           }
           struct.setSuccessIsSet(true);
@@ -106594,13 +106594,13 @@ package org.apache.hadoop.hive.metastore.api;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1730 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<java.lang.String>(_list1730.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1731;
-                  for (int _i1732 = 0; _i1732 < _list1730.size; ++_i1732)
+                  org.apache.thrift.protocol.TList _list1738 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<java.lang.String>(_list1738.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1739;
+                  for (int _i1740 = 0; _i1740 < _list1738.size; ++_i1740)
                   {
-                    _elem1731 = iprot.readString();
-                    struct.success.add(_elem1731);
+                    _elem1739 = iprot.readString();
+                    struct.success.add(_elem1739);
                   }
                   iprot.readListEnd();
                 }
@@ -106653,9 +106653,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (java.lang.String _iter1733 : struct.success)
+            for (java.lang.String _iter1741 : struct.success)
             {
-              oprot.writeString(_iter1733);
+              oprot.writeString(_iter1741);
             }
             oprot.writeListEnd();
           }
@@ -106710,9 +106710,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (java.lang.String _iter1734 : struct.success)
+            for (java.lang.String _iter1742 : struct.success)
             {
-              oprot.writeString(_iter1734);
+              oprot.writeString(_iter1742);
             }
           }
         }
@@ -106733,13 +106733,13 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1735 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.success = new java.util.ArrayList<java.lang.String>(_list1735.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem1736;
-            for (int _i1737 = 0; _i1737 < _list1735.size; ++_i1737)
+            org.apache.thrift.protocol.TList _list1743 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.success = new java.util.ArrayList<java.lang.String>(_list1743.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem1744;
+            for (int _i1745 = 0; _i1745 < _list1743.size; ++_i1745)
             {
-              _elem1736 = iprot.readString();
-              struct.success.add(_elem1736);
+              _elem1744 = iprot.readString();
+              struct.success.add(_elem1744);
             }
           }
           struct.setSuccessIsSet(true);
@@ -113586,14 +113586,14 @@ package org.apache.hadoop.hive.metastore.api;
             case 1: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1738 = iprot.readListBegin();
-                  struct.new_parts = new java.util.ArrayList<Partition>(_list1738.size);
-                  @org.apache.thrift.annotation.Nullable Partition _elem1739;
-                  for (int _i1740 = 0; _i1740 < _list1738.size; ++_i1740)
+                  org.apache.thrift.protocol.TList _list1746 = iprot.readListBegin();
+                  struct.new_parts = new java.util.ArrayList<Partition>(_list1746.size);
+                  @org.apache.thrift.annotation.Nullable Partition _elem1747;
+                  for (int _i1748 = 0; _i1748 < _list1746.size; ++_i1748)
                   {
-                    _elem1739 = new Partition();
-                    _elem1739.read(iprot);
-                    struct.new_parts.add(_elem1739);
+                    _elem1747 = new Partition();
+                    _elem1747.read(iprot);
+                    struct.new_parts.add(_elem1747);
                   }
                   iprot.readListEnd();
                 }
@@ -113619,9 +113619,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size()));
-            for (Partition _iter1741 : struct.new_parts)
+            for (Partition _iter1749 : struct.new_parts)
             {
-              _iter1741.write(oprot);
+              _iter1749.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -113652,9 +113652,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (Partition _iter1742 : struct.new_parts)
+            for (Partition _iter1750 : struct.new_parts)
             {
-              _iter1742.write(oprot);
+              _iter1750.write(oprot);
             }
           }
         }
@@ -113666,14 +113666,14 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1743 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.new_parts = new java.util.ArrayList<Partition>(_list1743.size);
-            @org.apache.thrift.annotation.Nullable Partition _elem1744;
-            for (int _i1745 = 0; _i1745 < _list1743.size; ++_i1745)
+            org.apache.thrift.protocol.TList _list1751 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.new_parts = new java.util.ArrayList<Partition>(_list1751.size);
+            @org.apache.thrift.annotation.Nullable Partition _elem1752;
+            for (int _i1753 = 0; _i1753 < _list1751.size; ++_i1753)
             {
-              _elem1744 = new Partition();
-              _elem1744.read(iprot);
-              struct.new_parts.add(_elem1744);
+              _elem1752 = new Partition();
+              _elem1752.read(iprot);
+              struct.new_parts.add(_elem1752);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -114680,14 +114680,14 @@ package org.apache.hadoop.hive.metastore.api;
             case 1: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1746 = iprot.readListBegin();
-                  struct.new_parts = new java.util.ArrayList<PartitionSpec>(_list1746.size);
-                  @org.apache.thrift.annotation.Nullable PartitionSpec _elem1747;
-                  for (int _i1748 = 0; _i1748 < _list1746.size; ++_i1748)
+                  org.apache.thrift.protocol.TList _list1754 = iprot.readListBegin();
+                  struct.new_parts = new java.util.ArrayList<PartitionSpec>(_list1754.size);
+                  @org.apache.thrift.annotation.Nullable PartitionSpec _elem1755;
+                  for (int _i1756 = 0; _i1756 < _list1754.size; ++_i1756)
                   {
-                    _elem1747 = new PartitionSpec();
-                    _elem1747.read(iprot);
-                    struct.new_parts.add(_elem1747);
+                    _elem1755 = new PartitionSpec();
+                    _elem1755.read(iprot);
+                    struct.new_parts.add(_elem1755);
                   }
                   iprot.readListEnd();
                 }
@@ -114713,9 +114713,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size()));
-            for (PartitionSpec _iter1749 : struct.new_parts)
+            for (PartitionSpec _iter1757 : struct.new_parts)
             {
-              _iter1749.write(oprot);
+              _iter1757.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -114746,9 +114746,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (PartitionSpec _iter1750 : struct.new_parts)
+            for (PartitionSpec _iter1758 : struct.new_parts)
             {
-              _iter1750.write(oprot);
+              _iter1758.write(oprot);
             }
           }
         }
@@ -114760,14 +114760,14 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1751 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.new_parts = new java.util.ArrayList<PartitionSpec>(_list1751.size);
-            @org.apache.thrift.annotation.Nullable PartitionSpec _elem1752;
-            for (int _i1753 = 0; _i1753 < _list1751.size; ++_i1753)
+            org.apache.thrift.protocol.TList _list1759 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.new_parts = new java.util.ArrayList<PartitionSpec>(_list1759.size);
+            @org.apache.thrift.annotation.Nullable PartitionSpec _elem1760;
+            for (int _i1761 = 0; _i1761 < _list1759.size; ++_i1761)
             {
-              _elem1752 = new PartitionSpec();
-              _elem1752.read(iprot);
-              struct.new_parts.add(_elem1752);
+              _elem1760 = new PartitionSpec();
+              _elem1760.read(iprot);
+              struct.new_parts.add(_elem1760);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -115949,13 +115949,13 @@ package org.apache.hadoop.hive.metastore.api;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1754 = iprot.readListBegin();
-                  struct.part_vals = new java.util.ArrayList<java.lang.String>(_list1754.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1755;
-                  for (int _i1756 = 0; _i1756 < _list1754.size; ++_i1756)
+                  org.apache.thrift.protocol.TList _list1762 = iprot.readListBegin();
+                  struct.part_vals = new java.util.ArrayList<java.lang.String>(_list1762.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1763;
+                  for (int _i1764 = 0; _i1764 < _list1762.size; ++_i1764)
                   {
-                    _elem1755 = iprot.readString();
-                    struct.part_vals.add(_elem1755);
+                    _elem1763 = iprot.readString();
+                    struct.part_vals.add(_elem1763);
                   }
                   iprot.readListEnd();
                 }
@@ -115991,9 +115991,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (java.lang.String _iter1757 : struct.part_vals)
+            for (java.lang.String _iter1765 : struct.part_vals)
             {
-              oprot.writeString(_iter1757);
+              oprot.writeString(_iter1765);
             }
             oprot.writeListEnd();
           }
@@ -116036,9 +116036,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (java.lang.String _iter1758 : struct.part_vals)
+            for (java.lang.String _iter1766 : struct.part_vals)
             {
-              oprot.writeString(_iter1758);
+              oprot.writeString(_iter1766);
             }
           }
         }
@@ -116058,13 +116058,13 @@ package org.apache.hadoop.hive.metastore.api;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1759 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.part_vals = new java.util.ArrayList<java.lang.String>(_list1759.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem1760;
-            for (int _i1761 = 0; _i1761 < _list1759.size; ++_i1761)
+            org.apache.thrift.protocol.TList _list1767 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.part_vals = new java.util.ArrayList<java.lang.String>(_list1767.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem1768;
+            for (int _i1769 = 0; _i1769 < _list1767.size; ++_i1769)
             {
-              _elem1760 = iprot.readString();
-              struct.part_vals.add(_elem1760);
+              _elem1768 = iprot.readString();
+              struct.part_vals.add(_elem1768);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -118390,13 +118390,13 @@ package org.apache.hadoop.hive.metastore.api;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1762 = iprot.readListBegin();
-                  struct.part_vals = new java.util.ArrayList<java.lang.String>(_list1762.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1763;
-                  for (int _i1764 = 0; _i1764 < _list1762.size; ++_i1764)
+                  org.apache.thrift.protocol.TList _list1770 = iprot.readListBegin();
+                  struct.part_vals = new java.util.ArrayList<java.lang.String>(_list1770.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1771;
+                  for (int _i1772 = 0; _i1772 < _list1770.size; ++_i1772)
                   {
-                    _elem1763 = iprot.readString();
-                    struct.part_vals.add(_elem1763);
+                    _elem1771 = iprot.readString();
+                    struct.part_vals.add(_elem1771);
                   }
                   iprot.readListEnd();
                 }
@@ -118441,9 +118441,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (java.lang.String _iter1765 : struct.part_vals)
+            for (java.lang.String _iter1773 : struct.part_vals)
             {
-              oprot.writeString(_iter1765);
+              oprot.writeString(_iter1773);
             }
             oprot.writeListEnd();
           }
@@ -118494,9 +118494,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (java.lang.String _iter1766 : struct.part_vals)
+            for (java.lang.String _iter1774 : struct.part_vals)
             {
-              oprot.writeString(_iter1766);
+              oprot.writeString(_iter1774);
             }
           }
         }
@@ -118519,13 +118519,13 @@ package org.apache.hadoop.hive.metastore.api;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1767 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.part_vals = new java.util.ArrayList<java.lang.String>(_list1767.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem1768;
-            for (int _i1769 = 0; _i1769 < _list1767.size; ++_i1769)
+            org.apache.thrift.protocol.TList _list1775 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.part_vals = new java.util.ArrayList<java.lang.String>(_list1775.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem1776;
+            for (int _i1777 = 0; _i1777 < _list1775.size; ++_i1777)
             {
-              _elem1768 = iprot.readString();
-              struct.part_vals.add(_elem1768);
+              _elem1776 = iprot.readString();
+              struct.part_vals.add(_elem1776);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -122417,13 +122417,13 @@ package org.apache.hadoop.hive.metastore.api;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1770 = iprot.readListBegin();
-                  struct.part_vals = new java.util.ArrayList<java.lang.String>(_list1770.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1771;
-                  for (int _i1772 = 0; _i1772 < _list1770.size; ++_i1772)
+                  org.apache.thrift.protocol.TList _list1778 = iprot.readListBegin();
+                  struct.part_vals = new java.util.ArrayList<java.lang.String>(_list1778.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1779;
+                  for (int _i1780 = 0; _i1780 < _list1778.size; ++_i1780)
                   {
-                    _elem1771 = iprot.readString();
-                    struct.part_vals.add(_elem1771);
+                    _elem1779 = iprot.readString();
+                    struct.part_vals.add(_elem1779);
                   }
                   iprot.readListEnd();
                 }
@@ -122467,9 +122467,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (java.lang.String _iter1773 : struct.part_vals)
+            for (java.lang.String _iter1781 : struct.part_vals)
             {
-              oprot.writeString(_iter1773);
+              oprot.writeString(_iter1781);
             }
             oprot.writeListEnd();
           }
@@ -122518,9 +122518,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (java.lang.String _iter1774 : struct.part_vals)
+            for (java.lang.String _iter1782 : struct.part_vals)
             {
-              oprot.writeString(_iter1774);
+              oprot.writeString(_iter1782);
             }
           }
         }
@@ -122543,13 +122543,13 @@ package org.apache.hadoop.hive.metastore.api;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1775 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.part_vals = new java.util.ArrayList<java.lang.String>(_list1775.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem1776;
-            for (int _i1777 = 0; _i1777 < _list1775.size; ++_i1777)
+            org.apache.thrift.protocol.TList _list1783 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.part_vals = new java.util.ArrayList<java.lang.String>(_list1783.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem1784;
+            for (int _i1785 = 0; _i1785 < _list1783.size; ++_i1785)
             {
-              _elem1776 = iprot.readString();
-              struct.part_vals.add(_elem1776);
+              _elem1784 = iprot.readString();
+              struct.part_vals.add(_elem1784);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -123791,13 +123791,13 @@ package org.apache.hadoop.hive.metastore.api;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1778 = iprot.readListBegin();
-                  struct.part_vals = new java.util.ArrayList<java.lang.String>(_list1778.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1779;
-                  for (int _i1780 = 0; _i1780 < _list1778.size; ++_i1780)
+                  org.apache.thrift.protocol.TList _list1786 = iprot.readListBegin();
+                  struct.part_vals = new java.util.ArrayList<java.lang.String>(_list1786.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1787;
+                  for (int _i1788 = 0; _i1788 < _list1786.size; ++_i1788)
                   {
-                    _elem1779 = iprot.readString();
-                    struct.part_vals.add(_elem1779);
+                    _elem1787 = iprot.readString();
+                    struct.part_vals.add(_elem1787);
                   }
                   iprot.readListEnd();
                 }
@@ -123850,9 +123850,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (java.lang.String _iter1781 : struct.part_vals)
+            for (java.lang.String _iter1789 : struct.part_vals)
             {
-              oprot.writeString(_iter1781);
+              oprot.writeString(_iter1789);
             }
             oprot.writeListEnd();
           }
@@ -123909,9 +123909,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (java.lang.String _iter1782 : struct.part_vals)
+            for (java.lang.String _iter1790 : struct.part_vals)
             {
-              oprot.writeString(_iter1782);
+              oprot.writeString(_iter1790);
             }
           }
         }
@@ -123937,13 +123937,13 @@ package org.apache.hadoop.hive.metastore.api;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1783 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.part_vals = new java.util.ArrayList<java.lang.String>(_list1783.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem1784;
-            for (int _i1785 = 0; _i1785 < _list1783.size; ++_i1785)
+            org.apache.thrift.protocol.TList _list1791 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.part_vals = new java.util.ArrayList<java.lang.String>(_list1791.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem1792;
+            for (int _i1793 = 0; _i1793 < _list1791.size; ++_i1793)
             {
-              _elem1784 = iprot.readString();
-              struct.part_vals.add(_elem1784);
+              _elem1792 = iprot.readString();
+              struct.part_vals.add(_elem1792);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -128563,13 +128563,13 @@ package org.apache.hadoop.hive.metastore.api;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1786 = iprot.readListBegin();
-                  struct.part_vals = new java.util.ArrayList<java.lang.String>(_list1786.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1787;
-                  for (int _i1788 = 0; _i1788 < _list1786.size; ++_i1788)
+                  org.apache.thrift.protocol.TList _list1794 = iprot.readListBegin();
+                  struct.part_vals = new java.util.ArrayList<java.lang.String>(_list1794.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1795;
+                  for (int _i1796 = 0; _i1796 < _list1794.size; ++_i1796)
                   {
-                    _elem1787 = iprot.readString();
-                    struct.part_vals.add(_elem1787);
+                    _elem1795 = iprot.readString();
+                    struct.part_vals.add(_elem1795);
                   }
                   iprot.readListEnd();
                 }
@@ -128605,9 +128605,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (java.lang.String _iter1789 : struct.part_vals)
+            for (java.lang.String _iter1797 : struct.part_vals)
             {
-              oprot.writeString(_iter1789);
+              oprot.writeString(_iter1797);
             }
             oprot.writeListEnd();
           }
@@ -128650,9 +128650,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (java.lang.String _iter1790 : struct.part_vals)
+            for (java.lang.String _iter1798 : struct.part_vals)
             {
-              oprot.writeString(_iter1790);
+              oprot.writeString(_iter1798);
             }
           }
         }
@@ -128672,13 +128672,13 @@ package org.apache.hadoop.hive.metastore.api;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1791 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.part_vals = new java.util.ArrayList<java.lang.String>(_list1791.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem1792;
-            for (int _i1793 = 0; _i1793 < _list1791.size; ++_i1793)
+            org.apache.thrift.protocol.TList _list1799 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.part_vals = new java.util.ArrayList<java.lang.String>(_list1799.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem1800;
+            for (int _i1801 = 0; _i1801 < _list1799.size; ++_i1801)
             {
-              _elem1792 = iprot.readString();
-              struct.part_vals.add(_elem1792);
+              _elem1800 = iprot.readString();
+              struct.part_vals.add(_elem1800);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -130850,15 +130850,15 @@ package org.apache.hadoop.hive.metastore.api;
             case 1: // PARTITION_SPECS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map1794 = iprot.readMapBegin();
-                  struct.partitionSpecs = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map1794.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _key1795;
-                  @org.apache.thrift.annotation.Nullable java.lang.String _val1796;
-                  for (int _i1797 = 0; _i1797 < _map1794.size; ++_i1797)
+                  org.apache.thrift.protocol.TMap _map1802 = iprot.readMapBegin();
+                  struct.partitionSpecs = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map1802.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _key1803;
+                  @org.apache.thrift.annotation.Nullable java.lang.String _val1804;
+                  for (int _i1805 = 0; _i1805 < _map1802.size; ++_i1805)
                   {
-                    _key1795 = iprot.readString();
-                    _val1796 = iprot.readString();
-                    struct.partitionSpecs.put(_key1795, _val1796);
+                    _key1803 = iprot.readString();
+                    _val1804 = iprot.readString();
+                    struct.partitionSpecs.put(_key1803, _val1804);
                   }
                   iprot.readMapEnd();
                 }
@@ -130916,10 +130916,10 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(PARTITION_SPECS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.partitionSpecs.size()));
-            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter1798 : struct.partitionSpecs.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter1806 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter1798.getKey());
-              oprot.writeString(_iter1798.getValue());
+              oprot.writeString(_iter1806.getKey());
+              oprot.writeString(_iter1806.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -130982,10 +130982,10 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetPartitionSpecs()) {
           {
             oprot.writeI32(struct.partitionSpecs.size());
-            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter1799 : struct.partitionSpecs.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter1807 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter1799.getKey());
-              oprot.writeString(_iter1799.getValue());
+              oprot.writeString(_iter1807.getKey());
+              oprot.writeString(_iter1807.getValue());
             }
           }
         }
@@ -131009,15 +131009,15 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map1800 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
-            struct.partitionSpecs = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map1800.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _key1801;
-            @org.apache.thrift.annotation.Nullable java.lang.String _val1802;
-            for (int _i1803 = 0; _i1803 < _map1800.size; ++_i1803)
+            org.apache.thrift.protocol.TMap _map1808 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
+            struct.partitionSpecs = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map1808.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _key1809;
+            @org.apache.thrift.annotation.Nullable java.lang.String _val1810;
+            for (int _i1811 = 0; _i1811 < _map1808.size; ++_i1811)
             {
-              _key1801 = iprot.readString();
-              _val1802 = iprot.readString();
-              struct.partitionSpecs.put(_key1801, _val1802);
+              _key1809 = iprot.readString();
+              _val1810 = iprot.readString();
+              struct.partitionSpecs.put(_key1809, _val1810);
             }
           }
           struct.setPartitionSpecsIsSet(true);
@@ -132471,15 +132471,15 @@ package org.apache.hadoop.hive.metastore.api;
             case 1: // PARTITION_SPECS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map1804 = iprot.readMapBegin();
-                  struct.partitionSpecs = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map1804.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _key1805;
-                  @org.apache.thrift.annotation.Nullable java.lang.String _val1806;
-                  for (int _i1807 = 0; _i1807 < _map1804.size; ++_i1807)
+                  org.apache.thrift.protocol.TMap _map1812 = iprot.readMapBegin();
+                  struct.partitionSpecs = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map1812.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _key1813;
+                  @org.apache.thrift.annotation.Nullable java.lang.String _val1814;
+                  for (int _i1815 = 0; _i1815 < _map1812.size; ++_i1815)
                   {
-                    _key1805 = iprot.readString();
-                    _val1806 = iprot.readString();
-                    struct.partitionSpecs.put(_key1805, _val1806);
+                    _key1813 = iprot.readString();
+                    _val1814 = iprot.readString();
+                    struct.partitionSpecs.put(_key1813, _val1814);
                   }
                   iprot.readMapEnd();
                 }
@@ -132537,10 +132537,10 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(PARTITION_SPECS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.partitionSpecs.size()));
-            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter1808 : struct.partitionSpecs.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter1816 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter1808.getKey());
-              oprot.writeString(_iter1808.getValue());
+              oprot.writeString(_iter1816.getKey());
+              oprot.writeString(_iter1816.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -132603,10 +132603,10 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetPartitionSpecs()) {
           {
             oprot.writeI32(struct.partitionSpecs.size());
-            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter1809 : struct.partitionSpecs.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter1817 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter1809.getKey());
-              oprot.writeString(_iter1809.getValue());
+              oprot.writeString(_iter1817.getKey());
+              oprot.writeString(_iter1817.getValue());
             }
           }
         }
@@ -132630,15 +132630,15 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map1810 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
-            struct.partitionSpecs = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map1810.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _key1811;
-            @org.apache.thrift.annotation.Nullable java.lang.String _val1812;
-            for (int _i1813 = 0; _i1813 < _map1810.size; ++_i1813)
+            org.apache.thrift.protocol.TMap _map1818 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
+            struct.partitionSpecs = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map1818.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _key1819;
+            @org.apache.thrift.annotation.Nullable java.lang.String _val1820;
+            for (int _i1821 = 0; _i1821 < _map1818.size; ++_i1821)
             {
-              _key1811 = iprot.readString();
-              _val1812 = iprot.readString();
-              struct.partitionSpecs.put(_key1811, _val1812);
+              _key1819 = iprot.readString();
+              _val1820 = iprot.readString();
+              struct.partitionSpecs.put(_key1819, _val1820);
             }
           }
           struct.setPartitionSpecsIsSet(true);
@@ -133308,14 +133308,14 @@ package org.apache.hadoop.hive.metastore.api;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1814 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<Partition>(_list1814.size);
-                  @org.apache.thrift.annotation.Nullable Partition _elem1815;
-                  for (int _i1816 = 0; _i1816 < _list1814.size; ++_i1816)
+                  org.apache.thrift.protocol.TList _list1822 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<Partition>(_list1822.size);
+                  @org.apache.thrift.annotation.Nullable Partition _elem1823;
+                  for (int _i1824 = 0; _i1824 < _list1822.size; ++_i1824)
                   {
-                    _elem1815 = new Partition();
-                    _elem1815.read(iprot);
-                    struct.success.add(_elem1815);
+                    _elem1823 = new Partition();
+                    _elem1823.read(iprot);
+                    struct.success.add(_elem1823);
                   }
                   iprot.readListEnd();
                 }
@@ -133377,9 +133377,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Partition _iter1817 : struct.success)
+            for (Partition _iter1825 : struct.success)
             {
-              _iter1817.write(oprot);
+              _iter1825.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -133442,9 +133442,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Partition _iter1818 : struct.success)
+            for (Partition _iter1826 : struct.success)
             {
-              _iter1818.write(oprot);
+              _iter1826.write(oprot);
             }
           }
         }
@@ -133468,14 +133468,14 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1819 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.success = new java.util.ArrayList<Partition>(_list1819.size);
-            @org.apache.thrift.annotation.Nullable Partition _elem1820;
-            for (int _i1821 = 0; _i1821 < _list1819.size; ++_i1821)
+            org.apache.thrift.protocol.TList _list1827 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.success = new java.util.ArrayList<Partition>(_list1827.size);
+            @org.apache.thrift.annotation.Nullable Partition _elem1828;
+            for (int _i1829 = 0; _i1829 < _list1827.size; ++_i1829)
             {
-              _elem1820 = new Partition();
-              _elem1820.read(iprot);
-              struct.success.add(_elem1820);
+              _elem1828 = new Partition();
+              _elem1828.read(iprot);
+              struct.success.add(_elem1828);
             }
           }
           struct.setSuccessIsSet(true);
@@ -134180,13 +134180,13 @@ package org.apache.hadoop.hive.metastore.api;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1822 = iprot.readListBegin();
-                  struct.part_vals = new java.util.ArrayList<java.lang.String>(_list1822.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1823;
-                  for (int _i1824 = 0; _i1824 < _list1822.size; ++_i1824)
+                  org.apache.thrift.protocol.TList _list1830 = iprot.readListBegin();
+                  struct.part_vals = new java.util.ArrayList<java.lang.String>(_list1830.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1831;
+                  for (int _i1832 = 0; _i1832 < _list1830.size; ++_i1832)
                   {
-                    _elem1823 = iprot.readString();
-                    struct.part_vals.add(_elem1823);
+                    _elem1831 = iprot.readString();
+                    struct.part_vals.add(_elem1831);
                   }
                   iprot.readListEnd();
                 }
@@ -134206,13 +134206,13 @@ package org.apache.hadoop.hive.metastore.api;
             case 5: // GROUP_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1825 = iprot.readListBegin();
-                  struct.group_names = new java.util.ArrayList<java.lang.String>(_list1825.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1826;
-                  for (int _i1827 = 0; _i1827 < _list1825.size; ++_i1827)
+                  org.apache.thrift.protocol.TList _list1833 = iprot.readListBegin();
+                  struct.group_names = new java.util.ArrayList<java.lang.String>(_list1833.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1834;
+                  for (int _i1835 = 0; _i1835 < _list1833.size; ++_i1835)
                   {
-                    _elem1826 = iprot.readString();
-                    struct.group_names.add(_elem1826);
+                    _elem1834 = iprot.readString();
+                    struct.group_names.add(_elem1834);
                   }
                   iprot.readListEnd();
                 }
@@ -134248,9 +134248,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (java.lang.String _iter1828 : struct.part_vals)
+            for (java.lang.String _iter1836 : struct.part_vals)
             {
-              oprot.writeString(_iter1828);
+              oprot.writeString(_iter1836);
             }
             oprot.writeListEnd();
           }
@@ -134265,9 +134265,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.group_names.size()));
-            for (java.lang.String _iter1829 : struct.group_names)
+            for (java.lang.String _iter1837 : struct.group_names)
             {
-              oprot.writeString(_iter1829);
+              oprot.writeString(_iter1837);
             }
             oprot.writeListEnd();
           }
@@ -134316,9 +134316,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (java.lang.String _iter1830 : struct.part_vals)
+            for (java.lang.String _iter1838 : struct.part_vals)
             {
-              oprot.writeString(_iter1830);
+              oprot.writeString(_iter1838);
             }
           }
         }
@@ -134328,9 +134328,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetGroup_names()) {
           {
             oprot.writeI32(struct.group_names.size());
-            for (java.lang.String _iter1831 : struct.group_names)
+            for (java.lang.String _iter1839 : struct.group_names)
             {
-              oprot.writeString(_iter1831);
+              oprot.writeString(_iter1839);
             }
           }
         }
@@ -134350,13 +134350,13 @@ package org.apache.hadoop.hive.metastore.api;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1832 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.part_vals = new java.util.ArrayList<java.lang.String>(_list1832.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem1833;
-            for (int _i1834 = 0; _i1834 < _list1832.size; ++_i1834)
+            org.apache.thrift.protocol.TList _list1840 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.part_vals = new java.util.ArrayList<java.lang.String>(_list1840.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem1841;
+            for (int _i1842 = 0; _i1842 < _list1840.size; ++_i1842)
             {
-              _elem1833 = iprot.readString();
-              struct.part_vals.add(_elem1833);
+              _elem1841 = iprot.readString();
+              struct.part_vals.add(_elem1841);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -134367,13 +134367,13 @@ package org.apache.hadoop.hive.metastore.api;
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol.TList _list1835 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.group_names = new java.util.ArrayList<java.lang.String>(_list1835.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem1836;
-            for (int _i1837 = 0; _i1837 < _list1835.size; ++_i1837)
+            org.apache.thrift.protocol.TList _list1843 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.group_names = new java.util.ArrayList<java.lang.String>(_list1843.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem1844;
+            for (int _i1845 = 0; _i1845 < _list1843.size; ++_i1845)
             {
-              _elem1836 = iprot.readString();
-              struct.group_names.add(_elem1836);
+              _elem1844 = iprot.readString();
+              struct.group_names.add(_elem1844);
             }
           }
           struct.setGroup_namesIsSet(true);
@@ -137160,14 +137160,14 @@ package org.apache.hadoop.hive.metastore.api;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1838 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<Partition>(_list1838.size);
-                  @org.apache.thrift.annotation.Nullable Partition _elem1839;
-                  for (int _i1840 = 0; _i1840 < _list1838.size; ++_i1840)
+                  org.apache.thrift.protocol.TList _list1846 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<Partition>(_list1846.size);
+                  @org.apache.thrift.annotation.Nullable Partition _elem1847;
+                  for (int _i1848 = 0; _i1848 < _list1846.size; ++_i1848)
                   {
-                    _elem1839 = new Partition();
-                    _elem1839.read(iprot);
-                    struct.success.add(_elem1839);
+                    _elem1847 = new Partition();
+                    _elem1847.read(iprot);
+                    struct.success.add(_elem1847);
                   }
                   iprot.readListEnd();
                 }
@@ -137211,9 +137211,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Partition _iter1841 : struct.success)
+            for (Partition _iter1849 : struct.success)
             {
-              _iter1841.write(oprot);
+              _iter1849.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -137260,9 +137260,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Partition _iter1842 : struct.success)
+            for (Partition _iter1850 : struct.success)
             {
-              _iter1842.write(oprot);
+              _iter1850.write(oprot);
             }
           }
         }
@@ -137280,14 +137280,14 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1843 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.success = new java.util.ArrayList<Partition>(_list1843.size);
-            @org.apache.thrift.annotation.Nullable Partition _elem1844;
-            for (int _i1845 = 0; _i1845 < _list1843.size; ++_i1845)
+            org.apache.thrift.protocol.TList _list1851 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.success = new java.util.ArrayList<Partition>(_list1851.size);
+            @org.apache.thrift.annotation.Nullable Partition _elem1852;
+            for (int _i1853 = 0; _i1853 < _list1851.size; ++_i1853)
             {
-              _elem1844 = new Partition();
-              _elem1844.read(iprot);
-              struct.success.add(_elem1844);
+              _elem1852 = new Partition();
+              _elem1852.read(iprot);
+              struct.success.add(_elem1852);
             }
           }
           struct.setSuccessIsSet(true);
@@ -138925,13 +138925,13 @@ package org.apache.hadoop.hive.metastore.api;
             case 5: // GROUP_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1846 = iprot.readListBegin();
-                  struct.group_names = new java.util.ArrayList<java.lang.String>(_list1846.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1847;
-                  for (int _i1848 = 0; _i1848 < _list1846.size; ++_i1848)
+                  org.apache.thrift.protocol.TList _list1854 = iprot.readListBegin();
+                  struct.group_names = new java.util.ArrayList<java.lang.String>(_list1854.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1855;
+                  for (int _i1856 = 0; _i1856 < _list1854.size; ++_i1856)
                   {
-                    _elem1847 = iprot.readString();
-                    struct.group_names.add(_elem1847);
+                    _elem1855 = iprot.readString();
+                    struct.group_names.add(_elem1855);
                   }
                   iprot.readListEnd();
                 }
@@ -138975,9 +138975,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.group_names.size()));
-            for (java.lang.String _iter1849 : struct.group_names)
+            for (java.lang.String _iter1857 : struct.group_names)
             {
-              oprot.writeString(_iter1849);
+              oprot.writeString(_iter1857);
             }
             oprot.writeListEnd();
           }
@@ -139032,9 +139032,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetGroup_names()) {
           {
             oprot.writeI32(struct.group_names.size());
-            for (java.lang.String _iter1850 : struct.group_names)
+            for (java.lang.String _iter1858 : struct.group_names)
             {
-              oprot.writeString(_iter1850);
+              oprot.writeString(_iter1858);
             }
           }
         }
@@ -139062,13 +139062,13 @@ package org.apache.hadoop.hive.metastore.api;
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol.TList _list1851 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.group_names = new java.util.ArrayList<java.lang.String>(_list1851.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem1852;
-            for (int _i1853 = 0; _i1853 < _list1851.size; ++_i1853)
+            org.apache.thrift.protocol.TList _list1859 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.group_names = new java.util.ArrayList<java.lang.String>(_list1859.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem1860;
+            for (int _i1861 = 0; _i1861 < _list1859.size; ++_i1861)
             {
-              _elem1852 = iprot.readString();
-              struct.group_names.add(_elem1852);
+              _elem1860 = iprot.readString();
+              struct.group_names.add(_elem1860);
             }
           }
           struct.setGroup_namesIsSet(true);
@@ -139560,14 +139560,14 @@ package org.apache.hadoop.hive.metastore.api;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1854 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<Partition>(_list1854.size);
-                  @org.apache.thrift.annotation.Nullable Partition _elem1855;
-                  for (int _i1856 = 0; _i1856 < _list1854.size; ++_i1856)
+                  org.apache.thrift.protocol.TList _list1862 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<Partition>(_list1862.size);
+                  @org.apache.thrift.annotation.Nullable Partition _elem1863;
+                  for (int _i1864 = 0; _i1864 < _list1862.size; ++_i1864)
                   {
-                    _elem1855 = new Partition();
-                    _elem1855.read(iprot);
-                    struct.success.add(_elem1855);
+                    _elem1863 = new Partition();
+                    _elem1863.read(iprot);
+                    struct.success.add(_elem1863);
                   }
                   iprot.readListEnd();
                 }
@@ -139611,9 +139611,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Partition _iter1857 : struct.success)
+            for (Partition _iter1865 : struct.success)
             {
-              _iter1857.write(oprot);
+              _iter1865.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -139660,9 +139660,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Partition _iter1858 : struct.success)
+            for (Partition _iter1866 : struct.success)
             {
-              _iter1858.write(oprot);
+              _iter1866.write(oprot);
             }
           }
         }
@@ -139680,14 +139680,14 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1859 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.success = new java.util.ArrayList<Partition>(_list1859.size);
-            @org.apache.thrift.annotation.Nullable Partition _elem1860;
-            for (int _i1861 = 0; _i1861 < _list1859.size; ++_i1861)
+            org.apache.thrift.protocol.TList _list1867 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.success = new java.util.ArrayList<Partition>(_list1867.size);
+            @org.apache.thrift.annotation.Nullable Partition _elem1868;
+            for (int _i1869 = 0; _i1869 < _list1867.size; ++_i1869)
             {
-              _elem1860 = new Partition();
-              _elem1860.read(iprot);
-              struct.success.add(_elem1860);
+              _elem1868 = new Partition();
+              _elem1868.read(iprot);
+              struct.success.add(_elem1868);
             }
           }
           struct.setSuccessIsSet(true);
@@ -140756,14 +140756,14 @@ package org.apache.hadoop.hive.metastore.api;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1862 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<PartitionSpec>(_list1862.size);
-                  @org.apache.thrift.annotation.Nullable PartitionSpec _elem1863;
-                  for (int _i1864 = 0; _i1864 < _list1862.size; ++_i1864)
+                  org.apache.thrift.protocol.TList _list1870 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<PartitionSpec>(_list1870.size);
+                  @org.apache.thrift.annotation.Nullable PartitionSpec _elem1871;
+                  for (int _i1872 = 0; _i1872 < _list1870.size; ++_i1872)
                   {
-                    _elem1863 = new PartitionSpec();
-                    _elem1863.read(iprot);
-                    struct.success.add(_elem1863);
+                    _elem1871 = new PartitionSpec();
+                    _elem1871.read(iprot);
+                    struct.success.add(_elem1871);
                   }
                   iprot.readListEnd();
                 }
@@ -140807,9 +140807,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (PartitionSpec _iter1865 : struct.success)
+            for (PartitionSpec _iter1873 : struct.success)
             {
-              _iter1865.write(oprot);
+              _iter1873.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -140856,9 +140856,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (PartitionSpec _iter1866 : struct.success)
+            for (PartitionSpec _iter1874 : struct.success)
             {
-              _iter1866.write(oprot);
+              _iter1874.write(oprot);
             }
           }
         }
@@ -140876,14 +140876,14 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1867 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.success = new java.util.ArrayList<PartitionSpec>(_list1867.size);
-            @org.apache.thrift.annotation.Nullable PartitionSpec _elem1868;
-            for (int _i1869 = 0; _i1869 < _list1867.size; ++_i1869)
+            org.apache.thrift.protocol.TList _list1875 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.success = new java.util.ArrayList<PartitionSpec>(_list1875.size);
+            @org.apache.thrift.annotation.Nullable PartitionSpec _elem1876;
+            for (int _i1877 = 0; _i1877 < _list1875.size; ++_i1877)
             {
-              _elem1868 = new PartitionSpec();
-              _elem1868.read(iprot);
-              struct.success.add(_elem1868);
+              _elem1876 = new PartitionSpec();
+              _elem1876.read(iprot);
+              struct.success.add(_elem1876);
             }
           }
           struct.setSuccessIsSet(true);
@@ -141949,13 +141949,13 @@ package org.apache.hadoop.hive.metastore.api;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1870 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<java.lang.String>(_list1870.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1871;
-                  for (int _i1872 = 0; _i1872 < _list1870.size; ++_i1872)
+                  org.apache.thrift.protocol.TList _list1878 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<java.lang.String>(_list1878.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem1879;
+                  for (int _i1880 = 0; _i1880 < _list1878.size; ++_i1880)
                   {
-                    _elem1871 = iprot.readString();
-                    struct.success.add(_elem1871);
+                    _elem1879 = iprot.readString();
+                    struct.success.add(_elem1879);
                   }
                   iprot.readListEnd();
                 }
@@ -141999,9 +141999,9 @@ package org.apache.hadoop.hive.metastore.api;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (java.lang.String _iter1873 : struct.success)
+            for (java.lang.String _iter1881 : struct.success)
             {
-              oprot.writeString(_iter1873);
+              oprot.writeString(_iter1881);
             }
             oprot.writeListEnd();
           }
@@ -142048,9 +142048,9 @@ package org.apache.hadoop.hive.metastore.api;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (java.lang.String _iter1874 : struct.success)
+            for (java.lang.String _iter1882 : struct.success)
             {
-              oprot.writeString(_iter1874);
+              oprot.writeString(_iter1882);
             }
           }
         }
@@ -142068,13 +142068,13 @@ package org.apache.hadoop.hive.metastore.api;
         java.util.BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1875 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.success = new java.util.ArrayList<java.lang.String>(_list1875.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem1876;
-            for (int _i1877 = 0; _i1877 < _list1875.size; ++_i1877)
+            org.apache.thrift.protocol.TList _list1883 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.success = new java.util.ArrayList<java.lang.String>(_list1883.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem1884;
+            for (int _i1885 = 0; _i1885 < _list1883.size; ++_i1885)
             {
-              _elem1876 = iprot.readString();
-              struct.success.add(_elem1876);
+              _elem1884 = iprot.readString();
+              struct.success.add(_elem1884);
             }
           }
           struct.setSuccessIsSet(true);
@@ -143615,13 +143615,13 @@ package org.apache.hadoop.hive.metastore.api;
             case 3: // PART_VALS
... 8081 lines suppressed ...