You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by kr...@apache.org on 2022/01/28 14:37:10 UTC

[hive] branch master updated: HIVE-25878: Unable to compile cpp metastore thrift client (Krisztian Kasa, reviewed by Zoltan Haindrich)

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

krisztiankasa 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 38c3079  HIVE-25878: Unable to compile cpp metastore thrift client (Krisztian Kasa, reviewed by Zoltan Haindrich)
38c3079 is described below

commit 38c30790937a6d6f132854532dd24483d46ab014
Author: Krisztian Kasa <ka...@gmail.com>
AuthorDate: Fri Jan 28 15:36:54 2022 +0100

    HIVE-25878: Unable to compile cpp metastore thrift client (Krisztian Kasa, reviewed by Zoltan Haindrich)
---
 .../hive/ql/metadata/MaterializedViewMetadata.java |   11 +-
 .../hadoop/hive/metastore/txn/TestTxnHandler.java  |    2 +-
 .../src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp | 2490 +++----
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp    | 7565 ++++++++++----------
 .../src/gen/thrift/gen-cpp/hive_metastore_types.h  |  110 +-
 .../hive/metastore/api/CreationMetadata.java       |   38 +-
 .../hadoop/hive/metastore/api/SourceTable.java     |    5 +-
 .../thrift/gen-php/metastore/AbortTxnsRequest.php  |   18 +-
 .../metastore/AddCheckConstraintRequest.php        |   20 +-
 .../metastore/AddDefaultConstraintRequest.php      |   20 +-
 .../gen-php/metastore/AddDynamicPartitions.php     |   18 +-
 .../gen-php/metastore/AddForeignKeyRequest.php     |   20 +-
 .../metastore/AddNotNullConstraintRequest.php      |   20 +-
 .../gen-php/metastore/AddPartitionsRequest.php     |   20 +-
 .../gen-php/metastore/AddPartitionsResult.php      |   20 +-
 .../gen-php/metastore/AddPrimaryKeyRequest.php     |   20 +-
 .../metastore/AddUniqueConstraintRequest.php       |   20 +-
 .../src/gen/thrift/gen-php/metastore/AggrStats.php |   20 +-
 .../metastore/AllocateTableWriteIdsRequest.php     |   38 +-
 .../metastore/AllocateTableWriteIdsResponse.php    |   20 +-
 .../gen-php/metastore/AlterPartitionsRequest.php   |   20 +-
 .../thrift/gen-php/metastore/AlterTableRequest.php |   18 +-
 .../gen-php/metastore/CheckConstraintsResponse.php |   20 +-
 .../gen-php/metastore/ClearFileMetadataRequest.php |   18 +-
 .../gen-php/metastore/ClientCapabilities.php       |   18 +-
 .../thrift/gen-php/metastore/ColumnStatistics.php  |   20 +-
 .../thrift/gen-php/metastore/CommitTxnRequest.php  |   20 +-
 .../thrift/gen-php/metastore/CompactionRequest.php |   26 +-
 .../gen-php/metastore/CreateDatabaseRequest.php    |   26 +-
 .../gen-php/metastore/CreateTableRequest.php       |  138 +-
 .../thrift/gen-php/metastore/CreationMetadata.php  |   20 +-
 .../gen/thrift/gen-php/metastore/DataConnector.php |   26 +-
 .../metastore/DefaultConstraintsResponse.php       |   20 +-
 .../gen-php/metastore/DropPartitionsResult.php     |   20 +-
 .../thrift/gen-php/metastore/ExtendedTableInfo.php |   36 +-
 .../gen/thrift/gen-php/metastore/FileMetadata.php  |   18 +-
 .../gen-php/metastore/FindSchemasByColsResp.php    |   20 +-
 .../thrift/gen-php/metastore/FireEventRequest.php  |   18 +-
 .../gen-php/metastore/FireEventRequestData.php     |   20 +-
 .../thrift/gen-php/metastore/FireEventResponse.php |   18 +-
 .../gen-php/metastore/ForeignKeysResponse.php      |   20 +-
 .../src/gen/thrift/gen-php/metastore/Function.php  |   20 +-
 .../gen-php/metastore/GetAllFunctionsResponse.php  |   20 +-
 .../gen-php/metastore/GetDatabaseRequest.php       |   18 +-
 .../thrift/gen-php/metastore/GetFieldsResponse.php |   20 +-
 .../metastore/GetFileMetadataByExprRequest.php     |   18 +-
 .../metastore/GetFileMetadataByExprResult.php      |   28 +-
 .../gen-php/metastore/GetFileMetadataRequest.php   |   18 +-
 .../gen-php/metastore/GetFileMetadataResult.php    |   26 +-
 .../GetLatestCommittedCompactionInfoRequest.php    |   18 +-
 .../GetLatestCommittedCompactionInfoResponse.php   |   20 +-
 .../gen-php/metastore/GetOpenTxnsInfoResponse.php  |   20 +-
 .../gen-php/metastore/GetOpenTxnsRequest.php       |   18 +-
 .../gen-php/metastore/GetOpenTxnsResponse.php      |   18 +-
 .../metastore/GetPartitionNamesPsRequest.php       |   18 +-
 .../metastore/GetPartitionNamesPsResponse.php      |   18 +-
 .../gen-php/metastore/GetPartitionRequest.php      |   18 +-
 .../metastore/GetPartitionsByNamesRequest.php      |   36 +-
 .../metastore/GetPartitionsByNamesResult.php       |   20 +-
 .../gen-php/metastore/GetPartitionsFilterSpec.php  |   18 +-
 .../metastore/GetPartitionsPsWithAuthRequest.php   |   36 +-
 .../metastore/GetPartitionsPsWithAuthResponse.php  |   20 +-
 .../gen-php/metastore/GetPartitionsRequest.php     |   36 +-
 .../gen-php/metastore/GetPartitionsResponse.php    |   20 +-
 .../gen-php/metastore/GetProjectionsSpec.php       |   18 +-
 .../thrift/gen-php/metastore/GetSchemaResponse.php |   20 +-
 .../thrift/gen-php/metastore/GetTableRequest.php   |   18 +-
 .../gen-php/metastore/GetTablesExtRequest.php      |   18 +-
 .../thrift/gen-php/metastore/GetTablesRequest.php  |   36 +-
 .../thrift/gen-php/metastore/GetTablesResult.php   |   20 +-
 .../gen-php/metastore/GetValidWriteIdsRequest.php  |   18 +-
 .../gen-php/metastore/GetValidWriteIdsResponse.php |   20 +-
 .../metastore/HeartbeatTxnRangeResponse.php        |   36 +-
 .../gen-php/metastore/InsertEventRequestData.php   |   72 +-
 .../gen/thrift/gen-php/metastore/LockRequest.php   |   20 +-
 .../metastore/NotNullConstraintsResponse.php       |   20 +-
 .../gen-php/metastore/NotificationEventRequest.php |   18 +-
 .../metastore/NotificationEventResponse.php        |   20 +-
 .../thrift/gen-php/metastore/ObjectDictionary.php  |   44 +-
 .../thrift/gen-php/metastore/OpenTxnRequest.php    |   18 +-
 .../thrift/gen-php/metastore/OpenTxnsResponse.php  |   18 +-
 .../src/gen/thrift/gen-php/metastore/Partition.php |   44 +-
 .../metastore/PartitionListComposingSpec.php       |   20 +-
 .../metastore/PartitionSpecWithSharedSD.php        |   20 +-
 .../gen-php/metastore/PartitionValuesRequest.php   |   40 +-
 .../gen-php/metastore/PartitionValuesResponse.php  |   20 +-
 .../gen-php/metastore/PartitionValuesRow.php       |   18 +-
 .../gen-php/metastore/PartitionWithoutSD.php       |   44 +-
 .../gen-php/metastore/PartitionsByExprResult.php   |   20 +-
 .../gen-php/metastore/PartitionsResponse.php       |   20 +-
 .../metastore/PartitionsSpecByExprResult.php       |   20 +-
 .../gen-php/metastore/PartitionsStatsRequest.php   |   36 +-
 .../gen-php/metastore/PartitionsStatsResult.php    |   46 +-
 .../gen-php/metastore/PrimaryKeysResponse.php      |   20 +-
 .../gen-php/metastore/PutFileMetadataRequest.php   |   36 +-
 .../gen-php/metastore/RenamePartitionRequest.php   |   18 +-
 .../thrift/gen-php/metastore/ReplLastIdInfo.php    |   18 +-
 .../metastore/ReplTblWriteIdStateRequest.php       |   18 +-
 .../gen-php/metastore/ReplicationMetricList.php    |   20 +-
 .../thrift/gen-php/metastore/RequestPartsSpec.php  |   38 +-
 .../src/gen/thrift/gen-php/metastore/Schema.php    |   46 +-
 .../gen/thrift/gen-php/metastore/SchemaVersion.php |   20 +-
 .../metastore/SetPartitionsStatsRequest.php        |   20 +-
 .../gen-php/metastore/ShowCompactResponse.php      |   20 +-
 .../thrift/gen-php/metastore/ShowLocksResponse.php |   20 +-
 .../src/gen/thrift/gen-php/metastore/Table.php     |   82 +-
 .../thrift/gen-php/metastore/TableStatsRequest.php |   18 +-
 .../thrift/gen-php/metastore/TableStatsResult.php  |   20 +-
 .../gen-php/metastore/TableValidWriteIds.php       |   18 +-
 .../ThriftHiveMetastore_add_partitions_args.php    |   20 +-
 ...riftHiveMetastore_add_partitions_pspec_args.php |   20 +-
 .../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 +-
 ...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 +-
 .../metastore/UniqueConstraintsResponse.php        |   20 +-
 .../gen-php/metastore/WMFullResourcePlan.php       |   80 +-
 .../metastore/WMGetAllResourcePlanResponse.php     |   20 +-
 .../WMGetTriggersForResourePlanResponse.php        |   20 +-
 .../metastore/WMValidateResourcePlanResponse.php   |   36 +-
 .../metastore/WriteNotificationLogBatchRequest.php |   20 +-
 .../metastore/WriteNotificationLogRequest.php      |   18 +-
 .../src/gen/thrift/gen-py/hive_metastore/ttypes.py |  232 +-
 .../src/gen/thrift/gen-rb/hive_metastore_types.rb  |   58 +-
 .../src/main/thrift/hive_metastore.thrift          |   16 +-
 .../apache/hadoop/hive/metastore/ObjectStore.java  |    2 +-
 .../metastore/client/builder/TableBuilder.java     |    4 +-
 .../hadoop/hive/metastore/TestObjectStore.java     |    4 +-
 .../hive/metastore/client/TestGetTableMeta.java    |    2 +-
 194 files changed, 7413 insertions(+), 7408 deletions(-)

diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/MaterializedViewMetadata.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/MaterializedViewMetadata.java
index 16c47f4..12bfd38 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/MaterializedViewMetadata.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/MaterializedViewMetadata.java
@@ -22,10 +22,13 @@ import org.apache.hadoop.hive.common.TableName;
 import org.apache.hadoop.hive.metastore.api.CreationMetadata;
 import org.apache.hadoop.hive.metastore.api.SourceTable;
 
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Set;
 import java.util.stream.Collectors;
 
 import static java.util.Collections.emptySet;
+import static java.util.Collections.unmodifiableList;
 import static java.util.Collections.unmodifiableSet;
 
 public class MaterializedViewMetadata {
@@ -39,7 +42,7 @@ public class MaterializedViewMetadata {
           String catalogName, String dbName, String mvName, Set<SourceTable> sourceTables, String validTxnList) {
     this.creationMetadata = new CreationMetadata(catalogName, dbName, mvName, toFullTableNames(sourceTables));
     this.creationMetadata.setValidTxnList(validTxnList);
-    this.creationMetadata.setSourceTables(unmodifiableSet(sourceTables));
+    this.creationMetadata.setSourceTables(unmodifiableList(new ArrayList<>(sourceTables)));
   }
 
   public Set<String> getSourceTableFullNames() {
@@ -50,7 +53,7 @@ public class MaterializedViewMetadata {
     return toFullTableNames(creationMetadata.getSourceTables());
   }
 
-  private Set<String> toFullTableNames(Set<SourceTable> sourceTables) {
+  private Set<String> toFullTableNames(Collection<SourceTable> sourceTables) {
     return unmodifiableSet(sourceTables.stream()
             .map(sourceTable -> TableName.getDbTable(
                     sourceTable.getTable().getDbName(), sourceTable.getTable().getTableName()))
@@ -70,12 +73,12 @@ public class MaterializedViewMetadata {
             .collect(Collectors.toSet()));
   }
 
-  public Set<SourceTable> getSourceTables() {
+  public Collection<SourceTable> getSourceTables() {
     if (!creationMetadata.isSetSourceTables()) {
       return emptySet();
     }
 
-    return unmodifiableSet(creationMetadata.getSourceTables());
+    return unmodifiableList(creationMetadata.getSourceTables());
   }
 
   public String getValidTxnList() {
diff --git a/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java b/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
index 7a8bc85..13da819 100644
--- a/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
+++ b/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
@@ -1915,7 +1915,7 @@ public class TestTxnHandler {
     CreationMetadata creationMetadata = new CreationMetadata();
     creationMetadata.setDbName("default");
     creationMetadata.setTblName("mat1");
-    creationMetadata.setSourceTables(new HashSet<SourceTable>() {{ add(sourceTable); }});
+    creationMetadata.setSourceTables(Collections.singletonList(sourceTable));
     creationMetadata.setValidTxnList(validTxnWriteIdList.toString());
 
     Materialization materialization = txnHandler.getMaterializationInvalidationInfo(creationMetadata);
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 2c187b0..26218bf 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@ -2561,14 +2561,14 @@ uint32_t ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1753;
-            ::apache::thrift::protocol::TType _etype1756;
-            xfer += iprot->readListBegin(_etype1756, _size1753);
-            this->success.resize(_size1753);
-            uint32_t _i1757;
-            for (_i1757 = 0; _i1757 < _size1753; ++_i1757)
+            uint32_t _size1752;
+            ::apache::thrift::protocol::TType _etype1755;
+            xfer += iprot->readListBegin(_etype1755, _size1752);
+            this->success.resize(_size1752);
+            uint32_t _i1756;
+            for (_i1756 = 0; _i1756 < _size1752; ++_i1756)
             {
-              xfer += iprot->readString(this->success[_i1757]);
+              xfer += iprot->readString(this->success[_i1756]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2607,10 +2607,10 @@ uint32_t ThriftHiveMetastore_get_databases_result::write(::apache::thrift::proto
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1758;
-      for (_iter1758 = this->success.begin(); _iter1758 != this->success.end(); ++_iter1758)
+      std::vector<std::string> ::const_iterator _iter1757;
+      for (_iter1757 = this->success.begin(); _iter1757 != this->success.end(); ++_iter1757)
       {
-        xfer += oprot->writeString((*_iter1758));
+        xfer += oprot->writeString((*_iter1757));
       }
       xfer += oprot->writeListEnd();
     }
@@ -2655,14 +2655,14 @@ uint32_t ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1759;
-            ::apache::thrift::protocol::TType _etype1762;
-            xfer += iprot->readListBegin(_etype1762, _size1759);
-            (*(this->success)).resize(_size1759);
-            uint32_t _i1763;
-            for (_i1763 = 0; _i1763 < _size1759; ++_i1763)
+            uint32_t _size1758;
+            ::apache::thrift::protocol::TType _etype1761;
+            xfer += iprot->readListBegin(_etype1761, _size1758);
+            (*(this->success)).resize(_size1758);
+            uint32_t _i1762;
+            for (_i1762 = 0; _i1762 < _size1758; ++_i1762)
             {
-              xfer += iprot->readString((*(this->success))[_i1763]);
+              xfer += iprot->readString((*(this->success))[_i1762]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2779,14 +2779,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1764;
-            ::apache::thrift::protocol::TType _etype1767;
-            xfer += iprot->readListBegin(_etype1767, _size1764);
-            this->success.resize(_size1764);
-            uint32_t _i1768;
-            for (_i1768 = 0; _i1768 < _size1764; ++_i1768)
+            uint32_t _size1763;
+            ::apache::thrift::protocol::TType _etype1766;
+            xfer += iprot->readListBegin(_etype1766, _size1763);
+            this->success.resize(_size1763);
+            uint32_t _i1767;
+            for (_i1767 = 0; _i1767 < _size1763; ++_i1767)
             {
-              xfer += iprot->readString(this->success[_i1768]);
+              xfer += iprot->readString(this->success[_i1767]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2825,10 +2825,10 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::write(::apache::thrift::p
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1769;
-      for (_iter1769 = this->success.begin(); _iter1769 != this->success.end(); ++_iter1769)
+      std::vector<std::string> ::const_iterator _iter1768;
+      for (_iter1768 = this->success.begin(); _iter1768 != this->success.end(); ++_iter1768)
       {
-        xfer += oprot->writeString((*_iter1769));
+        xfer += oprot->writeString((*_iter1768));
       }
       xfer += oprot->writeListEnd();
     }
@@ -2873,14 +2873,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1770;
-            ::apache::thrift::protocol::TType _etype1773;
-            xfer += iprot->readListBegin(_etype1773, _size1770);
-            (*(this->success)).resize(_size1770);
-            uint32_t _i1774;
-            for (_i1774 = 0; _i1774 < _size1770; ++_i1774)
+            uint32_t _size1769;
+            ::apache::thrift::protocol::TType _etype1772;
+            xfer += iprot->readListBegin(_etype1772, _size1769);
+            (*(this->success)).resize(_size1769);
+            uint32_t _i1773;
+            for (_i1773 = 0; _i1773 < _size1769; ++_i1773)
             {
-              xfer += iprot->readString((*(this->success))[_i1774]);
+              xfer += iprot->readString((*(this->success))[_i1773]);
             }
             xfer += iprot->readListEnd();
           }
@@ -3933,14 +3933,14 @@ uint32_t ThriftHiveMetastore_get_dataconnectors_result::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1775;
-            ::apache::thrift::protocol::TType _etype1778;
-            xfer += iprot->readListBegin(_etype1778, _size1775);
-            this->success.resize(_size1775);
-            uint32_t _i1779;
-            for (_i1779 = 0; _i1779 < _size1775; ++_i1779)
+            uint32_t _size1774;
+            ::apache::thrift::protocol::TType _etype1777;
+            xfer += iprot->readListBegin(_etype1777, _size1774);
+            this->success.resize(_size1774);
+            uint32_t _i1778;
+            for (_i1778 = 0; _i1778 < _size1774; ++_i1778)
             {
-              xfer += iprot->readString(this->success[_i1779]);
+              xfer += iprot->readString(this->success[_i1778]);
             }
             xfer += iprot->readListEnd();
           }
@@ -3979,10 +3979,10 @@ uint32_t ThriftHiveMetastore_get_dataconnectors_result::write(::apache::thrift::
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1780;
-      for (_iter1780 = this->success.begin(); _iter1780 != this->success.end(); ++_iter1780)
+      std::vector<std::string> ::const_iterator _iter1779;
+      for (_iter1779 = this->success.begin(); _iter1779 != this->success.end(); ++_iter1779)
       {
-        xfer += oprot->writeString((*_iter1780));
+        xfer += oprot->writeString((*_iter1779));
       }
       xfer += oprot->writeListEnd();
     }
@@ -4027,14 +4027,14 @@ uint32_t ThriftHiveMetastore_get_dataconnectors_presult::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1781;
-            ::apache::thrift::protocol::TType _etype1784;
-            xfer += iprot->readListBegin(_etype1784, _size1781);
-            (*(this->success)).resize(_size1781);
-            uint32_t _i1785;
-            for (_i1785 = 0; _i1785 < _size1781; ++_i1785)
+            uint32_t _size1780;
+            ::apache::thrift::protocol::TType _etype1783;
+            xfer += iprot->readListBegin(_etype1783, _size1780);
+            (*(this->success)).resize(_size1780);
+            uint32_t _i1784;
+            for (_i1784 = 0; _i1784 < _size1780; ++_i1784)
             {
-              xfer += iprot->readString((*(this->success))[_i1785]);
+              xfer += iprot->readString((*(this->success))[_i1784]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5096,17 +5096,17 @@ uint32_t ThriftHiveMetastore_get_type_all_result::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size1786;
-            ::apache::thrift::protocol::TType _ktype1787;
-            ::apache::thrift::protocol::TType _vtype1788;
-            xfer += iprot->readMapBegin(_ktype1787, _vtype1788, _size1786);
-            uint32_t _i1790;
-            for (_i1790 = 0; _i1790 < _size1786; ++_i1790)
+            uint32_t _size1785;
+            ::apache::thrift::protocol::TType _ktype1786;
+            ::apache::thrift::protocol::TType _vtype1787;
+            xfer += iprot->readMapBegin(_ktype1786, _vtype1787, _size1785);
+            uint32_t _i1789;
+            for (_i1789 = 0; _i1789 < _size1785; ++_i1789)
             {
-              std::string _key1791;
-              xfer += iprot->readString(_key1791);
-              Type& _val1792 = this->success[_key1791];
-              xfer += _val1792.read(iprot);
+              std::string _key1790;
+              xfer += iprot->readString(_key1790);
+              Type& _val1791 = this->success[_key1790];
+              xfer += _val1791.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -5145,11 +5145,11 @@ uint32_t ThriftHiveMetastore_get_type_all_result::write(::apache::thrift::protoc
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::map<std::string, Type> ::const_iterator _iter1793;
-      for (_iter1793 = this->success.begin(); _iter1793 != this->success.end(); ++_iter1793)
+      std::map<std::string, Type> ::const_iterator _iter1792;
+      for (_iter1792 = this->success.begin(); _iter1792 != this->success.end(); ++_iter1792)
       {
-        xfer += oprot->writeString(_iter1793->first);
-        xfer += _iter1793->second.write(oprot);
+        xfer += oprot->writeString(_iter1792->first);
+        xfer += _iter1792->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -5194,17 +5194,17 @@ uint32_t ThriftHiveMetastore_get_type_all_presult::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size1794;
-            ::apache::thrift::protocol::TType _ktype1795;
-            ::apache::thrift::protocol::TType _vtype1796;
-            xfer += iprot->readMapBegin(_ktype1795, _vtype1796, _size1794);
-            uint32_t _i1798;
-            for (_i1798 = 0; _i1798 < _size1794; ++_i1798)
+            uint32_t _size1793;
+            ::apache::thrift::protocol::TType _ktype1794;
+            ::apache::thrift::protocol::TType _vtype1795;
+            xfer += iprot->readMapBegin(_ktype1794, _vtype1795, _size1793);
+            uint32_t _i1797;
+            for (_i1797 = 0; _i1797 < _size1793; ++_i1797)
             {
-              std::string _key1799;
-              xfer += iprot->readString(_key1799);
-              Type& _val1800 = (*(this->success))[_key1799];
-              xfer += _val1800.read(iprot);
+              std::string _key1798;
+              xfer += iprot->readString(_key1798);
+              Type& _val1799 = (*(this->success))[_key1798];
+              xfer += _val1799.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -5358,14 +5358,14 @@ uint32_t ThriftHiveMetastore_get_fields_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1801;
-            ::apache::thrift::protocol::TType _etype1804;
-            xfer += iprot->readListBegin(_etype1804, _size1801);
-            this->success.resize(_size1801);
-            uint32_t _i1805;
-            for (_i1805 = 0; _i1805 < _size1801; ++_i1805)
+            uint32_t _size1800;
+            ::apache::thrift::protocol::TType _etype1803;
+            xfer += iprot->readListBegin(_etype1803, _size1800);
+            this->success.resize(_size1800);
+            uint32_t _i1804;
+            for (_i1804 = 0; _i1804 < _size1800; ++_i1804)
             {
-              xfer += this->success[_i1805].read(iprot);
+              xfer += this->success[_i1804].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5420,10 +5420,10 @@ uint32_t ThriftHiveMetastore_get_fields_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter1806;
-      for (_iter1806 = this->success.begin(); _iter1806 != this->success.end(); ++_iter1806)
+      std::vector<FieldSchema> ::const_iterator _iter1805;
+      for (_iter1805 = this->success.begin(); _iter1805 != this->success.end(); ++_iter1805)
       {
-        xfer += (*_iter1806).write(oprot);
+        xfer += (*_iter1805).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -5476,14 +5476,14 @@ uint32_t ThriftHiveMetastore_get_fields_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1807;
-            ::apache::thrift::protocol::TType _etype1810;
-            xfer += iprot->readListBegin(_etype1810, _size1807);
-            (*(this->success)).resize(_size1807);
-            uint32_t _i1811;
-            for (_i1811 = 0; _i1811 < _size1807; ++_i1811)
+            uint32_t _size1806;
+            ::apache::thrift::protocol::TType _etype1809;
+            xfer += iprot->readListBegin(_etype1809, _size1806);
+            (*(this->success)).resize(_size1806);
+            uint32_t _i1810;
+            for (_i1810 = 0; _i1810 < _size1806; ++_i1810)
             {
-              xfer += (*(this->success))[_i1811].read(iprot);
+              xfer += (*(this->success))[_i1810].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5669,14 +5669,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1812;
-            ::apache::thrift::protocol::TType _etype1815;
-            xfer += iprot->readListBegin(_etype1815, _size1812);
-            this->success.resize(_size1812);
-            uint32_t _i1816;
-            for (_i1816 = 0; _i1816 < _size1812; ++_i1816)
+            uint32_t _size1811;
+            ::apache::thrift::protocol::TType _etype1814;
+            xfer += iprot->readListBegin(_etype1814, _size1811);
+            this->success.resize(_size1811);
+            uint32_t _i1815;
+            for (_i1815 = 0; _i1815 < _size1811; ++_i1815)
             {
-              xfer += this->success[_i1816].read(iprot);
+              xfer += this->success[_i1815].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5731,10 +5731,10 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::write(:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter1817;
-      for (_iter1817 = this->success.begin(); _iter1817 != this->success.end(); ++_iter1817)
+      std::vector<FieldSchema> ::const_iterator _iter1816;
+      for (_iter1816 = this->success.begin(); _iter1816 != this->success.end(); ++_iter1816)
       {
-        xfer += (*_iter1817).write(oprot);
+        xfer += (*_iter1816).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -5787,14 +5787,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1818;
-            ::apache::thrift::protocol::TType _etype1821;
-            xfer += iprot->readListBegin(_etype1821, _size1818);
-            (*(this->success)).resize(_size1818);
-            uint32_t _i1822;
-            for (_i1822 = 0; _i1822 < _size1818; ++_i1822)
+            uint32_t _size1817;
+            ::apache::thrift::protocol::TType _etype1820;
+            xfer += iprot->readListBegin(_etype1820, _size1817);
+            (*(this->success)).resize(_size1817);
+            uint32_t _i1821;
+            for (_i1821 = 0; _i1821 < _size1817; ++_i1821)
             {
-              xfer += (*(this->success))[_i1822].read(iprot);
+              xfer += (*(this->success))[_i1821].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6211,14 +6211,14 @@ uint32_t ThriftHiveMetastore_get_schema_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1823;
-            ::apache::thrift::protocol::TType _etype1826;
-            xfer += iprot->readListBegin(_etype1826, _size1823);
-            this->success.resize(_size1823);
-            uint32_t _i1827;
-            for (_i1827 = 0; _i1827 < _size1823; ++_i1827)
+            uint32_t _size1822;
+            ::apache::thrift::protocol::TType _etype1825;
+            xfer += iprot->readListBegin(_etype1825, _size1822);
+            this->success.resize(_size1822);
+            uint32_t _i1826;
+            for (_i1826 = 0; _i1826 < _size1822; ++_i1826)
             {
-              xfer += this->success[_i1827].read(iprot);
+              xfer += this->success[_i1826].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6273,10 +6273,10 @@ uint32_t ThriftHiveMetastore_get_schema_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter1828;
-      for (_iter1828 = this->success.begin(); _iter1828 != this->success.end(); ++_iter1828)
+      std::vector<FieldSchema> ::const_iterator _iter1827;
+      for (_iter1827 = this->success.begin(); _iter1827 != this->success.end(); ++_iter1827)
       {
-        xfer += (*_iter1828).write(oprot);
+        xfer += (*_iter1827).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -6329,14 +6329,14 @@ uint32_t ThriftHiveMetastore_get_schema_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1829;
-            ::apache::thrift::protocol::TType _etype1832;
-            xfer += iprot->readListBegin(_etype1832, _size1829);
-            (*(this->success)).resize(_size1829);
-            uint32_t _i1833;
-            for (_i1833 = 0; _i1833 < _size1829; ++_i1833)
+            uint32_t _size1828;
+            ::apache::thrift::protocol::TType _etype1831;
+            xfer += iprot->readListBegin(_etype1831, _size1828);
+            (*(this->success)).resize(_size1828);
+            uint32_t _i1832;
+            for (_i1832 = 0; _i1832 < _size1828; ++_i1832)
             {
-              xfer += (*(this->success))[_i1833].read(iprot);
+              xfer += (*(this->success))[_i1832].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6522,14 +6522,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1834;
-            ::apache::thrift::protocol::TType _etype1837;
-            xfer += iprot->readListBegin(_etype1837, _size1834);
-            this->success.resize(_size1834);
-            uint32_t _i1838;
-            for (_i1838 = 0; _i1838 < _size1834; ++_i1838)
+            uint32_t _size1833;
+            ::apache::thrift::protocol::TType _etype1836;
+            xfer += iprot->readListBegin(_etype1836, _size1833);
+            this->success.resize(_size1833);
+            uint32_t _i1837;
+            for (_i1837 = 0; _i1837 < _size1833; ++_i1837)
             {
-              xfer += this->success[_i1838].read(iprot);
+              xfer += this->success[_i1837].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6584,10 +6584,10 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::write(:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter1839;
-      for (_iter1839 = this->success.begin(); _iter1839 != this->success.end(); ++_iter1839)
+      std::vector<FieldSchema> ::const_iterator _iter1838;
+      for (_iter1838 = this->success.begin(); _iter1838 != this->success.end(); ++_iter1838)
       {
-        xfer += (*_iter1839).write(oprot);
+        xfer += (*_iter1838).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -6640,14 +6640,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1840;
-            ::apache::thrift::protocol::TType _etype1843;
-            xfer += iprot->readListBegin(_etype1843, _size1840);
-            (*(this->success)).resize(_size1840);
-            uint32_t _i1844;
-            for (_i1844 = 0; _i1844 < _size1840; ++_i1844)
+            uint32_t _size1839;
+            ::apache::thrift::protocol::TType _etype1842;
+            xfer += iprot->readListBegin(_etype1842, _size1839);
+            (*(this->success)).resize(_size1839);
+            uint32_t _i1843;
+            for (_i1843 = 0; _i1843 < _size1839; ++_i1843)
             {
-              xfer += (*(this->success))[_i1844].read(iprot);
+              xfer += (*(this->success))[_i1843].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7487,14 +7487,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->primaryKeys.clear();
-            uint32_t _size1845;
-            ::apache::thrift::protocol::TType _etype1848;
-            xfer += iprot->readListBegin(_etype1848, _size1845);
-            this->primaryKeys.resize(_size1845);
-            uint32_t _i1849;
-            for (_i1849 = 0; _i1849 < _size1845; ++_i1849)
+            uint32_t _size1844;
+            ::apache::thrift::protocol::TType _etype1847;
+            xfer += iprot->readListBegin(_etype1847, _size1844);
+            this->primaryKeys.resize(_size1844);
+            uint32_t _i1848;
+            for (_i1848 = 0; _i1848 < _size1844; ++_i1848)
             {
-              xfer += this->primaryKeys[_i1849].read(iprot);
+              xfer += this->primaryKeys[_i1848].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7507,14 +7507,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->foreignKeys.clear();
-            uint32_t _size1850;
-            ::apache::thrift::protocol::TType _etype1853;
-            xfer += iprot->readListBegin(_etype1853, _size1850);
-            this->foreignKeys.resize(_size1850);
-            uint32_t _i1854;
-            for (_i1854 = 0; _i1854 < _size1850; ++_i1854)
+            uint32_t _size1849;
+            ::apache::thrift::protocol::TType _etype1852;
+            xfer += iprot->readListBegin(_etype1852, _size1849);
+            this->foreignKeys.resize(_size1849);
+            uint32_t _i1853;
+            for (_i1853 = 0; _i1853 < _size1849; ++_i1853)
             {
-              xfer += this->foreignKeys[_i1854].read(iprot);
+              xfer += this->foreignKeys[_i1853].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7527,14 +7527,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->uniqueConstraints.clear();
-            uint32_t _size1855;
-            ::apache::thrift::protocol::TType _etype1858;
-            xfer += iprot->readListBegin(_etype1858, _size1855);
-            this->uniqueConstraints.resize(_size1855);
-            uint32_t _i1859;
-            for (_i1859 = 0; _i1859 < _size1855; ++_i1859)
+            uint32_t _size1854;
+            ::apache::thrift::protocol::TType _etype1857;
+            xfer += iprot->readListBegin(_etype1857, _size1854);
+            this->uniqueConstraints.resize(_size1854);
+            uint32_t _i1858;
+            for (_i1858 = 0; _i1858 < _size1854; ++_i1858)
             {
-              xfer += this->uniqueConstraints[_i1859].read(iprot);
+              xfer += this->uniqueConstraints[_i1858].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7547,14 +7547,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->notNullConstraints.clear();
-            uint32_t _size1860;
-            ::apache::thrift::protocol::TType _etype1863;
-            xfer += iprot->readListBegin(_etype1863, _size1860);
-            this->notNullConstraints.resize(_size1860);
-            uint32_t _i1864;
-            for (_i1864 = 0; _i1864 < _size1860; ++_i1864)
+            uint32_t _size1859;
+            ::apache::thrift::protocol::TType _etype1862;
+            xfer += iprot->readListBegin(_etype1862, _size1859);
+            this->notNullConstraints.resize(_size1859);
+            uint32_t _i1863;
+            for (_i1863 = 0; _i1863 < _size1859; ++_i1863)
             {
-              xfer += this->notNullConstraints[_i1864].read(iprot);
+              xfer += this->notNullConstraints[_i1863].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7567,14 +7567,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->defaultConstraints.clear();
-            uint32_t _size1865;
-            ::apache::thrift::protocol::TType _etype1868;
-            xfer += iprot->readListBegin(_etype1868, _size1865);
-            this->defaultConstraints.resize(_size1865);
-            uint32_t _i1869;
-            for (_i1869 = 0; _i1869 < _size1865; ++_i1869)
+            uint32_t _size1864;
+            ::apache::thrift::protocol::TType _etype1867;
+            xfer += iprot->readListBegin(_etype1867, _size1864);
+            this->defaultConstraints.resize(_size1864);
+            uint32_t _i1868;
+            for (_i1868 = 0; _i1868 < _size1864; ++_i1868)
             {
-              xfer += this->defaultConstraints[_i1869].read(iprot);
+              xfer += this->defaultConstraints[_i1868].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7587,14 +7587,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->checkConstraints.clear();
-            uint32_t _size1870;
-            ::apache::thrift::protocol::TType _etype1873;
-            xfer += iprot->readListBegin(_etype1873, _size1870);
-            this->checkConstraints.resize(_size1870);
-            uint32_t _i1874;
-            for (_i1874 = 0; _i1874 < _size1870; ++_i1874)
+            uint32_t _size1869;
+            ::apache::thrift::protocol::TType _etype1872;
+            xfer += iprot->readListBegin(_etype1872, _size1869);
+            this->checkConstraints.resize(_size1869);
+            uint32_t _i1873;
+            for (_i1873 = 0; _i1873 < _size1869; ++_i1873)
             {
-              xfer += this->checkConstraints[_i1874].read(iprot);
+              xfer += this->checkConstraints[_i1873].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7627,10 +7627,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->primaryKeys.size()));
-    std::vector<SQLPrimaryKey> ::const_iterator _iter1875;
-    for (_iter1875 = this->primaryKeys.begin(); _iter1875 != this->primaryKeys.end(); ++_iter1875)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter1874;
+    for (_iter1874 = this->primaryKeys.begin(); _iter1874 != this->primaryKeys.end(); ++_iter1874)
     {
-      xfer += (*_iter1875).write(oprot);
+      xfer += (*_iter1874).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7639,10 +7639,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->foreignKeys.size()));
-    std::vector<SQLForeignKey> ::const_iterator _iter1876;
-    for (_iter1876 = this->foreignKeys.begin(); _iter1876 != this->foreignKeys.end(); ++_iter1876)
+    std::vector<SQLForeignKey> ::const_iterator _iter1875;
+    for (_iter1875 = this->foreignKeys.begin(); _iter1875 != this->foreignKeys.end(); ++_iter1875)
     {
-      xfer += (*_iter1876).write(oprot);
+      xfer += (*_iter1875).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7651,10 +7651,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 4);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->uniqueConstraints.size()));
-    std::vector<SQLUniqueConstraint> ::const_iterator _iter1877;
-    for (_iter1877 = this->uniqueConstraints.begin(); _iter1877 != this->uniqueConstraints.end(); ++_iter1877)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter1876;
+    for (_iter1876 = this->uniqueConstraints.begin(); _iter1876 != this->uniqueConstraints.end(); ++_iter1876)
     {
-      xfer += (*_iter1877).write(oprot);
+      xfer += (*_iter1876).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7663,10 +7663,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->notNullConstraints.size()));
-    std::vector<SQLNotNullConstraint> ::const_iterator _iter1878;
-    for (_iter1878 = this->notNullConstraints.begin(); _iter1878 != this->notNullConstraints.end(); ++_iter1878)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter1877;
+    for (_iter1877 = this->notNullConstraints.begin(); _iter1877 != this->notNullConstraints.end(); ++_iter1877)
     {
-      xfer += (*_iter1878).write(oprot);
+      xfer += (*_iter1877).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7675,10 +7675,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("defaultConstraints", ::apache::thrift::protocol::T_LIST, 6);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->defaultConstraints.size()));
-    std::vector<SQLDefaultConstraint> ::const_iterator _iter1879;
-    for (_iter1879 = this->defaultConstraints.begin(); _iter1879 != this->defaultConstraints.end(); ++_iter1879)
+    std::vector<SQLDefaultConstraint> ::const_iterator _iter1878;
+    for (_iter1878 = this->defaultConstraints.begin(); _iter1878 != this->defaultConstraints.end(); ++_iter1878)
     {
-      xfer += (*_iter1879).write(oprot);
+      xfer += (*_iter1878).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7687,10 +7687,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("checkConstraints", ::apache::thrift::protocol::T_LIST, 7);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->checkConstraints.size()));
-    std::vector<SQLCheckConstraint> ::const_iterator _iter1880;
-    for (_iter1880 = this->checkConstraints.begin(); _iter1880 != this->checkConstraints.end(); ++_iter1880)
+    std::vector<SQLCheckConstraint> ::const_iterator _iter1879;
+    for (_iter1879 = this->checkConstraints.begin(); _iter1879 != this->checkConstraints.end(); ++_iter1879)
     {
-      xfer += (*_iter1880).write(oprot);
+      xfer += (*_iter1879).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7718,10 +7718,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->primaryKeys)).size()));
-    std::vector<SQLPrimaryKey> ::const_iterator _iter1881;
-    for (_iter1881 = (*(this->primaryKeys)).begin(); _iter1881 != (*(this->primaryKeys)).end(); ++_iter1881)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter1880;
+    for (_iter1880 = (*(this->primaryKeys)).begin(); _iter1880 != (*(this->primaryKeys)).end(); ++_iter1880)
     {
-      xfer += (*_iter1881).write(oprot);
+      xfer += (*_iter1880).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7730,10 +7730,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->foreignKeys)).size()));
-    std::vector<SQLForeignKey> ::const_iterator _iter1882;
-    for (_iter1882 = (*(this->foreignKeys)).begin(); _iter1882 != (*(this->foreignKeys)).end(); ++_iter1882)
+    std::vector<SQLForeignKey> ::const_iterator _iter1881;
+    for (_iter1881 = (*(this->foreignKeys)).begin(); _iter1881 != (*(this->foreignKeys)).end(); ++_iter1881)
     {
-      xfer += (*_iter1882).write(oprot);
+      xfer += (*_iter1881).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7742,10 +7742,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 4);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->uniqueConstraints)).size()));
-    std::vector<SQLUniqueConstraint> ::const_iterator _iter1883;
-    for (_iter1883 = (*(this->uniqueConstraints)).begin(); _iter1883 != (*(this->uniqueConstraints)).end(); ++_iter1883)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter1882;
+    for (_iter1882 = (*(this->uniqueConstraints)).begin(); _iter1882 != (*(this->uniqueConstraints)).end(); ++_iter1882)
     {
-      xfer += (*_iter1883).write(oprot);
+      xfer += (*_iter1882).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7754,10 +7754,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->notNullConstraints)).size()));
-    std::vector<SQLNotNullConstraint> ::const_iterator _iter1884;
-    for (_iter1884 = (*(this->notNullConstraints)).begin(); _iter1884 != (*(this->notNullConstraints)).end(); ++_iter1884)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter1883;
+    for (_iter1883 = (*(this->notNullConstraints)).begin(); _iter1883 != (*(this->notNullConstraints)).end(); ++_iter1883)
     {
-      xfer += (*_iter1884).write(oprot);
+      xfer += (*_iter1883).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7766,10 +7766,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("defaultConstraints", ::apache::thrift::protocol::T_LIST, 6);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->defaultConstraints)).size()));
-    std::vector<SQLDefaultConstraint> ::const_iterator _iter1885;
-    for (_iter1885 = (*(this->defaultConstraints)).begin(); _iter1885 != (*(this->defaultConstraints)).end(); ++_iter1885)
+    std::vector<SQLDefaultConstraint> ::const_iterator _iter1884;
+    for (_iter1884 = (*(this->defaultConstraints)).begin(); _iter1884 != (*(this->defaultConstraints)).end(); ++_iter1884)
     {
-      xfer += (*_iter1885).write(oprot);
+      xfer += (*_iter1884).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7778,10 +7778,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("checkConstraints", ::apache::thrift::protocol::T_LIST, 7);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->checkConstraints)).size()));
-    std::vector<SQLCheckConstraint> ::const_iterator _iter1886;
-    for (_iter1886 = (*(this->checkConstraints)).begin(); _iter1886 != (*(this->checkConstraints)).end(); ++_iter1886)
+    std::vector<SQLCheckConstraint> ::const_iterator _iter1885;
+    for (_iter1885 = (*(this->checkConstraints)).begin(); _iter1885 != (*(this->checkConstraints)).end(); ++_iter1885)
     {
-      xfer += (*_iter1886).write(oprot);
+      xfer += (*_iter1885).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -10463,14 +10463,14 @@ uint32_t ThriftHiveMetastore_truncate_table_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partNames.clear();
-            uint32_t _size1887;
-            ::apache::thrift::protocol::TType _etype1890;
-            xfer += iprot->readListBegin(_etype1890, _size1887);
-            this->partNames.resize(_size1887);
-            uint32_t _i1891;
-            for (_i1891 = 0; _i1891 < _size1887; ++_i1891)
+            uint32_t _size1886;
+            ::apache::thrift::protocol::TType _etype1889;
+            xfer += iprot->readListBegin(_etype1889, _size1886);
+            this->partNames.resize(_size1886);
+            uint32_t _i1890;
+            for (_i1890 = 0; _i1890 < _size1886; ++_i1890)
             {
-              xfer += iprot->readString(this->partNames[_i1891]);
+              xfer += iprot->readString(this->partNames[_i1890]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10507,10 +10507,10 @@ uint32_t ThriftHiveMetastore_truncate_table_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partNames.size()));
-    std::vector<std::string> ::const_iterator _iter1892;
-    for (_iter1892 = this->partNames.begin(); _iter1892 != this->partNames.end(); ++_iter1892)
+    std::vector<std::string> ::const_iterator _iter1891;
+    for (_iter1891 = this->partNames.begin(); _iter1891 != this->partNames.end(); ++_iter1891)
     {
-      xfer += oprot->writeString((*_iter1892));
+      xfer += oprot->writeString((*_iter1891));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10542,10 +10542,10 @@ uint32_t ThriftHiveMetastore_truncate_table_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partNames)).size()));
-    std::vector<std::string> ::const_iterator _iter1893;
-    for (_iter1893 = (*(this->partNames)).begin(); _iter1893 != (*(this->partNames)).end(); ++_iter1893)
+    std::vector<std::string> ::const_iterator _iter1892;
+    for (_iter1892 = (*(this->partNames)).begin(); _iter1892 != (*(this->partNames)).end(); ++_iter1892)
     {
-      xfer += oprot->writeString((*_iter1893));
+      xfer += oprot->writeString((*_iter1892));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10996,14 +10996,14 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1894;
-            ::apache::thrift::protocol::TType _etype1897;
-            xfer += iprot->readListBegin(_etype1897, _size1894);
-            this->success.resize(_size1894);
-            uint32_t _i1898;
-            for (_i1898 = 0; _i1898 < _size1894; ++_i1898)
+            uint32_t _size1893;
+            ::apache::thrift::protocol::TType _etype1896;
+            xfer += iprot->readListBegin(_etype1896, _size1893);
+            this->success.resize(_size1893);
+            uint32_t _i1897;
+            for (_i1897 = 0; _i1897 < _size1893; ++_i1897)
             {
-              xfer += iprot->readString(this->success[_i1898]);
+              xfer += iprot->readString(this->success[_i1897]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11042,10 +11042,10 @@ uint32_t ThriftHiveMetastore_get_tables_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1899;
-      for (_iter1899 = this->success.begin(); _iter1899 != this->success.end(); ++_iter1899)
+      std::vector<std::string> ::const_iterator _iter1898;
+      for (_iter1898 = this->success.begin(); _iter1898 != this->success.end(); ++_iter1898)
       {
-        xfer += oprot->writeString((*_iter1899));
+        xfer += oprot->writeString((*_iter1898));
       }
       xfer += oprot->writeListEnd();
     }
@@ -11090,14 +11090,14 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1900;
-            ::apache::thrift::protocol::TType _etype1903;
-            xfer += iprot->readListBegin(_etype1903, _size1900);
-            (*(this->success)).resize(_size1900);
-            uint32_t _i1904;
-            for (_i1904 = 0; _i1904 < _size1900; ++_i1904)
+            uint32_t _size1899;
+            ::apache::thrift::protocol::TType _etype1902;
+            xfer += iprot->readListBegin(_etype1902, _size1899);
+            (*(this->success)).resize(_size1899);
+            uint32_t _i1903;
+            for (_i1903 = 0; _i1903 < _size1899; ++_i1903)
             {
-              xfer += iprot->readString((*(this->success))[_i1904]);
+              xfer += iprot->readString((*(this->success))[_i1903]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11267,14 +11267,14 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_result::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1905;
-            ::apache::thrift::protocol::TType _etype1908;
-            xfer += iprot->readListBegin(_etype1908, _size1905);
-            this->success.resize(_size1905);
-            uint32_t _i1909;
-            for (_i1909 = 0; _i1909 < _size1905; ++_i1909)
+            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)
             {
-              xfer += iprot->readString(this->success[_i1909]);
+              xfer += iprot->readString(this->success[_i1908]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11313,10 +11313,10 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_result::write(::apache::thrift::
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1910;
-      for (_iter1910 = this->success.begin(); _iter1910 != this->success.end(); ++_iter1910)
+      std::vector<std::string> ::const_iterator _iter1909;
+      for (_iter1909 = this->success.begin(); _iter1909 != this->success.end(); ++_iter1909)
       {
-        xfer += oprot->writeString((*_iter1910));
+        xfer += oprot->writeString((*_iter1909));
       }
       xfer += oprot->writeListEnd();
     }
@@ -11361,14 +11361,14 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_presult::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1911;
-            ::apache::thrift::protocol::TType _etype1914;
-            xfer += iprot->readListBegin(_etype1914, _size1911);
-            (*(this->success)).resize(_size1911);
-            uint32_t _i1915;
-            for (_i1915 = 0; _i1915 < _size1911; ++_i1915)
+            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 += iprot->readString((*(this->success))[_i1915]);
+              xfer += iprot->readString((*(this->success))[_i1914]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11485,14 +11485,14 @@ uint32_t ThriftHiveMetastore_get_all_materialized_view_objects_for_rewriting_res
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1916;
-            ::apache::thrift::protocol::TType _etype1919;
-            xfer += iprot->readListBegin(_etype1919, _size1916);
-            this->success.resize(_size1916);
-            uint32_t _i1920;
-            for (_i1920 = 0; _i1920 < _size1916; ++_i1920)
+            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)
             {
-              xfer += this->success[_i1920].read(iprot);
+              xfer += this->success[_i1919].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -11531,10 +11531,10 @@ uint32_t ThriftHiveMetastore_get_all_materialized_view_objects_for_rewriting_res
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Table> ::const_iterator _iter1921;
-      for (_iter1921 = this->success.begin(); _iter1921 != this->success.end(); ++_iter1921)
+      std::vector<Table> ::const_iterator _iter1920;
+      for (_iter1920 = this->success.begin(); _iter1920 != this->success.end(); ++_iter1920)
       {
-        xfer += (*_iter1921).write(oprot);
+        xfer += (*_iter1920).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -11579,14 +11579,14 @@ uint32_t ThriftHiveMetastore_get_all_materialized_view_objects_for_rewriting_pre
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1922;
-            ::apache::thrift::protocol::TType _etype1925;
-            xfer += iprot->readListBegin(_etype1925, _size1922);
-            (*(this->success)).resize(_size1922);
-            uint32_t _i1926;
-            for (_i1926 = 0; _i1926 < _size1922; ++_i1926)
+            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))[_i1926].read(iprot);
+              xfer += (*(this->success))[_i1925].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -11724,14 +11724,14 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_result::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1927;
-            ::apache::thrift::protocol::TType _etype1930;
-            xfer += iprot->readListBegin(_etype1930, _size1927);
-            this->success.resize(_size1927);
-            uint32_t _i1931;
-            for (_i1931 = 0; _i1931 < _size1927; ++_i1931)
+            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)
             {
-              xfer += iprot->readString(this->success[_i1931]);
+              xfer += iprot->readString(this->success[_i1930]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11770,10 +11770,10 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_result::write(
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1932;
-      for (_iter1932 = this->success.begin(); _iter1932 != this->success.end(); ++_iter1932)
+      std::vector<std::string> ::const_iterator _iter1931;
+      for (_iter1931 = this->success.begin(); _iter1931 != this->success.end(); ++_iter1931)
       {
-        xfer += oprot->writeString((*_iter1932));
+        xfer += oprot->writeString((*_iter1931));
       }
       xfer += oprot->writeListEnd();
     }
@@ -11818,14 +11818,14 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_presult::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1933;
-            ::apache::thrift::protocol::TType _etype1936;
-            xfer += iprot->readListBegin(_etype1936, _size1933);
-            (*(this->success)).resize(_size1933);
-            uint32_t _i1937;
-            for (_i1937 = 0; _i1937 < _size1933; ++_i1937)
+            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 += iprot->readString((*(this->success))[_i1937]);
+              xfer += iprot->readString((*(this->success))[_i1936]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11900,14 +11900,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_types.clear();
-            uint32_t _size1938;
-            ::apache::thrift::protocol::TType _etype1941;
-            xfer += iprot->readListBegin(_etype1941, _size1938);
-            this->tbl_types.resize(_size1938);
-            uint32_t _i1942;
-            for (_i1942 = 0; _i1942 < _size1938; ++_i1942)
+            uint32_t _size1937;
+            ::apache::thrift::protocol::TType _etype1940;
+            xfer += iprot->readListBegin(_etype1940, _size1937);
+            this->tbl_types.resize(_size1937);
+            uint32_t _i1941;
+            for (_i1941 = 0; _i1941 < _size1937; ++_i1941)
             {
-              xfer += iprot->readString(this->tbl_types[_i1942]);
+              xfer += iprot->readString(this->tbl_types[_i1941]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11944,10 +11944,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_types.size()));
-    std::vector<std::string> ::const_iterator _iter1943;
-    for (_iter1943 = this->tbl_types.begin(); _iter1943 != this->tbl_types.end(); ++_iter1943)
+    std::vector<std::string> ::const_iterator _iter1942;
+    for (_iter1942 = this->tbl_types.begin(); _iter1942 != this->tbl_types.end(); ++_iter1942)
     {
-      xfer += oprot->writeString((*_iter1943));
+      xfer += oprot->writeString((*_iter1942));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11979,10 +11979,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_types)).size()));
-    std::vector<std::string> ::const_iterator _iter1944;
-    for (_iter1944 = (*(this->tbl_types)).begin(); _iter1944 != (*(this->tbl_types)).end(); ++_iter1944)
+    std::vector<std::string> ::const_iterator _iter1943;
+    for (_iter1943 = (*(this->tbl_types)).begin(); _iter1943 != (*(this->tbl_types)).end(); ++_iter1943)
     {
-      xfer += oprot->writeString((*_iter1944));
+      xfer += oprot->writeString((*_iter1943));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12023,14 +12023,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1945;
-            ::apache::thrift::protocol::TType _etype1948;
-            xfer += iprot->readListBegin(_etype1948, _size1945);
-            this->success.resize(_size1945);
-            uint32_t _i1949;
-            for (_i1949 = 0; _i1949 < _size1945; ++_i1949)
+            uint32_t _size1944;
+            ::apache::thrift::protocol::TType _etype1947;
+            xfer += iprot->readListBegin(_etype1947, _size1944);
+            this->success.resize(_size1944);
+            uint32_t _i1948;
+            for (_i1948 = 0; _i1948 < _size1944; ++_i1948)
             {
-              xfer += this->success[_i1949].read(iprot);
+              xfer += this->success[_i1948].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12069,10 +12069,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<TableMeta> ::const_iterator _iter1950;
-      for (_iter1950 = this->success.begin(); _iter1950 != this->success.end(); ++_iter1950)
+      std::vector<TableMeta> ::const_iterator _iter1949;
+      for (_iter1949 = this->success.begin(); _iter1949 != this->success.end(); ++_iter1949)
       {
-        xfer += (*_iter1950).write(oprot);
+        xfer += (*_iter1949).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -12117,14 +12117,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1951;
-            ::apache::thrift::protocol::TType _etype1954;
-            xfer += iprot->readListBegin(_etype1954, _size1951);
-            (*(this->success)).resize(_size1951);
-            uint32_t _i1955;
-            for (_i1955 = 0; _i1955 < _size1951; ++_i1955)
+            uint32_t _size1950;
+            ::apache::thrift::protocol::TType _etype1953;
+            xfer += iprot->readListBegin(_etype1953, _size1950);
+            (*(this->success)).resize(_size1950);
+            uint32_t _i1954;
+            for (_i1954 = 0; _i1954 < _size1950; ++_i1954)
             {
-              xfer += (*(this->success))[_i1955].read(iprot);
+              xfer += (*(this->success))[_i1954].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12262,14 +12262,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1956;
-            ::apache::thrift::protocol::TType _etype1959;
-            xfer += iprot->readListBegin(_etype1959, _size1956);
-            this->success.resize(_size1956);
-            uint32_t _i1960;
-            for (_i1960 = 0; _i1960 < _size1956; ++_i1960)
+            uint32_t _size1955;
+            ::apache::thrift::protocol::TType _etype1958;
+            xfer += iprot->readListBegin(_etype1958, _size1955);
+            this->success.resize(_size1955);
+            uint32_t _i1959;
+            for (_i1959 = 0; _i1959 < _size1955; ++_i1959)
             {
-              xfer += iprot->readString(this->success[_i1960]);
+              xfer += iprot->readString(this->success[_i1959]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12308,10 +12308,10 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1961;
-      for (_iter1961 = this->success.begin(); _iter1961 != this->success.end(); ++_iter1961)
+      std::vector<std::string> ::const_iterator _iter1960;
+      for (_iter1960 = this->success.begin(); _iter1960 != this->success.end(); ++_iter1960)
       {
-        xfer += oprot->writeString((*_iter1961));
+        xfer += oprot->writeString((*_iter1960));
       }
       xfer += oprot->writeListEnd();
     }
@@ -12356,14 +12356,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1962;
-            ::apache::thrift::protocol::TType _etype1965;
-            xfer += iprot->readListBegin(_etype1965, _size1962);
-            (*(this->success)).resize(_size1962);
-            uint32_t _i1966;
-            for (_i1966 = 0; _i1966 < _size1962; ++_i1966)
+            uint32_t _size1961;
+            ::apache::thrift::protocol::TType _etype1964;
+            xfer += iprot->readListBegin(_etype1964, _size1961);
+            (*(this->success)).resize(_size1961);
+            uint32_t _i1965;
+            for (_i1965 = 0; _i1965 < _size1961; ++_i1965)
             {
-              xfer += iprot->readString((*(this->success))[_i1966]);
+              xfer += iprot->readString((*(this->success))[_i1965]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12673,14 +12673,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_names.clear();
-            uint32_t _size1967;
-            ::apache::thrift::protocol::TType _etype1970;
-            xfer += iprot->readListBegin(_etype1970, _size1967);
-            this->tbl_names.resize(_size1967);
-            uint32_t _i1971;
-            for (_i1971 = 0; _i1971 < _size1967; ++_i1971)
+            uint32_t _size1966;
+            ::apache::thrift::protocol::TType _etype1969;
+            xfer += iprot->readListBegin(_etype1969, _size1966);
+            this->tbl_names.resize(_size1966);
+            uint32_t _i1970;
+            for (_i1970 = 0; _i1970 < _size1966; ++_i1970)
             {
-              xfer += iprot->readString(this->tbl_names[_i1971]);
+              xfer += iprot->readString(this->tbl_names[_i1970]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12713,10 +12713,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::write(::apache::thr
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_names.size()));
-    std::vector<std::string> ::const_iterator _iter1972;
-    for (_iter1972 = this->tbl_names.begin(); _iter1972 != this->tbl_names.end(); ++_iter1972)
+    std::vector<std::string> ::const_iterator _iter1971;
+    for (_iter1971 = this->tbl_names.begin(); _iter1971 != this->tbl_names.end(); ++_iter1971)
     {
-      xfer += oprot->writeString((*_iter1972));
+      xfer += oprot->writeString((*_iter1971));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12744,10 +12744,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_pargs::write(::apache::th
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_names)).size()));
-    std::vector<std::string> ::const_iterator _iter1973;
-    for (_iter1973 = (*(this->tbl_names)).begin(); _iter1973 != (*(this->tbl_names)).end(); ++_iter1973)
+    std::vector<std::string> ::const_iterator _iter1972;
+    for (_iter1972 = (*(this->tbl_names)).begin(); _iter1972 != (*(this->tbl_names)).end(); ++_iter1972)
     {
-      xfer += oprot->writeString((*_iter1973));
+      xfer += oprot->writeString((*_iter1972));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12788,14 +12788,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1974;
-            ::apache::thrift::protocol::TType _etype1977;
-            xfer += iprot->readListBegin(_etype1977, _size1974);
-            this->success.resize(_size1974);
-            uint32_t _i1978;
-            for (_i1978 = 0; _i1978 < _size1974; ++_i1978)
+            uint32_t _size1973;
+            ::apache::thrift::protocol::TType _etype1976;
+            xfer += iprot->readListBegin(_etype1976, _size1973);
+            this->success.resize(_size1973);
+            uint32_t _i1977;
+            for (_i1977 = 0; _i1977 < _size1973; ++_i1977)
             {
-              xfer += this->success[_i1978].read(iprot);
+              xfer += this->success[_i1977].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12826,10 +12826,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::write(::apache::t
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Table> ::const_iterator _iter1979;
-      for (_iter1979 = this->success.begin(); _iter1979 != this->success.end(); ++_iter1979)
+      std::vector<Table> ::const_iterator _iter1978;
+      for (_iter1978 = this->success.begin(); _iter1978 != this->success.end(); ++_iter1978)
       {
-        xfer += (*_iter1979).write(oprot);
+        xfer += (*_iter1978).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -12870,14 +12870,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1980;
-            ::apache::thrift::protocol::TType _etype1983;
-            xfer += iprot->readListBegin(_etype1983, _size1980);
-            (*(this->success)).resize(_size1980);
-            uint32_t _i1984;
-            for (_i1984 = 0; _i1984 < _size1980; ++_i1984)
+            uint32_t _size1979;
+            ::apache::thrift::protocol::TType _etype1982;
+            xfer += iprot->readListBegin(_etype1982, _size1979);
+            (*(this->success)).resize(_size1979);
+            uint32_t _i1983;
+            for (_i1983 = 0; _i1983 < _size1979; ++_i1983)
             {
-              xfer += (*(this->success))[_i1984].read(iprot);
+              xfer += (*(this->success))[_i1983].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13007,14 +13007,14 @@ uint32_t ThriftHiveMetastore_get_tables_ext_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1985;
-            ::apache::thrift::protocol::TType _etype1988;
-            xfer += iprot->readListBegin(_etype1988, _size1985);
-            this->success.resize(_size1985);
-            uint32_t _i1989;
-            for (_i1989 = 0; _i1989 < _size1985; ++_i1989)
+            uint32_t _size1984;
+            ::apache::thrift::protocol::TType _etype1987;
+            xfer += iprot->readListBegin(_etype1987, _size1984);
+            this->success.resize(_size1984);
+            uint32_t _i1988;
+            for (_i1988 = 0; _i1988 < _size1984; ++_i1988)
             {
-              xfer += this->success[_i1989].read(iprot);
+              xfer += this->success[_i1988].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13053,10 +13053,10 @@ uint32_t ThriftHiveMetastore_get_tables_ext_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<ExtendedTableInfo> ::const_iterator _iter1990;
-      for (_iter1990 = this->success.begin(); _iter1990 != this->success.end(); ++_iter1990)
+      std::vector<ExtendedTableInfo> ::const_iterator _iter1989;
+      for (_iter1989 = this->success.begin(); _iter1989 != this->success.end(); ++_iter1989)
       {
-        xfer += (*_iter1990).write(oprot);
+        xfer += (*_iter1989).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -13101,14 +13101,14 @@ uint32_t ThriftHiveMetastore_get_tables_ext_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1991;
-            ::apache::thrift::protocol::TType _etype1994;
-            xfer += iprot->readListBegin(_etype1994, _size1991);
-            (*(this->success)).resize(_size1991);
-            uint32_t _i1995;
-            for (_i1995 = 0; _i1995 < _size1991; ++_i1995)
+            uint32_t _size1990;
+            ::apache::thrift::protocol::TType _etype1993;
+            xfer += iprot->readListBegin(_etype1993, _size1990);
+            (*(this->success)).resize(_size1990);
+            uint32_t _i1994;
+            for (_i1994 = 0; _i1994 < _size1990; ++_i1994)
             {
-              xfer += (*(this->success))[_i1995].read(iprot);
+              xfer += (*(this->success))[_i1994].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14290,14 +14290,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1996;
-            ::apache::thrift::protocol::TType _etype1999;
-            xfer += iprot->readListBegin(_etype1999, _size1996);
-            this->success.resize(_size1996);
-            uint32_t _i2000;
-            for (_i2000 = 0; _i2000 < _size1996; ++_i2000)
+            uint32_t _size1995;
+            ::apache::thrift::protocol::TType _etype1998;
+            xfer += iprot->readListBegin(_etype1998, _size1995);
+            this->success.resize(_size1995);
+            uint32_t _i1999;
+            for (_i1999 = 0; _i1999 < _size1995; ++_i1999)
             {
-              xfer += iprot->readString(this->success[_i2000]);
+              xfer += iprot->readString(this->success[_i1999]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14352,10 +14352,10 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::write(::apache::t
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter2001;
-      for (_iter2001 = this->success.begin(); _iter2001 != this->success.end(); ++_iter2001)
+      std::vector<std::string> ::const_iterator _iter2000;
+      for (_iter2000 = this->success.begin(); _iter2000 != this->success.end(); ++_iter2000)
       {
-        xfer += oprot->writeString((*_iter2001));
+        xfer += oprot->writeString((*_iter2000));
       }
       xfer += oprot->writeListEnd();
     }
@@ -14408,14 +14408,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2002;
-            ::apache::thrift::protocol::TType _etype2005;
-            xfer += iprot->readListBegin(_etype2005, _size2002);
-            (*(this->success)).resize(_size2002);
-            uint32_t _i2006;
-            for (_i2006 = 0; _i2006 < _size2002; ++_i2006)
+            uint32_t _size2001;
+            ::apache::thrift::protocol::TType _etype2004;
+            xfer += iprot->readListBegin(_etype2004, _size2001);
+            (*(this->success)).resize(_size2001);
+            uint32_t _i2005;
+            for (_i2005 = 0; _i2005 < _size2001; ++_i2005)
             {
-              xfer += iprot->readString((*(this->success))[_i2006]);
+              xfer += iprot->readString((*(this->success))[_i2005]);
             }
             xfer += iprot->readListEnd();
           }
@@ -15976,14 +15976,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size2007;
-            ::apache::thrift::protocol::TType _etype2010;
-            xfer += iprot->readListBegin(_etype2010, _size2007);
-            this->new_parts.resize(_size2007);
-            uint32_t _i2011;
-            for (_i2011 = 0; _i2011 < _size2007; ++_i2011)
+            uint32_t _size2006;
+            ::apache::thrift::protocol::TType _etype2009;
+            xfer += iprot->readListBegin(_etype2009, _size2006);
+            this->new_parts.resize(_size2006);
+            uint32_t _i2010;
+            for (_i2010 = 0; _i2010 < _size2006; ++_i2010)
             {
-              xfer += this->new_parts[_i2011].read(iprot);
+              xfer += this->new_parts[_i2010].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -16012,10 +16012,10 @@ uint32_t ThriftHiveMetastore_add_partitions_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->new_parts.size()));
-    std::vector<Partition> ::const_iterator _iter2012;
-    for (_iter2012 = this->new_parts.begin(); _iter2012 != this->new_parts.end(); ++_iter2012)
+    std::vector<Partition> ::const_iterator _iter2011;
+    for (_iter2011 = this->new_parts.begin(); _iter2011 != this->new_parts.end(); ++_iter2011)
     {
-      xfer += (*_iter2012).write(oprot);
+      xfer += (*_iter2011).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -16039,10 +16039,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->new_parts)).size()));
-    std::vector<Partition> ::const_iterator _iter2013;
-    for (_iter2013 = (*(this->new_parts)).begin(); _iter2013 != (*(this->new_parts)).end(); ++_iter2013)
+    std::vector<Partition> ::const_iterator _iter2012;
+    for (_iter2012 = (*(this->new_parts)).begin(); _iter2012 != (*(this->new_parts)).end(); ++_iter2012)
     {
-      xfer += (*_iter2013).write(oprot);
+      xfer += (*_iter2012).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -16251,14 +16251,14 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_args::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size2014;
-            ::apache::thrift::protocol::TType _etype2017;
-            xfer += iprot->readListBegin(_etype2017, _size2014);
-            this->new_parts.resize(_size2014);
-            uint32_t _i2018;
-            for (_i2018 = 0; _i2018 < _size2014; ++_i2018)
+            uint32_t _size2013;
+            ::apache::thrift::protocol::TType _etype2016;
+            xfer += iprot->readListBegin(_etype2016, _size2013);
+            this->new_parts.resize(_size2013);
+            uint32_t _i2017;
+            for (_i2017 = 0; _i2017 < _size2013; ++_i2017)
             {
-              xfer += this->new_parts[_i2018].read(iprot);
+              xfer += this->new_parts[_i2017].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -16287,10 +16287,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_args::write(::apache::thrift::
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->new_parts.size()));
-    std::vector<PartitionSpec> ::const_iterator _iter2019;
-    for (_iter2019 = this->new_parts.begin(); _iter2019 != this->new_parts.end(); ++_iter2019)
+    std::vector<PartitionSpec> ::const_iterator _iter2018;
+    for (_iter2018 = this->new_parts.begin(); _iter2018 != this->new_parts.end(); ++_iter2018)
     {
-      xfer += (*_iter2019).write(oprot);
+      xfer += (*_iter2018).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -16314,10 +16314,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_pargs::write(::apache::thrift:
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->new_parts)).size()));
-    std::vector<PartitionSpec> ::const_iterator _iter2020;
-    for (_iter2020 = (*(this->new_parts)).begin(); _iter2020 != (*(this->new_parts)).end(); ++_iter2020)
+    std::vector<PartitionSpec> ::const_iterator _iter2019;
+    for (_iter2019 = (*(this->new_parts)).begin(); _iter2019 != (*(this->new_parts)).end(); ++_iter2019)
     {
-      xfer += (*_iter2020).write(oprot);
+      xfer += (*_iter2019).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -16542,14 +16542,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size2021;
-            ::apache::thrift::protocol::TType _etype2024;
-            xfer += iprot->readListBegin(_etype2024, _size2021);
-            this->part_vals.resize(_size2021);
-            uint32_t _i2025;
-            for (_i2025 = 0; _i2025 < _size2021; ++_i2025)
+            uint32_t _size2020;
+            ::apache::thrift::protocol::TType _etype2023;
+            xfer += iprot->readListBegin(_etype2023, _size2020);
+            this->part_vals.resize(_size2020);
+            uint32_t _i2024;
+            for (_i2024 = 0; _i2024 < _size2020; ++_i2024)
             {
-              xfer += iprot->readString(this->part_vals[_i2025]);
+              xfer += iprot->readString(this->part_vals[_i2024]);
             }
             xfer += iprot->readListEnd();
           }
@@ -16586,10 +16586,10 @@ uint32_t ThriftHiveMetastore_append_partition_args::write(::apache::thrift::prot
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter2026;
-    for (_iter2026 = this->part_vals.begin(); _iter2026 != this->part_vals.end(); ++_iter2026)
+    std::vector<std::string> ::const_iterator _iter2025;
+    for (_iter2025 = this->part_vals.begin(); _iter2025 != this->part_vals.end(); ++_iter2025)
     {
-      xfer += oprot->writeString((*_iter2026));
+      xfer += oprot->writeString((*_iter2025));
     }
     xfer += oprot->writeListEnd();
   }
@@ -16621,10 +16621,10 @@ uint32_t ThriftHiveMetastore_append_partition_pargs::write(::apache::thrift::pro
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter2027;
-    for (_iter2027 = (*(this->part_vals)).begin(); _iter2027 != (*(this->part_vals)).end(); ++_iter2027)
+    std::vector<std::string> ::const_iterator _iter2026;
+    for (_iter2026 = (*(this->part_vals)).begin(); _iter2026 != (*(this->part_vals)).end(); ++_iter2026)
     {
-      xfer += oprot->writeString((*_iter2027));
+      xfer += oprot->writeString((*_iter2026));
     }
     xfer += oprot->writeListEnd();
   }
@@ -17096,14 +17096,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size2028;
-            ::apache::thrift::protocol::TType _etype2031;
-            xfer += iprot->readListBegin(_etype2031, _size2028);
-            this->part_vals.resize(_size2028);
-            uint32_t _i2032;
-            for (_i2032 = 0; _i2032 < _size2028; ++_i2032)
+            uint32_t _size2027;
+            ::apache::thrift::protocol::TType _etype2030;
+            xfer += iprot->readListBegin(_etype2030, _size2027);
+            this->part_vals.resize(_size2027);
+            uint32_t _i2031;
+            for (_i2031 = 0; _i2031 < _size2027; ++_i2031)
             {
-              xfer += iprot->readString(this->part_vals[_i2032]);
+              xfer += iprot->readString(this->part_vals[_i2031]);
             }
             xfer += iprot->readListEnd();
           }
@@ -17148,10 +17148,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::wri
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter2033;
-    for (_iter2033 = this->part_vals.begin(); _iter2033 != this->part_vals.end(); ++_iter2033)
+    std::vector<std::string> ::const_iterator _iter2032;
+    for (_iter2032 = this->part_vals.begin(); _iter2032 != this->part_vals.end(); ++_iter2032)
     {
-      xfer += oprot->writeString((*_iter2033));
+      xfer += oprot->writeString((*_iter2032));
     }
     xfer += oprot->writeListEnd();
   }
@@ -17187,10 +17187,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_pargs::wr
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter2034;
-    for (_iter2034 = (*(this->part_vals)).begin(); _iter2034 != (*(this->part_vals)).end(); ++_iter2034)
+    std::vector<std::string> ::const_iterator _iter2033;
+    for (_iter2033 = (*(this->part_vals)).begin(); _iter2033 != (*(this->part_vals)).end(); ++_iter2033)
     {
-      xfer += oprot->writeString((*_iter2034));
+      xfer += oprot->writeString((*_iter2033));
     }
     xfer += oprot->writeListEnd();
   }
@@ -17993,14 +17993,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size2035;
-            ::apache::thrift::protocol::TType _etype2038;
-            xfer += iprot->readListBegin(_etype2038, _size2035);
-            this->part_vals.resize(_size2035);
-            uint32_t _i2039;
-            for (_i2039 = 0; _i2039 < _size2035; ++_i2039)
+            uint32_t _size2034;
+            ::apache::thrift::protocol::TType _etype2037;
+            xfer += iprot->readListBegin(_etype2037, _size2034);
+            this->part_vals.resize(_size2034);
+            uint32_t _i2038;
+            for (_i2038 = 0; _i2038 < _size2034; ++_i2038)
             {
-              xfer += iprot->readString(this->part_vals[_i2039]);
+              xfer += iprot->readString(this->part_vals[_i2038]);
             }
             xfer += iprot->readListEnd();
           }
@@ -18045,10 +18045,10 @@ uint32_t ThriftHiveMetastore_drop_partition_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter2040;
-    for (_iter2040 = this->part_vals.begin(); _iter2040 != this->part_vals.end(); ++_iter2040)
+    std::vector<std::string> ::const_iterator _iter2039;
+    for (_iter2039 = this->part_vals.begin(); _iter2039 != this->part_vals.end(); ++_iter2039)
     {
-      xfer += oprot->writeString((*_iter2040));
+      xfer += oprot->writeString((*_iter2039));
     }
     xfer += oprot->writeListEnd();
   }
@@ -18084,10 +18084,10 @@ uint32_t ThriftHiveMetastore_drop_partition_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter2041;
-    for (_iter2041 = (*(this->part_vals)).begin(); _iter2041 != (*(this->part_vals)).end(); ++_iter2041)
+    std::vector<std::string> ::const_iterator _iter2040;
+    for (_iter2040 = (*(this->part_vals)).begin(); _iter2040 != (*(this->part_vals)).end(); ++_iter2040)
     {
-      xfer += oprot->writeString((*_iter2041));
+      xfer += oprot->writeString((*_iter2040));
     }
     xfer += oprot->writeListEnd();
   }
@@ -18296,14 +18296,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size2042;
-            ::apache::thrift::protocol::TType _etype2045;
-            xfer += iprot->readListBegin(_etype2045, _size2042);
-            this->part_vals.resize(_size2042);
-            uint32_t _i2046;
-            for (_i2046 = 0; _i2046 < _size2042; ++_i2046)
+            uint32_t _size2041;
+            ::apache::thrift::protocol::TType _etype2044;
+            xfer += iprot->readListBegin(_etype2044, _size2041);
+            this->part_vals.resize(_size2041);
+            uint32_t _i2045;
+            for (_i2045 = 0; _i2045 < _size2041; ++_i2045)
             {
-              xfer += iprot->readString(this->part_vals[_i2046]);
+              xfer += iprot->readString(this->part_vals[_i2045]);
             }
             xfer += iprot->readListEnd();
           }
@@ -18356,10 +18356,10 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::write
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter2047;
-    for (_iter2047 = this->part_vals.begin(); _iter2047 != this->part_vals.end(); ++_iter2047)
+    std::vector<std::string> ::const_iterator _iter2046;
+    for (_iter2046 = this->part_vals.begin(); _iter2046 != this->part_vals.end(); ++_iter2046)
     {
-      xfer += oprot->writeString((*_iter2047));
+      xfer += oprot->writeString((*_iter2046));
     }
     xfer += oprot->writeListEnd();
   }
@@ -18399,10 +18399,10 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_pargs::writ
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter2048;
-    for (_iter2048 = (*(this->part_vals)).begin(); _iter2048 != (*(this->part_vals)).end(); ++_iter2048)
+    std::vector<std::string> ::const_iterator _iter2047;
+    for (_iter2047 = (*(this->part_vals)).begin(); _iter2047 != (*(this->part_vals)).end(); ++_iter2047)
     {
-      xfer += oprot->writeString((*_iter2048));
+      xfer += oprot->writeString((*_iter2047));
     }
     xfer += oprot->writeListEnd();
   }
@@ -19408,14 +19408,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size2049;
-            ::apache::thrift::protocol::TType _etype2052;
-            xfer += iprot->readListBegin(_etype2052, _size2049);
-            this->part_vals.resize(_size2049);
-            uint32_t _i2053;
-            for (_i2053 = 0; _i2053 < _size2049; ++_i2053)
+            uint32_t _size2048;
+            ::apache::thrift::protocol::TType _etype2051;
+            xfer += iprot->readListBegin(_etype2051, _size2048);
+            this->part_vals.resize(_size2048);
+            uint32_t _i2052;
+            for (_i2052 = 0; _i2052 < _size2048; ++_i2052)
             {
-              xfer += iprot->readString(this->part_vals[_i2053]);
+              xfer += iprot->readString(this->part_vals[_i2052]);
             }
             xfer += iprot->readListEnd();
           }
@@ -19452,10 +19452,10 @@ uint32_t ThriftHiveMetastore_get_partition_args::write(::apache::thrift::protoco
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter2054;
-    for (_iter2054 = this->part_vals.begin(); _iter2054 != this->part_vals.end(); ++_iter2054)
+    std::vector<std::string> ::const_iterator _iter2053;
+    for (_iter2053 = this->part_vals.begin(); _iter2053 != this->part_vals.end(); ++_iter2053)
     {
-      xfer += oprot->writeString((*_iter2054));
+      xfer += oprot->writeString((*_iter2053));
     }
     xfer += oprot->writeListEnd();
   }
@@ -19487,10 +19487,10 @@ uint32_t ThriftHiveMetastore_get_partition_pargs::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter2055;
-    for (_iter2055 = (*(this->part_vals)).begin(); _iter2055 != (*(this->part_vals)).end(); ++_iter2055)
+    std::vector<std::string> ::const_iterator _iter2054;
+    for (_iter2054 = (*(this->part_vals)).begin(); _iter2054 != (*(this->part_vals)).end(); ++_iter2054)
     {
-      xfer += oprot->writeString((*_iter2055));
+      xfer += oprot->writeString((*_iter2054));
     }
     xfer += oprot->writeListEnd();
   }
@@ -19906,17 +19906,17 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size2056;
-            ::apache::thrift::protocol::TType _ktype2057;
-            ::apache::thrift::protocol::TType _vtype2058;
-            xfer += iprot->readMapBegin(_ktype2057, _vtype2058, _size2056);
-            uint32_t _i2060;
-            for (_i2060 = 0; _i2060 < _size2056; ++_i2060)
+            uint32_t _size2055;
+            ::apache::thrift::protocol::TType _ktype2056;
+            ::apache::thrift::protocol::TType _vtype2057;
+            xfer += iprot->readMapBegin(_ktype2056, _vtype2057, _size2055);
+            uint32_t _i2059;
+            for (_i2059 = 0; _i2059 < _size2055; ++_i2059)
             {
-              std::string _key2061;
-              xfer += iprot->readString(_key2061);
-              std::string& _val2062 = this->partitionSpecs[_key2061];
-              xfer += iprot->readString(_val2062);
+              std::string _key2060;
+              xfer += iprot->readString(_key2060);
+              std::string& _val2061 = this->partitionSpecs[_key2060];
+              xfer += iprot->readString(_val2061);
             }
             xfer += iprot->readMapEnd();
           }
@@ -19977,11 +19977,11 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::write(::apache::thrift::pr
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionSpecs.size()));
-    std::map<std::string, std::string> ::const_iterator _iter2063;
-    for (_iter2063 = this->partitionSpecs.begin(); _iter2063 != this->partitionSpecs.end(); ++_iter2063)
+    std::map<std::string, std::string> ::const_iterator _iter2062;
+    for (_iter2062 = this->partitionSpecs.begin(); _iter2062 != this->partitionSpecs.end(); ++_iter2062)
     {
-      xfer += oprot->writeString(_iter2063->first);
-      xfer += oprot->writeString(_iter2063->second);
+      xfer += oprot->writeString(_iter2062->first);
+      xfer += oprot->writeString(_iter2062->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -20021,11 +20021,11 @@ uint32_t ThriftHiveMetastore_exchange_partition_pargs::write(::apache::thrift::p
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partitionSpecs)).size()));
-    std::map<std::string, std::string> ::const_iterator _iter2064;
-    for (_iter2064 = (*(this->partitionSpecs)).begin(); _iter2064 != (*(this->partitionSpecs)).end(); ++_iter2064)
+    std::map<std::string, std::string> ::const_iterator _iter2063;
+    for (_iter2063 = (*(this->partitionSpecs)).begin(); _iter2063 != (*(this->partitionSpecs)).end(); ++_iter2063)
     {
-      xfer += oprot->writeString(_iter2064->first);
-      xfer += oprot->writeString(_iter2064->second);
+      xfer += oprot->writeString(_iter2063->first);
+      xfer += oprot->writeString(_iter2063->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -20270,17 +20270,17 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size2065;
-            ::apache::thrift::protocol::TType _ktype2066;
-            ::apache::thrift::protocol::TType _vtype2067;
-            xfer += iprot->readMapBegin(_ktype2066, _vtype2067, _size2065);
-            uint32_t _i2069;
-            for (_i2069 = 0; _i2069 < _size2065; ++_i2069)
+            uint32_t _size2064;
+            ::apache::thrift::protocol::TType _ktype2065;
+            ::apache::thrift::protocol::TType _vtype2066;
+            xfer += iprot->readMapBegin(_ktype2065, _vtype2066, _size2064);
+            uint32_t _i2068;
+            for (_i2068 = 0; _i2068 < _size2064; ++_i2068)
             {
-              std::string _key2070;
-              xfer += iprot->readString(_key2070);
-              std::string& _val2071 = this->partitionSpecs[_key2070];
-              xfer += iprot->readString(_val2071);
+              std::string _key2069;
+              xfer += iprot->readString(_key2069);
+              std::string& _val2070 = this->partitionSpecs[_key2069];
+              xfer += iprot->readString(_val2070);
             }
             xfer += iprot->readMapEnd();
           }
@@ -20341,11 +20341,11 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::write(::apache::thrift::p
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionSpecs.size()));
-    std::map<std::string, std::string> ::const_iterator _iter2072;
-    for (_iter2072 = this->partitionSpecs.begin(); _iter2072 != this->partitionSpecs.end(); ++_iter2072)
+    std::map<std::string, std::string> ::const_iterator _iter2071;
+    for (_iter2071 = this->partitionSpecs.begin(); _iter2071 != this->partitionSpecs.end(); ++_iter2071)
     {
-      xfer += oprot->writeString(_iter2072->first);
-      xfer += oprot->writeString(_iter2072->second);
+      xfer += oprot->writeString(_iter2071->first);
+      xfer += oprot->writeString(_iter2071->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -20385,11 +20385,11 @@ uint32_t ThriftHiveMetastore_exchange_partitions_pargs::write(::apache::thrift::
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partitionSpecs)).size()));
-    std::map<std::string, std::string> ::const_iterator _iter2073;
-    for (_iter2073 = (*(this->partitionSpecs)).begin(); _iter2073 != (*(this->partitionSpecs)).end(); ++_iter2073)
+    std::map<std::string, std::string> ::const_iterator _iter2072;
+    for (_iter2072 = (*(this->partitionSpecs)).begin(); _iter2072 != (*(this->partitionSpecs)).end(); ++_iter2072)
     {
-      xfer += oprot->writeString(_iter2073->first);
-      xfer += oprot->writeString(_iter2073->second);
+      xfer += oprot->writeString(_iter2072->first);
+      xfer += oprot->writeString(_iter2072->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -20446,14 +20446,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2074;
-            ::apache::thrift::protocol::TType _etype2077;
-            xfer += iprot->readListBegin(_etype2077, _size2074);
-            this->success.resize(_size2074);
-            uint32_t _i2078;
-            for (_i2078 = 0; _i2078 < _size2074; ++_i2078)
+            uint32_t _size2073;
+            ::apache::thrift::protocol::TType _etype2076;
+            xfer += iprot->readListBegin(_etype2076, _size2073);
+            this->success.resize(_size2073);
+            uint32_t _i2077;
+            for (_i2077 = 0; _i2077 < _size2073; ++_i2077)
             {
-              xfer += this->success[_i2078].read(iprot);
+              xfer += this->success[_i2077].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -20516,10 +20516,10 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::write(::apache::thrift:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter2079;
-      for (_iter2079 = this->success.begin(); _iter2079 != this->success.end(); ++_iter2079)
+      std::vector<Partition> ::const_iterator _iter2078;
+      for (_iter2078 = this->success.begin(); _iter2078 != this->success.end(); ++_iter2078)
       {
-        xfer += (*_iter2079).write(oprot);
+        xfer += (*_iter2078).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -20576,14 +20576,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_presult::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2080;
-            ::apache::thrift::protocol::TType _etype2083;
-            xfer += iprot->readListBegin(_etype2083, _size2080);
-            (*(this->success)).resize(_size2080);
-            uint32_t _i2084;
-            for (_i2084 = 0; _i2084 < _size2080; ++_i2084)
+            uint32_t _size2079;
+            ::apache::thrift::protocol::TType _etype2082;
+            xfer += iprot->readListBegin(_etype2082, _size2079);
+            (*(this->success)).resize(_size2079);
+            uint32_t _i2083;
+            for (_i2083 = 0; _i2083 < _size2079; ++_i2083)
             {
-              xfer += (*(this->success))[_i2084].read(iprot);
+              xfer += (*(this->success))[_i2083].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -20682,14 +20682,14 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size2085;
-            ::apache::thrift::protocol::TType _etype2088;
-            xfer += iprot->readListBegin(_etype2088, _size2085);
-            this->part_vals.resize(_size2085);
-            uint32_t _i2089;
-            for (_i2089 = 0; _i2089 < _size2085; ++_i2089)
+            uint32_t _size2084;
+            ::apache::thrift::protocol::TType _etype2087;
+            xfer += iprot->readListBegin(_etype2087, _size2084);
+            this->part_vals.resize(_size2084);
+            uint32_t _i2088;
+            for (_i2088 = 0; _i2088 < _size2084; ++_i2088)
             {
-              xfer += iprot->readString(this->part_vals[_i2089]);
+              xfer += iprot->readString(this->part_vals[_i2088]);
             }
             xfer += iprot->readListEnd();
           }
@@ -20710,14 +20710,14 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size2090;
-            ::apache::thrift::protocol::TType _etype2093;
-            xfer += iprot->readListBegin(_etype2093, _size2090);
-            this->group_names.resize(_size2090);
-            uint32_t _i2094;
-            for (_i2094 = 0; _i2094 < _size2090; ++_i2094)
+            uint32_t _size2089;
+            ::apache::thrift::protocol::TType _etype2092;
+            xfer += iprot->readListBegin(_etype2092, _size2089);
+            this->group_names.resize(_size2089);
+            uint32_t _i2093;
+            for (_i2093 = 0; _i2093 < _size2089; ++_i2093)
             {
-              xfer += iprot->readString(this->group_names[_i2094]);
+              xfer += iprot->readString(this->group_names[_i2093]);
             }
             xfer += iprot->readListEnd();
           }
@@ -20754,10 +20754,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrif
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter2095;
-    for (_iter2095 = this->part_vals.begin(); _iter2095 != this->part_vals.end(); ++_iter2095)
+    std::vector<std::string> ::const_iterator _iter2094;
+    for (_iter2094 = this->part_vals.begin(); _iter2094 != this->part_vals.end(); ++_iter2094)
     {
-      xfer += oprot->writeString((*_iter2095));
+      xfer += oprot->writeString((*_iter2094));
     }
     xfer += oprot->writeListEnd();
   }
@@ -20770,10 +20770,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrif
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter2096;
-    for (_iter2096 = this->group_names.begin(); _iter2096 != this->group_names.end(); ++_iter2096)
+    std::vector<std::string> ::const_iterator _iter2095;
+    for (_iter2095 = this->group_names.begin(); _iter2095 != this->group_names.end(); ++_iter2095)
     {
-      xfer += oprot->writeString((*_iter2096));
+      xfer += oprot->writeString((*_iter2095));
     }
     xfer += oprot->writeListEnd();
   }
@@ -20805,10 +20805,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thri
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter2097;
-    for (_iter2097 = (*(this->part_vals)).begin(); _iter2097 != (*(this->part_vals)).end(); ++_iter2097)
+    std::vector<std::string> ::const_iterator _iter2096;
+    for (_iter2096 = (*(this->part_vals)).begin(); _iter2096 != (*(this->part_vals)).end(); ++_iter2096)
     {
-      xfer += oprot->writeString((*_iter2097));
+      xfer += oprot->writeString((*_iter2096));
     }
     xfer += oprot->writeListEnd();
   }
@@ -20821,10 +20821,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thri
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter2098;
-    for (_iter2098 = (*(this->group_names)).begin(); _iter2098 != (*(this->group_names)).end(); ++_iter2098)
+    std::vector<std::string> ::const_iterator _iter2097;
+    for (_iter2097 = (*(this->group_names)).begin(); _iter2097 != (*(this->group_names)).end(); ++_iter2097)
     {
-      xfer += oprot->writeString((*_iter2098));
+      xfer += oprot->writeString((*_iter2097));
     }
     xfer += oprot->writeListEnd();
   }
@@ -21383,14 +21383,14 @@ uint32_t ThriftHiveMetastore_get_partitions_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _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 _size2098;
+            ::apache::thrift::protocol::TType _etype2101;
+            xfer += iprot->readListBegin(_etype2101, _size2098);
+            this->success.resize(_size2098);
+            uint32_t _i2102;
+            for (_i2102 = 0; _i2102 < _size2098; ++_i2102)
             {
-              xfer += this->success[_i2103].read(iprot);
+              xfer += this->success[_i2102].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -21437,10 +21437,10 @@ uint32_t ThriftHiveMetastore_get_partitions_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter2104;
-      for (_iter2104 = this->success.begin(); _iter2104 != this->success.end(); ++_iter2104)
+      std::vector<Partition> ::const_iterator _iter2103;
+      for (_iter2103 = this->success.begin(); _iter2103 != this->success.end(); ++_iter2103)
       {
-        xfer += (*_iter2104).write(oprot);
+        xfer += (*_iter2103).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -21489,14 +21489,14 @@ uint32_t ThriftHiveMetastore_get_partitions_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _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 _size2104;
+            ::apache::thrift::protocol::TType _etype2107;
+            xfer += iprot->readListBegin(_etype2107, _size2104);
+            (*(this->success)).resize(_size2104);
+            uint32_t _i2108;
+            for (_i2108 = 0; _i2108 < _size2104; ++_i2108)
             {
-              xfer += (*(this->success))[_i2109].read(iprot);
+              xfer += (*(this->success))[_i2108].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -21822,14 +21822,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::read(::apache::thrif
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size2110;
-            ::apache::thrift::protocol::TType _etype2113;
-            xfer += iprot->readListBegin(_etype2113, _size2110);
-            this->group_names.resize(_size2110);
-            uint32_t _i2114;
-            for (_i2114 = 0; _i2114 < _size2110; ++_i2114)
+            uint32_t _size2109;
+            ::apache::thrift::protocol::TType _etype2112;
+            xfer += iprot->readListBegin(_etype2112, _size2109);
+            this->group_names.resize(_size2109);
+            uint32_t _i2113;
+            for (_i2113 = 0; _i2113 < _size2109; ++_i2113)
             {
-              xfer += iprot->readString(this->group_names[_i2114]);
+              xfer += iprot->readString(this->group_names[_i2113]);
             }
             xfer += iprot->readListEnd();
           }
@@ -21874,10 +21874,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::write(::apache::thri
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter2115;
-    for (_iter2115 = this->group_names.begin(); _iter2115 != this->group_names.end(); ++_iter2115)
+    std::vector<std::string> ::const_iterator _iter2114;
+    for (_iter2114 = this->group_names.begin(); _iter2114 != this->group_names.end(); ++_iter2114)
     {
-      xfer += oprot->writeString((*_iter2115));
+      xfer += oprot->writeString((*_iter2114));
     }
     xfer += oprot->writeListEnd();
   }
@@ -21917,10 +21917,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_pargs::write(::apache::thr
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter2116;
-    for (_iter2116 = (*(this->group_names)).begin(); _iter2116 != (*(this->group_names)).end(); ++_iter2116)
+    std::vector<std::string> ::const_iterator _iter2115;
+    for (_iter2115 = (*(this->group_names)).begin(); _iter2115 != (*(this->group_names)).end(); ++_iter2115)
     {
-      xfer += oprot->writeString((*_iter2116));
+      xfer += oprot->writeString((*_iter2115));
     }
     xfer += oprot->writeListEnd();
   }
@@ -21961,14 +21961,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2117;
-            ::apache::thrift::protocol::TType _etype2120;
-            xfer += iprot->readListBegin(_etype2120, _size2117);
-            this->success.resize(_size2117);
-            uint32_t _i2121;
-            for (_i2121 = 0; _i2121 < _size2117; ++_i2121)
+            uint32_t _size2116;
+            ::apache::thrift::protocol::TType _etype2119;
+            xfer += iprot->readListBegin(_etype2119, _size2116);
+            this->success.resize(_size2116);
+            uint32_t _i2120;
+            for (_i2120 = 0; _i2120 < _size2116; ++_i2120)
             {
-              xfer += this->success[_i2121].read(iprot);
+              xfer += this->success[_i2120].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -22015,10 +22015,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::write(::apache::th
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter2122;
-      for (_iter2122 = this->success.begin(); _iter2122 != this->success.end(); ++_iter2122)
+      std::vector<Partition> ::const_iterator _iter2121;
+      for (_iter2121 = this->success.begin(); _iter2121 != this->success.end(); ++_iter2121)
       {
-        xfer += (*_iter2122).write(oprot);
+        xfer += (*_iter2121).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -22067,14 +22067,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_presult::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2123;
-            ::apache::thrift::protocol::TType _etype2126;
-            xfer += iprot->readListBegin(_etype2126, _size2123);
-            (*(this->success)).resize(_size2123);
-            uint32_t _i2127;
-            for (_i2127 = 0; _i2127 < _size2123; ++_i2127)
+            uint32_t _size2122;
+            ::apache::thrift::protocol::TType _etype2125;
+            xfer += iprot->readListBegin(_etype2125, _size2122);
+            (*(this->success)).resize(_size2122);
+            uint32_t _i2126;
+            for (_i2126 = 0; _i2126 < _size2122; ++_i2126)
             {
-              xfer += (*(this->success))[_i2127].read(iprot);
+              xfer += (*(this->success))[_i2126].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -22252,14 +22252,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2128;
-            ::apache::thrift::protocol::TType _etype2131;
-            xfer += iprot->readListBegin(_etype2131, _size2128);
-            this->success.resize(_size2128);
-            uint32_t _i2132;
-            for (_i2132 = 0; _i2132 < _size2128; ++_i2132)
+            uint32_t _size2127;
+            ::apache::thrift::protocol::TType _etype2130;
+            xfer += iprot->readListBegin(_etype2130, _size2127);
+            this->success.resize(_size2127);
+            uint32_t _i2131;
+            for (_i2131 = 0; _i2131 < _size2127; ++_i2131)
             {
-              xfer += this->success[_i2132].read(iprot);
+              xfer += this->success[_i2131].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -22306,10 +22306,10 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::write(::apache::thrift
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<PartitionSpec> ::const_iterator _iter2133;
-      for (_iter2133 = this->success.begin(); _iter2133 != this->success.end(); ++_iter2133)
+      std::vector<PartitionSpec> ::const_iterator _iter2132;
+      for (_iter2132 = this->success.begin(); _iter2132 != this->success.end(); ++_iter2132)
       {
-        xfer += (*_iter2133).write(oprot);
+        xfer += (*_iter2132).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -22358,14 +22358,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_presult::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2134;
-            ::apache::thrift::protocol::TType _etype2137;
-            xfer += iprot->readListBegin(_etype2137, _size2134);
-            (*(this->success)).resize(_size2134);
-            uint32_t _i2138;
-            for (_i2138 = 0; _i2138 < _size2134; ++_i2138)
+            uint32_t _size2133;
+            ::apache::thrift::protocol::TType _etype2136;
+            xfer += iprot->readListBegin(_etype2136, _size2133);
+            (*(this->success)).resize(_size2133);
+            uint32_t _i2137;
+            for (_i2137 = 0; _i2137 < _size2133; ++_i2137)
             {
-              xfer += (*(this->success))[_i2138].read(iprot);
+              xfer += (*(this->success))[_i2137].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -22543,14 +22543,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2139;
-            ::apache::thrift::protocol::TType _etype2142;
-            xfer += iprot->readListBegin(_etype2142, _size2139);
-            this->success.resize(_size2139);
-            uint32_t _i2143;
-            for (_i2143 = 0; _i2143 < _size2139; ++_i2143)
+            uint32_t _size2138;
+            ::apache::thrift::protocol::TType _etype2141;
+            xfer += iprot->readListBegin(_etype2141, _size2138);
+            this->success.resize(_size2138);
+            uint32_t _i2142;
+            for (_i2142 = 0; _i2142 < _size2138; ++_i2142)
             {
-              xfer += iprot->readString(this->success[_i2143]);
+              xfer += iprot->readString(this->success[_i2142]);
             }
             xfer += iprot->readListEnd();
           }
@@ -22597,10 +22597,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::write(::apache::thrift:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter2144;
-      for (_iter2144 = this->success.begin(); _iter2144 != this->success.end(); ++_iter2144)
+      std::vector<std::string> ::const_iterator _iter2143;
+      for (_iter2143 = this->success.begin(); _iter2143 != this->success.end(); ++_iter2143)
       {
-        xfer += oprot->writeString((*_iter2144));
+        xfer += oprot->writeString((*_iter2143));
       }
       xfer += oprot->writeListEnd();
     }
@@ -22649,14 +22649,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_presult::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2145;
-            ::apache::thrift::protocol::TType _etype2148;
-            xfer += iprot->readListBegin(_etype2148, _size2145);
-            (*(this->success)).resize(_size2145);
-            uint32_t _i2149;
-            for (_i2149 = 0; _i2149 < _size2145; ++_i2149)
+            uint32_t _size2144;
+            ::apache::thrift::protocol::TType _etype2147;
+            xfer += iprot->readListBegin(_etype2147, _size2144);
+            (*(this->success)).resize(_size2144);
+            uint32_t _i2148;
+            for (_i2148 = 0; _i2148 < _size2144; ++_i2148)
             {
-              xfer += iprot->readString((*(this->success))[_i2149]);
+              xfer += iprot->readString((*(this->success))[_i2148]);
             }
             xfer += iprot->readListEnd();
           }
@@ -22966,14 +22966,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_args::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size2150;
-            ::apache::thrift::protocol::TType _etype2153;
-            xfer += iprot->readListBegin(_etype2153, _size2150);
-            this->part_vals.resize(_size2150);
-            uint32_t _i2154;
-            for (_i2154 = 0; _i2154 < _size2150; ++_i2154)
+            uint32_t _size2149;
+            ::apache::thrift::protocol::TType _etype2152;
+            xfer += iprot->readListBegin(_etype2152, _size2149);
+            this->part_vals.resize(_size2149);
+            uint32_t _i2153;
+            for (_i2153 = 0; _i2153 < _size2149; ++_i2153)
             {
-              xfer += iprot->readString(this->part_vals[_i2154]);
+              xfer += iprot->readString(this->part_vals[_i2153]);
             }
             xfer += iprot->readListEnd();
           }
@@ -23018,10 +23018,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_args::write(::apache::thrift::pro
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter2155;
-    for (_iter2155 = this->part_vals.begin(); _iter2155 != this->part_vals.end(); ++_iter2155)
+    std::vector<std::string> ::const_iterator _iter2154;
+    for (_iter2154 = this->part_vals.begin(); _iter2154 != this->part_vals.end(); ++_iter2154)
     {
-      xfer += oprot->writeString((*_iter2155));
+      xfer += oprot->writeString((*_iter2154));
     }
     xfer += oprot->writeListEnd();
   }
@@ -23057,10 +23057,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_pargs::write(::apache::thrift::pr
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter2156;
-    for (_iter2156 = (*(this->part_vals)).begin(); _iter2156 != (*(this->part_vals)).end(); ++_iter2156)
+    std::vector<std::string> ::const_iterator _iter2155;
+    for (_iter2155 = (*(this->part_vals)).begin(); _iter2155 != (*(this->part_vals)).end(); ++_iter2155)
     {
-      xfer += oprot->writeString((*_iter2156));
+      xfer += oprot->writeString((*_iter2155));
     }
     xfer += oprot->writeListEnd();
   }
@@ -23105,14 +23105,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2157;
-            ::apache::thrift::protocol::TType _etype2160;
-            xfer += iprot->readListBegin(_etype2160, _size2157);
-            this->success.resize(_size2157);
-            uint32_t _i2161;
-            for (_i2161 = 0; _i2161 < _size2157; ++_i2161)
+            uint32_t _size2156;
+            ::apache::thrift::protocol::TType _etype2159;
+            xfer += iprot->readListBegin(_etype2159, _size2156);
+            this->success.resize(_size2156);
+            uint32_t _i2160;
+            for (_i2160 = 0; _i2160 < _size2156; ++_i2160)
             {
-              xfer += this->success[_i2161].read(iprot);
+              xfer += this->success[_i2160].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -23159,10 +23159,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_result::write(::apache::thrift::p
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter2162;
-      for (_iter2162 = this->success.begin(); _iter2162 != this->success.end(); ++_iter2162)
+      std::vector<Partition> ::const_iterator _iter2161;
+      for (_iter2161 = this->success.begin(); _iter2161 != this->success.end(); ++_iter2161)
       {
-        xfer += (*_iter2162).write(oprot);
+        xfer += (*_iter2161).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -23211,14 +23211,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2163;
-            ::apache::thrift::protocol::TType _etype2166;
-            xfer += iprot->readListBegin(_etype2166, _size2163);
-            (*(this->success)).resize(_size2163);
-            uint32_t _i2167;
-            for (_i2167 = 0; _i2167 < _size2163; ++_i2167)
+            uint32_t _size2162;
+            ::apache::thrift::protocol::TType _etype2165;
+            xfer += iprot->readListBegin(_etype2165, _size2162);
+            (*(this->success)).resize(_size2162);
+            uint32_t _i2166;
+            for (_i2166 = 0; _i2166 < _size2162; ++_i2166)
             {
-              xfer += (*(this->success))[_i2167].read(iprot);
+              xfer += (*(this->success))[_i2166].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -23301,14 +23301,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size2168;
-            ::apache::thrift::protocol::TType _etype2171;
-            xfer += iprot->readListBegin(_etype2171, _size2168);
-            this->part_vals.resize(_size2168);
-            uint32_t _i2172;
-            for (_i2172 = 0; _i2172 < _size2168; ++_i2172)
+            uint32_t _size2167;
+            ::apache::thrift::protocol::TType _etype2170;
+            xfer += iprot->readListBegin(_etype2170, _size2167);
+            this->part_vals.resize(_size2167);
+            uint32_t _i2171;
+            for (_i2171 = 0; _i2171 < _size2167; ++_i2171)
             {
-              xfer += iprot->readString(this->part_vals[_i2172]);
+              xfer += iprot->readString(this->part_vals[_i2171]);
             }
             xfer += iprot->readListEnd();
           }
@@ -23337,14 +23337,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size2173;
-            ::apache::thrift::protocol::TType _etype2176;
-            xfer += iprot->readListBegin(_etype2176, _size2173);
-            this->group_names.resize(_size2173);
-            uint32_t _i2177;
-            for (_i2177 = 0; _i2177 < _size2173; ++_i2177)
+            uint32_t _size2172;
+            ::apache::thrift::protocol::TType _etype2175;
+            xfer += iprot->readListBegin(_etype2175, _size2172);
+            this->group_names.resize(_size2172);
+            uint32_t _i2176;
+            for (_i2176 = 0; _i2176 < _size2172; ++_i2176)
             {
-              xfer += iprot->readString(this->group_names[_i2177]);
+              xfer += iprot->readString(this->group_names[_i2176]);
             }
             xfer += iprot->readListEnd();
           }
@@ -23381,10 +23381,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::write(::apache::t
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter2178;
-    for (_iter2178 = this->part_vals.begin(); _iter2178 != this->part_vals.end(); ++_iter2178)
+    std::vector<std::string> ::const_iterator _iter2177;
+    for (_iter2177 = this->part_vals.begin(); _iter2177 != this->part_vals.end(); ++_iter2177)
     {
-      xfer += oprot->writeString((*_iter2178));
+      xfer += oprot->writeString((*_iter2177));
     }
     xfer += oprot->writeListEnd();
   }
@@ -23401,10 +23401,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::write(::apache::t
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 6);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter2179;
-    for (_iter2179 = this->group_names.begin(); _iter2179 != this->group_names.end(); ++_iter2179)
+    std::vector<std::string> ::const_iterator _iter2178;
+    for (_iter2178 = this->group_names.begin(); _iter2178 != this->group_names.end(); ++_iter2178)
     {
-      xfer += oprot->writeString((*_iter2179));
+      xfer += oprot->writeString((*_iter2178));
     }
     xfer += oprot->writeListEnd();
   }
@@ -23436,10 +23436,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_pargs::write(::apache::
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter2180;
-    for (_iter2180 = (*(this->part_vals)).begin(); _iter2180 != (*(this->part_vals)).end(); ++_iter2180)
+    std::vector<std::string> ::const_iterator _iter2179;
+    for (_iter2179 = (*(this->part_vals)).begin(); _iter2179 != (*(this->part_vals)).end(); ++_iter2179)
     {
-      xfer += oprot->writeString((*_iter2180));
+      xfer += oprot->writeString((*_iter2179));
     }
     xfer += oprot->writeListEnd();
   }
@@ -23456,10 +23456,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_pargs::write(::apache::
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 6);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter2181;
-    for (_iter2181 = (*(this->group_names)).begin(); _iter2181 != (*(this->group_names)).end(); ++_iter2181)
+    std::vector<std::string> ::const_iterator _iter2180;
+    for (_iter2180 = (*(this->group_names)).begin(); _iter2180 != (*(this->group_names)).end(); ++_iter2180)
     {
-      xfer += oprot->writeString((*_iter2181));
+      xfer += oprot->writeString((*_iter2180));
     }
     xfer += oprot->writeListEnd();
   }
@@ -23500,14 +23500,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_result::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2182;
-            ::apache::thrift::protocol::TType _etype2185;
-            xfer += iprot->readListBegin(_etype2185, _size2182);
-            this->success.resize(_size2182);
-            uint32_t _i2186;
-            for (_i2186 = 0; _i2186 < _size2182; ++_i2186)
+            uint32_t _size2181;
+            ::apache::thrift::protocol::TType _etype2184;
+            xfer += iprot->readListBegin(_etype2184, _size2181);
+            this->success.resize(_size2181);
+            uint32_t _i2185;
+            for (_i2185 = 0; _i2185 < _size2181; ++_i2185)
             {
-              xfer += this->success[_i2186].read(iprot);
+              xfer += this->success[_i2185].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -23554,10 +23554,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_result::write(::apache:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter2187;
-      for (_iter2187 = this->success.begin(); _iter2187 != this->success.end(); ++_iter2187)
+      std::vector<Partition> ::const_iterator _iter2186;
+      for (_iter2186 = this->success.begin(); _iter2186 != this->success.end(); ++_iter2186)
       {
-        xfer += (*_iter2187).write(oprot);
+        xfer += (*_iter2186).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -23606,14 +23606,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_presult::read(::apache:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2188;
-            ::apache::thrift::protocol::TType _etype2191;
-            xfer += iprot->readListBegin(_etype2191, _size2188);
-            (*(this->success)).resize(_size2188);
-            uint32_t _i2192;
-            for (_i2192 = 0; _i2192 < _size2188; ++_i2192)
+            uint32_t _size2187;
+            ::apache::thrift::protocol::TType _etype2190;
+            xfer += iprot->readListBegin(_etype2190, _size2187);
+            (*(this->success)).resize(_size2187);
+            uint32_t _i2191;
+            for (_i2191 = 0; _i2191 < _size2187; ++_i2191)
             {
-              xfer += (*(this->success))[_i2192].read(iprot);
+              xfer += (*(this->success))[_i2191].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -23923,14 +23923,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_args::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size2193;
-            ::apache::thrift::protocol::TType _etype2196;
-            xfer += iprot->readListBegin(_etype2196, _size2193);
-            this->part_vals.resize(_size2193);
-            uint32_t _i2197;
-            for (_i2197 = 0; _i2197 < _size2193; ++_i2197)
+            uint32_t _size2192;
+            ::apache::thrift::protocol::TType _etype2195;
+            xfer += iprot->readListBegin(_etype2195, _size2192);
+            this->part_vals.resize(_size2192);
+            uint32_t _i2196;
+            for (_i2196 = 0; _i2196 < _size2192; ++_i2196)
             {
-              xfer += iprot->readString(this->part_vals[_i2197]);
+              xfer += iprot->readString(this->part_vals[_i2196]);
             }
             xfer += iprot->readListEnd();
           }
@@ -23975,10 +23975,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_args::write(::apache::thrift
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter2198;
-    for (_iter2198 = this->part_vals.begin(); _iter2198 != this->part_vals.end(); ++_iter2198)
+    std::vector<std::string> ::const_iterator _iter2197;
+    for (_iter2197 = this->part_vals.begin(); _iter2197 != this->part_vals.end(); ++_iter2197)
     {
-      xfer += oprot->writeString((*_iter2198));
+      xfer += oprot->writeString((*_iter2197));
     }
     xfer += oprot->writeListEnd();
   }
@@ -24014,10 +24014,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_pargs::write(::apache::thrif
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter2199;
-    for (_iter2199 = (*(this->part_vals)).begin(); _iter2199 != (*(this->part_vals)).end(); ++_iter2199)
+    std::vector<std::string> ::const_iterator _iter2198;
+    for (_iter2198 = (*(this->part_vals)).begin(); _iter2198 != (*(this->part_vals)).end(); ++_iter2198)
     {
-      xfer += oprot->writeString((*_iter2199));
+      xfer += oprot->writeString((*_iter2198));
     }
     xfer += oprot->writeListEnd();
   }
@@ -24062,14 +24062,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_result::read(::apache::thrif
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2200;
-            ::apache::thrift::protocol::TType _etype2203;
-            xfer += iprot->readListBegin(_etype2203, _size2200);
-            this->success.resize(_size2200);
-            uint32_t _i2204;
-            for (_i2204 = 0; _i2204 < _size2200; ++_i2204)
+            uint32_t _size2199;
+            ::apache::thrift::protocol::TType _etype2202;
+            xfer += iprot->readListBegin(_etype2202, _size2199);
+            this->success.resize(_size2199);
+            uint32_t _i2203;
+            for (_i2203 = 0; _i2203 < _size2199; ++_i2203)
             {
-              xfer += iprot->readString(this->success[_i2204]);
+              xfer += iprot->readString(this->success[_i2203]);
             }
             xfer += iprot->readListEnd();
           }
@@ -24116,10 +24116,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_result::write(::apache::thri
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter2205;
-      for (_iter2205 = this->success.begin(); _iter2205 != this->success.end(); ++_iter2205)
+      std::vector<std::string> ::const_iterator _iter2204;
+      for (_iter2204 = this->success.begin(); _iter2204 != this->success.end(); ++_iter2204)
       {
-        xfer += oprot->writeString((*_iter2205));
+        xfer += oprot->writeString((*_iter2204));
       }
       xfer += oprot->writeListEnd();
     }
@@ -24168,14 +24168,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_presult::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2206;
-            ::apache::thrift::protocol::TType _etype2209;
-            xfer += iprot->readListBegin(_etype2209, _size2206);
-            (*(this->success)).resize(_size2206);
-            uint32_t _i2210;
-            for (_i2210 = 0; _i2210 < _size2206; ++_i2210)
+            uint32_t _size2205;
+            ::apache::thrift::protocol::TType _etype2208;
+            xfer += iprot->readListBegin(_etype2208, _size2205);
+            (*(this->success)).resize(_size2205);
+            uint32_t _i2209;
+            for (_i2209 = 0; _i2209 < _size2205; ++_i2209)
             {
-              xfer += iprot->readString((*(this->success))[_i2210]);
+              xfer += iprot->readString((*(this->success))[_i2209]);
             }
             xfer += iprot->readListEnd();
           }
@@ -24548,14 +24548,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_req_result::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2211;
-            ::apache::thrift::protocol::TType _etype2214;
-            xfer += iprot->readListBegin(_etype2214, _size2211);
-            this->success.resize(_size2211);
-            uint32_t _i2215;
-            for (_i2215 = 0; _i2215 < _size2211; ++_i2215)
+            uint32_t _size2210;
+            ::apache::thrift::protocol::TType _etype2213;
+            xfer += iprot->readListBegin(_etype2213, _size2210);
+            this->success.resize(_size2210);
+            uint32_t _i2214;
+            for (_i2214 = 0; _i2214 < _size2210; ++_i2214)
             {
-              xfer += iprot->readString(this->success[_i2215]);
+              xfer += iprot->readString(this->success[_i2214]);
             }
             xfer += iprot->readListEnd();
           }
@@ -24602,10 +24602,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_req_result::write(::apache::thr
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter2216;
-      for (_iter2216 = this->success.begin(); _iter2216 != this->success.end(); ++_iter2216)
+      std::vector<std::string> ::const_iterator _iter2215;
+      for (_iter2215 = this->success.begin(); _iter2215 != this->success.end(); ++_iter2215)
       {
-        xfer += oprot->writeString((*_iter2216));
+        xfer += oprot->writeString((*_iter2215));
       }
       xfer += oprot->writeListEnd();
     }
@@ -24654,14 +24654,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_req_presult::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2217;
-            ::apache::thrift::protocol::TType _etype2220;
-            xfer += iprot->readListBegin(_etype2220, _size2217);
-            (*(this->success)).resize(_size2217);
-            uint32_t _i2221;
-            for (_i2221 = 0; _i2221 < _size2217; ++_i2221)
+            uint32_t _size2216;
+            ::apache::thrift::protocol::TType _etype2219;
+            xfer += iprot->readListBegin(_etype2219, _size2216);
+            (*(this->success)).resize(_size2216);
+            uint32_t _i2220;
+            for (_i2220 = 0; _i2220 < _size2216; ++_i2220)
             {
-              xfer += iprot->readString((*(this->success))[_i2221]);
+              xfer += iprot->readString((*(this->success))[_i2220]);
             }
             xfer += iprot->readListEnd();
           }
@@ -24855,14 +24855,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_filter_result::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2222;
-            ::apache::thrift::protocol::TType _etype2225;
-            xfer += iprot->readListBegin(_etype2225, _size2222);
-            this->success.resize(_size2222);
-            uint32_t _i2226;
-            for (_i2226 = 0; _i2226 < _size2222; ++_i2226)
+            uint32_t _size2221;
+            ::apache::thrift::protocol::TType _etype2224;
+            xfer += iprot->readListBegin(_etype2224, _size2221);
+            this->success.resize(_size2221);
+            uint32_t _i2225;
+            for (_i2225 = 0; _i2225 < _size2221; ++_i2225)
             {
-              xfer += this->success[_i2226].read(iprot);
+              xfer += this->success[_i2225].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -24909,10 +24909,10 @@ uint32_t ThriftHiveMetastore_get_partitions_by_filter_result::write(::apache::th
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter2227;
-      for (_iter2227 = this->success.begin(); _iter2227 != this->success.end(); ++_iter2227)
+      std::vector<Partition> ::const_iterator _iter2226;
+      for (_iter2226 = this->success.begin(); _iter2226 != this->success.end(); ++_iter2226)
       {
-        xfer += (*_iter2227).write(oprot);
+        xfer += (*_iter2226).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -24961,14 +24961,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_filter_presult::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2228;
-            ::apache::thrift::protocol::TType _etype2231;
-            xfer += iprot->readListBegin(_etype2231, _size2228);
-            (*(this->success)).resize(_size2228);
-            uint32_t _i2232;
-            for (_i2232 = 0; _i2232 < _size2228; ++_i2232)
+            uint32_t _size2227;
+            ::apache::thrift::protocol::TType _etype2230;
+            xfer += iprot->readListBegin(_etype2230, _size2227);
+            (*(this->success)).resize(_size2227);
+            uint32_t _i2231;
+            for (_i2231 = 0; _i2231 < _size2227; ++_i2231)
             {
-              xfer += (*(this->success))[_i2232].read(iprot);
+              xfer += (*(this->success))[_i2231].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -25162,14 +25162,14 @@ uint32_t ThriftHiveMetastore_get_part_specs_by_filter_result::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2233;
-            ::apache::thrift::protocol::TType _etype2236;
-            xfer += iprot->readListBegin(_etype2236, _size2233);
-            this->success.resize(_size2233);
-            uint32_t _i2237;
-            for (_i2237 = 0; _i2237 < _size2233; ++_i2237)
+            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[_i2237].read(iprot);
+              xfer += this->success[_i2236].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -25216,10 +25216,10 @@ uint32_t ThriftHiveMetastore_get_part_specs_by_filter_result::write(::apache::th
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<PartitionSpec> ::const_iterator _iter2238;
-      for (_iter2238 = this->success.begin(); _iter2238 != this->success.end(); ++_iter2238)
+      std::vector<PartitionSpec> ::const_iterator _iter2237;
+      for (_iter2237 = this->success.begin(); _iter2237 != this->success.end(); ++_iter2237)
       {
-        xfer += (*_iter2238).write(oprot);
+        xfer += (*_iter2237).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -25268,14 +25268,14 @@ uint32_t ThriftHiveMetastore_get_part_specs_by_filter_presult::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2239;
-            ::apache::thrift::protocol::TType _etype2242;
-            xfer += iprot->readListBegin(_etype2242, _size2239);
-            (*(this->success)).resize(_size2239);
-            uint32_t _i2243;
-            for (_i2243 = 0; _i2243 < _size2239; ++_i2243)
+            uint32_t _size2238;
+            ::apache::thrift::protocol::TType _etype2241;
+            xfer += iprot->readListBegin(_etype2241, _size2238);
+            (*(this->success)).resize(_size2238);
+            uint32_t _i2242;
+            for (_i2242 = 0; _i2242 < _size2238; ++_i2242)
             {
-              xfer += (*(this->success))[_i2243].read(iprot);
+              xfer += (*(this->success))[_i2242].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -26071,14 +26071,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_args::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->names.clear();
-            uint32_t _size2244;
-            ::apache::thrift::protocol::TType _etype2247;
-            xfer += iprot->readListBegin(_etype2247, _size2244);
-            this->names.resize(_size2244);
-            uint32_t _i2248;
-            for (_i2248 = 0; _i2248 < _size2244; ++_i2248)
+            uint32_t _size2243;
+            ::apache::thrift::protocol::TType _etype2246;
+            xfer += iprot->readListBegin(_etype2246, _size2243);
+            this->names.resize(_size2243);
+            uint32_t _i2247;
+            for (_i2247 = 0; _i2247 < _size2243; ++_i2247)
             {
-              xfer += iprot->readString(this->names[_i2248]);
+              xfer += iprot->readString(this->names[_i2247]);
             }
             xfer += iprot->readListEnd();
           }
@@ -26115,10 +26115,10 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_args::write(::apache::thrif
   xfer += oprot->writeFieldBegin("names", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->names.size()));
-    std::vector<std::string> ::const_iterator _iter2249;
-    for (_iter2249 = this->names.begin(); _iter2249 != this->names.end(); ++_iter2249)
+    std::vector<std::string> ::const_iterator _iter2248;
+    for (_iter2248 = this->names.begin(); _iter2248 != this->names.end(); ++_iter2248)
     {
-      xfer += oprot->writeString((*_iter2249));
+      xfer += oprot->writeString((*_iter2248));
     }
     xfer += oprot->writeListEnd();
   }
@@ -26150,10 +26150,10 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_pargs::write(::apache::thri
   xfer += oprot->writeFieldBegin("names", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->names)).size()));
-    std::vector<std::string> ::const_iterator _iter2250;
-    for (_iter2250 = (*(this->names)).begin(); _iter2250 != (*(this->names)).end(); ++_iter2250)
+    std::vector<std::string> ::const_iterator _iter2249;
+    for (_iter2249 = (*(this->names)).begin(); _iter2249 != (*(this->names)).end(); ++_iter2249)
     {
-      xfer += oprot->writeString((*_iter2250));
+      xfer += oprot->writeString((*_iter2249));
     }
     xfer += oprot->writeListEnd();
   }
@@ -26194,14 +26194,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_result::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2251;
-            ::apache::thrift::protocol::TType _etype2254;
-            xfer += iprot->readListBegin(_etype2254, _size2251);
-            this->success.resize(_size2251);
-            uint32_t _i2255;
-            for (_i2255 = 0; _i2255 < _size2251; ++_i2255)
+            uint32_t _size2250;
+            ::apache::thrift::protocol::TType _etype2253;
+            xfer += iprot->readListBegin(_etype2253, _size2250);
+            this->success.resize(_size2250);
+            uint32_t _i2254;
+            for (_i2254 = 0; _i2254 < _size2250; ++_i2254)
             {
-              xfer += this->success[_i2255].read(iprot);
+              xfer += this->success[_i2254].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -26248,10 +26248,10 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_result::write(::apache::thr
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter2256;
-      for (_iter2256 = this->success.begin(); _iter2256 != this->success.end(); ++_iter2256)
+      std::vector<Partition> ::const_iterator _iter2255;
+      for (_iter2255 = this->success.begin(); _iter2255 != this->success.end(); ++_iter2255)
       {
-        xfer += (*_iter2256).write(oprot);
+        xfer += (*_iter2255).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -26300,14 +26300,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_presult::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2257;
-            ::apache::thrift::protocol::TType _etype2260;
-            xfer += iprot->readListBegin(_etype2260, _size2257);
-            (*(this->success)).resize(_size2257);
-            uint32_t _i2261;
-            for (_i2261 = 0; _i2261 < _size2257; ++_i2261)
+            uint32_t _size2256;
+            ::apache::thrift::protocol::TType _etype2259;
+            xfer += iprot->readListBegin(_etype2259, _size2256);
+            (*(this->success)).resize(_size2256);
+            uint32_t _i2260;
+            for (_i2260 = 0; _i2260 < _size2256; ++_i2260)
             {
-              xfer += (*(this->success))[_i2261].read(iprot);
+              xfer += (*(this->success))[_i2260].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -26856,14 +26856,14 @@ uint32_t ThriftHiveMetastore_alter_partitions_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size2262;
-            ::apache::thrift::protocol::TType _etype2265;
-            xfer += iprot->readListBegin(_etype2265, _size2262);
-            this->new_parts.resize(_size2262);
-            uint32_t _i2266;
-            for (_i2266 = 0; _i2266 < _size2262; ++_i2266)
+            uint32_t _size2261;
+            ::apache::thrift::protocol::TType _etype2264;
+            xfer += iprot->readListBegin(_etype2264, _size2261);
+            this->new_parts.resize(_size2261);
+            uint32_t _i2265;
+            for (_i2265 = 0; _i2265 < _size2261; ++_i2265)
             {
-              xfer += this->new_parts[_i2266].read(iprot);
+              xfer += this->new_parts[_i2265].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -26900,10 +26900,10 @@ uint32_t ThriftHiveMetastore_alter_partitions_args::write(::apache::thrift::prot
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->new_parts.size()));
-    std::vector<Partition> ::const_iterator _iter2267;
-    for (_iter2267 = this->new_parts.begin(); _iter2267 != this->new_parts.end(); ++_iter2267)
+    std::vector<Partition> ::const_iterator _iter2266;
+    for (_iter2266 = this->new_parts.begin(); _iter2266 != this->new_parts.end(); ++_iter2266)
     {
-      xfer += (*_iter2267).write(oprot);
+      xfer += (*_iter2266).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -26935,10 +26935,10 @@ uint32_t ThriftHiveMetastore_alter_partitions_pargs::write(::apache::thrift::pro
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->new_parts)).size()));
-    std::vector<Partition> ::const_iterator _iter2268;
-    for (_iter2268 = (*(this->new_parts)).begin(); _iter2268 != (*(this->new_parts)).end(); ++_iter2268)
+    std::vector<Partition> ::const_iterator _iter2267;
+    for (_iter2267 = (*(this->new_parts)).begin(); _iter2267 != (*(this->new_parts)).end(); ++_iter2267)
     {
-      xfer += (*_iter2268).write(oprot);
+      xfer += (*_iter2267).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -27123,14 +27123,14 @@ uint32_t ThriftHiveMetastore_alter_partitions_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size2269;
-            ::apache::thrift::protocol::TType _etype2272;
-            xfer += iprot->readListBegin(_etype2272, _size2269);
-            this->new_parts.resize(_size2269);
-            uint32_t _i2273;
-            for (_i2273 = 0; _i2273 < _size2269; ++_i2273)
+            uint32_t _size2268;
+            ::apache::thrift::protocol::TType _etype2271;
+            xfer += iprot->readListBegin(_etype2271, _size2268);
+            this->new_parts.resize(_size2268);
+            uint32_t _i2272;
+            for (_i2272 = 0; _i2272 < _size2268; ++_i2272)
             {
-              xfer += this->new_parts[_i2273].read(iprot);
+              xfer += this->new_parts[_i2272].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -27175,10 +27175,10 @@ uint32_t ThriftHiveMetastore_alter_partitions_with_environment_context_args::wri
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->new_parts.size()));
-    std::vector<Partition> ::const_iterator _iter2274;
-    for (_iter2274 = this->new_parts.begin(); _iter2274 != this->new_parts.end(); ++_iter2274)
+    std::vector<Partition> ::const_iterator _iter2273;
+    for (_iter2273 = this->new_parts.begin(); _iter2273 != this->new_parts.end(); ++_iter2273)
     {
-      xfer += (*_iter2274).write(oprot);
+      xfer += (*_iter2273).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -27214,10 +27214,10 @@ uint32_t ThriftHiveMetastore_alter_partitions_with_environment_context_pargs::wr
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->new_parts)).size()));
-    std::vector<Partition> ::const_iterator _iter2275;
-    for (_iter2275 = (*(this->new_parts)).begin(); _iter2275 != (*(this->new_parts)).end(); ++_iter2275)
+    std::vector<Partition> ::const_iterator _iter2274;
+    for (_iter2274 = (*(this->new_parts)).begin(); _iter2274 != (*(this->new_parts)).end(); ++_iter2274)
     {
-      xfer += (*_iter2275).write(oprot);
+      xfer += (*_iter2274).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -27888,14 +27888,14 @@ uint32_t ThriftHiveMetastore_rename_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size2276;
-            ::apache::thrift::protocol::TType _etype2279;
-            xfer += iprot->readListBegin(_etype2279, _size2276);
-            this->part_vals.resize(_size2276);
-            uint32_t _i2280;
-            for (_i2280 = 0; _i2280 < _size2276; ++_i2280)
+            uint32_t _size2275;
+            ::apache::thrift::protocol::TType _etype2278;
+            xfer += iprot->readListBegin(_etype2278, _size2275);
+            this->part_vals.resize(_size2275);
+            uint32_t _i2279;
+            for (_i2279 = 0; _i2279 < _size2275; ++_i2279)
             {
-              xfer += iprot->readString(this->part_vals[_i2280]);
+              xfer += iprot->readString(this->part_vals[_i2279]);
             }
             xfer += iprot->readListEnd();
           }
@@ -27940,10 +27940,10 @@ uint32_t ThriftHiveMetastore_rename_partition_args::write(::apache::thrift::prot
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter2281;
-    for (_iter2281 = this->part_vals.begin(); _iter2281 != this->part_vals.end(); ++_iter2281)
+    std::vector<std::string> ::const_iterator _iter2280;
+    for (_iter2280 = this->part_vals.begin(); _iter2280 != this->part_vals.end(); ++_iter2280)
     {
-      xfer += oprot->writeString((*_iter2281));
+      xfer += oprot->writeString((*_iter2280));
     }
     xfer += oprot->writeListEnd();
   }
@@ -27979,10 +27979,10 @@ uint32_t ThriftHiveMetastore_rename_partition_pargs::write(::apache::thrift::pro
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter2282;
-    for (_iter2282 = (*(this->part_vals)).begin(); _iter2282 != (*(this->part_vals)).end(); ++_iter2282)
+    std::vector<std::string> ::const_iterator _iter2281;
+    for (_iter2281 = (*(this->part_vals)).begin(); _iter2281 != (*(this->part_vals)).end(); ++_iter2281)
     {
-      xfer += oprot->writeString((*_iter2282));
+      xfer += oprot->writeString((*_iter2281));
     }
     xfer += oprot->writeListEnd();
   }
@@ -28382,14 +28382,14 @@ uint32_t ThriftHiveMetastore_partition_name_has_valid_characters_args::read(::ap
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size2283;
-            ::apache::thrift::protocol::TType _etype2286;
-            xfer += iprot->readListBegin(_etype2286, _size2283);
-            this->part_vals.resize(_size2283);
-            uint32_t _i2287;
-            for (_i2287 = 0; _i2287 < _size2283; ++_i2287)
+            uint32_t _size2282;
+            ::apache::thrift::protocol::TType _etype2285;
+            xfer += iprot->readListBegin(_etype2285, _size2282);
+            this->part_vals.resize(_size2282);
+            uint32_t _i2286;
+            for (_i2286 = 0; _i2286 < _size2282; ++_i2286)
             {
-              xfer += iprot->readString(this->part_vals[_i2287]);
+              xfer += iprot->readString(this->part_vals[_i2286]);
             }
             xfer += iprot->readListEnd();
           }
@@ -28426,10 +28426,10 @@ uint32_t ThriftHiveMetastore_partition_name_has_valid_characters_args::write(::a
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter2288;
-    for (_iter2288 = this->part_vals.begin(); _iter2288 != this->part_vals.end(); ++_iter2288)
+    std::vector<std::string> ::const_iterator _iter2287;
+    for (_iter2287 = this->part_vals.begin(); _iter2287 != this->part_vals.end(); ++_iter2287)
     {
-      xfer += oprot->writeString((*_iter2288));
+      xfer += oprot->writeString((*_iter2287));
     }
     xfer += oprot->writeListEnd();
   }
@@ -28457,10 +28457,10 @@ uint32_t ThriftHiveMetastore_partition_name_has_valid_characters_pargs::write(::
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter2289;
-    for (_iter2289 = (*(this->part_vals)).begin(); _iter2289 != (*(this->part_vals)).end(); ++_iter2289)
+    std::vector<std::string> ::const_iterator _iter2288;
+    for (_iter2288 = (*(this->part_vals)).begin(); _iter2288 != (*(this->part_vals)).end(); ++_iter2288)
     {
-      xfer += oprot->writeString((*_iter2289));
+      xfer += oprot->writeString((*_iter2288));
     }
     xfer += oprot->writeListEnd();
   }
@@ -28935,14 +28935,14 @@ uint32_t ThriftHiveMetastore_partition_name_to_vals_result::read(::apache::thrif
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2290;
-            ::apache::thrift::protocol::TType _etype2293;
-            xfer += iprot->readListBegin(_etype2293, _size2290);
-            this->success.resize(_size2290);
-            uint32_t _i2294;
-            for (_i2294 = 0; _i2294 < _size2290; ++_i2294)
+            uint32_t _size2289;
+            ::apache::thrift::protocol::TType _etype2292;
+            xfer += iprot->readListBegin(_etype2292, _size2289);
+            this->success.resize(_size2289);
+            uint32_t _i2293;
+            for (_i2293 = 0; _i2293 < _size2289; ++_i2293)
             {
-              xfer += iprot->readString(this->success[_i2294]);
+              xfer += iprot->readString(this->success[_i2293]);
             }
             xfer += iprot->readListEnd();
           }
@@ -28981,10 +28981,10 @@ uint32_t ThriftHiveMetastore_partition_name_to_vals_result::write(::apache::thri
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter2295;
-      for (_iter2295 = this->success.begin(); _iter2295 != this->success.end(); ++_iter2295)
+      std::vector<std::string> ::const_iterator _iter2294;
+      for (_iter2294 = this->success.begin(); _iter2294 != this->success.end(); ++_iter2294)
       {
-        xfer += oprot->writeString((*_iter2295));
+        xfer += oprot->writeString((*_iter2294));
       }
       xfer += oprot->writeListEnd();
     }
@@ -29029,14 +29029,14 @@ uint32_t ThriftHiveMetastore_partition_name_to_vals_presult::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2296;
-            ::apache::thrift::protocol::TType _etype2299;
-            xfer += iprot->readListBegin(_etype2299, _size2296);
-            (*(this->success)).resize(_size2296);
-            uint32_t _i2300;
-            for (_i2300 = 0; _i2300 < _size2296; ++_i2300)
+            uint32_t _size2295;
+            ::apache::thrift::protocol::TType _etype2298;
+            xfer += iprot->readListBegin(_etype2298, _size2295);
+            (*(this->success)).resize(_size2295);
+            uint32_t _i2299;
+            for (_i2299 = 0; _i2299 < _size2295; ++_i2299)
             {
-              xfer += iprot->readString((*(this->success))[_i2300]);
+              xfer += iprot->readString((*(this->success))[_i2299]);
             }
             xfer += iprot->readListEnd();
           }
@@ -29174,17 +29174,17 @@ uint32_t ThriftHiveMetastore_partition_name_to_spec_result::read(::apache::thrif
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size2301;
-            ::apache::thrift::protocol::TType _ktype2302;
-            ::apache::thrift::protocol::TType _vtype2303;
-            xfer += iprot->readMapBegin(_ktype2302, _vtype2303, _size2301);
-            uint32_t _i2305;
-            for (_i2305 = 0; _i2305 < _size2301; ++_i2305)
+            uint32_t _size2300;
+            ::apache::thrift::protocol::TType _ktype2301;
+            ::apache::thrift::protocol::TType _vtype2302;
+            xfer += iprot->readMapBegin(_ktype2301, _vtype2302, _size2300);
+            uint32_t _i2304;
+            for (_i2304 = 0; _i2304 < _size2300; ++_i2304)
             {
-              std::string _key2306;
-              xfer += iprot->readString(_key2306);
-              std::string& _val2307 = this->success[_key2306];
-              xfer += iprot->readString(_val2307);
+              std::string _key2305;
+              xfer += iprot->readString(_key2305);
+              std::string& _val2306 = this->success[_key2305];
+              xfer += iprot->readString(_val2306);
             }
             xfer += iprot->readMapEnd();
           }
@@ -29223,11 +29223,11 @@ uint32_t ThriftHiveMetastore_partition_name_to_spec_result::write(::apache::thri
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::map<std::string, std::string> ::const_iterator _iter2308;
-      for (_iter2308 = this->success.begin(); _iter2308 != this->success.end(); ++_iter2308)
+      std::map<std::string, std::string> ::const_iterator _iter2307;
+      for (_iter2307 = this->success.begin(); _iter2307 != this->success.end(); ++_iter2307)
       {
-        xfer += oprot->writeString(_iter2308->first);
-        xfer += oprot->writeString(_iter2308->second);
+        xfer += oprot->writeString(_iter2307->first);
+        xfer += oprot->writeString(_iter2307->second);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -29272,17 +29272,17 @@ uint32_t ThriftHiveMetastore_partition_name_to_spec_presult::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size2309;
-            ::apache::thrift::protocol::TType _ktype2310;
-            ::apache::thrift::protocol::TType _vtype2311;
-            xfer += iprot->readMapBegin(_ktype2310, _vtype2311, _size2309);
-            uint32_t _i2313;
-            for (_i2313 = 0; _i2313 < _size2309; ++_i2313)
+            uint32_t _size2308;
+            ::apache::thrift::protocol::TType _ktype2309;
+            ::apache::thrift::protocol::TType _vtype2310;
+            xfer += iprot->readMapBegin(_ktype2309, _vtype2310, _size2308);
+            uint32_t _i2312;
+            for (_i2312 = 0; _i2312 < _size2308; ++_i2312)
             {
-              std::string _key2314;
-              xfer += iprot->readString(_key2314);
-              std::string& _val2315 = (*(this->success))[_key2314];
-              xfer += iprot->readString(_val2315);
+              std::string _key2313;
+              xfer += iprot->readString(_key2313);
+              std::string& _val2314 = (*(this->success))[_key2313];
+              xfer += iprot->readString(_val2314);
             }
             xfer += iprot->readMapEnd();
           }
@@ -29357,17 +29357,17 @@ uint32_t ThriftHiveMetastore_markPartitionForEvent_args::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->part_vals.clear();
-            uint32_t _size2316;
-            ::apache::thrift::protocol::TType _ktype2317;
-            ::apache::thrift::protocol::TType _vtype2318;
-            xfer += iprot->readMapBegin(_ktype2317, _vtype2318, _size2316);
-            uint32_t _i2320;
-            for (_i2320 = 0; _i2320 < _size2316; ++_i2320)
+            uint32_t _size2315;
+            ::apache::thrift::protocol::TType _ktype2316;
+            ::apache::thrift::protocol::TType _vtype2317;
+            xfer += iprot->readMapBegin(_ktype2316, _vtype2317, _size2315);
+            uint32_t _i2319;
+            for (_i2319 = 0; _i2319 < _size2315; ++_i2319)
             {
-              std::string _key2321;
-              xfer += iprot->readString(_key2321);
-              std::string& _val2322 = this->part_vals[_key2321];
-              xfer += iprot->readString(_val2322);
+              std::string _key2320;
+              xfer += iprot->readString(_key2320);
+              std::string& _val2321 = this->part_vals[_key2320];
+              xfer += iprot->readString(_val2321);
             }
             xfer += iprot->readMapEnd();
           }
@@ -29378,9 +29378,9 @@ uint32_t ThriftHiveMetastore_markPartitionForEvent_args::read(::apache::thrift::
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast2323;
-          xfer += iprot->readI32(ecast2323);
-          this->eventType = (PartitionEventType::type)ecast2323;
+          int32_t ecast2322;
+          xfer += iprot->readI32(ecast2322);
+          this->eventType = (PartitionEventType::type)ecast2322;
           this->__isset.eventType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -29414,11 +29414,11 @@ uint32_t ThriftHiveMetastore_markPartitionForEvent_args::write(::apache::thrift:
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_MAP, 3);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::map<std::string, std::string> ::const_iterator _iter2324;
-    for (_iter2324 = this->part_vals.begin(); _iter2324 != this->part_vals.end(); ++_iter2324)
+    std::map<std::string, std::string> ::const_iterator _iter2323;
+    for (_iter2323 = this->part_vals.begin(); _iter2323 != this->part_vals.end(); ++_iter2323)
     {
-      xfer += oprot->writeString(_iter2324->first);
-      xfer += oprot->writeString(_iter2324->second);
+      xfer += oprot->writeString(_iter2323->first);
+      xfer += oprot->writeString(_iter2323->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -29454,11 +29454,11 @@ uint32_t ThriftHiveMetastore_markPartitionForEvent_pargs::write(::apache::thrift
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_MAP, 3);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::map<std::string, std::string> ::const_iterator _iter2325;
-    for (_iter2325 = (*(this->part_vals)).begin(); _iter2325 != (*(this->part_vals)).end(); ++_iter2325)
+    std::map<std::string, std::string> ::const_iterator _iter2324;
+    for (_iter2324 = (*(this->part_vals)).begin(); _iter2324 != (*(this->part_vals)).end(); ++_iter2324)
     {
-      xfer += oprot->writeString(_iter2325->first);
-      xfer += oprot->writeString(_iter2325->second);
+      xfer += oprot->writeString(_iter2324->first);
+      xfer += oprot->writeString(_iter2324->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -29727,17 +29727,17 @@ uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_args::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->part_vals.clear();
-            uint32_t _size2326;
-            ::apache::thrift::protocol::TType _ktype2327;
-            ::apache::thrift::protocol::TType _vtype2328;
-            xfer += iprot->readMapBegin(_ktype2327, _vtype2328, _size2326);
-            uint32_t _i2330;
-            for (_i2330 = 0; _i2330 < _size2326; ++_i2330)
+            uint32_t _size2325;
+            ::apache::thrift::protocol::TType _ktype2326;
+            ::apache::thrift::protocol::TType _vtype2327;
+            xfer += iprot->readMapBegin(_ktype2326, _vtype2327, _size2325);
+            uint32_t _i2329;
+            for (_i2329 = 0; _i2329 < _size2325; ++_i2329)
             {
-              std::string _key2331;
-              xfer += iprot->readString(_key2331);
-              std::string& _val2332 = this->part_vals[_key2331];
-              xfer += iprot->readString(_val2332);
+              std::string _key2330;
+              xfer += iprot->readString(_key2330);
+              std::string& _val2331 = this->part_vals[_key2330];
+              xfer += iprot->readString(_val2331);
             }
             xfer += iprot->readMapEnd();
           }
@@ -29748,9 +29748,9 @@ uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_args::read(::apache::thri
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast2333;
-          xfer += iprot->readI32(ecast2333);
-          this->eventType = (PartitionEventType::type)ecast2333;
+          int32_t ecast2332;
+          xfer += iprot->readI32(ecast2332);
+          this->eventType = (PartitionEventType::type)ecast2332;
           this->__isset.eventType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -29784,11 +29784,11 @@ uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_args::write(::apache::thr
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_MAP, 3);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::map<std::string, std::string> ::const_iterator _iter2334;
-    for (_iter2334 = this->part_vals.begin(); _iter2334 != this->part_vals.end(); ++_iter2334)
+    std::map<std::string, std::string> ::const_iterator _iter2333;
+    for (_iter2333 = this->part_vals.begin(); _iter2333 != this->part_vals.end(); ++_iter2333)
     {
-      xfer += oprot->writeString(_iter2334->first);
-      xfer += oprot->writeString(_iter2334->second);
+      xfer += oprot->writeString(_iter2333->first);
+      xfer += oprot->writeString(_iter2333->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -29824,11 +29824,11 @@ uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_pargs::write(::apache::th
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_MAP, 3);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::map<std::string, std::string> ::const_iterator _iter2335;
-    for (_iter2335 = (*(this->part_vals)).begin(); _iter2335 != (*(this->part_vals)).end(); ++_iter2335)
+    std::map<std::string, std::string> ::const_iterator _iter2334;
+    for (_iter2334 = (*(this->part_vals)).begin(); _iter2334 != (*(this->part_vals)).end(); ++_iter2334)
     {
-      xfer += oprot->writeString(_iter2335->first);
-      xfer += oprot->writeString(_iter2335->second);
+      xfer += oprot->writeString(_iter2334->first);
+      xfer += oprot->writeString(_iter2334->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -35957,14 +35957,14 @@ uint32_t ThriftHiveMetastore_get_functions_result::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _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 _size2335;
+            ::apache::thrift::protocol::TType _etype2338;
+            xfer += iprot->readListBegin(_etype2338, _size2335);
+            this->success.resize(_size2335);
+            uint32_t _i2339;
+            for (_i2339 = 0; _i2339 < _size2335; ++_i2339)
             {
-              xfer += iprot->readString(this->success[_i2340]);
+              xfer += iprot->readString(this->success[_i2339]);
             }
             xfer += iprot->readListEnd();
           }
@@ -36003,10 +36003,10 @@ uint32_t ThriftHiveMetastore_get_functions_result::write(::apache::thrift::proto
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter2341;
-      for (_iter2341 = this->success.begin(); _iter2341 != this->success.end(); ++_iter2341)
+      std::vector<std::string> ::const_iterator _iter2340;
+      for (_iter2340 = this->success.begin(); _iter2340 != this->success.end(); ++_iter2340)
       {
-        xfer += oprot->writeString((*_iter2341));
+        xfer += oprot->writeString((*_iter2340));
       }
       xfer += oprot->writeListEnd();
     }
@@ -36051,14 +36051,14 @@ uint32_t ThriftHiveMetastore_get_functions_presult::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _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 _size2341;
+            ::apache::thrift::protocol::TType _etype2344;
+            xfer += iprot->readListBegin(_etype2344, _size2341);
+            (*(this->success)).resize(_size2341);
+            uint32_t _i2345;
+            for (_i2345 = 0; _i2345 < _size2341; ++_i2345)
             {
-              xfer += iprot->readString((*(this->success))[_i2346]);
+              xfer += iprot->readString((*(this->success))[_i2345]);
             }
             xfer += iprot->readListEnd();
           }
@@ -37018,14 +37018,14 @@ uint32_t ThriftHiveMetastore_get_role_names_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _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 _size2346;
+            ::apache::thrift::protocol::TType _etype2349;
+            xfer += iprot->readListBegin(_etype2349, _size2346);
+            this->success.resize(_size2346);
+            uint32_t _i2350;
+            for (_i2350 = 0; _i2350 < _size2346; ++_i2350)
             {
-              xfer += iprot->readString(this->success[_i2351]);
+              xfer += iprot->readString(this->success[_i2350]);
             }
             xfer += iprot->readListEnd();
           }
@@ -37064,10 +37064,10 @@ uint32_t ThriftHiveMetastore_get_role_names_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter2352;
-      for (_iter2352 = this->success.begin(); _iter2352 != this->success.end(); ++_iter2352)
+      std::vector<std::string> ::const_iterator _iter2351;
+      for (_iter2351 = this->success.begin(); _iter2351 != this->success.end(); ++_iter2351)
       {
-        xfer += oprot->writeString((*_iter2352));
+        xfer += oprot->writeString((*_iter2351));
       }
       xfer += oprot->writeListEnd();
     }
@@ -37112,14 +37112,14 @@ uint32_t ThriftHiveMetastore_get_role_names_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _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 _size2352;
+            ::apache::thrift::protocol::TType _etype2355;
+            xfer += iprot->readListBegin(_etype2355, _size2352);
+            (*(this->success)).resize(_size2352);
+            uint32_t _i2356;
+            for (_i2356 = 0; _i2356 < _size2352; ++_i2356)
             {
-              xfer += iprot->readString((*(this->success))[_i2357]);
+              xfer += iprot->readString((*(this->success))[_i2356]);
             }
             xfer += iprot->readListEnd();
           }
@@ -37192,9 +37192,9 @@ uint32_t ThriftHiveMetastore_grant_role_args::read(::apache::thrift::protocol::T
         break;
       case 3:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast2358;
-          xfer += iprot->readI32(ecast2358);
-          this->principal_type = (PrincipalType::type)ecast2358;
+          int32_t ecast2357;
+          xfer += iprot->readI32(ecast2357);
+          this->principal_type = (PrincipalType::type)ecast2357;
           this->__isset.principal_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -37210,9 +37210,9 @@ uint32_t ThriftHiveMetastore_grant_role_args::read(::apache::thrift::protocol::T
         break;
       case 5:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast2359;
-          xfer += iprot->readI32(ecast2359);
-          this->grantorType = (PrincipalType::type)ecast2359;
+          int32_t ecast2358;
+          xfer += iprot->readI32(ecast2358);
+          this->grantorType = (PrincipalType::type)ecast2358;
           this->__isset.grantorType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -37483,9 +37483,9 @@ uint32_t ThriftHiveMetastore_revoke_role_args::read(::apache::thrift::protocol::
         break;
       case 3:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast2360;
-          xfer += iprot->readI32(ecast2360);
-          this->principal_type = (PrincipalType::type)ecast2360;
+          int32_t ecast2359;
+          xfer += iprot->readI32(ecast2359);
+          this->principal_type = (PrincipalType::type)ecast2359;
           this->__isset.principal_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -37716,9 +37716,9 @@ uint32_t ThriftHiveMetastore_list_roles_args::read(::apache::thrift::protocol::T
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast2361;
-          xfer += iprot->readI32(ecast2361);
-          this->principal_type = (PrincipalType::type)ecast2361;
+          int32_t ecast2360;
+          xfer += iprot->readI32(ecast2360);
+          this->principal_type = (PrincipalType::type)ecast2360;
           this->__isset.principal_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -37807,14 +37807,14 @@ uint32_t ThriftHiveMetastore_list_roles_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2362;
-            ::apache::thrift::protocol::TType _etype2365;
-            xfer += iprot->readListBegin(_etype2365, _size2362);
-            this->success.resize(_size2362);
-            uint32_t _i2366;
-            for (_i2366 = 0; _i2366 < _size2362; ++_i2366)
+            uint32_t _size2361;
+            ::apache::thrift::protocol::TType _etype2364;
+            xfer += iprot->readListBegin(_etype2364, _size2361);
+            this->success.resize(_size2361);
+            uint32_t _i2365;
+            for (_i2365 = 0; _i2365 < _size2361; ++_i2365)
             {
-              xfer += this->success[_i2366].read(iprot);
+              xfer += this->success[_i2365].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -37853,10 +37853,10 @@ uint32_t ThriftHiveMetastore_list_roles_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Role> ::const_iterator _iter2367;
-      for (_iter2367 = this->success.begin(); _iter2367 != this->success.end(); ++_iter2367)
+      std::vector<Role> ::const_iterator _iter2366;
+      for (_iter2366 = this->success.begin(); _iter2366 != this->success.end(); ++_iter2366)
       {
-        xfer += (*_iter2367).write(oprot);
+        xfer += (*_iter2366).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -37901,14 +37901,14 @@ uint32_t ThriftHiveMetastore_list_roles_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2368;
-            ::apache::thrift::protocol::TType _etype2371;
-            xfer += iprot->readListBegin(_etype2371, _size2368);
-            (*(this->success)).resize(_size2368);
-            uint32_t _i2372;
-            for (_i2372 = 0; _i2372 < _size2368; ++_i2372)
+            uint32_t _size2367;
+            ::apache::thrift::protocol::TType _etype2370;
+            xfer += iprot->readListBegin(_etype2370, _size2367);
+            (*(this->success)).resize(_size2367);
+            uint32_t _i2371;
+            for (_i2371 = 0; _i2371 < _size2367; ++_i2371)
             {
-              xfer += (*(this->success))[_i2372].read(iprot);
+              xfer += (*(this->success))[_i2371].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -38604,14 +38604,14 @@ uint32_t ThriftHiveMetastore_get_privilege_set_args::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size2373;
-            ::apache::thrift::protocol::TType _etype2376;
-            xfer += iprot->readListBegin(_etype2376, _size2373);
-            this->group_names.resize(_size2373);
-            uint32_t _i2377;
-            for (_i2377 = 0; _i2377 < _size2373; ++_i2377)
+            uint32_t _size2372;
+            ::apache::thrift::protocol::TType _etype2375;
+            xfer += iprot->readListBegin(_etype2375, _size2372);
+            this->group_names.resize(_size2372);
+            uint32_t _i2376;
+            for (_i2376 = 0; _i2376 < _size2372; ++_i2376)
             {
-              xfer += iprot->readString(this->group_names[_i2377]);
+              xfer += iprot->readString(this->group_names[_i2376]);
             }
             xfer += iprot->readListEnd();
           }
@@ -38648,10 +38648,10 @@ uint32_t ThriftHiveMetastore_get_privilege_set_args::write(::apache::thrift::pro
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter2378;
-    for (_iter2378 = this->group_names.begin(); _iter2378 != this->group_names.end(); ++_iter2378)
+    std::vector<std::string> ::const_iterator _iter2377;
+    for (_iter2377 = this->group_names.begin(); _iter2377 != this->group_names.end(); ++_iter2377)
     {
-      xfer += oprot->writeString((*_iter2378));
+      xfer += oprot->writeString((*_iter2377));
     }
     xfer += oprot->writeListEnd();
   }
@@ -38683,10 +38683,10 @@ uint32_t ThriftHiveMetastore_get_privilege_set_pargs::write(::apache::thrift::pr
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter2379;
-    for (_iter2379 = (*(this->group_names)).begin(); _iter2379 != (*(this->group_names)).end(); ++_iter2379)
+    std::vector<std::string> ::const_iterator _iter2378;
+    for (_iter2378 = (*(this->group_names)).begin(); _iter2378 != (*(this->group_names)).end(); ++_iter2378)
     {
-      xfer += oprot->writeString((*_iter2379));
+      xfer += oprot->writeString((*_iter2378));
     }
     xfer += oprot->writeListEnd();
   }
@@ -38861,9 +38861,9 @@ uint32_t ThriftHiveMetastore_list_privileges_args::read(::apache::thrift::protoc
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast2380;
-          xfer += iprot->readI32(ecast2380);
-          this->principal_type = (PrincipalType::type)ecast2380;
+          int32_t ecast2379;
+          xfer += iprot->readI32(ecast2379);
+          this->principal_type = (PrincipalType::type)ecast2379;
           this->__isset.principal_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -38968,14 +38968,14 @@ uint32_t ThriftHiveMetastore_list_privileges_result::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2381;
-            ::apache::thrift::protocol::TType _etype2384;
-            xfer += iprot->readListBegin(_etype2384, _size2381);
-            this->success.resize(_size2381);
-            uint32_t _i2385;
-            for (_i2385 = 0; _i2385 < _size2381; ++_i2385)
+            uint32_t _size2380;
+            ::apache::thrift::protocol::TType _etype2383;
+            xfer += iprot->readListBegin(_etype2383, _size2380);
+            this->success.resize(_size2380);
+            uint32_t _i2384;
+            for (_i2384 = 0; _i2384 < _size2380; ++_i2384)
             {
-              xfer += this->success[_i2385].read(iprot);
+              xfer += this->success[_i2384].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -39014,10 +39014,10 @@ uint32_t ThriftHiveMetastore_list_privileges_result::write(::apache::thrift::pro
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<HiveObjectPrivilege> ::const_iterator _iter2386;
-      for (_iter2386 = this->success.begin(); _iter2386 != this->success.end(); ++_iter2386)
+      std::vector<HiveObjectPrivilege> ::const_iterator _iter2385;
+      for (_iter2385 = this->success.begin(); _iter2385 != this->success.end(); ++_iter2385)
       {
-        xfer += (*_iter2386).write(oprot);
+        xfer += (*_iter2385).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -39062,14 +39062,14 @@ uint32_t ThriftHiveMetastore_list_privileges_presult::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2387;
-            ::apache::thrift::protocol::TType _etype2390;
-            xfer += iprot->readListBegin(_etype2390, _size2387);
-            (*(this->success)).resize(_size2387);
-            uint32_t _i2391;
-            for (_i2391 = 0; _i2391 < _size2387; ++_i2391)
+            uint32_t _size2386;
+            ::apache::thrift::protocol::TType _etype2389;
+            xfer += iprot->readListBegin(_etype2389, _size2386);
+            (*(this->success)).resize(_size2386);
+            uint32_t _i2390;
+            for (_i2390 = 0; _i2390 < _size2386; ++_i2390)
             {
-              xfer += (*(this->success))[_i2391].read(iprot);
+              xfer += (*(this->success))[_i2390].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -39996,14 +39996,14 @@ uint32_t ThriftHiveMetastore_set_ugi_args::read(::apache::thrift::protocol::TPro
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size2392;
-            ::apache::thrift::protocol::TType _etype2395;
-            xfer += iprot->readListBegin(_etype2395, _size2392);
-            this->group_names.resize(_size2392);
-            uint32_t _i2396;
-            for (_i2396 = 0; _i2396 < _size2392; ++_i2396)
+            uint32_t _size2391;
+            ::apache::thrift::protocol::TType _etype2394;
+            xfer += iprot->readListBegin(_etype2394, _size2391);
+            this->group_names.resize(_size2391);
+            uint32_t _i2395;
+            for (_i2395 = 0; _i2395 < _size2391; ++_i2395)
             {
-              xfer += iprot->readString(this->group_names[_i2396]);
+              xfer += iprot->readString(this->group_names[_i2395]);
             }
             xfer += iprot->readListEnd();
           }
@@ -40036,10 +40036,10 @@ uint32_t ThriftHiveMetastore_set_ugi_args::write(::apache::thrift::protocol::TPr
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter2397;
-    for (_iter2397 = this->group_names.begin(); _iter2397 != this->group_names.end(); ++_iter2397)
+    std::vector<std::string> ::const_iterator _iter2396;
+    for (_iter2396 = this->group_names.begin(); _iter2396 != this->group_names.end(); ++_iter2396)
     {
-      xfer += oprot->writeString((*_iter2397));
+      xfer += oprot->writeString((*_iter2396));
     }
     xfer += oprot->writeListEnd();
   }
@@ -40067,10 +40067,10 @@ uint32_t ThriftHiveMetastore_set_ugi_pargs::write(::apache::thrift::protocol::TP
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter2398;
-    for (_iter2398 = (*(this->group_names)).begin(); _iter2398 != (*(this->group_names)).end(); ++_iter2398)
+    std::vector<std::string> ::const_iterator _iter2397;
+    for (_iter2397 = (*(this->group_names)).begin(); _iter2397 != (*(this->group_names)).end(); ++_iter2397)
     {
-      xfer += oprot->writeString((*_iter2398));
+      xfer += oprot->writeString((*_iter2397));
     }
     xfer += oprot->writeListEnd();
   }
@@ -40111,14 +40111,14 @@ uint32_t ThriftHiveMetastore_set_ugi_result::read(::apache::thrift::protocol::TP
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2399;
-            ::apache::thrift::protocol::TType _etype2402;
-            xfer += iprot->readListBegin(_etype2402, _size2399);
-            this->success.resize(_size2399);
-            uint32_t _i2403;
-            for (_i2403 = 0; _i2403 < _size2399; ++_i2403)
+            uint32_t _size2398;
+            ::apache::thrift::protocol::TType _etype2401;
+            xfer += iprot->readListBegin(_etype2401, _size2398);
+            this->success.resize(_size2398);
+            uint32_t _i2402;
+            for (_i2402 = 0; _i2402 < _size2398; ++_i2402)
             {
-              xfer += iprot->readString(this->success[_i2403]);
+              xfer += iprot->readString(this->success[_i2402]);
             }
             xfer += iprot->readListEnd();
           }
@@ -40157,10 +40157,10 @@ uint32_t ThriftHiveMetastore_set_ugi_result::write(::apache::thrift::protocol::T
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter2404;
-      for (_iter2404 = this->success.begin(); _iter2404 != this->success.end(); ++_iter2404)
+      std::vector<std::string> ::const_iterator _iter2403;
+      for (_iter2403 = this->success.begin(); _iter2403 != this->success.end(); ++_iter2403)
       {
-        xfer += oprot->writeString((*_iter2404));
+        xfer += oprot->writeString((*_iter2403));
       }
       xfer += oprot->writeListEnd();
     }
@@ -40205,14 +40205,14 @@ uint32_t ThriftHiveMetastore_set_ugi_presult::read(::apache::thrift::protocol::T
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2405;
-            ::apache::thrift::protocol::TType _etype2408;
-            xfer += iprot->readListBegin(_etype2408, _size2405);
-            (*(this->success)).resize(_size2405);
-            uint32_t _i2409;
-            for (_i2409 = 0; _i2409 < _size2405; ++_i2409)
+            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)
             {
-              xfer += iprot->readString((*(this->success))[_i2409]);
+              xfer += iprot->readString((*(this->success))[_i2408]);
             }
             xfer += iprot->readListEnd();
           }
@@ -41523,14 +41523,14 @@ uint32_t ThriftHiveMetastore_get_all_token_identifiers_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _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 _size2409;
+            ::apache::thrift::protocol::TType _etype2412;
+            xfer += iprot->readListBegin(_etype2412, _size2409);
+            this->success.resize(_size2409);
+            uint32_t _i2413;
+            for (_i2413 = 0; _i2413 < _size2409; ++_i2413)
             {
-              xfer += iprot->readString(this->success[_i2414]);
+              xfer += iprot->readString(this->success[_i2413]);
             }
             xfer += iprot->readListEnd();
           }
@@ -41561,10 +41561,10 @@ uint32_t ThriftHiveMetastore_get_all_token_identifiers_result::write(::apache::t
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter2415;
-      for (_iter2415 = this->success.begin(); _iter2415 != this->success.end(); ++_iter2415)
+      std::vector<std::string> ::const_iterator _iter2414;
+      for (_iter2414 = this->success.begin(); _iter2414 != this->success.end(); ++_iter2414)
       {
-        xfer += oprot->writeString((*_iter2415));
+        xfer += oprot->writeString((*_iter2414));
       }
       xfer += oprot->writeListEnd();
     }
@@ -41605,14 +41605,14 @@ uint32_t ThriftHiveMetastore_get_all_token_identifiers_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2416;
-            ::apache::thrift::protocol::TType _etype2419;
-            xfer += iprot->readListBegin(_etype2419, _size2416);
-            (*(this->success)).resize(_size2416);
-            uint32_t _i2420;
-            for (_i2420 = 0; _i2420 < _size2416; ++_i2420)
+            uint32_t _size2415;
+            ::apache::thrift::protocol::TType _etype2418;
+            xfer += iprot->readListBegin(_etype2418, _size2415);
+            (*(this->success)).resize(_size2415);
+            uint32_t _i2419;
+            for (_i2419 = 0; _i2419 < _size2415; ++_i2419)
             {
-              xfer += iprot->readString((*(this->success))[_i2420]);
+              xfer += iprot->readString((*(this->success))[_i2419]);
             }
             xfer += iprot->readListEnd();
           }
@@ -42338,14 +42338,14 @@ uint32_t ThriftHiveMetastore_get_master_keys_result::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2421;
-            ::apache::thrift::protocol::TType _etype2424;
-            xfer += iprot->readListBegin(_etype2424, _size2421);
-            this->success.resize(_size2421);
-            uint32_t _i2425;
-            for (_i2425 = 0; _i2425 < _size2421; ++_i2425)
+            uint32_t _size2420;
+            ::apache::thrift::protocol::TType _etype2423;
+            xfer += iprot->readListBegin(_etype2423, _size2420);
+            this->success.resize(_size2420);
+            uint32_t _i2424;
+            for (_i2424 = 0; _i2424 < _size2420; ++_i2424)
             {
-              xfer += iprot->readString(this->success[_i2425]);
+              xfer += iprot->readString(this->success[_i2424]);
             }
             xfer += iprot->readListEnd();
           }
@@ -42376,10 +42376,10 @@ uint32_t ThriftHiveMetastore_get_master_keys_result::write(::apache::thrift::pro
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter2426;
-      for (_iter2426 = this->success.begin(); _iter2426 != this->success.end(); ++_iter2426)
+      std::vector<std::string> ::const_iterator _iter2425;
+      for (_iter2425 = this->success.begin(); _iter2425 != this->success.end(); ++_iter2425)
       {
-        xfer += oprot->writeString((*_iter2426));
+        xfer += oprot->writeString((*_iter2425));
       }
       xfer += oprot->writeListEnd();
     }
@@ -42420,14 +42420,14 @@ uint32_t ThriftHiveMetastore_get_master_keys_presult::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2427;
-            ::apache::thrift::protocol::TType _etype2430;
-            xfer += iprot->readListBegin(_etype2430, _size2427);
-            (*(this->success)).resize(_size2427);
-            uint32_t _i2431;
-            for (_i2431 = 0; _i2431 < _size2427; ++_i2431)
+            uint32_t _size2426;
+            ::apache::thrift::protocol::TType _etype2429;
+            xfer += iprot->readListBegin(_etype2429, _size2426);
+            (*(this->success)).resize(_size2426);
+            uint32_t _i2430;
+            for (_i2430 = 0; _i2430 < _size2426; ++_i2430)
             {
-              xfer += iprot->readString((*(this->success))[_i2431]);
+              xfer += iprot->readString((*(this->success))[_i2430]);
             }
             xfer += iprot->readListEnd();
           }
@@ -47680,14 +47680,14 @@ uint32_t ThriftHiveMetastore_find_columns_with_stats_result::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2432;
-            ::apache::thrift::protocol::TType _etype2435;
-            xfer += iprot->readListBegin(_etype2435, _size2432);
-            this->success.resize(_size2432);
-            uint32_t _i2436;
-            for (_i2436 = 0; _i2436 < _size2432; ++_i2436)
+            uint32_t _size2431;
+            ::apache::thrift::protocol::TType _etype2434;
+            xfer += iprot->readListBegin(_etype2434, _size2431);
+            this->success.resize(_size2431);
+            uint32_t _i2435;
+            for (_i2435 = 0; _i2435 < _size2431; ++_i2435)
             {
-              xfer += iprot->readString(this->success[_i2436]);
+              xfer += iprot->readString(this->success[_i2435]);
             }
             xfer += iprot->readListEnd();
           }
@@ -47718,10 +47718,10 @@ uint32_t ThriftHiveMetastore_find_columns_with_stats_result::write(::apache::thr
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter2437;
-      for (_iter2437 = this->success.begin(); _iter2437 != this->success.end(); ++_iter2437)
+      std::vector<std::string> ::const_iterator _iter2436;
+      for (_iter2436 = this->success.begin(); _iter2436 != this->success.end(); ++_iter2436)
       {
-        xfer += oprot->writeString((*_iter2437));
+        xfer += oprot->writeString((*_iter2436));
       }
       xfer += oprot->writeListEnd();
     }
@@ -47762,14 +47762,14 @@ uint32_t ThriftHiveMetastore_find_columns_with_stats_presult::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2438;
-            ::apache::thrift::protocol::TType _etype2441;
-            xfer += iprot->readListBegin(_etype2441, _size2438);
-            (*(this->success)).resize(_size2438);
-            uint32_t _i2442;
-            for (_i2442 = 0; _i2442 < _size2438; ++_i2442)
+            uint32_t _size2437;
+            ::apache::thrift::protocol::TType _etype2440;
+            xfer += iprot->readListBegin(_etype2440, _size2437);
+            (*(this->success)).resize(_size2437);
+            uint32_t _i2441;
+            for (_i2441 = 0; _i2441 < _size2437; ++_i2441)
             {
-              xfer += iprot->readString((*(this->success))[_i2442]);
+              xfer += iprot->readString((*(this->success))[_i2441]);
             }
             xfer += iprot->readListEnd();
           }
@@ -57505,14 +57505,14 @@ uint32_t ThriftHiveMetastore_get_schema_all_versions_result::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2443;
-            ::apache::thrift::protocol::TType _etype2446;
-            xfer += iprot->readListBegin(_etype2446, _size2443);
-            this->success.resize(_size2443);
-            uint32_t _i2447;
-            for (_i2447 = 0; _i2447 < _size2443; ++_i2447)
+            uint32_t _size2442;
+            ::apache::thrift::protocol::TType _etype2445;
+            xfer += iprot->readListBegin(_etype2445, _size2442);
+            this->success.resize(_size2442);
+            uint32_t _i2446;
+            for (_i2446 = 0; _i2446 < _size2442; ++_i2446)
             {
-              xfer += this->success[_i2447].read(iprot);
+              xfer += this->success[_i2446].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -57559,10 +57559,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 _iter2448;
-      for (_iter2448 = this->success.begin(); _iter2448 != this->success.end(); ++_iter2448)
+      std::vector<SchemaVersion> ::const_iterator _iter2447;
+      for (_iter2447 = this->success.begin(); _iter2447 != this->success.end(); ++_iter2447)
       {
-        xfer += (*_iter2448).write(oprot);
+        xfer += (*_iter2447).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -57611,14 +57611,14 @@ uint32_t ThriftHiveMetastore_get_schema_all_versions_presult::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2449;
-            ::apache::thrift::protocol::TType _etype2452;
-            xfer += iprot->readListBegin(_etype2452, _size2449);
-            (*(this->success)).resize(_size2449);
-            uint32_t _i2453;
-            for (_i2453 = 0; _i2453 < _size2449; ++_i2453)
+            uint32_t _size2448;
+            ::apache::thrift::protocol::TType _etype2451;
+            xfer += iprot->readListBegin(_etype2451, _size2448);
+            (*(this->success)).resize(_size2448);
+            uint32_t _i2452;
+            for (_i2452 = 0; _i2452 < _size2448; ++_i2452)
             {
-              xfer += (*(this->success))[_i2453].read(iprot);
+              xfer += (*(this->success))[_i2452].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -59671,14 +59671,14 @@ uint32_t ThriftHiveMetastore_get_runtime_stats_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2454;
-            ::apache::thrift::protocol::TType _etype2457;
-            xfer += iprot->readListBegin(_etype2457, _size2454);
-            this->success.resize(_size2454);
-            uint32_t _i2458;
-            for (_i2458 = 0; _i2458 < _size2454; ++_i2458)
+            uint32_t _size2453;
+            ::apache::thrift::protocol::TType _etype2456;
+            xfer += iprot->readListBegin(_etype2456, _size2453);
+            this->success.resize(_size2453);
+            uint32_t _i2457;
+            for (_i2457 = 0; _i2457 < _size2453; ++_i2457)
             {
-              xfer += this->success[_i2458].read(iprot);
+              xfer += this->success[_i2457].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -59717,10 +59717,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 _iter2459;
-      for (_iter2459 = this->success.begin(); _iter2459 != this->success.end(); ++_iter2459)
+      std::vector<RuntimeStat> ::const_iterator _iter2458;
+      for (_iter2458 = this->success.begin(); _iter2458 != this->success.end(); ++_iter2458)
       {
-        xfer += (*_iter2459).write(oprot);
+        xfer += (*_iter2458).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -59765,14 +59765,14 @@ uint32_t ThriftHiveMetastore_get_runtime_stats_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2460;
-            ::apache::thrift::protocol::TType _etype2463;
-            xfer += iprot->readListBegin(_etype2463, _size2460);
-            (*(this->success)).resize(_size2460);
-            uint32_t _i2464;
-            for (_i2464 = 0; _i2464 < _size2460; ++_i2464)
+            uint32_t _size2459;
+            ::apache::thrift::protocol::TType _etype2462;
+            xfer += iprot->readListBegin(_etype2462, _size2459);
+            (*(this->success)).resize(_size2459);
+            uint32_t _i2463;
+            for (_i2463 = 0; _i2463 < _size2459; ++_i2463)
             {
-              xfer += (*(this->success))[_i2464].read(iprot);
+              xfer += (*(this->success))[_i2463].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -62207,14 +62207,14 @@ uint32_t ThriftHiveMetastore_get_all_stored_procedures_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size2465;
-            ::apache::thrift::protocol::TType _etype2468;
-            xfer += iprot->readListBegin(_etype2468, _size2465);
-            this->success.resize(_size2465);
-            uint32_t _i2469;
-            for (_i2469 = 0; _i2469 < _size2465; ++_i2469)
+            uint32_t _size2464;
+            ::apache::thrift::protocol::TType _etype2467;
+            xfer += iprot->readListBegin(_etype2467, _size2464);
+            this->success.resize(_size2464);
+            uint32_t _i2468;
+            for (_i2468 = 0; _i2468 < _size2464; ++_i2468)
             {
-              xfer += iprot->readString(this->success[_i2469]);
+              xfer += iprot->readString(this->success[_i2468]);
             }
             xfer += iprot->readListEnd();
           }
@@ -62253,10 +62253,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 _iter2470;
-      for (_iter2470 = this->success.begin(); _iter2470 != this->success.end(); ++_iter2470)
+      std::vector<std::string> ::const_iterator _iter2469;
+      for (_iter2469 = this->success.begin(); _iter2469 != this->success.end(); ++_iter2469)
       {
-        xfer += oprot->writeString((*_iter2470));
+        xfer += oprot->writeString((*_iter2469));
       }
       xfer += oprot->writeListEnd();
     }
@@ -62301,14 +62301,14 @@ uint32_t ThriftHiveMetastore_get_all_stored_procedures_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size2471;
-            ::apache::thrift::protocol::TType _etype2474;
-            xfer += iprot->readListBegin(_etype2474, _size2471);
-            (*(this->success)).resize(_size2471);
-            uint32_t _i2475;
-            for (_i2475 = 0; _i2475 < _size2471; ++_i2475)
+            uint32_t _size2470;
+            ::apache::thrift::protocol::TType _etype2473;
+            xfer += iprot->readListBegin(_etype2473, _size2470);
+            (*(this->success)).resize(_size2470);
+            uint32_t _i2474;
+            for (_i2474 = 0; _i2474 < _size2470; ++_i2474)
             {
-              xfer += iprot->readString((*(this->success))[_i2475]);
+              xfer += iprot->readString((*(this->success))[_i2474]);
             }
             xfer += iprot->readListEnd();
           }
@@ -62860,14 +62860,14 @@ uint32_t ThriftHiveMetastore_get_all_packages_result::read(::apache::thrift::pro
         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 _size2475;
+            ::apache::thrift::protocol::TType _etype2478;
+            xfer += iprot->readListBegin(_etype2478, _size2475);
+            this->success.resize(_size2475);
+            uint32_t _i2479;
+            for (_i2479 = 0; _i2479 < _size2475; ++_i2479)
             {
-              xfer += iprot->readString(this->success[_i2480]);
+              xfer += iprot->readString(this->success[_i2479]);
             }
             xfer += iprot->readListEnd();
           }
@@ -62906,10 +62906,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 _iter2481;
-      for (_iter2481 = this->success.begin(); _iter2481 != this->success.end(); ++_iter2481)
+      std::vector<std::string> ::const_iterator _iter2480;
+      for (_iter2480 = this->success.begin(); _iter2480 != this->success.end(); ++_iter2480)
       {
-        xfer += oprot->writeString((*_iter2481));
+        xfer += oprot->writeString((*_iter2480));
       }
       xfer += oprot->writeListEnd();
     }
@@ -62954,14 +62954,14 @@ uint32_t ThriftHiveMetastore_get_all_packages_presult::read(::apache::thrift::pr
         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 _size2481;
+            ::apache::thrift::protocol::TType _etype2484;
+            xfer += iprot->readListBegin(_etype2484, _size2481);
+            (*(this->success)).resize(_size2481);
+            uint32_t _i2485;
+            for (_i2485 = 0; _i2485 < _size2481; ++_i2485)
             {
-              xfer += iprot->readString((*(this->success))[_i2486]);
+              xfer += iprot->readString((*(this->success))[_i2485]);
             }
             xfer += iprot->readListEnd();
           }
@@ -63286,14 +63286,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 _size2487;
-            ::apache::thrift::protocol::TType _etype2490;
-            xfer += iprot->readListBegin(_etype2490, _size2487);
-            this->success.resize(_size2487);
-            uint32_t _i2491;
-            for (_i2491 = 0; _i2491 < _size2487; ++_i2491)
+            uint32_t _size2486;
+            ::apache::thrift::protocol::TType _etype2489;
+            xfer += iprot->readListBegin(_etype2489, _size2486);
+            this->success.resize(_size2486);
+            uint32_t _i2490;
+            for (_i2490 = 0; _i2490 < _size2486; ++_i2490)
             {
-              xfer += this->success[_i2491].read(iprot);
+              xfer += this->success[_i2490].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -63332,10 +63332,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 _iter2492;
-      for (_iter2492 = this->success.begin(); _iter2492 != this->success.end(); ++_iter2492)
+      std::vector<WriteEventInfo> ::const_iterator _iter2491;
+      for (_iter2491 = this->success.begin(); _iter2491 != this->success.end(); ++_iter2491)
       {
-        xfer += (*_iter2492).write(oprot);
+        xfer += (*_iter2491).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -63380,14 +63380,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 _size2493;
-            ::apache::thrift::protocol::TType _etype2496;
-            xfer += iprot->readListBegin(_etype2496, _size2493);
-            (*(this->success)).resize(_size2493);
-            uint32_t _i2497;
-            for (_i2497 = 0; _i2497 < _size2493; ++_i2497)
+            uint32_t _size2492;
+            ::apache::thrift::protocol::TType _etype2495;
+            xfer += iprot->readListBegin(_etype2495, _size2492);
+            (*(this->success)).resize(_size2492);
+            uint32_t _i2496;
+            for (_i2496 = 0; _i2496 < _size2492; ++_i2496)
             {
-              xfer += (*(this->success))[_i2497].read(iprot);
+              xfer += (*(this->success))[_i2496].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 0c32ef3..f6afabb 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
@@ -8086,167 +8086,6 @@ void StorageDescriptor::printTo(std::ostream& out) const {
 }
 
 
-SourceTable::~SourceTable() noexcept {
-}
-
-
-void SourceTable::__set_table(const Table& val) {
-  this->table = val;
-}
-
-void SourceTable::__set_insertedCount(const int64_t val) {
-  this->insertedCount = val;
-}
-
-void SourceTable::__set_updatedCount(const int64_t val) {
-  this->updatedCount = val;
-}
-
-void SourceTable::__set_deletedCount(const int64_t val) {
-  this->deletedCount = val;
-}
-std::ostream& operator<<(std::ostream& out, const SourceTable& obj)
-{
-  obj.printTo(out);
-  return out;
-}
-
-
-uint32_t SourceTable::read(::apache::thrift::protocol::TProtocol* iprot) {
-
-  ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
-  uint32_t xfer = 0;
-  std::string fname;
-  ::apache::thrift::protocol::TType ftype;
-  int16_t fid;
-
-  xfer += iprot->readStructBegin(fname);
-
-  using ::apache::thrift::protocol::TProtocolException;
-
-  bool isset_table = false;
-  bool isset_insertedCount = false;
-  bool isset_updatedCount = false;
-  bool isset_deletedCount = false;
-
-  while (true)
-  {
-    xfer += iprot->readFieldBegin(fname, ftype, fid);
-    if (ftype == ::apache::thrift::protocol::T_STOP) {
-      break;
-    }
-    switch (fid)
-    {
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->table.read(iprot);
-          isset_table = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 2:
-        if (ftype == ::apache::thrift::protocol::T_I64) {
-          xfer += iprot->readI64(this->insertedCount);
-          isset_insertedCount = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 3:
-        if (ftype == ::apache::thrift::protocol::T_I64) {
-          xfer += iprot->readI64(this->updatedCount);
-          isset_updatedCount = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 4:
-        if (ftype == ::apache::thrift::protocol::T_I64) {
-          xfer += iprot->readI64(this->deletedCount);
-          isset_deletedCount = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  if (!isset_table)
-    throw TProtocolException(TProtocolException::INVALID_DATA);
-  if (!isset_insertedCount)
-    throw TProtocolException(TProtocolException::INVALID_DATA);
-  if (!isset_updatedCount)
-    throw TProtocolException(TProtocolException::INVALID_DATA);
-  if (!isset_deletedCount)
-    throw TProtocolException(TProtocolException::INVALID_DATA);
-  return xfer;
-}
-
-uint32_t SourceTable::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("SourceTable");
-
-  xfer += oprot->writeFieldBegin("table", ::apache::thrift::protocol::T_STRUCT, 1);
-  xfer += this->table.write(oprot);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("insertedCount", ::apache::thrift::protocol::T_I64, 2);
-  xfer += oprot->writeI64(this->insertedCount);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("updatedCount", ::apache::thrift::protocol::T_I64, 3);
-  xfer += oprot->writeI64(this->updatedCount);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("deletedCount", ::apache::thrift::protocol::T_I64, 4);
-  xfer += oprot->writeI64(this->deletedCount);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-void swap(SourceTable &a, SourceTable &b) {
-  using ::std::swap;
-  swap(a.table, b.table);
-  swap(a.insertedCount, b.insertedCount);
-  swap(a.updatedCount, b.updatedCount);
-  swap(a.deletedCount, b.deletedCount);
-}
-
-SourceTable::SourceTable(const SourceTable& other295) {
-  table = other295.table;
-  insertedCount = other295.insertedCount;
-  updatedCount = other295.updatedCount;
-  deletedCount = other295.deletedCount;
-}
-SourceTable& SourceTable::operator=(const SourceTable& other296) {
-  table = other296.table;
-  insertedCount = other296.insertedCount;
-  updatedCount = other296.updatedCount;
-  deletedCount = other296.deletedCount;
-  return *this;
-}
-void SourceTable::printTo(std::ostream& out) const {
-  using ::apache::thrift::to_string;
-  out << "SourceTable(";
-  out << "table=" << to_string(table);
-  out << ", " << "insertedCount=" << to_string(insertedCount);
-  out << ", " << "updatedCount=" << to_string(updatedCount);
-  out << ", " << "deletedCount=" << to_string(deletedCount);
-  out << ")";
-}
-
-
 CreationMetadata::~CreationMetadata() noexcept {
 }
 
@@ -8277,7 +8116,7 @@ void CreationMetadata::__set_materializationTime(const int64_t val) {
 __isset.materializationTime = true;
 }
 
-void CreationMetadata::__set_sourceTables(const std::set<SourceTable> & val) {
+void CreationMetadata::__set_sourceTables(const std::vector<SourceTable> & val) {
   this->sourceTables = val;
 __isset.sourceTables = true;
 }
@@ -8341,15 +8180,15 @@ uint32_t CreationMetadata::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             this->tablesUsed.clear();
-            uint32_t _size297;
-            ::apache::thrift::protocol::TType _etype300;
-            xfer += iprot->readSetBegin(_etype300, _size297);
-            uint32_t _i301;
-            for (_i301 = 0; _i301 < _size297; ++_i301)
+            uint32_t _size295;
+            ::apache::thrift::protocol::TType _etype298;
+            xfer += iprot->readSetBegin(_etype298, _size295);
+            uint32_t _i299;
+            for (_i299 = 0; _i299 < _size295; ++_i299)
             {
-              std::string _elem302;
-              xfer += iprot->readString(_elem302);
-              this->tablesUsed.insert(_elem302);
+              std::string _elem300;
+              xfer += iprot->readString(_elem300);
+              this->tablesUsed.insert(_elem300);
             }
             xfer += iprot->readSetEnd();
           }
@@ -8375,20 +8214,19 @@ uint32_t CreationMetadata::read(::apache::thrift::protocol::TProtocol* iprot) {
         }
         break;
       case 7:
-        if (ftype == ::apache::thrift::protocol::T_SET) {
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->sourceTables.clear();
-            uint32_t _size303;
-            ::apache::thrift::protocol::TType _etype306;
-            xfer += iprot->readSetBegin(_etype306, _size303);
-            uint32_t _i307;
-            for (_i307 = 0; _i307 < _size303; ++_i307)
+            uint32_t _size301;
+            ::apache::thrift::protocol::TType _etype304;
+            xfer += iprot->readListBegin(_etype304, _size301);
+            this->sourceTables.resize(_size301);
+            uint32_t _i305;
+            for (_i305 = 0; _i305 < _size301; ++_i305)
             {
-              SourceTable _elem308;
-              xfer += _elem308.read(iprot);
-              this->sourceTables.insert(_elem308);
+              xfer += this->sourceTables[_i305].read(iprot);
             }
-            xfer += iprot->readSetEnd();
+            xfer += iprot->readListEnd();
           }
           this->__isset.sourceTables = true;
         } else {
@@ -8435,10 +8273,10 @@ uint32_t CreationMetadata::write(::apache::thrift::protocol::TProtocol* oprot) c
   xfer += oprot->writeFieldBegin("tablesUsed", ::apache::thrift::protocol::T_SET, 4);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tablesUsed.size()));
-    std::set<std::string> ::const_iterator _iter309;
-    for (_iter309 = this->tablesUsed.begin(); _iter309 != this->tablesUsed.end(); ++_iter309)
+    std::set<std::string> ::const_iterator _iter306;
+    for (_iter306 = this->tablesUsed.begin(); _iter306 != this->tablesUsed.end(); ++_iter306)
     {
-      xfer += oprot->writeString((*_iter309));
+      xfer += oprot->writeString((*_iter306));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -8455,15 +8293,15 @@ uint32_t CreationMetadata::write(::apache::thrift::protocol::TProtocol* oprot) c
     xfer += oprot->writeFieldEnd();
   }
   if (this->__isset.sourceTables) {
-    xfer += oprot->writeFieldBegin("sourceTables", ::apache::thrift::protocol::T_SET, 7);
+    xfer += oprot->writeFieldBegin("sourceTables", ::apache::thrift::protocol::T_LIST, 7);
     {
-      xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->sourceTables.size()));
-      std::set<SourceTable> ::const_iterator _iter310;
-      for (_iter310 = this->sourceTables.begin(); _iter310 != this->sourceTables.end(); ++_iter310)
+      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->sourceTables.size()));
+      std::vector<SourceTable> ::const_iterator _iter307;
+      for (_iter307 = this->sourceTables.begin(); _iter307 != this->sourceTables.end(); ++_iter307)
       {
-        xfer += (*_iter310).write(oprot);
+        xfer += (*_iter307).write(oprot);
       }
-      xfer += oprot->writeSetEnd();
+      xfer += oprot->writeListEnd();
     }
     xfer += oprot->writeFieldEnd();
   }
@@ -8484,25 +8322,25 @@ void swap(CreationMetadata &a, CreationMetadata &b) {
   swap(a.__isset, b.__isset);
 }
 
-CreationMetadata::CreationMetadata(const CreationMetadata& other311) {
-  catName = other311.catName;
-  dbName = other311.dbName;
-  tblName = other311.tblName;
-  tablesUsed = other311.tablesUsed;
-  validTxnList = other311.validTxnList;
-  materializationTime = other311.materializationTime;
-  sourceTables = other311.sourceTables;
-  __isset = other311.__isset;
+CreationMetadata::CreationMetadata(const CreationMetadata& other308) {
+  catName = other308.catName;
+  dbName = other308.dbName;
+  tblName = other308.tblName;
+  tablesUsed = other308.tablesUsed;
+  validTxnList = other308.validTxnList;
+  materializationTime = other308.materializationTime;
+  sourceTables = other308.sourceTables;
+  __isset = other308.__isset;
 }
-CreationMetadata& CreationMetadata::operator=(const CreationMetadata& other312) {
-  catName = other312.catName;
-  dbName = other312.dbName;
-  tblName = other312.tblName;
-  tablesUsed = other312.tablesUsed;
-  validTxnList = other312.validTxnList;
-  materializationTime = other312.materializationTime;
-  sourceTables = other312.sourceTables;
-  __isset = other312.__isset;
+CreationMetadata& CreationMetadata::operator=(const CreationMetadata& other309) {
+  catName = other309.catName;
+  dbName = other309.dbName;
+  tblName = other309.tblName;
+  tablesUsed = other309.tablesUsed;
+  validTxnList = other309.validTxnList;
+  materializationTime = other309.materializationTime;
+  sourceTables = other309.sourceTables;
+  __isset = other309.__isset;
   return *this;
 }
 void CreationMetadata::printTo(std::ostream& out) const {
@@ -8656,19 +8494,19 @@ void swap(BooleanColumnStatsData &a, BooleanColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-BooleanColumnStatsData::BooleanColumnStatsData(const BooleanColumnStatsData& other313) {
-  numTrues = other313.numTrues;
-  numFalses = other313.numFalses;
-  numNulls = other313.numNulls;
-  bitVectors = other313.bitVectors;
-  __isset = other313.__isset;
+BooleanColumnStatsData::BooleanColumnStatsData(const BooleanColumnStatsData& other310) {
+  numTrues = other310.numTrues;
+  numFalses = other310.numFalses;
+  numNulls = other310.numNulls;
+  bitVectors = other310.bitVectors;
+  __isset = other310.__isset;
 }
-BooleanColumnStatsData& BooleanColumnStatsData::operator=(const BooleanColumnStatsData& other314) {
-  numTrues = other314.numTrues;
-  numFalses = other314.numFalses;
-  numNulls = other314.numNulls;
-  bitVectors = other314.bitVectors;
-  __isset = other314.__isset;
+BooleanColumnStatsData& BooleanColumnStatsData::operator=(const BooleanColumnStatsData& other311) {
+  numTrues = other311.numTrues;
+  numFalses = other311.numFalses;
+  numNulls = other311.numNulls;
+  bitVectors = other311.bitVectors;
+  __isset = other311.__isset;
   return *this;
 }
 void BooleanColumnStatsData::printTo(std::ostream& out) const {
@@ -8837,21 +8675,21 @@ void swap(DoubleColumnStatsData &a, DoubleColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-DoubleColumnStatsData::DoubleColumnStatsData(const DoubleColumnStatsData& other315) {
-  lowValue = other315.lowValue;
-  highValue = other315.highValue;
-  numNulls = other315.numNulls;
-  numDVs = other315.numDVs;
-  bitVectors = other315.bitVectors;
-  __isset = other315.__isset;
+DoubleColumnStatsData::DoubleColumnStatsData(const DoubleColumnStatsData& other312) {
+  lowValue = other312.lowValue;
+  highValue = other312.highValue;
+  numNulls = other312.numNulls;
+  numDVs = other312.numDVs;
+  bitVectors = other312.bitVectors;
+  __isset = other312.__isset;
 }
-DoubleColumnStatsData& DoubleColumnStatsData::operator=(const DoubleColumnStatsData& other316) {
-  lowValue = other316.lowValue;
-  highValue = other316.highValue;
-  numNulls = other316.numNulls;
-  numDVs = other316.numDVs;
-  bitVectors = other316.bitVectors;
-  __isset = other316.__isset;
+DoubleColumnStatsData& DoubleColumnStatsData::operator=(const DoubleColumnStatsData& other313) {
+  lowValue = other313.lowValue;
+  highValue = other313.highValue;
+  numNulls = other313.numNulls;
+  numDVs = other313.numDVs;
+  bitVectors = other313.bitVectors;
+  __isset = other313.__isset;
   return *this;
 }
 void DoubleColumnStatsData::printTo(std::ostream& out) const {
@@ -9021,21 +8859,21 @@ void swap(LongColumnStatsData &a, LongColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-LongColumnStatsData::LongColumnStatsData(const LongColumnStatsData& other317) {
-  lowValue = other317.lowValue;
-  highValue = other317.highValue;
-  numNulls = other317.numNulls;
-  numDVs = other317.numDVs;
-  bitVectors = other317.bitVectors;
-  __isset = other317.__isset;
+LongColumnStatsData::LongColumnStatsData(const LongColumnStatsData& other314) {
+  lowValue = other314.lowValue;
+  highValue = other314.highValue;
+  numNulls = other314.numNulls;
+  numDVs = other314.numDVs;
+  bitVectors = other314.bitVectors;
+  __isset = other314.__isset;
 }
-LongColumnStatsData& LongColumnStatsData::operator=(const LongColumnStatsData& other318) {
-  lowValue = other318.lowValue;
-  highValue = other318.highValue;
-  numNulls = other318.numNulls;
-  numDVs = other318.numDVs;
-  bitVectors = other318.bitVectors;
-  __isset = other318.__isset;
+LongColumnStatsData& LongColumnStatsData::operator=(const LongColumnStatsData& other315) {
+  lowValue = other315.lowValue;
+  highValue = other315.highValue;
+  numNulls = other315.numNulls;
+  numDVs = other315.numDVs;
+  bitVectors = other315.bitVectors;
+  __isset = other315.__isset;
   return *this;
 }
 void LongColumnStatsData::printTo(std::ostream& out) const {
@@ -9207,21 +9045,21 @@ void swap(StringColumnStatsData &a, StringColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-StringColumnStatsData::StringColumnStatsData(const StringColumnStatsData& other319) {
-  maxColLen = other319.maxColLen;
-  avgColLen = other319.avgColLen;
-  numNulls = other319.numNulls;
-  numDVs = other319.numDVs;
-  bitVectors = other319.bitVectors;
-  __isset = other319.__isset;
+StringColumnStatsData::StringColumnStatsData(const StringColumnStatsData& other316) {
+  maxColLen = other316.maxColLen;
+  avgColLen = other316.avgColLen;
+  numNulls = other316.numNulls;
+  numDVs = other316.numDVs;
+  bitVectors = other316.bitVectors;
+  __isset = other316.__isset;
 }
-StringColumnStatsData& StringColumnStatsData::operator=(const StringColumnStatsData& other320) {
-  maxColLen = other320.maxColLen;
-  avgColLen = other320.avgColLen;
-  numNulls = other320.numNulls;
-  numDVs = other320.numDVs;
-  bitVectors = other320.bitVectors;
-  __isset = other320.__isset;
+StringColumnStatsData& StringColumnStatsData::operator=(const StringColumnStatsData& other317) {
+  maxColLen = other317.maxColLen;
+  avgColLen = other317.avgColLen;
+  numNulls = other317.numNulls;
+  numDVs = other317.numDVs;
+  bitVectors = other317.bitVectors;
+  __isset = other317.__isset;
   return *this;
 }
 void StringColumnStatsData::printTo(std::ostream& out) const {
@@ -9373,19 +9211,19 @@ void swap(BinaryColumnStatsData &a, BinaryColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-BinaryColumnStatsData::BinaryColumnStatsData(const BinaryColumnStatsData& other321) {
-  maxColLen = other321.maxColLen;
-  avgColLen = other321.avgColLen;
-  numNulls = other321.numNulls;
-  bitVectors = other321.bitVectors;
-  __isset = other321.__isset;
+BinaryColumnStatsData::BinaryColumnStatsData(const BinaryColumnStatsData& other318) {
+  maxColLen = other318.maxColLen;
+  avgColLen = other318.avgColLen;
+  numNulls = other318.numNulls;
+  bitVectors = other318.bitVectors;
+  __isset = other318.__isset;
 }
-BinaryColumnStatsData& BinaryColumnStatsData::operator=(const BinaryColumnStatsData& other322) {
-  maxColLen = other322.maxColLen;
-  avgColLen = other322.avgColLen;
-  numNulls = other322.numNulls;
-  bitVectors = other322.bitVectors;
-  __isset = other322.__isset;
+BinaryColumnStatsData& BinaryColumnStatsData::operator=(const BinaryColumnStatsData& other319) {
+  maxColLen = other319.maxColLen;
+  avgColLen = other319.avgColLen;
+  numNulls = other319.numNulls;
+  bitVectors = other319.bitVectors;
+  __isset = other319.__isset;
   return *this;
 }
 void BinaryColumnStatsData::printTo(std::ostream& out) const {
@@ -9496,13 +9334,13 @@ void swap(Decimal &a, Decimal &b) {
   swap(a.unscaled, b.unscaled);
 }
 
-Decimal::Decimal(const Decimal& other323) {
-  scale = other323.scale;
-  unscaled = other323.unscaled;
+Decimal::Decimal(const Decimal& other320) {
+  scale = other320.scale;
+  unscaled = other320.unscaled;
 }
-Decimal& Decimal::operator=(const Decimal& other324) {
-  scale = other324.scale;
-  unscaled = other324.unscaled;
+Decimal& Decimal::operator=(const Decimal& other321) {
+  scale = other321.scale;
+  unscaled = other321.unscaled;
   return *this;
 }
 void Decimal::printTo(std::ostream& out) const {
@@ -9669,21 +9507,21 @@ void swap(DecimalColumnStatsData &a, DecimalColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-DecimalColumnStatsData::DecimalColumnStatsData(const DecimalColumnStatsData& other325) {
-  lowValue = other325.lowValue;
-  highValue = other325.highValue;
-  numNulls = other325.numNulls;
-  numDVs = other325.numDVs;
-  bitVectors = other325.bitVectors;
-  __isset = other325.__isset;
+DecimalColumnStatsData::DecimalColumnStatsData(const DecimalColumnStatsData& other322) {
+  lowValue = other322.lowValue;
+  highValue = other322.highValue;
+  numNulls = other322.numNulls;
+  numDVs = other322.numDVs;
+  bitVectors = other322.bitVectors;
+  __isset = other322.__isset;
 }
-DecimalColumnStatsData& DecimalColumnStatsData::operator=(const DecimalColumnStatsData& other326) {
-  lowValue = other326.lowValue;
-  highValue = other326.highValue;
-  numNulls = other326.numNulls;
-  numDVs = other326.numDVs;
-  bitVectors = other326.bitVectors;
-  __isset = other326.__isset;
+DecimalColumnStatsData& DecimalColumnStatsData::operator=(const DecimalColumnStatsData& other323) {
+  lowValue = other323.lowValue;
+  highValue = other323.highValue;
+  numNulls = other323.numNulls;
+  numDVs = other323.numDVs;
+  bitVectors = other323.bitVectors;
+  __isset = other323.__isset;
   return *this;
 }
 void DecimalColumnStatsData::printTo(std::ostream& out) const {
@@ -9775,11 +9613,11 @@ void swap(Date &a, Date &b) {
   swap(a.daysSinceEpoch, b.daysSinceEpoch);
 }
 
-Date::Date(const Date& other327) {
-  daysSinceEpoch = other327.daysSinceEpoch;
+Date::Date(const Date& other324) {
+  daysSinceEpoch = other324.daysSinceEpoch;
 }
-Date& Date::operator=(const Date& other328) {
-  daysSinceEpoch = other328.daysSinceEpoch;
+Date& Date::operator=(const Date& other325) {
+  daysSinceEpoch = other325.daysSinceEpoch;
   return *this;
 }
 void Date::printTo(std::ostream& out) const {
@@ -9945,21 +9783,21 @@ void swap(DateColumnStatsData &a, DateColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-DateColumnStatsData::DateColumnStatsData(const DateColumnStatsData& other329) {
-  lowValue = other329.lowValue;
-  highValue = other329.highValue;
-  numNulls = other329.numNulls;
-  numDVs = other329.numDVs;
-  bitVectors = other329.bitVectors;
-  __isset = other329.__isset;
+DateColumnStatsData::DateColumnStatsData(const DateColumnStatsData& other326) {
+  lowValue = other326.lowValue;
+  highValue = other326.highValue;
+  numNulls = other326.numNulls;
+  numDVs = other326.numDVs;
+  bitVectors = other326.bitVectors;
+  __isset = other326.__isset;
 }
-DateColumnStatsData& DateColumnStatsData::operator=(const DateColumnStatsData& other330) {
-  lowValue = other330.lowValue;
-  highValue = other330.highValue;
-  numNulls = other330.numNulls;
-  numDVs = other330.numDVs;
-  bitVectors = other330.bitVectors;
-  __isset = other330.__isset;
+DateColumnStatsData& DateColumnStatsData::operator=(const DateColumnStatsData& other327) {
+  lowValue = other327.lowValue;
+  highValue = other327.highValue;
+  numNulls = other327.numNulls;
+  numDVs = other327.numDVs;
+  bitVectors = other327.bitVectors;
+  __isset = other327.__isset;
   return *this;
 }
 void DateColumnStatsData::printTo(std::ostream& out) const {
@@ -10051,11 +9889,11 @@ void swap(Timestamp &a, Timestamp &b) {
   swap(a.secondsSinceEpoch, b.secondsSinceEpoch);
 }
 
-Timestamp::Timestamp(const Timestamp& other331) {
-  secondsSinceEpoch = other331.secondsSinceEpoch;
+Timestamp::Timestamp(const Timestamp& other328) {
+  secondsSinceEpoch = other328.secondsSinceEpoch;
 }
-Timestamp& Timestamp::operator=(const Timestamp& other332) {
-  secondsSinceEpoch = other332.secondsSinceEpoch;
+Timestamp& Timestamp::operator=(const Timestamp& other329) {
+  secondsSinceEpoch = other329.secondsSinceEpoch;
   return *this;
 }
 void Timestamp::printTo(std::ostream& out) const {
@@ -10221,21 +10059,21 @@ void swap(TimestampColumnStatsData &a, TimestampColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-TimestampColumnStatsData::TimestampColumnStatsData(const TimestampColumnStatsData& other333) {
-  lowValue = other333.lowValue;
-  highValue = other333.highValue;
-  numNulls = other333.numNulls;
-  numDVs = other333.numDVs;
-  bitVectors = other333.bitVectors;
-  __isset = other333.__isset;
+TimestampColumnStatsData::TimestampColumnStatsData(const TimestampColumnStatsData& other330) {
+  lowValue = other330.lowValue;
+  highValue = other330.highValue;
+  numNulls = other330.numNulls;
+  numDVs = other330.numDVs;
+  bitVectors = other330.bitVectors;
+  __isset = other330.__isset;
 }
-TimestampColumnStatsData& TimestampColumnStatsData::operator=(const TimestampColumnStatsData& other334) {
-  lowValue = other334.lowValue;
-  highValue = other334.highValue;
-  numNulls = other334.numNulls;
-  numDVs = other334.numDVs;
-  bitVectors = other334.bitVectors;
-  __isset = other334.__isset;
+TimestampColumnStatsData& TimestampColumnStatsData::operator=(const TimestampColumnStatsData& other331) {
+  lowValue = other331.lowValue;
+  highValue = other331.highValue;
+  numNulls = other331.numNulls;
+  numDVs = other331.numDVs;
+  bitVectors = other331.bitVectors;
+  __isset = other331.__isset;
   return *this;
 }
 void TimestampColumnStatsData::printTo(std::ostream& out) const {
@@ -10460,27 +10298,27 @@ void swap(ColumnStatisticsData &a, ColumnStatisticsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-ColumnStatisticsData::ColumnStatisticsData(const ColumnStatisticsData& other335) {
-  booleanStats = other335.booleanStats;
-  longStats = other335.longStats;
-  doubleStats = other335.doubleStats;
-  stringStats = other335.stringStats;
-  binaryStats = other335.binaryStats;
-  decimalStats = other335.decimalStats;
-  dateStats = other335.dateStats;
-  timestampStats = other335.timestampStats;
-  __isset = other335.__isset;
-}
-ColumnStatisticsData& ColumnStatisticsData::operator=(const ColumnStatisticsData& other336) {
-  booleanStats = other336.booleanStats;
-  longStats = other336.longStats;
-  doubleStats = other336.doubleStats;
-  stringStats = other336.stringStats;
-  binaryStats = other336.binaryStats;
-  decimalStats = other336.decimalStats;
-  dateStats = other336.dateStats;
-  timestampStats = other336.timestampStats;
-  __isset = other336.__isset;
+ColumnStatisticsData::ColumnStatisticsData(const ColumnStatisticsData& other332) {
+  booleanStats = other332.booleanStats;
+  longStats = other332.longStats;
+  doubleStats = other332.doubleStats;
+  stringStats = other332.stringStats;
+  binaryStats = other332.binaryStats;
+  decimalStats = other332.decimalStats;
+  dateStats = other332.dateStats;
+  timestampStats = other332.timestampStats;
+  __isset = other332.__isset;
+}
+ColumnStatisticsData& ColumnStatisticsData::operator=(const ColumnStatisticsData& other333) {
+  booleanStats = other333.booleanStats;
+  longStats = other333.longStats;
+  doubleStats = other333.doubleStats;
+  stringStats = other333.stringStats;
+  binaryStats = other333.binaryStats;
+  decimalStats = other333.decimalStats;
+  dateStats = other333.dateStats;
+  timestampStats = other333.timestampStats;
+  __isset = other333.__isset;
   return *this;
 }
 void ColumnStatisticsData::printTo(std::ostream& out) const {
@@ -10615,15 +10453,15 @@ void swap(ColumnStatisticsObj &a, ColumnStatisticsObj &b) {
   swap(a.statsData, b.statsData);
 }
 
-ColumnStatisticsObj::ColumnStatisticsObj(const ColumnStatisticsObj& other337) {
-  colName = other337.colName;
-  colType = other337.colType;
-  statsData = other337.statsData;
+ColumnStatisticsObj::ColumnStatisticsObj(const ColumnStatisticsObj& other334) {
+  colName = other334.colName;
+  colType = other334.colType;
+  statsData = other334.statsData;
 }
-ColumnStatisticsObj& ColumnStatisticsObj::operator=(const ColumnStatisticsObj& other338) {
-  colName = other338.colName;
-  colType = other338.colType;
-  statsData = other338.statsData;
+ColumnStatisticsObj& ColumnStatisticsObj::operator=(const ColumnStatisticsObj& other335) {
+  colName = other335.colName;
+  colType = other335.colType;
+  statsData = other335.statsData;
   return *this;
 }
 void ColumnStatisticsObj::printTo(std::ostream& out) const {
@@ -10811,23 +10649,23 @@ void swap(ColumnStatisticsDesc &a, ColumnStatisticsDesc &b) {
   swap(a.__isset, b.__isset);
 }
 
-ColumnStatisticsDesc::ColumnStatisticsDesc(const ColumnStatisticsDesc& other339) {
-  isTblLevel = other339.isTblLevel;
-  dbName = other339.dbName;
-  tableName = other339.tableName;
-  partName = other339.partName;
-  lastAnalyzed = other339.lastAnalyzed;
-  catName = other339.catName;
-  __isset = other339.__isset;
+ColumnStatisticsDesc::ColumnStatisticsDesc(const ColumnStatisticsDesc& other336) {
+  isTblLevel = other336.isTblLevel;
+  dbName = other336.dbName;
+  tableName = other336.tableName;
+  partName = other336.partName;
+  lastAnalyzed = other336.lastAnalyzed;
+  catName = other336.catName;
+  __isset = other336.__isset;
 }
-ColumnStatisticsDesc& ColumnStatisticsDesc::operator=(const ColumnStatisticsDesc& other340) {
-  isTblLevel = other340.isTblLevel;
-  dbName = other340.dbName;
-  tableName = other340.tableName;
-  partName = other340.partName;
-  lastAnalyzed = other340.lastAnalyzed;
-  catName = other340.catName;
-  __isset = other340.__isset;
+ColumnStatisticsDesc& ColumnStatisticsDesc::operator=(const ColumnStatisticsDesc& other337) {
+  isTblLevel = other337.isTblLevel;
+  dbName = other337.dbName;
+  tableName = other337.tableName;
+  partName = other337.partName;
+  lastAnalyzed = other337.lastAnalyzed;
+  catName = other337.catName;
+  __isset = other337.__isset;
   return *this;
 }
 void ColumnStatisticsDesc::printTo(std::ostream& out) const {
@@ -10906,14 +10744,14 @@ uint32_t ColumnStatistics::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->statsObj.clear();
-            uint32_t _size341;
-            ::apache::thrift::protocol::TType _etype344;
-            xfer += iprot->readListBegin(_etype344, _size341);
-            this->statsObj.resize(_size341);
-            uint32_t _i345;
-            for (_i345 = 0; _i345 < _size341; ++_i345)
+            uint32_t _size338;
+            ::apache::thrift::protocol::TType _etype341;
+            xfer += iprot->readListBegin(_etype341, _size338);
+            this->statsObj.resize(_size338);
+            uint32_t _i342;
+            for (_i342 = 0; _i342 < _size338; ++_i342)
             {
-              xfer += this->statsObj[_i345].read(iprot);
+              xfer += this->statsObj[_i342].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -10966,10 +10804,10 @@ uint32_t ColumnStatistics::write(::apache::thrift::protocol::TProtocol* oprot) c
   xfer += oprot->writeFieldBegin("statsObj", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->statsObj.size()));
-    std::vector<ColumnStatisticsObj> ::const_iterator _iter346;
-    for (_iter346 = this->statsObj.begin(); _iter346 != this->statsObj.end(); ++_iter346)
+    std::vector<ColumnStatisticsObj> ::const_iterator _iter343;
+    for (_iter343 = this->statsObj.begin(); _iter343 != this->statsObj.end(); ++_iter343)
     {
-      xfer += (*_iter346).write(oprot);
+      xfer += (*_iter343).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -10999,19 +10837,19 @@ void swap(ColumnStatistics &a, ColumnStatistics &b) {
   swap(a.__isset, b.__isset);
 }
 
-ColumnStatistics::ColumnStatistics(const ColumnStatistics& other347) {
-  statsDesc = other347.statsDesc;
-  statsObj = other347.statsObj;
-  isStatsCompliant = other347.isStatsCompliant;
-  engine = other347.engine;
-  __isset = other347.__isset;
+ColumnStatistics::ColumnStatistics(const ColumnStatistics& other344) {
+  statsDesc = other344.statsDesc;
+  statsObj = other344.statsObj;
+  isStatsCompliant = other344.isStatsCompliant;
+  engine = other344.engine;
+  __isset = other344.__isset;
 }
-ColumnStatistics& ColumnStatistics::operator=(const ColumnStatistics& other348) {
-  statsDesc = other348.statsDesc;
-  statsObj = other348.statsObj;
-  isStatsCompliant = other348.isStatsCompliant;
-  engine = other348.engine;
-  __isset = other348.__isset;
+ColumnStatistics& ColumnStatistics::operator=(const ColumnStatistics& other345) {
+  statsDesc = other345.statsDesc;
+  statsObj = other345.statsObj;
+  isStatsCompliant = other345.isStatsCompliant;
+  engine = other345.engine;
+  __isset = other345.__isset;
   return *this;
 }
 void ColumnStatistics::printTo(std::ostream& out) const {
@@ -11088,14 +10926,14 @@ uint32_t FileMetadata::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->data.clear();
-            uint32_t _size349;
-            ::apache::thrift::protocol::TType _etype352;
-            xfer += iprot->readListBegin(_etype352, _size349);
-            this->data.resize(_size349);
-            uint32_t _i353;
-            for (_i353 = 0; _i353 < _size349; ++_i353)
+            uint32_t _size346;
+            ::apache::thrift::protocol::TType _etype349;
+            xfer += iprot->readListBegin(_etype349, _size346);
+            this->data.resize(_size346);
+            uint32_t _i350;
+            for (_i350 = 0; _i350 < _size346; ++_i350)
             {
-              xfer += iprot->readBinary(this->data[_i353]);
+              xfer += iprot->readBinary(this->data[_i350]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11132,10 +10970,10 @@ uint32_t FileMetadata::write(::apache::thrift::protocol::TProtocol* oprot) const
   xfer += oprot->writeFieldBegin("data", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->data.size()));
-    std::vector<std::string> ::const_iterator _iter354;
-    for (_iter354 = this->data.begin(); _iter354 != this->data.end(); ++_iter354)
+    std::vector<std::string> ::const_iterator _iter351;
+    for (_iter351 = this->data.begin(); _iter351 != this->data.end(); ++_iter351)
     {
-      xfer += oprot->writeBinary((*_iter354));
+      xfer += oprot->writeBinary((*_iter351));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11154,17 +10992,17 @@ void swap(FileMetadata &a, FileMetadata &b) {
   swap(a.__isset, b.__isset);
 }
 
-FileMetadata::FileMetadata(const FileMetadata& other355) {
-  type = other355.type;
-  version = other355.version;
-  data = other355.data;
-  __isset = other355.__isset;
+FileMetadata::FileMetadata(const FileMetadata& other352) {
+  type = other352.type;
+  version = other352.version;
+  data = other352.data;
+  __isset = other352.__isset;
 }
-FileMetadata& FileMetadata::operator=(const FileMetadata& other356) {
-  type = other356.type;
-  version = other356.version;
-  data = other356.data;
-  __isset = other356.__isset;
+FileMetadata& FileMetadata::operator=(const FileMetadata& other353) {
+  type = other353.type;
+  version = other353.version;
+  data = other353.data;
+  __isset = other353.__isset;
   return *this;
 }
 void FileMetadata::printTo(std::ostream& out) const {
@@ -11217,26 +11055,26 @@ uint32_t ObjectDictionary::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->values.clear();
-            uint32_t _size357;
-            ::apache::thrift::protocol::TType _ktype358;
-            ::apache::thrift::protocol::TType _vtype359;
-            xfer += iprot->readMapBegin(_ktype358, _vtype359, _size357);
-            uint32_t _i361;
-            for (_i361 = 0; _i361 < _size357; ++_i361)
+            uint32_t _size354;
+            ::apache::thrift::protocol::TType _ktype355;
+            ::apache::thrift::protocol::TType _vtype356;
+            xfer += iprot->readMapBegin(_ktype355, _vtype356, _size354);
+            uint32_t _i358;
+            for (_i358 = 0; _i358 < _size354; ++_i358)
             {
-              std::string _key362;
-              xfer += iprot->readString(_key362);
-              std::vector<std::string> & _val363 = this->values[_key362];
+              std::string _key359;
+              xfer += iprot->readString(_key359);
+              std::vector<std::string> & _val360 = this->values[_key359];
               {
-                _val363.clear();
-                uint32_t _size364;
-                ::apache::thrift::protocol::TType _etype367;
-                xfer += iprot->readListBegin(_etype367, _size364);
-                _val363.resize(_size364);
-                uint32_t _i368;
-                for (_i368 = 0; _i368 < _size364; ++_i368)
+                _val360.clear();
+                uint32_t _size361;
+                ::apache::thrift::protocol::TType _etype364;
+                xfer += iprot->readListBegin(_etype364, _size361);
+                _val360.resize(_size361);
+                uint32_t _i365;
+                for (_i365 = 0; _i365 < _size361; ++_i365)
                 {
-                  xfer += iprot->readBinary(_val363[_i368]);
+                  xfer += iprot->readBinary(_val360[_i365]);
                 }
                 xfer += iprot->readListEnd();
               }
@@ -11270,16 +11108,16 @@ uint32_t ObjectDictionary::write(::apache::thrift::protocol::TProtocol* oprot) c
   xfer += oprot->writeFieldBegin("values", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_LIST, static_cast<uint32_t>(this->values.size()));
-    std::map<std::string, std::vector<std::string> > ::const_iterator _iter369;
-    for (_iter369 = this->values.begin(); _iter369 != this->values.end(); ++_iter369)
+    std::map<std::string, std::vector<std::string> > ::const_iterator _iter366;
+    for (_iter366 = this->values.begin(); _iter366 != this->values.end(); ++_iter366)
     {
-      xfer += oprot->writeString(_iter369->first);
+      xfer += oprot->writeString(_iter366->first);
       {
-        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(_iter369->second.size()));
-        std::vector<std::string> ::const_iterator _iter370;
-        for (_iter370 = _iter369->second.begin(); _iter370 != _iter369->second.end(); ++_iter370)
+        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(_iter366->second.size()));
+        std::vector<std::string> ::const_iterator _iter367;
+        for (_iter367 = _iter366->second.begin(); _iter367 != _iter366->second.end(); ++_iter367)
         {
-          xfer += oprot->writeBinary((*_iter370));
+          xfer += oprot->writeBinary((*_iter367));
         }
         xfer += oprot->writeListEnd();
       }
@@ -11298,11 +11136,11 @@ void swap(ObjectDictionary &a, ObjectDictionary &b) {
   swap(a.values, b.values);
 }
 
-ObjectDictionary::ObjectDictionary(const ObjectDictionary& other371) {
-  values = other371.values;
+ObjectDictionary::ObjectDictionary(const ObjectDictionary& other368) {
+  values = other368.values;
 }
-ObjectDictionary& ObjectDictionary::operator=(const ObjectDictionary& other372) {
-  values = other372.values;
+ObjectDictionary& ObjectDictionary::operator=(const ObjectDictionary& other369) {
+  values = other369.values;
   return *this;
 }
 void ObjectDictionary::printTo(std::ostream& out) const {
@@ -11532,14 +11370,14 @@ uint32_t Table::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitionKeys.clear();
-            uint32_t _size373;
-            ::apache::thrift::protocol::TType _etype376;
-            xfer += iprot->readListBegin(_etype376, _size373);
-            this->partitionKeys.resize(_size373);
-            uint32_t _i377;
-            for (_i377 = 0; _i377 < _size373; ++_i377)
+            uint32_t _size370;
+            ::apache::thrift::protocol::TType _etype373;
+            xfer += iprot->readListBegin(_etype373, _size370);
+            this->partitionKeys.resize(_size370);
+            uint32_t _i374;
+            for (_i374 = 0; _i374 < _size370; ++_i374)
             {
-              xfer += this->partitionKeys[_i377].read(iprot);
+              xfer += this->partitionKeys[_i374].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -11552,17 +11390,17 @@ uint32_t Table::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->parameters.clear();
-            uint32_t _size378;
-            ::apache::thrift::protocol::TType _ktype379;
-            ::apache::thrift::protocol::TType _vtype380;
-            xfer += iprot->readMapBegin(_ktype379, _vtype380, _size378);
-            uint32_t _i382;
-            for (_i382 = 0; _i382 < _size378; ++_i382)
+            uint32_t _size375;
+            ::apache::thrift::protocol::TType _ktype376;
+            ::apache::thrift::protocol::TType _vtype377;
+            xfer += iprot->readMapBegin(_ktype376, _vtype377, _size375);
+            uint32_t _i379;
+            for (_i379 = 0; _i379 < _size375; ++_i379)
             {
-              std::string _key383;
-              xfer += iprot->readString(_key383);
-              std::string& _val384 = this->parameters[_key383];
-              xfer += iprot->readString(_val384);
+              std::string _key380;
+              xfer += iprot->readString(_key380);
+              std::string& _val381 = this->parameters[_key380];
+              xfer += iprot->readString(_val381);
             }
             xfer += iprot->readMapEnd();
           }
@@ -11637,9 +11475,9 @@ uint32_t Table::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 18:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast385;
-          xfer += iprot->readI32(ecast385);
-          this->ownerType = (PrincipalType::type)ecast385;
+          int32_t ecast382;
+          xfer += iprot->readI32(ecast382);
+          this->ownerType = (PrincipalType::type)ecast382;
           this->__isset.ownerType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -11681,14 +11519,14 @@ uint32_t Table::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->requiredReadCapabilities.clear();
-            uint32_t _size386;
-            ::apache::thrift::protocol::TType _etype389;
-            xfer += iprot->readListBegin(_etype389, _size386);
-            this->requiredReadCapabilities.resize(_size386);
-            uint32_t _i390;
-            for (_i390 = 0; _i390 < _size386; ++_i390)
+            uint32_t _size383;
+            ::apache::thrift::protocol::TType _etype386;
+            xfer += iprot->readListBegin(_etype386, _size383);
+            this->requiredReadCapabilities.resize(_size383);
+            uint32_t _i387;
+            for (_i387 = 0; _i387 < _size383; ++_i387)
             {
-              xfer += iprot->readString(this->requiredReadCapabilities[_i390]);
+              xfer += iprot->readString(this->requiredReadCapabilities[_i387]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11701,14 +11539,14 @@ uint32_t Table::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->requiredWriteCapabilities.clear();
-            uint32_t _size391;
-            ::apache::thrift::protocol::TType _etype394;
-            xfer += iprot->readListBegin(_etype394, _size391);
-            this->requiredWriteCapabilities.resize(_size391);
-            uint32_t _i395;
-            for (_i395 = 0; _i395 < _size391; ++_i395)
+            uint32_t _size388;
+            ::apache::thrift::protocol::TType _etype391;
+            xfer += iprot->readListBegin(_etype391, _size388);
+            this->requiredWriteCapabilities.resize(_size388);
+            uint32_t _i392;
+            for (_i392 = 0; _i392 < _size388; ++_i392)
             {
-              xfer += iprot->readString(this->requiredWriteCapabilities[_i395]);
+              xfer += iprot->readString(this->requiredWriteCapabilities[_i392]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11797,10 +11635,10 @@ uint32_t Table::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("partitionKeys", ::apache::thrift::protocol::T_LIST, 8);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->partitionKeys.size()));
-    std::vector<FieldSchema> ::const_iterator _iter396;
-    for (_iter396 = this->partitionKeys.begin(); _iter396 != this->partitionKeys.end(); ++_iter396)
+    std::vector<FieldSchema> ::const_iterator _iter393;
+    for (_iter393 = this->partitionKeys.begin(); _iter393 != this->partitionKeys.end(); ++_iter393)
     {
-      xfer += (*_iter396).write(oprot);
+      xfer += (*_iter393).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -11809,11 +11647,11 @@ uint32_t Table::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 9);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->parameters.size()));
-    std::map<std::string, std::string> ::const_iterator _iter397;
-    for (_iter397 = this->parameters.begin(); _iter397 != this->parameters.end(); ++_iter397)
+    std::map<std::string, std::string> ::const_iterator _iter394;
+    for (_iter394 = this->parameters.begin(); _iter394 != this->parameters.end(); ++_iter394)
     {
-      xfer += oprot->writeString(_iter397->first);
-      xfer += oprot->writeString(_iter397->second);
+      xfer += oprot->writeString(_iter394->first);
+      xfer += oprot->writeString(_iter394->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -11885,10 +11723,10 @@ uint32_t Table::write(::apache::thrift::protocol::TProtocol* oprot) const {
     xfer += oprot->writeFieldBegin("requiredReadCapabilities", ::apache::thrift::protocol::T_LIST, 23);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->requiredReadCapabilities.size()));
-      std::vector<std::string> ::const_iterator _iter398;
-      for (_iter398 = this->requiredReadCapabilities.begin(); _iter398 != this->requiredReadCapabilities.end(); ++_iter398)
+      std::vector<std::string> ::const_iterator _iter395;
+      for (_iter395 = this->requiredReadCapabilities.begin(); _iter395 != this->requiredReadCapabilities.end(); ++_iter395)
       {
-        xfer += oprot->writeString((*_iter398));
+        xfer += oprot->writeString((*_iter395));
       }
       xfer += oprot->writeListEnd();
     }
@@ -11898,10 +11736,10 @@ uint32_t Table::write(::apache::thrift::protocol::TProtocol* oprot) const {
     xfer += oprot->writeFieldBegin("requiredWriteCapabilities", ::apache::thrift::protocol::T_LIST, 24);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->requiredWriteCapabilities.size()));
-      std::vector<std::string> ::const_iterator _iter399;
-      for (_iter399 = this->requiredWriteCapabilities.begin(); _iter399 != this->requiredWriteCapabilities.end(); ++_iter399)
+      std::vector<std::string> ::const_iterator _iter396;
+      for (_iter396 = this->requiredWriteCapabilities.begin(); _iter396 != this->requiredWriteCapabilities.end(); ++_iter396)
       {
-        xfer += oprot->writeString((*_iter399));
+        xfer += oprot->writeString((*_iter396));
       }
       xfer += oprot->writeListEnd();
     }
@@ -11965,67 +11803,67 @@ void swap(Table &a, Table &b) {
   swap(a.__isset, b.__isset);
 }
 
-Table::Table(const Table& other400) {
-  tableName = other400.tableName;
-  dbName = other400.dbName;
-  owner = other400.owner;
-  createTime = other400.createTime;
-  lastAccessTime = other400.lastAccessTime;
-  retention = other400.retention;
-  sd = other400.sd;
-  partitionKeys = other400.partitionKeys;
-  parameters = other400.parameters;
-  viewOriginalText = other400.viewOriginalText;
-  viewExpandedText = other400.viewExpandedText;
-  tableType = other400.tableType;
-  privileges = other400.privileges;
-  temporary = other400.temporary;
-  rewriteEnabled = other400.rewriteEnabled;
-  creationMetadata = other400.creationMetadata;
-  catName = other400.catName;
-  ownerType = other400.ownerType;
-  writeId = other400.writeId;
-  isStatsCompliant = other400.isStatsCompliant;
-  colStats = other400.colStats;
-  accessType = other400.accessType;
-  requiredReadCapabilities = other400.requiredReadCapabilities;
-  requiredWriteCapabilities = other400.requiredWriteCapabilities;
-  id = other400.id;
-  fileMetadata = other400.fileMetadata;
-  dictionary = other400.dictionary;
-  txnId = other400.txnId;
-  __isset = other400.__isset;
-}
-Table& Table::operator=(const Table& other401) {
-  tableName = other401.tableName;
-  dbName = other401.dbName;
-  owner = other401.owner;
-  createTime = other401.createTime;
-  lastAccessTime = other401.lastAccessTime;
-  retention = other401.retention;
-  sd = other401.sd;
-  partitionKeys = other401.partitionKeys;
-  parameters = other401.parameters;
-  viewOriginalText = other401.viewOriginalText;
-  viewExpandedText = other401.viewExpandedText;
-  tableType = other401.tableType;
-  privileges = other401.privileges;
-  temporary = other401.temporary;
-  rewriteEnabled = other401.rewriteEnabled;
-  creationMetadata = other401.creationMetadata;
-  catName = other401.catName;
-  ownerType = other401.ownerType;
-  writeId = other401.writeId;
-  isStatsCompliant = other401.isStatsCompliant;
-  colStats = other401.colStats;
-  accessType = other401.accessType;
-  requiredReadCapabilities = other401.requiredReadCapabilities;
-  requiredWriteCapabilities = other401.requiredWriteCapabilities;
-  id = other401.id;
-  fileMetadata = other401.fileMetadata;
-  dictionary = other401.dictionary;
-  txnId = other401.txnId;
-  __isset = other401.__isset;
+Table::Table(const Table& other397) {
+  tableName = other397.tableName;
+  dbName = other397.dbName;
+  owner = other397.owner;
+  createTime = other397.createTime;
+  lastAccessTime = other397.lastAccessTime;
+  retention = other397.retention;
+  sd = other397.sd;
+  partitionKeys = other397.partitionKeys;
+  parameters = other397.parameters;
+  viewOriginalText = other397.viewOriginalText;
+  viewExpandedText = other397.viewExpandedText;
+  tableType = other397.tableType;
+  privileges = other397.privileges;
+  temporary = other397.temporary;
+  rewriteEnabled = other397.rewriteEnabled;
+  creationMetadata = other397.creationMetadata;
+  catName = other397.catName;
+  ownerType = other397.ownerType;
+  writeId = other397.writeId;
+  isStatsCompliant = other397.isStatsCompliant;
+  colStats = other397.colStats;
+  accessType = other397.accessType;
+  requiredReadCapabilities = other397.requiredReadCapabilities;
+  requiredWriteCapabilities = other397.requiredWriteCapabilities;
+  id = other397.id;
+  fileMetadata = other397.fileMetadata;
+  dictionary = other397.dictionary;
+  txnId = other397.txnId;
+  __isset = other397.__isset;
+}
+Table& Table::operator=(const Table& other398) {
+  tableName = other398.tableName;
+  dbName = other398.dbName;
+  owner = other398.owner;
+  createTime = other398.createTime;
+  lastAccessTime = other398.lastAccessTime;
+  retention = other398.retention;
+  sd = other398.sd;
+  partitionKeys = other398.partitionKeys;
+  parameters = other398.parameters;
+  viewOriginalText = other398.viewOriginalText;
+  viewExpandedText = other398.viewExpandedText;
+  tableType = other398.tableType;
+  privileges = other398.privileges;
+  temporary = other398.temporary;
+  rewriteEnabled = other398.rewriteEnabled;
+  creationMetadata = other398.creationMetadata;
+  catName = other398.catName;
+  ownerType = other398.ownerType;
+  writeId = other398.writeId;
+  isStatsCompliant = other398.isStatsCompliant;
+  colStats = other398.colStats;
+  accessType = other398.accessType;
+  requiredReadCapabilities = other398.requiredReadCapabilities;
+  requiredWriteCapabilities = other398.requiredWriteCapabilities;
+  id = other398.id;
+  fileMetadata = other398.fileMetadata;
+  dictionary = other398.dictionary;
+  txnId = other398.txnId;
+  __isset = other398.__isset;
   return *this;
 }
 void Table::printTo(std::ostream& out) const {
@@ -12063,6 +11901,167 @@ void Table::printTo(std::ostream& out) const {
 }
 
 
+SourceTable::~SourceTable() noexcept {
+}
+
+
+void SourceTable::__set_table(const Table& val) {
+  this->table = val;
+}
+
+void SourceTable::__set_insertedCount(const int64_t val) {
+  this->insertedCount = val;
+}
+
+void SourceTable::__set_updatedCount(const int64_t val) {
+  this->updatedCount = val;
+}
+
+void SourceTable::__set_deletedCount(const int64_t val) {
+  this->deletedCount = val;
+}
+std::ostream& operator<<(std::ostream& out, const SourceTable& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+
+uint32_t SourceTable::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+  bool isset_table = false;
+  bool isset_insertedCount = false;
+  bool isset_updatedCount = false;
+  bool isset_deletedCount = false;
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->table.read(iprot);
+          isset_table = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_I64) {
+          xfer += iprot->readI64(this->insertedCount);
+          isset_insertedCount = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_I64) {
+          xfer += iprot->readI64(this->updatedCount);
+          isset_updatedCount = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 4:
+        if (ftype == ::apache::thrift::protocol::T_I64) {
+          xfer += iprot->readI64(this->deletedCount);
+          isset_deletedCount = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_table)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_insertedCount)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_updatedCount)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_deletedCount)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t SourceTable::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("SourceTable");
+
+  xfer += oprot->writeFieldBegin("table", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += this->table.write(oprot);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("insertedCount", ::apache::thrift::protocol::T_I64, 2);
+  xfer += oprot->writeI64(this->insertedCount);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("updatedCount", ::apache::thrift::protocol::T_I64, 3);
+  xfer += oprot->writeI64(this->updatedCount);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("deletedCount", ::apache::thrift::protocol::T_I64, 4);
+  xfer += oprot->writeI64(this->deletedCount);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(SourceTable &a, SourceTable &b) {
+  using ::std::swap;
+  swap(a.table, b.table);
+  swap(a.insertedCount, b.insertedCount);
+  swap(a.updatedCount, b.updatedCount);
+  swap(a.deletedCount, b.deletedCount);
+}
+
+SourceTable::SourceTable(const SourceTable& other399) {
+  table = other399.table;
+  insertedCount = other399.insertedCount;
+  updatedCount = other399.updatedCount;
+  deletedCount = other399.deletedCount;
+}
+SourceTable& SourceTable::operator=(const SourceTable& other400) {
+  table = other400.table;
+  insertedCount = other400.insertedCount;
+  updatedCount = other400.updatedCount;
+  deletedCount = other400.deletedCount;
+  return *this;
+}
+void SourceTable::printTo(std::ostream& out) const {
+  using ::apache::thrift::to_string;
+  out << "SourceTable(";
+  out << "table=" << to_string(table);
+  out << ", " << "insertedCount=" << to_string(insertedCount);
+  out << ", " << "updatedCount=" << to_string(updatedCount);
+  out << ", " << "deletedCount=" << to_string(deletedCount);
+  out << ")";
+}
+
+
 Partition::~Partition() noexcept {
 }
 
@@ -12156,14 +12155,14 @@ uint32_t Partition::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->values.clear();
-            uint32_t _size402;
-            ::apache::thrift::protocol::TType _etype405;
-            xfer += iprot->readListBegin(_etype405, _size402);
-            this->values.resize(_size402);
-            uint32_t _i406;
-            for (_i406 = 0; _i406 < _size402; ++_i406)
+            uint32_t _size401;
+            ::apache::thrift::protocol::TType _etype404;
+            xfer += iprot->readListBegin(_etype404, _size401);
+            this->values.resize(_size401);
+            uint32_t _i405;
+            for (_i405 = 0; _i405 < _size401; ++_i405)
             {
-              xfer += iprot->readString(this->values[_i406]);
+              xfer += iprot->readString(this->values[_i405]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12216,17 +12215,17 @@ uint32_t Partition::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->parameters.clear();
-            uint32_t _size407;
-            ::apache::thrift::protocol::TType _ktype408;
-            ::apache::thrift::protocol::TType _vtype409;
-            xfer += iprot->readMapBegin(_ktype408, _vtype409, _size407);
-            uint32_t _i411;
-            for (_i411 = 0; _i411 < _size407; ++_i411)
+            uint32_t _size406;
+            ::apache::thrift::protocol::TType _ktype407;
+            ::apache::thrift::protocol::TType _vtype408;
+            xfer += iprot->readMapBegin(_ktype407, _vtype408, _size406);
+            uint32_t _i410;
+            for (_i410 = 0; _i410 < _size406; ++_i410)
             {
-              std::string _key412;
-              xfer += iprot->readString(_key412);
-              std::string& _val413 = this->parameters[_key412];
-              xfer += iprot->readString(_val413);
+              std::string _key411;
+              xfer += iprot->readString(_key411);
+              std::string& _val412 = this->parameters[_key411];
+              xfer += iprot->readString(_val412);
             }
             xfer += iprot->readMapEnd();
           }
@@ -12303,10 +12302,10 @@ uint32_t Partition::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("values", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->values.size()));
-    std::vector<std::string> ::const_iterator _iter414;
-    for (_iter414 = this->values.begin(); _iter414 != this->values.end(); ++_iter414)
+    std::vector<std::string> ::const_iterator _iter413;
+    for (_iter413 = this->values.begin(); _iter413 != this->values.end(); ++_iter413)
     {
-      xfer += oprot->writeString((*_iter414));
+      xfer += oprot->writeString((*_iter413));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12335,11 +12334,11 @@ uint32_t Partition::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 7);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->parameters.size()));
-    std::map<std::string, std::string> ::const_iterator _iter415;
-    for (_iter415 = this->parameters.begin(); _iter415 != this->parameters.end(); ++_iter415)
+    std::map<std::string, std::string> ::const_iterator _iter414;
+    for (_iter414 = this->parameters.begin(); _iter414 != this->parameters.end(); ++_iter414)
     {
-      xfer += oprot->writeString(_iter415->first);
-      xfer += oprot->writeString(_iter415->second);
+      xfer += oprot->writeString(_iter414->first);
+      xfer += oprot->writeString(_iter414->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -12398,7 +12397,23 @@ void swap(Partition &a, Partition &b) {
   swap(a.__isset, b.__isset);
 }
 
-Partition::Partition(const Partition& other416) {
+Partition::Partition(const Partition& other415) {
+  values = other415.values;
+  dbName = other415.dbName;
+  tableName = other415.tableName;
+  createTime = other415.createTime;
+  lastAccessTime = other415.lastAccessTime;
+  sd = other415.sd;
+  parameters = other415.parameters;
+  privileges = other415.privileges;
+  catName = other415.catName;
+  writeId = other415.writeId;
+  isStatsCompliant = other415.isStatsCompliant;
+  colStats = other415.colStats;
+  fileMetadata = other415.fileMetadata;
+  __isset = other415.__isset;
+}
+Partition& Partition::operator=(const Partition& other416) {
   values = other416.values;
   dbName = other416.dbName;
   tableName = other416.tableName;
@@ -12413,22 +12428,6 @@ Partition::Partition(const Partition& other416) {
   colStats = other416.colStats;
   fileMetadata = other416.fileMetadata;
   __isset = other416.__isset;
-}
-Partition& Partition::operator=(const Partition& other417) {
-  values = other417.values;
-  dbName = other417.dbName;
-  tableName = other417.tableName;
-  createTime = other417.createTime;
-  lastAccessTime = other417.lastAccessTime;
-  sd = other417.sd;
-  parameters = other417.parameters;
-  privileges = other417.privileges;
-  catName = other417.catName;
-  writeId = other417.writeId;
-  isStatsCompliant = other417.isStatsCompliant;
-  colStats = other417.colStats;
-  fileMetadata = other417.fileMetadata;
-  __isset = other417.__isset;
   return *this;
 }
 void Partition::printTo(std::ostream& out) const {
@@ -12511,14 +12510,14 @@ uint32_t PartitionWithoutSD::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->values.clear();
-            uint32_t _size418;
-            ::apache::thrift::protocol::TType _etype421;
-            xfer += iprot->readListBegin(_etype421, _size418);
-            this->values.resize(_size418);
-            uint32_t _i422;
-            for (_i422 = 0; _i422 < _size418; ++_i422)
+            uint32_t _size417;
+            ::apache::thrift::protocol::TType _etype420;
+            xfer += iprot->readListBegin(_etype420, _size417);
+            this->values.resize(_size417);
+            uint32_t _i421;
+            for (_i421 = 0; _i421 < _size417; ++_i421)
             {
-              xfer += iprot->readString(this->values[_i422]);
+              xfer += iprot->readString(this->values[_i421]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12555,17 +12554,17 @@ uint32_t PartitionWithoutSD::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->parameters.clear();
-            uint32_t _size423;
-            ::apache::thrift::protocol::TType _ktype424;
-            ::apache::thrift::protocol::TType _vtype425;
-            xfer += iprot->readMapBegin(_ktype424, _vtype425, _size423);
-            uint32_t _i427;
-            for (_i427 = 0; _i427 < _size423; ++_i427)
+            uint32_t _size422;
+            ::apache::thrift::protocol::TType _ktype423;
+            ::apache::thrift::protocol::TType _vtype424;
+            xfer += iprot->readMapBegin(_ktype423, _vtype424, _size422);
+            uint32_t _i426;
+            for (_i426 = 0; _i426 < _size422; ++_i426)
             {
-              std::string _key428;
-              xfer += iprot->readString(_key428);
-              std::string& _val429 = this->parameters[_key428];
-              xfer += iprot->readString(_val429);
+              std::string _key427;
+              xfer += iprot->readString(_key427);
+              std::string& _val428 = this->parameters[_key427];
+              xfer += iprot->readString(_val428);
             }
             xfer += iprot->readMapEnd();
           }
@@ -12602,10 +12601,10 @@ uint32_t PartitionWithoutSD::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("values", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->values.size()));
-    std::vector<std::string> ::const_iterator _iter430;
-    for (_iter430 = this->values.begin(); _iter430 != this->values.end(); ++_iter430)
+    std::vector<std::string> ::const_iterator _iter429;
+    for (_iter429 = this->values.begin(); _iter429 != this->values.end(); ++_iter429)
     {
-      xfer += oprot->writeString((*_iter430));
+      xfer += oprot->writeString((*_iter429));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12626,11 +12625,11 @@ uint32_t PartitionWithoutSD::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 5);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->parameters.size()));
-    std::map<std::string, std::string> ::const_iterator _iter431;
-    for (_iter431 = this->parameters.begin(); _iter431 != this->parameters.end(); ++_iter431)
+    std::map<std::string, std::string> ::const_iterator _iter430;
+    for (_iter430 = this->parameters.begin(); _iter430 != this->parameters.end(); ++_iter430)
     {
-      xfer += oprot->writeString(_iter431->first);
-      xfer += oprot->writeString(_iter431->second);
+      xfer += oprot->writeString(_iter430->first);
+      xfer += oprot->writeString(_iter430->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -12657,7 +12656,16 @@ void swap(PartitionWithoutSD &a, PartitionWithoutSD &b) {
   swap(a.__isset, b.__isset);
 }
 
-PartitionWithoutSD::PartitionWithoutSD(const PartitionWithoutSD& other432) {
+PartitionWithoutSD::PartitionWithoutSD(const PartitionWithoutSD& other431) {
+  values = other431.values;
+  createTime = other431.createTime;
+  lastAccessTime = other431.lastAccessTime;
+  relativePath = other431.relativePath;
+  parameters = other431.parameters;
+  privileges = other431.privileges;
+  __isset = other431.__isset;
+}
+PartitionWithoutSD& PartitionWithoutSD::operator=(const PartitionWithoutSD& other432) {
   values = other432.values;
   createTime = other432.createTime;
   lastAccessTime = other432.lastAccessTime;
@@ -12665,15 +12673,6 @@ PartitionWithoutSD::PartitionWithoutSD(const PartitionWithoutSD& other432) {
   parameters = other432.parameters;
   privileges = other432.privileges;
   __isset = other432.__isset;
-}
-PartitionWithoutSD& PartitionWithoutSD::operator=(const PartitionWithoutSD& other433) {
-  values = other433.values;
-  createTime = other433.createTime;
-  lastAccessTime = other433.lastAccessTime;
-  relativePath = other433.relativePath;
-  parameters = other433.parameters;
-  privileges = other433.privileges;
-  __isset = other433.__isset;
   return *this;
 }
 void PartitionWithoutSD::printTo(std::ostream& out) const {
@@ -12732,14 +12731,14 @@ uint32_t PartitionSpecWithSharedSD::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitions.clear();
-            uint32_t _size434;
-            ::apache::thrift::protocol::TType _etype437;
-            xfer += iprot->readListBegin(_etype437, _size434);
-            this->partitions.resize(_size434);
-            uint32_t _i438;
-            for (_i438 = 0; _i438 < _size434; ++_i438)
+            uint32_t _size433;
+            ::apache::thrift::protocol::TType _etype436;
+            xfer += iprot->readListBegin(_etype436, _size433);
+            this->partitions.resize(_size433);
+            uint32_t _i437;
+            for (_i437 = 0; _i437 < _size433; ++_i437)
             {
-              xfer += this->partitions[_i438].read(iprot);
+              xfer += this->partitions[_i437].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12776,10 +12775,10 @@ uint32_t PartitionSpecWithSharedSD::write(::apache::thrift::protocol::TProtocol*
   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<PartitionWithoutSD> ::const_iterator _iter439;
-    for (_iter439 = this->partitions.begin(); _iter439 != this->partitions.end(); ++_iter439)
+    std::vector<PartitionWithoutSD> ::const_iterator _iter438;
+    for (_iter438 = this->partitions.begin(); _iter438 != this->partitions.end(); ++_iter438)
     {
-      xfer += (*_iter439).write(oprot);
+      xfer += (*_iter438).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -12801,15 +12800,15 @@ void swap(PartitionSpecWithSharedSD &a, PartitionSpecWithSharedSD &b) {
   swap(a.__isset, b.__isset);
 }
 
-PartitionSpecWithSharedSD::PartitionSpecWithSharedSD(const PartitionSpecWithSharedSD& other440) {
+PartitionSpecWithSharedSD::PartitionSpecWithSharedSD(const PartitionSpecWithSharedSD& other439) {
+  partitions = other439.partitions;
+  sd = other439.sd;
+  __isset = other439.__isset;
+}
+PartitionSpecWithSharedSD& PartitionSpecWithSharedSD::operator=(const PartitionSpecWithSharedSD& other440) {
   partitions = other440.partitions;
   sd = other440.sd;
   __isset = other440.__isset;
-}
-PartitionSpecWithSharedSD& PartitionSpecWithSharedSD::operator=(const PartitionSpecWithSharedSD& other441) {
-  partitions = other441.partitions;
-  sd = other441.sd;
-  __isset = other441.__isset;
   return *this;
 }
 void PartitionSpecWithSharedSD::printTo(std::ostream& out) const {
@@ -12860,14 +12859,14 @@ uint32_t PartitionListComposingSpec::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitions.clear();
-            uint32_t _size442;
-            ::apache::thrift::protocol::TType _etype445;
-            xfer += iprot->readListBegin(_etype445, _size442);
-            this->partitions.resize(_size442);
-            uint32_t _i446;
-            for (_i446 = 0; _i446 < _size442; ++_i446)
+            uint32_t _size441;
+            ::apache::thrift::protocol::TType _etype444;
+            xfer += iprot->readListBegin(_etype444, _size441);
+            this->partitions.resize(_size441);
+            uint32_t _i445;
+            for (_i445 = 0; _i445 < _size441; ++_i445)
             {
-              xfer += this->partitions[_i446].read(iprot);
+              xfer += this->partitions[_i445].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12896,10 +12895,10 @@ uint32_t PartitionListComposingSpec::write(::apache::thrift::protocol::TProtocol
   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 _iter447;
-    for (_iter447 = this->partitions.begin(); _iter447 != this->partitions.end(); ++_iter447)
+    std::vector<Partition> ::const_iterator _iter446;
+    for (_iter446 = this->partitions.begin(); _iter446 != this->partitions.end(); ++_iter446)
     {
-      xfer += (*_iter447).write(oprot);
+      xfer += (*_iter446).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -12916,13 +12915,13 @@ void swap(PartitionListComposingSpec &a, PartitionListComposingSpec &b) {
   swap(a.__isset, b.__isset);
 }
 
-PartitionListComposingSpec::PartitionListComposingSpec(const PartitionListComposingSpec& other448) {
+PartitionListComposingSpec::PartitionListComposingSpec(const PartitionListComposingSpec& other447) {
+  partitions = other447.partitions;
+  __isset = other447.__isset;
+}
+PartitionListComposingSpec& PartitionListComposingSpec::operator=(const PartitionListComposingSpec& other448) {
   partitions = other448.partitions;
   __isset = other448.__isset;
-}
-PartitionListComposingSpec& PartitionListComposingSpec::operator=(const PartitionListComposingSpec& other449) {
-  partitions = other449.partitions;
-  __isset = other449.__isset;
   return *this;
 }
 void PartitionListComposingSpec::printTo(std::ostream& out) const {
@@ -13137,7 +13136,18 @@ void swap(PartitionSpec &a, PartitionSpec &b) {
   swap(a.__isset, b.__isset);
 }
 
-PartitionSpec::PartitionSpec(const PartitionSpec& other450) {
+PartitionSpec::PartitionSpec(const PartitionSpec& other449) {
+  dbName = other449.dbName;
+  tableName = other449.tableName;
+  rootPath = other449.rootPath;
+  sharedSDPartitionSpec = other449.sharedSDPartitionSpec;
+  partitionList = other449.partitionList;
+  catName = other449.catName;
+  writeId = other449.writeId;
+  isStatsCompliant = other449.isStatsCompliant;
+  __isset = other449.__isset;
+}
+PartitionSpec& PartitionSpec::operator=(const PartitionSpec& other450) {
   dbName = other450.dbName;
   tableName = other450.tableName;
   rootPath = other450.rootPath;
@@ -13147,17 +13157,6 @@ PartitionSpec::PartitionSpec(const PartitionSpec& other450) {
   writeId = other450.writeId;
   isStatsCompliant = other450.isStatsCompliant;
   __isset = other450.__isset;
-}
-PartitionSpec& PartitionSpec::operator=(const PartitionSpec& other451) {
-  dbName = other451.dbName;
-  tableName = other451.tableName;
-  rootPath = other451.rootPath;
-  sharedSDPartitionSpec = other451.sharedSDPartitionSpec;
-  partitionList = other451.partitionList;
-  catName = other451.catName;
-  writeId = other451.writeId;
-  isStatsCompliant = other451.isStatsCompliant;
-  __isset = other451.__isset;
   return *this;
 }
 void PartitionSpec::printTo(std::ostream& out) const {
@@ -13225,14 +13224,14 @@ uint32_t AggrStats::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->colStats.clear();
-            uint32_t _size452;
-            ::apache::thrift::protocol::TType _etype455;
-            xfer += iprot->readListBegin(_etype455, _size452);
-            this->colStats.resize(_size452);
-            uint32_t _i456;
-            for (_i456 = 0; _i456 < _size452; ++_i456)
+            uint32_t _size451;
+            ::apache::thrift::protocol::TType _etype454;
+            xfer += iprot->readListBegin(_etype454, _size451);
+            this->colStats.resize(_size451);
+            uint32_t _i455;
+            for (_i455 = 0; _i455 < _size451; ++_i455)
             {
-              xfer += this->colStats[_i456].read(iprot);
+              xfer += this->colStats[_i455].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13281,10 +13280,10 @@ uint32_t AggrStats::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("colStats", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->colStats.size()));
-    std::vector<ColumnStatisticsObj> ::const_iterator _iter457;
-    for (_iter457 = this->colStats.begin(); _iter457 != this->colStats.end(); ++_iter457)
+    std::vector<ColumnStatisticsObj> ::const_iterator _iter456;
+    for (_iter456 = this->colStats.begin(); _iter456 != this->colStats.end(); ++_iter456)
     {
-      xfer += (*_iter457).write(oprot);
+      xfer += (*_iter456).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -13312,17 +13311,17 @@ void swap(AggrStats &a, AggrStats &b) {
   swap(a.__isset, b.__isset);
 }
 
-AggrStats::AggrStats(const AggrStats& other458) {
+AggrStats::AggrStats(const AggrStats& other457) {
+  colStats = other457.colStats;
+  partsFound = other457.partsFound;
+  isStatsCompliant = other457.isStatsCompliant;
+  __isset = other457.__isset;
+}
+AggrStats& AggrStats::operator=(const AggrStats& other458) {
   colStats = other458.colStats;
   partsFound = other458.partsFound;
   isStatsCompliant = other458.isStatsCompliant;
   __isset = other458.__isset;
-}
-AggrStats& AggrStats::operator=(const AggrStats& other459) {
-  colStats = other459.colStats;
-  partsFound = other459.partsFound;
-  isStatsCompliant = other459.isStatsCompliant;
-  __isset = other459.__isset;
   return *this;
 }
 void AggrStats::printTo(std::ostream& out) const {
@@ -13395,14 +13394,14 @@ uint32_t SetPartitionsStatsRequest::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->colStats.clear();
-            uint32_t _size460;
-            ::apache::thrift::protocol::TType _etype463;
-            xfer += iprot->readListBegin(_etype463, _size460);
-            this->colStats.resize(_size460);
-            uint32_t _i464;
-            for (_i464 = 0; _i464 < _size460; ++_i464)
+            uint32_t _size459;
+            ::apache::thrift::protocol::TType _etype462;
+            xfer += iprot->readListBegin(_etype462, _size459);
+            this->colStats.resize(_size459);
+            uint32_t _i463;
+            for (_i463 = 0; _i463 < _size459; ++_i463)
             {
-              xfer += this->colStats[_i464].read(iprot);
+              xfer += this->colStats[_i463].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13467,10 +13466,10 @@ uint32_t SetPartitionsStatsRequest::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("colStats", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->colStats.size()));
-    std::vector<ColumnStatistics> ::const_iterator _iter465;
-    for (_iter465 = this->colStats.begin(); _iter465 != this->colStats.end(); ++_iter465)
+    std::vector<ColumnStatistics> ::const_iterator _iter464;
+    for (_iter464 = this->colStats.begin(); _iter464 != this->colStats.end(); ++_iter464)
     {
-      xfer += (*_iter465).write(oprot);
+      xfer += (*_iter464).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -13510,21 +13509,21 @@ void swap(SetPartitionsStatsRequest &a, SetPartitionsStatsRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-SetPartitionsStatsRequest::SetPartitionsStatsRequest(const SetPartitionsStatsRequest& other466) {
+SetPartitionsStatsRequest::SetPartitionsStatsRequest(const SetPartitionsStatsRequest& other465) {
+  colStats = other465.colStats;
+  needMerge = other465.needMerge;
+  writeId = other465.writeId;
+  validWriteIdList = other465.validWriteIdList;
+  engine = other465.engine;
+  __isset = other465.__isset;
+}
+SetPartitionsStatsRequest& SetPartitionsStatsRequest::operator=(const SetPartitionsStatsRequest& other466) {
   colStats = other466.colStats;
   needMerge = other466.needMerge;
   writeId = other466.writeId;
   validWriteIdList = other466.validWriteIdList;
   engine = other466.engine;
   __isset = other466.__isset;
-}
-SetPartitionsStatsRequest& SetPartitionsStatsRequest::operator=(const SetPartitionsStatsRequest& other467) {
-  colStats = other467.colStats;
-  needMerge = other467.needMerge;
-  writeId = other467.writeId;
-  validWriteIdList = other467.validWriteIdList;
-  engine = other467.engine;
-  __isset = other467.__isset;
   return *this;
 }
 void SetPartitionsStatsRequest::printTo(std::ostream& out) const {
@@ -13616,11 +13615,11 @@ void swap(SetPartitionsStatsResponse &a, SetPartitionsStatsResponse &b) {
   swap(a.result, b.result);
 }
 
-SetPartitionsStatsResponse::SetPartitionsStatsResponse(const SetPartitionsStatsResponse& other468) {
-  result = other468.result;
+SetPartitionsStatsResponse::SetPartitionsStatsResponse(const SetPartitionsStatsResponse& other467) {
+  result = other467.result;
 }
-SetPartitionsStatsResponse& SetPartitionsStatsResponse::operator=(const SetPartitionsStatsResponse& other469) {
-  result = other469.result;
+SetPartitionsStatsResponse& SetPartitionsStatsResponse::operator=(const SetPartitionsStatsResponse& other468) {
+  result = other468.result;
   return *this;
 }
 void SetPartitionsStatsResponse::printTo(std::ostream& out) const {
@@ -13674,14 +13673,14 @@ uint32_t Schema::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->fieldSchemas.clear();
-            uint32_t _size470;
-            ::apache::thrift::protocol::TType _etype473;
-            xfer += iprot->readListBegin(_etype473, _size470);
-            this->fieldSchemas.resize(_size470);
-            uint32_t _i474;
-            for (_i474 = 0; _i474 < _size470; ++_i474)
+            uint32_t _size469;
+            ::apache::thrift::protocol::TType _etype472;
+            xfer += iprot->readListBegin(_etype472, _size469);
+            this->fieldSchemas.resize(_size469);
+            uint32_t _i473;
+            for (_i473 = 0; _i473 < _size469; ++_i473)
             {
-              xfer += this->fieldSchemas[_i474].read(iprot);
+              xfer += this->fieldSchemas[_i473].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13694,17 +13693,17 @@ uint32_t Schema::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->properties.clear();
-            uint32_t _size475;
-            ::apache::thrift::protocol::TType _ktype476;
-            ::apache::thrift::protocol::TType _vtype477;
-            xfer += iprot->readMapBegin(_ktype476, _vtype477, _size475);
-            uint32_t _i479;
-            for (_i479 = 0; _i479 < _size475; ++_i479)
+            uint32_t _size474;
+            ::apache::thrift::protocol::TType _ktype475;
+            ::apache::thrift::protocol::TType _vtype476;
+            xfer += iprot->readMapBegin(_ktype475, _vtype476, _size474);
+            uint32_t _i478;
+            for (_i478 = 0; _i478 < _size474; ++_i478)
             {
-              std::string _key480;
-              xfer += iprot->readString(_key480);
-              std::string& _val481 = this->properties[_key480];
-              xfer += iprot->readString(_val481);
+              std::string _key479;
+              xfer += iprot->readString(_key479);
+              std::string& _val480 = this->properties[_key479];
+              xfer += iprot->readString(_val480);
             }
             xfer += iprot->readMapEnd();
           }
@@ -13733,10 +13732,10 @@ uint32_t Schema::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("fieldSchemas", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->fieldSchemas.size()));
-    std::vector<FieldSchema> ::const_iterator _iter482;
-    for (_iter482 = this->fieldSchemas.begin(); _iter482 != this->fieldSchemas.end(); ++_iter482)
+    std::vector<FieldSchema> ::const_iterator _iter481;
+    for (_iter481 = this->fieldSchemas.begin(); _iter481 != this->fieldSchemas.end(); ++_iter481)
     {
-      xfer += (*_iter482).write(oprot);
+      xfer += (*_iter481).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -13745,11 +13744,11 @@ uint32_t Schema::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("properties", ::apache::thrift::protocol::T_MAP, 2);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->properties.size()));
-    std::map<std::string, std::string> ::const_iterator _iter483;
-    for (_iter483 = this->properties.begin(); _iter483 != this->properties.end(); ++_iter483)
+    std::map<std::string, std::string> ::const_iterator _iter482;
+    for (_iter482 = this->properties.begin(); _iter482 != this->properties.end(); ++_iter482)
     {
-      xfer += oprot->writeString(_iter483->first);
-      xfer += oprot->writeString(_iter483->second);
+      xfer += oprot->writeString(_iter482->first);
+      xfer += oprot->writeString(_iter482->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -13767,15 +13766,15 @@ void swap(Schema &a, Schema &b) {
   swap(a.__isset, b.__isset);
 }
 
-Schema::Schema(const Schema& other484) {
+Schema::Schema(const Schema& other483) {
+  fieldSchemas = other483.fieldSchemas;
+  properties = other483.properties;
+  __isset = other483.__isset;
+}
+Schema& Schema::operator=(const Schema& other484) {
   fieldSchemas = other484.fieldSchemas;
   properties = other484.properties;
   __isset = other484.__isset;
-}
-Schema& Schema::operator=(const Schema& other485) {
-  fieldSchemas = other485.fieldSchemas;
-  properties = other485.properties;
-  __isset = other485.__isset;
   return *this;
 }
 void Schema::printTo(std::ostream& out) const {
@@ -13942,21 +13941,21 @@ void swap(PrimaryKeysRequest &a, PrimaryKeysRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-PrimaryKeysRequest::PrimaryKeysRequest(const PrimaryKeysRequest& other486) {
+PrimaryKeysRequest::PrimaryKeysRequest(const PrimaryKeysRequest& other485) {
+  db_name = other485.db_name;
+  tbl_name = other485.tbl_name;
+  catName = other485.catName;
+  validWriteIdList = other485.validWriteIdList;
+  tableId = other485.tableId;
+  __isset = other485.__isset;
+}
+PrimaryKeysRequest& PrimaryKeysRequest::operator=(const PrimaryKeysRequest& other486) {
   db_name = other486.db_name;
   tbl_name = other486.tbl_name;
   catName = other486.catName;
   validWriteIdList = other486.validWriteIdList;
   tableId = other486.tableId;
   __isset = other486.__isset;
-}
-PrimaryKeysRequest& PrimaryKeysRequest::operator=(const PrimaryKeysRequest& other487) {
-  db_name = other487.db_name;
-  tbl_name = other487.tbl_name;
-  catName = other487.catName;
-  validWriteIdList = other487.validWriteIdList;
-  tableId = other487.tableId;
-  __isset = other487.__isset;
   return *this;
 }
 void PrimaryKeysRequest::printTo(std::ostream& out) const {
@@ -14011,14 +14010,14 @@ uint32_t PrimaryKeysResponse::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->primaryKeys.clear();
-            uint32_t _size488;
-            ::apache::thrift::protocol::TType _etype491;
-            xfer += iprot->readListBegin(_etype491, _size488);
-            this->primaryKeys.resize(_size488);
-            uint32_t _i492;
-            for (_i492 = 0; _i492 < _size488; ++_i492)
+            uint32_t _size487;
+            ::apache::thrift::protocol::TType _etype490;
+            xfer += iprot->readListBegin(_etype490, _size487);
+            this->primaryKeys.resize(_size487);
+            uint32_t _i491;
+            for (_i491 = 0; _i491 < _size487; ++_i491)
             {
-              xfer += this->primaryKeys[_i492].read(iprot);
+              xfer += this->primaryKeys[_i491].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14049,10 +14048,10 @@ uint32_t PrimaryKeysResponse::write(::apache::thrift::protocol::TProtocol* oprot
   xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->primaryKeys.size()));
-    std::vector<SQLPrimaryKey> ::const_iterator _iter493;
-    for (_iter493 = this->primaryKeys.begin(); _iter493 != this->primaryKeys.end(); ++_iter493)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter492;
+    for (_iter492 = this->primaryKeys.begin(); _iter492 != this->primaryKeys.end(); ++_iter492)
     {
-      xfer += (*_iter493).write(oprot);
+      xfer += (*_iter492).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -14068,11 +14067,11 @@ void swap(PrimaryKeysResponse &a, PrimaryKeysResponse &b) {
   swap(a.primaryKeys, b.primaryKeys);
 }
 
-PrimaryKeysResponse::PrimaryKeysResponse(const PrimaryKeysResponse& other494) {
-  primaryKeys = other494.primaryKeys;
+PrimaryKeysResponse::PrimaryKeysResponse(const PrimaryKeysResponse& other493) {
+  primaryKeys = other493.primaryKeys;
 }
-PrimaryKeysResponse& PrimaryKeysResponse::operator=(const PrimaryKeysResponse& other495) {
-  primaryKeys = other495.primaryKeys;
+PrimaryKeysResponse& PrimaryKeysResponse::operator=(const PrimaryKeysResponse& other494) {
+  primaryKeys = other494.primaryKeys;
   return *this;
 }
 void PrimaryKeysResponse::printTo(std::ostream& out) const {
@@ -14266,7 +14265,17 @@ void swap(ForeignKeysRequest &a, ForeignKeysRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-ForeignKeysRequest::ForeignKeysRequest(const ForeignKeysRequest& other496) {
+ForeignKeysRequest::ForeignKeysRequest(const ForeignKeysRequest& other495) {
+  parent_db_name = other495.parent_db_name;
+  parent_tbl_name = other495.parent_tbl_name;
+  foreign_db_name = other495.foreign_db_name;
+  foreign_tbl_name = other495.foreign_tbl_name;
+  catName = other495.catName;
+  validWriteIdList = other495.validWriteIdList;
+  tableId = other495.tableId;
+  __isset = other495.__isset;
+}
+ForeignKeysRequest& ForeignKeysRequest::operator=(const ForeignKeysRequest& other496) {
   parent_db_name = other496.parent_db_name;
   parent_tbl_name = other496.parent_tbl_name;
   foreign_db_name = other496.foreign_db_name;
@@ -14275,16 +14284,6 @@ ForeignKeysRequest::ForeignKeysRequest(const ForeignKeysRequest& other496) {
   validWriteIdList = other496.validWriteIdList;
   tableId = other496.tableId;
   __isset = other496.__isset;
-}
-ForeignKeysRequest& ForeignKeysRequest::operator=(const ForeignKeysRequest& other497) {
-  parent_db_name = other497.parent_db_name;
-  parent_tbl_name = other497.parent_tbl_name;
-  foreign_db_name = other497.foreign_db_name;
-  foreign_tbl_name = other497.foreign_tbl_name;
-  catName = other497.catName;
-  validWriteIdList = other497.validWriteIdList;
-  tableId = other497.tableId;
-  __isset = other497.__isset;
   return *this;
 }
 void ForeignKeysRequest::printTo(std::ostream& out) const {
@@ -14341,14 +14340,14 @@ uint32_t ForeignKeysResponse::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->foreignKeys.clear();
-            uint32_t _size498;
-            ::apache::thrift::protocol::TType _etype501;
-            xfer += iprot->readListBegin(_etype501, _size498);
-            this->foreignKeys.resize(_size498);
-            uint32_t _i502;
-            for (_i502 = 0; _i502 < _size498; ++_i502)
+            uint32_t _size497;
+            ::apache::thrift::protocol::TType _etype500;
+            xfer += iprot->readListBegin(_etype500, _size497);
+            this->foreignKeys.resize(_size497);
+            uint32_t _i501;
+            for (_i501 = 0; _i501 < _size497; ++_i501)
             {
-              xfer += this->foreignKeys[_i502].read(iprot);
+              xfer += this->foreignKeys[_i501].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14379,10 +14378,10 @@ uint32_t ForeignKeysResponse::write(::apache::thrift::protocol::TProtocol* oprot
   xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->foreignKeys.size()));
-    std::vector<SQLForeignKey> ::const_iterator _iter503;
-    for (_iter503 = this->foreignKeys.begin(); _iter503 != this->foreignKeys.end(); ++_iter503)
+    std::vector<SQLForeignKey> ::const_iterator _iter502;
+    for (_iter502 = this->foreignKeys.begin(); _iter502 != this->foreignKeys.end(); ++_iter502)
     {
-      xfer += (*_iter503).write(oprot);
+      xfer += (*_iter502).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -14398,11 +14397,11 @@ void swap(ForeignKeysResponse &a, ForeignKeysResponse &b) {
   swap(a.foreignKeys, b.foreignKeys);
 }
 
-ForeignKeysResponse::ForeignKeysResponse(const ForeignKeysResponse& other504) {
-  foreignKeys = other504.foreignKeys;
+ForeignKeysResponse::ForeignKeysResponse(const ForeignKeysResponse& other503) {
+  foreignKeys = other503.foreignKeys;
 }
-ForeignKeysResponse& ForeignKeysResponse::operator=(const ForeignKeysResponse& other505) {
-  foreignKeys = other505.foreignKeys;
+ForeignKeysResponse& ForeignKeysResponse::operator=(const ForeignKeysResponse& other504) {
+  foreignKeys = other504.foreignKeys;
   return *this;
 }
 void ForeignKeysResponse::printTo(std::ostream& out) const {
@@ -14569,21 +14568,21 @@ void swap(UniqueConstraintsRequest &a, UniqueConstraintsRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-UniqueConstraintsRequest::UniqueConstraintsRequest(const UniqueConstraintsRequest& other506) {
+UniqueConstraintsRequest::UniqueConstraintsRequest(const UniqueConstraintsRequest& other505) {
+  catName = other505.catName;
+  db_name = other505.db_name;
+  tbl_name = other505.tbl_name;
+  validWriteIdList = other505.validWriteIdList;
+  tableId = other505.tableId;
+  __isset = other505.__isset;
+}
+UniqueConstraintsRequest& UniqueConstraintsRequest::operator=(const UniqueConstraintsRequest& other506) {
   catName = other506.catName;
   db_name = other506.db_name;
   tbl_name = other506.tbl_name;
   validWriteIdList = other506.validWriteIdList;
   tableId = other506.tableId;
   __isset = other506.__isset;
-}
-UniqueConstraintsRequest& UniqueConstraintsRequest::operator=(const UniqueConstraintsRequest& other507) {
-  catName = other507.catName;
-  db_name = other507.db_name;
-  tbl_name = other507.tbl_name;
-  validWriteIdList = other507.validWriteIdList;
-  tableId = other507.tableId;
-  __isset = other507.__isset;
   return *this;
 }
 void UniqueConstraintsRequest::printTo(std::ostream& out) const {
@@ -14638,14 +14637,14 @@ uint32_t UniqueConstraintsResponse::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->uniqueConstraints.clear();
-            uint32_t _size508;
-            ::apache::thrift::protocol::TType _etype511;
-            xfer += iprot->readListBegin(_etype511, _size508);
-            this->uniqueConstraints.resize(_size508);
-            uint32_t _i512;
-            for (_i512 = 0; _i512 < _size508; ++_i512)
+            uint32_t _size507;
+            ::apache::thrift::protocol::TType _etype510;
+            xfer += iprot->readListBegin(_etype510, _size507);
+            this->uniqueConstraints.resize(_size507);
+            uint32_t _i511;
+            for (_i511 = 0; _i511 < _size507; ++_i511)
             {
-              xfer += this->uniqueConstraints[_i512].read(iprot);
+              xfer += this->uniqueConstraints[_i511].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14676,10 +14675,10 @@ uint32_t UniqueConstraintsResponse::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->uniqueConstraints.size()));
-    std::vector<SQLUniqueConstraint> ::const_iterator _iter513;
-    for (_iter513 = this->uniqueConstraints.begin(); _iter513 != this->uniqueConstraints.end(); ++_iter513)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter512;
+    for (_iter512 = this->uniqueConstraints.begin(); _iter512 != this->uniqueConstraints.end(); ++_iter512)
     {
-      xfer += (*_iter513).write(oprot);
+      xfer += (*_iter512).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -14695,11 +14694,11 @@ void swap(UniqueConstraintsResponse &a, UniqueConstraintsResponse &b) {
   swap(a.uniqueConstraints, b.uniqueConstraints);
 }
 
-UniqueConstraintsResponse::UniqueConstraintsResponse(const UniqueConstraintsResponse& other514) {
-  uniqueConstraints = other514.uniqueConstraints;
+UniqueConstraintsResponse::UniqueConstraintsResponse(const UniqueConstraintsResponse& other513) {
+  uniqueConstraints = other513.uniqueConstraints;
 }
-UniqueConstraintsResponse& UniqueConstraintsResponse::operator=(const UniqueConstraintsResponse& other515) {
-  uniqueConstraints = other515.uniqueConstraints;
+UniqueConstraintsResponse& UniqueConstraintsResponse::operator=(const UniqueConstraintsResponse& other514) {
+  uniqueConstraints = other514.uniqueConstraints;
   return *this;
 }
 void UniqueConstraintsResponse::printTo(std::ostream& out) const {
@@ -14866,21 +14865,21 @@ void swap(NotNullConstraintsRequest &a, NotNullConstraintsRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-NotNullConstraintsRequest::NotNullConstraintsRequest(const NotNullConstraintsRequest& other516) {
+NotNullConstraintsRequest::NotNullConstraintsRequest(const NotNullConstraintsRequest& other515) {
+  catName = other515.catName;
+  db_name = other515.db_name;
+  tbl_name = other515.tbl_name;
+  validWriteIdList = other515.validWriteIdList;
+  tableId = other515.tableId;
+  __isset = other515.__isset;
+}
+NotNullConstraintsRequest& NotNullConstraintsRequest::operator=(const NotNullConstraintsRequest& other516) {
   catName = other516.catName;
   db_name = other516.db_name;
   tbl_name = other516.tbl_name;
   validWriteIdList = other516.validWriteIdList;
   tableId = other516.tableId;
   __isset = other516.__isset;
-}
-NotNullConstraintsRequest& NotNullConstraintsRequest::operator=(const NotNullConstraintsRequest& other517) {
-  catName = other517.catName;
-  db_name = other517.db_name;
-  tbl_name = other517.tbl_name;
-  validWriteIdList = other517.validWriteIdList;
-  tableId = other517.tableId;
-  __isset = other517.__isset;
   return *this;
 }
 void NotNullConstraintsRequest::printTo(std::ostream& out) const {
@@ -14935,14 +14934,14 @@ uint32_t NotNullConstraintsResponse::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->notNullConstraints.clear();
-            uint32_t _size518;
-            ::apache::thrift::protocol::TType _etype521;
-            xfer += iprot->readListBegin(_etype521, _size518);
-            this->notNullConstraints.resize(_size518);
-            uint32_t _i522;
-            for (_i522 = 0; _i522 < _size518; ++_i522)
+            uint32_t _size517;
+            ::apache::thrift::protocol::TType _etype520;
+            xfer += iprot->readListBegin(_etype520, _size517);
+            this->notNullConstraints.resize(_size517);
+            uint32_t _i521;
+            for (_i521 = 0; _i521 < _size517; ++_i521)
             {
-              xfer += this->notNullConstraints[_i522].read(iprot);
+              xfer += this->notNullConstraints[_i521].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14973,10 +14972,10 @@ uint32_t NotNullConstraintsResponse::write(::apache::thrift::protocol::TProtocol
   xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->notNullConstraints.size()));
-    std::vector<SQLNotNullConstraint> ::const_iterator _iter523;
-    for (_iter523 = this->notNullConstraints.begin(); _iter523 != this->notNullConstraints.end(); ++_iter523)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter522;
+    for (_iter522 = this->notNullConstraints.begin(); _iter522 != this->notNullConstraints.end(); ++_iter522)
     {
-      xfer += (*_iter523).write(oprot);
+      xfer += (*_iter522).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -14992,11 +14991,11 @@ void swap(NotNullConstraintsResponse &a, NotNullConstraintsResponse &b) {
   swap(a.notNullConstraints, b.notNullConstraints);
 }
 
-NotNullConstraintsResponse::NotNullConstraintsResponse(const NotNullConstraintsResponse& other524) {
-  notNullConstraints = other524.notNullConstraints;
+NotNullConstraintsResponse::NotNullConstraintsResponse(const NotNullConstraintsResponse& other523) {
+  notNullConstraints = other523.notNullConstraints;
 }
-NotNullConstraintsResponse& NotNullConstraintsResponse::operator=(const NotNullConstraintsResponse& other525) {
-  notNullConstraints = other525.notNullConstraints;
+NotNullConstraintsResponse& NotNullConstraintsResponse::operator=(const NotNullConstraintsResponse& other524) {
+  notNullConstraints = other524.notNullConstraints;
   return *this;
 }
 void NotNullConstraintsResponse::printTo(std::ostream& out) const {
@@ -15163,21 +15162,21 @@ void swap(DefaultConstraintsRequest &a, DefaultConstraintsRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-DefaultConstraintsRequest::DefaultConstraintsRequest(const DefaultConstraintsRequest& other526) {
+DefaultConstraintsRequest::DefaultConstraintsRequest(const DefaultConstraintsRequest& other525) {
+  catName = other525.catName;
+  db_name = other525.db_name;
+  tbl_name = other525.tbl_name;
+  validWriteIdList = other525.validWriteIdList;
+  tableId = other525.tableId;
+  __isset = other525.__isset;
+}
+DefaultConstraintsRequest& DefaultConstraintsRequest::operator=(const DefaultConstraintsRequest& other526) {
   catName = other526.catName;
   db_name = other526.db_name;
   tbl_name = other526.tbl_name;
   validWriteIdList = other526.validWriteIdList;
   tableId = other526.tableId;
   __isset = other526.__isset;
-}
-DefaultConstraintsRequest& DefaultConstraintsRequest::operator=(const DefaultConstraintsRequest& other527) {
-  catName = other527.catName;
-  db_name = other527.db_name;
-  tbl_name = other527.tbl_name;
-  validWriteIdList = other527.validWriteIdList;
-  tableId = other527.tableId;
-  __isset = other527.__isset;
   return *this;
 }
 void DefaultConstraintsRequest::printTo(std::ostream& out) const {
@@ -15232,14 +15231,14 @@ uint32_t DefaultConstraintsResponse::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->defaultConstraints.clear();
-            uint32_t _size528;
-            ::apache::thrift::protocol::TType _etype531;
-            xfer += iprot->readListBegin(_etype531, _size528);
-            this->defaultConstraints.resize(_size528);
-            uint32_t _i532;
-            for (_i532 = 0; _i532 < _size528; ++_i532)
+            uint32_t _size527;
+            ::apache::thrift::protocol::TType _etype530;
+            xfer += iprot->readListBegin(_etype530, _size527);
+            this->defaultConstraints.resize(_size527);
+            uint32_t _i531;
+            for (_i531 = 0; _i531 < _size527; ++_i531)
             {
-              xfer += this->defaultConstraints[_i532].read(iprot);
+              xfer += this->defaultConstraints[_i531].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -15270,10 +15269,10 @@ uint32_t DefaultConstraintsResponse::write(::apache::thrift::protocol::TProtocol
   xfer += oprot->writeFieldBegin("defaultConstraints", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->defaultConstraints.size()));
-    std::vector<SQLDefaultConstraint> ::const_iterator _iter533;
-    for (_iter533 = this->defaultConstraints.begin(); _iter533 != this->defaultConstraints.end(); ++_iter533)
+    std::vector<SQLDefaultConstraint> ::const_iterator _iter532;
+    for (_iter532 = this->defaultConstraints.begin(); _iter532 != this->defaultConstraints.end(); ++_iter532)
     {
-      xfer += (*_iter533).write(oprot);
+      xfer += (*_iter532).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -15289,11 +15288,11 @@ void swap(DefaultConstraintsResponse &a, DefaultConstraintsResponse &b) {
   swap(a.defaultConstraints, b.defaultConstraints);
 }
 
-DefaultConstraintsResponse::DefaultConstraintsResponse(const DefaultConstraintsResponse& other534) {
-  defaultConstraints = other534.defaultConstraints;
+DefaultConstraintsResponse::DefaultConstraintsResponse(const DefaultConstraintsResponse& other533) {
+  defaultConstraints = other533.defaultConstraints;
 }
-DefaultConstraintsResponse& DefaultConstraintsResponse::operator=(const DefaultConstraintsResponse& other535) {
-  defaultConstraints = other535.defaultConstraints;
+DefaultConstraintsResponse& DefaultConstraintsResponse::operator=(const DefaultConstraintsResponse& other534) {
+  defaultConstraints = other534.defaultConstraints;
   return *this;
 }
 void DefaultConstraintsResponse::printTo(std::ostream& out) const {
@@ -15460,21 +15459,21 @@ void swap(CheckConstraintsRequest &a, CheckConstraintsRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-CheckConstraintsRequest::CheckConstraintsRequest(const CheckConstraintsRequest& other536) {
+CheckConstraintsRequest::CheckConstraintsRequest(const CheckConstraintsRequest& other535) {
+  catName = other535.catName;
+  db_name = other535.db_name;
+  tbl_name = other535.tbl_name;
+  validWriteIdList = other535.validWriteIdList;
+  tableId = other535.tableId;
+  __isset = other535.__isset;
+}
+CheckConstraintsRequest& CheckConstraintsRequest::operator=(const CheckConstraintsRequest& other536) {
   catName = other536.catName;
   db_name = other536.db_name;
   tbl_name = other536.tbl_name;
   validWriteIdList = other536.validWriteIdList;
   tableId = other536.tableId;
   __isset = other536.__isset;
-}
-CheckConstraintsRequest& CheckConstraintsRequest::operator=(const CheckConstraintsRequest& other537) {
-  catName = other537.catName;
-  db_name = other537.db_name;
-  tbl_name = other537.tbl_name;
-  validWriteIdList = other537.validWriteIdList;
-  tableId = other537.tableId;
-  __isset = other537.__isset;
   return *this;
 }
 void CheckConstraintsRequest::printTo(std::ostream& out) const {
@@ -15529,14 +15528,14 @@ uint32_t CheckConstraintsResponse::read(::apache::thrift::protocol::TProtocol* i
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->checkConstraints.clear();
-            uint32_t _size538;
-            ::apache::thrift::protocol::TType _etype541;
-            xfer += iprot->readListBegin(_etype541, _size538);
-            this->checkConstraints.resize(_size538);
-            uint32_t _i542;
-            for (_i542 = 0; _i542 < _size538; ++_i542)
+            uint32_t _size537;
+            ::apache::thrift::protocol::TType _etype540;
+            xfer += iprot->readListBegin(_etype540, _size537);
+            this->checkConstraints.resize(_size537);
+            uint32_t _i541;
+            for (_i541 = 0; _i541 < _size537; ++_i541)
             {
-              xfer += this->checkConstraints[_i542].read(iprot);
+              xfer += this->checkConstraints[_i541].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -15567,10 +15566,10 @@ uint32_t CheckConstraintsResponse::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("checkConstraints", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->checkConstraints.size()));
-    std::vector<SQLCheckConstraint> ::const_iterator _iter543;
-    for (_iter543 = this->checkConstraints.begin(); _iter543 != this->checkConstraints.end(); ++_iter543)
+    std::vector<SQLCheckConstraint> ::const_iterator _iter542;
+    for (_iter542 = this->checkConstraints.begin(); _iter542 != this->checkConstraints.end(); ++_iter542)
     {
-      xfer += (*_iter543).write(oprot);
+      xfer += (*_iter542).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -15586,11 +15585,11 @@ void swap(CheckConstraintsResponse &a, CheckConstraintsResponse &b) {
   swap(a.checkConstraints, b.checkConstraints);
 }
 
-CheckConstraintsResponse::CheckConstraintsResponse(const CheckConstraintsResponse& other544) {
-  checkConstraints = other544.checkConstraints;
+CheckConstraintsResponse::CheckConstraintsResponse(const CheckConstraintsResponse& other543) {
+  checkConstraints = other543.checkConstraints;
 }
-CheckConstraintsResponse& CheckConstraintsResponse::operator=(const CheckConstraintsResponse& other545) {
-  checkConstraints = other545.checkConstraints;
+CheckConstraintsResponse& CheckConstraintsResponse::operator=(const CheckConstraintsResponse& other544) {
+  checkConstraints = other544.checkConstraints;
   return *this;
 }
 void CheckConstraintsResponse::printTo(std::ostream& out) const {
@@ -15757,21 +15756,21 @@ void swap(AllTableConstraintsRequest &a, AllTableConstraintsRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-AllTableConstraintsRequest::AllTableConstraintsRequest(const AllTableConstraintsRequest& other546) {
+AllTableConstraintsRequest::AllTableConstraintsRequest(const AllTableConstraintsRequest& other545) {
+  dbName = other545.dbName;
+  tblName = other545.tblName;
+  catName = other545.catName;
+  validWriteIdList = other545.validWriteIdList;
+  tableId = other545.tableId;
+  __isset = other545.__isset;
+}
+AllTableConstraintsRequest& AllTableConstraintsRequest::operator=(const AllTableConstraintsRequest& other546) {
   dbName = other546.dbName;
   tblName = other546.tblName;
   catName = other546.catName;
   validWriteIdList = other546.validWriteIdList;
   tableId = other546.tableId;
   __isset = other546.__isset;
-}
-AllTableConstraintsRequest& AllTableConstraintsRequest::operator=(const AllTableConstraintsRequest& other547) {
-  dbName = other547.dbName;
-  tblName = other547.tblName;
-  catName = other547.catName;
-  validWriteIdList = other547.validWriteIdList;
-  tableId = other547.tableId;
-  __isset = other547.__isset;
   return *this;
 }
 void AllTableConstraintsRequest::printTo(std::ostream& out) const {
@@ -15863,11 +15862,11 @@ void swap(AllTableConstraintsResponse &a, AllTableConstraintsResponse &b) {
   swap(a.allTableConstraints, b.allTableConstraints);
 }
 
-AllTableConstraintsResponse::AllTableConstraintsResponse(const AllTableConstraintsResponse& other548) {
-  allTableConstraints = other548.allTableConstraints;
+AllTableConstraintsResponse::AllTableConstraintsResponse(const AllTableConstraintsResponse& other547) {
+  allTableConstraints = other547.allTableConstraints;
 }
-AllTableConstraintsResponse& AllTableConstraintsResponse::operator=(const AllTableConstraintsResponse& other549) {
-  allTableConstraints = other549.allTableConstraints;
+AllTableConstraintsResponse& AllTableConstraintsResponse::operator=(const AllTableConstraintsResponse& other548) {
+  allTableConstraints = other548.allTableConstraints;
   return *this;
 }
 void AllTableConstraintsResponse::printTo(std::ostream& out) const {
@@ -16015,19 +16014,19 @@ void swap(DropConstraintRequest &a, DropConstraintRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-DropConstraintRequest::DropConstraintRequest(const DropConstraintRequest& other550) {
+DropConstraintRequest::DropConstraintRequest(const DropConstraintRequest& other549) {
+  dbname = other549.dbname;
+  tablename = other549.tablename;
+  constraintname = other549.constraintname;
+  catName = other549.catName;
+  __isset = other549.__isset;
+}
+DropConstraintRequest& DropConstraintRequest::operator=(const DropConstraintRequest& other550) {
   dbname = other550.dbname;
   tablename = other550.tablename;
   constraintname = other550.constraintname;
   catName = other550.catName;
   __isset = other550.__isset;
-}
-DropConstraintRequest& DropConstraintRequest::operator=(const DropConstraintRequest& other551) {
-  dbname = other551.dbname;
-  tablename = other551.tablename;
-  constraintname = other551.constraintname;
-  catName = other551.catName;
-  __isset = other551.__isset;
   return *this;
 }
 void DropConstraintRequest::printTo(std::ostream& out) const {
@@ -16081,14 +16080,14 @@ uint32_t AddPrimaryKeyRequest::read(::apache::thrift::protocol::TProtocol* iprot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->primaryKeyCols.clear();
-            uint32_t _size552;
-            ::apache::thrift::protocol::TType _etype555;
-            xfer += iprot->readListBegin(_etype555, _size552);
-            this->primaryKeyCols.resize(_size552);
-            uint32_t _i556;
-            for (_i556 = 0; _i556 < _size552; ++_i556)
+            uint32_t _size551;
+            ::apache::thrift::protocol::TType _etype554;
+            xfer += iprot->readListBegin(_etype554, _size551);
+            this->primaryKeyCols.resize(_size551);
+            uint32_t _i555;
+            for (_i555 = 0; _i555 < _size551; ++_i555)
             {
-              xfer += this->primaryKeyCols[_i556].read(iprot);
+              xfer += this->primaryKeyCols[_i555].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -16119,10 +16118,10 @@ uint32_t AddPrimaryKeyRequest::write(::apache::thrift::protocol::TProtocol* opro
   xfer += oprot->writeFieldBegin("primaryKeyCols", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->primaryKeyCols.size()));
-    std::vector<SQLPrimaryKey> ::const_iterator _iter557;
-    for (_iter557 = this->primaryKeyCols.begin(); _iter557 != this->primaryKeyCols.end(); ++_iter557)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter556;
+    for (_iter556 = this->primaryKeyCols.begin(); _iter556 != this->primaryKeyCols.end(); ++_iter556)
     {
-      xfer += (*_iter557).write(oprot);
+      xfer += (*_iter556).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -16138,11 +16137,11 @@ void swap(AddPrimaryKeyRequest &a, AddPrimaryKeyRequest &b) {
   swap(a.primaryKeyCols, b.primaryKeyCols);
 }
 
-AddPrimaryKeyRequest::AddPrimaryKeyRequest(const AddPrimaryKeyRequest& other558) {
-  primaryKeyCols = other558.primaryKeyCols;
+AddPrimaryKeyRequest::AddPrimaryKeyRequest(const AddPrimaryKeyRequest& other557) {
+  primaryKeyCols = other557.primaryKeyCols;
 }
-AddPrimaryKeyRequest& AddPrimaryKeyRequest::operator=(const AddPrimaryKeyRequest& other559) {
-  primaryKeyCols = other559.primaryKeyCols;
+AddPrimaryKeyRequest& AddPrimaryKeyRequest::operator=(const AddPrimaryKeyRequest& other558) {
+  primaryKeyCols = other558.primaryKeyCols;
   return *this;
 }
 void AddPrimaryKeyRequest::printTo(std::ostream& out) const {
@@ -16193,14 +16192,14 @@ uint32_t AddForeignKeyRequest::read(::apache::thrift::protocol::TProtocol* iprot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->foreignKeyCols.clear();
-            uint32_t _size560;
-            ::apache::thrift::protocol::TType _etype563;
-            xfer += iprot->readListBegin(_etype563, _size560);
-            this->foreignKeyCols.resize(_size560);
-            uint32_t _i564;
-            for (_i564 = 0; _i564 < _size560; ++_i564)
+            uint32_t _size559;
+            ::apache::thrift::protocol::TType _etype562;
+            xfer += iprot->readListBegin(_etype562, _size559);
+            this->foreignKeyCols.resize(_size559);
+            uint32_t _i563;
+            for (_i563 = 0; _i563 < _size559; ++_i563)
             {
-              xfer += this->foreignKeyCols[_i564].read(iprot);
+              xfer += this->foreignKeyCols[_i563].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -16231,10 +16230,10 @@ uint32_t AddForeignKeyRequest::write(::apache::thrift::protocol::TProtocol* opro
   xfer += oprot->writeFieldBegin("foreignKeyCols", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->foreignKeyCols.size()));
-    std::vector<SQLForeignKey> ::const_iterator _iter565;
-    for (_iter565 = this->foreignKeyCols.begin(); _iter565 != this->foreignKeyCols.end(); ++_iter565)
+    std::vector<SQLForeignKey> ::const_iterator _iter564;
+    for (_iter564 = this->foreignKeyCols.begin(); _iter564 != this->foreignKeyCols.end(); ++_iter564)
     {
-      xfer += (*_iter565).write(oprot);
+      xfer += (*_iter564).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -16250,11 +16249,11 @@ void swap(AddForeignKeyRequest &a, AddForeignKeyRequest &b) {
   swap(a.foreignKeyCols, b.foreignKeyCols);
 }
 
-AddForeignKeyRequest::AddForeignKeyRequest(const AddForeignKeyRequest& other566) {
-  foreignKeyCols = other566.foreignKeyCols;
+AddForeignKeyRequest::AddForeignKeyRequest(const AddForeignKeyRequest& other565) {
+  foreignKeyCols = other565.foreignKeyCols;
 }
-AddForeignKeyRequest& AddForeignKeyRequest::operator=(const AddForeignKeyRequest& other567) {
-  foreignKeyCols = other567.foreignKeyCols;
+AddForeignKeyRequest& AddForeignKeyRequest::operator=(const AddForeignKeyRequest& other566) {
+  foreignKeyCols = other566.foreignKeyCols;
   return *this;
 }
 void AddForeignKeyRequest::printTo(std::ostream& out) const {
@@ -16305,14 +16304,14 @@ uint32_t AddUniqueConstraintRequest::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->uniqueConstraintCols.clear();
-            uint32_t _size568;
-            ::apache::thrift::protocol::TType _etype571;
-            xfer += iprot->readListBegin(_etype571, _size568);
-            this->uniqueConstraintCols.resize(_size568);
-            uint32_t _i572;
-            for (_i572 = 0; _i572 < _size568; ++_i572)
+            uint32_t _size567;
+            ::apache::thrift::protocol::TType _etype570;
+            xfer += iprot->readListBegin(_etype570, _size567);
+            this->uniqueConstraintCols.resize(_size567);
+            uint32_t _i571;
+            for (_i571 = 0; _i571 < _size567; ++_i571)
             {
-              xfer += this->uniqueConstraintCols[_i572].read(iprot);
+              xfer += this->uniqueConstraintCols[_i571].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -16343,10 +16342,10 @@ uint32_t AddUniqueConstraintRequest::write(::apache::thrift::protocol::TProtocol
   xfer += oprot->writeFieldBegin("uniqueConstraintCols", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->uniqueConstraintCols.size()));
-    std::vector<SQLUniqueConstraint> ::const_iterator _iter573;
-    for (_iter573 = this->uniqueConstraintCols.begin(); _iter573 != this->uniqueConstraintCols.end(); ++_iter573)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter572;
+    for (_iter572 = this->uniqueConstraintCols.begin(); _iter572 != this->uniqueConstraintCols.end(); ++_iter572)
     {
-      xfer += (*_iter573).write(oprot);
+      xfer += (*_iter572).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -16362,11 +16361,11 @@ void swap(AddUniqueConstraintRequest &a, AddUniqueConstraintRequest &b) {
   swap(a.uniqueConstraintCols, b.uniqueConstraintCols);
 }
 
-AddUniqueConstraintRequest::AddUniqueConstraintRequest(const AddUniqueConstraintRequest& other574) {
-  uniqueConstraintCols = other574.uniqueConstraintCols;
+AddUniqueConstraintRequest::AddUniqueConstraintRequest(const AddUniqueConstraintRequest& other573) {
+  uniqueConstraintCols = other573.uniqueConstraintCols;
 }
-AddUniqueConstraintRequest& AddUniqueConstraintRequest::operator=(const AddUniqueConstraintRequest& other575) {
-  uniqueConstraintCols = other575.uniqueConstraintCols;
+AddUniqueConstraintRequest& AddUniqueConstraintRequest::operator=(const AddUniqueConstraintRequest& other574) {
+  uniqueConstraintCols = other574.uniqueConstraintCols;
   return *this;
 }
 void AddUniqueConstraintRequest::printTo(std::ostream& out) const {
@@ -16417,14 +16416,14 @@ uint32_t AddNotNullConstraintRequest::read(::apache::thrift::protocol::TProtocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->notNullConstraintCols.clear();
-            uint32_t _size576;
-            ::apache::thrift::protocol::TType _etype579;
-            xfer += iprot->readListBegin(_etype579, _size576);
-            this->notNullConstraintCols.resize(_size576);
-            uint32_t _i580;
-            for (_i580 = 0; _i580 < _size576; ++_i580)
+            uint32_t _size575;
+            ::apache::thrift::protocol::TType _etype578;
+            xfer += iprot->readListBegin(_etype578, _size575);
+            this->notNullConstraintCols.resize(_size575);
+            uint32_t _i579;
+            for (_i579 = 0; _i579 < _size575; ++_i579)
             {
-              xfer += this->notNullConstraintCols[_i580].read(iprot);
+              xfer += this->notNullConstraintCols[_i579].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -16455,10 +16454,10 @@ uint32_t AddNotNullConstraintRequest::write(::apache::thrift::protocol::TProtoco
   xfer += oprot->writeFieldBegin("notNullConstraintCols", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->notNullConstraintCols.size()));
-    std::vector<SQLNotNullConstraint> ::const_iterator _iter581;
-    for (_iter581 = this->notNullConstraintCols.begin(); _iter581 != this->notNullConstraintCols.end(); ++_iter581)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter580;
+    for (_iter580 = this->notNullConstraintCols.begin(); _iter580 != this->notNullConstraintCols.end(); ++_iter580)
     {
-      xfer += (*_iter581).write(oprot);
+      xfer += (*_iter580).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -16474,11 +16473,11 @@ void swap(AddNotNullConstraintRequest &a, AddNotNullConstraintRequest &b) {
   swap(a.notNullConstraintCols, b.notNullConstraintCols);
 }
 
-AddNotNullConstraintRequest::AddNotNullConstraintRequest(const AddNotNullConstraintRequest& other582) {
-  notNullConstraintCols = other582.notNullConstraintCols;
+AddNotNullConstraintRequest::AddNotNullConstraintRequest(const AddNotNullConstraintRequest& other581) {
+  notNullConstraintCols = other581.notNullConstraintCols;
 }
-AddNotNullConstraintRequest& AddNotNullConstraintRequest::operator=(const AddNotNullConstraintRequest& other583) {
-  notNullConstraintCols = other583.notNullConstraintCols;
+AddNotNullConstraintRequest& AddNotNullConstraintRequest::operator=(const AddNotNullConstraintRequest& other582) {
+  notNullConstraintCols = other582.notNullConstraintCols;
   return *this;
 }
 void AddNotNullConstraintRequest::printTo(std::ostream& out) const {
@@ -16529,14 +16528,14 @@ uint32_t AddDefaultConstraintRequest::read(::apache::thrift::protocol::TProtocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->defaultConstraintCols.clear();
-            uint32_t _size584;
-            ::apache::thrift::protocol::TType _etype587;
-            xfer += iprot->readListBegin(_etype587, _size584);
-            this->defaultConstraintCols.resize(_size584);
-            uint32_t _i588;
-            for (_i588 = 0; _i588 < _size584; ++_i588)
+            uint32_t _size583;
+            ::apache::thrift::protocol::TType _etype586;
+            xfer += iprot->readListBegin(_etype586, _size583);
+            this->defaultConstraintCols.resize(_size583);
+            uint32_t _i587;
+            for (_i587 = 0; _i587 < _size583; ++_i587)
             {
-              xfer += this->defaultConstraintCols[_i588].read(iprot);
+              xfer += this->defaultConstraintCols[_i587].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -16567,10 +16566,10 @@ uint32_t AddDefaultConstraintRequest::write(::apache::thrift::protocol::TProtoco
   xfer += oprot->writeFieldBegin("defaultConstraintCols", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->defaultConstraintCols.size()));
-    std::vector<SQLDefaultConstraint> ::const_iterator _iter589;
-    for (_iter589 = this->defaultConstraintCols.begin(); _iter589 != this->defaultConstraintCols.end(); ++_iter589)
+    std::vector<SQLDefaultConstraint> ::const_iterator _iter588;
+    for (_iter588 = this->defaultConstraintCols.begin(); _iter588 != this->defaultConstraintCols.end(); ++_iter588)
     {
-      xfer += (*_iter589).write(oprot);
+      xfer += (*_iter588).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -16586,11 +16585,11 @@ void swap(AddDefaultConstraintRequest &a, AddDefaultConstraintRequest &b) {
   swap(a.defaultConstraintCols, b.defaultConstraintCols);
 }
 
-AddDefaultConstraintRequest::AddDefaultConstraintRequest(const AddDefaultConstraintRequest& other590) {
-  defaultConstraintCols = other590.defaultConstraintCols;
+AddDefaultConstraintRequest::AddDefaultConstraintRequest(const AddDefaultConstraintRequest& other589) {
+  defaultConstraintCols = other589.defaultConstraintCols;
 }
-AddDefaultConstraintRequest& AddDefaultConstraintRequest::operator=(const AddDefaultConstraintRequest& other591) {
-  defaultConstraintCols = other591.defaultConstraintCols;
+AddDefaultConstraintRequest& AddDefaultConstraintRequest::operator=(const AddDefaultConstraintRequest& other590) {
+  defaultConstraintCols = other590.defaultConstraintCols;
   return *this;
 }
 void AddDefaultConstraintRequest::printTo(std::ostream& out) const {
@@ -16641,14 +16640,14 @@ uint32_t AddCheckConstraintRequest::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->checkConstraintCols.clear();
-            uint32_t _size592;
-            ::apache::thrift::protocol::TType _etype595;
-            xfer += iprot->readListBegin(_etype595, _size592);
-            this->checkConstraintCols.resize(_size592);
-            uint32_t _i596;
-            for (_i596 = 0; _i596 < _size592; ++_i596)
+            uint32_t _size591;
+            ::apache::thrift::protocol::TType _etype594;
+            xfer += iprot->readListBegin(_etype594, _size591);
+            this->checkConstraintCols.resize(_size591);
+            uint32_t _i595;
+            for (_i595 = 0; _i595 < _size591; ++_i595)
             {
-              xfer += this->checkConstraintCols[_i596].read(iprot);
+              xfer += this->checkConstraintCols[_i595].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -16679,10 +16678,10 @@ uint32_t AddCheckConstraintRequest::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("checkConstraintCols", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->checkConstraintCols.size()));
-    std::vector<SQLCheckConstraint> ::const_iterator _iter597;
-    for (_iter597 = this->checkConstraintCols.begin(); _iter597 != this->checkConstraintCols.end(); ++_iter597)
+    std::vector<SQLCheckConstraint> ::const_iterator _iter596;
+    for (_iter596 = this->checkConstraintCols.begin(); _iter596 != this->checkConstraintCols.end(); ++_iter596)
     {
-      xfer += (*_iter597).write(oprot);
+      xfer += (*_iter596).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -16698,11 +16697,11 @@ void swap(AddCheckConstraintRequest &a, AddCheckConstraintRequest &b) {
   swap(a.checkConstraintCols, b.checkConstraintCols);
 }
 
-AddCheckConstraintRequest::AddCheckConstraintRequest(const AddCheckConstraintRequest& other598) {
-  checkConstraintCols = other598.checkConstraintCols;
+AddCheckConstraintRequest::AddCheckConstraintRequest(const AddCheckConstraintRequest& other597) {
+  checkConstraintCols = other597.checkConstraintCols;
 }
-AddCheckConstraintRequest& AddCheckConstraintRequest::operator=(const AddCheckConstraintRequest& other599) {
-  checkConstraintCols = other599.checkConstraintCols;
+AddCheckConstraintRequest& AddCheckConstraintRequest::operator=(const AddCheckConstraintRequest& other598) {
+  checkConstraintCols = other598.checkConstraintCols;
   return *this;
 }
 void AddCheckConstraintRequest::printTo(std::ostream& out) const {
@@ -16758,14 +16757,14 @@ uint32_t PartitionsByExprResult::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitions.clear();
-            uint32_t _size600;
-            ::apache::thrift::protocol::TType _etype603;
-            xfer += iprot->readListBegin(_etype603, _size600);
-            this->partitions.resize(_size600);
-            uint32_t _i604;
-            for (_i604 = 0; _i604 < _size600; ++_i604)
+            uint32_t _size599;
+            ::apache::thrift::protocol::TType _etype602;
+            xfer += iprot->readListBegin(_etype602, _size599);
+            this->partitions.resize(_size599);
+            uint32_t _i603;
+            for (_i603 = 0; _i603 < _size599; ++_i603)
             {
-              xfer += this->partitions[_i604].read(iprot);
+              xfer += this->partitions[_i603].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -16806,10 +16805,10 @@ uint32_t PartitionsByExprResult::write(::apache::thrift::protocol::TProtocol* op
   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 _iter605;
-    for (_iter605 = this->partitions.begin(); _iter605 != this->partitions.end(); ++_iter605)
+    std::vector<Partition> ::const_iterator _iter604;
+    for (_iter604 = this->partitions.begin(); _iter604 != this->partitions.end(); ++_iter604)
     {
-      xfer += (*_iter605).write(oprot);
+      xfer += (*_iter604).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -16830,13 +16829,13 @@ void swap(PartitionsByExprResult &a, PartitionsByExprResult &b) {
   swap(a.hasUnknownPartitions, b.hasUnknownPartitions);
 }
 
-PartitionsByExprResult::PartitionsByExprResult(const PartitionsByExprResult& other606) {
+PartitionsByExprResult::PartitionsByExprResult(const PartitionsByExprResult& other605) {
+  partitions = other605.partitions;
+  hasUnknownPartitions = other605.hasUnknownPartitions;
+}
+PartitionsByExprResult& PartitionsByExprResult::operator=(const PartitionsByExprResult& other606) {
   partitions = other606.partitions;
   hasUnknownPartitions = other606.hasUnknownPartitions;
-}
-PartitionsByExprResult& PartitionsByExprResult::operator=(const PartitionsByExprResult& other607) {
-  partitions = other607.partitions;
-  hasUnknownPartitions = other607.hasUnknownPartitions;
   return *this;
 }
 void PartitionsByExprResult::printTo(std::ostream& out) const {
@@ -16893,14 +16892,14 @@ uint32_t PartitionsSpecByExprResult::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitionsSpec.clear();
-            uint32_t _size608;
-            ::apache::thrift::protocol::TType _etype611;
-            xfer += iprot->readListBegin(_etype611, _size608);
-            this->partitionsSpec.resize(_size608);
-            uint32_t _i612;
-            for (_i612 = 0; _i612 < _size608; ++_i612)
+            uint32_t _size607;
+            ::apache::thrift::protocol::TType _etype610;
+            xfer += iprot->readListBegin(_etype610, _size607);
+            this->partitionsSpec.resize(_size607);
+            uint32_t _i611;
+            for (_i611 = 0; _i611 < _size607; ++_i611)
             {
-              xfer += this->partitionsSpec[_i612].read(iprot);
+              xfer += this->partitionsSpec[_i611].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -16941,10 +16940,10 @@ uint32_t PartitionsSpecByExprResult::write(::apache::thrift::protocol::TProtocol
   xfer += oprot->writeFieldBegin("partitionsSpec", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->partitionsSpec.size()));
-    std::vector<PartitionSpec> ::const_iterator _iter613;
-    for (_iter613 = this->partitionsSpec.begin(); _iter613 != this->partitionsSpec.end(); ++_iter613)
+    std::vector<PartitionSpec> ::const_iterator _iter612;
+    for (_iter612 = this->partitionsSpec.begin(); _iter612 != this->partitionsSpec.end(); ++_iter612)
     {
-      xfer += (*_iter613).write(oprot);
+      xfer += (*_iter612).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -16965,13 +16964,13 @@ void swap(PartitionsSpecByExprResult &a, PartitionsSpecByExprResult &b) {
   swap(a.hasUnknownPartitions, b.hasUnknownPartitions);
 }
 
-PartitionsSpecByExprResult::PartitionsSpecByExprResult(const PartitionsSpecByExprResult& other614) {
+PartitionsSpecByExprResult::PartitionsSpecByExprResult(const PartitionsSpecByExprResult& other613) {
+  partitionsSpec = other613.partitionsSpec;
+  hasUnknownPartitions = other613.hasUnknownPartitions;
+}
+PartitionsSpecByExprResult& PartitionsSpecByExprResult::operator=(const PartitionsSpecByExprResult& other614) {
   partitionsSpec = other614.partitionsSpec;
   hasUnknownPartitions = other614.hasUnknownPartitions;
-}
-PartitionsSpecByExprResult& PartitionsSpecByExprResult::operator=(const PartitionsSpecByExprResult& other615) {
-  partitionsSpec = other615.partitionsSpec;
-  hasUnknownPartitions = other615.hasUnknownPartitions;
   return *this;
 }
 void PartitionsSpecByExprResult::printTo(std::ostream& out) const {
@@ -17215,7 +17214,19 @@ void swap(PartitionsByExprRequest &a, PartitionsByExprRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-PartitionsByExprRequest::PartitionsByExprRequest(const PartitionsByExprRequest& other616) {
+PartitionsByExprRequest::PartitionsByExprRequest(const PartitionsByExprRequest& other615) {
+  dbName = other615.dbName;
+  tblName = other615.tblName;
+  expr = other615.expr;
+  defaultPartitionName = other615.defaultPartitionName;
+  maxParts = other615.maxParts;
+  catName = other615.catName;
+  order = other615.order;
+  validWriteIdList = other615.validWriteIdList;
+  id = other615.id;
+  __isset = other615.__isset;
+}
+PartitionsByExprRequest& PartitionsByExprRequest::operator=(const PartitionsByExprRequest& other616) {
   dbName = other616.dbName;
   tblName = other616.tblName;
   expr = other616.expr;
@@ -17226,18 +17237,6 @@ PartitionsByExprRequest::PartitionsByExprRequest(const PartitionsByExprRequest&
   validWriteIdList = other616.validWriteIdList;
   id = other616.id;
   __isset = other616.__isset;
-}
-PartitionsByExprRequest& PartitionsByExprRequest::operator=(const PartitionsByExprRequest& other617) {
-  dbName = other617.dbName;
-  tblName = other617.tblName;
-  expr = other617.expr;
-  defaultPartitionName = other617.defaultPartitionName;
-  maxParts = other617.maxParts;
-  catName = other617.catName;
-  order = other617.order;
-  validWriteIdList = other617.validWriteIdList;
-  id = other617.id;
-  __isset = other617.__isset;
   return *this;
 }
 void PartitionsByExprRequest::printTo(std::ostream& out) const {
@@ -17301,14 +17300,14 @@ uint32_t TableStatsResult::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tableStats.clear();
-            uint32_t _size618;
-            ::apache::thrift::protocol::TType _etype621;
-            xfer += iprot->readListBegin(_etype621, _size618);
-            this->tableStats.resize(_size618);
-            uint32_t _i622;
-            for (_i622 = 0; _i622 < _size618; ++_i622)
+            uint32_t _size617;
+            ::apache::thrift::protocol::TType _etype620;
+            xfer += iprot->readListBegin(_etype620, _size617);
+            this->tableStats.resize(_size617);
+            uint32_t _i621;
+            for (_i621 = 0; _i621 < _size617; ++_i621)
             {
-              xfer += this->tableStats[_i622].read(iprot);
+              xfer += this->tableStats[_i621].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -17347,10 +17346,10 @@ uint32_t TableStatsResult::write(::apache::thrift::protocol::TProtocol* oprot) c
   xfer += oprot->writeFieldBegin("tableStats", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->tableStats.size()));
-    std::vector<ColumnStatisticsObj> ::const_iterator _iter623;
-    for (_iter623 = this->tableStats.begin(); _iter623 != this->tableStats.end(); ++_iter623)
+    std::vector<ColumnStatisticsObj> ::const_iterator _iter622;
+    for (_iter622 = this->tableStats.begin(); _iter622 != this->tableStats.end(); ++_iter622)
     {
-      xfer += (*_iter623).write(oprot);
+      xfer += (*_iter622).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -17373,15 +17372,15 @@ void swap(TableStatsResult &a, TableStatsResult &b) {
   swap(a.__isset, b.__isset);
 }
 
-TableStatsResult::TableStatsResult(const TableStatsResult& other624) {
+TableStatsResult::TableStatsResult(const TableStatsResult& other623) {
+  tableStats = other623.tableStats;
+  isStatsCompliant = other623.isStatsCompliant;
+  __isset = other623.__isset;
+}
+TableStatsResult& TableStatsResult::operator=(const TableStatsResult& other624) {
   tableStats = other624.tableStats;
   isStatsCompliant = other624.isStatsCompliant;
   __isset = other624.__isset;
-}
-TableStatsResult& TableStatsResult::operator=(const TableStatsResult& other625) {
-  tableStats = other625.tableStats;
-  isStatsCompliant = other625.isStatsCompliant;
-  __isset = other625.__isset;
   return *this;
 }
 void TableStatsResult::printTo(std::ostream& out) const {
@@ -17438,26 +17437,26 @@ uint32_t PartitionsStatsResult::read(::apache::thrift::protocol::TProtocol* ipro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partStats.clear();
-            uint32_t _size626;
-            ::apache::thrift::protocol::TType _ktype627;
-            ::apache::thrift::protocol::TType _vtype628;
-            xfer += iprot->readMapBegin(_ktype627, _vtype628, _size626);
-            uint32_t _i630;
-            for (_i630 = 0; _i630 < _size626; ++_i630)
+            uint32_t _size625;
+            ::apache::thrift::protocol::TType _ktype626;
+            ::apache::thrift::protocol::TType _vtype627;
+            xfer += iprot->readMapBegin(_ktype626, _vtype627, _size625);
+            uint32_t _i629;
+            for (_i629 = 0; _i629 < _size625; ++_i629)
             {
-              std::string _key631;
-              xfer += iprot->readString(_key631);
-              std::vector<ColumnStatisticsObj> & _val632 = this->partStats[_key631];
+              std::string _key630;
+              xfer += iprot->readString(_key630);
+              std::vector<ColumnStatisticsObj> & _val631 = this->partStats[_key630];
               {
-                _val632.clear();
-                uint32_t _size633;
-                ::apache::thrift::protocol::TType _etype636;
-                xfer += iprot->readListBegin(_etype636, _size633);
-                _val632.resize(_size633);
-                uint32_t _i637;
-                for (_i637 = 0; _i637 < _size633; ++_i637)
+                _val631.clear();
+                uint32_t _size632;
+                ::apache::thrift::protocol::TType _etype635;
+                xfer += iprot->readListBegin(_etype635, _size632);
+                _val631.resize(_size632);
+                uint32_t _i636;
+                for (_i636 = 0; _i636 < _size632; ++_i636)
                 {
-                  xfer += _val632[_i637].read(iprot);
+                  xfer += _val631[_i636].read(iprot);
                 }
                 xfer += iprot->readListEnd();
               }
@@ -17499,16 +17498,16 @@ uint32_t PartitionsStatsResult::write(::apache::thrift::protocol::TProtocol* opr
   xfer += oprot->writeFieldBegin("partStats", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_LIST, static_cast<uint32_t>(this->partStats.size()));
-    std::map<std::string, std::vector<ColumnStatisticsObj> > ::const_iterator _iter638;
-    for (_iter638 = this->partStats.begin(); _iter638 != this->partStats.end(); ++_iter638)
+    std::map<std::string, std::vector<ColumnStatisticsObj> > ::const_iterator _iter637;
+    for (_iter637 = this->partStats.begin(); _iter637 != this->partStats.end(); ++_iter637)
     {
-      xfer += oprot->writeString(_iter638->first);
+      xfer += oprot->writeString(_iter637->first);
       {
-        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(_iter638->second.size()));
-        std::vector<ColumnStatisticsObj> ::const_iterator _iter639;
-        for (_iter639 = _iter638->second.begin(); _iter639 != _iter638->second.end(); ++_iter639)
+        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(_iter637->second.size()));
+        std::vector<ColumnStatisticsObj> ::const_iterator _iter638;
+        for (_iter638 = _iter637->second.begin(); _iter638 != _iter637->second.end(); ++_iter638)
         {
-          xfer += (*_iter639).write(oprot);
+          xfer += (*_iter638).write(oprot);
         }
         xfer += oprot->writeListEnd();
       }
@@ -17534,15 +17533,15 @@ void swap(PartitionsStatsResult &a, PartitionsStatsResult &b) {
   swap(a.__isset, b.__isset);
 }
 
-PartitionsStatsResult::PartitionsStatsResult(const PartitionsStatsResult& other640) {
+PartitionsStatsResult::PartitionsStatsResult(const PartitionsStatsResult& other639) {
+  partStats = other639.partStats;
+  isStatsCompliant = other639.isStatsCompliant;
+  __isset = other639.__isset;
+}
+PartitionsStatsResult& PartitionsStatsResult::operator=(const PartitionsStatsResult& other640) {
   partStats = other640.partStats;
   isStatsCompliant = other640.isStatsCompliant;
   __isset = other640.__isset;
-}
-PartitionsStatsResult& PartitionsStatsResult::operator=(const PartitionsStatsResult& other641) {
-  partStats = other641.partStats;
-  isStatsCompliant = other641.isStatsCompliant;
-  __isset = other641.__isset;
   return *this;
 }
 void PartitionsStatsResult::printTo(std::ostream& out) const {
@@ -17640,14 +17639,14 @@ uint32_t TableStatsRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->colNames.clear();
-            uint32_t _size642;
-            ::apache::thrift::protocol::TType _etype645;
-            xfer += iprot->readListBegin(_etype645, _size642);
-            this->colNames.resize(_size642);
-            uint32_t _i646;
-            for (_i646 = 0; _i646 < _size642; ++_i646)
+            uint32_t _size641;
+            ::apache::thrift::protocol::TType _etype644;
+            xfer += iprot->readListBegin(_etype644, _size641);
+            this->colNames.resize(_size641);
+            uint32_t _i645;
+            for (_i645 = 0; _i645 < _size641; ++_i645)
             {
-              xfer += iprot->readString(this->colNames[_i646]);
+              xfer += iprot->readString(this->colNames[_i645]);
             }
             xfer += iprot->readListEnd();
           }
@@ -17724,10 +17723,10 @@ uint32_t TableStatsRequest::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("colNames", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->colNames.size()));
-    std::vector<std::string> ::const_iterator _iter647;
-    for (_iter647 = this->colNames.begin(); _iter647 != this->colNames.end(); ++_iter647)
+    std::vector<std::string> ::const_iterator _iter646;
+    for (_iter646 = this->colNames.begin(); _iter646 != this->colNames.end(); ++_iter646)
     {
-      xfer += oprot->writeString((*_iter647));
+      xfer += oprot->writeString((*_iter646));
     }
     xfer += oprot->writeListEnd();
   }
@@ -17769,7 +17768,17 @@ void swap(TableStatsRequest &a, TableStatsRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-TableStatsRequest::TableStatsRequest(const TableStatsRequest& other648) {
+TableStatsRequest::TableStatsRequest(const TableStatsRequest& other647) {
+  dbName = other647.dbName;
+  tblName = other647.tblName;
+  colNames = other647.colNames;
+  catName = other647.catName;
+  validWriteIdList = other647.validWriteIdList;
+  engine = other647.engine;
+  id = other647.id;
+  __isset = other647.__isset;
+}
+TableStatsRequest& TableStatsRequest::operator=(const TableStatsRequest& other648) {
   dbName = other648.dbName;
   tblName = other648.tblName;
   colNames = other648.colNames;
@@ -17778,16 +17787,6 @@ TableStatsRequest::TableStatsRequest(const TableStatsRequest& other648) {
   engine = other648.engine;
   id = other648.id;
   __isset = other648.__isset;
-}
-TableStatsRequest& TableStatsRequest::operator=(const TableStatsRequest& other649) {
-  dbName = other649.dbName;
-  tblName = other649.tblName;
-  colNames = other649.colNames;
-  catName = other649.catName;
-  validWriteIdList = other649.validWriteIdList;
-  engine = other649.engine;
-  id = other649.id;
-  __isset = other649.__isset;
   return *this;
 }
 void TableStatsRequest::printTo(std::ostream& out) const {
@@ -17890,14 +17889,14 @@ uint32_t PartitionsStatsRequest::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->colNames.clear();
-            uint32_t _size650;
-            ::apache::thrift::protocol::TType _etype653;
-            xfer += iprot->readListBegin(_etype653, _size650);
-            this->colNames.resize(_size650);
-            uint32_t _i654;
-            for (_i654 = 0; _i654 < _size650; ++_i654)
+            uint32_t _size649;
+            ::apache::thrift::protocol::TType _etype652;
+            xfer += iprot->readListBegin(_etype652, _size649);
+            this->colNames.resize(_size649);
+            uint32_t _i653;
+            for (_i653 = 0; _i653 < _size649; ++_i653)
             {
-              xfer += iprot->readString(this->colNames[_i654]);
+              xfer += iprot->readString(this->colNames[_i653]);
             }
             xfer += iprot->readListEnd();
           }
@@ -17910,14 +17909,14 @@ uint32_t PartitionsStatsRequest::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partNames.clear();
-            uint32_t _size655;
-            ::apache::thrift::protocol::TType _etype658;
-            xfer += iprot->readListBegin(_etype658, _size655);
-            this->partNames.resize(_size655);
-            uint32_t _i659;
-            for (_i659 = 0; _i659 < _size655; ++_i659)
+            uint32_t _size654;
+            ::apache::thrift::protocol::TType _etype657;
+            xfer += iprot->readListBegin(_etype657, _size654);
+            this->partNames.resize(_size654);
+            uint32_t _i658;
+            for (_i658 = 0; _i658 < _size654; ++_i658)
             {
-              xfer += iprot->readString(this->partNames[_i659]);
+              xfer += iprot->readString(this->partNames[_i658]);
             }
             xfer += iprot->readListEnd();
           }
@@ -17988,10 +17987,10 @@ uint32_t PartitionsStatsRequest::write(::apache::thrift::protocol::TProtocol* op
   xfer += oprot->writeFieldBegin("colNames", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->colNames.size()));
-    std::vector<std::string> ::const_iterator _iter660;
-    for (_iter660 = this->colNames.begin(); _iter660 != this->colNames.end(); ++_iter660)
+    std::vector<std::string> ::const_iterator _iter659;
+    for (_iter659 = this->colNames.begin(); _iter659 != this->colNames.end(); ++_iter659)
     {
-      xfer += oprot->writeString((*_iter660));
+      xfer += oprot->writeString((*_iter659));
     }
     xfer += oprot->writeListEnd();
   }
@@ -18000,10 +17999,10 @@ uint32_t PartitionsStatsRequest::write(::apache::thrift::protocol::TProtocol* op
   xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 4);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partNames.size()));
-    std::vector<std::string> ::const_iterator _iter661;
-    for (_iter661 = this->partNames.begin(); _iter661 != this->partNames.end(); ++_iter661)
+    std::vector<std::string> ::const_iterator _iter660;
+    for (_iter660 = this->partNames.begin(); _iter660 != this->partNames.end(); ++_iter660)
     {
-      xfer += oprot->writeString((*_iter661));
+      xfer += oprot->writeString((*_iter660));
     }
     xfer += oprot->writeListEnd();
   }
@@ -18040,7 +18039,17 @@ void swap(PartitionsStatsRequest &a, PartitionsStatsRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-PartitionsStatsRequest::PartitionsStatsRequest(const PartitionsStatsRequest& other662) {
+PartitionsStatsRequest::PartitionsStatsRequest(const PartitionsStatsRequest& other661) {
+  dbName = other661.dbName;
+  tblName = other661.tblName;
+  colNames = other661.colNames;
+  partNames = other661.partNames;
+  catName = other661.catName;
+  validWriteIdList = other661.validWriteIdList;
+  engine = other661.engine;
+  __isset = other661.__isset;
+}
+PartitionsStatsRequest& PartitionsStatsRequest::operator=(const PartitionsStatsRequest& other662) {
   dbName = other662.dbName;
   tblName = other662.tblName;
   colNames = other662.colNames;
@@ -18049,16 +18058,6 @@ PartitionsStatsRequest::PartitionsStatsRequest(const PartitionsStatsRequest& oth
   validWriteIdList = other662.validWriteIdList;
   engine = other662.engine;
   __isset = other662.__isset;
-}
-PartitionsStatsRequest& PartitionsStatsRequest::operator=(const PartitionsStatsRequest& other663) {
-  dbName = other663.dbName;
-  tblName = other663.tblName;
-  colNames = other663.colNames;
-  partNames = other663.partNames;
-  catName = other663.catName;
-  validWriteIdList = other663.validWriteIdList;
-  engine = other663.engine;
-  __isset = other663.__isset;
   return *this;
 }
 void PartitionsStatsRequest::printTo(std::ostream& out) const {
@@ -18120,14 +18119,14 @@ uint32_t AddPartitionsResult::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitions.clear();
-            uint32_t _size664;
-            ::apache::thrift::protocol::TType _etype667;
-            xfer += iprot->readListBegin(_etype667, _size664);
-            this->partitions.resize(_size664);
-            uint32_t _i668;
-            for (_i668 = 0; _i668 < _size664; ++_i668)
+            uint32_t _size663;
+            ::apache::thrift::protocol::TType _etype666;
+            xfer += iprot->readListBegin(_etype666, _size663);
+            this->partitions.resize(_size663);
+            uint32_t _i667;
+            for (_i667 = 0; _i667 < _size663; ++_i667)
             {
-              xfer += this->partitions[_i668].read(iprot);
+              xfer += this->partitions[_i667].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -18165,10 +18164,10 @@ uint32_t AddPartitionsResult::write(::apache::thrift::protocol::TProtocol* oprot
     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 _iter669;
-      for (_iter669 = this->partitions.begin(); _iter669 != this->partitions.end(); ++_iter669)
+      std::vector<Partition> ::const_iterator _iter668;
+      for (_iter668 = this->partitions.begin(); _iter668 != this->partitions.end(); ++_iter668)
       {
-        xfer += (*_iter669).write(oprot);
+        xfer += (*_iter668).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -18191,15 +18190,15 @@ void swap(AddPartitionsResult &a, AddPartitionsResult &b) {
   swap(a.__isset, b.__isset);
 }
 
-AddPartitionsResult::AddPartitionsResult(const AddPartitionsResult& other670) {
+AddPartitionsResult::AddPartitionsResult(const AddPartitionsResult& other669) {
+  partitions = other669.partitions;
+  isStatsCompliant = other669.isStatsCompliant;
+  __isset = other669.__isset;
+}
+AddPartitionsResult& AddPartitionsResult::operator=(const AddPartitionsResult& other670) {
   partitions = other670.partitions;
   isStatsCompliant = other670.isStatsCompliant;
   __isset = other670.__isset;
-}
-AddPartitionsResult& AddPartitionsResult::operator=(const AddPartitionsResult& other671) {
-  partitions = other671.partitions;
-  isStatsCompliant = other671.isStatsCompliant;
-  __isset = other671.__isset;
   return *this;
 }
 void AddPartitionsResult::printTo(std::ostream& out) const {
@@ -18297,14 +18296,14 @@ uint32_t AddPartitionsRequest::read(::apache::thrift::protocol::TProtocol* iprot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->parts.clear();
-            uint32_t _size672;
-            ::apache::thrift::protocol::TType _etype675;
-            xfer += iprot->readListBegin(_etype675, _size672);
-            this->parts.resize(_size672);
-            uint32_t _i676;
-            for (_i676 = 0; _i676 < _size672; ++_i676)
+            uint32_t _size671;
+            ::apache::thrift::protocol::TType _etype674;
+            xfer += iprot->readListBegin(_etype674, _size671);
+            this->parts.resize(_size671);
+            uint32_t _i675;
+            for (_i675 = 0; _i675 < _size671; ++_i675)
             {
-              xfer += this->parts[_i676].read(iprot);
+              xfer += this->parts[_i675].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -18381,10 +18380,10 @@ uint32_t AddPartitionsRequest::write(::apache::thrift::protocol::TProtocol* opro
   xfer += oprot->writeFieldBegin("parts", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->parts.size()));
-    std::vector<Partition> ::const_iterator _iter677;
-    for (_iter677 = this->parts.begin(); _iter677 != this->parts.end(); ++_iter677)
+    std::vector<Partition> ::const_iterator _iter676;
+    for (_iter676 = this->parts.begin(); _iter676 != this->parts.end(); ++_iter676)
     {
-      xfer += (*_iter677).write(oprot);
+      xfer += (*_iter676).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -18426,7 +18425,17 @@ void swap(AddPartitionsRequest &a, AddPartitionsRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-AddPartitionsRequest::AddPartitionsRequest(const AddPartitionsRequest& other678) {
+AddPartitionsRequest::AddPartitionsRequest(const AddPartitionsRequest& other677) {
+  dbName = other677.dbName;
+  tblName = other677.tblName;
+  parts = other677.parts;
+  ifNotExists = other677.ifNotExists;
+  needResult = other677.needResult;
+  catName = other677.catName;
+  validWriteIdList = other677.validWriteIdList;
+  __isset = other677.__isset;
+}
+AddPartitionsRequest& AddPartitionsRequest::operator=(const AddPartitionsRequest& other678) {
   dbName = other678.dbName;
   tblName = other678.tblName;
   parts = other678.parts;
@@ -18435,16 +18444,6 @@ AddPartitionsRequest::AddPartitionsRequest(const AddPartitionsRequest& other678)
   catName = other678.catName;
   validWriteIdList = other678.validWriteIdList;
   __isset = other678.__isset;
-}
-AddPartitionsRequest& AddPartitionsRequest::operator=(const AddPartitionsRequest& other679) {
-  dbName = other679.dbName;
-  tblName = other679.tblName;
-  parts = other679.parts;
-  ifNotExists = other679.ifNotExists;
-  needResult = other679.needResult;
-  catName = other679.catName;
-  validWriteIdList = other679.validWriteIdList;
-  __isset = other679.__isset;
   return *this;
 }
 void AddPartitionsRequest::printTo(std::ostream& out) const {
@@ -18501,14 +18500,14 @@ uint32_t DropPartitionsResult::read(::apache::thrift::protocol::TProtocol* iprot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitions.clear();
-            uint32_t _size680;
-            ::apache::thrift::protocol::TType _etype683;
-            xfer += iprot->readListBegin(_etype683, _size680);
-            this->partitions.resize(_size680);
-            uint32_t _i684;
-            for (_i684 = 0; _i684 < _size680; ++_i684)
+            uint32_t _size679;
+            ::apache::thrift::protocol::TType _etype682;
+            xfer += iprot->readListBegin(_etype682, _size679);
+            this->partitions.resize(_size679);
+            uint32_t _i683;
+            for (_i683 = 0; _i683 < _size679; ++_i683)
             {
-              xfer += this->partitions[_i684].read(iprot);
+              xfer += this->partitions[_i683].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -18538,10 +18537,10 @@ uint32_t DropPartitionsResult::write(::apache::thrift::protocol::TProtocol* opro
     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 _iter685;
-      for (_iter685 = this->partitions.begin(); _iter685 != this->partitions.end(); ++_iter685)
+      std::vector<Partition> ::const_iterator _iter684;
+      for (_iter684 = this->partitions.begin(); _iter684 != this->partitions.end(); ++_iter684)
       {
-        xfer += (*_iter685).write(oprot);
+        xfer += (*_iter684).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -18558,13 +18557,13 @@ void swap(DropPartitionsResult &a, DropPartitionsResult &b) {
   swap(a.__isset, b.__isset);
 }
 
-DropPartitionsResult::DropPartitionsResult(const DropPartitionsResult& other686) {
+DropPartitionsResult::DropPartitionsResult(const DropPartitionsResult& other685) {
+  partitions = other685.partitions;
+  __isset = other685.__isset;
+}
+DropPartitionsResult& DropPartitionsResult::operator=(const DropPartitionsResult& other686) {
   partitions = other686.partitions;
   __isset = other686.__isset;
-}
-DropPartitionsResult& DropPartitionsResult::operator=(const DropPartitionsResult& other687) {
-  partitions = other687.partitions;
-  __isset = other687.__isset;
   return *this;
 }
 void DropPartitionsResult::printTo(std::ostream& out) const {
@@ -18672,15 +18671,15 @@ void swap(DropPartitionsExpr &a, DropPartitionsExpr &b) {
   swap(a.__isset, b.__isset);
 }
 
-DropPartitionsExpr::DropPartitionsExpr(const DropPartitionsExpr& other688) {
+DropPartitionsExpr::DropPartitionsExpr(const DropPartitionsExpr& other687) {
+  expr = other687.expr;
+  partArchiveLevel = other687.partArchiveLevel;
+  __isset = other687.__isset;
+}
+DropPartitionsExpr& DropPartitionsExpr::operator=(const DropPartitionsExpr& other688) {
   expr = other688.expr;
   partArchiveLevel = other688.partArchiveLevel;
   __isset = other688.__isset;
-}
-DropPartitionsExpr& DropPartitionsExpr::operator=(const DropPartitionsExpr& other689) {
-  expr = other689.expr;
-  partArchiveLevel = other689.partArchiveLevel;
-  __isset = other689.__isset;
   return *this;
 }
 void DropPartitionsExpr::printTo(std::ostream& out) const {
@@ -18737,14 +18736,14 @@ uint32_t RequestPartsSpec::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->names.clear();
-            uint32_t _size690;
-            ::apache::thrift::protocol::TType _etype693;
-            xfer += iprot->readListBegin(_etype693, _size690);
-            this->names.resize(_size690);
-            uint32_t _i694;
-            for (_i694 = 0; _i694 < _size690; ++_i694)
+            uint32_t _size689;
+            ::apache::thrift::protocol::TType _etype692;
+            xfer += iprot->readListBegin(_etype692, _size689);
+            this->names.resize(_size689);
+            uint32_t _i693;
+            for (_i693 = 0; _i693 < _size689; ++_i693)
             {
-              xfer += iprot->readString(this->names[_i694]);
+              xfer += iprot->readString(this->names[_i693]);
             }
             xfer += iprot->readListEnd();
           }
@@ -18757,14 +18756,14 @@ uint32_t RequestPartsSpec::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->exprs.clear();
-            uint32_t _size695;
-            ::apache::thrift::protocol::TType _etype698;
-            xfer += iprot->readListBegin(_etype698, _size695);
-            this->exprs.resize(_size695);
-            uint32_t _i699;
-            for (_i699 = 0; _i699 < _size695; ++_i699)
+            uint32_t _size694;
+            ::apache::thrift::protocol::TType _etype697;
+            xfer += iprot->readListBegin(_etype697, _size694);
+            this->exprs.resize(_size694);
+            uint32_t _i698;
+            for (_i698 = 0; _i698 < _size694; ++_i698)
             {
-              xfer += this->exprs[_i699].read(iprot);
+              xfer += this->exprs[_i698].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -18794,10 +18793,10 @@ uint32_t RequestPartsSpec::write(::apache::thrift::protocol::TProtocol* oprot) c
     xfer += oprot->writeFieldBegin("names", ::apache::thrift::protocol::T_LIST, 1);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->names.size()));
-      std::vector<std::string> ::const_iterator _iter700;
-      for (_iter700 = this->names.begin(); _iter700 != this->names.end(); ++_iter700)
+      std::vector<std::string> ::const_iterator _iter699;
+      for (_iter699 = this->names.begin(); _iter699 != this->names.end(); ++_iter699)
       {
-        xfer += oprot->writeString((*_iter700));
+        xfer += oprot->writeString((*_iter699));
       }
       xfer += oprot->writeListEnd();
     }
@@ -18807,10 +18806,10 @@ uint32_t RequestPartsSpec::write(::apache::thrift::protocol::TProtocol* oprot) c
     xfer += oprot->writeFieldBegin("exprs", ::apache::thrift::protocol::T_LIST, 2);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->exprs.size()));
-      std::vector<DropPartitionsExpr> ::const_iterator _iter701;
-      for (_iter701 = this->exprs.begin(); _iter701 != this->exprs.end(); ++_iter701)
+      std::vector<DropPartitionsExpr> ::const_iterator _iter700;
+      for (_iter700 = this->exprs.begin(); _iter700 != this->exprs.end(); ++_iter700)
       {
-        xfer += (*_iter701).write(oprot);
+        xfer += (*_iter700).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -18828,15 +18827,15 @@ void swap(RequestPartsSpec &a, RequestPartsSpec &b) {
   swap(a.__isset, b.__isset);
 }
 
-RequestPartsSpec::RequestPartsSpec(const RequestPartsSpec& other702) {
+RequestPartsSpec::RequestPartsSpec(const RequestPartsSpec& other701) {
+  names = other701.names;
+  exprs = other701.exprs;
+  __isset = other701.__isset;
+}
+RequestPartsSpec& RequestPartsSpec::operator=(const RequestPartsSpec& other702) {
   names = other702.names;
   exprs = other702.exprs;
   __isset = other702.__isset;
-}
-RequestPartsSpec& RequestPartsSpec::operator=(const RequestPartsSpec& other703) {
-  names = other703.names;
-  exprs = other703.exprs;
-  __isset = other703.__isset;
   return *this;
 }
 void RequestPartsSpec::printTo(std::ostream& out) const {
@@ -19080,7 +19079,19 @@ void swap(DropPartitionsRequest &a, DropPartitionsRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-DropPartitionsRequest::DropPartitionsRequest(const DropPartitionsRequest& other704) {
+DropPartitionsRequest::DropPartitionsRequest(const DropPartitionsRequest& other703) {
+  dbName = other703.dbName;
+  tblName = other703.tblName;
+  parts = other703.parts;
+  deleteData = other703.deleteData;
+  ifExists = other703.ifExists;
+  ignoreProtection = other703.ignoreProtection;
+  environmentContext = other703.environmentContext;
+  needResult = other703.needResult;
+  catName = other703.catName;
+  __isset = other703.__isset;
+}
+DropPartitionsRequest& DropPartitionsRequest::operator=(const DropPartitionsRequest& other704) {
   dbName = other704.dbName;
   tblName = other704.tblName;
   parts = other704.parts;
@@ -19091,18 +19102,6 @@ DropPartitionsRequest::DropPartitionsRequest(const DropPartitionsRequest& other7
   needResult = other704.needResult;
   catName = other704.catName;
   __isset = other704.__isset;
-}
-DropPartitionsRequest& DropPartitionsRequest::operator=(const DropPartitionsRequest& other705) {
-  dbName = other705.dbName;
-  tblName = other705.tblName;
-  parts = other705.parts;
-  deleteData = other705.deleteData;
-  ifExists = other705.ifExists;
-  ignoreProtection = other705.ignoreProtection;
-  environmentContext = other705.environmentContext;
-  needResult = other705.needResult;
-  catName = other705.catName;
-  __isset = other705.__isset;
   return *this;
 }
 void DropPartitionsRequest::printTo(std::ostream& out) const {
@@ -19222,14 +19221,14 @@ uint32_t PartitionValuesRequest::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitionKeys.clear();
-            uint32_t _size706;
-            ::apache::thrift::protocol::TType _etype709;
-            xfer += iprot->readListBegin(_etype709, _size706);
-            this->partitionKeys.resize(_size706);
-            uint32_t _i710;
-            for (_i710 = 0; _i710 < _size706; ++_i710)
+            uint32_t _size705;
+            ::apache::thrift::protocol::TType _etype708;
+            xfer += iprot->readListBegin(_etype708, _size705);
+            this->partitionKeys.resize(_size705);
+            uint32_t _i709;
+            for (_i709 = 0; _i709 < _size705; ++_i709)
             {
-              xfer += this->partitionKeys[_i710].read(iprot);
+              xfer += this->partitionKeys[_i709].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -19258,14 +19257,14 @@ uint32_t PartitionValuesRequest::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitionOrder.clear();
-            uint32_t _size711;
-            ::apache::thrift::protocol::TType _etype714;
-            xfer += iprot->readListBegin(_etype714, _size711);
-            this->partitionOrder.resize(_size711);
-            uint32_t _i715;
-            for (_i715 = 0; _i715 < _size711; ++_i715)
+            uint32_t _size710;
+            ::apache::thrift::protocol::TType _etype713;
+            xfer += iprot->readListBegin(_etype713, _size710);
+            this->partitionOrder.resize(_size710);
+            uint32_t _i714;
+            for (_i714 = 0; _i714 < _size710; ++_i714)
             {
-              xfer += this->partitionOrder[_i715].read(iprot);
+              xfer += this->partitionOrder[_i714].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -19340,10 +19339,10 @@ uint32_t PartitionValuesRequest::write(::apache::thrift::protocol::TProtocol* op
   xfer += oprot->writeFieldBegin("partitionKeys", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->partitionKeys.size()));
-    std::vector<FieldSchema> ::const_iterator _iter716;
-    for (_iter716 = this->partitionKeys.begin(); _iter716 != this->partitionKeys.end(); ++_iter716)
+    std::vector<FieldSchema> ::const_iterator _iter715;
+    for (_iter715 = this->partitionKeys.begin(); _iter715 != this->partitionKeys.end(); ++_iter715)
     {
-      xfer += (*_iter716).write(oprot);
+      xfer += (*_iter715).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -19363,10 +19362,10 @@ uint32_t PartitionValuesRequest::write(::apache::thrift::protocol::TProtocol* op
     xfer += oprot->writeFieldBegin("partitionOrder", ::apache::thrift::protocol::T_LIST, 6);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->partitionOrder.size()));
-      std::vector<FieldSchema> ::const_iterator _iter717;
-      for (_iter717 = this->partitionOrder.begin(); _iter717 != this->partitionOrder.end(); ++_iter717)
+      std::vector<FieldSchema> ::const_iterator _iter716;
+      for (_iter716 = this->partitionOrder.begin(); _iter716 != this->partitionOrder.end(); ++_iter716)
       {
-        xfer += (*_iter717).write(oprot);
+        xfer += (*_iter716).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -19412,7 +19411,20 @@ void swap(PartitionValuesRequest &a, PartitionValuesRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-PartitionValuesRequest::PartitionValuesRequest(const PartitionValuesRequest& other718) {
+PartitionValuesRequest::PartitionValuesRequest(const PartitionValuesRequest& other717) {
+  dbName = other717.dbName;
+  tblName = other717.tblName;
+  partitionKeys = other717.partitionKeys;
+  applyDistinct = other717.applyDistinct;
+  filter = other717.filter;
+  partitionOrder = other717.partitionOrder;
+  ascending = other717.ascending;
+  maxParts = other717.maxParts;
+  catName = other717.catName;
+  validWriteIdList = other717.validWriteIdList;
+  __isset = other717.__isset;
+}
+PartitionValuesRequest& PartitionValuesRequest::operator=(const PartitionValuesRequest& other718) {
   dbName = other718.dbName;
   tblName = other718.tblName;
   partitionKeys = other718.partitionKeys;
@@ -19424,19 +19436,6 @@ PartitionValuesRequest::PartitionValuesRequest(const PartitionValuesRequest& oth
   catName = other718.catName;
   validWriteIdList = other718.validWriteIdList;
   __isset = other718.__isset;
-}
-PartitionValuesRequest& PartitionValuesRequest::operator=(const PartitionValuesRequest& other719) {
-  dbName = other719.dbName;
-  tblName = other719.tblName;
-  partitionKeys = other719.partitionKeys;
-  applyDistinct = other719.applyDistinct;
-  filter = other719.filter;
-  partitionOrder = other719.partitionOrder;
-  ascending = other719.ascending;
-  maxParts = other719.maxParts;
-  catName = other719.catName;
-  validWriteIdList = other719.validWriteIdList;
-  __isset = other719.__isset;
   return *this;
 }
 void PartitionValuesRequest::printTo(std::ostream& out) const {
@@ -19496,14 +19495,14 @@ uint32_t PartitionValuesRow::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->row.clear();
-            uint32_t _size720;
-            ::apache::thrift::protocol::TType _etype723;
-            xfer += iprot->readListBegin(_etype723, _size720);
-            this->row.resize(_size720);
-            uint32_t _i724;
-            for (_i724 = 0; _i724 < _size720; ++_i724)
+            uint32_t _size719;
+            ::apache::thrift::protocol::TType _etype722;
+            xfer += iprot->readListBegin(_etype722, _size719);
+            this->row.resize(_size719);
+            uint32_t _i723;
+            for (_i723 = 0; _i723 < _size719; ++_i723)
             {
-              xfer += iprot->readString(this->row[_i724]);
+              xfer += iprot->readString(this->row[_i723]);
             }
             xfer += iprot->readListEnd();
           }
@@ -19534,10 +19533,10 @@ uint32_t PartitionValuesRow::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("row", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->row.size()));
-    std::vector<std::string> ::const_iterator _iter725;
-    for (_iter725 = this->row.begin(); _iter725 != this->row.end(); ++_iter725)
+    std::vector<std::string> ::const_iterator _iter724;
+    for (_iter724 = this->row.begin(); _iter724 != this->row.end(); ++_iter724)
     {
-      xfer += oprot->writeString((*_iter725));
+      xfer += oprot->writeString((*_iter724));
     }
     xfer += oprot->writeListEnd();
   }
@@ -19553,11 +19552,11 @@ void swap(PartitionValuesRow &a, PartitionValuesRow &b) {
   swap(a.row, b.row);
 }
 
-PartitionValuesRow::PartitionValuesRow(const PartitionValuesRow& other726) {
-  row = other726.row;
+PartitionValuesRow::PartitionValuesRow(const PartitionValuesRow& other725) {
+  row = other725.row;
 }
-PartitionValuesRow& PartitionValuesRow::operator=(const PartitionValuesRow& other727) {
-  row = other727.row;
+PartitionValuesRow& PartitionValuesRow::operator=(const PartitionValuesRow& other726) {
+  row = other726.row;
   return *this;
 }
 void PartitionValuesRow::printTo(std::ostream& out) const {
@@ -19608,14 +19607,14 @@ uint32_t PartitionValuesResponse::read(::apache::thrift::protocol::TProtocol* ip
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitionValues.clear();
-            uint32_t _size728;
-            ::apache::thrift::protocol::TType _etype731;
-            xfer += iprot->readListBegin(_etype731, _size728);
-            this->partitionValues.resize(_size728);
-            uint32_t _i732;
-            for (_i732 = 0; _i732 < _size728; ++_i732)
+            uint32_t _size727;
+            ::apache::thrift::protocol::TType _etype730;
+            xfer += iprot->readListBegin(_etype730, _size727);
+            this->partitionValues.resize(_size727);
+            uint32_t _i731;
+            for (_i731 = 0; _i731 < _size727; ++_i731)
             {
-              xfer += this->partitionValues[_i732].read(iprot);
+              xfer += this->partitionValues[_i731].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -19646,10 +19645,10 @@ uint32_t PartitionValuesResponse::write(::apache::thrift::protocol::TProtocol* o
   xfer += oprot->writeFieldBegin("partitionValues", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->partitionValues.size()));
-    std::vector<PartitionValuesRow> ::const_iterator _iter733;
-    for (_iter733 = this->partitionValues.begin(); _iter733 != this->partitionValues.end(); ++_iter733)
+    std::vector<PartitionValuesRow> ::const_iterator _iter732;
+    for (_iter732 = this->partitionValues.begin(); _iter732 != this->partitionValues.end(); ++_iter732)
     {
-      xfer += (*_iter733).write(oprot);
+      xfer += (*_iter732).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -19665,11 +19664,11 @@ void swap(PartitionValuesResponse &a, PartitionValuesResponse &b) {
   swap(a.partitionValues, b.partitionValues);
 }
 
-PartitionValuesResponse::PartitionValuesResponse(const PartitionValuesResponse& other734) {
-  partitionValues = other734.partitionValues;
+PartitionValuesResponse::PartitionValuesResponse(const PartitionValuesResponse& other733) {
+  partitionValues = other733.partitionValues;
 }
-PartitionValuesResponse& PartitionValuesResponse::operator=(const PartitionValuesResponse& other735) {
-  partitionValues = other735.partitionValues;
+PartitionValuesResponse& PartitionValuesResponse::operator=(const PartitionValuesResponse& other734) {
+  partitionValues = other734.partitionValues;
   return *this;
 }
 void PartitionValuesResponse::printTo(std::ostream& out) const {
@@ -19781,14 +19780,14 @@ uint32_t GetPartitionsByNamesRequest::read(::apache::thrift::protocol::TProtocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->names.clear();
-            uint32_t _size736;
-            ::apache::thrift::protocol::TType _etype739;
-            xfer += iprot->readListBegin(_etype739, _size736);
-            this->names.resize(_size736);
-            uint32_t _i740;
-            for (_i740 = 0; _i740 < _size736; ++_i740)
+            uint32_t _size735;
+            ::apache::thrift::protocol::TType _etype738;
+            xfer += iprot->readListBegin(_etype738, _size735);
+            this->names.resize(_size735);
+            uint32_t _i739;
+            for (_i739 = 0; _i739 < _size735; ++_i739)
             {
-              xfer += iprot->readString(this->names[_i740]);
+              xfer += iprot->readString(this->names[_i739]);
             }
             xfer += iprot->readListEnd();
           }
@@ -19809,14 +19808,14 @@ uint32_t GetPartitionsByNamesRequest::read(::apache::thrift::protocol::TProtocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->processorCapabilities.clear();
-            uint32_t _size741;
-            ::apache::thrift::protocol::TType _etype744;
-            xfer += iprot->readListBegin(_etype744, _size741);
-            this->processorCapabilities.resize(_size741);
-            uint32_t _i745;
-            for (_i745 = 0; _i745 < _size741; ++_i745)
+            uint32_t _size740;
+            ::apache::thrift::protocol::TType _etype743;
+            xfer += iprot->readListBegin(_etype743, _size740);
+            this->processorCapabilities.resize(_size740);
+            uint32_t _i744;
+            for (_i744 = 0; _i744 < _size740; ++_i744)
             {
-              xfer += iprot->readString(this->processorCapabilities[_i745]);
+              xfer += iprot->readString(this->processorCapabilities[_i744]);
             }
             xfer += iprot->readListEnd();
           }
@@ -19898,10 +19897,10 @@ uint32_t GetPartitionsByNamesRequest::write(::apache::thrift::protocol::TProtoco
     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 _iter746;
-      for (_iter746 = this->names.begin(); _iter746 != this->names.end(); ++_iter746)
+      std::vector<std::string> ::const_iterator _iter745;
+      for (_iter745 = this->names.begin(); _iter745 != this->names.end(); ++_iter745)
       {
-        xfer += oprot->writeString((*_iter746));
+        xfer += oprot->writeString((*_iter745));
       }
       xfer += oprot->writeListEnd();
     }
@@ -19916,10 +19915,10 @@ uint32_t GetPartitionsByNamesRequest::write(::apache::thrift::protocol::TProtoco
     xfer += oprot->writeFieldBegin("processorCapabilities", ::apache::thrift::protocol::T_LIST, 5);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->processorCapabilities.size()));
-      std::vector<std::string> ::const_iterator _iter747;
-      for (_iter747 = this->processorCapabilities.begin(); _iter747 != this->processorCapabilities.end(); ++_iter747)
+      std::vector<std::string> ::const_iterator _iter746;
+      for (_iter746 = this->processorCapabilities.begin(); _iter746 != this->processorCapabilities.end(); ++_iter746)
       {
-        xfer += oprot->writeString((*_iter747));
+        xfer += oprot->writeString((*_iter746));
       }
       xfer += oprot->writeListEnd();
     }
@@ -19970,7 +19969,20 @@ void swap(GetPartitionsByNamesRequest &a, GetPartitionsByNamesRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-GetPartitionsByNamesRequest::GetPartitionsByNamesRequest(const GetPartitionsByNamesRequest& other748) {
+GetPartitionsByNamesRequest::GetPartitionsByNamesRequest(const GetPartitionsByNamesRequest& other747) {
+  db_name = other747.db_name;
+  tbl_name = other747.tbl_name;
+  names = other747.names;
+  get_col_stats = other747.get_col_stats;
+  processorCapabilities = other747.processorCapabilities;
+  processorIdentifier = other747.processorIdentifier;
+  engine = other747.engine;
+  validWriteIdList = other747.validWriteIdList;
+  getFileMetadata = other747.getFileMetadata;
+  id = other747.id;
+  __isset = other747.__isset;
+}
+GetPartitionsByNamesRequest& GetPartitionsByNamesRequest::operator=(const GetPartitionsByNamesRequest& other748) {
   db_name = other748.db_name;
   tbl_name = other748.tbl_name;
   names = other748.names;
@@ -19982,19 +19994,6 @@ GetPartitionsByNamesRequest::GetPartitionsByNamesRequest(const GetPartitionsByNa
   getFileMetadata = other748.getFileMetadata;
   id = other748.id;
   __isset = other748.__isset;
-}
-GetPartitionsByNamesRequest& GetPartitionsByNamesRequest::operator=(const GetPartitionsByNamesRequest& other749) {
-  db_name = other749.db_name;
-  tbl_name = other749.tbl_name;
-  names = other749.names;
-  get_col_stats = other749.get_col_stats;
-  processorCapabilities = other749.processorCapabilities;
-  processorIdentifier = other749.processorIdentifier;
-  engine = other749.engine;
-  validWriteIdList = other749.validWriteIdList;
-  getFileMetadata = other749.getFileMetadata;
-  id = other749.id;
-  __isset = other749.__isset;
   return *this;
 }
 void GetPartitionsByNamesRequest::printTo(std::ostream& out) const {
@@ -20059,14 +20058,14 @@ uint32_t GetPartitionsByNamesResult::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitions.clear();
-            uint32_t _size750;
-            ::apache::thrift::protocol::TType _etype753;
-            xfer += iprot->readListBegin(_etype753, _size750);
-            this->partitions.resize(_size750);
-            uint32_t _i754;
-            for (_i754 = 0; _i754 < _size750; ++_i754)
+            uint32_t _size749;
+            ::apache::thrift::protocol::TType _etype752;
+            xfer += iprot->readListBegin(_etype752, _size749);
+            this->partitions.resize(_size749);
+            uint32_t _i753;
+            for (_i753 = 0; _i753 < _size749; ++_i753)
             {
-              xfer += this->partitions[_i754].read(iprot);
+              xfer += this->partitions[_i753].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -20105,10 +20104,10 @@ uint32_t GetPartitionsByNamesResult::write(::apache::thrift::protocol::TProtocol
   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 _iter755;
-    for (_iter755 = this->partitions.begin(); _iter755 != this->partitions.end(); ++_iter755)
+    std::vector<Partition> ::const_iterator _iter754;
+    for (_iter754 = this->partitions.begin(); _iter754 != this->partitions.end(); ++_iter754)
     {
-      xfer += (*_iter755).write(oprot);
+      xfer += (*_iter754).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -20131,15 +20130,15 @@ void swap(GetPartitionsByNamesResult &a, GetPartitionsByNamesResult &b) {
   swap(a.__isset, b.__isset);
 }
 
-GetPartitionsByNamesResult::GetPartitionsByNamesResult(const GetPartitionsByNamesResult& other756) {
+GetPartitionsByNamesResult::GetPartitionsByNamesResult(const GetPartitionsByNamesResult& other755) {
+  partitions = other755.partitions;
+  dictionary = other755.dictionary;
+  __isset = other755.__isset;
+}
+GetPartitionsByNamesResult& GetPartitionsByNamesResult::operator=(const GetPartitionsByNamesResult& other756) {
   partitions = other756.partitions;
   dictionary = other756.dictionary;
   __isset = other756.__isset;
-}
-GetPartitionsByNamesResult& GetPartitionsByNamesResult::operator=(const GetPartitionsByNamesResult& other757) {
-  partitions = other757.partitions;
-  dictionary = other757.dictionary;
-  __isset = other757.__isset;
   return *this;
 }
 void GetPartitionsByNamesResult::printTo(std::ostream& out) const {
@@ -20255,17 +20254,17 @@ uint32_t DataConnector::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->parameters.clear();
-            uint32_t _size758;
-            ::apache::thrift::protocol::TType _ktype759;
-            ::apache::thrift::protocol::TType _vtype760;
-            xfer += iprot->readMapBegin(_ktype759, _vtype760, _size758);
-            uint32_t _i762;
-            for (_i762 = 0; _i762 < _size758; ++_i762)
+            uint32_t _size757;
+            ::apache::thrift::protocol::TType _ktype758;
+            ::apache::thrift::protocol::TType _vtype759;
+            xfer += iprot->readMapBegin(_ktype758, _vtype759, _size757);
+            uint32_t _i761;
+            for (_i761 = 0; _i761 < _size757; ++_i761)
             {
-              std::string _key763;
-              xfer += iprot->readString(_key763);
-              std::string& _val764 = this->parameters[_key763];
-              xfer += iprot->readString(_val764);
+              std::string _key762;
+              xfer += iprot->readString(_key762);
+              std::string& _val763 = this->parameters[_key762];
+              xfer += iprot->readString(_val763);
             }
             xfer += iprot->readMapEnd();
           }
@@ -20284,9 +20283,9 @@ uint32_t DataConnector::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 7:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast765;
-          xfer += iprot->readI32(ecast765);
-          this->ownerType = (PrincipalType::type)ecast765;
+          int32_t ecast764;
+          xfer += iprot->readI32(ecast764);
+          this->ownerType = (PrincipalType::type)ecast764;
           this->__isset.ownerType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -20338,11 +20337,11 @@ uint32_t DataConnector::write(::apache::thrift::protocol::TProtocol* oprot) cons
     xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 5);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->parameters.size()));
-      std::map<std::string, std::string> ::const_iterator _iter766;
-      for (_iter766 = this->parameters.begin(); _iter766 != this->parameters.end(); ++_iter766)
+      std::map<std::string, std::string> ::const_iterator _iter765;
+      for (_iter765 = this->parameters.begin(); _iter765 != this->parameters.end(); ++_iter765)
       {
-        xfer += oprot->writeString(_iter766->first);
-        xfer += oprot->writeString(_iter766->second);
+        xfer += oprot->writeString(_iter765->first);
+        xfer += oprot->writeString(_iter765->second);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -20381,7 +20380,18 @@ void swap(DataConnector &a, DataConnector &b) {
   swap(a.__isset, b.__isset);
 }
 
-DataConnector::DataConnector(const DataConnector& other767) {
+DataConnector::DataConnector(const DataConnector& other766) {
+  name = other766.name;
+  type = other766.type;
+  url = other766.url;
+  description = other766.description;
+  parameters = other766.parameters;
+  ownerName = other766.ownerName;
+  ownerType = other766.ownerType;
+  createTime = other766.createTime;
+  __isset = other766.__isset;
+}
+DataConnector& DataConnector::operator=(const DataConnector& other767) {
   name = other767.name;
   type = other767.type;
   url = other767.url;
@@ -20391,17 +20401,6 @@ DataConnector::DataConnector(const DataConnector& other767) {
   ownerType = other767.ownerType;
   createTime = other767.createTime;
   __isset = other767.__isset;
-}
-DataConnector& DataConnector::operator=(const DataConnector& other768) {
-  name = other768.name;
-  type = other768.type;
-  url = other768.url;
-  description = other768.description;
-  parameters = other768.parameters;
-  ownerName = other768.ownerName;
-  ownerType = other768.ownerType;
-  createTime = other768.createTime;
-  __isset = other768.__isset;
   return *this;
 }
 void DataConnector::printTo(std::ostream& out) const {
@@ -20460,9 +20459,9 @@ uint32_t ResourceUri::read(::apache::thrift::protocol::TProtocol* iprot) {
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast769;
-          xfer += iprot->readI32(ecast769);
-          this->resourceType = (ResourceType::type)ecast769;
+          int32_t ecast768;
+          xfer += iprot->readI32(ecast768);
+          this->resourceType = (ResourceType::type)ecast768;
           this->__isset.resourceType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -20513,15 +20512,15 @@ void swap(ResourceUri &a, ResourceUri &b) {
   swap(a.__isset, b.__isset);
 }
 
-ResourceUri::ResourceUri(const ResourceUri& other770) {
+ResourceUri::ResourceUri(const ResourceUri& other769) {
+  resourceType = other769.resourceType;
+  uri = other769.uri;
+  __isset = other769.__isset;
+}
+ResourceUri& ResourceUri::operator=(const ResourceUri& other770) {
   resourceType = other770.resourceType;
   uri = other770.uri;
   __isset = other770.__isset;
-}
-ResourceUri& ResourceUri::operator=(const ResourceUri& other771) {
-  resourceType = other771.resourceType;
-  uri = other771.uri;
-  __isset = other771.__isset;
   return *this;
 }
 void ResourceUri::printTo(std::ostream& out) const {
@@ -20635,9 +20634,9 @@ uint32_t Function::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 5:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast772;
-          xfer += iprot->readI32(ecast772);
-          this->ownerType = (PrincipalType::type)ecast772;
+          int32_t ecast771;
+          xfer += iprot->readI32(ecast771);
+          this->ownerType = (PrincipalType::type)ecast771;
           this->__isset.ownerType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -20653,9 +20652,9 @@ uint32_t Function::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 7:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast773;
-          xfer += iprot->readI32(ecast773);
-          this->functionType = (FunctionType::type)ecast773;
+          int32_t ecast772;
+          xfer += iprot->readI32(ecast772);
+          this->functionType = (FunctionType::type)ecast772;
           this->__isset.functionType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -20665,14 +20664,14 @@ uint32_t Function::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->resourceUris.clear();
-            uint32_t _size774;
-            ::apache::thrift::protocol::TType _etype777;
-            xfer += iprot->readListBegin(_etype777, _size774);
-            this->resourceUris.resize(_size774);
-            uint32_t _i778;
-            for (_i778 = 0; _i778 < _size774; ++_i778)
+            uint32_t _size773;
+            ::apache::thrift::protocol::TType _etype776;
+            xfer += iprot->readListBegin(_etype776, _size773);
+            this->resourceUris.resize(_size773);
+            uint32_t _i777;
+            for (_i777 = 0; _i777 < _size773; ++_i777)
             {
-              xfer += this->resourceUris[_i778].read(iprot);
+              xfer += this->resourceUris[_i777].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -20737,10 +20736,10 @@ uint32_t Function::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("resourceUris", ::apache::thrift::protocol::T_LIST, 8);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->resourceUris.size()));
-    std::vector<ResourceUri> ::const_iterator _iter779;
-    for (_iter779 = this->resourceUris.begin(); _iter779 != this->resourceUris.end(); ++_iter779)
+    std::vector<ResourceUri> ::const_iterator _iter778;
+    for (_iter778 = this->resourceUris.begin(); _iter778 != this->resourceUris.end(); ++_iter778)
     {
-      xfer += (*_iter779).write(oprot);
+      xfer += (*_iter778).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -20770,7 +20769,19 @@ void swap(Function &a, Function &b) {
   swap(a.__isset, b.__isset);
 }
 
-Function::Function(const Function& other780) {
+Function::Function(const Function& other779) {
+  functionName = other779.functionName;
+  dbName = other779.dbName;
+  className = other779.className;
+  ownerName = other779.ownerName;
+  ownerType = other779.ownerType;
+  createTime = other779.createTime;
+  functionType = other779.functionType;
+  resourceUris = other779.resourceUris;
+  catName = other779.catName;
+  __isset = other779.__isset;
+}
+Function& Function::operator=(const Function& other780) {
   functionName = other780.functionName;
   dbName = other780.dbName;
   className = other780.className;
@@ -20781,18 +20792,6 @@ Function::Function(const Function& other780) {
   resourceUris = other780.resourceUris;
   catName = other780.catName;
   __isset = other780.__isset;
-}
-Function& Function::operator=(const Function& other781) {
-  functionName = other781.functionName;
-  dbName = other781.dbName;
-  className = other781.className;
-  ownerName = other781.ownerName;
-  ownerType = other781.ownerType;
-  createTime = other781.createTime;
-  functionType = other781.functionType;
-  resourceUris = other781.resourceUris;
-  catName = other781.catName;
-  __isset = other781.__isset;
   return *this;
 }
 void Function::printTo(std::ostream& out) const {
@@ -20897,9 +20896,9 @@ uint32_t TxnInfo::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast782;
-          xfer += iprot->readI32(ecast782);
-          this->state = (TxnState::type)ecast782;
+          int32_t ecast781;
+          xfer += iprot->readI32(ecast781);
+          this->state = (TxnState::type)ecast781;
           isset_state = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -21046,7 +21045,19 @@ void swap(TxnInfo &a, TxnInfo &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnInfo::TxnInfo(const TxnInfo& other783) {
+TxnInfo::TxnInfo(const TxnInfo& other782) {
+  id = other782.id;
+  state = other782.state;
+  user = other782.user;
+  hostname = other782.hostname;
+  agentInfo = other782.agentInfo;
+  heartbeatCount = other782.heartbeatCount;
+  metaInfo = other782.metaInfo;
+  startedTime = other782.startedTime;
+  lastHeartbeatTime = other782.lastHeartbeatTime;
+  __isset = other782.__isset;
+}
+TxnInfo& TxnInfo::operator=(const TxnInfo& other783) {
   id = other783.id;
   state = other783.state;
   user = other783.user;
@@ -21057,18 +21068,6 @@ TxnInfo::TxnInfo(const TxnInfo& other783) {
   startedTime = other783.startedTime;
   lastHeartbeatTime = other783.lastHeartbeatTime;
   __isset = other783.__isset;
-}
-TxnInfo& TxnInfo::operator=(const TxnInfo& other784) {
-  id = other784.id;
-  state = other784.state;
-  user = other784.user;
-  hostname = other784.hostname;
-  agentInfo = other784.agentInfo;
-  heartbeatCount = other784.heartbeatCount;
-  metaInfo = other784.metaInfo;
-  startedTime = other784.startedTime;
-  lastHeartbeatTime = other784.lastHeartbeatTime;
-  __isset = other784.__isset;
   return *this;
 }
 void TxnInfo::printTo(std::ostream& out) const {
@@ -21140,14 +21139,14 @@ uint32_t GetOpenTxnsInfoResponse::read(::apache::thrift::protocol::TProtocol* ip
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->open_txns.clear();
-            uint32_t _size785;
-            ::apache::thrift::protocol::TType _etype788;
-            xfer += iprot->readListBegin(_etype788, _size785);
-            this->open_txns.resize(_size785);
-            uint32_t _i789;
-            for (_i789 = 0; _i789 < _size785; ++_i789)
+            uint32_t _size784;
+            ::apache::thrift::protocol::TType _etype787;
+            xfer += iprot->readListBegin(_etype787, _size784);
+            this->open_txns.resize(_size784);
+            uint32_t _i788;
+            for (_i788 = 0; _i788 < _size784; ++_i788)
             {
-              xfer += this->open_txns[_i789].read(iprot);
+              xfer += this->open_txns[_i788].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -21184,10 +21183,10 @@ uint32_t GetOpenTxnsInfoResponse::write(::apache::thrift::protocol::TProtocol* o
   xfer += oprot->writeFieldBegin("open_txns", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->open_txns.size()));
-    std::vector<TxnInfo> ::const_iterator _iter790;
-    for (_iter790 = this->open_txns.begin(); _iter790 != this->open_txns.end(); ++_iter790)
+    std::vector<TxnInfo> ::const_iterator _iter789;
+    for (_iter789 = this->open_txns.begin(); _iter789 != this->open_txns.end(); ++_iter789)
     {
-      xfer += (*_iter790).write(oprot);
+      xfer += (*_iter789).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -21204,13 +21203,13 @@ void swap(GetOpenTxnsInfoResponse &a, GetOpenTxnsInfoResponse &b) {
   swap(a.open_txns, b.open_txns);
 }
 
-GetOpenTxnsInfoResponse::GetOpenTxnsInfoResponse(const GetOpenTxnsInfoResponse& other791) {
+GetOpenTxnsInfoResponse::GetOpenTxnsInfoResponse(const GetOpenTxnsInfoResponse& other790) {
+  txn_high_water_mark = other790.txn_high_water_mark;
+  open_txns = other790.open_txns;
+}
+GetOpenTxnsInfoResponse& GetOpenTxnsInfoResponse::operator=(const GetOpenTxnsInfoResponse& other791) {
   txn_high_water_mark = other791.txn_high_water_mark;
   open_txns = other791.open_txns;
-}
-GetOpenTxnsInfoResponse& GetOpenTxnsInfoResponse::operator=(const GetOpenTxnsInfoResponse& other792) {
-  txn_high_water_mark = other792.txn_high_water_mark;
-  open_txns = other792.open_txns;
   return *this;
 }
 void GetOpenTxnsInfoResponse::printTo(std::ostream& out) const {
@@ -21285,14 +21284,14 @@ uint32_t GetOpenTxnsResponse::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->open_txns.clear();
-            uint32_t _size793;
-            ::apache::thrift::protocol::TType _etype796;
-            xfer += iprot->readListBegin(_etype796, _size793);
-            this->open_txns.resize(_size793);
-            uint32_t _i797;
-            for (_i797 = 0; _i797 < _size793; ++_i797)
+            uint32_t _size792;
+            ::apache::thrift::protocol::TType _etype795;
+            xfer += iprot->readListBegin(_etype795, _size792);
+            this->open_txns.resize(_size792);
+            uint32_t _i796;
+            for (_i796 = 0; _i796 < _size792; ++_i796)
             {
-              xfer += iprot->readI64(this->open_txns[_i797]);
+              xfer += iprot->readI64(this->open_txns[_i796]);
             }
             xfer += iprot->readListEnd();
           }
@@ -21347,10 +21346,10 @@ uint32_t GetOpenTxnsResponse::write(::apache::thrift::protocol::TProtocol* oprot
   xfer += oprot->writeFieldBegin("open_txns", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->open_txns.size()));
-    std::vector<int64_t> ::const_iterator _iter798;
-    for (_iter798 = this->open_txns.begin(); _iter798 != this->open_txns.end(); ++_iter798)
+    std::vector<int64_t> ::const_iterator _iter797;
+    for (_iter797 = this->open_txns.begin(); _iter797 != this->open_txns.end(); ++_iter797)
     {
-      xfer += oprot->writeI64((*_iter798));
+      xfer += oprot->writeI64((*_iter797));
     }
     xfer += oprot->writeListEnd();
   }
@@ -21379,19 +21378,19 @@ void swap(GetOpenTxnsResponse &a, GetOpenTxnsResponse &b) {
   swap(a.__isset, b.__isset);
 }
 
-GetOpenTxnsResponse::GetOpenTxnsResponse(const GetOpenTxnsResponse& other799) {
+GetOpenTxnsResponse::GetOpenTxnsResponse(const GetOpenTxnsResponse& other798) {
+  txn_high_water_mark = other798.txn_high_water_mark;
+  open_txns = other798.open_txns;
+  min_open_txn = other798.min_open_txn;
+  abortedBits = other798.abortedBits;
+  __isset = other798.__isset;
+}
+GetOpenTxnsResponse& GetOpenTxnsResponse::operator=(const GetOpenTxnsResponse& other799) {
   txn_high_water_mark = other799.txn_high_water_mark;
   open_txns = other799.open_txns;
   min_open_txn = other799.min_open_txn;
   abortedBits = other799.abortedBits;
   __isset = other799.__isset;
-}
-GetOpenTxnsResponse& GetOpenTxnsResponse::operator=(const GetOpenTxnsResponse& other800) {
-  txn_high_water_mark = other800.txn_high_water_mark;
-  open_txns = other800.open_txns;
-  min_open_txn = other800.min_open_txn;
-  abortedBits = other800.abortedBits;
-  __isset = other800.__isset;
   return *this;
 }
 void GetOpenTxnsResponse::printTo(std::ostream& out) const {
@@ -21515,14 +21514,14 @@ uint32_t OpenTxnRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->replSrcTxnIds.clear();
-            uint32_t _size801;
-            ::apache::thrift::protocol::TType _etype804;
-            xfer += iprot->readListBegin(_etype804, _size801);
-            this->replSrcTxnIds.resize(_size801);
-            uint32_t _i805;
-            for (_i805 = 0; _i805 < _size801; ++_i805)
+            uint32_t _size800;
+            ::apache::thrift::protocol::TType _etype803;
+            xfer += iprot->readListBegin(_etype803, _size800);
+            this->replSrcTxnIds.resize(_size800);
+            uint32_t _i804;
+            for (_i804 = 0; _i804 < _size800; ++_i804)
             {
-              xfer += iprot->readI64(this->replSrcTxnIds[_i805]);
+              xfer += iprot->readI64(this->replSrcTxnIds[_i804]);
             }
             xfer += iprot->readListEnd();
           }
@@ -21533,9 +21532,9 @@ uint32_t OpenTxnRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 7:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast806;
-          xfer += iprot->readI32(ecast806);
-          this->txn_type = (TxnType::type)ecast806;
+          int32_t ecast805;
+          xfer += iprot->readI32(ecast805);
+          this->txn_type = (TxnType::type)ecast805;
           this->__isset.txn_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -21590,10 +21589,10 @@ uint32_t OpenTxnRequest::write(::apache::thrift::protocol::TProtocol* oprot) con
     xfer += oprot->writeFieldBegin("replSrcTxnIds", ::apache::thrift::protocol::T_LIST, 6);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->replSrcTxnIds.size()));
-      std::vector<int64_t> ::const_iterator _iter807;
-      for (_iter807 = this->replSrcTxnIds.begin(); _iter807 != this->replSrcTxnIds.end(); ++_iter807)
+      std::vector<int64_t> ::const_iterator _iter806;
+      for (_iter806 = this->replSrcTxnIds.begin(); _iter806 != this->replSrcTxnIds.end(); ++_iter806)
       {
-        xfer += oprot->writeI64((*_iter807));
+        xfer += oprot->writeI64((*_iter806));
       }
       xfer += oprot->writeListEnd();
     }
@@ -21621,7 +21620,17 @@ void swap(OpenTxnRequest &a, OpenTxnRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-OpenTxnRequest::OpenTxnRequest(const OpenTxnRequest& other808) {
+OpenTxnRequest::OpenTxnRequest(const OpenTxnRequest& other807) {
+  num_txns = other807.num_txns;
+  user = other807.user;
+  hostname = other807.hostname;
+  agentInfo = other807.agentInfo;
+  replPolicy = other807.replPolicy;
+  replSrcTxnIds = other807.replSrcTxnIds;
+  txn_type = other807.txn_type;
+  __isset = other807.__isset;
+}
+OpenTxnRequest& OpenTxnRequest::operator=(const OpenTxnRequest& other808) {
   num_txns = other808.num_txns;
   user = other808.user;
   hostname = other808.hostname;
@@ -21630,16 +21639,6 @@ OpenTxnRequest::OpenTxnRequest(const OpenTxnRequest& other808) {
   replSrcTxnIds = other808.replSrcTxnIds;
   txn_type = other808.txn_type;
   __isset = other808.__isset;
-}
-OpenTxnRequest& OpenTxnRequest::operator=(const OpenTxnRequest& other809) {
-  num_txns = other809.num_txns;
-  user = other809.user;
-  hostname = other809.hostname;
-  agentInfo = other809.agentInfo;
-  replPolicy = other809.replPolicy;
-  replSrcTxnIds = other809.replSrcTxnIds;
-  txn_type = other809.txn_type;
-  __isset = other809.__isset;
   return *this;
 }
 void OpenTxnRequest::printTo(std::ostream& out) const {
@@ -21696,14 +21695,14 @@ uint32_t OpenTxnsResponse::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->txn_ids.clear();
-            uint32_t _size810;
-            ::apache::thrift::protocol::TType _etype813;
-            xfer += iprot->readListBegin(_etype813, _size810);
-            this->txn_ids.resize(_size810);
-            uint32_t _i814;
-            for (_i814 = 0; _i814 < _size810; ++_i814)
+            uint32_t _size809;
+            ::apache::thrift::protocol::TType _etype812;
+            xfer += iprot->readListBegin(_etype812, _size809);
+            this->txn_ids.resize(_size809);
+            uint32_t _i813;
+            for (_i813 = 0; _i813 < _size809; ++_i813)
             {
-              xfer += iprot->readI64(this->txn_ids[_i814]);
+              xfer += iprot->readI64(this->txn_ids[_i813]);
             }
             xfer += iprot->readListEnd();
           }
@@ -21734,10 +21733,10 @@ uint32_t OpenTxnsResponse::write(::apache::thrift::protocol::TProtocol* oprot) c
   xfer += oprot->writeFieldBegin("txn_ids", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->txn_ids.size()));
-    std::vector<int64_t> ::const_iterator _iter815;
-    for (_iter815 = this->txn_ids.begin(); _iter815 != this->txn_ids.end(); ++_iter815)
+    std::vector<int64_t> ::const_iterator _iter814;
+    for (_iter814 = this->txn_ids.begin(); _iter814 != this->txn_ids.end(); ++_iter814)
     {
-      xfer += oprot->writeI64((*_iter815));
+      xfer += oprot->writeI64((*_iter814));
     }
     xfer += oprot->writeListEnd();
   }
@@ -21753,11 +21752,11 @@ void swap(OpenTxnsResponse &a, OpenTxnsResponse &b) {
   swap(a.txn_ids, b.txn_ids);
 }
 
-OpenTxnsResponse::OpenTxnsResponse(const OpenTxnsResponse& other816) {
-  txn_ids = other816.txn_ids;
+OpenTxnsResponse::OpenTxnsResponse(const OpenTxnsResponse& other815) {
+  txn_ids = other815.txn_ids;
 }
-OpenTxnsResponse& OpenTxnsResponse::operator=(const OpenTxnsResponse& other817) {
-  txn_ids = other817.txn_ids;
+OpenTxnsResponse& OpenTxnsResponse::operator=(const OpenTxnsResponse& other816) {
+  txn_ids = other816.txn_ids;
   return *this;
 }
 void OpenTxnsResponse::printTo(std::ostream& out) const {
@@ -21832,9 +21831,9 @@ uint32_t AbortTxnRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 3:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast818;
-          xfer += iprot->readI32(ecast818);
-          this->txn_type = (TxnType::type)ecast818;
+          int32_t ecast817;
+          xfer += iprot->readI32(ecast817);
+          this->txn_type = (TxnType::type)ecast817;
           this->__isset.txn_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -21886,17 +21885,17 @@ void swap(AbortTxnRequest &a, AbortTxnRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-AbortTxnRequest::AbortTxnRequest(const AbortTxnRequest& other819) {
+AbortTxnRequest::AbortTxnRequest(const AbortTxnRequest& other818) {
+  txnid = other818.txnid;
+  replPolicy = other818.replPolicy;
+  txn_type = other818.txn_type;
+  __isset = other818.__isset;
+}
+AbortTxnRequest& AbortTxnRequest::operator=(const AbortTxnRequest& other819) {
   txnid = other819.txnid;
   replPolicy = other819.replPolicy;
   txn_type = other819.txn_type;
   __isset = other819.__isset;
-}
-AbortTxnRequest& AbortTxnRequest::operator=(const AbortTxnRequest& other820) {
-  txnid = other820.txnid;
-  replPolicy = other820.replPolicy;
-  txn_type = other820.txn_type;
-  __isset = other820.__isset;
   return *this;
 }
 void AbortTxnRequest::printTo(std::ostream& out) const {
@@ -21949,14 +21948,14 @@ uint32_t AbortTxnsRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->txn_ids.clear();
-            uint32_t _size821;
-            ::apache::thrift::protocol::TType _etype824;
-            xfer += iprot->readListBegin(_etype824, _size821);
-            this->txn_ids.resize(_size821);
-            uint32_t _i825;
-            for (_i825 = 0; _i825 < _size821; ++_i825)
+            uint32_t _size820;
+            ::apache::thrift::protocol::TType _etype823;
+            xfer += iprot->readListBegin(_etype823, _size820);
+            this->txn_ids.resize(_size820);
+            uint32_t _i824;
+            for (_i824 = 0; _i824 < _size820; ++_i824)
             {
-              xfer += iprot->readI64(this->txn_ids[_i825]);
+              xfer += iprot->readI64(this->txn_ids[_i824]);
             }
             xfer += iprot->readListEnd();
           }
@@ -21987,10 +21986,10 @@ uint32_t AbortTxnsRequest::write(::apache::thrift::protocol::TProtocol* oprot) c
   xfer += oprot->writeFieldBegin("txn_ids", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->txn_ids.size()));
-    std::vector<int64_t> ::const_iterator _iter826;
-    for (_iter826 = this->txn_ids.begin(); _iter826 != this->txn_ids.end(); ++_iter826)
+    std::vector<int64_t> ::const_iterator _iter825;
+    for (_iter825 = this->txn_ids.begin(); _iter825 != this->txn_ids.end(); ++_iter825)
     {
-      xfer += oprot->writeI64((*_iter826));
+      xfer += oprot->writeI64((*_iter825));
     }
     xfer += oprot->writeListEnd();
   }
@@ -22006,11 +22005,11 @@ void swap(AbortTxnsRequest &a, AbortTxnsRequest &b) {
   swap(a.txn_ids, b.txn_ids);
 }
 
-AbortTxnsRequest::AbortTxnsRequest(const AbortTxnsRequest& other827) {
-  txn_ids = other827.txn_ids;
+AbortTxnsRequest::AbortTxnsRequest(const AbortTxnsRequest& other826) {
+  txn_ids = other826.txn_ids;
 }
-AbortTxnsRequest& AbortTxnsRequest::operator=(const AbortTxnsRequest& other828) {
-  txn_ids = other828.txn_ids;
+AbortTxnsRequest& AbortTxnsRequest::operator=(const AbortTxnsRequest& other827) {
+  txn_ids = other827.txn_ids;
   return *this;
 }
 void AbortTxnsRequest::printTo(std::ostream& out) const {
@@ -22138,15 +22137,15 @@ void swap(CommitTxnKeyValue &a, CommitTxnKeyValue &b) {
   swap(a.value, b.value);
 }
 
-CommitTxnKeyValue::CommitTxnKeyValue(const CommitTxnKeyValue& other829) {
+CommitTxnKeyValue::CommitTxnKeyValue(const CommitTxnKeyValue& other828) {
+  tableId = other828.tableId;
+  key = other828.key;
+  value = other828.value;
+}
+CommitTxnKeyValue& CommitTxnKeyValue::operator=(const CommitTxnKeyValue& other829) {
   tableId = other829.tableId;
   key = other829.key;
   value = other829.value;
-}
-CommitTxnKeyValue& CommitTxnKeyValue::operator=(const CommitTxnKeyValue& other830) {
-  tableId = other830.tableId;
-  key = other830.key;
-  value = other830.value;
   return *this;
 }
 void CommitTxnKeyValue::printTo(std::ostream& out) const {
@@ -22354,7 +22353,17 @@ void swap(WriteEventInfo &a, WriteEventInfo &b) {
   swap(a.__isset, b.__isset);
 }
 
-WriteEventInfo::WriteEventInfo(const WriteEventInfo& other831) {
+WriteEventInfo::WriteEventInfo(const WriteEventInfo& other830) {
+  writeId = other830.writeId;
+  database = other830.database;
+  table = other830.table;
+  files = other830.files;
+  partition = other830.partition;
+  tableObj = other830.tableObj;
+  partitionObj = other830.partitionObj;
+  __isset = other830.__isset;
+}
+WriteEventInfo& WriteEventInfo::operator=(const WriteEventInfo& other831) {
   writeId = other831.writeId;
   database = other831.database;
   table = other831.table;
@@ -22363,16 +22372,6 @@ WriteEventInfo::WriteEventInfo(const WriteEventInfo& other831) {
   tableObj = other831.tableObj;
   partitionObj = other831.partitionObj;
   __isset = other831.__isset;
-}
-WriteEventInfo& WriteEventInfo::operator=(const WriteEventInfo& other832) {
-  writeId = other832.writeId;
-  database = other832.database;
-  table = other832.table;
-  files = other832.files;
-  partition = other832.partition;
-  tableObj = other832.tableObj;
-  partitionObj = other832.partitionObj;
-  __isset = other832.__isset;
   return *this;
 }
 void WriteEventInfo::printTo(std::ostream& out) const {
@@ -22481,14 +22480,14 @@ uint32_t ReplLastIdInfo::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitionList.clear();
-            uint32_t _size833;
-            ::apache::thrift::protocol::TType _etype836;
-            xfer += iprot->readListBegin(_etype836, _size833);
-            this->partitionList.resize(_size833);
-            uint32_t _i837;
-            for (_i837 = 0; _i837 < _size833; ++_i837)
+            uint32_t _size832;
+            ::apache::thrift::protocol::TType _etype835;
+            xfer += iprot->readListBegin(_etype835, _size832);
+            this->partitionList.resize(_size832);
+            uint32_t _i836;
+            for (_i836 = 0; _i836 < _size832; ++_i836)
             {
-              xfer += iprot->readString(this->partitionList[_i837]);
+              xfer += iprot->readString(this->partitionList[_i836]);
             }
             xfer += iprot->readListEnd();
           }
@@ -22540,10 +22539,10 @@ uint32_t ReplLastIdInfo::write(::apache::thrift::protocol::TProtocol* oprot) con
     xfer += oprot->writeFieldBegin("partitionList", ::apache::thrift::protocol::T_LIST, 5);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionList.size()));
-      std::vector<std::string> ::const_iterator _iter838;
-      for (_iter838 = this->partitionList.begin(); _iter838 != this->partitionList.end(); ++_iter838)
+      std::vector<std::string> ::const_iterator _iter837;
+      for (_iter837 = this->partitionList.begin(); _iter837 != this->partitionList.end(); ++_iter837)
       {
-        xfer += oprot->writeString((*_iter838));
+        xfer += oprot->writeString((*_iter837));
       }
       xfer += oprot->writeListEnd();
     }
@@ -22564,21 +22563,21 @@ void swap(ReplLastIdInfo &a, ReplLastIdInfo &b) {
   swap(a.__isset, b.__isset);
 }
 
-ReplLastIdInfo::ReplLastIdInfo(const ReplLastIdInfo& other839) {
+ReplLastIdInfo::ReplLastIdInfo(const ReplLastIdInfo& other838) {
+  database = other838.database;
+  lastReplId = other838.lastReplId;
+  table = other838.table;
+  catalog = other838.catalog;
+  partitionList = other838.partitionList;
+  __isset = other838.__isset;
+}
+ReplLastIdInfo& ReplLastIdInfo::operator=(const ReplLastIdInfo& other839) {
   database = other839.database;
   lastReplId = other839.lastReplId;
   table = other839.table;
   catalog = other839.catalog;
   partitionList = other839.partitionList;
   __isset = other839.__isset;
-}
-ReplLastIdInfo& ReplLastIdInfo::operator=(const ReplLastIdInfo& other840) {
-  database = other840.database;
-  lastReplId = other840.lastReplId;
-  table = other840.table;
-  catalog = other840.catalog;
-  partitionList = other840.partitionList;
-  __isset = other840.__isset;
   return *this;
 }
 void ReplLastIdInfo::printTo(std::ostream& out) const {
@@ -22730,17 +22729,17 @@ void swap(UpdateTransactionalStatsRequest &a, UpdateTransactionalStatsRequest &b
   swap(a.deletedCount, b.deletedCount);
 }
 
-UpdateTransactionalStatsRequest::UpdateTransactionalStatsRequest(const UpdateTransactionalStatsRequest& other841) {
+UpdateTransactionalStatsRequest::UpdateTransactionalStatsRequest(const UpdateTransactionalStatsRequest& other840) {
+  tableId = other840.tableId;
+  insertCount = other840.insertCount;
+  updatedCount = other840.updatedCount;
+  deletedCount = other840.deletedCount;
+}
+UpdateTransactionalStatsRequest& UpdateTransactionalStatsRequest::operator=(const UpdateTransactionalStatsRequest& other841) {
   tableId = other841.tableId;
   insertCount = other841.insertCount;
   updatedCount = other841.updatedCount;
   deletedCount = other841.deletedCount;
-}
-UpdateTransactionalStatsRequest& UpdateTransactionalStatsRequest::operator=(const UpdateTransactionalStatsRequest& other842) {
-  tableId = other842.tableId;
-  insertCount = other842.insertCount;
-  updatedCount = other842.updatedCount;
-  deletedCount = other842.deletedCount;
   return *this;
 }
 void UpdateTransactionalStatsRequest::printTo(std::ostream& out) const {
@@ -22840,14 +22839,14 @@ uint32_t CommitTxnRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->writeEventInfos.clear();
-            uint32_t _size843;
-            ::apache::thrift::protocol::TType _etype846;
-            xfer += iprot->readListBegin(_etype846, _size843);
-            this->writeEventInfos.resize(_size843);
-            uint32_t _i847;
-            for (_i847 = 0; _i847 < _size843; ++_i847)
+            uint32_t _size842;
+            ::apache::thrift::protocol::TType _etype845;
+            xfer += iprot->readListBegin(_etype845, _size842);
+            this->writeEventInfos.resize(_size842);
+            uint32_t _i846;
+            for (_i846 = 0; _i846 < _size842; ++_i846)
             {
-              xfer += this->writeEventInfos[_i847].read(iprot);
+              xfer += this->writeEventInfos[_i846].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -22882,9 +22881,9 @@ uint32_t CommitTxnRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 7:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast848;
-          xfer += iprot->readI32(ecast848);
-          this->txn_type = (TxnType::type)ecast848;
+          int32_t ecast847;
+          xfer += iprot->readI32(ecast847);
+          this->txn_type = (TxnType::type)ecast847;
           this->__isset.txn_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -22922,10 +22921,10 @@ uint32_t CommitTxnRequest::write(::apache::thrift::protocol::TProtocol* oprot) c
     xfer += oprot->writeFieldBegin("writeEventInfos", ::apache::thrift::protocol::T_LIST, 3);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->writeEventInfos.size()));
-      std::vector<WriteEventInfo> ::const_iterator _iter849;
-      for (_iter849 = this->writeEventInfos.begin(); _iter849 != this->writeEventInfos.end(); ++_iter849)
+      std::vector<WriteEventInfo> ::const_iterator _iter848;
+      for (_iter848 = this->writeEventInfos.begin(); _iter848 != this->writeEventInfos.end(); ++_iter848)
       {
-        xfer += (*_iter849).write(oprot);
+        xfer += (*_iter848).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -22968,7 +22967,17 @@ void swap(CommitTxnRequest &a, CommitTxnRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-CommitTxnRequest::CommitTxnRequest(const CommitTxnRequest& other850) {
+CommitTxnRequest::CommitTxnRequest(const CommitTxnRequest& other849) {
+  txnid = other849.txnid;
+  replPolicy = other849.replPolicy;
+  writeEventInfos = other849.writeEventInfos;
+  replLastIdInfo = other849.replLastIdInfo;
+  keyValue = other849.keyValue;
+  exclWriteEnabled = other849.exclWriteEnabled;
+  txn_type = other849.txn_type;
+  __isset = other849.__isset;
+}
+CommitTxnRequest& CommitTxnRequest::operator=(const CommitTxnRequest& other850) {
   txnid = other850.txnid;
   replPolicy = other850.replPolicy;
   writeEventInfos = other850.writeEventInfos;
@@ -22977,16 +22986,6 @@ CommitTxnRequest::CommitTxnRequest(const CommitTxnRequest& other850) {
   exclWriteEnabled = other850.exclWriteEnabled;
   txn_type = other850.txn_type;
   __isset = other850.__isset;
-}
-CommitTxnRequest& CommitTxnRequest::operator=(const CommitTxnRequest& other851) {
-  txnid = other851.txnid;
-  replPolicy = other851.replPolicy;
-  writeEventInfos = other851.writeEventInfos;
-  replLastIdInfo = other851.replLastIdInfo;
-  keyValue = other851.keyValue;
-  exclWriteEnabled = other851.exclWriteEnabled;
-  txn_type = other851.txn_type;
-  __isset = other851.__isset;
   return *this;
 }
 void CommitTxnRequest::printTo(std::ostream& out) const {
@@ -23108,14 +23107,14 @@ uint32_t ReplTblWriteIdStateRequest::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partNames.clear();
-            uint32_t _size852;
-            ::apache::thrift::protocol::TType _etype855;
-            xfer += iprot->readListBegin(_etype855, _size852);
-            this->partNames.resize(_size852);
-            uint32_t _i856;
-            for (_i856 = 0; _i856 < _size852; ++_i856)
+            uint32_t _size851;
+            ::apache::thrift::protocol::TType _etype854;
+            xfer += iprot->readListBegin(_etype854, _size851);
+            this->partNames.resize(_size851);
+            uint32_t _i855;
+            for (_i855 = 0; _i855 < _size851; ++_i855)
             {
-              xfer += iprot->readString(this->partNames[_i856]);
+              xfer += iprot->readString(this->partNames[_i855]);
             }
             xfer += iprot->readListEnd();
           }
@@ -23175,10 +23174,10 @@ uint32_t ReplTblWriteIdStateRequest::write(::apache::thrift::protocol::TProtocol
     xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 6);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partNames.size()));
-      std::vector<std::string> ::const_iterator _iter857;
-      for (_iter857 = this->partNames.begin(); _iter857 != this->partNames.end(); ++_iter857)
+      std::vector<std::string> ::const_iterator _iter856;
+      for (_iter856 = this->partNames.begin(); _iter856 != this->partNames.end(); ++_iter856)
       {
-        xfer += oprot->writeString((*_iter857));
+        xfer += oprot->writeString((*_iter856));
       }
       xfer += oprot->writeListEnd();
     }
@@ -23200,7 +23199,16 @@ void swap(ReplTblWriteIdStateRequest &a, ReplTblWriteIdStateRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-ReplTblWriteIdStateRequest::ReplTblWriteIdStateRequest(const ReplTblWriteIdStateRequest& other858) {
+ReplTblWriteIdStateRequest::ReplTblWriteIdStateRequest(const ReplTblWriteIdStateRequest& other857) {
+  validWriteIdlist = other857.validWriteIdlist;
+  user = other857.user;
+  hostName = other857.hostName;
+  dbName = other857.dbName;
+  tableName = other857.tableName;
+  partNames = other857.partNames;
+  __isset = other857.__isset;
+}
+ReplTblWriteIdStateRequest& ReplTblWriteIdStateRequest::operator=(const ReplTblWriteIdStateRequest& other858) {
   validWriteIdlist = other858.validWriteIdlist;
   user = other858.user;
   hostName = other858.hostName;
@@ -23208,15 +23216,6 @@ ReplTblWriteIdStateRequest::ReplTblWriteIdStateRequest(const ReplTblWriteIdState
   tableName = other858.tableName;
   partNames = other858.partNames;
   __isset = other858.__isset;
-}
-ReplTblWriteIdStateRequest& ReplTblWriteIdStateRequest::operator=(const ReplTblWriteIdStateRequest& other859) {
-  validWriteIdlist = other859.validWriteIdlist;
-  user = other859.user;
-  hostName = other859.hostName;
-  dbName = other859.dbName;
-  tableName = other859.tableName;
-  partNames = other859.partNames;
-  __isset = other859.__isset;
   return *this;
 }
 void ReplTblWriteIdStateRequest::printTo(std::ostream& out) const {
@@ -23282,14 +23281,14 @@ uint32_t GetValidWriteIdsRequest::read(::apache::thrift::protocol::TProtocol* ip
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->fullTableNames.clear();
-            uint32_t _size860;
-            ::apache::thrift::protocol::TType _etype863;
-            xfer += iprot->readListBegin(_etype863, _size860);
-            this->fullTableNames.resize(_size860);
-            uint32_t _i864;
-            for (_i864 = 0; _i864 < _size860; ++_i864)
+            uint32_t _size859;
+            ::apache::thrift::protocol::TType _etype862;
+            xfer += iprot->readListBegin(_etype862, _size859);
+            this->fullTableNames.resize(_size859);
+            uint32_t _i863;
+            for (_i863 = 0; _i863 < _size859; ++_i863)
             {
-              xfer += iprot->readString(this->fullTableNames[_i864]);
+              xfer += iprot->readString(this->fullTableNames[_i863]);
             }
             xfer += iprot->readListEnd();
           }
@@ -23336,10 +23335,10 @@ uint32_t GetValidWriteIdsRequest::write(::apache::thrift::protocol::TProtocol* o
   xfer += oprot->writeFieldBegin("fullTableNames", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->fullTableNames.size()));
-    std::vector<std::string> ::const_iterator _iter865;
-    for (_iter865 = this->fullTableNames.begin(); _iter865 != this->fullTableNames.end(); ++_iter865)
+    std::vector<std::string> ::const_iterator _iter864;
+    for (_iter864 = this->fullTableNames.begin(); _iter864 != this->fullTableNames.end(); ++_iter864)
     {
-      xfer += oprot->writeString((*_iter865));
+      xfer += oprot->writeString((*_iter864));
     }
     xfer += oprot->writeListEnd();
   }
@@ -23368,17 +23367,17 @@ void swap(GetValidWriteIdsRequest &a, GetValidWriteIdsRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-GetValidWriteIdsRequest::GetValidWriteIdsRequest(const GetValidWriteIdsRequest& other866) {
+GetValidWriteIdsRequest::GetValidWriteIdsRequest(const GetValidWriteIdsRequest& other865) {
+  fullTableNames = other865.fullTableNames;
+  validTxnList = other865.validTxnList;
+  writeId = other865.writeId;
+  __isset = other865.__isset;
+}
+GetValidWriteIdsRequest& GetValidWriteIdsRequest::operator=(const GetValidWriteIdsRequest& other866) {
   fullTableNames = other866.fullTableNames;
   validTxnList = other866.validTxnList;
   writeId = other866.writeId;
   __isset = other866.__isset;
-}
-GetValidWriteIdsRequest& GetValidWriteIdsRequest::operator=(const GetValidWriteIdsRequest& other867) {
-  fullTableNames = other867.fullTableNames;
-  validTxnList = other867.validTxnList;
-  writeId = other867.writeId;
-  __isset = other867.__isset;
   return *this;
 }
 void GetValidWriteIdsRequest::printTo(std::ostream& out) const {
@@ -23467,14 +23466,14 @@ uint32_t TableValidWriteIds::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->invalidWriteIds.clear();
-            uint32_t _size868;
-            ::apache::thrift::protocol::TType _etype871;
-            xfer += iprot->readListBegin(_etype871, _size868);
-            this->invalidWriteIds.resize(_size868);
-            uint32_t _i872;
-            for (_i872 = 0; _i872 < _size868; ++_i872)
+            uint32_t _size867;
+            ::apache::thrift::protocol::TType _etype870;
+            xfer += iprot->readListBegin(_etype870, _size867);
+            this->invalidWriteIds.resize(_size867);
+            uint32_t _i871;
+            for (_i871 = 0; _i871 < _size867; ++_i871)
             {
-              xfer += iprot->readI64(this->invalidWriteIds[_i872]);
+              xfer += iprot->readI64(this->invalidWriteIds[_i871]);
             }
             xfer += iprot->readListEnd();
           }
@@ -23535,10 +23534,10 @@ uint32_t TableValidWriteIds::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("invalidWriteIds", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->invalidWriteIds.size()));
-    std::vector<int64_t> ::const_iterator _iter873;
-    for (_iter873 = this->invalidWriteIds.begin(); _iter873 != this->invalidWriteIds.end(); ++_iter873)
+    std::vector<int64_t> ::const_iterator _iter872;
+    for (_iter872 = this->invalidWriteIds.begin(); _iter872 != this->invalidWriteIds.end(); ++_iter872)
     {
-      xfer += oprot->writeI64((*_iter873));
+      xfer += oprot->writeI64((*_iter872));
     }
     xfer += oprot->writeListEnd();
   }
@@ -23568,21 +23567,21 @@ void swap(TableValidWriteIds &a, TableValidWriteIds &b) {
   swap(a.__isset, b.__isset);
 }
 
-TableValidWriteIds::TableValidWriteIds(const TableValidWriteIds& other874) {
+TableValidWriteIds::TableValidWriteIds(const TableValidWriteIds& other873) {
+  fullTableName = other873.fullTableName;
+  writeIdHighWaterMark = other873.writeIdHighWaterMark;
+  invalidWriteIds = other873.invalidWriteIds;
+  minOpenWriteId = other873.minOpenWriteId;
+  abortedBits = other873.abortedBits;
+  __isset = other873.__isset;
+}
+TableValidWriteIds& TableValidWriteIds::operator=(const TableValidWriteIds& other874) {
   fullTableName = other874.fullTableName;
   writeIdHighWaterMark = other874.writeIdHighWaterMark;
   invalidWriteIds = other874.invalidWriteIds;
   minOpenWriteId = other874.minOpenWriteId;
   abortedBits = other874.abortedBits;
   __isset = other874.__isset;
-}
-TableValidWriteIds& TableValidWriteIds::operator=(const TableValidWriteIds& other875) {
-  fullTableName = other875.fullTableName;
-  writeIdHighWaterMark = other875.writeIdHighWaterMark;
-  invalidWriteIds = other875.invalidWriteIds;
-  minOpenWriteId = other875.minOpenWriteId;
-  abortedBits = other875.abortedBits;
-  __isset = other875.__isset;
   return *this;
 }
 void TableValidWriteIds::printTo(std::ostream& out) const {
@@ -23637,14 +23636,14 @@ uint32_t GetValidWriteIdsResponse::read(::apache::thrift::protocol::TProtocol* i
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tblValidWriteIds.clear();
-            uint32_t _size876;
-            ::apache::thrift::protocol::TType _etype879;
-            xfer += iprot->readListBegin(_etype879, _size876);
-            this->tblValidWriteIds.resize(_size876);
-            uint32_t _i880;
-            for (_i880 = 0; _i880 < _size876; ++_i880)
+            uint32_t _size875;
+            ::apache::thrift::protocol::TType _etype878;
+            xfer += iprot->readListBegin(_etype878, _size875);
+            this->tblValidWriteIds.resize(_size875);
+            uint32_t _i879;
+            for (_i879 = 0; _i879 < _size875; ++_i879)
             {
-              xfer += this->tblValidWriteIds[_i880].read(iprot);
+              xfer += this->tblValidWriteIds[_i879].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -23675,10 +23674,10 @@ uint32_t GetValidWriteIdsResponse::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("tblValidWriteIds", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->tblValidWriteIds.size()));
-    std::vector<TableValidWriteIds> ::const_iterator _iter881;
-    for (_iter881 = this->tblValidWriteIds.begin(); _iter881 != this->tblValidWriteIds.end(); ++_iter881)
+    std::vector<TableValidWriteIds> ::const_iterator _iter880;
+    for (_iter880 = this->tblValidWriteIds.begin(); _iter880 != this->tblValidWriteIds.end(); ++_iter880)
     {
-      xfer += (*_iter881).write(oprot);
+      xfer += (*_iter880).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -23694,11 +23693,11 @@ void swap(GetValidWriteIdsResponse &a, GetValidWriteIdsResponse &b) {
   swap(a.tblValidWriteIds, b.tblValidWriteIds);
 }
 
-GetValidWriteIdsResponse::GetValidWriteIdsResponse(const GetValidWriteIdsResponse& other882) {
-  tblValidWriteIds = other882.tblValidWriteIds;
+GetValidWriteIdsResponse::GetValidWriteIdsResponse(const GetValidWriteIdsResponse& other881) {
+  tblValidWriteIds = other881.tblValidWriteIds;
 }
-GetValidWriteIdsResponse& GetValidWriteIdsResponse::operator=(const GetValidWriteIdsResponse& other883) {
-  tblValidWriteIds = other883.tblValidWriteIds;
+GetValidWriteIdsResponse& GetValidWriteIdsResponse::operator=(const GetValidWriteIdsResponse& other882) {
+  tblValidWriteIds = other882.tblValidWriteIds;
   return *this;
 }
 void GetValidWriteIdsResponse::printTo(std::ostream& out) const {
@@ -23806,13 +23805,13 @@ void swap(TxnToWriteId &a, TxnToWriteId &b) {
   swap(a.writeId, b.writeId);
 }
 
-TxnToWriteId::TxnToWriteId(const TxnToWriteId& other884) {
+TxnToWriteId::TxnToWriteId(const TxnToWriteId& other883) {
+  txnId = other883.txnId;
+  writeId = other883.writeId;
+}
+TxnToWriteId& TxnToWriteId::operator=(const TxnToWriteId& other884) {
   txnId = other884.txnId;
   writeId = other884.writeId;
-}
-TxnToWriteId& TxnToWriteId::operator=(const TxnToWriteId& other885) {
-  txnId = other885.txnId;
-  writeId = other885.writeId;
   return *this;
 }
 void TxnToWriteId::printTo(std::ostream& out) const {
@@ -23900,14 +23899,14 @@ uint32_t AllocateTableWriteIdsRequest::read(::apache::thrift::protocol::TProtoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->txnIds.clear();
... 16439 lines suppressed ...