You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jc...@apache.org on 2018/02/04 22:07:28 UTC

[01/16] hive git commit: HIVE-18546: Remove unnecessary code introduced in HIVE-14498 (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Repository: hive
Updated Branches:
  refs/heads/master 4a33ec8fc -> 0a328f030


http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb b/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
index 3a11a05..bc58cfe 100644
--- a/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -827,7 +827,7 @@ class Table
     PRIVILEGES => {:type => ::Thrift::Types::STRUCT, :name => 'privileges', :class => ::PrincipalPrivilegeSet, :optional => true},
     TEMPORARY => {:type => ::Thrift::Types::BOOL, :name => 'temporary', :default => false, :optional => true},
     REWRITEENABLED => {:type => ::Thrift::Types::BOOL, :name => 'rewriteEnabled', :optional => true},
-    CREATIONMETADATA => {:type => ::Thrift::Types::MAP, :name => 'creationMetadata', :key => {:type => ::Thrift::Types::STRING}, :value => {:type => ::Thrift::Types::STRUCT, :class => ::BasicTxnInfo}, :optional => true}
+    CREATIONMETADATA => {:type => ::Thrift::Types::STRUCT, :name => 'creationMetadata', :class => ::CreationMetadata, :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -2673,16 +2673,14 @@ end
 class BasicTxnInfo
   include ::Thrift::Struct, ::Thrift::Struct_Union
   ISNULL = 1
-  ID = 2
-  TIME = 3
-  TXNID = 4
-  DBNAME = 5
-  TABLENAME = 6
-  PARTITIONNAME = 7
+  TIME = 2
+  TXNID = 3
+  DBNAME = 4
+  TABLENAME = 5
+  PARTITIONNAME = 6
 
   FIELDS = {
     ISNULL => {:type => ::Thrift::Types::BOOL, :name => 'isnull'},
-    ID => {:type => ::Thrift::Types::I64, :name => 'id', :optional => true},
     TIME => {:type => ::Thrift::Types::I64, :name => 'time', :optional => true},
     TXNID => {:type => ::Thrift::Types::I64, :name => 'txnid', :optional => true},
     DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbname', :optional => true},
@@ -2699,21 +2697,26 @@ class BasicTxnInfo
   ::Thrift::Struct.generate_accessors self
 end
 
-class TxnsSnapshot
+class CreationMetadata
   include ::Thrift::Struct, ::Thrift::Struct_Union
-  TXN_HIGH_WATER_MARK = 1
-  OPEN_TXNS = 2
+  DBNAME = 1
+  TBLNAME = 2
+  TABLESUSED = 3
+  VALIDTXNLIST = 4
 
   FIELDS = {
-    TXN_HIGH_WATER_MARK => {:type => ::Thrift::Types::I64, :name => 'txn_high_water_mark'},
-    OPEN_TXNS => {:type => ::Thrift::Types::LIST, :name => 'open_txns', :element => {:type => ::Thrift::Types::I64}}
+    DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'},
+    TBLNAME => {:type => ::Thrift::Types::STRING, :name => 'tblName'},
+    TABLESUSED => {:type => ::Thrift::Types::SET, :name => 'tablesUsed', :element => {:type => ::Thrift::Types::STRING}},
+    VALIDTXNLIST => {:type => ::Thrift::Types::STRING, :name => 'validTxnList', :optional => true}
   }
 
   def struct_fields; FIELDS; end
 
   def validate
-    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field txn_high_water_mark is unset!') unless @txn_high_water_mark
-    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field open_txns is unset!') unless @open_txns
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field dbName is unset!') unless @dbName
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field tblName is unset!') unless @tblName
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field tablesUsed is unset!') unless @tablesUsed
   end
 
   ::Thrift::Struct.generate_accessors self

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb b/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
index a788c08..ec88131 100644
--- a/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
+++ b/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
@@ -2546,36 +2546,6 @@ module ThriftHiveMetastore
       return
     end
 
-    def get_last_completed_transaction_for_tables(db_names, table_names, txns_snapshot)
-      send_get_last_completed_transaction_for_tables(db_names, table_names, txns_snapshot)
-      return recv_get_last_completed_transaction_for_tables()
-    end
-
-    def send_get_last_completed_transaction_for_tables(db_names, table_names, txns_snapshot)
-      send_message('get_last_completed_transaction_for_tables', Get_last_completed_transaction_for_tables_args, :db_names => db_names, :table_names => table_names, :txns_snapshot => txns_snapshot)
-    end
-
-    def recv_get_last_completed_transaction_for_tables()
-      result = receive_message(Get_last_completed_transaction_for_tables_result)
-      return result.success unless result.success.nil?
-      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_last_completed_transaction_for_tables failed: unknown result')
-    end
-
-    def get_last_completed_transaction_for_table(db_name, table_name, txns_snapshot)
-      send_get_last_completed_transaction_for_table(db_name, table_name, txns_snapshot)
-      return recv_get_last_completed_transaction_for_table()
-    end
-
-    def send_get_last_completed_transaction_for_table(db_name, table_name, txns_snapshot)
-      send_message('get_last_completed_transaction_for_table', Get_last_completed_transaction_for_table_args, :db_name => db_name, :table_name => table_name, :txns_snapshot => txns_snapshot)
-    end
-
-    def recv_get_last_completed_transaction_for_table()
-      result = receive_message(Get_last_completed_transaction_for_table_result)
-      return result.success unless result.success.nil?
-      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_last_completed_transaction_for_table failed: unknown result')
-    end
-
     def get_next_notification(rqst)
       send_get_next_notification(rqst)
       return recv_get_next_notification()
@@ -4986,20 +4956,6 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'add_dynamic_partitions', seqid)
     end
 
-    def process_get_last_completed_transaction_for_tables(seqid, iprot, oprot)
-      args = read_args(iprot, Get_last_completed_transaction_for_tables_args)
-      result = Get_last_completed_transaction_for_tables_result.new()
-      result.success = @handler.get_last_completed_transaction_for_tables(args.db_names, args.table_names, args.txns_snapshot)
-      write_result(result, oprot, 'get_last_completed_transaction_for_tables', seqid)
-    end
-
-    def process_get_last_completed_transaction_for_table(seqid, iprot, oprot)
-      args = read_args(iprot, Get_last_completed_transaction_for_table_args)
-      result = Get_last_completed_transaction_for_table_result.new()
-      result.success = @handler.get_last_completed_transaction_for_table(args.db_name, args.table_name, args.txns_snapshot)
-      write_result(result, oprot, 'get_last_completed_transaction_for_table', seqid)
-    end
-
     def process_get_next_notification(seqid, iprot, oprot)
       args = read_args(iprot, Get_next_notification_args)
       result = Get_next_notification_result.new()
@@ -11088,78 +11044,6 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
-  class Get_last_completed_transaction_for_tables_args
-    include ::Thrift::Struct, ::Thrift::Struct_Union
-    DB_NAMES = 1
-    TABLE_NAMES = 2
-    TXNS_SNAPSHOT = 3
-
-    FIELDS = {
-      DB_NAMES => {:type => ::Thrift::Types::LIST, :name => 'db_names', :element => {:type => ::Thrift::Types::STRING}},
-      TABLE_NAMES => {:type => ::Thrift::Types::LIST, :name => 'table_names', :element => {:type => ::Thrift::Types::STRING}},
-      TXNS_SNAPSHOT => {:type => ::Thrift::Types::STRUCT, :name => 'txns_snapshot', :class => ::TxnsSnapshot}
-    }
-
-    def struct_fields; FIELDS; end
-
-    def validate
-    end
-
-    ::Thrift::Struct.generate_accessors self
-  end
-
-  class Get_last_completed_transaction_for_tables_result
-    include ::Thrift::Struct, ::Thrift::Struct_Union
-    SUCCESS = 0
-
-    FIELDS = {
-      SUCCESS => {:type => ::Thrift::Types::LIST, :name => 'success', :element => {:type => ::Thrift::Types::STRUCT, :class => ::BasicTxnInfo}}
-    }
-
-    def struct_fields; FIELDS; end
-
-    def validate
-    end
-
-    ::Thrift::Struct.generate_accessors self
-  end
-
-  class Get_last_completed_transaction_for_table_args
-    include ::Thrift::Struct, ::Thrift::Struct_Union
-    DB_NAME = 1
-    TABLE_NAME = 2
-    TXNS_SNAPSHOT = 3
-
-    FIELDS = {
-      DB_NAME => {:type => ::Thrift::Types::STRING, :name => 'db_name'},
-      TABLE_NAME => {:type => ::Thrift::Types::STRING, :name => 'table_name'},
-      TXNS_SNAPSHOT => {:type => ::Thrift::Types::STRUCT, :name => 'txns_snapshot', :class => ::TxnsSnapshot}
-    }
-
-    def struct_fields; FIELDS; end
-
-    def validate
-    end
-
-    ::Thrift::Struct.generate_accessors self
-  end
-
-  class Get_last_completed_transaction_for_table_result
-    include ::Thrift::Struct, ::Thrift::Struct_Union
-    SUCCESS = 0
-
-    FIELDS = {
-      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::BasicTxnInfo}
-    }
-
-    def struct_fields; FIELDS; end
-
-    def validate
-    end
-
-    ::Thrift::Struct.generate_accessors self
-  end
-
   class Get_next_notification_args
     include ::Thrift::Struct, ::Thrift::Struct_Union
     RQST = 1

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index ecc4644..8dc9b6a 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -6951,19 +6951,6 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     }
 
     @Override
-    public List<BasicTxnInfo> get_last_completed_transaction_for_tables(
-        List<String> dbNames, List<String> tableNames, TxnsSnapshot txnsSnapshot)
-            throws TException {
-      return getTxnHandler().getLastCompletedTransactionForTables(dbNames, tableNames, txnsSnapshot);
-    }
-
-    @Override
-    public BasicTxnInfo get_last_completed_transaction_for_table(String dbName, String tableName, TxnsSnapshot txnsSnapshot)
-        throws TException {
-      return getTxnHandler().getLastCompletedTransactionForTable(dbName, tableName, txnsSnapshot);
-    }
-
-    @Override
     public NotificationEventsCountResponse get_notification_events_count(NotificationEventsCountRequest rqst)
             throws TException {
       try {

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index a3cb17b..2e76e17 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -2170,25 +2170,6 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   }
 
   @Override
-  public List<BasicTxnInfo> getLastCompletedTransactionForTables(
-      List<String> dbNames, List<String> tableNames, ValidTxnList txnList)
-          throws TException {
-    TxnsSnapshot txnsSnapshot = new TxnsSnapshot();
-    txnsSnapshot.setTxn_high_water_mark(txnList.getHighWatermark());
-    txnsSnapshot.setOpen_txns(Arrays.asList(ArrayUtils.toObject(txnList.getInvalidTransactions())));
-    return client.get_last_completed_transaction_for_tables(dbNames, tableNames, txnsSnapshot);
-  }
-
-  @Override
-  public BasicTxnInfo getLastCompletedTransactionForTable(String dbName, String tableName, ValidTxnList txnList)
-      throws TException {
-    TxnsSnapshot txnsSnapshot = new TxnsSnapshot();
-    txnsSnapshot.setTxn_high_water_mark(txnList.getHighWatermark());
-    txnsSnapshot.setOpen_txns(Arrays.asList(ArrayUtils.toObject(txnList.getInvalidTransactions())));
-    return client.get_last_completed_transaction_for_table(dbName, tableName, txnsSnapshot);
-  }
-
-  @Override
   public ValidTxnList getValidTxns() throws TException {
     return TxnUtils.createValidReadTxnList(client.get_open_txns(), 0);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index 8ec8b3b..96d4590 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -1338,24 +1338,6 @@ public interface IMetaStoreClient {
           throws MetaException, TException;
 
   /**
-   * Get the last completed transaction for the given tables. Although transactions in Hive
-   * might happen concurrently, the order is based on the actual commit to the metastore
-   * table holding the completed transactions.
-   */
-  @InterfaceAudience.LimitedPrivate({"HCatalog"})
-  List<BasicTxnInfo> getLastCompletedTransactionForTables(List<String> dbNames, List<String> tableNames, ValidTxnList txnList)
-      throws TException;
-
-  /**
-   * Get the last completed transaction for the given table. Although transactions in Hive
-   * might happen concurrently, the order is based on the actual commit to the metastore
-   * table holding the completed transactions.
-   */
-  @InterfaceAudience.LimitedPrivate({"HCatalog"})
-  BasicTxnInfo getLastCompletedTransactionForTable(String dbName, String tableName, ValidTxnList txnList)
-      throws TException;
-
-  /**
    * Get a structure that details valid transactions.
    * @return list of valid transactions
    * @throws TException

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MaterializationsInvalidationCache.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MaterializationsInvalidationCache.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MaterializationsInvalidationCache.java
index de912d5..20e4e8d 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MaterializationsInvalidationCache.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MaterializationsInvalidationCache.java
@@ -17,15 +17,18 @@
  */
 package org.apache.hadoop.hive.metastore;
 
+import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 
+import org.apache.hadoop.hive.common.ValidReadTxnList;
+import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.metastore.api.BasicTxnInfo;
 import org.apache.hadoop.hive.metastore.api.Materialization;
 import org.apache.hadoop.hive.metastore.api.MetaException;
@@ -65,8 +68,8 @@ public final class MaterializationsInvalidationCache {
    * happen. This is useful to quickly check the invalidation time for a given materialized
    * view. 
    */
-  private final ConcurrentMap<String, ConcurrentSkipListSet<TableModificationKey>> tableModifications =
-      new ConcurrentHashMap<String, ConcurrentSkipListSet<TableModificationKey>>();
+  private final ConcurrentMap<String, ConcurrentSkipListMap<Long, Long>> tableModifications =
+      new ConcurrentHashMap<String, ConcurrentSkipListMap<Long, Long>>();
 
   /* Whether the cache has been initialized or not. */
   private boolean initialized;
@@ -112,7 +115,7 @@ public final class MaterializationsInvalidationCache {
       try {
         for (String dbName : store.getAllDatabases()) {
           for (Table mv : store.getTableObjectsByName(dbName, store.getTables(dbName, null, TableType.MATERIALIZED_VIEW))) {
-            addMaterializedView(mv, ImmutableSet.copyOf(mv.getCreationMetadata().keySet()), OpType.LOAD);
+            addMaterializedView(mv, ImmutableSet.copyOf(mv.getCreationMetadata().getTablesUsed()), OpType.LOAD);
           }
         }
         LOG.info("Initialized materializations invalidation cache");
@@ -160,53 +163,46 @@ public final class MaterializationsInvalidationCache {
     // Start the process to add materialization to the cache
     // Before loading the materialization in the cache, we need to update some
     // important information in the registry to account for rewriting invalidation
-    for (String qNameTableUsed : tablesUsed) {
-      // First we insert a new tree set to keep table modifications, unless it already exists
-      ConcurrentSkipListSet<TableModificationKey> modificationsTree =
-          new ConcurrentSkipListSet<TableModificationKey>();
-      final ConcurrentSkipListSet<TableModificationKey> prevModificationsTree = tableModifications.putIfAbsent(
-          qNameTableUsed, modificationsTree);
-      if (prevModificationsTree != null) {
-        modificationsTree = prevModificationsTree;
-      }
-      // We obtain the access time to the table when the materialized view was created.
-      // This is a map from table fully qualified name to last modification before MV creation.
-      BasicTxnInfo e = materializedViewTable.getCreationMetadata().get(qNameTableUsed);
-      if (e.isIsnull()) {
-        // This can happen when the materialized view was created on non-transactional tables
-        // with rewrite disabled but then it was enabled by alter statement
-        continue;
-      }
-      final TableModificationKey lastModificationBeforeCreation =
-          new TableModificationKey(e.getId(), e.getTime());
-      modificationsTree.add(lastModificationBeforeCreation);
-      if (opType == OpType.LOAD) {
+    String txnListString = materializedViewTable.getCreationMetadata().getValidTxnList();
+    if (txnListString == null) {
+      // This can happen when the materialized view was created on non-transactional tables
+      return;
+    }
+    if (opType == OpType.CREATE || opType == OpType.ALTER) {
+      // You store the materialized view
+      cq.put(materializedViewTable.getTableName(),
+          new MaterializationInvalidationInfo(materializedViewTable, tablesUsed));
+    } else {
+      ValidTxnList txnList = new ValidReadTxnList(txnListString);
+      for (String qNameTableUsed : tablesUsed) {
+        // First we insert a new tree set to keep table modifications, unless it already exists
+        ConcurrentSkipListMap<Long, Long> modificationsTree =
+                new ConcurrentSkipListMap<Long, Long>();
+        final ConcurrentSkipListMap<Long, Long> prevModificationsTree = tableModifications.putIfAbsent(
+                qNameTableUsed, modificationsTree);
+        if (prevModificationsTree != null) {
+          modificationsTree = prevModificationsTree;
+        }
         // If we are not creating the MV at this instant, but instead it was created previously
-        // and we are loading it into the cache, we need to go through the transaction logs and
+        // and we are loading it into the cache, we need to go through the transaction entries and
         // check if the MV is still valid.
         try {
           String[] names =  qNameTableUsed.split("\\.");
-          BasicTxnInfo e2 = txnStore.getFirstCompletedTransactionForTableAfterCommit(
-              names[0], names[1], lastModificationBeforeCreation.id);
-          if (!e2.isIsnull()) {
-            modificationsTree.add(new TableModificationKey(e2.getId(), e2.getTime()));
+          BasicTxnInfo e = txnStore.getFirstCompletedTransactionForTableAfterCommit(
+                  names[0], names[1], txnList);
+          if (!e.isIsnull()) {
+            modificationsTree.put(e.getTxnid(), e.getTime());
             // We do not need to do anything more for current table, as we detected
             // a modification event that was in the metastore.
             continue;
           }
         } catch (MetaException ex) {
           LOG.debug("Materialized view " +
-              Warehouse.getQualifiedName(materializedViewTable.getDbName(), materializedViewTable.getTableName()) +
-              " ignored; error loading view into invalidation cache", ex);
+                  Warehouse.getQualifiedName(materializedViewTable.getDbName(), materializedViewTable.getTableName()) +
+                  " ignored; error loading view into invalidation cache", ex);
           return;
         }
       }
-    }
-    if (opType == OpType.CREATE || opType == OpType.ALTER) {
-      // You store the materialized view
-      cq.put(materializedViewTable.getTableName(),
-          new MaterializationInvalidationInfo(materializedViewTable, tablesUsed));
-    } else {
       // For LOAD, you only add it if it does exist as you might be loading an outdated MV
       cq.putIfAbsent(materializedViewTable.getTableName(),
           new MaterializationInvalidationInfo(materializedViewTable, tablesUsed));
@@ -218,23 +214,23 @@ public final class MaterializationsInvalidationCache {
   }
 
   /**
-   * This method is called when a table is modified. That way we can keep a track of the
+   * This method is called when a table is modified. That way we can keep track of the
    * invalidation for the MVs that use that table.
    */
   public void notifyTableModification(String dbName, String tableName,
-      long eventId, long newModificationTime) {
+      long txnId, long newModificationTime) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("Notification for table {} in database {} received -> id: {}, time: {}",
-          tableName, dbName, eventId, newModificationTime);
+          tableName, dbName, txnId, newModificationTime);
     }
-    ConcurrentSkipListSet<TableModificationKey> modificationsTree =
-        new ConcurrentSkipListSet<TableModificationKey>();
-    final ConcurrentSkipListSet<TableModificationKey> prevModificationsTree =
+    ConcurrentSkipListMap<Long, Long> modificationsTree =
+        new ConcurrentSkipListMap<Long, Long>();
+    final ConcurrentSkipListMap<Long, Long> prevModificationsTree =
         tableModifications.putIfAbsent(Warehouse.getQualifiedName(dbName, tableName), modificationsTree);
     if (prevModificationsTree != null) {
       modificationsTree = prevModificationsTree;
     }
-    modificationsTree.add(new TableModificationKey(eventId, newModificationTime));
+    modificationsTree.put(txnId, newModificationTime);
   }
 
   /**
@@ -296,69 +292,49 @@ public final class MaterializationsInvalidationCache {
   }
 
   private long getInvalidationTime(MaterializationInvalidationInfo materialization) {
+    String txnListString = materialization.getMaterializationTable().getCreationMetadata().getValidTxnList();
+    if (txnListString == null) {
+      // This can happen when the materialization was created on non-transactional tables
+      return Long.MIN_VALUE;
+    }
+
+    // We will obtain the modification time as follows.
+    // First, we obtain the first element after high watermark (if any)
+    // Then, we iterate through the elements from min open txn till high
+    // watermark, updating the modification time after creation if needed
+    ValidTxnList txnList = new ValidReadTxnList(txnListString);
     long firstModificationTimeAfterCreation = 0L;
     for (String qNameTableUsed : materialization.getTablesUsed()) {
-      BasicTxnInfo e = materialization.getMaterializationTable().getCreationMetadata().get(qNameTableUsed);
-      if (e == null) {
-        // This can happen when the materialized view was created on non-transactional tables
-        // with rewrite disabled but then it was enabled by alter statement
-        return Long.MIN_VALUE;
-      }
-      final TableModificationKey lastModificationBeforeCreation =
-          new TableModificationKey(e.getId(), e.getTime());
-      final TableModificationKey post = tableModifications.get(qNameTableUsed)
-          .higher(lastModificationBeforeCreation);
-      if (post != null) {
+      final Long tn = tableModifications.get(qNameTableUsed)
+          .higherKey(txnList.getHighWatermark());
+      if (tn != null) {
         if (firstModificationTimeAfterCreation == 0L ||
-            post.time < firstModificationTimeAfterCreation) {
-          firstModificationTimeAfterCreation = post.time;
+            tn < firstModificationTimeAfterCreation) {
+          firstModificationTimeAfterCreation = tn;
         }
       }
-    }
-    return firstModificationTimeAfterCreation;
-  }
-
-  private static class TableModificationKey implements Comparable<TableModificationKey> {
-    private long id;
-    private long time;
-
-    private TableModificationKey(long id, long time) {
-      this.id = id;
-      this.time = time;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      if(this == obj) {
-        return true;
-      }
-      if((obj == null) || (obj.getClass() != this.getClass())) {
-        return false;
-      }
-      TableModificationKey tableModificationKey = (TableModificationKey) obj;
-      return id == tableModificationKey.id && time == tableModificationKey.time;
-    }
-
-    @Override
-    public int hashCode() {
-      int hash = 7;
-      hash = 31 * hash + Long.hashCode(id);
-      hash = 31 * hash + Long.hashCode(time);
-      return hash;
-    }
-
-    @Override
-    public int compareTo(TableModificationKey other) {
-      if (id == other.id) {
-        return Long.compare(time, other.time);
+      // Min open txn might be null if there were no open transactions
+      // when this transaction was being executed
+      if (txnList.getMinOpenTxn() != null) {
+        // Invalid transaction list is sorted
+        int pos = 0;
+        for (Map.Entry<Long, Long> t : tableModifications.get(qNameTableUsed)
+                .subMap(txnList.getMinOpenTxn(), txnList.getHighWatermark()).entrySet()) {
+          while (pos < txnList.getInvalidTransactions().length &&
+              txnList.getInvalidTransactions()[pos] != t.getKey()) {
+            pos++;
+          }
+          if (pos >= txnList.getInvalidTransactions().length) {
+            break;
+          }
+          if (firstModificationTimeAfterCreation == 0L ||
+              t.getValue() < firstModificationTimeAfterCreation) {
+            firstModificationTimeAfterCreation = t.getValue();
+          }
+        }
       }
-      return Long.compare(id, other.id);
-    }
-
-    @Override
-    public String toString() {
-      return "TableModificationKey{" + id + "," + time + "}";
     }
+    return firstModificationTimeAfterCreation;
   }
 
   private enum OpType {

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index b3d99a1..3d1c67f 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -68,7 +68,6 @@ import javax.jdo.identity.IntIdentity;
 import javax.sql.DataSource;
 
 import org.apache.commons.collections.CollectionUtils;
-import org.apache.commons.collections.MapUtils;
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.commons.lang.exception.ExceptionUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -80,10 +79,10 @@ import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.metastore.MetaStoreDirectSql.SqlFilterForPushdown;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
-import org.apache.hadoop.hive.metastore.api.BasicTxnInfo;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+import org.apache.hadoop.hive.metastore.api.CreationMetadata;
 import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -149,6 +148,7 @@ import org.apache.hadoop.hive.metastore.metrics.Metrics;
 import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
 import org.apache.hadoop.hive.metastore.model.MColumnDescriptor;
 import org.apache.hadoop.hive.metastore.model.MConstraint;
+import org.apache.hadoop.hive.metastore.model.MCreationMetadata;
 import org.apache.hadoop.hive.metastore.model.MDBPrivilege;
 import org.apache.hadoop.hive.metastore.model.MDatabase;
 import org.apache.hadoop.hive.metastore.model.MDelegationToken;
@@ -193,10 +193,7 @@ import org.apache.hadoop.hive.metastore.utils.JavaUtils;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.ColStatsObjWithSourceInfo;
 import org.apache.hadoop.hive.metastore.utils.ObjectPair;
-import org.apache.thrift.TDeserializer;
 import org.apache.thrift.TException;
-import org.apache.thrift.TSerializer;
-import org.apache.thrift.protocol.TJSONProtocol;
 import org.datanucleus.AbstractNucleusContext;
 import org.datanucleus.ClassLoaderResolver;
 import org.datanucleus.ClassLoaderResolverImpl;
@@ -1150,7 +1147,7 @@ public class ObjectStore implements RawStore, Configurable {
         if (MetaStoreUtils.isMaterializedViewTable(tbl)) {
           // Add to the invalidation cache
           MaterializationsInvalidationCache.get().createMaterializedView(
-              tbl, tbl.getCreationMetadata().keySet());
+              tbl, tbl.getCreationMetadata().getTablesUsed());
         }
       }
     }
@@ -1236,6 +1233,14 @@ public class ObjectStore implements RawStore, Configurable {
         }
 
         preDropStorageDescriptor(tbl.getSd());
+
+        if (tbl.getCreationMetadata() != null) {
+          // Remove creation metadata
+          MCreationMetadata mcm = tbl.getCreationMetadata();
+          tbl.setCreationMetadata(null);
+          pm.deletePersistent(mcm);
+        }
+
         // then remove the table
         pm.deletePersistentAll(tbl);
       }
@@ -1552,6 +1557,11 @@ public class ObjectStore implements RawStore, Configurable {
         pm.retrieveAll(mtbl.getSd().getCD());
         nmtbl.mcd = mtbl.getSd().getCD();
       }
+      // Retrieve creation metadata if needed
+      if (mtbl != null &&
+          TableType.MATERIALIZED_VIEW.toString().equals(mtbl.getTableType())) {
+        mtbl.setCreationMetadata(getCreationMetadata(db, table));
+      }
       commited = commitTransaction();
     } finally {
       rollbackAndCleanup(commited, query);
@@ -1560,6 +1570,25 @@ public class ObjectStore implements RawStore, Configurable {
     return nmtbl;
   }
 
+  private MCreationMetadata getCreationMetadata(String dbName, String tblName) {
+    boolean commited = false;
+    MCreationMetadata mcm = null;
+    Query query = null;
+    try {
+      openTransaction();
+      query = pm.newQuery(
+          MCreationMetadata.class, "tblName == table && dbName == db");
+      query.declareParameters("java.lang.String table, java.lang.String db");
+      query.setUnique(true);
+      mcm = (MCreationMetadata) query.execute(tblName, dbName);
+      pm.retrieve(mcm);
+      commited = commitTransaction();
+    } finally {
+      rollbackAndCleanup(commited, query);
+    }
+    return mcm;
+  }
+
   private MTable getMTable(String db, String table) {
     AttachedMTableInfo nmtbl = getMTable(db, table, false);
     return nmtbl.mtbl;
@@ -1885,56 +1914,37 @@ public class ObjectStore implements RawStore, Configurable {
             .getSkewedColValueLocationMaps()), sd.isStoredAsSubDirectories());
   }
 
-  private Map<String, String> convertToMCreationMetadata(
-      Map<String, BasicTxnInfo> m) throws MetaException {
+  private MCreationMetadata convertToMCreationMetadata(
+      CreationMetadata m) throws MetaException {
     if (m == null) {
       return null;
     }
-    Map<String, String> r = new HashMap<>();
-    for (Entry<String, BasicTxnInfo> e : m.entrySet()) {
-      r.put(e.getKey(), serializeBasicTransactionInfo(e.getValue()));
+    Set<MTable> tablesUsed = new HashSet<>();
+    for (String fullyQualifiedName : m.getTablesUsed()) {
+      String[] names =  fullyQualifiedName.split("\\.");
+      tablesUsed.add(getMTable(names[0], names[1], false).mtbl);
     }
-    return r;
+    return new MCreationMetadata(m.getDbName(), m.getTblName(),
+        tablesUsed, m.getValidTxnList());
   }
 
-  private Map<String, BasicTxnInfo> convertToCreationMetadata(
-      Map<String, String> m) throws MetaException {
-    if (m == null) {
+  private CreationMetadata convertToCreationMetadata(
+      MCreationMetadata s) throws MetaException {
+    if (s == null) {
       return null;
     }
-    Map<String, BasicTxnInfo> r = new HashMap<>();
-    for (Entry<String, String> e : m.entrySet()) {
-      r.put(e.getKey(), deserializeBasicTransactionInfo(e.getValue()));
-    }
-    return r;
-  }
-
-  private String serializeBasicTransactionInfo(BasicTxnInfo entry)
-      throws MetaException {
-    if (entry.isIsnull()) {
-      return "";
-    }
-    try {
-      TSerializer serializer = new TSerializer(new TJSONProtocol.Factory());
-      return serializer.toString(entry, "UTF-8");
-    } catch (TException e) {
-      throw new MetaException("Error serializing object " + entry + ": " + e.toString());
-    }
-  }
-
-  private BasicTxnInfo deserializeBasicTransactionInfo(String s)
-      throws MetaException {
-    if (s.equals("")) {
-      return new BasicTxnInfo(true);
+    Set<String> tablesUsed = new HashSet<>();
+    for (MTable mtbl : s.getTables()) {
+      tablesUsed.add(
+          Warehouse.getQualifiedName(
+              mtbl.getDatabase().getName(), mtbl.getTableName()));
     }
-    try {
-      TDeserializer deserializer = new TDeserializer(new TJSONProtocol.Factory());
-      BasicTxnInfo r = new BasicTxnInfo();
-      deserializer.deserialize(r, s, "UTF-8");
-      return r;
-    } catch (TException e) {
-      throw new MetaException("Error deserializing object " + s + ": " + e.toString());
+    CreationMetadata r = new CreationMetadata(
+        s.getDbName(), s.getTblName(), tablesUsed);
+    if (s.getTxnList() != null) {
+      r.setValidTxnList(s.getTxnList());
     }
+    return r;
   }
 
   @Override
@@ -3703,9 +3713,10 @@ public class ObjectStore implements RawStore, Configurable {
       oldt.setViewOriginalText(newt.getViewOriginalText());
       oldt.setViewExpandedText(newt.getViewExpandedText());
       oldt.setRewriteEnabled(newt.isRewriteEnabled());
-      registerCreationSignature = !MapUtils.isEmpty(newt.getCreationMetadata());
+      registerCreationSignature = newt.getCreationMetadata() != null;
       if (registerCreationSignature) {
-        oldt.setCreationMetadata(newt.getCreationMetadata());
+        oldt.getCreationMetadata().setTables(newt.getCreationMetadata().getTables());
+        oldt.getCreationMetadata().setTxnList(newt.getCreationMetadata().getTxnList());
       }
 
       // commit the changes
@@ -3718,7 +3729,7 @@ public class ObjectStore implements RawStore, Configurable {
             registerCreationSignature) {
           // Add to the invalidation cache if the creation signature has changed
           MaterializationsInvalidationCache.get().alterMaterializedView(
-              newTable, newTable.getCreationMetadata().keySet());
+              newTable, newTable.getCreationMetadata().getTablesUsed());
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MCreationMetadata.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MCreationMetadata.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MCreationMetadata.java
new file mode 100644
index 0000000..1133cb1
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MCreationMetadata.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.model;
+
+import java.util.Set;
+
+/**
+ * Represents the creation metadata of a materialization.
+ * It includes the database and table name for the materialization,
+ * the set of tables that it uses, and the valid transaction list
+ * when it was created.
+ */
+public class MCreationMetadata {
+
+  private String dbName;
+  private String tblName;
+  private Set<MTable> tables;
+  private String txnList;
+
+  public MCreationMetadata() {
+  }
+
+  public MCreationMetadata(String dbName, String tblName,
+      Set<MTable> tables, String txnList) {
+    this.dbName = dbName;
+    this.tblName = tblName;
+    this.tables = tables;
+    this.txnList = txnList;
+  }
+
+  public Set<MTable> getTables() {
+    return tables;
+  }
+
+  public void setTables(Set<MTable> tables) {
+    this.tables = tables;
+  }
+
+  public String getTxnList() {
+    return txnList;
+  }
+
+  public void setTxnList(String txnList) {
+    this.txnList = txnList;
+  }
+
+  public String getDbName() {
+    return dbName;
+  }
+
+  public void setDbName(String dbName) {
+    this.dbName = dbName;
+  }
+
+  public String getTblName() {
+    return tblName;
+  }
+
+  public void setTblName(String tblName) {
+    this.tblName = tblName;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MTable.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MTable.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MTable.java
index 6c40ae8..aea16ad 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MTable.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MTable.java
@@ -35,7 +35,7 @@ public class MTable {
   private String viewOriginalText;
   private String viewExpandedText;
   private boolean rewriteEnabled;
-  private Map<String, String> creationMetadata;
+  private MCreationMetadata creationMetadata;
   private String tableType;
 
   public MTable() {}
@@ -57,7 +57,7 @@ public class MTable {
   public MTable(String tableName, MDatabase database, MStorageDescriptor sd, String owner,
       int createTime, int lastAccessTime, int retention, List<MFieldSchema> partitionKeys,
       Map<String, String> parameters, String viewOriginalText, String viewExpandedText,
-      boolean rewriteEnabled, Map<String, String> creationMetadata,
+      boolean rewriteEnabled, MCreationMetadata creationMetadata,
       String tableType) {
     this.tableName = tableName;
     this.database = database;
@@ -176,14 +176,14 @@ public class MTable {
   /**
    * @return the metadata information related to a materialized view creation
    */
-  public Map<String, String> getCreationMetadata() {
+  public MCreationMetadata getCreationMetadata() {
     return creationMetadata;
   }
 
   /**
    * @param creationMetadata the metadata information to set
    */
-  public void setCreationMetadata(Map<String, String> creationMetadata) {
+  public void setCreationMetadata(MCreationMetadata creationMetadata) {
     this.creationMetadata = creationMetadata;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
index 3a558b4..1bb976c 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
@@ -27,6 +27,7 @@ import java.sql.SQLFeatureNotSupportedException;
 import java.sql.Savepoint;
 import java.sql.Statement;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.BitSet;
 import java.util.Calendar;
 import java.util.Collections;
@@ -54,11 +55,13 @@ import org.apache.commons.dbcp.ConnectionFactory;
 import org.apache.commons.dbcp.DriverManagerConnectionFactory;
 import org.apache.commons.dbcp.PoolableConnectionFactory;
 import org.apache.commons.dbcp.PoolingDataSource;
+import org.apache.commons.lang.ArrayUtils;
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.commons.pool.impl.GenericObjectPool;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.common.classification.RetrySemantics;
 import org.apache.hadoop.hive.metastore.DatabaseProduct;
 import org.apache.hadoop.hive.metastore.MaterializationsInvalidationCache;
@@ -103,7 +106,6 @@ import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
 import org.apache.hadoop.hive.metastore.api.TxnInfo;
 import org.apache.hadoop.hive.metastore.api.TxnOpenException;
 import org.apache.hadoop.hive.metastore.api.TxnState;
-import org.apache.hadoop.hive.metastore.api.TxnsSnapshot;
 import org.apache.hadoop.hive.metastore.api.UnlockRequest;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
@@ -840,13 +842,13 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         dbConn.commit();
 
         // Update registry with modifications
-        s = "select ctc_database, ctc_table, ctc_id, ctc_timestamp from COMPLETED_TXN_COMPONENTS where ctc_txnid = " + txnid;
+        s = "select ctc_database, ctc_table, ctc_timestamp from COMPLETED_TXN_COMPONENTS where ctc_txnid = " + txnid;
         rs = stmt.executeQuery(s);
         if (rs.next()) {
           LOG.debug("Going to register table modification in invalidation cache <" + s + ">");
           MaterializationsInvalidationCache.get().notifyTableModification(
-              rs.getString(1), rs.getString(2), rs.getLong(3),
-              rs.getTimestamp(4, Calendar.getInstance(TimeZone.getTimeZone("UTC"))).getTime());
+              rs.getString(1), rs.getString(2), txnid,
+              rs.getTimestamp(3, Calendar.getInstance(TimeZone.getTimeZone("UTC"))).getTime());
         }
         close(rs);
         dbConn.commit();
@@ -913,73 +915,16 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
   }
 
   /**
-   * Gets the information of the last transaction committed for the input table
-   * given the transaction snapshot provided.
-   */
-  @Override
-  @RetrySemantics.ReadOnly
-  public List<BasicTxnInfo> getLastCompletedTransactionForTables(
-      List<String> dbNames, List<String> tableNames, TxnsSnapshot txnsSnapshot) throws MetaException {
-    List<BasicTxnInfo> r = new ArrayList<>();
-    for (int i = 0; i < dbNames.size(); i++) {
-      r.add(getLastCompletedTransactionForTable(dbNames.get(i), tableNames.get(i), txnsSnapshot));
-    }
-    return r;
-  }
-
-  /**
-   * Gets the information of the last transaction committed for the input table
-   * given the transaction snapshot provided.
-   */
-  @Override
-  @RetrySemantics.ReadOnly
-  public BasicTxnInfo getLastCompletedTransactionForTable(
-      String inputDbName, String inputTableName, TxnsSnapshot txnsSnapshot) throws MetaException {
-    Connection dbConn = null;
-    Statement stmt = null;
-    ResultSet rs = null;
-    try {
-      dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-      stmt = dbConn.createStatement();
-      stmt.setMaxRows(1);
-      String s = "select ctc_id, ctc_timestamp, ctc_txnid, ctc_database, ctc_table "
-          + "from COMPLETED_TXN_COMPONENTS "
-          + "where ctc_database=" + quoteString(inputDbName) + " and ctc_table=" + quoteString(inputTableName)
-          + " and ctc_txnid <= " + txnsSnapshot.getTxn_high_water_mark()
-          + (txnsSnapshot.getOpen_txns().isEmpty() ?
-              " " : " and ctc_txnid NOT IN(" + StringUtils.join(",", txnsSnapshot.getOpen_txns()) + ") ")
-          + "order by ctc_id desc";
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Going to execute query <" + s + ">");
-      }
-      rs = stmt.executeQuery(s);
-      if(!rs.next()) {
-        return new BasicTxnInfo(true);
-      }
-      final BasicTxnInfo txnInfo = new BasicTxnInfo(false);
-      txnInfo.setId(rs.getLong(1));
-      txnInfo.setTime(rs.getTimestamp(2, Calendar.getInstance(TimeZone.getTimeZone("UTC"))).getTime());
-      txnInfo.setTxnid(rs.getLong(3));
-      txnInfo.setDbname(rs.getString(4));
-      txnInfo.setTablename(rs.getString(5));
-      return txnInfo;
-    } catch (SQLException ex) {
-      LOG.warn("getLastCompletedTransactionForTable failed due to " + getMessage(ex), ex);
-      throw new MetaException("Unable to retrieve commits information due to " + StringUtils.stringifyException(ex));
-    } finally {
-      close(rs, stmt, dbConn);
-    }
-  }
-
-  /**
    * Gets the information of the first transaction for the given table
    * after the transaction with the input id was committed (if any). 
    */
   @Override
   @RetrySemantics.ReadOnly
   public BasicTxnInfo getFirstCompletedTransactionForTableAfterCommit(
-      String inputDbName, String inputTableName, long incrementalIdentifier)
+      String inputDbName, String inputTableName, ValidTxnList txnList)
           throws MetaException {
+    final List<Long> openTxns = Arrays.asList(ArrayUtils.toObject(txnList.getInvalidTransactions()));
+
     Connection dbConn = null;
     Statement stmt = null;
     ResultSet rs = null;
@@ -987,23 +932,26 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
       dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
       stmt = dbConn.createStatement();
       stmt.setMaxRows(1);
-      String s = "select ctc_id, ctc_timestamp, ctc_txnid, ctc_database, ctc_table "
+      String s = "select ctc_timestamp, ctc_txnid, ctc_database, ctc_table "
           + "from COMPLETED_TXN_COMPONENTS "
           + "where ctc_database=" + quoteString(inputDbName) + " and ctc_table=" + quoteString(inputTableName)
-          + " and ctc_id > " + incrementalIdentifier + " order by ctc_id asc";
+          + " and ctc_txnid > " + txnList.getHighWatermark()
+          + (txnList.getInvalidTransactions().length == 0 ?
+              " " : " or ctc_txnid IN(" + StringUtils.join(",", openTxns) + ") ")
+          + "order by ctc_timestamp asc";
       if (LOG.isDebugEnabled()) {
         LOG.debug("Going to execute query <" + s + ">");
       }
       rs = stmt.executeQuery(s);
+
       if(!rs.next()) {
         return new BasicTxnInfo(true);
       }
       final BasicTxnInfo txnInfo = new BasicTxnInfo(false);
-      txnInfo.setId(rs.getLong(1));
-      txnInfo.setTime(rs.getTimestamp(2, Calendar.getInstance(TimeZone.getTimeZone("UTC"))).getTime());
-      txnInfo.setTxnid(rs.getLong(3));
-      txnInfo.setDbname(rs.getString(4));
-      txnInfo.setTablename(rs.getString(5));
+      txnInfo.setTime(rs.getTimestamp(1, Calendar.getInstance(TimeZone.getTimeZone("UTC"))).getTime());
+      txnInfo.setTxnid(rs.getLong(2));
+      txnInfo.setDbname(rs.getString(3));
+      txnInfo.setTablename(rs.getString(4));
       return txnInfo;
     } catch (SQLException ex) {
       LOG.warn("getLastCompletedTransactionForTable failed due to " + getMessage(ex), ex);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
index 42f90cd..3e27034 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
@@ -21,6 +21,7 @@ import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.common.classification.RetrySemantics;
 import org.apache.hadoop.hive.metastore.api.*;
 
@@ -113,33 +114,14 @@ public interface TxnStore extends Configurable {
     throws NoSuchTxnException, TxnAbortedException,  MetaException;
 
   /**
-   * Get the last transaction corresponding to given databases and tables.
-   * @return
-   * @throws MetaException
-   */
-  @RetrySemantics.Idempotent
-  public List<BasicTxnInfo> getLastCompletedTransactionForTables(
-      List<String> dbNames, List<String> tableNames, TxnsSnapshot txnsSnapshot)
-          throws MetaException;
-
-  /**
-   * Get the last transaction corresponding to given database and table.
-   * @return
-   * @throws MetaException
-   */
-  @RetrySemantics.Idempotent
-  public BasicTxnInfo getLastCompletedTransactionForTable(
-      String inputDbName, String inputTableName, TxnsSnapshot txnsSnapshot)
-          throws MetaException;
-
-  /**
-   * Get the first transaction corresponding to given database and table after incremental id.
+   * Get the first transaction corresponding to given database and table after transactions
+   * referenced in the transaction snapshot.
    * @return
    * @throws MetaException
    */
   @RetrySemantics.Idempotent
   public BasicTxnInfo getFirstCompletedTransactionForTableAfterCommit(
-      String inputDbName, String inputTableName, long id)
+      String inputDbName, String inputTableName, ValidTxnList txnList)
           throws MetaException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/main/resources/package.jdo
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/resources/package.jdo b/standalone-metastore/src/main/resources/package.jdo
index 3da09a5..f408de5 100644
--- a/standalone-metastore/src/main/resources/package.jdo
+++ b/standalone-metastore/src/main/resources/package.jdo
@@ -25,8 +25,8 @@
   Non-indexed VARCHAR: 4000 bytes (max length on Oracle 9i/10g/11g)
 
 -->
-<jdo>  
-  <package name="org.apache.hadoop.hive.metastore.model">  
+<jdo>
+  <package name="org.apache.hadoop.hive.metastore.model">
     <class name="MDatabase" identity-type="datastore" table="DBS" detachable="true">  
       <datastore-identity>
         <column name="DB_ID"/>
@@ -185,17 +185,27 @@
       <field name="tableType">
         <column name="TBL_TYPE" length="128" jdbc-type="VARCHAR"/>
       </field>
-      <field name="creationMetadata" table="MV_CREATION_METADATA">
-        <map key-type="java.lang.String" value-type="java.lang.String"/>
+    </class>
+
+    <class name="MCreationMetadata" identity-type="datastore" table="MV_CREATION_METADATA" detachable="true">
+      <datastore-identity>
+        <column name="MV_CREATION_METADATA_ID"/>
+      </datastore-identity>
+      <field name="dbName">
+        <column name="DB_NAME" length="128" jdbc-type="VARCHAR"/>
+      </field>
+      <field name="tblName">
+        <column name="TBL_NAME" length="256" jdbc-type="VARCHAR"/>
+      </field>
+      <field name="tables" table="MV_TABLES_USED">
+        <collection element-type="MTable"/>
         <join>
-          <column name="TBL_ID"/>
+          <column name="MV_CREATION_METADATA_ID"/>
         </join>
-        <key>
-           <column name="TBL_NAME" length="180" jdbc-type="VARCHAR"/>
-        </key>
-        <value>
-           <column name="LAST_TRANSACTION_INFO" jdbc-type="LONGVARCHAR"/>
-        </value>
+        <element column="TBL_ID"/>
+      </field>
+      <field name="txnList">
+        <column name="TXN_LIST" jdbc-type="CLOB" allows-null="true"/>
       </field>
     </class>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/main/thrift/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/thrift/hive_metastore.thrift b/standalone-metastore/src/main/thrift/hive_metastore.thrift
index 93f3e53..371b975 100644
--- a/standalone-metastore/src/main/thrift/hive_metastore.thrift
+++ b/standalone-metastore/src/main/thrift/hive_metastore.thrift
@@ -327,7 +327,7 @@ struct Table {
   13: optional PrincipalPrivilegeSet privileges,
   14: optional bool temporary=false,
   15: optional bool rewriteEnabled,     // rewrite enabled or not
-  16: optional map<string, BasicTxnInfo> creationMetadata   // only for MVs, it stores table name used -> last modification before MV creation
+  16: optional CreationMetadata creationMetadata   // only for MVs, it stores table names used and txn list at MV creation
 }
 
 struct Partition {
@@ -858,17 +858,18 @@ struct AddDynamicPartitions {
 
 struct BasicTxnInfo {
     1: required bool isnull,
-    2: optional i64 id,
-    3: optional i64 time,
-    4: optional i64 txnid,
-    5: optional string dbname,
-    6: optional string tablename,
-    7: optional string partitionname
+    2: optional i64 time,
+    3: optional i64 txnid,
+    4: optional string dbname,
+    5: optional string tablename,
+    6: optional string partitionname
 }
 
-struct TxnsSnapshot {
-    1: required i64 txn_high_water_mark,
-    2: required list<i64> open_txns
+struct CreationMetadata {
+    1: required string dbName,
+    2: required string tblName,
+    3: required set<string> tablesUsed,
+    4: optional string validTxnList
 }
 
 struct NotificationEventRequest {
@@ -1815,8 +1816,6 @@ service ThriftHiveMetastore extends fb303.FacebookService
   CompactionResponse compact2(1:CompactionRequest rqst) 
   ShowCompactResponse show_compact(1:ShowCompactRequest rqst)
   void add_dynamic_partitions(1:AddDynamicPartitions rqst) throws (1:NoSuchTxnException o1, 2:TxnAbortedException o2)
-  list<BasicTxnInfo> get_last_completed_transaction_for_tables(1:list<string> db_names, 2:list<string> table_names, 3:TxnsSnapshot txns_snapshot)
-  BasicTxnInfo get_last_completed_transaction_for_table(1:string db_name, 2:string table_name, 3:TxnsSnapshot txns_snapshot)
 
   // Notification logging calls
   NotificationEventResponse get_next_notification(1:NotificationEventRequest rqst) 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java
index b9a8f61..bd61df6 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java
@@ -225,7 +225,6 @@ public class TestCachedStore {
     Table tbl1 =
         new Table(tblName1, dbName, tblOwner, 0, 0, 0, sd, new ArrayList<>(), tblParams,
             null, null, TableType.MANAGED_TABLE.toString());
-    tbl1.setCreationMetadata(new HashMap<String, BasicTxnInfo>());
     cachedStore.createTable(tbl1);
     tbl1 = cachedStore.getTable(dbName, tblName1);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestGetTableMeta.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestGetTableMeta.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestGetTableMeta.java
index 71cac2f..dcf1eb7 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestGetTableMeta.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestGetTableMeta.java
@@ -23,8 +23,10 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Set;
 
+import com.google.common.collect.ImmutableSet;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.metastore.api.CreationMetadata;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.TableMeta;
@@ -149,7 +151,9 @@ public class TestGetTableMeta {
 
 
     if (type == TableType.MATERIALIZED_VIEW) {
-      table.setCreationMetadata(new HashMap<>());
+      CreationMetadata cm = new CreationMetadata(
+          dbName, tableName, ImmutableSet.of());
+      table.setCreationMetadata(cm);
     }
 
     if (type == TableType.EXTERNAL_TABLE) {

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java
index abc400a..00f38ee 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java
@@ -286,7 +286,7 @@ public class TestTablesCreateDropAlterTruncate {
     Assert.assertNull("Comparing ViewOriginalText", createdTable.getViewOriginalText());
     Assert.assertNull("Comparing ViewExpandedText", createdTable.getViewExpandedText());
     Assert.assertEquals("Comparing TableType", "MANAGED_TABLE", createdTable.getTableType());
-    Assert.assertTrue("Creation metadata should be empty", createdTable.getCreationMetadata().isEmpty());
+    Assert.assertTrue("Creation metadata should be empty", createdTable.getCreationMetadata() == null);
 
     // Storage Descriptor data
     StorageDescriptor createdSd = createdTable.getSd();


[02/16] hive git commit: HIVE-18546: Remove unnecessary code introduced in HIVE-14498 (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index 25e9a88..5598859 100644
--- a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -3487,7 +3487,7 @@ class Table:
     (13, TType.STRUCT, 'privileges', (PrincipalPrivilegeSet, PrincipalPrivilegeSet.thrift_spec), None, ), # 13
     (14, TType.BOOL, 'temporary', None, False, ), # 14
     (15, TType.BOOL, 'rewriteEnabled', None, None, ), # 15
-    (16, TType.MAP, 'creationMetadata', (TType.STRING,None,TType.STRUCT,(BasicTxnInfo, BasicTxnInfo.thrift_spec)), None, ), # 16
+    (16, TType.STRUCT, 'creationMetadata', (CreationMetadata, CreationMetadata.thrift_spec), None, ), # 16
   )
 
   def __init__(self, tableName=None, dbName=None, owner=None, createTime=None, lastAccessTime=None, retention=None, sd=None, partitionKeys=None, parameters=None, viewOriginalText=None, viewExpandedText=None, tableType=None, privileges=None, temporary=thrift_spec[14][4], rewriteEnabled=None, creationMetadata=None,):
@@ -3607,15 +3607,9 @@ class Table:
         else:
           iprot.skip(ftype)
       elif fid == 16:
-        if ftype == TType.MAP:
-          self.creationMetadata = {}
-          (_ktype182, _vtype183, _size181 ) = iprot.readMapBegin()
-          for _i185 in xrange(_size181):
-            _key186 = iprot.readString()
-            _val187 = BasicTxnInfo()
-            _val187.read(iprot)
-            self.creationMetadata[_key186] = _val187
-          iprot.readMapEnd()
+        if ftype == TType.STRUCT:
+          self.creationMetadata = CreationMetadata()
+          self.creationMetadata.read(iprot)
         else:
           iprot.skip(ftype)
       else:
@@ -3659,16 +3653,16 @@ class Table:
     if self.partitionKeys is not None:
       oprot.writeFieldBegin('partitionKeys', TType.LIST, 8)
       oprot.writeListBegin(TType.STRUCT, len(self.partitionKeys))
-      for iter188 in self.partitionKeys:
-        iter188.write(oprot)
+      for iter181 in self.partitionKeys:
+        iter181.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.parameters is not None:
       oprot.writeFieldBegin('parameters', TType.MAP, 9)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.parameters))
-      for kiter189,viter190 in self.parameters.items():
-        oprot.writeString(kiter189)
-        oprot.writeString(viter190)
+      for kiter182,viter183 in self.parameters.items():
+        oprot.writeString(kiter182)
+        oprot.writeString(viter183)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.viewOriginalText is not None:
@@ -3696,12 +3690,8 @@ class Table:
       oprot.writeBool(self.rewriteEnabled)
       oprot.writeFieldEnd()
     if self.creationMetadata is not None:
-      oprot.writeFieldBegin('creationMetadata', TType.MAP, 16)
-      oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.creationMetadata))
-      for kiter191,viter192 in self.creationMetadata.items():
-        oprot.writeString(kiter191)
-        viter192.write(oprot)
-      oprot.writeMapEnd()
+      oprot.writeFieldBegin('creationMetadata', TType.STRUCT, 16)
+      self.creationMetadata.write(oprot)
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
@@ -3788,10 +3778,10 @@ class Partition:
       if fid == 1:
         if ftype == TType.LIST:
           self.values = []
-          (_etype196, _size193) = iprot.readListBegin()
-          for _i197 in xrange(_size193):
-            _elem198 = iprot.readString()
-            self.values.append(_elem198)
+          (_etype187, _size184) = iprot.readListBegin()
+          for _i188 in xrange(_size184):
+            _elem189 = iprot.readString()
+            self.values.append(_elem189)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -3824,11 +3814,11 @@ class Partition:
       elif fid == 7:
         if ftype == TType.MAP:
           self.parameters = {}
-          (_ktype200, _vtype201, _size199 ) = iprot.readMapBegin()
-          for _i203 in xrange(_size199):
-            _key204 = iprot.readString()
-            _val205 = iprot.readString()
-            self.parameters[_key204] = _val205
+          (_ktype191, _vtype192, _size190 ) = iprot.readMapBegin()
+          for _i194 in xrange(_size190):
+            _key195 = iprot.readString()
+            _val196 = iprot.readString()
+            self.parameters[_key195] = _val196
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -3851,8 +3841,8 @@ class Partition:
     if self.values is not None:
       oprot.writeFieldBegin('values', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.values))
-      for iter206 in self.values:
-        oprot.writeString(iter206)
+      for iter197 in self.values:
+        oprot.writeString(iter197)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.dbName is not None:
@@ -3878,9 +3868,9 @@ class Partition:
     if self.parameters is not None:
       oprot.writeFieldBegin('parameters', TType.MAP, 7)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.parameters))
-      for kiter207,viter208 in self.parameters.items():
-        oprot.writeString(kiter207)
-        oprot.writeString(viter208)
+      for kiter198,viter199 in self.parameters.items():
+        oprot.writeString(kiter198)
+        oprot.writeString(viter199)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.privileges is not None:
@@ -3958,10 +3948,10 @@ class PartitionWithoutSD:
       if fid == 1:
         if ftype == TType.LIST:
           self.values = []
-          (_etype212, _size209) = iprot.readListBegin()
-          for _i213 in xrange(_size209):
-            _elem214 = iprot.readString()
-            self.values.append(_elem214)
+          (_etype203, _size200) = iprot.readListBegin()
+          for _i204 in xrange(_size200):
+            _elem205 = iprot.readString()
+            self.values.append(_elem205)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -3983,11 +3973,11 @@ class PartitionWithoutSD:
       elif fid == 5:
         if ftype == TType.MAP:
           self.parameters = {}
-          (_ktype216, _vtype217, _size215 ) = iprot.readMapBegin()
-          for _i219 in xrange(_size215):
-            _key220 = iprot.readString()
-            _val221 = iprot.readString()
-            self.parameters[_key220] = _val221
+          (_ktype207, _vtype208, _size206 ) = iprot.readMapBegin()
+          for _i210 in xrange(_size206):
+            _key211 = iprot.readString()
+            _val212 = iprot.readString()
+            self.parameters[_key211] = _val212
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -4010,8 +4000,8 @@ class PartitionWithoutSD:
     if self.values is not None:
       oprot.writeFieldBegin('values', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.values))
-      for iter222 in self.values:
-        oprot.writeString(iter222)
+      for iter213 in self.values:
+        oprot.writeString(iter213)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.createTime is not None:
@@ -4029,9 +4019,9 @@ class PartitionWithoutSD:
     if self.parameters is not None:
       oprot.writeFieldBegin('parameters', TType.MAP, 5)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.parameters))
-      for kiter223,viter224 in self.parameters.items():
-        oprot.writeString(kiter223)
-        oprot.writeString(viter224)
+      for kiter214,viter215 in self.parameters.items():
+        oprot.writeString(kiter214)
+        oprot.writeString(viter215)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.privileges is not None:
@@ -4095,11 +4085,11 @@ class PartitionSpecWithSharedSD:
       if fid == 1:
         if ftype == TType.LIST:
           self.partitions = []
-          (_etype228, _size225) = iprot.readListBegin()
-          for _i229 in xrange(_size225):
-            _elem230 = PartitionWithoutSD()
-            _elem230.read(iprot)
-            self.partitions.append(_elem230)
+          (_etype219, _size216) = iprot.readListBegin()
+          for _i220 in xrange(_size216):
+            _elem221 = PartitionWithoutSD()
+            _elem221.read(iprot)
+            self.partitions.append(_elem221)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -4122,8 +4112,8 @@ class PartitionSpecWithSharedSD:
     if self.partitions is not None:
       oprot.writeFieldBegin('partitions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.partitions))
-      for iter231 in self.partitions:
-        iter231.write(oprot)
+      for iter222 in self.partitions:
+        iter222.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.sd is not None:
@@ -4180,11 +4170,11 @@ class PartitionListComposingSpec:
       if fid == 1:
         if ftype == TType.LIST:
           self.partitions = []
-          (_etype235, _size232) = iprot.readListBegin()
-          for _i236 in xrange(_size232):
-            _elem237 = Partition()
-            _elem237.read(iprot)
-            self.partitions.append(_elem237)
+          (_etype226, _size223) = iprot.readListBegin()
+          for _i227 in xrange(_size223):
+            _elem228 = Partition()
+            _elem228.read(iprot)
+            self.partitions.append(_elem228)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -4201,8 +4191,8 @@ class PartitionListComposingSpec:
     if self.partitions is not None:
       oprot.writeFieldBegin('partitions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.partitions))
-      for iter238 in self.partitions:
-        iter238.write(oprot)
+      for iter229 in self.partitions:
+        iter229.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -4441,11 +4431,11 @@ class Index:
       elif fid == 9:
         if ftype == TType.MAP:
           self.parameters = {}
-          (_ktype240, _vtype241, _size239 ) = iprot.readMapBegin()
-          for _i243 in xrange(_size239):
-            _key244 = iprot.readString()
-            _val245 = iprot.readString()
-            self.parameters[_key244] = _val245
+          (_ktype231, _vtype232, _size230 ) = iprot.readMapBegin()
+          for _i234 in xrange(_size230):
+            _key235 = iprot.readString()
+            _val236 = iprot.readString()
+            self.parameters[_key235] = _val236
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -4499,9 +4489,9 @@ class Index:
     if self.parameters is not None:
       oprot.writeFieldBegin('parameters', TType.MAP, 9)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.parameters))
-      for kiter246,viter247 in self.parameters.items():
-        oprot.writeString(kiter246)
-        oprot.writeString(viter247)
+      for kiter237,viter238 in self.parameters.items():
+        oprot.writeString(kiter237)
+        oprot.writeString(viter238)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.deferredRebuild is not None:
@@ -5929,11 +5919,11 @@ class ColumnStatistics:
       elif fid == 2:
         if ftype == TType.LIST:
           self.statsObj = []
-          (_etype251, _size248) = iprot.readListBegin()
-          for _i252 in xrange(_size248):
-            _elem253 = ColumnStatisticsObj()
-            _elem253.read(iprot)
-            self.statsObj.append(_elem253)
+          (_etype242, _size239) = iprot.readListBegin()
+          for _i243 in xrange(_size239):
+            _elem244 = ColumnStatisticsObj()
+            _elem244.read(iprot)
+            self.statsObj.append(_elem244)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -5954,8 +5944,8 @@ class ColumnStatistics:
     if self.statsObj is not None:
       oprot.writeFieldBegin('statsObj', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.statsObj))
-      for iter254 in self.statsObj:
-        iter254.write(oprot)
+      for iter245 in self.statsObj:
+        iter245.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -6015,11 +6005,11 @@ class AggrStats:
       if fid == 1:
         if ftype == TType.LIST:
           self.colStats = []
-          (_etype258, _size255) = iprot.readListBegin()
-          for _i259 in xrange(_size255):
-            _elem260 = ColumnStatisticsObj()
-            _elem260.read(iprot)
-            self.colStats.append(_elem260)
+          (_etype249, _size246) = iprot.readListBegin()
+          for _i250 in xrange(_size246):
+            _elem251 = ColumnStatisticsObj()
+            _elem251.read(iprot)
+            self.colStats.append(_elem251)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -6041,8 +6031,8 @@ class AggrStats:
     if self.colStats is not None:
       oprot.writeFieldBegin('colStats', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.colStats))
-      for iter261 in self.colStats:
-        iter261.write(oprot)
+      for iter252 in self.colStats:
+        iter252.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.partsFound is not None:
@@ -6106,11 +6096,11 @@ class SetPartitionsStatsRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.colStats = []
-          (_etype265, _size262) = iprot.readListBegin()
-          for _i266 in xrange(_size262):
-            _elem267 = ColumnStatistics()
-            _elem267.read(iprot)
-            self.colStats.append(_elem267)
+          (_etype256, _size253) = iprot.readListBegin()
+          for _i257 in xrange(_size253):
+            _elem258 = ColumnStatistics()
+            _elem258.read(iprot)
+            self.colStats.append(_elem258)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -6132,8 +6122,8 @@ class SetPartitionsStatsRequest:
     if self.colStats is not None:
       oprot.writeFieldBegin('colStats', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.colStats))
-      for iter268 in self.colStats:
-        iter268.write(oprot)
+      for iter259 in self.colStats:
+        iter259.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.needMerge is not None:
@@ -6195,22 +6185,22 @@ class Schema:
       if fid == 1:
         if ftype == TType.LIST:
           self.fieldSchemas = []
-          (_etype272, _size269) = iprot.readListBegin()
-          for _i273 in xrange(_size269):
-            _elem274 = FieldSchema()
-            _elem274.read(iprot)
-            self.fieldSchemas.append(_elem274)
+          (_etype263, _size260) = iprot.readListBegin()
+          for _i264 in xrange(_size260):
+            _elem265 = FieldSchema()
+            _elem265.read(iprot)
+            self.fieldSchemas.append(_elem265)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.MAP:
           self.properties = {}
-          (_ktype276, _vtype277, _size275 ) = iprot.readMapBegin()
-          for _i279 in xrange(_size275):
-            _key280 = iprot.readString()
-            _val281 = iprot.readString()
-            self.properties[_key280] = _val281
+          (_ktype267, _vtype268, _size266 ) = iprot.readMapBegin()
+          for _i270 in xrange(_size266):
+            _key271 = iprot.readString()
+            _val272 = iprot.readString()
+            self.properties[_key271] = _val272
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -6227,16 +6217,16 @@ class Schema:
     if self.fieldSchemas is not None:
       oprot.writeFieldBegin('fieldSchemas', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.fieldSchemas))
-      for iter282 in self.fieldSchemas:
-        iter282.write(oprot)
+      for iter273 in self.fieldSchemas:
+        iter273.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.properties is not None:
       oprot.writeFieldBegin('properties', TType.MAP, 2)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.properties))
-      for kiter283,viter284 in self.properties.items():
-        oprot.writeString(kiter283)
-        oprot.writeString(viter284)
+      for kiter274,viter275 in self.properties.items():
+        oprot.writeString(kiter274)
+        oprot.writeString(viter275)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -6289,11 +6279,11 @@ class EnvironmentContext:
       if fid == 1:
         if ftype == TType.MAP:
           self.properties = {}
-          (_ktype286, _vtype287, _size285 ) = iprot.readMapBegin()
-          for _i289 in xrange(_size285):
-            _key290 = iprot.readString()
-            _val291 = iprot.readString()
-            self.properties[_key290] = _val291
+          (_ktype277, _vtype278, _size276 ) = iprot.readMapBegin()
+          for _i280 in xrange(_size276):
+            _key281 = iprot.readString()
+            _val282 = iprot.readString()
+            self.properties[_key281] = _val282
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -6310,9 +6300,9 @@ class EnvironmentContext:
     if self.properties is not None:
       oprot.writeFieldBegin('properties', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.properties))
-      for kiter292,viter293 in self.properties.items():
-        oprot.writeString(kiter292)
-        oprot.writeString(viter293)
+      for kiter283,viter284 in self.properties.items():
+        oprot.writeString(kiter283)
+        oprot.writeString(viter284)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -6446,11 +6436,11 @@ class PrimaryKeysResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.primaryKeys = []
-          (_etype297, _size294) = iprot.readListBegin()
-          for _i298 in xrange(_size294):
-            _elem299 = SQLPrimaryKey()
-            _elem299.read(iprot)
-            self.primaryKeys.append(_elem299)
+          (_etype288, _size285) = iprot.readListBegin()
+          for _i289 in xrange(_size285):
+            _elem290 = SQLPrimaryKey()
+            _elem290.read(iprot)
+            self.primaryKeys.append(_elem290)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -6467,8 +6457,8 @@ class PrimaryKeysResponse:
     if self.primaryKeys is not None:
       oprot.writeFieldBegin('primaryKeys', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.primaryKeys))
-      for iter300 in self.primaryKeys:
-        iter300.write(oprot)
+      for iter291 in self.primaryKeys:
+        iter291.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -6626,11 +6616,11 @@ class ForeignKeysResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.foreignKeys = []
-          (_etype304, _size301) = iprot.readListBegin()
-          for _i305 in xrange(_size301):
-            _elem306 = SQLForeignKey()
-            _elem306.read(iprot)
-            self.foreignKeys.append(_elem306)
+          (_etype295, _size292) = iprot.readListBegin()
+          for _i296 in xrange(_size292):
+            _elem297 = SQLForeignKey()
+            _elem297.read(iprot)
+            self.foreignKeys.append(_elem297)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -6647,8 +6637,8 @@ class ForeignKeysResponse:
     if self.foreignKeys is not None:
       oprot.writeFieldBegin('foreignKeys', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.foreignKeys))
-      for iter307 in self.foreignKeys:
-        iter307.write(oprot)
+      for iter298 in self.foreignKeys:
+        iter298.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -6784,11 +6774,11 @@ class UniqueConstraintsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.uniqueConstraints = []
-          (_etype311, _size308) = iprot.readListBegin()
-          for _i312 in xrange(_size308):
-            _elem313 = SQLUniqueConstraint()
-            _elem313.read(iprot)
-            self.uniqueConstraints.append(_elem313)
+          (_etype302, _size299) = iprot.readListBegin()
+          for _i303 in xrange(_size299):
+            _elem304 = SQLUniqueConstraint()
+            _elem304.read(iprot)
+            self.uniqueConstraints.append(_elem304)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -6805,8 +6795,8 @@ class UniqueConstraintsResponse:
     if self.uniqueConstraints is not None:
       oprot.writeFieldBegin('uniqueConstraints', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.uniqueConstraints))
-      for iter314 in self.uniqueConstraints:
-        iter314.write(oprot)
+      for iter305 in self.uniqueConstraints:
+        iter305.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -6942,11 +6932,11 @@ class NotNullConstraintsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.notNullConstraints = []
-          (_etype318, _size315) = iprot.readListBegin()
-          for _i319 in xrange(_size315):
-            _elem320 = SQLNotNullConstraint()
-            _elem320.read(iprot)
-            self.notNullConstraints.append(_elem320)
+          (_etype309, _size306) = iprot.readListBegin()
+          for _i310 in xrange(_size306):
+            _elem311 = SQLNotNullConstraint()
+            _elem311.read(iprot)
+            self.notNullConstraints.append(_elem311)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -6963,8 +6953,8 @@ class NotNullConstraintsResponse:
     if self.notNullConstraints is not None:
       oprot.writeFieldBegin('notNullConstraints', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.notNullConstraints))
-      for iter321 in self.notNullConstraints:
-        iter321.write(oprot)
+      for iter312 in self.notNullConstraints:
+        iter312.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -7115,11 +7105,11 @@ class AddPrimaryKeyRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.primaryKeyCols = []
-          (_etype325, _size322) = iprot.readListBegin()
-          for _i326 in xrange(_size322):
-            _elem327 = SQLPrimaryKey()
-            _elem327.read(iprot)
-            self.primaryKeyCols.append(_elem327)
+          (_etype316, _size313) = iprot.readListBegin()
+          for _i317 in xrange(_size313):
+            _elem318 = SQLPrimaryKey()
+            _elem318.read(iprot)
+            self.primaryKeyCols.append(_elem318)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -7136,8 +7126,8 @@ class AddPrimaryKeyRequest:
     if self.primaryKeyCols is not None:
       oprot.writeFieldBegin('primaryKeyCols', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.primaryKeyCols))
-      for iter328 in self.primaryKeyCols:
-        iter328.write(oprot)
+      for iter319 in self.primaryKeyCols:
+        iter319.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -7191,11 +7181,11 @@ class AddForeignKeyRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.foreignKeyCols = []
-          (_etype332, _size329) = iprot.readListBegin()
-          for _i333 in xrange(_size329):
-            _elem334 = SQLForeignKey()
-            _elem334.read(iprot)
-            self.foreignKeyCols.append(_elem334)
+          (_etype323, _size320) = iprot.readListBegin()
+          for _i324 in xrange(_size320):
+            _elem325 = SQLForeignKey()
+            _elem325.read(iprot)
+            self.foreignKeyCols.append(_elem325)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -7212,8 +7202,8 @@ class AddForeignKeyRequest:
     if self.foreignKeyCols is not None:
       oprot.writeFieldBegin('foreignKeyCols', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.foreignKeyCols))
-      for iter335 in self.foreignKeyCols:
-        iter335.write(oprot)
+      for iter326 in self.foreignKeyCols:
+        iter326.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -7267,11 +7257,11 @@ class AddUniqueConstraintRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.uniqueConstraintCols = []
-          (_etype339, _size336) = iprot.readListBegin()
-          for _i340 in xrange(_size336):
-            _elem341 = SQLUniqueConstraint()
-            _elem341.read(iprot)
-            self.uniqueConstraintCols.append(_elem341)
+          (_etype330, _size327) = iprot.readListBegin()
+          for _i331 in xrange(_size327):
+            _elem332 = SQLUniqueConstraint()
+            _elem332.read(iprot)
+            self.uniqueConstraintCols.append(_elem332)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -7288,8 +7278,8 @@ class AddUniqueConstraintRequest:
     if self.uniqueConstraintCols is not None:
       oprot.writeFieldBegin('uniqueConstraintCols', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.uniqueConstraintCols))
-      for iter342 in self.uniqueConstraintCols:
-        iter342.write(oprot)
+      for iter333 in self.uniqueConstraintCols:
+        iter333.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -7343,11 +7333,11 @@ class AddNotNullConstraintRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.notNullConstraintCols = []
-          (_etype346, _size343) = iprot.readListBegin()
-          for _i347 in xrange(_size343):
-            _elem348 = SQLNotNullConstraint()
-            _elem348.read(iprot)
-            self.notNullConstraintCols.append(_elem348)
+          (_etype337, _size334) = iprot.readListBegin()
+          for _i338 in xrange(_size334):
+            _elem339 = SQLNotNullConstraint()
+            _elem339.read(iprot)
+            self.notNullConstraintCols.append(_elem339)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -7364,8 +7354,8 @@ class AddNotNullConstraintRequest:
     if self.notNullConstraintCols is not None:
       oprot.writeFieldBegin('notNullConstraintCols', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.notNullConstraintCols))
-      for iter349 in self.notNullConstraintCols:
-        iter349.write(oprot)
+      for iter340 in self.notNullConstraintCols:
+        iter340.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -7422,11 +7412,11 @@ class PartitionsByExprResult:
       if fid == 1:
         if ftype == TType.LIST:
           self.partitions = []
-          (_etype353, _size350) = iprot.readListBegin()
-          for _i354 in xrange(_size350):
-            _elem355 = Partition()
-            _elem355.read(iprot)
-            self.partitions.append(_elem355)
+          (_etype344, _size341) = iprot.readListBegin()
+          for _i345 in xrange(_size341):
+            _elem346 = Partition()
+            _elem346.read(iprot)
+            self.partitions.append(_elem346)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -7448,8 +7438,8 @@ class PartitionsByExprResult:
     if self.partitions is not None:
       oprot.writeFieldBegin('partitions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.partitions))
-      for iter356 in self.partitions:
-        iter356.write(oprot)
+      for iter347 in self.partitions:
+        iter347.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.hasUnknownPartitions is not None:
@@ -7633,11 +7623,11 @@ class TableStatsResult:
       if fid == 1:
         if ftype == TType.LIST:
           self.tableStats = []
-          (_etype360, _size357) = iprot.readListBegin()
-          for _i361 in xrange(_size357):
-            _elem362 = ColumnStatisticsObj()
-            _elem362.read(iprot)
-            self.tableStats.append(_elem362)
+          (_etype351, _size348) = iprot.readListBegin()
+          for _i352 in xrange(_size348):
+            _elem353 = ColumnStatisticsObj()
+            _elem353.read(iprot)
+            self.tableStats.append(_elem353)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -7654,8 +7644,8 @@ class TableStatsResult:
     if self.tableStats is not None:
       oprot.writeFieldBegin('tableStats', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.tableStats))
-      for iter363 in self.tableStats:
-        iter363.write(oprot)
+      for iter354 in self.tableStats:
+        iter354.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -7709,17 +7699,17 @@ class PartitionsStatsResult:
       if fid == 1:
         if ftype == TType.MAP:
           self.partStats = {}
-          (_ktype365, _vtype366, _size364 ) = iprot.readMapBegin()
-          for _i368 in xrange(_size364):
-            _key369 = iprot.readString()
-            _val370 = []
-            (_etype374, _size371) = iprot.readListBegin()
-            for _i375 in xrange(_size371):
-              _elem376 = ColumnStatisticsObj()
-              _elem376.read(iprot)
-              _val370.append(_elem376)
+          (_ktype356, _vtype357, _size355 ) = iprot.readMapBegin()
+          for _i359 in xrange(_size355):
+            _key360 = iprot.readString()
+            _val361 = []
+            (_etype365, _size362) = iprot.readListBegin()
+            for _i366 in xrange(_size362):
+              _elem367 = ColumnStatisticsObj()
+              _elem367.read(iprot)
+              _val361.append(_elem367)
             iprot.readListEnd()
-            self.partStats[_key369] = _val370
+            self.partStats[_key360] = _val361
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -7736,11 +7726,11 @@ class PartitionsStatsResult:
     if self.partStats is not None:
       oprot.writeFieldBegin('partStats', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.LIST, len(self.partStats))
-      for kiter377,viter378 in self.partStats.items():
-        oprot.writeString(kiter377)
-        oprot.writeListBegin(TType.STRUCT, len(viter378))
-        for iter379 in viter378:
-          iter379.write(oprot)
+      for kiter368,viter369 in self.partStats.items():
+        oprot.writeString(kiter368)
+        oprot.writeListBegin(TType.STRUCT, len(viter369))
+        for iter370 in viter369:
+          iter370.write(oprot)
         oprot.writeListEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
@@ -7811,10 +7801,10 @@ class TableStatsRequest:
       elif fid == 3:
         if ftype == TType.LIST:
           self.colNames = []
-          (_etype383, _size380) = iprot.readListBegin()
-          for _i384 in xrange(_size380):
-            _elem385 = iprot.readString()
-            self.colNames.append(_elem385)
+          (_etype374, _size371) = iprot.readListBegin()
+          for _i375 in xrange(_size371):
+            _elem376 = iprot.readString()
+            self.colNames.append(_elem376)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -7839,8 +7829,8 @@ class TableStatsRequest:
     if self.colNames is not None:
       oprot.writeFieldBegin('colNames', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.colNames))
-      for iter386 in self.colNames:
-        oprot.writeString(iter386)
+      for iter377 in self.colNames:
+        oprot.writeString(iter377)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -7919,20 +7909,20 @@ class PartitionsStatsRequest:
       elif fid == 3:
         if ftype == TType.LIST:
           self.colNames = []
-          (_etype390, _size387) = iprot.readListBegin()
-          for _i391 in xrange(_size387):
-            _elem392 = iprot.readString()
-            self.colNames.append(_elem392)
+          (_etype381, _size378) = iprot.readListBegin()
+          for _i382 in xrange(_size378):
+            _elem383 = iprot.readString()
+            self.colNames.append(_elem383)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.LIST:
           self.partNames = []
-          (_etype396, _size393) = iprot.readListBegin()
-          for _i397 in xrange(_size393):
-            _elem398 = iprot.readString()
-            self.partNames.append(_elem398)
+          (_etype387, _size384) = iprot.readListBegin()
+          for _i388 in xrange(_size384):
+            _elem389 = iprot.readString()
+            self.partNames.append(_elem389)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -7957,15 +7947,15 @@ class PartitionsStatsRequest:
     if self.colNames is not None:
       oprot.writeFieldBegin('colNames', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.colNames))
-      for iter399 in self.colNames:
-        oprot.writeString(iter399)
+      for iter390 in self.colNames:
+        oprot.writeString(iter390)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.partNames is not None:
       oprot.writeFieldBegin('partNames', TType.LIST, 4)
       oprot.writeListBegin(TType.STRING, len(self.partNames))
-      for iter400 in self.partNames:
-        oprot.writeString(iter400)
+      for iter391 in self.partNames:
+        oprot.writeString(iter391)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -8028,11 +8018,11 @@ class AddPartitionsResult:
       if fid == 1:
         if ftype == TType.LIST:
           self.partitions = []
-          (_etype404, _size401) = iprot.readListBegin()
-          for _i405 in xrange(_size401):
-            _elem406 = Partition()
-            _elem406.read(iprot)
-            self.partitions.append(_elem406)
+          (_etype395, _size392) = iprot.readListBegin()
+          for _i396 in xrange(_size392):
+            _elem397 = Partition()
+            _elem397.read(iprot)
+            self.partitions.append(_elem397)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -8049,8 +8039,8 @@ class AddPartitionsResult:
     if self.partitions is not None:
       oprot.writeFieldBegin('partitions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.partitions))
-      for iter407 in self.partitions:
-        iter407.write(oprot)
+      for iter398 in self.partitions:
+        iter398.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -8124,11 +8114,11 @@ class AddPartitionsRequest:
       elif fid == 3:
         if ftype == TType.LIST:
           self.parts = []
-          (_etype411, _size408) = iprot.readListBegin()
-          for _i412 in xrange(_size408):
-            _elem413 = Partition()
-            _elem413.read(iprot)
-            self.parts.append(_elem413)
+          (_etype402, _size399) = iprot.readListBegin()
+          for _i403 in xrange(_size399):
+            _elem404 = Partition()
+            _elem404.read(iprot)
+            self.parts.append(_elem404)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -8163,8 +8153,8 @@ class AddPartitionsRequest:
     if self.parts is not None:
       oprot.writeFieldBegin('parts', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.parts))
-      for iter414 in self.parts:
-        iter414.write(oprot)
+      for iter405 in self.parts:
+        iter405.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.ifNotExists is not None:
@@ -8236,11 +8226,11 @@ class DropPartitionsResult:
       if fid == 1:
         if ftype == TType.LIST:
           self.partitions = []
-          (_etype418, _size415) = iprot.readListBegin()
-          for _i419 in xrange(_size415):
-            _elem420 = Partition()
-            _elem420.read(iprot)
-            self.partitions.append(_elem420)
+          (_etype409, _size406) = iprot.readListBegin()
+          for _i410 in xrange(_size406):
+            _elem411 = Partition()
+            _elem411.read(iprot)
+            self.partitions.append(_elem411)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -8257,8 +8247,8 @@ class DropPartitionsResult:
     if self.partitions is not None:
       oprot.writeFieldBegin('partitions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.partitions))
-      for iter421 in self.partitions:
-        iter421.write(oprot)
+      for iter412 in self.partitions:
+        iter412.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -8393,21 +8383,21 @@ class RequestPartsSpec:
       if fid == 1:
         if ftype == TType.LIST:
           self.names = []
-          (_etype425, _size422) = iprot.readListBegin()
-          for _i426 in xrange(_size422):
-            _elem427 = iprot.readString()
-            self.names.append(_elem427)
+          (_etype416, _size413) = iprot.readListBegin()
+          for _i417 in xrange(_size413):
+            _elem418 = iprot.readString()
+            self.names.append(_elem418)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.LIST:
           self.exprs = []
-          (_etype431, _size428) = iprot.readListBegin()
-          for _i432 in xrange(_size428):
-            _elem433 = DropPartitionsExpr()
-            _elem433.read(iprot)
-            self.exprs.append(_elem433)
+          (_etype422, _size419) = iprot.readListBegin()
+          for _i423 in xrange(_size419):
+            _elem424 = DropPartitionsExpr()
+            _elem424.read(iprot)
+            self.exprs.append(_elem424)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -8424,15 +8414,15 @@ class RequestPartsSpec:
     if self.names is not None:
       oprot.writeFieldBegin('names', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.names))
-      for iter434 in self.names:
-        oprot.writeString(iter434)
+      for iter425 in self.names:
+        oprot.writeString(iter425)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.exprs is not None:
       oprot.writeFieldBegin('exprs', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.exprs))
-      for iter435 in self.exprs:
-        iter435.write(oprot)
+      for iter426 in self.exprs:
+        iter426.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -8680,11 +8670,11 @@ class PartitionValuesRequest:
       elif fid == 3:
         if ftype == TType.LIST:
           self.partitionKeys = []
-          (_etype439, _size436) = iprot.readListBegin()
-          for _i440 in xrange(_size436):
-            _elem441 = FieldSchema()
-            _elem441.read(iprot)
-            self.partitionKeys.append(_elem441)
+          (_etype430, _size427) = iprot.readListBegin()
+          for _i431 in xrange(_size427):
+            _elem432 = FieldSchema()
+            _elem432.read(iprot)
+            self.partitionKeys.append(_elem432)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -8701,11 +8691,11 @@ class PartitionValuesRequest:
       elif fid == 6:
         if ftype == TType.LIST:
           self.partitionOrder = []
-          (_etype445, _size442) = iprot.readListBegin()
-          for _i446 in xrange(_size442):
-            _elem447 = FieldSchema()
-            _elem447.read(iprot)
-            self.partitionOrder.append(_elem447)
+          (_etype436, _size433) = iprot.readListBegin()
+          for _i437 in xrange(_size433):
+            _elem438 = FieldSchema()
+            _elem438.read(iprot)
+            self.partitionOrder.append(_elem438)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -8740,8 +8730,8 @@ class PartitionValuesRequest:
     if self.partitionKeys is not None:
       oprot.writeFieldBegin('partitionKeys', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.partitionKeys))
-      for iter448 in self.partitionKeys:
-        iter448.write(oprot)
+      for iter439 in self.partitionKeys:
+        iter439.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.applyDistinct is not None:
@@ -8755,8 +8745,8 @@ class PartitionValuesRequest:
     if self.partitionOrder is not None:
       oprot.writeFieldBegin('partitionOrder', TType.LIST, 6)
       oprot.writeListBegin(TType.STRUCT, len(self.partitionOrder))
-      for iter449 in self.partitionOrder:
-        iter449.write(oprot)
+      for iter440 in self.partitionOrder:
+        iter440.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.ascending is not None:
@@ -8829,10 +8819,10 @@ class PartitionValuesRow:
       if fid == 1:
         if ftype == TType.LIST:
           self.row = []
-          (_etype453, _size450) = iprot.readListBegin()
-          for _i454 in xrange(_size450):
-            _elem455 = iprot.readString()
-            self.row.append(_elem455)
+          (_etype444, _size441) = iprot.readListBegin()
+          for _i445 in xrange(_size441):
+            _elem446 = iprot.readString()
+            self.row.append(_elem446)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -8849,8 +8839,8 @@ class PartitionValuesRow:
     if self.row is not None:
       oprot.writeFieldBegin('row', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.row))
-      for iter456 in self.row:
-        oprot.writeString(iter456)
+      for iter447 in self.row:
+        oprot.writeString(iter447)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -8904,11 +8894,11 @@ class PartitionValuesResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.partitionValues = []
-          (_etype460, _size457) = iprot.readListBegin()
-          for _i461 in xrange(_size457):
-            _elem462 = PartitionValuesRow()
-            _elem462.read(iprot)
-            self.partitionValues.append(_elem462)
+          (_etype451, _size448) = iprot.readListBegin()
+          for _i452 in xrange(_size448):
+            _elem453 = PartitionValuesRow()
+            _elem453.read(iprot)
+            self.partitionValues.append(_elem453)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -8925,8 +8915,8 @@ class PartitionValuesResponse:
     if self.partitionValues is not None:
       oprot.writeFieldBegin('partitionValues', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.partitionValues))
-      for iter463 in self.partitionValues:
-        iter463.write(oprot)
+      for iter454 in self.partitionValues:
+        iter454.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -9114,11 +9104,11 @@ class Function:
       elif fid == 8:
         if ftype == TType.LIST:
           self.resourceUris = []
-          (_etype467, _size464) = iprot.readListBegin()
-          for _i468 in xrange(_size464):
-            _elem469 = ResourceUri()
-            _elem469.read(iprot)
-            self.resourceUris.append(_elem469)
+          (_etype458, _size455) = iprot.readListBegin()
+          for _i459 in xrange(_size455):
+            _elem460 = ResourceUri()
+            _elem460.read(iprot)
+            self.resourceUris.append(_elem460)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9163,8 +9153,8 @@ class Function:
     if self.resourceUris is not None:
       oprot.writeFieldBegin('resourceUris', TType.LIST, 8)
       oprot.writeListBegin(TType.STRUCT, len(self.resourceUris))
-      for iter470 in self.resourceUris:
-        iter470.write(oprot)
+      for iter461 in self.resourceUris:
+        iter461.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -9408,11 +9398,11 @@ class GetOpenTxnsInfoResponse:
       elif fid == 2:
         if ftype == TType.LIST:
           self.open_txns = []
-          (_etype474, _size471) = iprot.readListBegin()
-          for _i475 in xrange(_size471):
-            _elem476 = TxnInfo()
-            _elem476.read(iprot)
-            self.open_txns.append(_elem476)
+          (_etype465, _size462) = iprot.readListBegin()
+          for _i466 in xrange(_size462):
+            _elem467 = TxnInfo()
+            _elem467.read(iprot)
+            self.open_txns.append(_elem467)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9433,8 +9423,8 @@ class GetOpenTxnsInfoResponse:
     if self.open_txns is not None:
       oprot.writeFieldBegin('open_txns', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.open_txns))
-      for iter477 in self.open_txns:
-        iter477.write(oprot)
+      for iter468 in self.open_txns:
+        iter468.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -9505,10 +9495,10 @@ class GetOpenTxnsResponse:
       elif fid == 2:
         if ftype == TType.LIST:
           self.open_txns = []
-          (_etype481, _size478) = iprot.readListBegin()
-          for _i482 in xrange(_size478):
-            _elem483 = iprot.readI64()
-            self.open_txns.append(_elem483)
+          (_etype472, _size469) = iprot.readListBegin()
+          for _i473 in xrange(_size469):
+            _elem474 = iprot.readI64()
+            self.open_txns.append(_elem474)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9539,8 +9529,8 @@ class GetOpenTxnsResponse:
     if self.open_txns is not None:
       oprot.writeFieldBegin('open_txns', TType.LIST, 2)
       oprot.writeListBegin(TType.I64, len(self.open_txns))
-      for iter484 in self.open_txns:
-        oprot.writeI64(iter484)
+      for iter475 in self.open_txns:
+        oprot.writeI64(iter475)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.min_open_txn is not None:
@@ -9719,10 +9709,10 @@ class OpenTxnsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.txn_ids = []
-          (_etype488, _size485) = iprot.readListBegin()
-          for _i489 in xrange(_size485):
-            _elem490 = iprot.readI64()
-            self.txn_ids.append(_elem490)
+          (_etype479, _size476) = iprot.readListBegin()
+          for _i480 in xrange(_size476):
+            _elem481 = iprot.readI64()
+            self.txn_ids.append(_elem481)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9739,8 +9729,8 @@ class OpenTxnsResponse:
     if self.txn_ids is not None:
       oprot.writeFieldBegin('txn_ids', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.txn_ids))
-      for iter491 in self.txn_ids:
-        oprot.writeI64(iter491)
+      for iter482 in self.txn_ids:
+        oprot.writeI64(iter482)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -9861,10 +9851,10 @@ class AbortTxnsRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.txn_ids = []
-          (_etype495, _size492) = iprot.readListBegin()
-          for _i496 in xrange(_size492):
-            _elem497 = iprot.readI64()
-            self.txn_ids.append(_elem497)
+          (_etype486, _size483) = iprot.readListBegin()
+          for _i487 in xrange(_size483):
+            _elem488 = iprot.readI64()
+            self.txn_ids.append(_elem488)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9881,8 +9871,8 @@ class AbortTxnsRequest:
     if self.txn_ids is not None:
       oprot.writeFieldBegin('txn_ids', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.txn_ids))
-      for iter498 in self.txn_ids:
-        oprot.writeI64(iter498)
+      for iter489 in self.txn_ids:
+        oprot.writeI64(iter489)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -10177,11 +10167,11 @@ class LockRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.component = []
-          (_etype502, _size499) = iprot.readListBegin()
-          for _i503 in xrange(_size499):
-            _elem504 = LockComponent()
-            _elem504.read(iprot)
-            self.component.append(_elem504)
+          (_etype493, _size490) = iprot.readListBegin()
+          for _i494 in xrange(_size490):
+            _elem495 = LockComponent()
+            _elem495.read(iprot)
+            self.component.append(_elem495)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10218,8 +10208,8 @@ class LockRequest:
     if self.component is not None:
       oprot.writeFieldBegin('component', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.component))
-      for iter505 in self.component:
-        iter505.write(oprot)
+      for iter496 in self.component:
+        iter496.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.txnid is not None:
@@ -10917,11 +10907,11 @@ class ShowLocksResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.locks = []
-          (_etype509, _size506) = iprot.readListBegin()
-          for _i510 in xrange(_size506):
-            _elem511 = ShowLocksResponseElement()
-            _elem511.read(iprot)
-            self.locks.append(_elem511)
+          (_etype500, _size497) = iprot.readListBegin()
+          for _i501 in xrange(_size497):
+            _elem502 = ShowLocksResponseElement()
+            _elem502.read(iprot)
+            self.locks.append(_elem502)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10938,8 +10928,8 @@ class ShowLocksResponse:
     if self.locks is not None:
       oprot.writeFieldBegin('locks', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.locks))
-      for iter512 in self.locks:
-        iter512.write(oprot)
+      for iter503 in self.locks:
+        iter503.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -11154,20 +11144,20 @@ class HeartbeatTxnRangeResponse:
       if fid == 1:
         if ftype == TType.SET:
           self.aborted = set()
-          (_etype516, _size513) = iprot.readSetBegin()
-          for _i517 in xrange(_size513):
-            _elem518 = iprot.readI64()
-            self.aborted.add(_elem518)
+          (_etype507, _size504) = iprot.readSetBegin()
+          for _i508 in xrange(_size504):
+            _elem509 = iprot.readI64()
+            self.aborted.add(_elem509)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.SET:
           self.nosuch = set()
-          (_etype522, _size519) = iprot.readSetBegin()
-          for _i523 in xrange(_size519):
-            _elem524 = iprot.readI64()
-            self.nosuch.add(_elem524)
+          (_etype513, _size510) = iprot.readSetBegin()
+          for _i514 in xrange(_size510):
+            _elem515 = iprot.readI64()
+            self.nosuch.add(_elem515)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
@@ -11184,15 +11174,15 @@ class HeartbeatTxnRangeResponse:
     if self.aborted is not None:
       oprot.writeFieldBegin('aborted', TType.SET, 1)
       oprot.writeSetBegin(TType.I64, len(self.aborted))
-      for iter525 in self.aborted:
-        oprot.writeI64(iter525)
+      for iter516 in self.aborted:
+        oprot.writeI64(iter516)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     if self.nosuch is not None:
       oprot.writeFieldBegin('nosuch', TType.SET, 2)
       oprot.writeSetBegin(TType.I64, len(self.nosuch))
-      for iter526 in self.nosuch:
-        oprot.writeI64(iter526)
+      for iter517 in self.nosuch:
+        oprot.writeI64(iter517)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -11289,11 +11279,11 @@ class CompactionRequest:
       elif fid == 6:
         if ftype == TType.MAP:
           self.properties = {}
-          (_ktype528, _vtype529, _size527 ) = iprot.readMapBegin()
-          for _i531 in xrange(_size527):
-            _key532 = iprot.readString()
-            _val533 = iprot.readString()
-            self.properties[_key532] = _val533
+          (_ktype519, _vtype520, _size518 ) = iprot.readMapBegin()
+          for _i522 in xrange(_size518):
+            _key523 = iprot.readString()
+            _val524 = iprot.readString()
+            self.properties[_key523] = _val524
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -11330,9 +11320,9 @@ class CompactionRequest:
     if self.properties is not None:
       oprot.writeFieldBegin('properties', TType.MAP, 6)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.properties))
-      for kiter534,viter535 in self.properties.items():
-        oprot.writeString(kiter534)
-        oprot.writeString(viter535)
+      for kiter525,viter526 in self.properties.items():
+        oprot.writeString(kiter525)
+        oprot.writeString(viter526)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -11767,11 +11757,11 @@ class ShowCompactResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.compacts = []
-          (_etype539, _size536) = iprot.readListBegin()
-          for _i540 in xrange(_size536):
-            _elem541 = ShowCompactResponseElement()
-            _elem541.read(iprot)
-            self.compacts.append(_elem541)
+          (_etype530, _size527) = iprot.readListBegin()
+          for _i531 in xrange(_size527):
+            _elem532 = ShowCompactResponseElement()
+            _elem532.read(iprot)
+            self.compacts.append(_elem532)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11788,8 +11778,8 @@ class ShowCompactResponse:
     if self.compacts is not None:
       oprot.writeFieldBegin('compacts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.compacts))
-      for iter542 in self.compacts:
-        iter542.write(oprot)
+      for iter533 in self.compacts:
+        iter533.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -11870,10 +11860,10 @@ class AddDynamicPartitions:
       elif fid == 4:
         if ftype == TType.LIST:
           self.partitionnames = []
-          (_etype546, _size543) = iprot.readListBegin()
-          for _i547 in xrange(_size543):
-            _elem548 = iprot.readString()
-            self.partitionnames.append(_elem548)
+          (_etype537, _size534) = iprot.readListBegin()
+          for _i538 in xrange(_size534):
+            _elem539 = iprot.readString()
+            self.partitionnames.append(_elem539)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11907,8 +11897,8 @@ class AddDynamicPartitions:
     if self.partitionnames is not None:
       oprot.writeFieldBegin('partitionnames', TType.LIST, 4)
       oprot.writeListBegin(TType.STRING, len(self.partitionnames))
-      for iter549 in self.partitionnames:
-        oprot.writeString(iter549)
+      for iter540 in self.partitionnames:
+        oprot.writeString(iter540)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.operationType is not None:
@@ -11954,7 +11944,6 @@ class BasicTxnInfo:
   """
   Attributes:
    - isnull
-   - id
    - time
    - txnid
    - dbname
@@ -11965,17 +11954,15 @@ class BasicTxnInfo:
   thrift_spec = (
     None, # 0
     (1, TType.BOOL, 'isnull', None, None, ), # 1
-    (2, TType.I64, 'id', None, None, ), # 2
-    (3, TType.I64, 'time', None, None, ), # 3
-    (4, TType.I64, 'txnid', None, None, ), # 4
-    (5, TType.STRING, 'dbname', None, None, ), # 5
-    (6, TType.STRING, 'tablename', None, None, ), # 6
-    (7, TType.STRING, 'partitionname', None, None, ), # 7
+    (2, TType.I64, 'time', None, None, ), # 2
+    (3, TType.I64, 'txnid', None, None, ), # 3
+    (4, TType.STRING, 'dbname', None, None, ), # 4
+    (5, TType.STRING, 'tablename', None, None, ), # 5
+    (6, TType.STRING, 'partitionname', None, None, ), # 6
   )
 
-  def __init__(self, isnull=None, id=None, time=None, txnid=None, dbname=None, tablename=None, partitionname=None,):
+  def __init__(self, isnull=None, time=None, txnid=None, dbname=None, tablename=None, partitionname=None,):
     self.isnull = isnull
-    self.id = id
     self.time = time
     self.txnid = txnid
     self.dbname = dbname
@@ -11998,30 +11985,25 @@ class BasicTxnInfo:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.I64:
-          self.id = iprot.readI64()
-        else:
-          iprot.skip(ftype)
-      elif fid == 3:
-        if ftype == TType.I64:
           self.time = iprot.readI64()
         else:
           iprot.skip(ftype)
-      elif fid == 4:
+      elif fid == 3:
         if ftype == TType.I64:
           self.txnid = iprot.readI64()
         else:
           iprot.skip(ftype)
-      elif fid == 5:
+      elif fid == 4:
         if ftype == TType.STRING:
           self.dbname = iprot.readString()
         else:
           iprot.skip(ftype)
-      elif fid == 6:
+      elif fid == 5:
         if ftype == TType.STRING:
           self.tablename = iprot.readString()
         else:
           iprot.skip(ftype)
-      elif fid == 7:
+      elif fid == 6:
         if ftype == TType.STRING:
           self.partitionname = iprot.readString()
         else:
@@ -12040,28 +12022,24 @@ class BasicTxnInfo:
       oprot.writeFieldBegin('isnull', TType.BOOL, 1)
       oprot.writeBool(self.isnull)
       oprot.writeFieldEnd()
-    if self.id is not None:
-      oprot.writeFieldBegin('id', TType.I64, 2)
-      oprot.writeI64(self.id)
-      oprot.writeFieldEnd()
     if self.time is not None:
-      oprot.writeFieldBegin('time', TType.I64, 3)
+      oprot.writeFieldBegin('time', TType.I64, 2)
       oprot.writeI64(self.time)
       oprot.writeFieldEnd()
     if self.txnid is not None:
-      oprot.writeFieldBegin('txnid', TType.I64, 4)
+      oprot.writeFieldBegin('txnid', TType.I64, 3)
       oprot.writeI64(self.txnid)
       oprot.writeFieldEnd()
     if self.dbname is not None:
-      oprot.writeFieldBegin('dbname', TType.STRING, 5)
+      oprot.writeFieldBegin('dbname', TType.STRING, 4)
       oprot.writeString(self.dbname)
       oprot.writeFieldEnd()
     if self.tablename is not None:
-      oprot.writeFieldBegin('tablename', TType.STRING, 6)
+      oprot.writeFieldBegin('tablename', TType.STRING, 5)
       oprot.writeString(self.tablename)
       oprot.writeFieldEnd()
     if self.partitionname is not None:
-      oprot.writeFieldBegin('partitionname', TType.STRING, 7)
+      oprot.writeFieldBegin('partitionname', TType.STRING, 6)
       oprot.writeString(self.partitionname)
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -12076,7 +12054,6 @@ class BasicTxnInfo:
   def __hash__(self):
     value = 17
     value = (value * 31) ^ hash(self.isnull)
-    value = (value * 31) ^ hash(self.id)
     value = (value * 31) ^ hash(self.time)
     value = (value * 31) ^ hash(self.txnid)
     value = (value * 31) ^ hash(self.dbname)
@@ -12095,22 +12072,28 @@ class BasicTxnInfo:
   def __ne__(self, other):
     return not (self == other)
 
-class TxnsSnapshot:
+class CreationMetadata:
   """
   Attributes:
-   - txn_high_water_mark
-   - open_txns
+   - dbName
+   - tblName
+   - tablesUsed
+   - validTxnList
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.I64, 'txn_high_water_mark', None, None, ), # 1
-    (2, TType.LIST, 'open_txns', (TType.I64,None), None, ), # 2
+    (1, TType.STRING, 'dbName', None, None, ), # 1
+    (2, TType.STRING, 'tblName', None, None, ), # 2
+    (3, TType.SET, 'tablesUsed', (TType.STRING,None), None, ), # 3
+    (4, TType.STRING, 'validTxnList', None, None, ), # 4
   )
 
-  def __init__(self, txn_high_water_mark=None, open_txns=None,):
-    self.txn_high_water_mark = txn_high_water_mark
-    self.open_txns = open_txns
+  def __init__(self, dbName=None, tblName=None, tablesUsed=None, validTxnList=None,):
+    self.dbName = dbName
+    self.tblName = tblName
+    self.tablesUsed = tablesUsed
+    self.validTxnList = validTxnList
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -12122,18 +12105,28 @@ class TxnsSnapshot:
       if ftype == TType.STOP:
         break
       if fid == 1:
-        if ftype == TType.I64:
-          self.txn_high_water_mark = iprot.readI64()
+        if ftype == TType.STRING:
+          self.dbName = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 2:
-        if ftype == TType.LIST:
-          self.open_txns = []
-          (_etype553, _size550) = iprot.readListBegin()
-          for _i554 in xrange(_size550):
-            _elem555 = iprot.readI64()
-            self.open_txns.append(_elem555)
-          iprot.readListEnd()
+        if ftype == TType.STRING:
+          self.tblName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.SET:
+          self.tablesUsed = set()
+          (_etype544, _size541) = iprot.readSetBegin()
+          for _i545 in xrange(_size541):
+            _elem546 = iprot.readString()
+            self.tablesUsed.add(_elem546)
+          iprot.readSetEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.STRING:
+          self.validTxnList = iprot.readString()
         else:
           iprot.skip(ftype)
       else:
@@ -12145,33 +12138,45 @@ class TxnsSnapshot:
     if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
       oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
       return
-    oprot.writeStructBegin('TxnsSnapshot')
-    if self.txn_high_water_mark is not None:
-      oprot.writeFieldBegin('txn_high_water_mark', TType.I64, 1)
-      oprot.writeI64(self.txn_high_water_mark)
+    oprot.writeStructBegin('CreationMetadata')
+    if self.dbName is not None:
+      oprot.writeFieldBegin('dbName', TType.STRING, 1)
+      oprot.writeString(self.dbName)
       oprot.writeFieldEnd()
-    if self.open_txns is not None:
-      oprot.writeFieldBegin('open_txns', TType.LIST, 2)
-      oprot.writeListBegin(TType.I64, len(self.open_txns))
-      for iter556 in self.open_txns:
-        oprot.writeI64(iter556)
-      oprot.writeListEnd()
+    if self.tblName is not None:
+      oprot.writeFieldBegin('tblName', TType.STRING, 2)
+      oprot.writeString(self.tblName)
+      oprot.writeFieldEnd()
+    if self.tablesUsed is not None:
+      oprot.writeFieldBegin('tablesUsed', TType.SET, 3)
+      oprot.writeSetBegin(TType.STRING, len(self.tablesUsed))
+      for iter547 in self.tablesUsed:
+        oprot.writeString(iter547)
+      oprot.writeSetEnd()
+      oprot.writeFieldEnd()
+    if self.validTxnList is not None:
+      oprot.writeFieldBegin('validTxnList', TType.STRING, 4)
+      oprot.writeString(self.validTxnList)
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
   def validate(self):
-    if self.txn_high_water_mark is None:
-      raise TProtocol.TProtocolException(message='Required field txn_high_water_mark is unset!')
-    if self.open_txns is None:
-      raise TProtocol.TProtocolException(message='Required field open_txns is unset!')
+    if self.dbName is None:
+      raise TProtocol.TProtocolException(message='Required field dbName is unset!')
+    if self.tblName is None:
+      raise TProtocol.TProtocolException(message='Required field tblName is unset!')
+    if self.tablesUsed is None:
+      raise TProtocol.TProtocolException(message='Required field tablesUsed is unset!')
     return
 
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.txn_high_water_mark)
-    value = (value * 31) ^ hash(self.open_txns)
+    value = (value * 31) ^ hash(self.dbName)
+    value = (value * 31) ^ hash(self.tblName)
+    value = (value * 31) ^ hash(self.tablesUsed)
+    value = (value * 31) ^ hash(self.validTxnList)
     return value
 
   def __repr__(self):
@@ -12442,11 +12447,11 @@ class NotificationEventResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.events = []
-          (_etype560, _size557) = iprot.readListBegin()
-          for _i561 in xrange(_size557):
-            _elem562 = NotificationEvent()
-            _elem562.read(iprot)
-            self.events.append(_elem562)
+          (_etype551, _size548) = iprot.readListBegin()
+          for _i552 in xrange(_size548):
+            _elem553 = NotificationEvent()
+            _elem553.read(iprot)
+            self.events.append(_elem553)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12463,8 +12468,8 @@ class NotificationEventResponse:
     if self.events is not None:
       oprot.writeFieldBegin('events', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.events))
-      for iter563 in self.events:
-        iter563.write(oprot)
+      for iter554 in self.events:
+        iter554.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -12745,20 +12750,20 @@ class InsertEventRequestData:
       elif fid == 2:
         if ftype == TType.LIST:
           self.filesAdded = []
-          (_etype567, _size564) = iprot.readListBegin()
-          for _i568 in xrange(_size564):
-            _elem569 = iprot.readString()
-            self.filesAdded.append(_elem569)
+          (_etype558, _size555) = iprot.readListBegin()
+          for _i559 in xrange(_size555):
+            _elem560 = iprot.readString()
+            self.filesAdded.append(_elem560)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.filesAddedChecksum = []
-          (_etype573, _size570) = iprot.readListBegin()
-          for _i574 in xrange(_size570):
-            _elem575 = iprot.readString()
-            self.filesAddedChecksum.append(_elem575)
+          (_etype564, _size561) = iprot.readListBegin()
+          for _i565 in xrange(_size561):
+            _elem566 = iprot.readString()
+            self.filesAddedChecksum.append(_elem566)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12779,15 +12784,15 @@ class InsertEventRequestData:
     if self.filesAdded is not None:
       oprot.writeFieldBegin('filesAdded', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.filesAdded))
-      for iter576 in self.filesAdded:
-        oprot.writeString(iter576)
+      for iter567 in self.filesAdded:
+        oprot.writeString(iter567)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.filesAddedChecksum is not None:
       oprot.writeFieldBegin('filesAddedChecksum', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.filesAddedChecksum))
-      for iter577 in self.filesAddedChecksum:
-        oprot.writeString(iter577)
+      for iter568 in self.filesAddedChecksum:
+        oprot.writeString(iter568)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -12942,10 +12947,10 @@ class FireEventRequest:
       elif fid == 5:
         if ftype == TType.LIST:
           self.partitionVals = []
-          (_etype581, _size578) = iprot.readListBegin()
-          for _i582 in xrange(_size578):
-            _elem583 = iprot.readString()
-            self.partitionVals.append(_elem583)
+          (_etype572, _size569) = iprot.readListBegin()
+          for _i573 in xrange(_size569):
+            _elem574 = iprot.readString()
+            self.partitionVals.append(_elem574)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12978,8 +12983,8 @@ class FireEventRequest:
     if self.partitionVals is not None:
       oprot.writeFieldBegin('partitionVals', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.partitionVals))
-      for iter584 in self.partitionVals:
-        oprot.writeString(iter584)
+      for iter575 in self.partitionVals:
+        oprot.writeString(iter575)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -13166,12 +13171,12 @@ class GetFileMetadataByExprResult:
       if fid == 1:
         if ftype == TType.MAP:
           self.metadata = {}
-          (_ktype586, _vtype587, _size585 ) = iprot.readMapBegin()
-          for _i589 in xrange(_size585):
-            _key590 = iprot.readI64()
-            _val591 = MetadataPpdResult()
-            _val591.read(iprot)
-            self.metadata[_key590] = _val591
+          (_ktype577, _vtype578, _size576 ) = iprot.readMapBegin()
+          for _i580 in xrange(_size576):
+            _key581 = iprot.readI64()
+            _val582 = MetadataPpdResult()
+            _val582.read(iprot)
+            self.metadata[_key581] = _val582
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -13193,9 +13198,9 @@ class GetFileMetadataByExprResult:
     if self.metadata is not None:
       oprot.writeFieldBegin('metadata', TType.MAP, 1)
       oprot.writeMapBegin(TType.I64, TType.STRUCT, len(self.metadata))
-      for kiter592,viter593 in self.metadata.items():
-        oprot.writeI64(kiter592)
-        viter593.write(oprot)
+      for kiter583,viter584 in self.metadata.items():
+        oprot.writeI64(kiter583)
+        viter584.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.isSupported is not None:
@@ -13265,10 +13270,10 @@ class GetFileMetadataByExprRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype597, _size594) = iprot.readListBegin()
-          for _i598 in xrange(_size594):
-            _elem599 = iprot.readI64()
-            self.fileIds.append(_elem599)
+          (_etype588, _size585) = iprot.readListBegin()
+          for _i589 in xrange(_size585):
+            _elem590 = iprot.readI64()
+            self.fileIds.append(_elem590)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13300,8 +13305,8 @@ class GetFileMetadataByExprRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter600 in self.fileIds:
-        oprot.writeI64(iter600)
+      for iter591 in self.fileIds:
+        oprot.writeI64(iter591)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.expr is not None:
@@ -13375,11 +13380,11 @@ class GetFileMetadataResult:
       if fid == 1:
         if ftype == TType.MAP:
           self.metadata = {}
-          (_ktype602, _vtype603, _size601 ) = iprot.readMapBegin()
-          for _i605 in xrange(_size601):
-            _key606 = iprot.readI64()
-            _val607 = iprot.readString()
-            self.metadata[_key606] = _val607
+          (_ktype593, _vtype594, _size592 ) = iprot.readMapBegin()
+          for _i596 in xrange(_size592):
+            _key597 = iprot.readI64()
+            _val598 = iprot.readString()
+            self.metadata[_key597] = _val598
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -13401,9 +13406,9 @@ class GetFileMetadataResult:
     if self.metadata is not None:
       oprot.writeFieldBegin('metadata', TType.MAP, 1)
       oprot.writeMapBegin(TType.I64, TType.STRING, len(self.metadata))
-      for kiter608,viter609 in self.metadata.items():
-        oprot.writeI64(kiter608)
-        oprot.writeString(viter609)
+      for kiter599,viter600 in self.metadata.items():
+        oprot.writeI64(kiter599)
+        oprot.writeString(viter600)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.isSupported is not None:
@@ -13464,10 +13469,10 @@ class GetFileMetadataRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype613, _size610) = iprot.readListBegin()
-          for _i614 in xrange(_size610):
-            _elem615 = iprot.readI64()
-            self.fileIds.append(_elem615)
+          (_etype604, _size601) = iprot.readListBegin()
+          for _i605 in xrange(_size601):
+            _elem606 = iprot.readI64()
+            self.fileIds.append(_elem606)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13484,8 +13489,8 @@ class GetFileMetadataRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter616 in self.fileIds:
-        oprot.writeI64(iter616)
+      for iter607 in self.fileIds:
+        oprot.writeI64(iter607)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -13591,20 +13596,20 @@ class PutFileMetadataRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype620, _size617) = iprot.readListBegin()
-          for _i621 in xrange(_size617):
-            _elem622 = iprot.readI64()
-            self.fileIds.append(_elem622)
+          (_etype611, _size608) = iprot.readListBegin()
+          for _i612 in xrange(_size608):
+            _elem613 = iprot.readI64()
+            self.fileIds.append(_elem613)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.LIST:
           self.metadata = []
-          (_etype626, _size623) = iprot.readListBegin()
-          for _i627 in xrange(_size623):
-            _elem628 = iprot.readString()
-            self.metadata.append(_elem628)
+          (_etype617, _size614) = iprot.readListBegin()
+          for _i618 in xrange(_size614):
+            _elem619 = iprot.readString()
+            self.metadata.append(_elem619)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13626,15 +13631,15 @@ class PutFileMetadataRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter629 in self.fileIds:
-        oprot.writeI64(iter629)
+      for iter620 in self.fileIds:
+        oprot.writeI64(iter620)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.metadata is not None:
       oprot.writeFieldBegin('metadata', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.metadata))
-      for iter630 in self.metadata:
-        oprot.writeString(iter630)
+      for iter621 in self.metadata:
+        oprot.writeString(iter621)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.type is not None:
@@ -13742,10 +13747,10 @@ class ClearFileMetadataRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype634, _size631) = iprot.readListBegin()
-          for _i635 in xrange(_size631):
-            _elem636 = iprot.readI64()
-            self.fileIds.append(_elem636)
+          (_etype625, _size622) = iprot.readListBegin()
+          for _i626 in xrange(_size622):
+            _elem627 = iprot.readI64()
+            self.fileIds.append(_elem627)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13762,8 +13767,8 @@ class ClearFileMetadataRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter637 in self.fileIds:
-        oprot.writeI64(iter637)
+      for iter628 in self.fileIds:
+        oprot.writeI64(iter628)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -13992,11 +13997,11 @@ class GetAllFunctionsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.functions = []
-          (_etype641, _size638) = iprot.readListBegin()
-          for _i642 in xrange(_size638):
-            _elem643 = Function()
-            _elem643.read(iprot)
-            self.functions.append(_elem643)
+          (_etype632, _size629) = iprot.readListBegin()
+          for _i633 in xrange(_size629):
+            _elem634 = Function()
+            _elem634.read(iprot)
+            self.functions.append(_elem634)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14013,8 +14018,8 @@ class GetAllFunctionsResponse:
     if self.functions is not None:
       oprot.writeFieldBegin('functions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.functions))
-      for iter644 in self.functions:
-        iter644.write(oprot)
+      for iter635 in self.functions:
+        iter635.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14066,10 +14071,10 @@ class ClientCapabilities:
       if fid == 1:
         if ftype == TType.LIST:
           self.values = []
-          (_etype648, _size645) = iprot.readListBegin()
-          for _i649 in xrange(_size645):
-            _elem650 = iprot.readI32()
-            self.values.append(_elem650)
+          (_etype639, _size636) = iprot.readListBegin()
+          for _i640 in xrange(_size636):
+            _elem641 = iprot.readI32()
+            self.values.append(_elem641)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14086,8 +14091,8 @@ class ClientCapabilities:
     if self.values is not None:
       oprot.writeFieldBegin('values', TType.LIST, 1)
       oprot.writeListBegin(TType.I32, len(self.values))
-      for iter651 in self.values:
-        oprot.writeI32(iter651)
+      for iter642 in self.values:
+        oprot.writeI32(iter642)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14316,10 +14321,10 @@ class GetTablesRequest:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tblNames = []
-          (_etype655, _size652) = iprot.readListBegin()
-          for _i656 in xrange(_size652):
-            _elem657 = iprot.readString()
-            self.tblNames.append(_elem657)
+          (_etype646, _size643) = iprot.readListBegin()
+          for _i647 in xrange(_size643):
+            _elem648 = iprot.readString()
+            self.tblNames.append(_elem648)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14346,8 +14351,8 @@ class GetTablesRequest:
     if self.tblNames is not None:
       oprot.writeFieldBegin('tblNames', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.tblNames))
-      for iter658 in self.tblNames:
-        oprot.writeString(iter658)
+      for iter649 in self.tblNames:
+        oprot.writeString(iter649)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.capabilities is not None:
@@ -14407,11 +14412,11 @@ class GetTablesResult:
       if fid == 1:
         if ftype == TType.LIST:
           self.tables = []
-          (_etype662, _size659) = iprot.readListBegin()
-          for _i663 in xrange(_size659):
-            _elem664 = Table()
-            _elem664.read(iprot)
-            self.tables.append(_elem664)
+          (_etype653, _size650) = iprot.readListBegin()
+          for _i654 in xrange(_size650):
+            _elem655 = Table()
+            _elem655.read(iprot)
+            self.tables.append(_elem655)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14428,8 +14433,8 @@ class GetTablesResult:
     if self.tables is not None:
       oprot.writeFieldBegin('tables', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.tables))
-      for iter665 in self.tables:
-        iter665.write(oprot)
+      for iter656 in self.tables:
+        iter656.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14733,10 +14738,10 @@ class Materialization:
       elif fid == 2:
         if ftype == TType.SET:
           self.tablesUsed = set()
-          (_etype669, _size666) = iprot.readSetBegin()
-          for _i670 in xrange(_size666):
-            _elem671 = iprot.readString()
-            self.tablesUsed.add(_elem671)
+          (_etype660, _size657) = iprot.readSetBegin()
+          for _i661 in xrange(_size657):
+            _elem662 = iprot.readString()
+            self.tablesUsed.add(_elem662)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
@@ -14762,8 +14767,8 @@ class Materialization:
     if self.tablesUsed is not None:
       oprot.writeFieldBegin('tablesUsed', TType.SET, 2)
       oprot.writeSetBegin(TType.STRING, len(self.tablesUsed))
-      for iter672 in self.tablesUsed:
-        oprot.writeString(iter672)
+      for iter663 in self.tablesUsed:
+        oprot.writeString(iter663)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     if self.invalidationTime is not None:
@@ -15665,44 +15670,44 @@ class WMFullResourcePlan:
       elif fid == 2:
         if ftype == TType.LIST:
           self.pools = []
-          (_etype676, _size673) = iprot.readListBegin()
-          for _i677 in xrange(_size673):
-            _elem678 = WMPool()
-            _elem678.read(iprot)
-            self.pools.append(_elem678)
+          (_etype667, _size664) = iprot.readListBegin()
+          for _i668 in xrange(_size664):
+            _elem669 = WMPool()
+            _elem669.read(iprot)
+            self.pools.append(_elem669)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.mappings = []
-          (_etype682, _size679) = iprot.readListBegin()
-          for _i683 in xrange(_size679):
-            _elem684 = WMMapping()
-            _elem684.read(iprot)
-            self.mappings.append(_elem684)
+          (_etype673, _size670) = iprot.readListBegin()
+          for _i674 in xrange(_size670):
+            _elem675 = WMMapping()
+            _elem675.read(iprot)
+            self.mappings.append(_elem675)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.LIST:
           self.triggers = []
-          (_etype688, _size685) = iprot.readListBegin()
-          for _i689 in xrange(_size685):
-            _elem690 = WMTrigger()
-            _elem690.read(iprot)
-            self.triggers.append(_elem690)
+          (_etype679, _size676) = iprot.readListBegin()
+          for _i680 in xrange(_size676):
+            _elem681 = WMTrigger()
+            _elem681.read(iprot)
+            self.triggers.append(_elem681)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.LIST:
           self.poolTriggers = []
-          (_etype694, _size691) = iprot.readListBegin()
-          for _i695 in xrange(_size691):
-            _elem696 = WMPoolTrigger()
-            _elem696.read(iprot)
-            self.poolTriggers.append(_elem696)
+          (_etype685, _size682) = iprot.readListBegin()
+          for _i686 in xrange(_size682):
+            _elem687 = WMPoolTrigger()
+            _elem687.read(iprot)
+            self.poolTriggers.append(_elem687)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15723,29 +15728,29 @@ class WMFullResourcePlan:
     if self.pools is not None:
       oprot.writeFieldBegin('pools', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.pools))
-      for iter697 in self.pools:
-        iter697.write(oprot)
+      for iter688 in self.pools:
+        iter688.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.mappings is not None:
       oprot.writeFieldBegin('mappings', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.mappings))
-      for iter698 in self.mappings:
-        iter698.write(oprot)
+      for iter689 in self.mappings:
+        iter689.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.triggers is not None:
       oprot.writeFieldBegin('triggers', TType.LIST, 4)
       oprot.writeListBegin(TType.STRUCT, len(self.triggers))
-      for iter699 in self.triggers:
-        iter699.write(oprot)
+      for iter690 in self.triggers:
+        iter690.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.poolTriggers is not None:
       oprot.writeFieldBegin('poolTriggers', TType.LIST, 5)
       oprot.writeListBegin(TType.STRUCT, len(self.poolTriggers))
-      for iter700 in self.poolTriggers:
-        iter700.write(oprot)
+      for iter691 in self.poolTriggers:
+        iter691.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16219,11 +16224,11 @@ class WMGetAllResourcePlanResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.resourcePlans = []
-          (_etype704, _size701) = iprot.readListBegin()
-          for _i705 in xrange(_size701):
-            _elem706 = WMResourcePlan()
-            _elem706.read(iprot)
-            self.resourcePlans.append(_elem706)
+          (_etype695, _size692) = iprot.readListBegin()
+          for _i696 in xrange(_size692):
+            _elem697 = WMResourcePlan()
+            _elem697.read(iprot)
+            self.resourcePlans.append(_elem697)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16240,8 +16245,8 @@ class WMGetAllResourcePlanResponse:
     if self.resourcePlans is not None:
       oprot.writeFieldBegin('resourcePlans', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.resourcePlans))
-      for iter707 in self.resourcePlans:
-        iter707.write(oprot)
+      for iter698 in self.resourcePlans:
+        iter698.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16545,20 +16550,20 @@ class WMValidateResourcePlanResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.errors = []
-          (_etype711, _size708) = iprot.readListBegin()
-          for _i712 in xrange(_size708):
-            _elem713 = iprot.readString()
-            self.errors.append(_elem713)
+          (_etype702, _size699) = iprot.readListBegin()
+          for _i703 in xrange(_size699):
+            _elem704 = iprot.readString()
+            self.errors.append(_elem704)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.LIST:
           self.warnings = []
-          (_etype717, _size714) = iprot.readListBegin()
-          for _i718 in xrange(_size714):
-            _elem719 = iprot.readString()
-            self.warnings.append(_elem719)
+          (_etype708, _size705) = iprot.readListBegin()
+          for _i709 in xrange(_size705):
+            _elem710 = iprot.readString()
+            self.warnings.append(_elem710)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16575,15 +16580,15 @@ class WMValidateResourcePlanResponse:
     if self.errors is not None:
       oprot.writeFieldBegin('errors', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.errors))
-      for iter720 in self.errors:
-        oprot.writeString(iter720)
+      for iter711 in self.errors:
+        oprot.writeString(iter711)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.warnings is not None:
       oprot.writeFieldBegin('warnings', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.warnings))
-      for iter721 in self.warnings:
-        oprot.writeString(iter721)
+      for iter712 in self.warnings:
+        oprot.writeString(iter712)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17160,11 +17165,11 @@ class WMGetTriggersForResourePlanResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.triggers = []
-          (_etype725, _size722) = iprot.readListBegin()
-          for _i726 in xrange(_size722):
-            _elem727 = WMTrigger()
-            _elem727.read(iprot)
-            self.triggers.append(_elem727)
+          (_etype716, _size713) = iprot.readLis

<TRUNCATED>

[14/16] hive git commit: HIVE-18546: Remove unnecessary code introduced in HIVE-14498 (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
index 42bc929..bfa17eb 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
@@ -174,8 +174,6 @@ class ThriftHiveMetastoreIf : virtual public  ::facebook::fb303::FacebookService
   virtual void compact2(CompactionResponse& _return, const CompactionRequest& rqst) = 0;
   virtual void show_compact(ShowCompactResponse& _return, const ShowCompactRequest& rqst) = 0;
   virtual void add_dynamic_partitions(const AddDynamicPartitions& rqst) = 0;
-  virtual void get_last_completed_transaction_for_tables(std::vector<BasicTxnInfo> & _return, const std::vector<std::string> & db_names, const std::vector<std::string> & table_names, const TxnsSnapshot& txns_snapshot) = 0;
-  virtual void get_last_completed_transaction_for_table(BasicTxnInfo& _return, const std::string& db_name, const std::string& table_name, const TxnsSnapshot& txns_snapshot) = 0;
   virtual void get_next_notification(NotificationEventResponse& _return, const NotificationEventRequest& rqst) = 0;
   virtual void get_current_notificationEventId(CurrentNotificationEventId& _return) = 0;
   virtual void get_notification_events_count(NotificationEventsCountResponse& _return, const NotificationEventsCountRequest& rqst) = 0;
@@ -718,12 +716,6 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p
   void add_dynamic_partitions(const AddDynamicPartitions& /* rqst */) {
     return;
   }
-  void get_last_completed_transaction_for_tables(std::vector<BasicTxnInfo> & /* _return */, const std::vector<std::string> & /* db_names */, const std::vector<std::string> & /* table_names */, const TxnsSnapshot& /* txns_snapshot */) {
-    return;
-  }
-  void get_last_completed_transaction_for_table(BasicTxnInfo& /* _return */, const std::string& /* db_name */, const std::string& /* table_name */, const TxnsSnapshot& /* txns_snapshot */) {
-    return;
-  }
   void get_next_notification(NotificationEventResponse& /* _return */, const NotificationEventRequest& /* rqst */) {
     return;
   }
@@ -19798,242 +19790,6 @@ class ThriftHiveMetastore_add_dynamic_partitions_presult {
 
 };
 
-typedef struct _ThriftHiveMetastore_get_last_completed_transaction_for_tables_args__isset {
-  _ThriftHiveMetastore_get_last_completed_transaction_for_tables_args__isset() : db_names(false), table_names(false), txns_snapshot(false) {}
-  bool db_names :1;
-  bool table_names :1;
-  bool txns_snapshot :1;
-} _ThriftHiveMetastore_get_last_completed_transaction_for_tables_args__isset;
-
-class ThriftHiveMetastore_get_last_completed_transaction_for_tables_args {
- public:
-
-  ThriftHiveMetastore_get_last_completed_transaction_for_tables_args(const ThriftHiveMetastore_get_last_completed_transaction_for_tables_args&);
-  ThriftHiveMetastore_get_last_completed_transaction_for_tables_args& operator=(const ThriftHiveMetastore_get_last_completed_transaction_for_tables_args&);
-  ThriftHiveMetastore_get_last_completed_transaction_for_tables_args() {
-  }
-
-  virtual ~ThriftHiveMetastore_get_last_completed_transaction_for_tables_args() throw();
-  std::vector<std::string>  db_names;
-  std::vector<std::string>  table_names;
-  TxnsSnapshot txns_snapshot;
-
-  _ThriftHiveMetastore_get_last_completed_transaction_for_tables_args__isset __isset;
-
-  void __set_db_names(const std::vector<std::string> & val);
-
-  void __set_table_names(const std::vector<std::string> & val);
-
-  void __set_txns_snapshot(const TxnsSnapshot& val);
-
-  bool operator == (const ThriftHiveMetastore_get_last_completed_transaction_for_tables_args & rhs) const
-  {
-    if (!(db_names == rhs.db_names))
-      return false;
-    if (!(table_names == rhs.table_names))
-      return false;
-    if (!(txns_snapshot == rhs.txns_snapshot))
-      return false;
-    return true;
-  }
-  bool operator != (const ThriftHiveMetastore_get_last_completed_transaction_for_tables_args &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const ThriftHiveMetastore_get_last_completed_transaction_for_tables_args & ) const;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-
-class ThriftHiveMetastore_get_last_completed_transaction_for_tables_pargs {
- public:
-
-
-  virtual ~ThriftHiveMetastore_get_last_completed_transaction_for_tables_pargs() throw();
-  const std::vector<std::string> * db_names;
-  const std::vector<std::string> * table_names;
-  const TxnsSnapshot* txns_snapshot;
-
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-typedef struct _ThriftHiveMetastore_get_last_completed_transaction_for_tables_result__isset {
-  _ThriftHiveMetastore_get_last_completed_transaction_for_tables_result__isset() : success(false) {}
-  bool success :1;
-} _ThriftHiveMetastore_get_last_completed_transaction_for_tables_result__isset;
-
-class ThriftHiveMetastore_get_last_completed_transaction_for_tables_result {
- public:
-
-  ThriftHiveMetastore_get_last_completed_transaction_for_tables_result(const ThriftHiveMetastore_get_last_completed_transaction_for_tables_result&);
-  ThriftHiveMetastore_get_last_completed_transaction_for_tables_result& operator=(const ThriftHiveMetastore_get_last_completed_transaction_for_tables_result&);
-  ThriftHiveMetastore_get_last_completed_transaction_for_tables_result() {
-  }
-
-  virtual ~ThriftHiveMetastore_get_last_completed_transaction_for_tables_result() throw();
-  std::vector<BasicTxnInfo>  success;
-
-  _ThriftHiveMetastore_get_last_completed_transaction_for_tables_result__isset __isset;
-
-  void __set_success(const std::vector<BasicTxnInfo> & val);
-
-  bool operator == (const ThriftHiveMetastore_get_last_completed_transaction_for_tables_result & rhs) const
-  {
-    if (!(success == rhs.success))
-      return false;
-    return true;
-  }
-  bool operator != (const ThriftHiveMetastore_get_last_completed_transaction_for_tables_result &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const ThriftHiveMetastore_get_last_completed_transaction_for_tables_result & ) const;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-typedef struct _ThriftHiveMetastore_get_last_completed_transaction_for_tables_presult__isset {
-  _ThriftHiveMetastore_get_last_completed_transaction_for_tables_presult__isset() : success(false) {}
-  bool success :1;
-} _ThriftHiveMetastore_get_last_completed_transaction_for_tables_presult__isset;
-
-class ThriftHiveMetastore_get_last_completed_transaction_for_tables_presult {
- public:
-
-
-  virtual ~ThriftHiveMetastore_get_last_completed_transaction_for_tables_presult() throw();
-  std::vector<BasicTxnInfo> * success;
-
-  _ThriftHiveMetastore_get_last_completed_transaction_for_tables_presult__isset __isset;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-
-};
-
-typedef struct _ThriftHiveMetastore_get_last_completed_transaction_for_table_args__isset {
-  _ThriftHiveMetastore_get_last_completed_transaction_for_table_args__isset() : db_name(false), table_name(false), txns_snapshot(false) {}
-  bool db_name :1;
-  bool table_name :1;
-  bool txns_snapshot :1;
-} _ThriftHiveMetastore_get_last_completed_transaction_for_table_args__isset;
-
-class ThriftHiveMetastore_get_last_completed_transaction_for_table_args {
- public:
-
-  ThriftHiveMetastore_get_last_completed_transaction_for_table_args(const ThriftHiveMetastore_get_last_completed_transaction_for_table_args&);
-  ThriftHiveMetastore_get_last_completed_transaction_for_table_args& operator=(const ThriftHiveMetastore_get_last_completed_transaction_for_table_args&);
-  ThriftHiveMetastore_get_last_completed_transaction_for_table_args() : db_name(), table_name() {
-  }
-
-  virtual ~ThriftHiveMetastore_get_last_completed_transaction_for_table_args() throw();
-  std::string db_name;
-  std::string table_name;
-  TxnsSnapshot txns_snapshot;
-
-  _ThriftHiveMetastore_get_last_completed_transaction_for_table_args__isset __isset;
-
-  void __set_db_name(const std::string& val);
-
-  void __set_table_name(const std::string& val);
-
-  void __set_txns_snapshot(const TxnsSnapshot& val);
-
-  bool operator == (const ThriftHiveMetastore_get_last_completed_transaction_for_table_args & rhs) const
-  {
-    if (!(db_name == rhs.db_name))
-      return false;
-    if (!(table_name == rhs.table_name))
-      return false;
-    if (!(txns_snapshot == rhs.txns_snapshot))
-      return false;
-    return true;
-  }
-  bool operator != (const ThriftHiveMetastore_get_last_completed_transaction_for_table_args &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const ThriftHiveMetastore_get_last_completed_transaction_for_table_args & ) const;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-
-class ThriftHiveMetastore_get_last_completed_transaction_for_table_pargs {
- public:
-
-
-  virtual ~ThriftHiveMetastore_get_last_completed_transaction_for_table_pargs() throw();
-  const std::string* db_name;
-  const std::string* table_name;
-  const TxnsSnapshot* txns_snapshot;
-
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-typedef struct _ThriftHiveMetastore_get_last_completed_transaction_for_table_result__isset {
-  _ThriftHiveMetastore_get_last_completed_transaction_for_table_result__isset() : success(false) {}
-  bool success :1;
-} _ThriftHiveMetastore_get_last_completed_transaction_for_table_result__isset;
-
-class ThriftHiveMetastore_get_last_completed_transaction_for_table_result {
- public:
-
-  ThriftHiveMetastore_get_last_completed_transaction_for_table_result(const ThriftHiveMetastore_get_last_completed_transaction_for_table_result&);
-  ThriftHiveMetastore_get_last_completed_transaction_for_table_result& operator=(const ThriftHiveMetastore_get_last_completed_transaction_for_table_result&);
-  ThriftHiveMetastore_get_last_completed_transaction_for_table_result() {
-  }
-
-  virtual ~ThriftHiveMetastore_get_last_completed_transaction_for_table_result() throw();
-  BasicTxnInfo success;
-
-  _ThriftHiveMetastore_get_last_completed_transaction_for_table_result__isset __isset;
-
-  void __set_success(const BasicTxnInfo& val);
-
-  bool operator == (const ThriftHiveMetastore_get_last_completed_transaction_for_table_result & rhs) const
-  {
-    if (!(success == rhs.success))
-      return false;
-    return true;
-  }
-  bool operator != (const ThriftHiveMetastore_get_last_completed_transaction_for_table_result &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const ThriftHiveMetastore_get_last_completed_transaction_for_table_result & ) const;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-typedef struct _ThriftHiveMetastore_get_last_completed_transaction_for_table_presult__isset {
-  _ThriftHiveMetastore_get_last_completed_transaction_for_table_presult__isset() : success(false) {}
-  bool success :1;
-} _ThriftHiveMetastore_get_last_completed_transaction_for_table_presult__isset;
-
-class ThriftHiveMetastore_get_last_completed_transaction_for_table_presult {
- public:
-
-
-  virtual ~ThriftHiveMetastore_get_last_completed_transaction_for_table_presult() throw();
-  BasicTxnInfo* success;
-
-  _ThriftHiveMetastore_get_last_completed_transaction_for_table_presult__isset __isset;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-
-};
-
 typedef struct _ThriftHiveMetastore_get_next_notification_args__isset {
   _ThriftHiveMetastore_get_next_notification_args__isset() : rqst(false) {}
   bool rqst :1;
@@ -23871,12 +23627,6 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public
   void add_dynamic_partitions(const AddDynamicPartitions& rqst);
   void send_add_dynamic_partitions(const AddDynamicPartitions& rqst);
   void recv_add_dynamic_partitions();
-  void get_last_completed_transaction_for_tables(std::vector<BasicTxnInfo> & _return, const std::vector<std::string> & db_names, const std::vector<std::string> & table_names, const TxnsSnapshot& txns_snapshot);
-  void send_get_last_completed_transaction_for_tables(const std::vector<std::string> & db_names, const std::vector<std::string> & table_names, const TxnsSnapshot& txns_snapshot);
-  void recv_get_last_completed_transaction_for_tables(std::vector<BasicTxnInfo> & _return);
-  void get_last_completed_transaction_for_table(BasicTxnInfo& _return, const std::string& db_name, const std::string& table_name, const TxnsSnapshot& txns_snapshot);
-  void send_get_last_completed_transaction_for_table(const std::string& db_name, const std::string& table_name, const TxnsSnapshot& txns_snapshot);
-  void recv_get_last_completed_transaction_for_table(BasicTxnInfo& _return);
   void get_next_notification(NotificationEventResponse& _return, const NotificationEventRequest& rqst);
   void send_get_next_notification(const NotificationEventRequest& rqst);
   void recv_get_next_notification(NotificationEventResponse& _return);
@@ -24126,8 +23876,6 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
   void process_compact2(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_show_compact(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_add_dynamic_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
-  void process_get_last_completed_transaction_for_tables(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
-  void process_get_last_completed_transaction_for_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_next_notification(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_current_notificationEventId(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_notification_events_count(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
@@ -24313,8 +24061,6 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
     processMap_["compact2"] = &ThriftHiveMetastoreProcessor::process_compact2;
     processMap_["show_compact"] = &ThriftHiveMetastoreProcessor::process_show_compact;
     processMap_["add_dynamic_partitions"] = &ThriftHiveMetastoreProcessor::process_add_dynamic_partitions;
-    processMap_["get_last_completed_transaction_for_tables"] = &ThriftHiveMetastoreProcessor::process_get_last_completed_transaction_for_tables;
-    processMap_["get_last_completed_transaction_for_table"] = &ThriftHiveMetastoreProcessor::process_get_last_completed_transaction_for_table;
     processMap_["get_next_notification"] = &ThriftHiveMetastoreProcessor::process_get_next_notification;
     processMap_["get_current_notificationEventId"] = &ThriftHiveMetastoreProcessor::process_get_current_notificationEventId;
     processMap_["get_notification_events_count"] = &ThriftHiveMetastoreProcessor::process_get_notification_events_count;
@@ -25832,26 +25578,6 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi
     ifaces_[i]->add_dynamic_partitions(rqst);
   }
 
-  void get_last_completed_transaction_for_tables(std::vector<BasicTxnInfo> & _return, const std::vector<std::string> & db_names, const std::vector<std::string> & table_names, const TxnsSnapshot& txns_snapshot) {
-    size_t sz = ifaces_.size();
-    size_t i = 0;
-    for (; i < (sz - 1); ++i) {
-      ifaces_[i]->get_last_completed_transaction_for_tables(_return, db_names, table_names, txns_snapshot);
-    }
-    ifaces_[i]->get_last_completed_transaction_for_tables(_return, db_names, table_names, txns_snapshot);
-    return;
-  }
-
-  void get_last_completed_transaction_for_table(BasicTxnInfo& _return, const std::string& db_name, const std::string& table_name, const TxnsSnapshot& txns_snapshot) {
-    size_t sz = ifaces_.size();
-    size_t i = 0;
-    for (; i < (sz - 1); ++i) {
-      ifaces_[i]->get_last_completed_transaction_for_table(_return, db_name, table_name, txns_snapshot);
-    }
-    ifaces_[i]->get_last_completed_transaction_for_table(_return, db_name, table_name, txns_snapshot);
-    return;
-  }
-
   void get_next_notification(NotificationEventResponse& _return, const NotificationEventRequest& rqst) {
     size_t sz = ifaces_.size();
     size_t i = 0;
@@ -26613,12 +26339,6 @@ class ThriftHiveMetastoreConcurrentClient : virtual public ThriftHiveMetastoreIf
   void add_dynamic_partitions(const AddDynamicPartitions& rqst);
   int32_t send_add_dynamic_partitions(const AddDynamicPartitions& rqst);
   void recv_add_dynamic_partitions(const int32_t seqid);
-  void get_last_completed_transaction_for_tables(std::vector<BasicTxnInfo> & _return, const std::vector<std::string> & db_names, const std::vector<std::string> & table_names, const TxnsSnapshot& txns_snapshot);
-  int32_t send_get_last_completed_transaction_for_tables(const std::vector<std::string> & db_names, const std::vector<std::string> & table_names, const TxnsSnapshot& txns_snapshot);
-  void recv_get_last_completed_transaction_for_tables(std::vector<BasicTxnInfo> & _return, const int32_t seqid);
-  void get_last_completed_transaction_for_table(BasicTxnInfo& _return, const std::string& db_name, const std::string& table_name, const TxnsSnapshot& txns_snapshot);
-  int32_t send_get_last_completed_transaction_for_table(const std::string& db_name, const std::string& table_name, const TxnsSnapshot& txns_snapshot);
-  void recv_get_last_completed_transaction_for_table(BasicTxnInfo& _return, const int32_t seqid);
   void get_next_notification(NotificationEventResponse& _return, const NotificationEventRequest& rqst);
   int32_t send_get_next_notification(const NotificationEventRequest& rqst);
   void recv_get_next_notification(NotificationEventResponse& _return, const int32_t seqid);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
index 6a2ff6c..cf9a171 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
@@ -782,16 +782,6 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("add_dynamic_partitions\n");
   }
 
-  void get_last_completed_transaction_for_tables(std::vector<BasicTxnInfo> & _return, const std::vector<std::string> & db_names, const std::vector<std::string> & table_names, const TxnsSnapshot& txns_snapshot) {
-    // Your implementation goes here
-    printf("get_last_completed_transaction_for_tables\n");
-  }
-
-  void get_last_completed_transaction_for_table(BasicTxnInfo& _return, const std::string& db_name, const std::string& table_name, const TxnsSnapshot& txns_snapshot) {
-    // Your implementation goes here
-    printf("get_last_completed_transaction_for_table\n");
-  }
-
   void get_next_notification(NotificationEventResponse& _return, const NotificationEventRequest& rqst) {
     // Your implementation goes here
     printf("get_next_notification\n");


[12/16] hive git commit: HIVE-18546: Remove unnecessary code introduced in HIVE-14498 (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
index df646a7..4c09bc8 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -400,7 +400,7 @@ class AddDynamicPartitions;
 
 class BasicTxnInfo;
 
-class TxnsSnapshot;
+class CreationMetadata;
 
 class NotificationEventRequest;
 
@@ -2415,7 +2415,7 @@ class Table {
   PrincipalPrivilegeSet privileges;
   bool temporary;
   bool rewriteEnabled;
-  std::map<std::string, BasicTxnInfo>  creationMetadata;
+  CreationMetadata creationMetadata;
 
   _Table__isset __isset;
 
@@ -2449,7 +2449,7 @@ class Table {
 
   void __set_rewriteEnabled(const bool val);
 
-  void __set_creationMetadata(const std::map<std::string, BasicTxnInfo> & val);
+  void __set_creationMetadata(const CreationMetadata& val);
 
   bool operator == (const Table & rhs) const
   {
@@ -7086,8 +7086,7 @@ inline std::ostream& operator<<(std::ostream& out, const AddDynamicPartitions& o
 }
 
 typedef struct _BasicTxnInfo__isset {
-  _BasicTxnInfo__isset() : id(false), time(false), txnid(false), dbname(false), tablename(false), partitionname(false) {}
-  bool id :1;
+  _BasicTxnInfo__isset() : time(false), txnid(false), dbname(false), tablename(false), partitionname(false) {}
   bool time :1;
   bool txnid :1;
   bool dbname :1;
@@ -7100,12 +7099,11 @@ class BasicTxnInfo {
 
   BasicTxnInfo(const BasicTxnInfo&);
   BasicTxnInfo& operator=(const BasicTxnInfo&);
-  BasicTxnInfo() : isnull(0), id(0), time(0), txnid(0), dbname(), tablename(), partitionname() {
+  BasicTxnInfo() : isnull(0), time(0), txnid(0), dbname(), tablename(), partitionname() {
   }
 
   virtual ~BasicTxnInfo() throw();
   bool isnull;
-  int64_t id;
   int64_t time;
   int64_t txnid;
   std::string dbname;
@@ -7116,8 +7114,6 @@ class BasicTxnInfo {
 
   void __set_isnull(const bool val);
 
-  void __set_id(const int64_t val);
-
   void __set_time(const int64_t val);
 
   void __set_txnid(const int64_t val);
@@ -7132,10 +7128,6 @@ class BasicTxnInfo {
   {
     if (!(isnull == rhs.isnull))
       return false;
-    if (__isset.id != rhs.__isset.id)
-      return false;
-    else if (__isset.id && !(id == rhs.id))
-      return false;
     if (__isset.time != rhs.__isset.time)
       return false;
     else if (__isset.time && !(time == rhs.time))
@@ -7178,36 +7170,54 @@ inline std::ostream& operator<<(std::ostream& out, const BasicTxnInfo& obj)
   return out;
 }
 
+typedef struct _CreationMetadata__isset {
+  _CreationMetadata__isset() : validTxnList(false) {}
+  bool validTxnList :1;
+} _CreationMetadata__isset;
 
-class TxnsSnapshot {
+class CreationMetadata {
  public:
 
-  TxnsSnapshot(const TxnsSnapshot&);
-  TxnsSnapshot& operator=(const TxnsSnapshot&);
-  TxnsSnapshot() : txn_high_water_mark(0) {
+  CreationMetadata(const CreationMetadata&);
+  CreationMetadata& operator=(const CreationMetadata&);
+  CreationMetadata() : dbName(), tblName(), validTxnList() {
   }
 
-  virtual ~TxnsSnapshot() throw();
-  int64_t txn_high_water_mark;
-  std::vector<int64_t>  open_txns;
+  virtual ~CreationMetadata() throw();
+  std::string dbName;
+  std::string tblName;
+  std::set<std::string>  tablesUsed;
+  std::string validTxnList;
 
-  void __set_txn_high_water_mark(const int64_t val);
+  _CreationMetadata__isset __isset;
 
-  void __set_open_txns(const std::vector<int64_t> & val);
+  void __set_dbName(const std::string& val);
+
+  void __set_tblName(const std::string& val);
 
-  bool operator == (const TxnsSnapshot & rhs) const
+  void __set_tablesUsed(const std::set<std::string> & val);
+
+  void __set_validTxnList(const std::string& val);
+
+  bool operator == (const CreationMetadata & rhs) const
   {
-    if (!(txn_high_water_mark == rhs.txn_high_water_mark))
+    if (!(dbName == rhs.dbName))
       return false;
-    if (!(open_txns == rhs.open_txns))
+    if (!(tblName == rhs.tblName))
+      return false;
+    if (!(tablesUsed == rhs.tablesUsed))
+      return false;
+    if (__isset.validTxnList != rhs.__isset.validTxnList)
+      return false;
+    else if (__isset.validTxnList && !(validTxnList == rhs.validTxnList))
       return false;
     return true;
   }
-  bool operator != (const TxnsSnapshot &rhs) const {
+  bool operator != (const CreationMetadata &rhs) const {
     return !(*this == rhs);
   }
 
-  bool operator < (const TxnsSnapshot & ) const;
+  bool operator < (const CreationMetadata & ) const;
 
   uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
   uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
@@ -7215,9 +7225,9 @@ class TxnsSnapshot {
   virtual void printTo(std::ostream& out) const;
 };
 
-void swap(TxnsSnapshot &a, TxnsSnapshot &b);
+void swap(CreationMetadata &a, CreationMetadata &b);
 
-inline std::ostream& operator<<(std::ostream& out, const TxnsSnapshot& obj)
+inline std::ostream& operator<<(std::ostream& out, const CreationMetadata& obj)
 {
   obj.printTo(out);
   return out;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java
index 398f8d4..0e5dbf7 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java
@@ -351,13 +351,13 @@ import org.slf4j.LoggerFactory;
           case 1: // TXN_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list558 = iprot.readListBegin();
-                struct.txn_ids = new ArrayList<Long>(_list558.size);
-                long _elem559;
-                for (int _i560 = 0; _i560 < _list558.size; ++_i560)
+                org.apache.thrift.protocol.TList _list548 = iprot.readListBegin();
+                struct.txn_ids = new ArrayList<Long>(_list548.size);
+                long _elem549;
+                for (int _i550 = 0; _i550 < _list548.size; ++_i550)
                 {
-                  _elem559 = iprot.readI64();
-                  struct.txn_ids.add(_elem559);
+                  _elem549 = iprot.readI64();
+                  struct.txn_ids.add(_elem549);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TXN_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.txn_ids.size()));
-          for (long _iter561 : struct.txn_ids)
+          for (long _iter551 : struct.txn_ids)
           {
-            oprot.writeI64(_iter561);
+            oprot.writeI64(_iter551);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.txn_ids.size());
-        for (long _iter562 : struct.txn_ids)
+        for (long _iter552 : struct.txn_ids)
         {
-          oprot.writeI64(_iter562);
+          oprot.writeI64(_iter552);
         }
       }
     }
@@ -421,13 +421,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, AbortTxnsRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list563 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.txn_ids = new ArrayList<Long>(_list563.size);
-        long _elem564;
-        for (int _i565 = 0; _i565 < _list563.size; ++_i565)
+        org.apache.thrift.protocol.TList _list553 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.txn_ids = new ArrayList<Long>(_list553.size);
+        long _elem554;
+        for (int _i555 = 0; _i555 < _list553.size; ++_i555)
         {
-          _elem564 = iprot.readI64();
-          struct.txn_ids.add(_elem564);
+          _elem554 = iprot.readI64();
+          struct.txn_ids.add(_elem554);
         }
       }
       struct.setTxn_idsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
index 2102aa5..dae233a 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
@@ -727,13 +727,13 @@ import org.slf4j.LoggerFactory;
           case 4: // PARTITIONNAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list616 = iprot.readListBegin();
-                struct.partitionnames = new ArrayList<String>(_list616.size);
-                String _elem617;
-                for (int _i618 = 0; _i618 < _list616.size; ++_i618)
+                org.apache.thrift.protocol.TList _list606 = iprot.readListBegin();
+                struct.partitionnames = new ArrayList<String>(_list606.size);
+                String _elem607;
+                for (int _i608 = 0; _i608 < _list606.size; ++_i608)
                 {
-                  _elem617 = iprot.readString();
-                  struct.partitionnames.add(_elem617);
+                  _elem607 = iprot.readString();
+                  struct.partitionnames.add(_elem607);
                 }
                 iprot.readListEnd();
               }
@@ -780,9 +780,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARTITIONNAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partitionnames.size()));
-          for (String _iter619 : struct.partitionnames)
+          for (String _iter609 : struct.partitionnames)
           {
-            oprot.writeString(_iter619);
+            oprot.writeString(_iter609);
           }
           oprot.writeListEnd();
         }
@@ -817,9 +817,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeString(struct.tablename);
       {
         oprot.writeI32(struct.partitionnames.size());
-        for (String _iter620 : struct.partitionnames)
+        for (String _iter610 : struct.partitionnames)
         {
-          oprot.writeString(_iter620);
+          oprot.writeString(_iter610);
         }
       }
       BitSet optionals = new BitSet();
@@ -842,13 +842,13 @@ import org.slf4j.LoggerFactory;
       struct.tablename = iprot.readString();
       struct.setTablenameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list621 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.partitionnames = new ArrayList<String>(_list621.size);
-        String _elem622;
-        for (int _i623 = 0; _i623 < _list621.size; ++_i623)
+        org.apache.thrift.protocol.TList _list611 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.partitionnames = new ArrayList<String>(_list611.size);
+        String _elem612;
+        for (int _i613 = 0; _i613 < _list611.size; ++_i613)
         {
-          _elem622 = iprot.readString();
-          struct.partitionnames.add(_elem622);
+          _elem612 = iprot.readString();
+          struct.partitionnames.add(_elem612);
         }
       }
       struct.setPartitionnamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddForeignKeyRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddForeignKeyRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddForeignKeyRequest.java
index a222529..c1c0dbf 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddForeignKeyRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddForeignKeyRequest.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // FOREIGN_KEY_COLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list372 = iprot.readListBegin();
-                struct.foreignKeyCols = new ArrayList<SQLForeignKey>(_list372.size);
-                SQLForeignKey _elem373;
-                for (int _i374 = 0; _i374 < _list372.size; ++_i374)
+                org.apache.thrift.protocol.TList _list362 = iprot.readListBegin();
+                struct.foreignKeyCols = new ArrayList<SQLForeignKey>(_list362.size);
+                SQLForeignKey _elem363;
+                for (int _i364 = 0; _i364 < _list362.size; ++_i364)
                 {
-                  _elem373 = new SQLForeignKey();
-                  _elem373.read(iprot);
-                  struct.foreignKeyCols.add(_elem373);
+                  _elem363 = new SQLForeignKey();
+                  _elem363.read(iprot);
+                  struct.foreignKeyCols.add(_elem363);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FOREIGN_KEY_COLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.foreignKeyCols.size()));
-          for (SQLForeignKey _iter375 : struct.foreignKeyCols)
+          for (SQLForeignKey _iter365 : struct.foreignKeyCols)
           {
-            _iter375.write(oprot);
+            _iter365.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.foreignKeyCols.size());
-        for (SQLForeignKey _iter376 : struct.foreignKeyCols)
+        for (SQLForeignKey _iter366 : struct.foreignKeyCols)
         {
-          _iter376.write(oprot);
+          _iter366.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, AddForeignKeyRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list377 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.foreignKeyCols = new ArrayList<SQLForeignKey>(_list377.size);
-        SQLForeignKey _elem378;
-        for (int _i379 = 0; _i379 < _list377.size; ++_i379)
+        org.apache.thrift.protocol.TList _list367 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.foreignKeyCols = new ArrayList<SQLForeignKey>(_list367.size);
+        SQLForeignKey _elem368;
+        for (int _i369 = 0; _i369 < _list367.size; ++_i369)
         {
-          _elem378 = new SQLForeignKey();
-          _elem378.read(iprot);
-          struct.foreignKeyCols.add(_elem378);
+          _elem368 = new SQLForeignKey();
+          _elem368.read(iprot);
+          struct.foreignKeyCols.add(_elem368);
         }
       }
       struct.setForeignKeyColsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddNotNullConstraintRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddNotNullConstraintRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddNotNullConstraintRequest.java
index ef23d30..0bd85f3 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddNotNullConstraintRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddNotNullConstraintRequest.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // NOT_NULL_CONSTRAINT_COLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list388 = iprot.readListBegin();
-                struct.notNullConstraintCols = new ArrayList<SQLNotNullConstraint>(_list388.size);
-                SQLNotNullConstraint _elem389;
-                for (int _i390 = 0; _i390 < _list388.size; ++_i390)
+                org.apache.thrift.protocol.TList _list378 = iprot.readListBegin();
+                struct.notNullConstraintCols = new ArrayList<SQLNotNullConstraint>(_list378.size);
+                SQLNotNullConstraint _elem379;
+                for (int _i380 = 0; _i380 < _list378.size; ++_i380)
                 {
-                  _elem389 = new SQLNotNullConstraint();
-                  _elem389.read(iprot);
-                  struct.notNullConstraintCols.add(_elem389);
+                  _elem379 = new SQLNotNullConstraint();
+                  _elem379.read(iprot);
+                  struct.notNullConstraintCols.add(_elem379);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(NOT_NULL_CONSTRAINT_COLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.notNullConstraintCols.size()));
-          for (SQLNotNullConstraint _iter391 : struct.notNullConstraintCols)
+          for (SQLNotNullConstraint _iter381 : struct.notNullConstraintCols)
           {
-            _iter391.write(oprot);
+            _iter381.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.notNullConstraintCols.size());
-        for (SQLNotNullConstraint _iter392 : struct.notNullConstraintCols)
+        for (SQLNotNullConstraint _iter382 : struct.notNullConstraintCols)
         {
-          _iter392.write(oprot);
+          _iter382.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, AddNotNullConstraintRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list393 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.notNullConstraintCols = new ArrayList<SQLNotNullConstraint>(_list393.size);
-        SQLNotNullConstraint _elem394;
-        for (int _i395 = 0; _i395 < _list393.size; ++_i395)
+        org.apache.thrift.protocol.TList _list383 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.notNullConstraintCols = new ArrayList<SQLNotNullConstraint>(_list383.size);
+        SQLNotNullConstraint _elem384;
+        for (int _i385 = 0; _i385 < _list383.size; ++_i385)
         {
-          _elem394 = new SQLNotNullConstraint();
-          _elem394.read(iprot);
-          struct.notNullConstraintCols.add(_elem394);
+          _elem384 = new SQLNotNullConstraint();
+          _elem384.read(iprot);
+          struct.notNullConstraintCols.add(_elem384);
         }
       }
       struct.setNotNullConstraintColsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java
index 13a2318..9119336 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java
@@ -704,14 +704,14 @@ import org.slf4j.LoggerFactory;
           case 3: // PARTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list462 = iprot.readListBegin();
-                struct.parts = new ArrayList<Partition>(_list462.size);
-                Partition _elem463;
-                for (int _i464 = 0; _i464 < _list462.size; ++_i464)
+                org.apache.thrift.protocol.TList _list452 = iprot.readListBegin();
+                struct.parts = new ArrayList<Partition>(_list452.size);
+                Partition _elem453;
+                for (int _i454 = 0; _i454 < _list452.size; ++_i454)
                 {
-                  _elem463 = new Partition();
-                  _elem463.read(iprot);
-                  struct.parts.add(_elem463);
+                  _elem453 = new Partition();
+                  _elem453.read(iprot);
+                  struct.parts.add(_elem453);
                 }
                 iprot.readListEnd();
               }
@@ -763,9 +763,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.parts.size()));
-          for (Partition _iter465 : struct.parts)
+          for (Partition _iter455 : struct.parts)
           {
-            _iter465.write(oprot);
+            _iter455.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -800,9 +800,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeString(struct.tblName);
       {
         oprot.writeI32(struct.parts.size());
-        for (Partition _iter466 : struct.parts)
+        for (Partition _iter456 : struct.parts)
         {
-          _iter466.write(oprot);
+          _iter456.write(oprot);
         }
       }
       oprot.writeBool(struct.ifNotExists);
@@ -824,14 +824,14 @@ import org.slf4j.LoggerFactory;
       struct.tblName = iprot.readString();
       struct.setTblNameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list467 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.parts = new ArrayList<Partition>(_list467.size);
-        Partition _elem468;
-        for (int _i469 = 0; _i469 < _list467.size; ++_i469)
+        org.apache.thrift.protocol.TList _list457 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.parts = new ArrayList<Partition>(_list457.size);
+        Partition _elem458;
+        for (int _i459 = 0; _i459 < _list457.size; ++_i459)
         {
-          _elem468 = new Partition();
-          _elem468.read(iprot);
-          struct.parts.add(_elem468);
+          _elem458 = new Partition();
+          _elem458.read(iprot);
+          struct.parts.add(_elem458);
         }
       }
       struct.setPartsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java
index 49ce6e1..57d4953 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java
@@ -346,14 +346,14 @@ import org.slf4j.LoggerFactory;
           case 1: // PARTITIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list454 = iprot.readListBegin();
-                struct.partitions = new ArrayList<Partition>(_list454.size);
-                Partition _elem455;
-                for (int _i456 = 0; _i456 < _list454.size; ++_i456)
+                org.apache.thrift.protocol.TList _list444 = iprot.readListBegin();
+                struct.partitions = new ArrayList<Partition>(_list444.size);
+                Partition _elem445;
+                for (int _i446 = 0; _i446 < _list444.size; ++_i446)
                 {
-                  _elem455 = new Partition();
-                  _elem455.read(iprot);
-                  struct.partitions.add(_elem455);
+                  _elem445 = new Partition();
+                  _elem445.read(iprot);
+                  struct.partitions.add(_elem445);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PARTITIONS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitions.size()));
-            for (Partition _iter457 : struct.partitions)
+            for (Partition _iter447 : struct.partitions)
             {
-              _iter457.write(oprot);
+              _iter447.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitions()) {
         {
           oprot.writeI32(struct.partitions.size());
-          for (Partition _iter458 : struct.partitions)
+          for (Partition _iter448 : struct.partitions)
           {
-            _iter458.write(oprot);
+            _iter448.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list459 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.partitions = new ArrayList<Partition>(_list459.size);
-          Partition _elem460;
-          for (int _i461 = 0; _i461 < _list459.size; ++_i461)
+          org.apache.thrift.protocol.TList _list449 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.partitions = new ArrayList<Partition>(_list449.size);
+          Partition _elem450;
+          for (int _i451 = 0; _i451 < _list449.size; ++_i451)
           {
-            _elem460 = new Partition();
-            _elem460.read(iprot);
-            struct.partitions.add(_elem460);
+            _elem450 = new Partition();
+            _elem450.read(iprot);
+            struct.partitions.add(_elem450);
           }
         }
         struct.setPartitionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPrimaryKeyRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPrimaryKeyRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPrimaryKeyRequest.java
index 478032a..900985b 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPrimaryKeyRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPrimaryKeyRequest.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // PRIMARY_KEY_COLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list364 = iprot.readListBegin();
-                struct.primaryKeyCols = new ArrayList<SQLPrimaryKey>(_list364.size);
-                SQLPrimaryKey _elem365;
-                for (int _i366 = 0; _i366 < _list364.size; ++_i366)
+                org.apache.thrift.protocol.TList _list354 = iprot.readListBegin();
+                struct.primaryKeyCols = new ArrayList<SQLPrimaryKey>(_list354.size);
+                SQLPrimaryKey _elem355;
+                for (int _i356 = 0; _i356 < _list354.size; ++_i356)
                 {
-                  _elem365 = new SQLPrimaryKey();
-                  _elem365.read(iprot);
-                  struct.primaryKeyCols.add(_elem365);
+                  _elem355 = new SQLPrimaryKey();
+                  _elem355.read(iprot);
+                  struct.primaryKeyCols.add(_elem355);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PRIMARY_KEY_COLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.primaryKeyCols.size()));
-          for (SQLPrimaryKey _iter367 : struct.primaryKeyCols)
+          for (SQLPrimaryKey _iter357 : struct.primaryKeyCols)
           {
-            _iter367.write(oprot);
+            _iter357.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.primaryKeyCols.size());
-        for (SQLPrimaryKey _iter368 : struct.primaryKeyCols)
+        for (SQLPrimaryKey _iter358 : struct.primaryKeyCols)
         {
-          _iter368.write(oprot);
+          _iter358.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, AddPrimaryKeyRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list369 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.primaryKeyCols = new ArrayList<SQLPrimaryKey>(_list369.size);
-        SQLPrimaryKey _elem370;
-        for (int _i371 = 0; _i371 < _list369.size; ++_i371)
+        org.apache.thrift.protocol.TList _list359 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.primaryKeyCols = new ArrayList<SQLPrimaryKey>(_list359.size);
+        SQLPrimaryKey _elem360;
+        for (int _i361 = 0; _i361 < _list359.size; ++_i361)
         {
-          _elem370 = new SQLPrimaryKey();
-          _elem370.read(iprot);
-          struct.primaryKeyCols.add(_elem370);
+          _elem360 = new SQLPrimaryKey();
+          _elem360.read(iprot);
+          struct.primaryKeyCols.add(_elem360);
         }
       }
       struct.setPrimaryKeyColsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddUniqueConstraintRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddUniqueConstraintRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddUniqueConstraintRequest.java
index b58f39f..df4f544 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddUniqueConstraintRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddUniqueConstraintRequest.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // UNIQUE_CONSTRAINT_COLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list380 = iprot.readListBegin();
-                struct.uniqueConstraintCols = new ArrayList<SQLUniqueConstraint>(_list380.size);
-                SQLUniqueConstraint _elem381;
-                for (int _i382 = 0; _i382 < _list380.size; ++_i382)
+                org.apache.thrift.protocol.TList _list370 = iprot.readListBegin();
+                struct.uniqueConstraintCols = new ArrayList<SQLUniqueConstraint>(_list370.size);
+                SQLUniqueConstraint _elem371;
+                for (int _i372 = 0; _i372 < _list370.size; ++_i372)
                 {
-                  _elem381 = new SQLUniqueConstraint();
-                  _elem381.read(iprot);
-                  struct.uniqueConstraintCols.add(_elem381);
+                  _elem371 = new SQLUniqueConstraint();
+                  _elem371.read(iprot);
+                  struct.uniqueConstraintCols.add(_elem371);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(UNIQUE_CONSTRAINT_COLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.uniqueConstraintCols.size()));
-          for (SQLUniqueConstraint _iter383 : struct.uniqueConstraintCols)
+          for (SQLUniqueConstraint _iter373 : struct.uniqueConstraintCols)
           {
-            _iter383.write(oprot);
+            _iter373.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.uniqueConstraintCols.size());
-        for (SQLUniqueConstraint _iter384 : struct.uniqueConstraintCols)
+        for (SQLUniqueConstraint _iter374 : struct.uniqueConstraintCols)
         {
-          _iter384.write(oprot);
+          _iter374.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, AddUniqueConstraintRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list385 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.uniqueConstraintCols = new ArrayList<SQLUniqueConstraint>(_list385.size);
-        SQLUniqueConstraint _elem386;
-        for (int _i387 = 0; _i387 < _list385.size; ++_i387)
+        org.apache.thrift.protocol.TList _list375 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.uniqueConstraintCols = new ArrayList<SQLUniqueConstraint>(_list375.size);
+        SQLUniqueConstraint _elem376;
+        for (int _i377 = 0; _i377 < _list375.size; ++_i377)
         {
-          _elem386 = new SQLUniqueConstraint();
-          _elem386.read(iprot);
-          struct.uniqueConstraintCols.add(_elem386);
+          _elem376 = new SQLUniqueConstraint();
+          _elem376.read(iprot);
+          struct.uniqueConstraintCols.add(_elem376);
         }
       }
       struct.setUniqueConstraintColsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AggrStats.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AggrStats.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AggrStats.java
index 54ef01f..c38c8c6 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AggrStats.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AggrStats.java
@@ -439,14 +439,14 @@ import org.slf4j.LoggerFactory;
           case 1: // COL_STATS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list288 = iprot.readListBegin();
-                struct.colStats = new ArrayList<ColumnStatisticsObj>(_list288.size);
-                ColumnStatisticsObj _elem289;
-                for (int _i290 = 0; _i290 < _list288.size; ++_i290)
+                org.apache.thrift.protocol.TList _list278 = iprot.readListBegin();
+                struct.colStats = new ArrayList<ColumnStatisticsObj>(_list278.size);
+                ColumnStatisticsObj _elem279;
+                for (int _i280 = 0; _i280 < _list278.size; ++_i280)
                 {
-                  _elem289 = new ColumnStatisticsObj();
-                  _elem289.read(iprot);
-                  struct.colStats.add(_elem289);
+                  _elem279 = new ColumnStatisticsObj();
+                  _elem279.read(iprot);
+                  struct.colStats.add(_elem279);
                 }
                 iprot.readListEnd();
               }
@@ -480,9 +480,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COL_STATS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.colStats.size()));
-          for (ColumnStatisticsObj _iter291 : struct.colStats)
+          for (ColumnStatisticsObj _iter281 : struct.colStats)
           {
-            _iter291.write(oprot);
+            _iter281.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -510,9 +510,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.colStats.size());
-        for (ColumnStatisticsObj _iter292 : struct.colStats)
+        for (ColumnStatisticsObj _iter282 : struct.colStats)
         {
-          _iter292.write(oprot);
+          _iter282.write(oprot);
         }
       }
       oprot.writeI64(struct.partsFound);
@@ -522,14 +522,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, AggrStats struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list293 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.colStats = new ArrayList<ColumnStatisticsObj>(_list293.size);
-        ColumnStatisticsObj _elem294;
-        for (int _i295 = 0; _i295 < _list293.size; ++_i295)
+        org.apache.thrift.protocol.TList _list283 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.colStats = new ArrayList<ColumnStatisticsObj>(_list283.size);
+        ColumnStatisticsObj _elem284;
+        for (int _i285 = 0; _i285 < _list283.size; ++_i285)
         {
-          _elem294 = new ColumnStatisticsObj();
-          _elem294.read(iprot);
-          struct.colStats.add(_elem294);
+          _elem284 = new ColumnStatisticsObj();
+          _elem284.read(iprot);
+          struct.colStats.add(_elem284);
         }
       }
       struct.setColStatsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/BasicTxnInfo.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/BasicTxnInfo.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/BasicTxnInfo.java
index f695e5d..da37d03 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/BasicTxnInfo.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/BasicTxnInfo.java
@@ -39,12 +39,11 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("BasicTxnInfo");
 
   private static final org.apache.thrift.protocol.TField ISNULL_FIELD_DESC = new org.apache.thrift.protocol.TField("isnull", org.apache.thrift.protocol.TType.BOOL, (short)1);
-  private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.I64, (short)2);
-  private static final org.apache.thrift.protocol.TField TIME_FIELD_DESC = new org.apache.thrift.protocol.TField("time", org.apache.thrift.protocol.TType.I64, (short)3);
-  private static final org.apache.thrift.protocol.TField TXNID_FIELD_DESC = new org.apache.thrift.protocol.TField("txnid", org.apache.thrift.protocol.TType.I64, (short)4);
-  private static final org.apache.thrift.protocol.TField DBNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbname", org.apache.thrift.protocol.TType.STRING, (short)5);
-  private static final org.apache.thrift.protocol.TField TABLENAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tablename", org.apache.thrift.protocol.TType.STRING, (short)6);
-  private static final org.apache.thrift.protocol.TField PARTITIONNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("partitionname", org.apache.thrift.protocol.TType.STRING, (short)7);
+  private static final org.apache.thrift.protocol.TField TIME_FIELD_DESC = new org.apache.thrift.protocol.TField("time", org.apache.thrift.protocol.TType.I64, (short)2);
+  private static final org.apache.thrift.protocol.TField TXNID_FIELD_DESC = new org.apache.thrift.protocol.TField("txnid", org.apache.thrift.protocol.TType.I64, (short)3);
+  private static final org.apache.thrift.protocol.TField DBNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbname", org.apache.thrift.protocol.TType.STRING, (short)4);
+  private static final org.apache.thrift.protocol.TField TABLENAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tablename", org.apache.thrift.protocol.TType.STRING, (short)5);
+  private static final org.apache.thrift.protocol.TField PARTITIONNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("partitionname", org.apache.thrift.protocol.TType.STRING, (short)6);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -53,7 +52,6 @@ import org.slf4j.LoggerFactory;
   }
 
   private boolean isnull; // required
-  private long id; // optional
   private long time; // optional
   private long txnid; // optional
   private String dbname; // optional
@@ -63,12 +61,11 @@ import org.slf4j.LoggerFactory;
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     ISNULL((short)1, "isnull"),
-    ID((short)2, "id"),
-    TIME((short)3, "time"),
-    TXNID((short)4, "txnid"),
-    DBNAME((short)5, "dbname"),
-    TABLENAME((short)6, "tablename"),
-    PARTITIONNAME((short)7, "partitionname");
+    TIME((short)2, "time"),
+    TXNID((short)3, "txnid"),
+    DBNAME((short)4, "dbname"),
+    TABLENAME((short)5, "tablename"),
+    PARTITIONNAME((short)6, "partitionname");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -85,17 +82,15 @@ import org.slf4j.LoggerFactory;
       switch(fieldId) {
         case 1: // ISNULL
           return ISNULL;
-        case 2: // ID
-          return ID;
-        case 3: // TIME
+        case 2: // TIME
           return TIME;
-        case 4: // TXNID
+        case 3: // TXNID
           return TXNID;
-        case 5: // DBNAME
+        case 4: // DBNAME
           return DBNAME;
-        case 6: // TABLENAME
+        case 5: // TABLENAME
           return TABLENAME;
-        case 7: // PARTITIONNAME
+        case 6: // PARTITIONNAME
           return PARTITIONNAME;
         default:
           return null;
@@ -138,18 +133,15 @@ import org.slf4j.LoggerFactory;
 
   // isset id assignments
   private static final int __ISNULL_ISSET_ID = 0;
-  private static final int __ID_ISSET_ID = 1;
-  private static final int __TIME_ISSET_ID = 2;
-  private static final int __TXNID_ISSET_ID = 3;
+  private static final int __TIME_ISSET_ID = 1;
+  private static final int __TXNID_ISSET_ID = 2;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.ID,_Fields.TIME,_Fields.TXNID,_Fields.DBNAME,_Fields.TABLENAME,_Fields.PARTITIONNAME};
+  private static final _Fields optionals[] = {_Fields.TIME,_Fields.TXNID,_Fields.DBNAME,_Fields.TABLENAME,_Fields.PARTITIONNAME};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
     tmpMap.put(_Fields.ISNULL, new org.apache.thrift.meta_data.FieldMetaData("isnull", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
-    tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
     tmpMap.put(_Fields.TIME, new org.apache.thrift.meta_data.FieldMetaData("time", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
     tmpMap.put(_Fields.TXNID, new org.apache.thrift.meta_data.FieldMetaData("txnid", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
@@ -181,7 +173,6 @@ import org.slf4j.LoggerFactory;
   public BasicTxnInfo(BasicTxnInfo other) {
     __isset_bitfield = other.__isset_bitfield;
     this.isnull = other.isnull;
-    this.id = other.id;
     this.time = other.time;
     this.txnid = other.txnid;
     if (other.isSetDbname()) {
@@ -203,8 +194,6 @@ import org.slf4j.LoggerFactory;
   public void clear() {
     setIsnullIsSet(false);
     this.isnull = false;
-    setIdIsSet(false);
-    this.id = 0;
     setTimeIsSet(false);
     this.time = 0;
     setTxnidIsSet(false);
@@ -236,28 +225,6 @@ import org.slf4j.LoggerFactory;
     __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ISNULL_ISSET_ID, value);
   }
 
-  public long getId() {
-    return this.id;
-  }
-
-  public void setId(long id) {
-    this.id = id;
-    setIdIsSet(true);
-  }
-
-  public void unsetId() {
-    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ID_ISSET_ID);
-  }
-
-  /** Returns true if field id is set (has been assigned a value) and false otherwise */
-  public boolean isSetId() {
-    return EncodingUtils.testBit(__isset_bitfield, __ID_ISSET_ID);
-  }
-
-  public void setIdIsSet(boolean value) {
-    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ID_ISSET_ID, value);
-  }
-
   public long getTime() {
     return this.time;
   }
@@ -381,14 +348,6 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
-    case ID:
-      if (value == null) {
-        unsetId();
-      } else {
-        setId((Long)value);
-      }
-      break;
-
     case TIME:
       if (value == null) {
         unsetTime();
@@ -437,9 +396,6 @@ import org.slf4j.LoggerFactory;
     case ISNULL:
       return isIsnull();
 
-    case ID:
-      return getId();
-
     case TIME:
       return getTime();
 
@@ -468,8 +424,6 @@ import org.slf4j.LoggerFactory;
     switch (field) {
     case ISNULL:
       return isSetIsnull();
-    case ID:
-      return isSetId();
     case TIME:
       return isSetTime();
     case TXNID:
@@ -506,15 +460,6 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
-    boolean this_present_id = true && this.isSetId();
-    boolean that_present_id = true && that.isSetId();
-    if (this_present_id || that_present_id) {
-      if (!(this_present_id && that_present_id))
-        return false;
-      if (this.id != that.id)
-        return false;
-    }
-
     boolean this_present_time = true && this.isSetTime();
     boolean that_present_time = true && that.isSetTime();
     if (this_present_time || that_present_time) {
@@ -572,11 +517,6 @@ import org.slf4j.LoggerFactory;
     if (present_isnull)
       list.add(isnull);
 
-    boolean present_id = true && (isSetId());
-    list.add(present_id);
-    if (present_id)
-      list.add(id);
-
     boolean present_time = true && (isSetTime());
     list.add(present_time);
     if (present_time)
@@ -623,16 +563,6 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(isSetId()).compareTo(other.isSetId());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetId()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
     lastComparison = Boolean.valueOf(isSetTime()).compareTo(other.isSetTime());
     if (lastComparison != 0) {
       return lastComparison;
@@ -706,12 +636,6 @@ import org.slf4j.LoggerFactory;
     sb.append("isnull:");
     sb.append(this.isnull);
     first = false;
-    if (isSetId()) {
-      if (!first) sb.append(", ");
-      sb.append("id:");
-      sb.append(this.id);
-      first = false;
-    }
     if (isSetTime()) {
       if (!first) sb.append(", ");
       sb.append("time:");
@@ -811,15 +735,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 2: // ID
-            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
-              struct.id = iprot.readI64();
-              struct.setIdIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          case 3: // TIME
+          case 2: // TIME
             if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
               struct.time = iprot.readI64();
               struct.setTimeIsSet(true);
@@ -827,7 +743,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 4: // TXNID
+          case 3: // TXNID
             if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
               struct.txnid = iprot.readI64();
               struct.setTxnidIsSet(true);
@@ -835,7 +751,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 5: // DBNAME
+          case 4: // DBNAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.dbname = iprot.readString();
               struct.setDbnameIsSet(true);
@@ -843,7 +759,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 6: // TABLENAME
+          case 5: // TABLENAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.tablename = iprot.readString();
               struct.setTablenameIsSet(true);
@@ -851,7 +767,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 7: // PARTITIONNAME
+          case 6: // PARTITIONNAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.partitionname = iprot.readString();
               struct.setPartitionnameIsSet(true);
@@ -875,11 +791,6 @@ import org.slf4j.LoggerFactory;
       oprot.writeFieldBegin(ISNULL_FIELD_DESC);
       oprot.writeBool(struct.isnull);
       oprot.writeFieldEnd();
-      if (struct.isSetId()) {
-        oprot.writeFieldBegin(ID_FIELD_DESC);
-        oprot.writeI64(struct.id);
-        oprot.writeFieldEnd();
-      }
       if (struct.isSetTime()) {
         oprot.writeFieldBegin(TIME_FIELD_DESC);
         oprot.writeI64(struct.time);
@@ -930,28 +841,22 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       oprot.writeBool(struct.isnull);
       BitSet optionals = new BitSet();
-      if (struct.isSetId()) {
-        optionals.set(0);
-      }
       if (struct.isSetTime()) {
-        optionals.set(1);
+        optionals.set(0);
       }
       if (struct.isSetTxnid()) {
-        optionals.set(2);
+        optionals.set(1);
       }
       if (struct.isSetDbname()) {
-        optionals.set(3);
+        optionals.set(2);
       }
       if (struct.isSetTablename()) {
-        optionals.set(4);
+        optionals.set(3);
       }
       if (struct.isSetPartitionname()) {
-        optionals.set(5);
-      }
-      oprot.writeBitSet(optionals, 6);
-      if (struct.isSetId()) {
-        oprot.writeI64(struct.id);
+        optionals.set(4);
       }
+      oprot.writeBitSet(optionals, 5);
       if (struct.isSetTime()) {
         oprot.writeI64(struct.time);
       }
@@ -974,28 +879,24 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol iprot = (TTupleProtocol) prot;
       struct.isnull = iprot.readBool();
       struct.setIsnullIsSet(true);
-      BitSet incoming = iprot.readBitSet(6);
+      BitSet incoming = iprot.readBitSet(5);
       if (incoming.get(0)) {
-        struct.id = iprot.readI64();
-        struct.setIdIsSet(true);
-      }
-      if (incoming.get(1)) {
         struct.time = iprot.readI64();
         struct.setTimeIsSet(true);
       }
-      if (incoming.get(2)) {
+      if (incoming.get(1)) {
         struct.txnid = iprot.readI64();
         struct.setTxnidIsSet(true);
       }
-      if (incoming.get(3)) {
+      if (incoming.get(2)) {
         struct.dbname = iprot.readString();
         struct.setDbnameIsSet(true);
       }
-      if (incoming.get(4)) {
+      if (incoming.get(3)) {
         struct.tablename = iprot.readString();
         struct.setTablenameIsSet(true);
       }
-      if (incoming.get(5)) {
+      if (incoming.get(4)) {
         struct.partitionname = iprot.readString();
         struct.setPartitionnameIsSet(true);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
index dbda2ab..16e85cf 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
@@ -351,13 +351,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list716 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list716.size);
-                long _elem717;
-                for (int _i718 = 0; _i718 < _list716.size; ++_i718)
+                org.apache.thrift.protocol.TList _list706 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list706.size);
+                long _elem707;
+                for (int _i708 = 0; _i708 < _list706.size; ++_i708)
                 {
-                  _elem717 = iprot.readI64();
-                  struct.fileIds.add(_elem717);
+                  _elem707 = iprot.readI64();
+                  struct.fileIds.add(_elem707);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size()));
-          for (long _iter719 : struct.fileIds)
+          for (long _iter709 : struct.fileIds)
           {
-            oprot.writeI64(_iter719);
+            oprot.writeI64(_iter709);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter720 : struct.fileIds)
+        for (long _iter710 : struct.fileIds)
         {
-          oprot.writeI64(_iter720);
+          oprot.writeI64(_iter710);
         }
       }
     }
@@ -421,13 +421,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, ClearFileMetadataRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list721 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list721.size);
-        long _elem722;
-        for (int _i723 = 0; _i723 < _list721.size; ++_i723)
+        org.apache.thrift.protocol.TList _list711 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list711.size);
+        long _elem712;
+        for (int _i713 = 0; _i713 < _list711.size; ++_i713)
         {
-          _elem722 = iprot.readI64();
-          struct.fileIds.add(_elem722);
+          _elem712 = iprot.readI64();
+          struct.fileIds.add(_elem712);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
index 0df33f1..816b61b 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
@@ -354,13 +354,13 @@ import org.slf4j.LoggerFactory;
           case 1: // VALUES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list732 = iprot.readListBegin();
-                struct.values = new ArrayList<ClientCapability>(_list732.size);
-                ClientCapability _elem733;
-                for (int _i734 = 0; _i734 < _list732.size; ++_i734)
+                org.apache.thrift.protocol.TList _list722 = iprot.readListBegin();
+                struct.values = new ArrayList<ClientCapability>(_list722.size);
+                ClientCapability _elem723;
+                for (int _i724 = 0; _i724 < _list722.size; ++_i724)
                 {
-                  _elem733 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
-                  struct.values.add(_elem733);
+                  _elem723 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
+                  struct.values.add(_elem723);
                 }
                 iprot.readListEnd();
               }
@@ -386,9 +386,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(VALUES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, struct.values.size()));
-          for (ClientCapability _iter735 : struct.values)
+          for (ClientCapability _iter725 : struct.values)
           {
-            oprot.writeI32(_iter735.getValue());
+            oprot.writeI32(_iter725.getValue());
           }
           oprot.writeListEnd();
         }
@@ -413,9 +413,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.values.size());
-        for (ClientCapability _iter736 : struct.values)
+        for (ClientCapability _iter726 : struct.values)
         {
-          oprot.writeI32(_iter736.getValue());
+          oprot.writeI32(_iter726.getValue());
         }
       }
     }
@@ -424,13 +424,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, ClientCapabilities struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list737 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32());
-        struct.values = new ArrayList<ClientCapability>(_list737.size);
-        ClientCapability _elem738;
-        for (int _i739 = 0; _i739 < _list737.size; ++_i739)
+        org.apache.thrift.protocol.TList _list727 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32());
+        struct.values = new ArrayList<ClientCapability>(_list727.size);
+        ClientCapability _elem728;
+        for (int _i729 = 0; _i729 < _list727.size; ++_i729)
         {
-          _elem738 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
-          struct.values.add(_elem738);
+          _elem728 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
+          struct.values.add(_elem728);
         }
       }
       struct.setValuesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatistics.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatistics.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatistics.java
index 962bb1c..765889e 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatistics.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatistics.java
@@ -451,14 +451,14 @@ import org.slf4j.LoggerFactory;
           case 2: // STATS_OBJ
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list280 = iprot.readListBegin();
-                struct.statsObj = new ArrayList<ColumnStatisticsObj>(_list280.size);
-                ColumnStatisticsObj _elem281;
-                for (int _i282 = 0; _i282 < _list280.size; ++_i282)
+                org.apache.thrift.protocol.TList _list270 = iprot.readListBegin();
+                struct.statsObj = new ArrayList<ColumnStatisticsObj>(_list270.size);
+                ColumnStatisticsObj _elem271;
+                for (int _i272 = 0; _i272 < _list270.size; ++_i272)
                 {
-                  _elem281 = new ColumnStatisticsObj();
-                  _elem281.read(iprot);
-                  struct.statsObj.add(_elem281);
+                  _elem271 = new ColumnStatisticsObj();
+                  _elem271.read(iprot);
+                  struct.statsObj.add(_elem271);
                 }
                 iprot.readListEnd();
               }
@@ -489,9 +489,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(STATS_OBJ_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.statsObj.size()));
-          for (ColumnStatisticsObj _iter283 : struct.statsObj)
+          for (ColumnStatisticsObj _iter273 : struct.statsObj)
           {
-            _iter283.write(oprot);
+            _iter273.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -517,9 +517,9 @@ import org.slf4j.LoggerFactory;
       struct.statsDesc.write(oprot);
       {
         oprot.writeI32(struct.statsObj.size());
-        for (ColumnStatisticsObj _iter284 : struct.statsObj)
+        for (ColumnStatisticsObj _iter274 : struct.statsObj)
         {
-          _iter284.write(oprot);
+          _iter274.write(oprot);
         }
       }
     }
@@ -531,14 +531,14 @@ import org.slf4j.LoggerFactory;
       struct.statsDesc.read(iprot);
       struct.setStatsDescIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list285 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.statsObj = new ArrayList<ColumnStatisticsObj>(_list285.size);
-        ColumnStatisticsObj _elem286;
-        for (int _i287 = 0; _i287 < _list285.size; ++_i287)
+        org.apache.thrift.protocol.TList _list275 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.statsObj = new ArrayList<ColumnStatisticsObj>(_list275.size);
+        ColumnStatisticsObj _elem276;
+        for (int _i277 = 0; _i277 < _list275.size; ++_i277)
         {
-          _elem286 = new ColumnStatisticsObj();
-          _elem286.read(iprot);
-          struct.statsObj.add(_elem286);
+          _elem276 = new ColumnStatisticsObj();
+          _elem276.read(iprot);
+          struct.statsObj.add(_elem276);
         }
       }
       struct.setStatsObjIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
index b92293e..6da2b88 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
@@ -814,15 +814,15 @@ import org.slf4j.LoggerFactory;
           case 6: // PROPERTIES
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map598 = iprot.readMapBegin();
-                struct.properties = new HashMap<String,String>(2*_map598.size);
-                String _key599;
-                String _val600;
-                for (int _i601 = 0; _i601 < _map598.size; ++_i601)
+                org.apache.thrift.protocol.TMap _map588 = iprot.readMapBegin();
+                struct.properties = new HashMap<String,String>(2*_map588.size);
+                String _key589;
+                String _val590;
+                for (int _i591 = 0; _i591 < _map588.size; ++_i591)
                 {
-                  _key599 = iprot.readString();
-                  _val600 = iprot.readString();
-                  struct.properties.put(_key599, _val600);
+                  _key589 = iprot.readString();
+                  _val590 = iprot.readString();
+                  struct.properties.put(_key589, _val590);
                 }
                 iprot.readMapEnd();
               }
@@ -878,10 +878,10 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PROPERTIES_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.properties.size()));
-            for (Map.Entry<String, String> _iter602 : struct.properties.entrySet())
+            for (Map.Entry<String, String> _iter592 : struct.properties.entrySet())
             {
-              oprot.writeString(_iter602.getKey());
-              oprot.writeString(_iter602.getValue());
+              oprot.writeString(_iter592.getKey());
+              oprot.writeString(_iter592.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -928,10 +928,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetProperties()) {
         {
           oprot.writeI32(struct.properties.size());
-          for (Map.Entry<String, String> _iter603 : struct.properties.entrySet())
+          for (Map.Entry<String, String> _iter593 : struct.properties.entrySet())
           {
-            oprot.writeString(_iter603.getKey());
-            oprot.writeString(_iter603.getValue());
+            oprot.writeString(_iter593.getKey());
+            oprot.writeString(_iter593.getValue());
           }
         }
       }
@@ -957,15 +957,15 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TMap _map604 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.properties = new HashMap<String,String>(2*_map604.size);
-          String _key605;
-          String _val606;
-          for (int _i607 = 0; _i607 < _map604.size; ++_i607)
+          org.apache.thrift.protocol.TMap _map594 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.properties = new HashMap<String,String>(2*_map594.size);
+          String _key595;
+          String _val596;
+          for (int _i597 = 0; _i597 < _map594.size; ++_i597)
           {
-            _key605 = iprot.readString();
-            _val606 = iprot.readString();
-            struct.properties.put(_key605, _val606);
+            _key595 = iprot.readString();
+            _val596 = iprot.readString();
+            struct.properties.put(_key595, _val596);
           }
         }
         struct.setPropertiesIsSet(true);


[03/16] hive git commit: HIVE-18546: Remove unnecessary code introduced in HIVE-14498 (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
index 988c01a..2e19105 100644
--- a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -1236,24 +1236,6 @@ class Iface(fb303.FacebookService.Iface):
     """
     pass
 
-  def get_last_completed_transaction_for_tables(self, db_names, table_names, txns_snapshot):
-    """
-    Parameters:
-     - db_names
-     - table_names
-     - txns_snapshot
-    """
-    pass
-
-  def get_last_completed_transaction_for_table(self, db_name, table_name, txns_snapshot):
-    """
-    Parameters:
-     - db_name
-     - table_name
-     - txns_snapshot
-    """
-    pass
-
   def get_next_notification(self, rqst):
     """
     Parameters:
@@ -6992,76 +6974,6 @@ class Client(fb303.FacebookService.Client, Iface):
       raise result.o2
     return
 
-  def get_last_completed_transaction_for_tables(self, db_names, table_names, txns_snapshot):
-    """
-    Parameters:
-     - db_names
-     - table_names
-     - txns_snapshot
-    """
-    self.send_get_last_completed_transaction_for_tables(db_names, table_names, txns_snapshot)
-    return self.recv_get_last_completed_transaction_for_tables()
-
-  def send_get_last_completed_transaction_for_tables(self, db_names, table_names, txns_snapshot):
-    self._oprot.writeMessageBegin('get_last_completed_transaction_for_tables', TMessageType.CALL, self._seqid)
-    args = get_last_completed_transaction_for_tables_args()
-    args.db_names = db_names
-    args.table_names = table_names
-    args.txns_snapshot = txns_snapshot
-    args.write(self._oprot)
-    self._oprot.writeMessageEnd()
-    self._oprot.trans.flush()
-
-  def recv_get_last_completed_transaction_for_tables(self):
-    iprot = self._iprot
-    (fname, mtype, rseqid) = iprot.readMessageBegin()
-    if mtype == TMessageType.EXCEPTION:
-      x = TApplicationException()
-      x.read(iprot)
-      iprot.readMessageEnd()
-      raise x
-    result = get_last_completed_transaction_for_tables_result()
-    result.read(iprot)
-    iprot.readMessageEnd()
-    if result.success is not None:
-      return result.success
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "get_last_completed_transaction_for_tables failed: unknown result")
-
-  def get_last_completed_transaction_for_table(self, db_name, table_name, txns_snapshot):
-    """
-    Parameters:
-     - db_name
-     - table_name
-     - txns_snapshot
-    """
-    self.send_get_last_completed_transaction_for_table(db_name, table_name, txns_snapshot)
-    return self.recv_get_last_completed_transaction_for_table()
-
-  def send_get_last_completed_transaction_for_table(self, db_name, table_name, txns_snapshot):
-    self._oprot.writeMessageBegin('get_last_completed_transaction_for_table', TMessageType.CALL, self._seqid)
-    args = get_last_completed_transaction_for_table_args()
-    args.db_name = db_name
-    args.table_name = table_name
-    args.txns_snapshot = txns_snapshot
-    args.write(self._oprot)
-    self._oprot.writeMessageEnd()
-    self._oprot.trans.flush()
-
-  def recv_get_last_completed_transaction_for_table(self):
-    iprot = self._iprot
-    (fname, mtype, rseqid) = iprot.readMessageBegin()
-    if mtype == TMessageType.EXCEPTION:
-      x = TApplicationException()
-      x.read(iprot)
-      iprot.readMessageEnd()
-      raise x
-    result = get_last_completed_transaction_for_table_result()
-    result.read(iprot)
-    iprot.readMessageEnd()
-    if result.success is not None:
-      return result.success
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "get_last_completed_transaction_for_table failed: unknown result")
-
   def get_next_notification(self, rqst):
     """
     Parameters:
@@ -8202,8 +8114,6 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     self._processMap["compact2"] = Processor.process_compact2
     self._processMap["show_compact"] = Processor.process_show_compact
     self._processMap["add_dynamic_partitions"] = Processor.process_add_dynamic_partitions
-    self._processMap["get_last_completed_transaction_for_tables"] = Processor.process_get_last_completed_transaction_for_tables
-    self._processMap["get_last_completed_transaction_for_table"] = Processor.process_get_last_completed_transaction_for_table
     self._processMap["get_next_notification"] = Processor.process_get_next_notification
     self._processMap["get_current_notificationEventId"] = Processor.process_get_current_notificationEventId
     self._processMap["get_notification_events_count"] = Processor.process_get_notification_events_count
@@ -12010,44 +11920,6 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
-  def process_get_last_completed_transaction_for_tables(self, seqid, iprot, oprot):
-    args = get_last_completed_transaction_for_tables_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = get_last_completed_transaction_for_tables_result()
-    try:
-      result.success = self._handler.get_last_completed_transaction_for_tables(args.db_names, args.table_names, args.txns_snapshot)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("get_last_completed_transaction_for_tables", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
-
-  def process_get_last_completed_transaction_for_table(self, seqid, iprot, oprot):
-    args = get_last_completed_transaction_for_table_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = get_last_completed_transaction_for_table_result()
-    try:
-      result.success = self._handler.get_last_completed_transaction_for_table(args.db_name, args.table_name, args.txns_snapshot)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("get_last_completed_transaction_for_table", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
-
   def process_get_next_notification(self, seqid, iprot, oprot):
     args = get_next_notification_args()
     args.read(iprot)
@@ -13638,10 +13510,10 @@ class get_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype732, _size729) = iprot.readListBegin()
-          for _i733 in xrange(_size729):
-            _elem734 = iprot.readString()
-            self.success.append(_elem734)
+          (_etype723, _size720) = iprot.readListBegin()
+          for _i724 in xrange(_size720):
+            _elem725 = iprot.readString()
+            self.success.append(_elem725)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13664,8 +13536,8 @@ class get_databases_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter735 in self.success:
-        oprot.writeString(iter735)
+      for iter726 in self.success:
+        oprot.writeString(iter726)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -13770,10 +13642,10 @@ class get_all_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype739, _size736) = iprot.readListBegin()
-          for _i740 in xrange(_size736):
-            _elem741 = iprot.readString()
-            self.success.append(_elem741)
+          (_etype730, _size727) = iprot.readListBegin()
+          for _i731 in xrange(_size727):
+            _elem732 = iprot.readString()
+            self.success.append(_elem732)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13796,8 +13668,8 @@ class get_all_databases_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter742 in self.success:
-        oprot.writeString(iter742)
+      for iter733 in self.success:
+        oprot.writeString(iter733)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -14567,12 +14439,12 @@ class get_type_all_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype744, _vtype745, _size743 ) = iprot.readMapBegin()
-          for _i747 in xrange(_size743):
-            _key748 = iprot.readString()
-            _val749 = Type()
-            _val749.read(iprot)
-            self.success[_key748] = _val749
+          (_ktype735, _vtype736, _size734 ) = iprot.readMapBegin()
+          for _i738 in xrange(_size734):
+            _key739 = iprot.readString()
+            _val740 = Type()
+            _val740.read(iprot)
+            self.success[_key739] = _val740
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -14595,9 +14467,9 @@ class get_type_all_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.MAP, 0)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.success))
-      for kiter750,viter751 in self.success.items():
-        oprot.writeString(kiter750)
-        viter751.write(oprot)
+      for kiter741,viter742 in self.success.items():
+        oprot.writeString(kiter741)
+        viter742.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -14740,11 +14612,11 @@ class get_fields_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype755, _size752) = iprot.readListBegin()
-          for _i756 in xrange(_size752):
-            _elem757 = FieldSchema()
-            _elem757.read(iprot)
-            self.success.append(_elem757)
+          (_etype746, _size743) = iprot.readListBegin()
+          for _i747 in xrange(_size743):
+            _elem748 = FieldSchema()
+            _elem748.read(iprot)
+            self.success.append(_elem748)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14779,8 +14651,8 @@ class get_fields_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter758 in self.success:
-        iter758.write(oprot)
+      for iter749 in self.success:
+        iter749.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -14947,11 +14819,11 @@ class get_fields_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype762, _size759) = iprot.readListBegin()
-          for _i763 in xrange(_size759):
-            _elem764 = FieldSchema()
-            _elem764.read(iprot)
-            self.success.append(_elem764)
+          (_etype753, _size750) = iprot.readListBegin()
+          for _i754 in xrange(_size750):
+            _elem755 = FieldSchema()
+            _elem755.read(iprot)
+            self.success.append(_elem755)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14986,8 +14858,8 @@ class get_fields_with_environment_context_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter765 in self.success:
-        iter765.write(oprot)
+      for iter756 in self.success:
+        iter756.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15140,11 +15012,11 @@ class get_schema_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype769, _size766) = iprot.readListBegin()
-          for _i770 in xrange(_size766):
-            _elem771 = FieldSchema()
-            _elem771.read(iprot)
-            self.success.append(_elem771)
+          (_etype760, _size757) = iprot.readListBegin()
+          for _i761 in xrange(_size757):
+            _elem762 = FieldSchema()
+            _elem762.read(iprot)
+            self.success.append(_elem762)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15179,8 +15051,8 @@ class get_schema_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter772 in self.success:
-        iter772.write(oprot)
+      for iter763 in self.success:
+        iter763.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15347,11 +15219,11 @@ class get_schema_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype776, _size773) = iprot.readListBegin()
-          for _i777 in xrange(_size773):
-            _elem778 = FieldSchema()
-            _elem778.read(iprot)
-            self.success.append(_elem778)
+          (_etype767, _size764) = iprot.readListBegin()
+          for _i768 in xrange(_size764):
+            _elem769 = FieldSchema()
+            _elem769.read(iprot)
+            self.success.append(_elem769)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15386,8 +15258,8 @@ class get_schema_with_environment_context_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter779 in self.success:
-        iter779.write(oprot)
+      for iter770 in self.success:
+        iter770.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15834,44 +15706,44 @@ class create_table_with_constraints_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.primaryKeys = []
-          (_etype783, _size780) = iprot.readListBegin()
-          for _i784 in xrange(_size780):
-            _elem785 = SQLPrimaryKey()
-            _elem785.read(iprot)
-            self.primaryKeys.append(_elem785)
+          (_etype774, _size771) = iprot.readListBegin()
+          for _i775 in xrange(_size771):
+            _elem776 = SQLPrimaryKey()
+            _elem776.read(iprot)
+            self.primaryKeys.append(_elem776)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.foreignKeys = []
-          (_etype789, _size786) = iprot.readListBegin()
-          for _i790 in xrange(_size786):
-            _elem791 = SQLForeignKey()
-            _elem791.read(iprot)
-            self.foreignKeys.append(_elem791)
+          (_etype780, _size777) = iprot.readListBegin()
+          for _i781 in xrange(_size777):
+            _elem782 = SQLForeignKey()
+            _elem782.read(iprot)
+            self.foreignKeys.append(_elem782)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.LIST:
           self.uniqueConstraints = []
-          (_etype795, _size792) = iprot.readListBegin()
-          for _i796 in xrange(_size792):
-            _elem797 = SQLUniqueConstraint()
-            _elem797.read(iprot)
-            self.uniqueConstraints.append(_elem797)
+          (_etype786, _size783) = iprot.readListBegin()
+          for _i787 in xrange(_size783):
+            _elem788 = SQLUniqueConstraint()
+            _elem788.read(iprot)
+            self.uniqueConstraints.append(_elem788)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.LIST:
           self.notNullConstraints = []
-          (_etype801, _size798) = iprot.readListBegin()
-          for _i802 in xrange(_size798):
-            _elem803 = SQLNotNullConstraint()
-            _elem803.read(iprot)
-            self.notNullConstraints.append(_elem803)
+          (_etype792, _size789) = iprot.readListBegin()
+          for _i793 in xrange(_size789):
+            _elem794 = SQLNotNullConstraint()
+            _elem794.read(iprot)
+            self.notNullConstraints.append(_elem794)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15892,29 +15764,29 @@ class create_table_with_constraints_args:
     if self.primaryKeys is not None:
       oprot.writeFieldBegin('primaryKeys', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.primaryKeys))
-      for iter804 in self.primaryKeys:
-        iter804.write(oprot)
+      for iter795 in self.primaryKeys:
+        iter795.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.foreignKeys is not None:
       oprot.writeFieldBegin('foreignKeys', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.foreignKeys))
-      for iter805 in self.foreignKeys:
-        iter805.write(oprot)
+      for iter796 in self.foreignKeys:
+        iter796.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.uniqueConstraints is not None:
       oprot.writeFieldBegin('uniqueConstraints', TType.LIST, 4)
       oprot.writeListBegin(TType.STRUCT, len(self.uniqueConstraints))
-      for iter806 in self.uniqueConstraints:
-        iter806.write(oprot)
+      for iter797 in self.uniqueConstraints:
+        iter797.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.notNullConstraints is not None:
       oprot.writeFieldBegin('notNullConstraints', TType.LIST, 5)
       oprot.writeListBegin(TType.STRUCT, len(self.notNullConstraints))
-      for iter807 in self.notNullConstraints:
-        iter807.write(oprot)
+      for iter798 in self.notNullConstraints:
+        iter798.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17180,10 +17052,10 @@ class truncate_table_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.partNames = []
-          (_etype811, _size808) = iprot.readListBegin()
-          for _i812 in xrange(_size808):
-            _elem813 = iprot.readString()
-            self.partNames.append(_elem813)
+          (_etype802, _size799) = iprot.readListBegin()
+          for _i803 in xrange(_size799):
+            _elem804 = iprot.readString()
+            self.partNames.append(_elem804)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17208,8 +17080,8 @@ class truncate_table_args:
     if self.partNames is not None:
       oprot.writeFieldBegin('partNames', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.partNames))
-      for iter814 in self.partNames:
-        oprot.writeString(iter814)
+      for iter805 in self.partNames:
+        oprot.writeString(iter805)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17409,10 +17281,10 @@ class get_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype818, _size815) = iprot.readListBegin()
-          for _i819 in xrange(_size815):
-            _elem820 = iprot.readString()
-            self.success.append(_elem820)
+          (_etype809, _size806) = iprot.readListBegin()
+          for _i810 in xrange(_size806):
+            _elem811 = iprot.readString()
+            self.success.append(_elem811)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17435,8 +17307,8 @@ class get_tables_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter821 in self.success:
-        oprot.writeString(iter821)
+      for iter812 in self.success:
+        oprot.writeString(iter812)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17586,10 +17458,10 @@ class get_tables_by_type_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype825, _size822) = iprot.readListBegin()
-          for _i826 in xrange(_size822):
-            _elem827 = iprot.readString()
-            self.success.append(_elem827)
+          (_etype816, _size813) = iprot.readListBegin()
+          for _i817 in xrange(_size813):
+            _elem818 = iprot.readString()
+            self.success.append(_elem818)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17612,8 +17484,8 @@ class get_tables_by_type_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter828 in self.success:
-        oprot.writeString(iter828)
+      for iter819 in self.success:
+        oprot.writeString(iter819)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17737,10 +17609,10 @@ class get_materialized_views_for_rewriting_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype832, _size829) = iprot.readListBegin()
-          for _i833 in xrange(_size829):
-            _elem834 = iprot.readString()
-            self.success.append(_elem834)
+          (_etype823, _size820) = iprot.readListBegin()
+          for _i824 in xrange(_size820):
+            _elem825 = iprot.readString()
+            self.success.append(_elem825)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17763,8 +17635,8 @@ class get_materialized_views_for_rewriting_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter835 in self.success:
-        oprot.writeString(iter835)
+      for iter826 in self.success:
+        oprot.writeString(iter826)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17837,10 +17709,10 @@ class get_table_meta_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.tbl_types = []
-          (_etype839, _size836) = iprot.readListBegin()
-          for _i840 in xrange(_size836):
-            _elem841 = iprot.readString()
-            self.tbl_types.append(_elem841)
+          (_etype830, _size827) = iprot.readListBegin()
+          for _i831 in xrange(_size827):
+            _elem832 = iprot.readString()
+            self.tbl_types.append(_elem832)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17865,8 +17737,8 @@ class get_table_meta_args:
     if self.tbl_types is not None:
       oprot.writeFieldBegin('tbl_types', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.tbl_types))
-      for iter842 in self.tbl_types:
-        oprot.writeString(iter842)
+      for iter833 in self.tbl_types:
+        oprot.writeString(iter833)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17922,11 +17794,11 @@ class get_table_meta_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype846, _size843) = iprot.readListBegin()
-          for _i847 in xrange(_size843):
-            _elem848 = TableMeta()
-            _elem848.read(iprot)
-            self.success.append(_elem848)
+          (_etype837, _size834) = iprot.readListBegin()
+          for _i838 in xrange(_size834):
+            _elem839 = TableMeta()
+            _elem839.read(iprot)
+            self.success.append(_elem839)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17949,8 +17821,8 @@ class get_table_meta_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter849 in self.success:
-        iter849.write(oprot)
+      for iter840 in self.success:
+        iter840.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18074,10 +17946,10 @@ class get_all_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype853, _size850) = iprot.readListBegin()
-          for _i854 in xrange(_size850):
-            _elem855 = iprot.readString()
-            self.success.append(_elem855)
+          (_etype844, _size841) = iprot.readListBegin()
+          for _i845 in xrange(_size841):
+            _elem846 = iprot.readString()
+            self.success.append(_elem846)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18100,8 +17972,8 @@ class get_all_tables_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter856 in self.success:
-        oprot.writeString(iter856)
+      for iter847 in self.success:
+        oprot.writeString(iter847)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18337,10 +18209,10 @@ class get_table_objects_by_name_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype860, _size857) = iprot.readListBegin()
-          for _i861 in xrange(_size857):
-            _elem862 = iprot.readString()
-            self.tbl_names.append(_elem862)
+          (_etype851, _size848) = iprot.readListBegin()
+          for _i852 in xrange(_size848):
+            _elem853 = iprot.readString()
+            self.tbl_names.append(_elem853)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18361,8 +18233,8 @@ class get_table_objects_by_name_args:
     if self.tbl_names is not None:
       oprot.writeFieldBegin('tbl_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.tbl_names))
-      for iter863 in self.tbl_names:
-        oprot.writeString(iter863)
+      for iter854 in self.tbl_names:
+        oprot.writeString(iter854)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18414,11 +18286,11 @@ class get_table_objects_by_name_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype867, _size864) = iprot.readListBegin()
-          for _i868 in xrange(_size864):
-            _elem869 = Table()
-            _elem869.read(iprot)
-            self.success.append(_elem869)
+          (_etype858, _size855) = iprot.readListBegin()
+          for _i859 in xrange(_size855):
+            _elem860 = Table()
+            _elem860.read(iprot)
+            self.success.append(_elem860)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18435,8 +18307,8 @@ class get_table_objects_by_name_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter870 in self.success:
-        iter870.write(oprot)
+      for iter861 in self.success:
+        iter861.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18828,10 +18700,10 @@ class get_materialization_invalidation_info_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype874, _size871) = iprot.readListBegin()
-          for _i875 in xrange(_size871):
-            _elem876 = iprot.readString()
-            self.tbl_names.append(_elem876)
+          (_etype865, _size862) = iprot.readListBegin()
+          for _i866 in xrange(_size862):
+            _elem867 = iprot.readString()
+            self.tbl_names.append(_elem867)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18852,8 +18724,8 @@ class get_materialization_invalidation_info_args:
     if self.tbl_names is not None:
       oprot.writeFieldBegin('tbl_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.tbl_names))
-      for iter877 in self.tbl_names:
-        oprot.writeString(iter877)
+      for iter868 in self.tbl_names:
+        oprot.writeString(iter868)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18914,12 +18786,12 @@ class get_materialization_invalidation_info_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype879, _vtype880, _size878 ) = iprot.readMapBegin()
-          for _i882 in xrange(_size878):
-            _key883 = iprot.readString()
-            _val884 = Materialization()
-            _val884.read(iprot)
-            self.success[_key883] = _val884
+          (_ktype870, _vtype871, _size869 ) = iprot.readMapBegin()
+          for _i873 in xrange(_size869):
+            _key874 = iprot.readString()
+            _val875 = Materialization()
+            _val875.read(iprot)
+            self.success[_key874] = _val875
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -18954,9 +18826,9 @@ class get_materialization_invalidation_info_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.MAP, 0)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.success))
-      for kiter885,viter886 in self.success.items():
-        oprot.writeString(kiter885)
-        viter886.write(oprot)
+      for kiter876,viter877 in self.success.items():
+        oprot.writeString(kiter876)
+        viter877.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19122,10 +18994,10 @@ class get_table_names_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype890, _size887) = iprot.readListBegin()
-          for _i891 in xrange(_size887):
-            _elem892 = iprot.readString()
-            self.success.append(_elem892)
+          (_etype881, _size878) = iprot.readListBegin()
+          for _i882 in xrange(_size878):
+            _elem883 = iprot.readString()
+            self.success.append(_elem883)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19160,8 +19032,8 @@ class get_table_names_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter893 in self.success:
-        oprot.writeString(iter893)
+      for iter884 in self.success:
+        oprot.writeString(iter884)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20131,11 +20003,11 @@ class add_partitions_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype897, _size894) = iprot.readListBegin()
-          for _i898 in xrange(_size894):
-            _elem899 = Partition()
-            _elem899.read(iprot)
-            self.new_parts.append(_elem899)
+          (_etype888, _size885) = iprot.readListBegin()
+          for _i889 in xrange(_size885):
+            _elem890 = Partition()
+            _elem890.read(iprot)
+            self.new_parts.append(_elem890)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20152,8 +20024,8 @@ class add_partitions_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter900 in self.new_parts:
-        iter900.write(oprot)
+      for iter891 in self.new_parts:
+        iter891.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20311,11 +20183,11 @@ class add_partitions_pspec_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype904, _size901) = iprot.readListBegin()
-          for _i905 in xrange(_size901):
-            _elem906 = PartitionSpec()
-            _elem906.read(iprot)
-            self.new_parts.append(_elem906)
+          (_etype895, _size892) = iprot.readListBegin()
+          for _i896 in xrange(_size892):
+            _elem897 = PartitionSpec()
+            _elem897.read(iprot)
+            self.new_parts.append(_elem897)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20332,8 +20204,8 @@ class add_partitions_pspec_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter907 in self.new_parts:
-        iter907.write(oprot)
+      for iter898 in self.new_parts:
+        iter898.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20507,10 +20379,10 @@ class append_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype911, _size908) = iprot.readListBegin()
-          for _i912 in xrange(_size908):
-            _elem913 = iprot.readString()
-            self.part_vals.append(_elem913)
+          (_etype902, _size899) = iprot.readListBegin()
+          for _i903 in xrange(_size899):
+            _elem904 = iprot.readString()
+            self.part_vals.append(_elem904)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20535,8 +20407,8 @@ class append_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter914 in self.part_vals:
-        oprot.writeString(iter914)
+      for iter905 in self.part_vals:
+        oprot.writeString(iter905)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20889,10 +20761,10 @@ class append_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype918, _size915) = iprot.readListBegin()
-          for _i919 in xrange(_size915):
-            _elem920 = iprot.readString()
-            self.part_vals.append(_elem920)
+          (_etype909, _size906) = iprot.readListBegin()
+          for _i910 in xrange(_size906):
+            _elem911 = iprot.readString()
+            self.part_vals.append(_elem911)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20923,8 +20795,8 @@ class append_partition_with_environment_context_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter921 in self.part_vals:
-        oprot.writeString(iter921)
+      for iter912 in self.part_vals:
+        oprot.writeString(iter912)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -21519,10 +21391,10 @@ class drop_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype925, _size922) = iprot.readListBegin()
-          for _i926 in xrange(_size922):
-            _elem927 = iprot.readString()
-            self.part_vals.append(_elem927)
+          (_etype916, _size913) = iprot.readListBegin()
+          for _i917 in xrange(_size913):
+            _elem918 = iprot.readString()
+            self.part_vals.append(_elem918)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21552,8 +21424,8 @@ class drop_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter928 in self.part_vals:
-        oprot.writeString(iter928)
+      for iter919 in self.part_vals:
+        oprot.writeString(iter919)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -21726,10 +21598,10 @@ class drop_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype932, _size929) = iprot.readListBegin()
-          for _i933 in xrange(_size929):
-            _elem934 = iprot.readString()
-            self.part_vals.append(_elem934)
+          (_etype923, _size920) = iprot.readListBegin()
+          for _i924 in xrange(_size920):
+            _elem925 = iprot.readString()
+            self.part_vals.append(_elem925)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21765,8 +21637,8 @@ class drop_partition_with_environment_context_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter935 in self.part_vals:
-        oprot.writeString(iter935)
+      for iter926 in self.part_vals:
+        oprot.writeString(iter926)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -22503,10 +22375,10 @@ class get_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype939, _size936) = iprot.readListBegin()
-          for _i940 in xrange(_size936):
-            _elem941 = iprot.readString()
-            self.part_vals.append(_elem941)
+          (_etype930, _size927) = iprot.readListBegin()
+          for _i931 in xrange(_size927):
+            _elem932 = iprot.readString()
+            self.part_vals.append(_elem932)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22531,8 +22403,8 @@ class get_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter942 in self.part_vals:
-        oprot.writeString(iter942)
+      for iter933 in self.part_vals:
+        oprot.writeString(iter933)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -22691,11 +22563,11 @@ class exchange_partition_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype944, _vtype945, _size943 ) = iprot.readMapBegin()
-          for _i947 in xrange(_size943):
-            _key948 = iprot.readString()
-            _val949 = iprot.readString()
-            self.partitionSpecs[_key948] = _val949
+          (_ktype935, _vtype936, _size934 ) = iprot.readMapBegin()
+          for _i938 in xrange(_size934):
+            _key939 = iprot.readString()
+            _val940 = iprot.readString()
+            self.partitionSpecs[_key939] = _val940
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -22732,9 +22604,9 @@ class exchange_partition_args:
     if self.partitionSpecs is not None:
       oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs))
-      for kiter950,viter951 in self.partitionSpecs.items():
-        oprot.writeString(kiter950)
-        oprot.writeString(viter951)
+      for kiter941,viter942 in self.partitionSpecs.items():
+        oprot.writeString(kiter941)
+        oprot.writeString(viter942)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -22939,11 +22811,11 @@ class exchange_partitions_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype953, _vtype954, _size952 ) = iprot.readMapBegin()
-          for _i956 in xrange(_size952):
-            _key957 = iprot.readString()
-            _val958 = iprot.readString()
-            self.partitionSpecs[_key957] = _val958
+          (_ktype944, _vtype945, _size943 ) = iprot.readMapBegin()
+          for _i947 in xrange(_size943):
+            _key948 = iprot.readString()
+            _val949 = iprot.readString()
+            self.partitionSpecs[_key948] = _val949
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -22980,9 +22852,9 @@ class exchange_partitions_args:
     if self.partitionSpecs is not None:
       oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs))
-      for kiter959,viter960 in self.partitionSpecs.items():
-        oprot.writeString(kiter959)
-        oprot.writeString(viter960)
+      for kiter950,viter951 in self.partitionSpecs.items():
+        oprot.writeString(kiter950)
+        oprot.writeString(viter951)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -23065,11 +22937,11 @@ class exchange_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype964, _size961) = iprot.readListBegin()
-          for _i965 in xrange(_size961):
-            _elem966 = Partition()
-            _elem966.read(iprot)
-            self.success.append(_elem966)
+          (_etype955, _size952) = iprot.readListBegin()
+          for _i956 in xrange(_size952):
+            _elem957 = Partition()
+            _elem957.read(iprot)
+            self.success.append(_elem957)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23110,8 +22982,8 @@ class exchange_partitions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter967 in self.success:
-        iter967.write(oprot)
+      for iter958 in self.success:
+        iter958.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -23205,10 +23077,10 @@ class get_partition_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype971, _size968) = iprot.readListBegin()
-          for _i972 in xrange(_size968):
-            _elem973 = iprot.readString()
-            self.part_vals.append(_elem973)
+          (_etype962, _size959) = iprot.readListBegin()
+          for _i963 in xrange(_size959):
+            _elem964 = iprot.readString()
+            self.part_vals.append(_elem964)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23220,10 +23092,10 @@ class get_partition_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype977, _size974) = iprot.readListBegin()
-          for _i978 in xrange(_size974):
-            _elem979 = iprot.readString()
-            self.group_names.append(_elem979)
+          (_etype968, _size965) = iprot.readListBegin()
+          for _i969 in xrange(_size965):
+            _elem970 = iprot.readString()
+            self.group_names.append(_elem970)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23248,8 +23120,8 @@ class get_partition_with_auth_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter980 in self.part_vals:
-        oprot.writeString(iter980)
+      for iter971 in self.part_vals:
+        oprot.writeString(iter971)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.user_name is not None:
@@ -23259,8 +23131,8 @@ class get_partition_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter981 in self.group_names:
-        oprot.writeString(iter981)
+      for iter972 in self.group_names:
+        oprot.writeString(iter972)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -23689,11 +23561,11 @@ class get_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype985, _size982) = iprot.readListBegin()
-          for _i986 in xrange(_size982):
-            _elem987 = Partition()
-            _elem987.read(iprot)
-            self.success.append(_elem987)
+          (_etype976, _size973) = iprot.readListBegin()
+          for _i977 in xrange(_size973):
+            _elem978 = Partition()
+            _elem978.read(iprot)
+            self.success.append(_elem978)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23722,8 +23594,8 @@ class get_partitions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter988 in self.success:
-        iter988.write(oprot)
+      for iter979 in self.success:
+        iter979.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -23817,10 +23689,10 @@ class get_partitions_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype992, _size989) = iprot.readListBegin()
-          for _i993 in xrange(_size989):
-            _elem994 = iprot.readString()
-            self.group_names.append(_elem994)
+          (_etype983, _size980) = iprot.readListBegin()
+          for _i984 in xrange(_size980):
+            _elem985 = iprot.readString()
+            self.group_names.append(_elem985)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23853,8 +23725,8 @@ class get_partitions_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter995 in self.group_names:
-        oprot.writeString(iter995)
+      for iter986 in self.group_names:
+        oprot.writeString(iter986)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -23915,11 +23787,11 @@ class get_partitions_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype999, _size996) = iprot.readListBegin()
-          for _i1000 in xrange(_size996):
-            _elem1001 = Partition()
-            _elem1001.read(iprot)
-            self.success.append(_elem1001)
+          (_etype990, _size987) = iprot.readListBegin()
+          for _i991 in xrange(_size987):
+            _elem992 = Partition()
+            _elem992.read(iprot)
+            self.success.append(_elem992)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23948,8 +23820,8 @@ class get_partitions_with_auth_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1002 in self.success:
-        iter1002.write(oprot)
+      for iter993 in self.success:
+        iter993.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -24107,11 +23979,11 @@ class get_partitions_pspec_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1006, _size1003) = iprot.readListBegin()
-          for _i1007 in xrange(_size1003):
-            _elem1008 = PartitionSpec()
-            _elem1008.read(iprot)
-            self.success.append(_elem1008)
+          (_etype997, _size994) = iprot.readListBegin()
+          for _i998 in xrange(_size994):
+            _elem999 = PartitionSpec()
+            _elem999.read(iprot)
+            self.success.append(_elem999)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24140,8 +24012,8 @@ class get_partitions_pspec_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1009 in self.success:
-        iter1009.write(oprot)
+      for iter1000 in self.success:
+        iter1000.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -24299,10 +24171,10 @@ class get_partition_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1013, _size1010) = iprot.readListBegin()
-          for _i1014 in xrange(_size1010):
-            _elem1015 = iprot.readString()
-            self.success.append(_elem1015)
+          (_etype1004, _size1001) = iprot.readListBegin()
+          for _i1005 in xrange(_size1001):
+            _elem1006 = iprot.readString()
+            self.success.append(_elem1006)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24331,8 +24203,8 @@ class get_partition_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1016 in self.success:
-        oprot.writeString(iter1016)
+      for iter1007 in self.success:
+        oprot.writeString(iter1007)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -24572,10 +24444,10 @@ class get_partitions_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1020, _size1017) = iprot.readListBegin()
-          for _i1021 in xrange(_size1017):
-            _elem1022 = iprot.readString()
-            self.part_vals.append(_elem1022)
+          (_etype1011, _size1008) = iprot.readListBegin()
+          for _i1012 in xrange(_size1008):
+            _elem1013 = iprot.readString()
+            self.part_vals.append(_elem1013)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24605,8 +24477,8 @@ class get_partitions_ps_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1023 in self.part_vals:
-        oprot.writeString(iter1023)
+      for iter1014 in self.part_vals:
+        oprot.writeString(iter1014)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -24670,11 +24542,11 @@ class get_partitions_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1027, _size1024) = iprot.readListBegin()
-          for _i1028 in xrange(_size1024):
-            _elem1029 = Partition()
-            _elem1029.read(iprot)
-            self.success.append(_elem1029)
+          (_etype1018, _size1015) = iprot.readListBegin()
+          for _i1019 in xrange(_size1015):
+            _elem1020 = Partition()
+            _elem1020.read(iprot)
+            self.success.append(_elem1020)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24703,8 +24575,8 @@ class get_partitions_ps_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1030 in self.success:
-        iter1030.write(oprot)
+      for iter1021 in self.success:
+        iter1021.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -24791,10 +24663,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1034, _size1031) = iprot.readListBegin()
-          for _i1035 in xrange(_size1031):
-            _elem1036 = iprot.readString()
-            self.part_vals.append(_elem1036)
+          (_etype1025, _size1022) = iprot.readListBegin()
+          for _i1026 in xrange(_size1022):
+            _elem1027 = iprot.readString()
+            self.part_vals.append(_elem1027)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24811,10 +24683,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 6:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1040, _size1037) = iprot.readListBegin()
-          for _i1041 in xrange(_size1037):
-            _elem1042 = iprot.readString()
-            self.group_names.append(_elem1042)
+          (_etype1031, _size1028) = iprot.readListBegin()
+          for _i1032 in xrange(_size1028):
+            _elem1033 = iprot.readString()
+            self.group_names.append(_elem1033)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24839,8 +24711,8 @@ class get_partitions_ps_with_auth_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1043 in self.part_vals:
-        oprot.writeString(iter1043)
+      for iter1034 in self.part_vals:
+        oprot.writeString(iter1034)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -24854,8 +24726,8 @@ class get_partitions_ps_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 6)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter1044 in self.group_names:
-        oprot.writeString(iter1044)
+      for iter1035 in self.group_names:
+        oprot.writeString(iter1035)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -24917,11 +24789,11 @@ class get_partitions_ps_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1048, _size1045) = iprot.readListBegin()
-          for _i1049 in xrange(_size1045):
-            _elem1050 = Partition()
-            _elem1050.read(iprot)
-            self.success.append(_elem1050)
+          (_etype1039, _size1036) = iprot.readListBegin()
+          for _i1040 in xrange(_size1036):
+            _elem1041 = Partition()
+            _elem1041.read(iprot)
+            self.success.append(_elem1041)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24950,8 +24822,8 @@ class get_partitions_ps_with_auth_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1051 in self.success:
-        iter1051.write(oprot)
+      for iter1042 in self.success:
+        iter1042.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -25032,10 +24904,10 @@ class get_partition_names_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1055, _size1052) = iprot.readListBegin()
-          for _i1056 in xrange(_size1052):
-            _elem1057 = iprot.readString()
-            self.part_vals.append(_elem1057)
+          (_etype1046, _size1043) = iprot.readListBegin()
+          for _i1047 in xrange(_size1043):
+            _elem1048 = iprot.readString()
+            self.part_vals.append(_elem1048)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25065,8 +24937,8 @@ class get_partition_names_ps_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1058 in self.part_vals:
-        oprot.writeString(iter1058)
+      for iter1049 in self.part_vals:
+        oprot.writeString(iter1049)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -25130,10 +25002,10 @@ class get_partition_names_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1062, _size1059) = iprot.readListBegin()
-          for _i1063 in xrange(_size1059):
-            _elem1064 = iprot.readString()
-            self.success.append(_elem1064)
+          (_etype1053, _size1050) = iprot.readListBegin()
+          for _i1054 in xrange(_size1050):
+            _elem1055 = iprot.readString()
+            self.success.append(_elem1055)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25162,8 +25034,8 @@ class get_partition_names_ps_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1065 in self.success:
-        oprot.writeString(iter1065)
+      for iter1056 in self.success:
+        oprot.writeString(iter1056)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -25334,11 +25206,11 @@ class get_partitions_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1069, _size1066) = iprot.readListBegin()
-          for _i1070 in xrange(_size1066):
-            _elem1071 = Partition()
-            _elem1071.read(iprot)
-            self.success.append(_elem1071)
+          (_etype1060, _size1057) = iprot.readListBegin()
+          for _i1061 in xrange(_size1057):
+            _elem1062 = Partition()
+            _elem1062.read(iprot)
+            self.success.append(_elem1062)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25367,8 +25239,8 @@ class get_partitions_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1072 in self.success:
-        iter1072.write(oprot)
+      for iter1063 in self.success:
+        iter1063.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -25539,11 +25411,11 @@ class get_part_specs_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1076, _size1073) = iprot.readListBegin()
-          for _i1077 in xrange(_size1073):
-            _elem1078 = PartitionSpec()
-            _elem1078.read(iprot)
-            self.success.append(_elem1078)
+          (_etype1067, _size1064) = iprot.readListBegin()
+          for _i1068 in xrange(_size1064):
+            _elem1069 = PartitionSpec()
+            _elem1069.read(iprot)
+            self.success.append(_elem1069)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25572,8 +25444,8 @@ class get_part_specs_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1079 in self.success:
-        iter1079.write(oprot)
+      for iter1070 in self.success:
+        iter1070.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -25993,10 +25865,10 @@ class get_partitions_by_names_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.names = []
-          (_etype1083, _size1080) = iprot.readListBegin()
-          for _i1084 in xrange(_size1080):
-            _elem1085 = iprot.readString()
-            self.names.append(_elem1085)
+          (_etype1074, _size1071) = iprot.readListBegin()
+          for _i1075 in xrange(_size1071):
+            _elem1076 = iprot.readString()
+            self.names.append(_elem1076)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26021,8 +25893,8 @@ class get_partitions_by_names_args:
     if self.names is not None:
       oprot.writeFieldBegin('names', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.names))
-      for iter1086 in self.names:
-        oprot.writeString(iter1086)
+      for iter1077 in self.names:
+        oprot.writeString(iter1077)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -26081,11 +25953,11 @@ class get_partitions_by_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1090, _size1087) = iprot.readListBegin()
-          for _i1091 in xrange(_size1087):
-            _elem1092 = Partition()
-            _elem1092.read(iprot)
-            self.success.append(_elem1092)
+          (_etype1081, _size1078) = iprot.readListBegin()
+          for _i1082 in xrange(_size1078):
+            _elem1083 = Partition()
+            _elem1083.read(iprot)
+            self.success.append(_elem1083)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26114,8 +25986,8 @@ class get_partitions_by_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1093 in self.success:
-        iter1093.write(oprot)
+      for iter1084 in self.success:
+        iter1084.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26365,11 +26237,11 @@ class alter_partitions_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1097, _size1094) = iprot.readListBegin()
-          for _i1098 in xrange(_size1094):
-            _elem1099 = Partition()
-            _elem1099.read(iprot)
-            self.new_parts.append(_elem1099)
+          (_etype1088, _size1085) = iprot.readListBegin()
+          for _i1089 in xrange(_size1085):
+            _elem1090 = Partition()
+            _elem1090.read(iprot)
+            self.new_parts.append(_elem1090)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26394,8 +26266,8 @@ class alter_partitions_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter1100 in self.new_parts:
-        iter1100.write(oprot)
+      for iter1091 in self.new_parts:
+        iter1091.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -26548,11 +26420,11 @@ class alter_partitions_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1104, _size1101) = iprot.readListBegin()
-          for _i1105 in xrange(_size1101):
-            _elem1106 = Partition()
-            _elem1106.read(iprot)
-            self.new_parts.append(_elem1106)
+          (_etype1095, _size1092) = iprot.readListBegin()
+          for _i1096 in xrange(_size1092):
+            _elem1097 = Partition()
+            _elem1097.read(iprot)
+            self.new_parts.append(_elem1097)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26583,8 +26455,8 @@ class alter_partitions_with_environment_context_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter1107 in self.new_parts:
-        iter1107.write(oprot)
+      for iter1098 in self.new_parts:
+        iter1098.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -26928,10 +26800,10 @@ class rename_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1111, _size1108) = iprot.readListBegin()
-          for _i1112 in xrange(_size1108):
-            _elem1113 = iprot.readString()
-            self.part_vals.append(_elem1113)
+          (_etype1102, _size1099) = iprot.readListBegin()
+          for _i1103 in xrange(_size1099):
+            _elem1104 = iprot.readString()
+            self.part_vals.append(_elem1104)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26962,8 +26834,8 @@ class rename_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1114 in self.part_vals:
-        oprot.writeString(iter1114)
+      for iter1105 in self.part_vals:
+        oprot.writeString(iter1105)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.new_part is not None:
@@ -27105,10 +26977,10 @@ class partition_name_has_valid_characters_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1118, _size1115) = iprot.readListBegin()
-          for _i1119 in xrange(_size1115):
-            _elem1120 = iprot.readString()
-            self.part_vals.append(_elem1120)
+          (_etype1109, _size1106) = iprot.readListBegin()
+          for _i1110 in xrange(_size1106):
+            _elem1111 = iprot.readString()
+            self.part_vals.append(_elem1111)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27130,8 +27002,8 @@ class partition_name_has_valid_characters_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1121 in self.part_vals:
-        oprot.writeString(iter1121)
+      for iter1112 in self.part_vals:
+        oprot.writeString(iter1112)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.throw_exception is not None:
@@ -27489,10 +27361,10 @@ class partition_name_to_vals_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1125, _size1122) = iprot.readListBegin()
-          for _i1126 in xrange(_size1122):
-            _elem1127 = iprot.readString()
-            self.success.append(_elem1127)
+          (_etype1116, _size1113) = iprot.readListBegin()
+          for _i1117 in xrange(_size1113):
+            _elem1118 = iprot.readString()
+            self.success.append(_elem1118)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27515,8 +27387,8 @@ class partition_name_to_vals_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1128 in self.success:
-        oprot.writeString(iter1128)
+      for iter1119 in self.success:
+        oprot.writeString(iter1119)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27640,11 +27512,11 @@ class partition_name_to_spec_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype1130, _vtype1131, _size1129 ) = iprot.readMapBegin()
-          for _i1133 in xrange(_size1129):
-            _key1134 = iprot.readString()
-            _val1135 = iprot.readString()
-            self.success[_key1134] = _val1135
+          (_ktype1121, _vtype1122, _size1120 ) = iprot.readMapBegin()
+          for _i1124 in xrange(_size1120):
+            _key1125 = iprot.readString()
+            _val1126 = iprot.readString()
+            self.success[_key1125] = _val1126
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -27667,9 +27539,9 @@ class partition_name_to_spec_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.MAP, 0)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.success))
-      for kiter1136,viter1137 in self.success.items():
-        oprot.writeString(kiter1136)
-        oprot.writeString(viter1137)
+      for kiter1127,viter1128 in self.success.items():
+        oprot.writeString(kiter1127)
+        oprot.writeString(viter1128)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27745,11 +27617,11 @@ class markPartitionForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype1139, _vtype1140, _size1138 ) = iprot.readMapBegin()
-          for _i1142 in xrange(_size1138):
-            _key1143 = iprot.readString()
-            _val1144 = iprot.readString()
-            self.part_vals[_key1143] = _val1144
+          (_ktype1130, _vtype1131, _size1129 ) = iprot.readMapBegin()
+          for _i1133 in xrange(_size1129):
+            _key1134 = iprot.readString()
+            _val1135 = iprot.readString()
+            self.part_vals[_key1134] = _val1135
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -27779,9 +27651,9 @@ class markPartitionForEvent_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals))
-      for kiter1145,viter1146 in self.part_vals.items():
-        oprot.writeString(kiter1145)
-        oprot.writeString(viter1146)
+      for kiter1136,viter1137 in self.part_vals.items():
+        oprot.writeString(kiter1136)
+        oprot.writeString(viter1137)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -27995,11 +27867,11 @@ class isPartitionMarkedForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype1148, _vtype1149, _size1147 ) = iprot.readMapBegin()
-          for _i1151 in xrange(_size1147):
-            _key1152 = iprot.readString()
-            _val1153 = iprot.readString()
-            self.part_vals[_key1152] = _val1153
+          (_ktype1139, _vtype1140, _size1138 ) = iprot.readMapBegin()
+          for _i1142 in xrange(_size1138):
+            _key1143 = iprot.readString()
+            _val1144 = iprot.readString()
+            self.part_vals[_key1143] = _val1144
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -28029,9 +27901,9 @@ class isPartitionMarkedForEvent_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals))
-      for kiter1154,viter1155 in self.part_vals.items():
-        oprot.writeString(kiter1154)
-        oprot.writeString(viter1155)
+      for kiter1145,viter1146 in self.part_vals.items():
+        oprot.writeString(kiter1145)
+        oprot.writeString(viter1146)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -29086,11 +28958,11 @@ class get_indexes_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1159, _size1156) = iprot.readListBegin()
-          for _i1160 in xrange(_size1156):
-            _elem1161 = Index()
-            _elem1161.read(iprot)
-            self.success.append(_elem1161)
+          (_etype1150, _size1147) = iprot.readListBegin()
+          for _i1151 in xrange(_size1147):
+            _elem1152 = Index()
+            _elem1152.read(iprot)
+            self.success.append(_elem1152)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29119,8 +28991,8 @@ class get_indexes_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1162 in self.success:
-        iter1162.write(oprot)
+      for iter1153 in self.success:
+        iter1153.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -29275,10 +29147,10 @@ class get_index_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1166, _size1163) = iprot.readListBegin()
-          for _i1167 in xrange(_size1163):
-            _elem1168 = iprot.readString()
-            self.success.append(_elem1168)
+          (_etype1157, _size1154) = iprot.readListBegin()
+          for _i1158 in xrange(_size1154):
+            _elem1159 = iprot.readString()
+            self.success.append(_elem1159)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29301,8 +29173,8 @@ class get_index_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1169 in self.success:
-        oprot.writeString(iter1169)
+      for iter1160 in self.success:
+        oprot.writeString(iter1160)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -32486,10 +32358,10 @@ class get_functions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1173, _size1170) = iprot.readListBegin()
-          for _i1174 in xrange(_size1170):
-            _elem1175 = iprot.readString()
-            self.success.append(_elem1175)
+          (_etype1164, _size1161) = iprot.readListBegin()
+          for _i1165 in xrange(_size1161):
+            _elem1166 = iprot.readString()
+            self.success.append(_elem1166)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -32512,8 +32384,8 @@ class get_functions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1176 in self.success:
-        oprot.writeString(iter1176)
+      for iter1167 in self.success:
+        oprot.writeString(iter1167)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -33201,10 +33073,10 @@ class get_role_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1180, _size1177) = iprot.readListBegin()
-          for _i1181 in xrange(_size1177):
-            _elem1182 = iprot.readString()
-            self.success.append(_elem1182)
+          (_etype1171, _size1168) = iprot.readListBegin()
+          for _i1172 in xrange(_size1168):
+            _elem1173 = iprot.readString()
+            self.success.append(_elem1173)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -33227,8 +33099,8 @@ class get_role_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1183 in self.success:
-        oprot.writeString(iter1183)
+      for iter1174 in self.success:
+        oprot.writeString(iter1174)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -33742,11 +33614,11 @@ class list_roles_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1187, _size1184) = iprot.readListBegin()
-          for _i1188 in xrange(_size1184):
-            _elem1189 = Role()
-            _elem1189.read(iprot)
-            self.success.append(_elem1189)
+          (_etype1178, _size1175) = iprot.readListBegin()
+          for _i1179 in xrange(_size1175):
+            _elem1180 = Role()
+            _elem1180.read(iprot)
+            self.success.append(_elem1180)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -33769,8 +33641,8 @@ class list_roles_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1190 in self.success:
-        iter1190.write(oprot)
+      for iter1181 in self.success:
+        iter1181.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -34279,10 +34151,10 @@ class get_privilege_set_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1194, _size1191) = iprot.readListBegin()
-          for _i1195 in xrange(_size1191):
-            _elem1196 = iprot.readString()
-            self.group_names.append(_elem1196)
+          (_etype1185, _size1182) = iprot.readListBegin()
+          for _i1186 in xrange(_size1182):
+            _elem1187 = iprot.readString()
+            self.group_names.append(_elem1187)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -34307,8 +34179,8 @@ class get_privilege_set_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter1197 in self.group_names:
-        oprot.writeString(iter1197)
+      for iter1188 in self.group_names:
+        oprot.writeString(iter1188)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -34535,11 +34407,11 @@ class list_privileges_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1201, _size1198) = iprot.readListBegin()
-          for _i1202 in xrange(_size1198):
-            _elem1203 = HiveObjectPrivilege()
-            _elem1203.read(iprot)
-            self.success.append(_elem1203)
+          (_etype1192, _size1189) = iprot.readListBegin()
+          for _i1193 in xrange(_size1189):
+            _elem1194 = HiveObjectPrivilege()
+            _elem1194.read(iprot)
+            self.success.append(_elem1194)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -34562,8 +34434,8 @@ class list_privileges_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1204 in self.success:
-        iter1204.write(oprot)
+      for iter1195 in self.success:
+        iter1195.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -35061,10 +34933,10 @@ class set_ugi_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1208, _size1205) = iprot.readListBegin()
-          for _i1209 in xrange(_size1205):
-            _elem1210 = iprot.readString()
-            self.group_names.append(_elem1210)
+          (_etype1199, _size1196) = iprot.readListBegin()
+          for _i1200 in xrange(_size1196):
+            _elem1201 = iprot.readString()
+            self.group_names.append(_elem1201)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -35085,8 +34957,8 @@ class set_ugi_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter1211 in self.group_names:
-        oprot.writeString(iter1211)
+      for iter1202 in self.group_names:
+        oprot.writeString(iter1202)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -35141,10 +35013,10 @@ class set_ugi_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1215, _size1212) = iprot.readListBegin()
-          for _i1216 in xrange(_size1212):
-            _elem1217 = iprot.readString()
-            self.success.append(_elem1217)
+          (_etype1206, _size1203) = iprot.readListBegin()
+          for _i1207 in xrange(_size1203):
+            _elem1208 = iprot.readString()
+            self.success.append(_elem1208)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -35167,8 +35039,8 @@ class set_ugi_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1218 in self.success:
-        oprot.writeString(iter1218)
+      for iter1209 in self.success:
+        oprot.writeString(iter1209)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -36100,10 +35972,10 @@ class get_all_token_identifiers_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1222, _size1219) = iprot.readListBegin()
-          for _i1223 in xrange(_size1219):
-            _elem1224 = iprot.readString()
-            self.success.append(_elem1224)
+          (_etype1213, _size1210) = iprot.readListBegin()
+          for _i1214 in xrange(_size1210):
+            _elem1215 = iprot.readString()
+            self.success.append(_elem1215)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -36120,8 +35992,8 @@ class get_all_token_identifiers_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1225 in self.success:
-        oprot.writeString(iter1225)
+      for iter1216 in self.success:
+        oprot.writeString(iter1216)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -36648,10 +36520,10 @@ class get_master_keys_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1229, _size1226) = iprot.readListBegin()
-          for _i1230 in xrange(_size1226):
-            _elem1231 = iprot.readString()
-            self.success.append(_elem1231)
+          (_etype1220, _size1217) = iprot.readListBegin()
+          for _i1221 in xrange(_size1217):
+            _elem1222 = iprot.readString()
+            self.success.append(_elem1222)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -36668,8 +36540,8 @@ class get_master_keys_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1232 in self.success:
-        oprot.writeString(iter1232)
+      for iter1223 in self.success:
+        oprot.writeString(iter1223)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -38878,344 +38750,6 @@ class add_dynamic_partitions_result:
   def __ne__(self, other):
     return not (self == other)
 
-class get_last_completed_transaction_for_tables_args:
-  """
-  Attributes:
-   - db_names
-   - table_names
-   - txns_snapshot
-  """
-
-  thrift_spec = (
-    None, # 0
-    (1, TType.LIST, 'db_names', (TType.STRING,None), None, ), # 1
-    (2, TType.LIST, 'table_names', (TType.STRING,None), None, ), # 2
-    (3, TType.STRUCT, 'txns_snapshot', (TxnsSnapshot, TxnsSnapshot.thrift_spec), None, ), # 3
-  )
-
-  def __init__(self, db_names=None, table_names=None, txns_snapshot=None,):
-    self.db_names = db_names
-    self.table_names = table_names
-    self.txns_snapshot = txns_snapshot
-
-  def read(self, iprot):
-    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
-      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
-      return
-    iprot.readStructBegin()
-    while True:
-      (fname, ftype, fid) = iprot.readFieldBegin()
-      if ftype == TType.STOP:
-        break
-      if fid == 1:
-        if ftype == TType.LIST:
-          self.db_names = []
-          (_etype1236, _size1233) = iprot.readListBegin()
-          for _i1237 in xrange(_size1233):
-            _elem1238 = iprot.readString()
-            self.db_names.append(_elem1238)
-          iprot.readListEnd()
-        else:
-          iprot.skip(ftype)
-      elif fid == 2:
-        if ftype == TType.LIST:
-          self.table_names = []
-          (_etype1242, _size1239) = iprot.readListBegin()
-          for _i1243 in xrange(_size1239):
-            _elem1244 = iprot.readString()
-            self.table_names.append(_elem1244)
-          iprot.readListEnd()
-        else:
-          iprot.skip(ftype)
-      elif fid == 3:
-        if ftype == TType.STRUCT:
-          self.txns_snapshot = TxnsSnapshot()
-          self.txns_snapshot.read(iprot)
-        else:
-          iprot.skip(ftype)
-      else:
-        iprot.skip(ftype)
-      iprot.readFieldEnd()
-    iprot.readStructEnd()
-
-  def write(self, oprot):
-    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
-      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
-      return
-    oprot.writeStructBegin('get_last_completed_transaction_for_tables_args')
-    if self.db_names is not None:
-      oprot.writeFieldBegin('db_names', TType.LIST, 1)
-      oprot.writeListBegin(TType.STRING, len(self.db_names))
-      for iter1245 in self.db_names:
-        oprot.writeString(iter1245)
-      oprot.writeListEnd()
-      oprot.writeFieldEnd()
-    if self.table_names is not None:
-      oprot.writeFieldBegin('table_names', TType.LIST, 2)
-      oprot.writeListBegin(TType.STRING, len(self.table_names))
-      for iter1246 in self.table_names:
-        oprot.writeString(iter1246)
-      oprot.writeListEnd()
-      oprot.writeFieldEnd()
-    if self.txns_snapshot is not None:
-      oprot.writeFieldBegin('txns_snapshot', TType.STRUCT, 3)
-      self.txns_snapshot.write(oprot)
-      oprot.writeFieldEnd()
-    oprot.writeFieldStop()
-    oprot.writeStructEnd()
-
-  def validate(self):
-    return
-
-
-  def __hash__(self):
-    value = 17
-    value = (value * 31) ^ hash(self.db_names)
-    value = (value * 31) ^ hash(self.table_names)
-    value = (value * 31) ^ hash(self.txns_snapshot)
-    return value
-
-  def __repr__(self):
-    L = ['%s=%r' % (key, value)
-      for key, value in self.__dict__.iteritems()]
-    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
-
-  def __eq__(self, other):
-    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
-
-  def __ne__(self, other):
-    return not (self == other)
-
-class get_last_completed_transaction_for_tables_result:
-  """
-  Attributes:
-   - success
-  """
-
-  thrift_spec = (
-    (0, TType.LIST, 'success', (TType.STRUCT,(BasicTxnInfo, BasicTxnInfo.thrift_spec)), None, ), # 0
-  )
-
-  def __init__(self, success=None,):
-    self.success = success
-
-  def read(self, iprot):
-    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
-      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
-      return
-    iprot.readStructBegin()
-    while True:
-      (fname, ftype, fid) = iprot.readFieldBegin()
-      if ftype == TType.STOP:
-        break
-      if fid == 0:
-        if ftype == TType.LIST:
-          self.success = []
-          (_etype1250, _size1247) = iprot.readListBegin()
-          for _i1251 in xrange(_size1247):
-            _elem1252 = BasicTxnInfo()
-            _elem1252.read(iprot)
-            self.success.append(_elem1252)
-          iprot.readListEnd()
-        else:
-          iprot.skip(ftype)
-      else:
-        iprot.skip(ftype)
-      iprot.readFieldEnd()
-    iprot.readStructEnd()
-
-  def write(self, oprot):
-    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
-      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
-      return
-    oprot.writeStructBegin('get_last_completed_transaction_for_tables_result')
-    if self.success is not None:
-      oprot.writeFieldBegin('success', TType.LIST, 0)
-      oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1253 in self.success:
-        iter1253.write(oprot)
-      oprot.writeListEnd()
-      oprot.writeFieldEnd()
-    oprot.writeFieldStop()
-    oprot.writeStructEnd()
-
-  def validate(self):
-    return
-
-
-  def __hash__(self):
-    value = 17
-    value = (value * 31) ^ hash(self.success)
-    return value
-
-  def __repr__(self):
-    L = ['%s=%r' % (key, value)
-      for key, value in self.__dict__.iteritems()]
-    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
-
-  def __eq__(self, other):
-    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
-
-  def __ne__(self, other):
-    return not (self == other)
-
-class get_last_completed_transaction_for_table_args:
-  """
-  Attributes:
-   - db_name
-   - table_name
-   - txns_snapshot
-  """
-
-  thrift_spec = (
-    None, # 0
-    (1, TType.STRING, 'db_name', None, N

<TRUNCATED>

[06/16] hive git commit: HIVE-18546: Remove unnecessary code introduced in HIVE-14498 (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php b/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
index bf7d466..9382c60 100644
--- a/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -1205,20 +1205,6 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    */
   public function add_dynamic_partitions(\metastore\AddDynamicPartitions $rqst);
   /**
-   * @param string[] $db_names
-   * @param string[] $table_names
-   * @param \metastore\TxnsSnapshot $txns_snapshot
-   * @return \metastore\BasicTxnInfo[]
-   */
-  public function get_last_completed_transaction_for_tables(array $db_names, array $table_names, \metastore\TxnsSnapshot $txns_snapshot);
-  /**
-   * @param string $db_name
-   * @param string $table_name
-   * @param \metastore\TxnsSnapshot $txns_snapshot
-   * @return \metastore\BasicTxnInfo
-   */
-  public function get_last_completed_transaction_for_table($db_name, $table_name, \metastore\TxnsSnapshot $txns_snapshot);
-  /**
    * @param \metastore\NotificationEventRequest $rqst
    * @return \metastore\NotificationEventResponse
    */
@@ -10101,112 +10087,6 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     return;
   }
 
-  public function get_last_completed_transaction_for_tables(array $db_names, array $table_names, \metastore\TxnsSnapshot $txns_snapshot)
-  {
-    $this->send_get_last_completed_transaction_for_tables($db_names, $table_names, $txns_snapshot);
-    return $this->recv_get_last_completed_transaction_for_tables();
-  }
-
-  public function send_get_last_completed_transaction_for_tables(array $db_names, array $table_names, \metastore\TxnsSnapshot $txns_snapshot)
-  {
-    $args = new \metastore\ThriftHiveMetastore_get_last_completed_transaction_for_tables_args();
-    $args->db_names = $db_names;
-    $args->table_names = $table_names;
-    $args->txns_snapshot = $txns_snapshot;
-    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
-    if ($bin_accel)
-    {
-      thrift_protocol_write_binary($this->output_, 'get_last_completed_transaction_for_tables', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
-    }
-    else
-    {
-      $this->output_->writeMessageBegin('get_last_completed_transaction_for_tables', TMessageType::CALL, $this->seqid_);
-      $args->write($this->output_);
-      $this->output_->writeMessageEnd();
-      $this->output_->getTransport()->flush();
-    }
-  }
-
-  public function recv_get_last_completed_transaction_for_tables()
-  {
-    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
-    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_get_last_completed_transaction_for_tables_result', $this->input_->isStrictRead());
-    else
-    {
-      $rseqid = 0;
-      $fname = null;
-      $mtype = 0;
-
-      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
-      if ($mtype == TMessageType::EXCEPTION) {
-        $x = new TApplicationException();
-        $x->read($this->input_);
-        $this->input_->readMessageEnd();
-        throw $x;
-      }
-      $result = new \metastore\ThriftHiveMetastore_get_last_completed_transaction_for_tables_result();
-      $result->read($this->input_);
-      $this->input_->readMessageEnd();
-    }
-    if ($result->success !== null) {
-      return $result->success;
-    }
-    throw new \Exception("get_last_completed_transaction_for_tables failed: unknown result");
-  }
-
-  public function get_last_completed_transaction_for_table($db_name, $table_name, \metastore\TxnsSnapshot $txns_snapshot)
-  {
-    $this->send_get_last_completed_transaction_for_table($db_name, $table_name, $txns_snapshot);
-    return $this->recv_get_last_completed_transaction_for_table();
-  }
-
-  public function send_get_last_completed_transaction_for_table($db_name, $table_name, \metastore\TxnsSnapshot $txns_snapshot)
-  {
-    $args = new \metastore\ThriftHiveMetastore_get_last_completed_transaction_for_table_args();
-    $args->db_name = $db_name;
-    $args->table_name = $table_name;
-    $args->txns_snapshot = $txns_snapshot;
-    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
-    if ($bin_accel)
-    {
-      thrift_protocol_write_binary($this->output_, 'get_last_completed_transaction_for_table', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
-    }
-    else
-    {
-      $this->output_->writeMessageBegin('get_last_completed_transaction_for_table', TMessageType::CALL, $this->seqid_);
-      $args->write($this->output_);
-      $this->output_->writeMessageEnd();
-      $this->output_->getTransport()->flush();
-    }
-  }
-
-  public function recv_get_last_completed_transaction_for_table()
-  {
-    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
-    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_get_last_completed_transaction_for_table_result', $this->input_->isStrictRead());
-    else
-    {
-      $rseqid = 0;
-      $fname = null;
-      $mtype = 0;
-
-      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
-      if ($mtype == TMessageType::EXCEPTION) {
-        $x = new TApplicationException();
-        $x->read($this->input_);
-        $this->input_->readMessageEnd();
-        throw $x;
-      }
-      $result = new \metastore\ThriftHiveMetastore_get_last_completed_transaction_for_table_result();
-      $result->read($this->input_);
-      $this->input_->readMessageEnd();
-    }
-    if ($result->success !== null) {
-      return $result->success;
-    }
-    throw new \Exception("get_last_completed_transaction_for_table failed: unknown result");
-  }
-
   public function get_next_notification(\metastore\NotificationEventRequest $rqst)
   {
     $this->send_get_next_notification($rqst);
@@ -12984,14 +12864,14 @@ class ThriftHiveMetastore_get_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size732 = 0;
-            $_etype735 = 0;
-            $xfer += $input->readListBegin($_etype735, $_size732);
-            for ($_i736 = 0; $_i736 < $_size732; ++$_i736)
+            $_size724 = 0;
+            $_etype727 = 0;
+            $xfer += $input->readListBegin($_etype727, $_size724);
+            for ($_i728 = 0; $_i728 < $_size724; ++$_i728)
             {
-              $elem737 = null;
-              $xfer += $input->readString($elem737);
-              $this->success []= $elem737;
+              $elem729 = null;
+              $xfer += $input->readString($elem729);
+              $this->success []= $elem729;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13027,9 +12907,9 @@ class ThriftHiveMetastore_get_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter738)
+          foreach ($this->success as $iter730)
           {
-            $xfer += $output->writeString($iter738);
+            $xfer += $output->writeString($iter730);
           }
         }
         $output->writeListEnd();
@@ -13160,14 +13040,14 @@ class ThriftHiveMetastore_get_all_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size739 = 0;
-            $_etype742 = 0;
-            $xfer += $input->readListBegin($_etype742, $_size739);
-            for ($_i743 = 0; $_i743 < $_size739; ++$_i743)
+            $_size731 = 0;
+            $_etype734 = 0;
+            $xfer += $input->readListBegin($_etype734, $_size731);
+            for ($_i735 = 0; $_i735 < $_size731; ++$_i735)
             {
-              $elem744 = null;
-              $xfer += $input->readString($elem744);
-              $this->success []= $elem744;
+              $elem736 = null;
+              $xfer += $input->readString($elem736);
+              $this->success []= $elem736;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13203,9 +13083,9 @@ class ThriftHiveMetastore_get_all_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter745)
+          foreach ($this->success as $iter737)
           {
-            $xfer += $output->writeString($iter745);
+            $xfer += $output->writeString($iter737);
           }
         }
         $output->writeListEnd();
@@ -14206,18 +14086,18 @@ class ThriftHiveMetastore_get_type_all_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size746 = 0;
-            $_ktype747 = 0;
-            $_vtype748 = 0;
-            $xfer += $input->readMapBegin($_ktype747, $_vtype748, $_size746);
-            for ($_i750 = 0; $_i750 < $_size746; ++$_i750)
+            $_size738 = 0;
+            $_ktype739 = 0;
+            $_vtype740 = 0;
+            $xfer += $input->readMapBegin($_ktype739, $_vtype740, $_size738);
+            for ($_i742 = 0; $_i742 < $_size738; ++$_i742)
             {
-              $key751 = '';
-              $val752 = new \metastore\Type();
-              $xfer += $input->readString($key751);
-              $val752 = new \metastore\Type();
-              $xfer += $val752->read($input);
-              $this->success[$key751] = $val752;
+              $key743 = '';
+              $val744 = new \metastore\Type();
+              $xfer += $input->readString($key743);
+              $val744 = new \metastore\Type();
+              $xfer += $val744->read($input);
+              $this->success[$key743] = $val744;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -14253,10 +14133,10 @@ class ThriftHiveMetastore_get_type_all_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $kiter753 => $viter754)
+          foreach ($this->success as $kiter745 => $viter746)
           {
-            $xfer += $output->writeString($kiter753);
-            $xfer += $viter754->write($output);
+            $xfer += $output->writeString($kiter745);
+            $xfer += $viter746->write($output);
           }
         }
         $output->writeMapEnd();
@@ -14460,15 +14340,15 @@ class ThriftHiveMetastore_get_fields_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size755 = 0;
-            $_etype758 = 0;
-            $xfer += $input->readListBegin($_etype758, $_size755);
-            for ($_i759 = 0; $_i759 < $_size755; ++$_i759)
+            $_size747 = 0;
+            $_etype750 = 0;
+            $xfer += $input->readListBegin($_etype750, $_size747);
+            for ($_i751 = 0; $_i751 < $_size747; ++$_i751)
             {
-              $elem760 = null;
-              $elem760 = new \metastore\FieldSchema();
-              $xfer += $elem760->read($input);
-              $this->success []= $elem760;
+              $elem752 = null;
+              $elem752 = new \metastore\FieldSchema();
+              $xfer += $elem752->read($input);
+              $this->success []= $elem752;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14520,9 +14400,9 @@ class ThriftHiveMetastore_get_fields_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter761)
+          foreach ($this->success as $iter753)
           {
-            $xfer += $iter761->write($output);
+            $xfer += $iter753->write($output);
           }
         }
         $output->writeListEnd();
@@ -14764,15 +14644,15 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size762 = 0;
-            $_etype765 = 0;
-            $xfer += $input->readListBegin($_etype765, $_size762);
-            for ($_i766 = 0; $_i766 < $_size762; ++$_i766)
+            $_size754 = 0;
+            $_etype757 = 0;
+            $xfer += $input->readListBegin($_etype757, $_size754);
+            for ($_i758 = 0; $_i758 < $_size754; ++$_i758)
             {
-              $elem767 = null;
-              $elem767 = new \metastore\FieldSchema();
-              $xfer += $elem767->read($input);
-              $this->success []= $elem767;
+              $elem759 = null;
+              $elem759 = new \metastore\FieldSchema();
+              $xfer += $elem759->read($input);
+              $this->success []= $elem759;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14824,9 +14704,9 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter768)
+          foreach ($this->success as $iter760)
           {
-            $xfer += $iter768->write($output);
+            $xfer += $iter760->write($output);
           }
         }
         $output->writeListEnd();
@@ -15040,15 +14920,15 @@ class ThriftHiveMetastore_get_schema_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size769 = 0;
-            $_etype772 = 0;
-            $xfer += $input->readListBegin($_etype772, $_size769);
-            for ($_i773 = 0; $_i773 < $_size769; ++$_i773)
+            $_size761 = 0;
+            $_etype764 = 0;
+            $xfer += $input->readListBegin($_etype764, $_size761);
+            for ($_i765 = 0; $_i765 < $_size761; ++$_i765)
             {
-              $elem774 = null;
-              $elem774 = new \metastore\FieldSchema();
-              $xfer += $elem774->read($input);
-              $this->success []= $elem774;
+              $elem766 = null;
+              $elem766 = new \metastore\FieldSchema();
+              $xfer += $elem766->read($input);
+              $this->success []= $elem766;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15100,9 +14980,9 @@ class ThriftHiveMetastore_get_schema_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter775)
+          foreach ($this->success as $iter767)
           {
-            $xfer += $iter775->write($output);
+            $xfer += $iter767->write($output);
           }
         }
         $output->writeListEnd();
@@ -15344,15 +15224,15 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size776 = 0;
-            $_etype779 = 0;
-            $xfer += $input->readListBegin($_etype779, $_size776);
-            for ($_i780 = 0; $_i780 < $_size776; ++$_i780)
+            $_size768 = 0;
+            $_etype771 = 0;
+            $xfer += $input->readListBegin($_etype771, $_size768);
+            for ($_i772 = 0; $_i772 < $_size768; ++$_i772)
             {
-              $elem781 = null;
-              $elem781 = new \metastore\FieldSchema();
-              $xfer += $elem781->read($input);
-              $this->success []= $elem781;
+              $elem773 = null;
+              $elem773 = new \metastore\FieldSchema();
+              $xfer += $elem773->read($input);
+              $this->success []= $elem773;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15404,9 +15284,9 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter782)
+          foreach ($this->success as $iter774)
           {
-            $xfer += $iter782->write($output);
+            $xfer += $iter774->write($output);
           }
         }
         $output->writeListEnd();
@@ -16046,15 +15926,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->primaryKeys = array();
-            $_size783 = 0;
-            $_etype786 = 0;
-            $xfer += $input->readListBegin($_etype786, $_size783);
-            for ($_i787 = 0; $_i787 < $_size783; ++$_i787)
+            $_size775 = 0;
+            $_etype778 = 0;
+            $xfer += $input->readListBegin($_etype778, $_size775);
+            for ($_i779 = 0; $_i779 < $_size775; ++$_i779)
             {
-              $elem788 = null;
-              $elem788 = new \metastore\SQLPrimaryKey();
-              $xfer += $elem788->read($input);
-              $this->primaryKeys []= $elem788;
+              $elem780 = null;
+              $elem780 = new \metastore\SQLPrimaryKey();
+              $xfer += $elem780->read($input);
+              $this->primaryKeys []= $elem780;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16064,15 +15944,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->foreignKeys = array();
-            $_size789 = 0;
-            $_etype792 = 0;
-            $xfer += $input->readListBegin($_etype792, $_size789);
-            for ($_i793 = 0; $_i793 < $_size789; ++$_i793)
+            $_size781 = 0;
+            $_etype784 = 0;
+            $xfer += $input->readListBegin($_etype784, $_size781);
+            for ($_i785 = 0; $_i785 < $_size781; ++$_i785)
             {
-              $elem794 = null;
-              $elem794 = new \metastore\SQLForeignKey();
-              $xfer += $elem794->read($input);
-              $this->foreignKeys []= $elem794;
+              $elem786 = null;
+              $elem786 = new \metastore\SQLForeignKey();
+              $xfer += $elem786->read($input);
+              $this->foreignKeys []= $elem786;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16082,15 +15962,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 4:
           if ($ftype == TType::LST) {
             $this->uniqueConstraints = array();
-            $_size795 = 0;
-            $_etype798 = 0;
-            $xfer += $input->readListBegin($_etype798, $_size795);
-            for ($_i799 = 0; $_i799 < $_size795; ++$_i799)
+            $_size787 = 0;
+            $_etype790 = 0;
+            $xfer += $input->readListBegin($_etype790, $_size787);
+            for ($_i791 = 0; $_i791 < $_size787; ++$_i791)
             {
-              $elem800 = null;
-              $elem800 = new \metastore\SQLUniqueConstraint();
-              $xfer += $elem800->read($input);
-              $this->uniqueConstraints []= $elem800;
+              $elem792 = null;
+              $elem792 = new \metastore\SQLUniqueConstraint();
+              $xfer += $elem792->read($input);
+              $this->uniqueConstraints []= $elem792;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16100,15 +15980,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->notNullConstraints = array();
-            $_size801 = 0;
-            $_etype804 = 0;
-            $xfer += $input->readListBegin($_etype804, $_size801);
-            for ($_i805 = 0; $_i805 < $_size801; ++$_i805)
+            $_size793 = 0;
+            $_etype796 = 0;
+            $xfer += $input->readListBegin($_etype796, $_size793);
+            for ($_i797 = 0; $_i797 < $_size793; ++$_i797)
             {
-              $elem806 = null;
-              $elem806 = new \metastore\SQLNotNullConstraint();
-              $xfer += $elem806->read($input);
-              $this->notNullConstraints []= $elem806;
+              $elem798 = null;
+              $elem798 = new \metastore\SQLNotNullConstraint();
+              $xfer += $elem798->read($input);
+              $this->notNullConstraints []= $elem798;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16144,9 +16024,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->primaryKeys));
         {
-          foreach ($this->primaryKeys as $iter807)
+          foreach ($this->primaryKeys as $iter799)
           {
-            $xfer += $iter807->write($output);
+            $xfer += $iter799->write($output);
           }
         }
         $output->writeListEnd();
@@ -16161,9 +16041,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->foreignKeys));
         {
-          foreach ($this->foreignKeys as $iter808)
+          foreach ($this->foreignKeys as $iter800)
           {
-            $xfer += $iter808->write($output);
+            $xfer += $iter800->write($output);
           }
         }
         $output->writeListEnd();
@@ -16178,9 +16058,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->uniqueConstraints));
         {
-          foreach ($this->uniqueConstraints as $iter809)
+          foreach ($this->uniqueConstraints as $iter801)
           {
-            $xfer += $iter809->write($output);
+            $xfer += $iter801->write($output);
           }
         }
         $output->writeListEnd();
@@ -16195,9 +16075,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->notNullConstraints));
         {
-          foreach ($this->notNullConstraints as $iter810)
+          foreach ($this->notNullConstraints as $iter802)
           {
-            $xfer += $iter810->write($output);
+            $xfer += $iter802->write($output);
           }
         }
         $output->writeListEnd();
@@ -17833,14 +17713,14 @@ class ThriftHiveMetastore_truncate_table_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->partNames = array();
-            $_size811 = 0;
-            $_etype814 = 0;
-            $xfer += $input->readListBegin($_etype814, $_size811);
-            for ($_i815 = 0; $_i815 < $_size811; ++$_i815)
+            $_size803 = 0;
+            $_etype806 = 0;
+            $xfer += $input->readListBegin($_etype806, $_size803);
+            for ($_i807 = 0; $_i807 < $_size803; ++$_i807)
             {
-              $elem816 = null;
-              $xfer += $input->readString($elem816);
-              $this->partNames []= $elem816;
+              $elem808 = null;
+              $xfer += $input->readString($elem808);
+              $this->partNames []= $elem808;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17878,9 +17758,9 @@ class ThriftHiveMetastore_truncate_table_args {
       {
         $output->writeListBegin(TType::STRING, count($this->partNames));
         {
-          foreach ($this->partNames as $iter817)
+          foreach ($this->partNames as $iter809)
           {
-            $xfer += $output->writeString($iter817);
+            $xfer += $output->writeString($iter809);
           }
         }
         $output->writeListEnd();
@@ -18131,14 +18011,14 @@ class ThriftHiveMetastore_get_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size818 = 0;
-            $_etype821 = 0;
-            $xfer += $input->readListBegin($_etype821, $_size818);
-            for ($_i822 = 0; $_i822 < $_size818; ++$_i822)
+            $_size810 = 0;
+            $_etype813 = 0;
+            $xfer += $input->readListBegin($_etype813, $_size810);
+            for ($_i814 = 0; $_i814 < $_size810; ++$_i814)
             {
-              $elem823 = null;
-              $xfer += $input->readString($elem823);
-              $this->success []= $elem823;
+              $elem815 = null;
+              $xfer += $input->readString($elem815);
+              $this->success []= $elem815;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18174,9 +18054,9 @@ class ThriftHiveMetastore_get_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter824)
+          foreach ($this->success as $iter816)
           {
-            $xfer += $output->writeString($iter824);
+            $xfer += $output->writeString($iter816);
           }
         }
         $output->writeListEnd();
@@ -18378,14 +18258,14 @@ class ThriftHiveMetastore_get_tables_by_type_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size825 = 0;
-            $_etype828 = 0;
-            $xfer += $input->readListBegin($_etype828, $_size825);
-            for ($_i829 = 0; $_i829 < $_size825; ++$_i829)
+            $_size817 = 0;
+            $_etype820 = 0;
+            $xfer += $input->readListBegin($_etype820, $_size817);
+            for ($_i821 = 0; $_i821 < $_size817; ++$_i821)
             {
-              $elem830 = null;
-              $xfer += $input->readString($elem830);
-              $this->success []= $elem830;
+              $elem822 = null;
+              $xfer += $input->readString($elem822);
+              $this->success []= $elem822;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18421,9 +18301,9 @@ class ThriftHiveMetastore_get_tables_by_type_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter831)
+          foreach ($this->success as $iter823)
           {
-            $xfer += $output->writeString($iter831);
+            $xfer += $output->writeString($iter823);
           }
         }
         $output->writeListEnd();
@@ -18579,14 +18459,14 @@ class ThriftHiveMetastore_get_materialized_views_for_rewriting_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size832 = 0;
-            $_etype835 = 0;
-            $xfer += $input->readListBegin($_etype835, $_size832);
-            for ($_i836 = 0; $_i836 < $_size832; ++$_i836)
+            $_size824 = 0;
+            $_etype827 = 0;
+            $xfer += $input->readListBegin($_etype827, $_size824);
+            for ($_i828 = 0; $_i828 < $_size824; ++$_i828)
             {
-              $elem837 = null;
-              $xfer += $input->readString($elem837);
-              $this->success []= $elem837;
+              $elem829 = null;
+              $xfer += $input->readString($elem829);
+              $this->success []= $elem829;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18622,9 +18502,9 @@ class ThriftHiveMetastore_get_materialized_views_for_rewriting_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter838)
+          foreach ($this->success as $iter830)
           {
-            $xfer += $output->writeString($iter838);
+            $xfer += $output->writeString($iter830);
           }
         }
         $output->writeListEnd();
@@ -18729,14 +18609,14 @@ class ThriftHiveMetastore_get_table_meta_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->tbl_types = array();
-            $_size839 = 0;
-            $_etype842 = 0;
-            $xfer += $input->readListBegin($_etype842, $_size839);
-            for ($_i843 = 0; $_i843 < $_size839; ++$_i843)
+            $_size831 = 0;
+            $_etype834 = 0;
+            $xfer += $input->readListBegin($_etype834, $_size831);
+            for ($_i835 = 0; $_i835 < $_size831; ++$_i835)
             {
-              $elem844 = null;
-              $xfer += $input->readString($elem844);
-              $this->tbl_types []= $elem844;
+              $elem836 = null;
+              $xfer += $input->readString($elem836);
+              $this->tbl_types []= $elem836;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18774,9 +18654,9 @@ class ThriftHiveMetastore_get_table_meta_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_types));
         {
-          foreach ($this->tbl_types as $iter845)
+          foreach ($this->tbl_types as $iter837)
           {
-            $xfer += $output->writeString($iter845);
+            $xfer += $output->writeString($iter837);
           }
         }
         $output->writeListEnd();
@@ -18853,15 +18733,15 @@ class ThriftHiveMetastore_get_table_meta_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size846 = 0;
-            $_etype849 = 0;
-            $xfer += $input->readListBegin($_etype849, $_size846);
-            for ($_i850 = 0; $_i850 < $_size846; ++$_i850)
+            $_size838 = 0;
+            $_etype841 = 0;
+            $xfer += $input->readListBegin($_etype841, $_size838);
+            for ($_i842 = 0; $_i842 < $_size838; ++$_i842)
             {
-              $elem851 = null;
-              $elem851 = new \metastore\TableMeta();
-              $xfer += $elem851->read($input);
-              $this->success []= $elem851;
+              $elem843 = null;
+              $elem843 = new \metastore\TableMeta();
+              $xfer += $elem843->read($input);
+              $this->success []= $elem843;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18897,9 +18777,9 @@ class ThriftHiveMetastore_get_table_meta_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter852)
+          foreach ($this->success as $iter844)
           {
-            $xfer += $iter852->write($output);
+            $xfer += $iter844->write($output);
           }
         }
         $output->writeListEnd();
@@ -19055,14 +18935,14 @@ class ThriftHiveMetastore_get_all_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size853 = 0;
-            $_etype856 = 0;
-            $xfer += $input->readListBegin($_etype856, $_size853);
-            for ($_i857 = 0; $_i857 < $_size853; ++$_i857)
+            $_size845 = 0;
+            $_etype848 = 0;
+            $xfer += $input->readListBegin($_etype848, $_size845);
+            for ($_i849 = 0; $_i849 < $_size845; ++$_i849)
             {
-              $elem858 = null;
-              $xfer += $input->readString($elem858);
-              $this->success []= $elem858;
+              $elem850 = null;
+              $xfer += $input->readString($elem850);
+              $this->success []= $elem850;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19098,9 +18978,9 @@ class ThriftHiveMetastore_get_all_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter859)
+          foreach ($this->success as $iter851)
           {
-            $xfer += $output->writeString($iter859);
+            $xfer += $output->writeString($iter851);
           }
         }
         $output->writeListEnd();
@@ -19415,14 +19295,14 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size860 = 0;
-            $_etype863 = 0;
-            $xfer += $input->readListBegin($_etype863, $_size860);
-            for ($_i864 = 0; $_i864 < $_size860; ++$_i864)
+            $_size852 = 0;
+            $_etype855 = 0;
+            $xfer += $input->readListBegin($_etype855, $_size852);
+            for ($_i856 = 0; $_i856 < $_size852; ++$_i856)
             {
-              $elem865 = null;
-              $xfer += $input->readString($elem865);
-              $this->tbl_names []= $elem865;
+              $elem857 = null;
+              $xfer += $input->readString($elem857);
+              $this->tbl_names []= $elem857;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19455,9 +19335,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter866)
+          foreach ($this->tbl_names as $iter858)
           {
-            $xfer += $output->writeString($iter866);
+            $xfer += $output->writeString($iter858);
           }
         }
         $output->writeListEnd();
@@ -19522,15 +19402,15 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size867 = 0;
-            $_etype870 = 0;
-            $xfer += $input->readListBegin($_etype870, $_size867);
-            for ($_i871 = 0; $_i871 < $_size867; ++$_i871)
+            $_size859 = 0;
+            $_etype862 = 0;
+            $xfer += $input->readListBegin($_etype862, $_size859);
+            for ($_i863 = 0; $_i863 < $_size859; ++$_i863)
             {
-              $elem872 = null;
-              $elem872 = new \metastore\Table();
-              $xfer += $elem872->read($input);
-              $this->success []= $elem872;
+              $elem864 = null;
+              $elem864 = new \metastore\Table();
+              $xfer += $elem864->read($input);
+              $this->success []= $elem864;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19558,9 +19438,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter873)
+          foreach ($this->success as $iter865)
           {
-            $xfer += $iter873->write($output);
+            $xfer += $iter865->write($output);
           }
         }
         $output->writeListEnd();
@@ -20087,14 +19967,14 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size874 = 0;
-            $_etype877 = 0;
-            $xfer += $input->readListBegin($_etype877, $_size874);
-            for ($_i878 = 0; $_i878 < $_size874; ++$_i878)
+            $_size866 = 0;
+            $_etype869 = 0;
+            $xfer += $input->readListBegin($_etype869, $_size866);
+            for ($_i870 = 0; $_i870 < $_size866; ++$_i870)
             {
-              $elem879 = null;
-              $xfer += $input->readString($elem879);
-              $this->tbl_names []= $elem879;
+              $elem871 = null;
+              $xfer += $input->readString($elem871);
+              $this->tbl_names []= $elem871;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20127,9 +20007,9 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter880)
+          foreach ($this->tbl_names as $iter872)
           {
-            $xfer += $output->writeString($iter880);
+            $xfer += $output->writeString($iter872);
           }
         }
         $output->writeListEnd();
@@ -20234,18 +20114,18 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size881 = 0;
-            $_ktype882 = 0;
-            $_vtype883 = 0;
-            $xfer += $input->readMapBegin($_ktype882, $_vtype883, $_size881);
-            for ($_i885 = 0; $_i885 < $_size881; ++$_i885)
+            $_size873 = 0;
+            $_ktype874 = 0;
+            $_vtype875 = 0;
+            $xfer += $input->readMapBegin($_ktype874, $_vtype875, $_size873);
+            for ($_i877 = 0; $_i877 < $_size873; ++$_i877)
             {
-              $key886 = '';
-              $val887 = new \metastore\Materialization();
-              $xfer += $input->readString($key886);
-              $val887 = new \metastore\Materialization();
-              $xfer += $val887->read($input);
-              $this->success[$key886] = $val887;
+              $key878 = '';
+              $val879 = new \metastore\Materialization();
+              $xfer += $input->readString($key878);
+              $val879 = new \metastore\Materialization();
+              $xfer += $val879->read($input);
+              $this->success[$key878] = $val879;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -20297,10 +20177,10 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $kiter888 => $viter889)
+          foreach ($this->success as $kiter880 => $viter881)
           {
-            $xfer += $output->writeString($kiter888);
-            $xfer += $viter889->write($output);
+            $xfer += $output->writeString($kiter880);
+            $xfer += $viter881->write($output);
           }
         }
         $output->writeMapEnd();
@@ -20536,14 +20416,14 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size890 = 0;
-            $_etype893 = 0;
-            $xfer += $input->readListBegin($_etype893, $_size890);
-            for ($_i894 = 0; $_i894 < $_size890; ++$_i894)
+            $_size882 = 0;
+            $_etype885 = 0;
+            $xfer += $input->readListBegin($_etype885, $_size882);
+            for ($_i886 = 0; $_i886 < $_size882; ++$_i886)
             {
-              $elem895 = null;
-              $xfer += $input->readString($elem895);
-              $this->success []= $elem895;
+              $elem887 = null;
+              $xfer += $input->readString($elem887);
+              $this->success []= $elem887;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20595,9 +20475,9 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter896)
+          foreach ($this->success as $iter888)
           {
-            $xfer += $output->writeString($iter896);
+            $xfer += $output->writeString($iter888);
           }
         }
         $output->writeListEnd();
@@ -21910,15 +21790,15 @@ class ThriftHiveMetastore_add_partitions_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size897 = 0;
-            $_etype900 = 0;
-            $xfer += $input->readListBegin($_etype900, $_size897);
-            for ($_i901 = 0; $_i901 < $_size897; ++$_i901)
+            $_size889 = 0;
+            $_etype892 = 0;
+            $xfer += $input->readListBegin($_etype892, $_size889);
+            for ($_i893 = 0; $_i893 < $_size889; ++$_i893)
             {
-              $elem902 = null;
-              $elem902 = new \metastore\Partition();
-              $xfer += $elem902->read($input);
-              $this->new_parts []= $elem902;
+              $elem894 = null;
+              $elem894 = new \metastore\Partition();
+              $xfer += $elem894->read($input);
+              $this->new_parts []= $elem894;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21946,9 +21826,9 @@ class ThriftHiveMetastore_add_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter903)
+          foreach ($this->new_parts as $iter895)
           {
-            $xfer += $iter903->write($output);
+            $xfer += $iter895->write($output);
           }
         }
         $output->writeListEnd();
@@ -22163,15 +22043,15 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size904 = 0;
-            $_etype907 = 0;
-            $xfer += $input->readListBegin($_etype907, $_size904);
-            for ($_i908 = 0; $_i908 < $_size904; ++$_i908)
+            $_size896 = 0;
+            $_etype899 = 0;
+            $xfer += $input->readListBegin($_etype899, $_size896);
+            for ($_i900 = 0; $_i900 < $_size896; ++$_i900)
             {
-              $elem909 = null;
-              $elem909 = new \metastore\PartitionSpec();
-              $xfer += $elem909->read($input);
-              $this->new_parts []= $elem909;
+              $elem901 = null;
+              $elem901 = new \metastore\PartitionSpec();
+              $xfer += $elem901->read($input);
+              $this->new_parts []= $elem901;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22199,9 +22079,9 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter910)
+          foreach ($this->new_parts as $iter902)
           {
-            $xfer += $iter910->write($output);
+            $xfer += $iter902->write($output);
           }
         }
         $output->writeListEnd();
@@ -22451,14 +22331,14 @@ class ThriftHiveMetastore_append_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size911 = 0;
-            $_etype914 = 0;
-            $xfer += $input->readListBegin($_etype914, $_size911);
-            for ($_i915 = 0; $_i915 < $_size911; ++$_i915)
+            $_size903 = 0;
+            $_etype906 = 0;
+            $xfer += $input->readListBegin($_etype906, $_size903);
+            for ($_i907 = 0; $_i907 < $_size903; ++$_i907)
             {
-              $elem916 = null;
-              $xfer += $input->readString($elem916);
-              $this->part_vals []= $elem916;
+              $elem908 = null;
+              $xfer += $input->readString($elem908);
+              $this->part_vals []= $elem908;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22496,9 +22376,9 @@ class ThriftHiveMetastore_append_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter917)
+          foreach ($this->part_vals as $iter909)
           {
-            $xfer += $output->writeString($iter917);
+            $xfer += $output->writeString($iter909);
           }
         }
         $output->writeListEnd();
@@ -23000,14 +22880,14 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size918 = 0;
-            $_etype921 = 0;
-            $xfer += $input->readListBegin($_etype921, $_size918);
-            for ($_i922 = 0; $_i922 < $_size918; ++$_i922)
+            $_size910 = 0;
+            $_etype913 = 0;
+            $xfer += $input->readListBegin($_etype913, $_size910);
+            for ($_i914 = 0; $_i914 < $_size910; ++$_i914)
             {
-              $elem923 = null;
-              $xfer += $input->readString($elem923);
-              $this->part_vals []= $elem923;
+              $elem915 = null;
+              $xfer += $input->readString($elem915);
+              $this->part_vals []= $elem915;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23053,9 +22933,9 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter924)
+          foreach ($this->part_vals as $iter916)
           {
-            $xfer += $output->writeString($iter924);
+            $xfer += $output->writeString($iter916);
           }
         }
         $output->writeListEnd();
@@ -23909,14 +23789,14 @@ class ThriftHiveMetastore_drop_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size925 = 0;
-            $_etype928 = 0;
-            $xfer += $input->readListBegin($_etype928, $_size925);
-            for ($_i929 = 0; $_i929 < $_size925; ++$_i929)
+            $_size917 = 0;
+            $_etype920 = 0;
+            $xfer += $input->readListBegin($_etype920, $_size917);
+            for ($_i921 = 0; $_i921 < $_size917; ++$_i921)
             {
-              $elem930 = null;
-              $xfer += $input->readString($elem930);
-              $this->part_vals []= $elem930;
+              $elem922 = null;
+              $xfer += $input->readString($elem922);
+              $this->part_vals []= $elem922;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23961,9 +23841,9 @@ class ThriftHiveMetastore_drop_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter931)
+          foreach ($this->part_vals as $iter923)
           {
-            $xfer += $output->writeString($iter931);
+            $xfer += $output->writeString($iter923);
           }
         }
         $output->writeListEnd();
@@ -24216,14 +24096,14 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size932 = 0;
-            $_etype935 = 0;
-            $xfer += $input->readListBegin($_etype935, $_size932);
-            for ($_i936 = 0; $_i936 < $_size932; ++$_i936)
+            $_size924 = 0;
+            $_etype927 = 0;
+            $xfer += $input->readListBegin($_etype927, $_size924);
+            for ($_i928 = 0; $_i928 < $_size924; ++$_i928)
             {
-              $elem937 = null;
-              $xfer += $input->readString($elem937);
-              $this->part_vals []= $elem937;
+              $elem929 = null;
+              $xfer += $input->readString($elem929);
+              $this->part_vals []= $elem929;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24276,9 +24156,9 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter938)
+          foreach ($this->part_vals as $iter930)
           {
-            $xfer += $output->writeString($iter938);
+            $xfer += $output->writeString($iter930);
           }
         }
         $output->writeListEnd();
@@ -25292,14 +25172,14 @@ class ThriftHiveMetastore_get_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size939 = 0;
-            $_etype942 = 0;
-            $xfer += $input->readListBegin($_etype942, $_size939);
-            for ($_i943 = 0; $_i943 < $_size939; ++$_i943)
+            $_size931 = 0;
+            $_etype934 = 0;
+            $xfer += $input->readListBegin($_etype934, $_size931);
+            for ($_i935 = 0; $_i935 < $_size931; ++$_i935)
             {
-              $elem944 = null;
-              $xfer += $input->readString($elem944);
-              $this->part_vals []= $elem944;
+              $elem936 = null;
+              $xfer += $input->readString($elem936);
+              $this->part_vals []= $elem936;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25337,9 +25217,9 @@ class ThriftHiveMetastore_get_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter945)
+          foreach ($this->part_vals as $iter937)
           {
-            $xfer += $output->writeString($iter945);
+            $xfer += $output->writeString($iter937);
           }
         }
         $output->writeListEnd();
@@ -25581,17 +25461,17 @@ class ThriftHiveMetastore_exchange_partition_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size946 = 0;
-            $_ktype947 = 0;
-            $_vtype948 = 0;
-            $xfer += $input->readMapBegin($_ktype947, $_vtype948, $_size946);
-            for ($_i950 = 0; $_i950 < $_size946; ++$_i950)
+            $_size938 = 0;
+            $_ktype939 = 0;
+            $_vtype940 = 0;
+            $xfer += $input->readMapBegin($_ktype939, $_vtype940, $_size938);
+            for ($_i942 = 0; $_i942 < $_size938; ++$_i942)
             {
-              $key951 = '';
-              $val952 = '';
-              $xfer += $input->readString($key951);
-              $xfer += $input->readString($val952);
-              $this->partitionSpecs[$key951] = $val952;
+              $key943 = '';
+              $val944 = '';
+              $xfer += $input->readString($key943);
+              $xfer += $input->readString($val944);
+              $this->partitionSpecs[$key943] = $val944;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -25647,10 +25527,10 @@ class ThriftHiveMetastore_exchange_partition_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter953 => $viter954)
+          foreach ($this->partitionSpecs as $kiter945 => $viter946)
           {
-            $xfer += $output->writeString($kiter953);
-            $xfer += $output->writeString($viter954);
+            $xfer += $output->writeString($kiter945);
+            $xfer += $output->writeString($viter946);
           }
         }
         $output->writeMapEnd();
@@ -25962,17 +25842,17 @@ class ThriftHiveMetastore_exchange_partitions_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size955 = 0;
-            $_ktype956 = 0;
-            $_vtype957 = 0;
-            $xfer += $input->readMapBegin($_ktype956, $_vtype957, $_size955);
-            for ($_i959 = 0; $_i959 < $_size955; ++$_i959)
+            $_size947 = 0;
+            $_ktype948 = 0;
+            $_vtype949 = 0;
+            $xfer += $input->readMapBegin($_ktype948, $_vtype949, $_size947);
+            for ($_i951 = 0; $_i951 < $_size947; ++$_i951)
             {
-              $key960 = '';
-              $val961 = '';
-              $xfer += $input->readString($key960);
-              $xfer += $input->readString($val961);
-              $this->partitionSpecs[$key960] = $val961;
+              $key952 = '';
+              $val953 = '';
+              $xfer += $input->readString($key952);
+              $xfer += $input->readString($val953);
+              $this->partitionSpecs[$key952] = $val953;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -26028,10 +25908,10 @@ class ThriftHiveMetastore_exchange_partitions_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter962 => $viter963)
+          foreach ($this->partitionSpecs as $kiter954 => $viter955)
           {
-            $xfer += $output->writeString($kiter962);
-            $xfer += $output->writeString($viter963);
+            $xfer += $output->writeString($kiter954);
+            $xfer += $output->writeString($viter955);
           }
         }
         $output->writeMapEnd();
@@ -26164,15 +26044,15 @@ class ThriftHiveMetastore_exchange_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size964 = 0;
-            $_etype967 = 0;
-            $xfer += $input->readListBegin($_etype967, $_size964);
-            for ($_i968 = 0; $_i968 < $_size964; ++$_i968)
+            $_size956 = 0;
+            $_etype959 = 0;
+            $xfer += $input->readListBegin($_etype959, $_size956);
+            for ($_i960 = 0; $_i960 < $_size956; ++$_i960)
             {
-              $elem969 = null;
-              $elem969 = new \metastore\Partition();
-              $xfer += $elem969->read($input);
-              $this->success []= $elem969;
+              $elem961 = null;
+              $elem961 = new \metastore\Partition();
+              $xfer += $elem961->read($input);
+              $this->success []= $elem961;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26232,9 +26112,9 @@ class ThriftHiveMetastore_exchange_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter970)
+          foreach ($this->success as $iter962)
           {
-            $xfer += $iter970->write($output);
+            $xfer += $iter962->write($output);
           }
         }
         $output->writeListEnd();
@@ -26380,14 +26260,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size971 = 0;
-            $_etype974 = 0;
-            $xfer += $input->readListBegin($_etype974, $_size971);
-            for ($_i975 = 0; $_i975 < $_size971; ++$_i975)
+            $_size963 = 0;
+            $_etype966 = 0;
+            $xfer += $input->readListBegin($_etype966, $_size963);
+            for ($_i967 = 0; $_i967 < $_size963; ++$_i967)
             {
-              $elem976 = null;
-              $xfer += $input->readString($elem976);
-              $this->part_vals []= $elem976;
+              $elem968 = null;
+              $xfer += $input->readString($elem968);
+              $this->part_vals []= $elem968;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26404,14 +26284,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size977 = 0;
-            $_etype980 = 0;
-            $xfer += $input->readListBegin($_etype980, $_size977);
-            for ($_i981 = 0; $_i981 < $_size977; ++$_i981)
+            $_size969 = 0;
+            $_etype972 = 0;
+            $xfer += $input->readListBegin($_etype972, $_size969);
+            for ($_i973 = 0; $_i973 < $_size969; ++$_i973)
             {
-              $elem982 = null;
-              $xfer += $input->readString($elem982);
-              $this->group_names []= $elem982;
+              $elem974 = null;
+              $xfer += $input->readString($elem974);
+              $this->group_names []= $elem974;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26449,9 +26329,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter983)
+          foreach ($this->part_vals as $iter975)
           {
-            $xfer += $output->writeString($iter983);
+            $xfer += $output->writeString($iter975);
           }
         }
         $output->writeListEnd();
@@ -26471,9 +26351,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter984)
+          foreach ($this->group_names as $iter976)
           {
-            $xfer += $output->writeString($iter984);
+            $xfer += $output->writeString($iter976);
           }
         }
         $output->writeListEnd();
@@ -27064,15 +26944,15 @@ class ThriftHiveMetastore_get_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size985 = 0;
-            $_etype988 = 0;
-            $xfer += $input->readListBegin($_etype988, $_size985);
-            for ($_i989 = 0; $_i989 < $_size985; ++$_i989)
+            $_size977 = 0;
+            $_etype980 = 0;
+            $xfer += $input->readListBegin($_etype980, $_size977);
+            for ($_i981 = 0; $_i981 < $_size977; ++$_i981)
             {
-              $elem990 = null;
-              $elem990 = new \metastore\Partition();
-              $xfer += $elem990->read($input);
-              $this->success []= $elem990;
+              $elem982 = null;
+              $elem982 = new \metastore\Partition();
+              $xfer += $elem982->read($input);
+              $this->success []= $elem982;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27116,9 +26996,9 @@ class ThriftHiveMetastore_get_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter991)
+          foreach ($this->success as $iter983)
           {
-            $xfer += $iter991->write($output);
+            $xfer += $iter983->write($output);
           }
         }
         $output->writeListEnd();
@@ -27264,14 +27144,14 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size992 = 0;
-            $_etype995 = 0;
-            $xfer += $input->readListBegin($_etype995, $_size992);
-            for ($_i996 = 0; $_i996 < $_size992; ++$_i996)
+            $_size984 = 0;
+            $_etype987 = 0;
+            $xfer += $input->readListBegin($_etype987, $_size984);
+            for ($_i988 = 0; $_i988 < $_size984; ++$_i988)
             {
-              $elem997 = null;
-              $xfer += $input->readString($elem997);
-              $this->group_names []= $elem997;
+              $elem989 = null;
+              $xfer += $input->readString($elem989);
+              $this->group_names []= $elem989;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27319,9 +27199,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter998)
+          foreach ($this->group_names as $iter990)
           {
-            $xfer += $output->writeString($iter998);
+            $xfer += $output->writeString($iter990);
           }
         }
         $output->writeListEnd();
@@ -27410,15 +27290,15 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size999 = 0;
-            $_etype1002 = 0;
-            $xfer += $input->readListBegin($_etype1002, $_size999);
-            for ($_i1003 = 0; $_i1003 < $_size999; ++$_i1003)
+            $_size991 = 0;
+            $_etype994 = 0;
+            $xfer += $input->readListBegin($_etype994, $_size991);
+            for ($_i995 = 0; $_i995 < $_size991; ++$_i995)
             {
-              $elem1004 = null;
-              $elem1004 = new \metastore\Partition();
-              $xfer += $elem1004->read($input);
-              $this->success []= $elem1004;
+              $elem996 = null;
+              $elem996 = new \metastore\Partition();
+              $xfer += $elem996->read($input);
+              $this->success []= $elem996;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27462,9 +27342,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1005)
+          foreach ($this->success as $iter997)
           {
-            $xfer += $iter1005->write($output);
+            $xfer += $iter997->write($output);
           }
         }
         $output->writeListEnd();
@@ -27684,15 +27564,15 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1006 = 0;
-            $_etype1009 = 0;
-            $xfer += $input->readListBegin($_etype1009, $_size1006);
-            for ($_i1010 = 0; $_i1010 < $_size1006; ++$_i1010)
+            $_size998 = 0;
+            $_etype1001 = 0;
+            $xfer += $input->readListBegin($_etype1001, $_size998);
+            for ($_i1002 = 0; $_i1002 < $_size998; ++$_i1002)
             {
-              $elem1011 = null;
-              $elem1011 = new \metastore\PartitionSpec();
-              $xfer += $elem1011->read($input);
-              $this->success []= $elem1011;
+              $elem1003 = null;
+              $elem1003 = new \metastore\PartitionSpec();
+              $xfer += $elem1003->read($input);
+              $this->success []= $elem1003;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27736,9 +27616,9 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1012)
+          foreach ($this->success as $iter1004)
           {
-            $xfer += $iter1012->write($output);
+            $xfer += $iter1004->write($output);
           }
         }
         $output->writeListEnd();
@@ -27957,14 +27837,14 @@ class ThriftHiveMetastore_get_partition_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1013 = 0;
-            $_etype1016 = 0;
-            $xfer += $input->readListBegin($_etype1016, $_size1013);
-            for ($_i1017 = 0; $_i1017 < $_size1013; ++$_i1017)
+            $_size1005 = 0;
+            $_etype1008 = 0;
+            $xfer += $input->readListBegin($_etype1008, $_size1005);
+            for ($_i1009 = 0; $_i1009 < $_size1005; ++$_i1009)
             {
-              $elem1018 = null;
-              $xfer += $input->readString($elem1018);
-              $this->success []= $elem1018;
+              $elem1010 = null;
+              $xfer += $input->readString($elem1010);
+              $this->success []= $elem1010;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28008,9 +27888,9 @@ class ThriftHiveMetastore_get_partition_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1019)
+          foreach ($this->success as $iter1011)
           {
-            $xfer += $output->writeString($iter1019);
+            $xfer += $output->writeString($iter1011);
           }
         }
         $output->writeListEnd();
@@ -28341,14 +28221,14 @@ class ThriftHiveMetastore_get_partitions_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1020 = 0;
-            $_etype1023 = 0;
-            $xfer += $input->readListBegin($_etype1023, $_size1020);
-            for ($_i1024 = 0; $_i1024 < $_size1020; ++$_i1024)
+            $_size1012 = 0;
+            $_etype1015 = 0;
+            $xfer += $input->readListBegin($_etype1015, $_size1012);
+            for ($_i1016 = 0; $_i1016 < $_size1012; ++$_i1016)
             {
-              $elem1025 = null;
-              $xfer += $input->readString($elem1025);
-              $this->part_vals []= $elem1025;
+              $elem1017 = null;
+              $xfer += $input->readString($elem1017);
+              $this->part_vals []= $elem1017;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28393,9 +28273,9 @@ class ThriftHiveMetastore_get_partitions_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1026)
+          foreach ($this->part_vals as $iter1018)
           {
-            $xfer += $output->writeString($iter1026);
+            $xfer += $output->writeString($iter1018);
           }
         }
         $output->writeListEnd();
@@ -28489,15 +28369,15 @@ class ThriftHiveMetastore_get_partitions_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1027 = 0;
-            $_etype1030 = 0;
-            $xfer += $input->readListBegin($_etype1030, $_size1027);
-            for ($_i1031 = 0; $_i1031 < $_size1027; ++$_i1031)
+            $_size1019 = 0;
+            $_etype1022 = 0;
+            $xfer += $input->readListBegin($_etype1022, $_size1019);
+            for ($_i1023 = 0; $_i1023 < $_size1019; ++$_i1023)
             {
-              $elem1032 = null;
-              $elem1032 = new \metastore\Partition();
-              $xfer += $elem1032->read($input);
-              $this->success []= $elem1032;
+              $elem1024 = null;
+              $elem1024 = new \metastore\Partition();
+              $xfer += $elem1024->read($input);
+              $this->success []= $elem1024;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28541,9 +28421,9 @@ class ThriftHiveMetastore_get_partitions_ps_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1033)
+          foreach ($this->success as $iter1025)
           {
-            $xfer += $iter1033->write($output);
+            $xfer += $iter1025->write($output);
           }
         }
         $output->writeListEnd();
@@ -28690,14 +28570,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1034 = 0;
-            $_etype1037 = 0;
-            $xfer += $input->readListBegin($_etype1037, $_size1034);
-            for ($_i1038 = 0; $_i1038 < $_size1034; ++$_i1038)
+            $_size1026 = 0;
+            $_etype1029 = 0;
+            $xfer += $input->readListBegin($_etype1029, $_size1026);
+            for ($_i1030 = 0; $_i1030 < $_size1026; ++$_i1030)
             {
-              $elem1039 = null;
-              $xfer += $input->readString($elem1039);
-              $this->part_vals []= $elem1039;
+              $elem1031 = null;
+              $xfer += $input->readString($elem1031);
+              $this->part_vals []= $elem1031;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28721,14 +28601,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1040 = 0;
-            $_etype1043 = 0;
-            $xfer += $input->readListBegin($_etype1043, $_size1040);
-            for ($_i1044 = 0; $_i1044 < $_size1040; ++$_i1044)
+            $_size1032 = 0;
+            $_etype1035 = 0;
+            $xfer += $input->readListBegin($_etype1035, $_size1032);
+            for ($_i1036 = 0; $_i1036 < $_size1032; ++$_i1036)
             {
-              $elem1045 = null;
-              $xfer += $input->readString($elem1045);
-              $this->group_names []= $elem1045;
+              $elem1037 = null;
+              $xfer += $input->readString($elem1037);
+              $this->group_names []= $elem1037;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28766,9 +28646,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1046)
+          foreach ($this->part_vals as $iter1038)
           {
-            $xfer += $output->writeString($iter1046);
+            $xfer += $output->writeString($iter1038);
           }
         }
         $output->writeListEnd();
@@ -28793,9 +28673,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1047)
+          foreach ($this->group_names as $iter1039)
           {
-            $xfer += $output->writeString($iter1047);
+            $xfer += $output->writeString($iter1039);
           }
         }
         $output->writeListEnd();
@@ -28884,15 +28764,15 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1048 = 0;
-            $_etype1051 = 0;
-            $xfer += $input->readListBegin($_etype1051, $_size1048);
-            for ($_i1052 = 0; $_i1052 < $_size1048; ++$_i1052)
+            $_size1040 = 0;
+            $_etype1043 = 0;
+            $xfer += $input->readListBegin($_etype1043, $_size1040);
+            for ($_i1044 = 0; $_i1044 < $_size1040; ++$_i1044)
             {
-              $elem1053 = null;
-              $elem1053 = new \metastore\Partition();
-              $xfer += $elem1053->read($input);
-              $this->success []= $elem1053;
+              $elem1045 = null;
+              $elem1045 = new \metastore\Partition();
+              $xfer += $elem1045->read($input);
+              $this->success []= $elem1045;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28936,9 +28816,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1054)
+          foreach ($this->success as $iter1046)
           {
-            $xfer += $iter1054->write($output);
+            $xfer += $iter1046->write($output);
           }
         }
         $output->writeListEnd();
@@ -29059,14 +28939,14 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1055 = 0;
-            $_etype1058 = 0;
-            $xfer += $input->readListBegin($_etype1058, $_size1055);
-            for ($_i1059 = 0; $_i1059 < $_size1055; ++$_i1059)
+            $_size1047 = 0;
+            $_etype1050 = 0;
+            $xfer += $input->readListBegin($_etype1050, $_size1047);
+            for ($_i1051 = 0; $_i1051 < $_size1047; ++$_i1051)
             {
-              $elem1060 = null;
-              $xfer += $input->readString($elem1060);
-              $this->part_vals []= $elem1060;
+              $elem1052 = null;
+              $xfer += $input->readString($elem1052);
+              $this->part_vals []= $elem1052;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29111,9 +28991,9 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1061)
+          foreach ($this->part_vals as $iter1053)
           {
-            $xfer += $output->writeString($iter1061);
+            $xfer += $output->writeString($iter1053);
           }
         }
         $output->writeListEnd();
@@ -29206,14 +29086,14 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1062 = 0;
-            $_etype1065 = 0;
-            $xfer += $input->readListBegin($_etype1065, $_size1062);
-            for ($_i1066 = 0; $_i1066 < $_size1062; ++$_i1066)
+            $_size1054 = 0;
+            $_etype1057 = 0;
+            $xfer += $input->readListBegin($_etype1057, $_size1054);
+            for ($_i1058 = 0; $_i1058 < $_size1054; ++$_i1058)
             {
-              $elem1067 = null;
-              $xfer += $input->readString($elem1067);
-              $this->success []= $elem1067;
+              $elem1059 = null;
+              $xfer += $input->readString($elem1059);
+              $this->success []= $elem1059;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29257,9 +29137,9 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1068)
+          foreach ($this->success as $iter1060)
           {
-            $xfer += $output->writeString($iter1068);
+            $xfer += $output->writeString($iter1060);
           }
         }
         $output->writeListEnd();
@@ -29502,15 +29382,15 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1069 = 0;
-            $_etype1072 = 0;
-            $xfer += $input->readListBegin($_etype1072, $_size1069);
-            for ($_i1073 = 0; $_i1073 < $_size1069; ++$_i1073)
+            $_size1061 = 0;
+            $_etype1064 = 0;
+            $xfer += $input->readListBegin($_etype1064, $_size1061);
+            for ($_i1065 = 0; $_i1065 < $_size1061; ++$_i1065)
             {
-              $elem1074 = null;
-              $elem1074 = new \metastore\Partition();
-              $xfer += $elem1074->read($input);
-              $this->success []= $elem1074;
+              $elem1066 = null;
+              $elem1066 = new \metastore\Partition();
+              $xfer += $elem1066->read($input);
+              $this->success []= $elem1066;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29554,9 +29434,9 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1075)
+          foreach ($this->success as $iter1067)
           {
-            $xfer += $iter1075->write($output);
+            $xfer += $iter1067->write($output);
           }
         }
         $output->writeListEnd();
@@ -29799,15 +29679,15 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1076 = 0;
-            $_etype1079 = 0;
-            $xfer += $input->readListBegin($_etype1079, $_size1076);
-            for ($_i1080 = 0; $_i1080 < $_size1076; ++$_i1080)
+            $_size1068 = 0;
+            $_etype1071 = 0;
+            $xfer += $input->readListBegin($_etype1071, $_size1068);
+            for ($_i1072 = 0; $_i1072 < $_size1068; ++$_i1072)
             {
-              $elem1081 = null;
-              $elem1081 = new \metastore\PartitionSpec();
-              $xfer += $elem1081->read($input);
-              $this->success []= $elem1081;
+              $elem1073 = null;
+              $elem1073 = new \metastore\PartitionSpec();
+              $xfer += $elem1073->read($input);
+              $this->success []= $elem1073;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29851,9 +29731,9 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1082)
+          foreach ($this->success as $iter1074)
           {
-            $xfer += $iter1082->write($output);
+            $xfer += $iter1074->write($output);
           }
         }
         $output->writeListEnd();
@@ -30419,14 +30299,14 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->names = array();
-            $_size1083 = 0;
-            $_etype1086 = 0;
-            $xfer += $input->readListBegin($_etype1086, $_size1083);
-            for ($_i1087 = 0; $_i1087 < $_size1083; ++$_i1087)
+            $_size1075 = 0;
+            $_etype1078 = 0;
+            $xfer += $input->readListBegin($_etype1078, $_size1075);
+            for ($_i1079 = 0; $_i1079 < $_size1075; ++$_i1079)
             {
-              $elem1088 = null;
-              $xfer += $input->readString($elem1088);
-              $this->names []= $elem1088;
+              $elem1080 = null;
+              $xfer += $input->readString($elem1080);
+              $this->names []= $elem1080;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30464,9 +30344,9 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
       {
         $output->writeListBegin(TType::STRING, count($this->names));
         {
-          foreach ($this->names as $iter1089)
+          foreach ($this->names as $iter1081)
           {
-            $xfer += $output->writeString($iter1089);
+            $xfer += $output->writeString($iter1081);
           }
         }
         $output->writeListEnd();
@@ -30555,15 +30435,15 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1090 = 0;
-            $_etype1093 = 0;
-            $xfer += $input->readListBegin($_etype1093, $_size1090);
-            for ($_i1094 = 0; $_i1094 < $_size1090; ++$_i1094)
+            $_size1082 = 0;
+            $_etype1085 = 0;
+            $xfer += $input->readListBegin($_etype1085, $_size1082);
+            for ($_i1086 = 0; $_i1086 < $_size1082; ++$_i1086)
             {
-              $elem1095 = null;
-              $elem1095 = new \metastore\Partition();
-              $xfer += $elem1095->read($input);
-              $this->success []= $elem1095;
+              $elem1087 = null;
+              $elem1087 = new \metastore\Partition();
+              $xfer += $elem1087->read($input);
+              $this->success []= $elem1087;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30607,9 +30487,9 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1096)
+          foreach ($this->success as $iter1088)
           {
-            $xfer += $iter1096->write($output);
+            $xfer += $iter1088->write($output);
           }
         }
         $output->writeListEnd();
@@ -30948,15 +30828,15 @@ class ThriftHiveMetastore_alter_partitions_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size1097 = 0;
-            $_etype1100 = 0;
-            $xfer += $input->readListBegin($_etype1100, $_size1097);
-            for ($_i1101 = 0; $_i1101 < $_size1097; ++$_i1101)
+            $_size1089 = 0;
+            $_etype1092 = 0;
+            $xfer += $input->readListBegin($_etype1092, $_size1089);
+            for ($_i1093 = 0; $_i1093 < $_size1089; ++$_i1093)
             {
-              $elem1102 = null;
-              $elem1102 = new \metastore\Partition();
-              $xfer += $elem1102->read($input);
-              $this->new_parts []= $elem1102;
+              $elem1094 = null;
+              $elem1094 = new \metastore\Partition();
+              $xfer += $elem1094->read($input);
+              $this->new_parts []= $elem1094;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30994,9 +30874,9 @@ class ThriftHiveMetastore_alter_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1103)
+          foreach ($this->new_parts as $iter1095)
           {
-            $xfer += $iter1103->write($output);
+            $xfer += $iter1095->write($output);
           }
         }
         $output->writeListEnd();
@@ -31211,15 +31091,15 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size1104 = 0;
-            $_etype1107 = 0;
-            $xfer += $input->readListBegin($_etype1107, $_size1104);
-            for ($_i1108 = 0; $_i1108 < $_size1104; ++$_i1108)
+            $_size1096 = 0;
+            $_etype1099 = 0;
+            $xfer += $input->readListBegin($_etype1099, $_size1096);
+            for ($_i1100 = 0; $_i1100 < $_size1096; ++$_i1100)
             {
-              $elem1109 = null;
-              $elem1109 = new \metastore\Partition();
-              $xfer += $elem1109->read($input);
-              $this->new_parts []= $elem1109;
+              $elem1101 = null;
+              $elem1101 = new \metastore\Partition();
+              $xfer += $elem1101->read($input);
+              $this->new_parts []= $elem1101;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31265,9 +31145,9 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1110)
+          foreach ($this->new_parts as $iter1102)
           {
-            $xfer += $iter1110->write($output);
+            $xfer += $iter1102->write($output);
           }
         }
         $output->writeListEnd();
@@ -31745,14 +31625,14 @@ class ThriftHiveMetastore_rename_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1111 = 0;
-            $_etype1114 = 0;
-            $xfer += $input->readListBegin($_etype1114, $_size1111);
-            for ($_i1115 = 0; $_i1115 < $_size1111; ++$_i1115)
+            $_size1103 = 0;
+            $_etype1106 = 0;
+            $xfer += $input->readListBegin($_etype1106, $_size1103);
+            for ($_i1107 = 0; $_i1107 < $_size1103; ++$_i1107)
             {
-              $elem1116 = null;
-              $xfer += $input->readString($elem1116);
-              $this->part_vals []= $elem1116;
+              $elem1108 = null;
+              $xfer += $input->readString($elem1108);
+              $this->part_vals []= $elem1108;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31798,9 +31678,9 @@ class ThriftHiveMetastore_rename_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1117)
+          foreach ($this->part_vals as $iter1109)
           {
-            $xfer += $output->writeString($iter1117);
+            $xfer += $output->writeString($iter1109);
           }
         }
         $output->writeListEnd();
@@ -31985,14 +31865,14 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1118 = 0;
-            $_etype1121 = 0;
-            $xfer += $input->readListBegin($_etype1121, $_size1118);
-            for ($_i1122 = 0; $_i1122 < $_size1118; ++$_i1122)
+            $_size1110 = 0;
+            $_etype1113 = 0;
+            $xfer += $input->readListBegin($_etype1113, $_size1110);
+            for ($_i1114 = 0; $_i1114 < $_size1110; ++$_i1114)
             {
-              $elem1123 = null;
-              $xfer += $input->readString($elem1123);
-              $this->part_vals []= $elem1123;
+              $elem1115 = null;
+              $xfer += $input->readString($elem1115);
+              $this->part_vals []= $elem1115;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32027,9 +31907,9 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1124)
+          foreach ($this->part_vals as $iter1116)
           {
-            $xfer += $output->writeString($iter1124);
+            $xfer += $output->writeString($iter1116);
           }
         }
         $output->writeListEnd();
@@ -32483,14 +32363,14 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1125 = 0;
-            $_etype1128 = 0;
-            $xfer += $input->readListBegin($_etype1128, $_size1125);
-            for ($_i1129 = 0; $_i1129 < $_size1125; ++$_i1129)
+            $_size1117 = 0;
+            $_etype1120 = 0;
+            $xfer += $input->readListBegin($_etype1120, $_size1117);
+            for ($_i1121 = 0; $_i1121 < $_size1117; ++$_i1121)
             {
-              $elem1130 = null;
-              $xfer += $input->readString($elem1130);
-              $this->success []= $elem1130;
+              $elem1122 = null;
+              $xfer += $input->readString($elem1122);
+              $this->success []= $elem1122;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32526,9 +32406,9 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1131)
+          foreach ($this->success as $iter1123)
           {
-            $xfer += $output->writeString($iter1131);
+            $xfer += $output->writeString($iter1123);
           }
         }
         $output->writeListEnd();
@@ -32688,17 +32568,17 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size1132 = 0;
-            $_ktype1133 = 0;
-            $_vtype1134 = 0;
-            $xfer += $input->readMapBegin($_ktype1133, $_vtype1134, $_size1132);
-            for ($_i1136 = 0; $_i1136 < $_size1132; ++$_i1136)
+            $_size1124 = 0;
+            $_ktype1125 = 0;
+            $_vtype1126 = 0;
+            $xfer += $input->readMapBegin($_ktype1125, $_vtype1126, $_size1124);
+            for ($_i1128 = 0; $_i1128 < $_size1124; ++$_i1128)
             {
-              $key1137 = '';
-              $val1138 = '';
-              $xfer += $input->readString($key1137);
-              $xfer += $input->readString($val1138);
-              $this->success[$key1137] = $val1138;
+              $key1129 = '';
+              $val1130 = '';
+              $xfer += $input->readString($key1129);
+              $xfer += $input->readString($val1130);
+              $this->success[$key1129] = $val1130;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -32734,10 +32614,10 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->success));
         {
-          foreach ($this->success as $kiter1139 => $viter1140)
+          foreach ($this->success as $kiter1131 => $viter1132)
           {
-            $xfer += $output->writeString($kiter1139);
-            $xfer += $output->writeString($viter1140);
+            $xfer += $output->writeString($kiter1131);
+            $xfer += $output->writeString($viter1132);
           }
         }
         $output->writeMapEnd();
@@ -32857,17 +32737,17 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size1141 = 0;
-            $_ktype1142 = 0;
-            $_vtype1143 = 0;
-            $xfer += $input->readMapBegin($_ktype1142, $_vtype1143, $_size1141);
-            for ($_i1145 = 0; $_i1145 < $_size1141; ++$_i1145)
+            $_size1133 = 0;
+            $_ktype1134 = 0;
+            $_vtype1135 = 0;
+            $xfer += $input->readMapBegin($_ktype1134, $_vtype1135, $_size1133);
+  

<TRUNCATED>

[10/16] hive git commit: HIVE-18546: Remove unnecessary code introduced in HIVE-14498 (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Index.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Index.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Index.java
index 556e7c6..8f2af25 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Index.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Index.java
@@ -1133,15 +1133,15 @@ import org.slf4j.LoggerFactory;
           case 9: // PARAMETERS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map270 = iprot.readMapBegin();
-                struct.parameters = new HashMap<String,String>(2*_map270.size);
-                String _key271;
-                String _val272;
-                for (int _i273 = 0; _i273 < _map270.size; ++_i273)
+                org.apache.thrift.protocol.TMap _map260 = iprot.readMapBegin();
+                struct.parameters = new HashMap<String,String>(2*_map260.size);
+                String _key261;
+                String _val262;
+                for (int _i263 = 0; _i263 < _map260.size; ++_i263)
                 {
-                  _key271 = iprot.readString();
-                  _val272 = iprot.readString();
-                  struct.parameters.put(_key271, _val272);
+                  _key261 = iprot.readString();
+                  _val262 = iprot.readString();
+                  struct.parameters.put(_key261, _val262);
                 }
                 iprot.readMapEnd();
               }
@@ -1211,10 +1211,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARAMETERS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.parameters.size()));
-          for (Map.Entry<String, String> _iter274 : struct.parameters.entrySet())
+          for (Map.Entry<String, String> _iter264 : struct.parameters.entrySet())
           {
-            oprot.writeString(_iter274.getKey());
-            oprot.writeString(_iter274.getValue());
+            oprot.writeString(_iter264.getKey());
+            oprot.writeString(_iter264.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -1299,10 +1299,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetParameters()) {
         {
           oprot.writeI32(struct.parameters.size());
-          for (Map.Entry<String, String> _iter275 : struct.parameters.entrySet())
+          for (Map.Entry<String, String> _iter265 : struct.parameters.entrySet())
           {
-            oprot.writeString(_iter275.getKey());
-            oprot.writeString(_iter275.getValue());
+            oprot.writeString(_iter265.getKey());
+            oprot.writeString(_iter265.getValue());
           }
         }
       }
@@ -1350,15 +1350,15 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(8)) {
         {
-          org.apache.thrift.protocol.TMap _map276 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.parameters = new HashMap<String,String>(2*_map276.size);
-          String _key277;
-          String _val278;
-          for (int _i279 = 0; _i279 < _map276.size; ++_i279)
+          org.apache.thrift.protocol.TMap _map266 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.parameters = new HashMap<String,String>(2*_map266.size);
+          String _key267;
+          String _val268;
+          for (int _i269 = 0; _i269 < _map266.size; ++_i269)
           {
-            _key277 = iprot.readString();
-            _val278 = iprot.readString();
-            struct.parameters.put(_key277, _val278);
+            _key267 = iprot.readString();
+            _val268 = iprot.readString();
+            struct.parameters.put(_key267, _val268);
           }
         }
         struct.setParametersIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
index 184f9d5..e23bc04 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
@@ -538,13 +538,13 @@ import org.slf4j.LoggerFactory;
           case 2: // FILES_ADDED
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list640 = iprot.readListBegin();
-                struct.filesAdded = new ArrayList<String>(_list640.size);
-                String _elem641;
-                for (int _i642 = 0; _i642 < _list640.size; ++_i642)
+                org.apache.thrift.protocol.TList _list630 = iprot.readListBegin();
+                struct.filesAdded = new ArrayList<String>(_list630.size);
+                String _elem631;
+                for (int _i632 = 0; _i632 < _list630.size; ++_i632)
                 {
-                  _elem641 = iprot.readString();
-                  struct.filesAdded.add(_elem641);
+                  _elem631 = iprot.readString();
+                  struct.filesAdded.add(_elem631);
                 }
                 iprot.readListEnd();
               }
@@ -556,13 +556,13 @@ import org.slf4j.LoggerFactory;
           case 3: // FILES_ADDED_CHECKSUM
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list643 = iprot.readListBegin();
-                struct.filesAddedChecksum = new ArrayList<String>(_list643.size);
-                String _elem644;
-                for (int _i645 = 0; _i645 < _list643.size; ++_i645)
+                org.apache.thrift.protocol.TList _list633 = iprot.readListBegin();
+                struct.filesAddedChecksum = new ArrayList<String>(_list633.size);
+                String _elem634;
+                for (int _i635 = 0; _i635 < _list633.size; ++_i635)
                 {
-                  _elem644 = iprot.readString();
-                  struct.filesAddedChecksum.add(_elem644);
+                  _elem634 = iprot.readString();
+                  struct.filesAddedChecksum.add(_elem634);
                 }
                 iprot.readListEnd();
               }
@@ -593,9 +593,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILES_ADDED_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filesAdded.size()));
-          for (String _iter646 : struct.filesAdded)
+          for (String _iter636 : struct.filesAdded)
           {
-            oprot.writeString(_iter646);
+            oprot.writeString(_iter636);
           }
           oprot.writeListEnd();
         }
@@ -606,9 +606,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(FILES_ADDED_CHECKSUM_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filesAddedChecksum.size()));
-            for (String _iter647 : struct.filesAddedChecksum)
+            for (String _iter637 : struct.filesAddedChecksum)
             {
-              oprot.writeString(_iter647);
+              oprot.writeString(_iter637);
             }
             oprot.writeListEnd();
           }
@@ -634,9 +634,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.filesAdded.size());
-        for (String _iter648 : struct.filesAdded)
+        for (String _iter638 : struct.filesAdded)
         {
-          oprot.writeString(_iter648);
+          oprot.writeString(_iter638);
         }
       }
       BitSet optionals = new BitSet();
@@ -653,9 +653,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetFilesAddedChecksum()) {
         {
           oprot.writeI32(struct.filesAddedChecksum.size());
-          for (String _iter649 : struct.filesAddedChecksum)
+          for (String _iter639 : struct.filesAddedChecksum)
           {
-            oprot.writeString(_iter649);
+            oprot.writeString(_iter639);
           }
         }
       }
@@ -665,13 +665,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, InsertEventRequestData struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list650 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.filesAdded = new ArrayList<String>(_list650.size);
-        String _elem651;
-        for (int _i652 = 0; _i652 < _list650.size; ++_i652)
+        org.apache.thrift.protocol.TList _list640 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.filesAdded = new ArrayList<String>(_list640.size);
+        String _elem641;
+        for (int _i642 = 0; _i642 < _list640.size; ++_i642)
         {
-          _elem651 = iprot.readString();
-          struct.filesAdded.add(_elem651);
+          _elem641 = iprot.readString();
+          struct.filesAdded.add(_elem641);
         }
       }
       struct.setFilesAddedIsSet(true);
@@ -682,13 +682,13 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list653 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.filesAddedChecksum = new ArrayList<String>(_list653.size);
-          String _elem654;
-          for (int _i655 = 0; _i655 < _list653.size; ++_i655)
+          org.apache.thrift.protocol.TList _list643 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.filesAddedChecksum = new ArrayList<String>(_list643.size);
+          String _elem644;
+          for (int _i645 = 0; _i645 < _list643.size; ++_i645)
           {
-            _elem654 = iprot.readString();
-            struct.filesAddedChecksum.add(_elem654);
+            _elem644 = iprot.readString();
+            struct.filesAddedChecksum.add(_elem644);
           }
         }
         struct.setFilesAddedChecksumIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
index b5d17cc..6aaed5c 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
@@ -689,14 +689,14 @@ import org.slf4j.LoggerFactory;
           case 1: // COMPONENT
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list566 = iprot.readListBegin();
-                struct.component = new ArrayList<LockComponent>(_list566.size);
-                LockComponent _elem567;
-                for (int _i568 = 0; _i568 < _list566.size; ++_i568)
+                org.apache.thrift.protocol.TList _list556 = iprot.readListBegin();
+                struct.component = new ArrayList<LockComponent>(_list556.size);
+                LockComponent _elem557;
+                for (int _i558 = 0; _i558 < _list556.size; ++_i558)
                 {
-                  _elem567 = new LockComponent();
-                  _elem567.read(iprot);
-                  struct.component.add(_elem567);
+                  _elem557 = new LockComponent();
+                  _elem557.read(iprot);
+                  struct.component.add(_elem557);
                 }
                 iprot.readListEnd();
               }
@@ -754,9 +754,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COMPONENT_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.component.size()));
-          for (LockComponent _iter569 : struct.component)
+          for (LockComponent _iter559 : struct.component)
           {
-            _iter569.write(oprot);
+            _iter559.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -803,9 +803,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.component.size());
-        for (LockComponent _iter570 : struct.component)
+        for (LockComponent _iter560 : struct.component)
         {
-          _iter570.write(oprot);
+          _iter560.write(oprot);
         }
       }
       oprot.writeString(struct.user);
@@ -830,14 +830,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, LockRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list571 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.component = new ArrayList<LockComponent>(_list571.size);
-        LockComponent _elem572;
-        for (int _i573 = 0; _i573 < _list571.size; ++_i573)
+        org.apache.thrift.protocol.TList _list561 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.component = new ArrayList<LockComponent>(_list561.size);
+        LockComponent _elem562;
+        for (int _i563 = 0; _i563 < _list561.size; ++_i563)
         {
-          _elem572 = new LockComponent();
-          _elem572.read(iprot);
-          struct.component.add(_elem572);
+          _elem562 = new LockComponent();
+          _elem562.read(iprot);
+          struct.component.add(_elem562);
         }
       }
       struct.setComponentIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java
index f217bf0..b399d66 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java
@@ -533,13 +533,13 @@ import org.slf4j.LoggerFactory;
           case 2: // TABLES_USED
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set756 = iprot.readSetBegin();
-                struct.tablesUsed = new HashSet<String>(2*_set756.size);
-                String _elem757;
-                for (int _i758 = 0; _i758 < _set756.size; ++_i758)
+                org.apache.thrift.protocol.TSet _set746 = iprot.readSetBegin();
+                struct.tablesUsed = new HashSet<String>(2*_set746.size);
+                String _elem747;
+                for (int _i748 = 0; _i748 < _set746.size; ++_i748)
                 {
-                  _elem757 = iprot.readString();
-                  struct.tablesUsed.add(_elem757);
+                  _elem747 = iprot.readString();
+                  struct.tablesUsed.add(_elem747);
                 }
                 iprot.readSetEnd();
               }
@@ -578,9 +578,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TABLES_USED_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, struct.tablesUsed.size()));
-          for (String _iter759 : struct.tablesUsed)
+          for (String _iter749 : struct.tablesUsed)
           {
-            oprot.writeString(_iter759);
+            oprot.writeString(_iter749);
           }
           oprot.writeSetEnd();
         }
@@ -609,9 +609,9 @@ import org.slf4j.LoggerFactory;
       struct.materializationTable.write(oprot);
       {
         oprot.writeI32(struct.tablesUsed.size());
-        for (String _iter760 : struct.tablesUsed)
+        for (String _iter750 : struct.tablesUsed)
         {
-          oprot.writeString(_iter760);
+          oprot.writeString(_iter750);
         }
       }
       oprot.writeI64(struct.invalidationTime);
@@ -624,13 +624,13 @@ import org.slf4j.LoggerFactory;
       struct.materializationTable.read(iprot);
       struct.setMaterializationTableIsSet(true);
       {
-        org.apache.thrift.protocol.TSet _set761 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.tablesUsed = new HashSet<String>(2*_set761.size);
-        String _elem762;
-        for (int _i763 = 0; _i763 < _set761.size; ++_i763)
+        org.apache.thrift.protocol.TSet _set751 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.tablesUsed = new HashSet<String>(2*_set751.size);
+        String _elem752;
+        for (int _i753 = 0; _i753 < _set751.size; ++_i753)
         {
-          _elem762 = iprot.readString();
-          struct.tablesUsed.add(_elem762);
+          _elem752 = iprot.readString();
+          struct.tablesUsed.add(_elem752);
         }
       }
       struct.setTablesUsedIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotNullConstraintsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotNullConstraintsResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotNullConstraintsResponse.java
index 3257a41..8566d3d 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotNullConstraintsResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotNullConstraintsResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // NOT_NULL_CONSTRAINTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list356 = iprot.readListBegin();
-                struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list356.size);
-                SQLNotNullConstraint _elem357;
-                for (int _i358 = 0; _i358 < _list356.size; ++_i358)
+                org.apache.thrift.protocol.TList _list346 = iprot.readListBegin();
+                struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list346.size);
+                SQLNotNullConstraint _elem347;
+                for (int _i348 = 0; _i348 < _list346.size; ++_i348)
                 {
-                  _elem357 = new SQLNotNullConstraint();
-                  _elem357.read(iprot);
-                  struct.notNullConstraints.add(_elem357);
+                  _elem347 = new SQLNotNullConstraint();
+                  _elem347.read(iprot);
+                  struct.notNullConstraints.add(_elem347);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(NOT_NULL_CONSTRAINTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.notNullConstraints.size()));
-          for (SQLNotNullConstraint _iter359 : struct.notNullConstraints)
+          for (SQLNotNullConstraint _iter349 : struct.notNullConstraints)
           {
-            _iter359.write(oprot);
+            _iter349.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.notNullConstraints.size());
-        for (SQLNotNullConstraint _iter360 : struct.notNullConstraints)
+        for (SQLNotNullConstraint _iter350 : struct.notNullConstraints)
         {
-          _iter360.write(oprot);
+          _iter350.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, NotNullConstraintsResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list361 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list361.size);
-        SQLNotNullConstraint _elem362;
-        for (int _i363 = 0; _i363 < _list361.size; ++_i363)
+        org.apache.thrift.protocol.TList _list351 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list351.size);
+        SQLNotNullConstraint _elem352;
+        for (int _i353 = 0; _i353 < _list351.size; ++_i353)
         {
-          _elem362 = new SQLNotNullConstraint();
-          _elem362.read(iprot);
-          struct.notNullConstraints.add(_elem362);
+          _elem352 = new SQLNotNullConstraint();
+          _elem352.read(iprot);
+          struct.notNullConstraints.add(_elem352);
         }
       }
       struct.setNotNullConstraintsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
index eb57844..a28350b 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // EVENTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list632 = iprot.readListBegin();
-                struct.events = new ArrayList<NotificationEvent>(_list632.size);
-                NotificationEvent _elem633;
-                for (int _i634 = 0; _i634 < _list632.size; ++_i634)
+                org.apache.thrift.protocol.TList _list622 = iprot.readListBegin();
+                struct.events = new ArrayList<NotificationEvent>(_list622.size);
+                NotificationEvent _elem623;
+                for (int _i624 = 0; _i624 < _list622.size; ++_i624)
                 {
-                  _elem633 = new NotificationEvent();
-                  _elem633.read(iprot);
-                  struct.events.add(_elem633);
+                  _elem623 = new NotificationEvent();
+                  _elem623.read(iprot);
+                  struct.events.add(_elem623);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(EVENTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.events.size()));
-          for (NotificationEvent _iter635 : struct.events)
+          for (NotificationEvent _iter625 : struct.events)
           {
-            _iter635.write(oprot);
+            _iter625.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.events.size());
-        for (NotificationEvent _iter636 : struct.events)
+        for (NotificationEvent _iter626 : struct.events)
         {
-          _iter636.write(oprot);
+          _iter626.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, NotificationEventResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list637 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.events = new ArrayList<NotificationEvent>(_list637.size);
-        NotificationEvent _elem638;
-        for (int _i639 = 0; _i639 < _list637.size; ++_i639)
+        org.apache.thrift.protocol.TList _list627 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.events = new ArrayList<NotificationEvent>(_list627.size);
+        NotificationEvent _elem628;
+        for (int _i629 = 0; _i629 < _list627.size; ++_i629)
         {
-          _elem638 = new NotificationEvent();
-          _elem638.read(iprot);
-          struct.events.add(_elem638);
+          _elem628 = new NotificationEvent();
+          _elem628.read(iprot);
+          struct.events.add(_elem628);
         }
       }
       struct.setEventsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java
index d1a1bf8..ee7ae39 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java
@@ -351,13 +351,13 @@ import org.slf4j.LoggerFactory;
           case 1: // TXN_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list550 = iprot.readListBegin();
-                struct.txn_ids = new ArrayList<Long>(_list550.size);
-                long _elem551;
-                for (int _i552 = 0; _i552 < _list550.size; ++_i552)
+                org.apache.thrift.protocol.TList _list540 = iprot.readListBegin();
+                struct.txn_ids = new ArrayList<Long>(_list540.size);
+                long _elem541;
+                for (int _i542 = 0; _i542 < _list540.size; ++_i542)
                 {
-                  _elem551 = iprot.readI64();
-                  struct.txn_ids.add(_elem551);
+                  _elem541 = iprot.readI64();
+                  struct.txn_ids.add(_elem541);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TXN_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.txn_ids.size()));
-          for (long _iter553 : struct.txn_ids)
+          for (long _iter543 : struct.txn_ids)
           {
-            oprot.writeI64(_iter553);
+            oprot.writeI64(_iter543);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.txn_ids.size());
-        for (long _iter554 : struct.txn_ids)
+        for (long _iter544 : struct.txn_ids)
         {
-          oprot.writeI64(_iter554);
+          oprot.writeI64(_iter544);
         }
       }
     }
@@ -421,13 +421,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, OpenTxnsResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list555 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.txn_ids = new ArrayList<Long>(_list555.size);
-        long _elem556;
-        for (int _i557 = 0; _i557 < _list555.size; ++_i557)
+        org.apache.thrift.protocol.TList _list545 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.txn_ids = new ArrayList<Long>(_list545.size);
+        long _elem546;
+        for (int _i547 = 0; _i547 < _list545.size; ++_i547)
         {
-          _elem556 = iprot.readI64();
-          struct.txn_ids.add(_elem556);
+          _elem546 = iprot.readI64();
+          struct.txn_ids.add(_elem546);
         }
       }
       struct.setTxn_idsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java
index 7ec6172..3a13753 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java
@@ -931,13 +931,13 @@ import org.slf4j.LoggerFactory;
           case 1: // VALUES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list218 = iprot.readListBegin();
-                struct.values = new ArrayList<String>(_list218.size);
-                String _elem219;
-                for (int _i220 = 0; _i220 < _list218.size; ++_i220)
+                org.apache.thrift.protocol.TList _list208 = iprot.readListBegin();
+                struct.values = new ArrayList<String>(_list208.size);
+                String _elem209;
+                for (int _i210 = 0; _i210 < _list208.size; ++_i210)
                 {
-                  _elem219 = iprot.readString();
-                  struct.values.add(_elem219);
+                  _elem209 = iprot.readString();
+                  struct.values.add(_elem209);
                 }
                 iprot.readListEnd();
               }
@@ -990,15 +990,15 @@ import org.slf4j.LoggerFactory;
           case 7: // PARAMETERS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map221 = iprot.readMapBegin();
-                struct.parameters = new HashMap<String,String>(2*_map221.size);
-                String _key222;
-                String _val223;
-                for (int _i224 = 0; _i224 < _map221.size; ++_i224)
+                org.apache.thrift.protocol.TMap _map211 = iprot.readMapBegin();
+                struct.parameters = new HashMap<String,String>(2*_map211.size);
+                String _key212;
+                String _val213;
+                for (int _i214 = 0; _i214 < _map211.size; ++_i214)
                 {
-                  _key222 = iprot.readString();
-                  _val223 = iprot.readString();
-                  struct.parameters.put(_key222, _val223);
+                  _key212 = iprot.readString();
+                  _val213 = iprot.readString();
+                  struct.parameters.put(_key212, _val213);
                 }
                 iprot.readMapEnd();
               }
@@ -1033,9 +1033,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(VALUES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.values.size()));
-          for (String _iter225 : struct.values)
+          for (String _iter215 : struct.values)
           {
-            oprot.writeString(_iter225);
+            oprot.writeString(_iter215);
           }
           oprot.writeListEnd();
         }
@@ -1066,10 +1066,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARAMETERS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.parameters.size()));
-          for (Map.Entry<String, String> _iter226 : struct.parameters.entrySet())
+          for (Map.Entry<String, String> _iter216 : struct.parameters.entrySet())
           {
-            oprot.writeString(_iter226.getKey());
-            oprot.writeString(_iter226.getValue());
+            oprot.writeString(_iter216.getKey());
+            oprot.writeString(_iter216.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -1128,9 +1128,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetValues()) {
         {
           oprot.writeI32(struct.values.size());
-          for (String _iter227 : struct.values)
+          for (String _iter217 : struct.values)
           {
-            oprot.writeString(_iter227);
+            oprot.writeString(_iter217);
           }
         }
       }
@@ -1152,10 +1152,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetParameters()) {
         {
           oprot.writeI32(struct.parameters.size());
-          for (Map.Entry<String, String> _iter228 : struct.parameters.entrySet())
+          for (Map.Entry<String, String> _iter218 : struct.parameters.entrySet())
           {
-            oprot.writeString(_iter228.getKey());
-            oprot.writeString(_iter228.getValue());
+            oprot.writeString(_iter218.getKey());
+            oprot.writeString(_iter218.getValue());
           }
         }
       }
@@ -1170,13 +1170,13 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(8);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list229 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.values = new ArrayList<String>(_list229.size);
-          String _elem230;
-          for (int _i231 = 0; _i231 < _list229.size; ++_i231)
+          org.apache.thrift.protocol.TList _list219 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.values = new ArrayList<String>(_list219.size);
+          String _elem220;
+          for (int _i221 = 0; _i221 < _list219.size; ++_i221)
           {
-            _elem230 = iprot.readString();
-            struct.values.add(_elem230);
+            _elem220 = iprot.readString();
+            struct.values.add(_elem220);
           }
         }
         struct.setValuesIsSet(true);
@@ -1204,15 +1204,15 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(6)) {
         {
-          org.apache.thrift.protocol.TMap _map232 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.parameters = new HashMap<String,String>(2*_map232.size);
-          String _key233;
-          String _val234;
-          for (int _i235 = 0; _i235 < _map232.size; ++_i235)
+          org.apache.thrift.protocol.TMap _map222 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.parameters = new HashMap<String,String>(2*_map222.size);
+          String _key223;
+          String _val224;
+          for (int _i225 = 0; _i225 < _map222.size; ++_i225)
           {
-            _key233 = iprot.readString();
-            _val234 = iprot.readString();
-            struct.parameters.put(_key233, _val234);
+            _key223 = iprot.readString();
+            _val224 = iprot.readString();
+            struct.parameters.put(_key223, _val224);
           }
         }
         struct.parameters = org.apache.hadoop.hive.metastore.utils.StringUtils.intern(struct.parameters); struct.setParametersIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionListComposingSpec.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionListComposingSpec.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionListComposingSpec.java
index 17b6c44..186eb23 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionListComposingSpec.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionListComposingSpec.java
@@ -350,14 +350,14 @@ import org.slf4j.LoggerFactory;
           case 1: // PARTITIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list262 = iprot.readListBegin();
-                struct.partitions = new ArrayList<Partition>(_list262.size);
-                Partition _elem263;
-                for (int _i264 = 0; _i264 < _list262.size; ++_i264)
+                org.apache.thrift.protocol.TList _list252 = iprot.readListBegin();
+                struct.partitions = new ArrayList<Partition>(_list252.size);
+                Partition _elem253;
+                for (int _i254 = 0; _i254 < _list252.size; ++_i254)
                 {
-                  _elem263 = new Partition();
-                  _elem263.read(iprot);
-                  struct.partitions.add(_elem263);
+                  _elem253 = new Partition();
+                  _elem253.read(iprot);
+                  struct.partitions.add(_elem253);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARTITIONS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitions.size()));
-          for (Partition _iter265 : struct.partitions)
+          for (Partition _iter255 : struct.partitions)
           {
-            _iter265.write(oprot);
+            _iter255.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -416,9 +416,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitions()) {
         {
           oprot.writeI32(struct.partitions.size());
-          for (Partition _iter266 : struct.partitions)
+          for (Partition _iter256 : struct.partitions)
           {
-            _iter266.write(oprot);
+            _iter256.write(oprot);
           }
         }
       }
@@ -430,14 +430,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list267 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.partitions = new ArrayList<Partition>(_list267.size);
-          Partition _elem268;
-          for (int _i269 = 0; _i269 < _list267.size; ++_i269)
+          org.apache.thrift.protocol.TList _list257 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.partitions = new ArrayList<Partition>(_list257.size);
+          Partition _elem258;
+          for (int _i259 = 0; _i259 < _list257.size; ++_i259)
           {
-            _elem268 = new Partition();
-            _elem268.read(iprot);
-            struct.partitions.add(_elem268);
+            _elem258 = new Partition();
+            _elem258.read(iprot);
+            struct.partitions.add(_elem258);
           }
         }
         struct.setPartitionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpecWithSharedSD.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpecWithSharedSD.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpecWithSharedSD.java
index 71bd08b..e7ab52a 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpecWithSharedSD.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpecWithSharedSD.java
@@ -434,14 +434,14 @@ import org.slf4j.LoggerFactory;
           case 1: // PARTITIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list254 = iprot.readListBegin();
-                struct.partitions = new ArrayList<PartitionWithoutSD>(_list254.size);
-                PartitionWithoutSD _elem255;
-                for (int _i256 = 0; _i256 < _list254.size; ++_i256)
+                org.apache.thrift.protocol.TList _list244 = iprot.readListBegin();
+                struct.partitions = new ArrayList<PartitionWithoutSD>(_list244.size);
+                PartitionWithoutSD _elem245;
+                for (int _i246 = 0; _i246 < _list244.size; ++_i246)
                 {
-                  _elem255 = new PartitionWithoutSD();
-                  _elem255.read(iprot);
-                  struct.partitions.add(_elem255);
+                  _elem245 = new PartitionWithoutSD();
+                  _elem245.read(iprot);
+                  struct.partitions.add(_elem245);
                 }
                 iprot.readListEnd();
               }
@@ -476,9 +476,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARTITIONS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitions.size()));
-          for (PartitionWithoutSD _iter257 : struct.partitions)
+          for (PartitionWithoutSD _iter247 : struct.partitions)
           {
-            _iter257.write(oprot);
+            _iter247.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -517,9 +517,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitions()) {
         {
           oprot.writeI32(struct.partitions.size());
-          for (PartitionWithoutSD _iter258 : struct.partitions)
+          for (PartitionWithoutSD _iter248 : struct.partitions)
           {
-            _iter258.write(oprot);
+            _iter248.write(oprot);
           }
         }
       }
@@ -534,14 +534,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list259 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.partitions = new ArrayList<PartitionWithoutSD>(_list259.size);
-          PartitionWithoutSD _elem260;
-          for (int _i261 = 0; _i261 < _list259.size; ++_i261)
+          org.apache.thrift.protocol.TList _list249 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.partitions = new ArrayList<PartitionWithoutSD>(_list249.size);
+          PartitionWithoutSD _elem250;
+          for (int _i251 = 0; _i251 < _list249.size; ++_i251)
           {
-            _elem260 = new PartitionWithoutSD();
-            _elem260.read(iprot);
-            struct.partitions.add(_elem260);
+            _elem250 = new PartitionWithoutSD();
+            _elem250.read(iprot);
+            struct.partitions.add(_elem250);
           }
         }
         struct.setPartitionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRequest.java
index 9db256d..2283c24 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRequest.java
@@ -961,14 +961,14 @@ import org.slf4j.LoggerFactory;
           case 3: // PARTITION_KEYS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list494 = iprot.readListBegin();
-                struct.partitionKeys = new ArrayList<FieldSchema>(_list494.size);
-                FieldSchema _elem495;
-                for (int _i496 = 0; _i496 < _list494.size; ++_i496)
+                org.apache.thrift.protocol.TList _list484 = iprot.readListBegin();
+                struct.partitionKeys = new ArrayList<FieldSchema>(_list484.size);
+                FieldSchema _elem485;
+                for (int _i486 = 0; _i486 < _list484.size; ++_i486)
                 {
-                  _elem495 = new FieldSchema();
-                  _elem495.read(iprot);
-                  struct.partitionKeys.add(_elem495);
+                  _elem485 = new FieldSchema();
+                  _elem485.read(iprot);
+                  struct.partitionKeys.add(_elem485);
                 }
                 iprot.readListEnd();
               }
@@ -996,14 +996,14 @@ import org.slf4j.LoggerFactory;
           case 6: // PARTITION_ORDER
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list497 = iprot.readListBegin();
-                struct.partitionOrder = new ArrayList<FieldSchema>(_list497.size);
-                FieldSchema _elem498;
-                for (int _i499 = 0; _i499 < _list497.size; ++_i499)
+                org.apache.thrift.protocol.TList _list487 = iprot.readListBegin();
+                struct.partitionOrder = new ArrayList<FieldSchema>(_list487.size);
+                FieldSchema _elem488;
+                for (int _i489 = 0; _i489 < _list487.size; ++_i489)
                 {
-                  _elem498 = new FieldSchema();
-                  _elem498.read(iprot);
-                  struct.partitionOrder.add(_elem498);
+                  _elem488 = new FieldSchema();
+                  _elem488.read(iprot);
+                  struct.partitionOrder.add(_elem488);
                 }
                 iprot.readListEnd();
               }
@@ -1055,9 +1055,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARTITION_KEYS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitionKeys.size()));
-          for (FieldSchema _iter500 : struct.partitionKeys)
+          for (FieldSchema _iter490 : struct.partitionKeys)
           {
-            _iter500.write(oprot);
+            _iter490.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1080,9 +1080,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PARTITION_ORDER_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitionOrder.size()));
-            for (FieldSchema _iter501 : struct.partitionOrder)
+            for (FieldSchema _iter491 : struct.partitionOrder)
             {
-              _iter501.write(oprot);
+              _iter491.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -1120,9 +1120,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeString(struct.tblName);
       {
         oprot.writeI32(struct.partitionKeys.size());
-        for (FieldSchema _iter502 : struct.partitionKeys)
+        for (FieldSchema _iter492 : struct.partitionKeys)
         {
-          _iter502.write(oprot);
+          _iter492.write(oprot);
         }
       }
       BitSet optionals = new BitSet();
@@ -1151,9 +1151,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitionOrder()) {
         {
           oprot.writeI32(struct.partitionOrder.size());
-          for (FieldSchema _iter503 : struct.partitionOrder)
+          for (FieldSchema _iter493 : struct.partitionOrder)
           {
-            _iter503.write(oprot);
+            _iter493.write(oprot);
           }
         }
       }
@@ -1173,14 +1173,14 @@ import org.slf4j.LoggerFactory;
       struct.tblName = iprot.readString();
       struct.setTblNameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list504 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.partitionKeys = new ArrayList<FieldSchema>(_list504.size);
-        FieldSchema _elem505;
-        for (int _i506 = 0; _i506 < _list504.size; ++_i506)
+        org.apache.thrift.protocol.TList _list494 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.partitionKeys = new ArrayList<FieldSchema>(_list494.size);
+        FieldSchema _elem495;
+        for (int _i496 = 0; _i496 < _list494.size; ++_i496)
         {
-          _elem505 = new FieldSchema();
-          _elem505.read(iprot);
-          struct.partitionKeys.add(_elem505);
+          _elem495 = new FieldSchema();
+          _elem495.read(iprot);
+          struct.partitionKeys.add(_elem495);
         }
       }
       struct.setPartitionKeysIsSet(true);
@@ -1195,14 +1195,14 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list507 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.partitionOrder = new ArrayList<FieldSchema>(_list507.size);
-          FieldSchema _elem508;
-          for (int _i509 = 0; _i509 < _list507.size; ++_i509)
+          org.apache.thrift.protocol.TList _list497 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.partitionOrder = new ArrayList<FieldSchema>(_list497.size);
+          FieldSchema _elem498;
+          for (int _i499 = 0; _i499 < _list497.size; ++_i499)
           {
-            _elem508 = new FieldSchema();
-            _elem508.read(iprot);
-            struct.partitionOrder.add(_elem508);
+            _elem498 = new FieldSchema();
+            _elem498.read(iprot);
+            struct.partitionOrder.add(_elem498);
           }
         }
         struct.setPartitionOrderIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesResponse.java
index 2781180..f551156 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // PARTITION_VALUES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list518 = iprot.readListBegin();
-                struct.partitionValues = new ArrayList<PartitionValuesRow>(_list518.size);
-                PartitionValuesRow _elem519;
-                for (int _i520 = 0; _i520 < _list518.size; ++_i520)
+                org.apache.thrift.protocol.TList _list508 = iprot.readListBegin();
+                struct.partitionValues = new ArrayList<PartitionValuesRow>(_list508.size);
+                PartitionValuesRow _elem509;
+                for (int _i510 = 0; _i510 < _list508.size; ++_i510)
                 {
-                  _elem519 = new PartitionValuesRow();
-                  _elem519.read(iprot);
-                  struct.partitionValues.add(_elem519);
+                  _elem509 = new PartitionValuesRow();
+                  _elem509.read(iprot);
+                  struct.partitionValues.add(_elem509);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARTITION_VALUES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitionValues.size()));
-          for (PartitionValuesRow _iter521 : struct.partitionValues)
+          for (PartitionValuesRow _iter511 : struct.partitionValues)
           {
-            _iter521.write(oprot);
+            _iter511.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.partitionValues.size());
-        for (PartitionValuesRow _iter522 : struct.partitionValues)
+        for (PartitionValuesRow _iter512 : struct.partitionValues)
         {
-          _iter522.write(oprot);
+          _iter512.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, PartitionValuesResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list523 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.partitionValues = new ArrayList<PartitionValuesRow>(_list523.size);
-        PartitionValuesRow _elem524;
-        for (int _i525 = 0; _i525 < _list523.size; ++_i525)
+        org.apache.thrift.protocol.TList _list513 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.partitionValues = new ArrayList<PartitionValuesRow>(_list513.size);
+        PartitionValuesRow _elem514;
+        for (int _i515 = 0; _i515 < _list513.size; ++_i515)
         {
-          _elem524 = new PartitionValuesRow();
-          _elem524.read(iprot);
-          struct.partitionValues.add(_elem524);
+          _elem514 = new PartitionValuesRow();
+          _elem514.read(iprot);
+          struct.partitionValues.add(_elem514);
         }
       }
       struct.setPartitionValuesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRow.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRow.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRow.java
index 595f350..3f3c3b9 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRow.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRow.java
@@ -351,13 +351,13 @@ import org.slf4j.LoggerFactory;
           case 1: // ROW
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list510 = iprot.readListBegin();
-                struct.row = new ArrayList<String>(_list510.size);
-                String _elem511;
-                for (int _i512 = 0; _i512 < _list510.size; ++_i512)
+                org.apache.thrift.protocol.TList _list500 = iprot.readListBegin();
+                struct.row = new ArrayList<String>(_list500.size);
+                String _elem501;
+                for (int _i502 = 0; _i502 < _list500.size; ++_i502)
                 {
-                  _elem511 = iprot.readString();
-                  struct.row.add(_elem511);
+                  _elem501 = iprot.readString();
+                  struct.row.add(_elem501);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(ROW_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.row.size()));
-          for (String _iter513 : struct.row)
+          for (String _iter503 : struct.row)
           {
-            oprot.writeString(_iter513);
+            oprot.writeString(_iter503);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.row.size());
-        for (String _iter514 : struct.row)
+        for (String _iter504 : struct.row)
         {
-          oprot.writeString(_iter514);
+          oprot.writeString(_iter504);
         }
       }
     }
@@ -421,13 +421,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, PartitionValuesRow struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list515 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.row = new ArrayList<String>(_list515.size);
-        String _elem516;
-        for (int _i517 = 0; _i517 < _list515.size; ++_i517)
+        org.apache.thrift.protocol.TList _list505 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.row = new ArrayList<String>(_list505.size);
+        String _elem506;
+        for (int _i507 = 0; _i507 < _list505.size; ++_i507)
         {
-          _elem516 = iprot.readString();
-          struct.row.add(_elem516);
+          _elem506 = iprot.readString();
+          struct.row.add(_elem506);
         }
       }
       struct.setRowIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionWithoutSD.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionWithoutSD.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionWithoutSD.java
index 1f5d314..ba8a7ca 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionWithoutSD.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionWithoutSD.java
@@ -766,13 +766,13 @@ import org.slf4j.LoggerFactory;
           case 1: // VALUES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list236 = iprot.readListBegin();
-                struct.values = new ArrayList<String>(_list236.size);
-                String _elem237;
-                for (int _i238 = 0; _i238 < _list236.size; ++_i238)
+                org.apache.thrift.protocol.TList _list226 = iprot.readListBegin();
+                struct.values = new ArrayList<String>(_list226.size);
+                String _elem227;
+                for (int _i228 = 0; _i228 < _list226.size; ++_i228)
                 {
-                  _elem237 = iprot.readString();
-                  struct.values.add(_elem237);
+                  _elem227 = iprot.readString();
+                  struct.values.add(_elem227);
                 }
                 iprot.readListEnd();
               }
@@ -808,15 +808,15 @@ import org.slf4j.LoggerFactory;
           case 5: // PARAMETERS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map239 = iprot.readMapBegin();
-                struct.parameters = new HashMap<String,String>(2*_map239.size);
-                String _key240;
-                String _val241;
-                for (int _i242 = 0; _i242 < _map239.size; ++_i242)
+                org.apache.thrift.protocol.TMap _map229 = iprot.readMapBegin();
+                struct.parameters = new HashMap<String,String>(2*_map229.size);
+                String _key230;
+                String _val231;
+                for (int _i232 = 0; _i232 < _map229.size; ++_i232)
                 {
-                  _key240 = iprot.readString();
-                  _val241 = iprot.readString();
-                  struct.parameters.put(_key240, _val241);
+                  _key230 = iprot.readString();
+                  _val231 = iprot.readString();
+                  struct.parameters.put(_key230, _val231);
                 }
                 iprot.readMapEnd();
               }
@@ -851,9 +851,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(VALUES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.values.size()));
-          for (String _iter243 : struct.values)
+          for (String _iter233 : struct.values)
           {
-            oprot.writeString(_iter243);
+            oprot.writeString(_iter233);
           }
           oprot.writeListEnd();
         }
@@ -874,10 +874,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARAMETERS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.parameters.size()));
-          for (Map.Entry<String, String> _iter244 : struct.parameters.entrySet())
+          for (Map.Entry<String, String> _iter234 : struct.parameters.entrySet())
           {
-            oprot.writeString(_iter244.getKey());
-            oprot.writeString(_iter244.getValue());
+            oprot.writeString(_iter234.getKey());
+            oprot.writeString(_iter234.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -930,9 +930,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetValues()) {
         {
           oprot.writeI32(struct.values.size());
-          for (String _iter245 : struct.values)
+          for (String _iter235 : struct.values)
           {
-            oprot.writeString(_iter245);
+            oprot.writeString(_iter235);
           }
         }
       }
@@ -948,10 +948,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetParameters()) {
         {
           oprot.writeI32(struct.parameters.size());
-          for (Map.Entry<String, String> _iter246 : struct.parameters.entrySet())
+          for (Map.Entry<String, String> _iter236 : struct.parameters.entrySet())
           {
-            oprot.writeString(_iter246.getKey());
-            oprot.writeString(_iter246.getValue());
+            oprot.writeString(_iter236.getKey());
+            oprot.writeString(_iter236.getValue());
           }
         }
       }
@@ -966,13 +966,13 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(6);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list247 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.values = new ArrayList<String>(_list247.size);
-          String _elem248;
-          for (int _i249 = 0; _i249 < _list247.size; ++_i249)
+          org.apache.thrift.protocol.TList _list237 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.values = new ArrayList<String>(_list237.size);
+          String _elem238;
+          for (int _i239 = 0; _i239 < _list237.size; ++_i239)
           {
-            _elem248 = iprot.readString();
-            struct.values.add(_elem248);
+            _elem238 = iprot.readString();
+            struct.values.add(_elem238);
           }
         }
         struct.setValuesIsSet(true);
@@ -991,15 +991,15 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(4)) {
         {
-          org.apache.thrift.protocol.TMap _map250 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.parameters = new HashMap<String,String>(2*_map250.size);
-          String _key251;
-          String _val252;
-          for (int _i253 = 0; _i253 < _map250.size; ++_i253)
+          org.apache.thrift.protocol.TMap _map240 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.parameters = new HashMap<String,String>(2*_map240.size);
+          String _key241;
+          String _val242;
+          for (int _i243 = 0; _i243 < _map240.size; ++_i243)
           {
-            _key251 = iprot.readString();
-            _val252 = iprot.readString();
-            struct.parameters.put(_key251, _val252);
+            _key241 = iprot.readString();
+            _val242 = iprot.readString();
+            struct.parameters.put(_key241, _val242);
           }
         }
         struct.setParametersIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java
index 9be6b48..3ccf5ee 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java
@@ -439,14 +439,14 @@ import org.slf4j.LoggerFactory;
           case 1: // PARTITIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list396 = iprot.readListBegin();
-                struct.partitions = new ArrayList<Partition>(_list396.size);
-                Partition _elem397;
-                for (int _i398 = 0; _i398 < _list396.size; ++_i398)
+                org.apache.thrift.protocol.TList _list386 = iprot.readListBegin();
+                struct.partitions = new ArrayList<Partition>(_list386.size);
+                Partition _elem387;
+                for (int _i388 = 0; _i388 < _list386.size; ++_i388)
                 {
-                  _elem397 = new Partition();
-                  _elem397.read(iprot);
-                  struct.partitions.add(_elem397);
+                  _elem387 = new Partition();
+                  _elem387.read(iprot);
+                  struct.partitions.add(_elem387);
                 }
                 iprot.readListEnd();
               }
@@ -480,9 +480,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARTITIONS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitions.size()));
-          for (Partition _iter399 : struct.partitions)
+          for (Partition _iter389 : struct.partitions)
           {
-            _iter399.write(oprot);
+            _iter389.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -510,9 +510,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.partitions.size());
-        for (Partition _iter400 : struct.partitions)
+        for (Partition _iter390 : struct.partitions)
         {
-          _iter400.write(oprot);
+          _iter390.write(oprot);
         }
       }
       oprot.writeBool(struct.hasUnknownPartitions);
@@ -522,14 +522,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, PartitionsByExprResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list401 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.partitions = new ArrayList<Partition>(_list401.size);
-        Partition _elem402;
-        for (int _i403 = 0; _i403 < _list401.size; ++_i403)
+        org.apache.thrift.protocol.TList _list391 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.partitions = new ArrayList<Partition>(_list391.size);
+        Partition _elem392;
+        for (int _i393 = 0; _i393 < _list391.size; ++_i393)
         {
-          _elem402 = new Partition();
-          _elem402.read(iprot);
-          struct.partitions.add(_elem402);
+          _elem392 = new Partition();
+          _elem392.read(iprot);
+          struct.partitions.add(_elem392);
         }
       }
       struct.setPartitionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
index 80910f8..9941fa5 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
@@ -639,13 +639,13 @@ import org.slf4j.LoggerFactory;
           case 3: // COL_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list438 = iprot.readListBegin();
-                struct.colNames = new ArrayList<String>(_list438.size);
-                String _elem439;
-                for (int _i440 = 0; _i440 < _list438.size; ++_i440)
+                org.apache.thrift.protocol.TList _list428 = iprot.readListBegin();
+                struct.colNames = new ArrayList<String>(_list428.size);
+                String _elem429;
+                for (int _i430 = 0; _i430 < _list428.size; ++_i430)
                 {
-                  _elem439 = iprot.readString();
-                  struct.colNames.add(_elem439);
+                  _elem429 = iprot.readString();
+                  struct.colNames.add(_elem429);
                 }
                 iprot.readListEnd();
               }
@@ -657,13 +657,13 @@ import org.slf4j.LoggerFactory;
           case 4: // PART_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list441 = iprot.readListBegin();
-                struct.partNames = new ArrayList<String>(_list441.size);
-                String _elem442;
-                for (int _i443 = 0; _i443 < _list441.size; ++_i443)
+                org.apache.thrift.protocol.TList _list431 = iprot.readListBegin();
+                struct.partNames = new ArrayList<String>(_list431.size);
+                String _elem432;
+                for (int _i433 = 0; _i433 < _list431.size; ++_i433)
                 {
-                  _elem442 = iprot.readString();
-                  struct.partNames.add(_elem442);
+                  _elem432 = iprot.readString();
+                  struct.partNames.add(_elem432);
                 }
                 iprot.readListEnd();
               }
@@ -699,9 +699,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COL_NAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.colNames.size()));
-          for (String _iter444 : struct.colNames)
+          for (String _iter434 : struct.colNames)
           {
-            oprot.writeString(_iter444);
+            oprot.writeString(_iter434);
           }
           oprot.writeListEnd();
         }
@@ -711,9 +711,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PART_NAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partNames.size()));
-          for (String _iter445 : struct.partNames)
+          for (String _iter435 : struct.partNames)
           {
-            oprot.writeString(_iter445);
+            oprot.writeString(_iter435);
           }
           oprot.writeListEnd();
         }
@@ -740,16 +740,16 @@ import org.slf4j.LoggerFactory;
       oprot.writeString(struct.tblName);
       {
         oprot.writeI32(struct.colNames.size());
-        for (String _iter446 : struct.colNames)
+        for (String _iter436 : struct.colNames)
         {
-          oprot.writeString(_iter446);
+          oprot.writeString(_iter436);
         }
       }
       {
         oprot.writeI32(struct.partNames.size());
-        for (String _iter447 : struct.partNames)
+        for (String _iter437 : struct.partNames)
         {
-          oprot.writeString(_iter447);
+          oprot.writeString(_iter437);
         }
       }
     }
@@ -762,24 +762,24 @@ import org.slf4j.LoggerFactory;
       struct.tblName = iprot.readString();
       struct.setTblNameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list448 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.colNames = new ArrayList<String>(_list448.size);
-        String _elem449;
-        for (int _i450 = 0; _i450 < _list448.size; ++_i450)
+        org.apache.thrift.protocol.TList _list438 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.colNames = new ArrayList<String>(_list438.size);
+        String _elem439;
+        for (int _i440 = 0; _i440 < _list438.size; ++_i440)
         {
-          _elem449 = iprot.readString();
-          struct.colNames.add(_elem449);
+          _elem439 = iprot.readString();
+          struct.colNames.add(_elem439);
         }
       }
       struct.setColNamesIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list451 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.partNames = new ArrayList<String>(_list451.size);
-        String _elem452;
-        for (int _i453 = 0; _i453 < _list451.size; ++_i453)
+        org.apache.thrift.protocol.TList _list441 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.partNames = new ArrayList<String>(_list441.size);
+        String _elem442;
+        for (int _i443 = 0; _i443 < _list441.size; ++_i443)
         {
-          _elem452 = iprot.readString();
-          struct.partNames.add(_elem452);
+          _elem442 = iprot.readString();
+          struct.partNames.add(_elem442);
         }
       }
       struct.setPartNamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java
index 41408eb..8a0e5a5 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java
@@ -363,26 +363,26 @@ import org.slf4j.LoggerFactory;
           case 1: // PART_STATS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map412 = iprot.readMapBegin();
-                struct.partStats = new HashMap<String,List<ColumnStatisticsObj>>(2*_map412.size);
-                String _key413;
-                List<ColumnStatisticsObj> _val414;
-                for (int _i415 = 0; _i415 < _map412.size; ++_i415)
+                org.apache.thrift.protocol.TMap _map402 = iprot.readMapBegin();
+                struct.partStats = new HashMap<String,List<ColumnStatisticsObj>>(2*_map402.size);
+                String _key403;
+                List<ColumnStatisticsObj> _val404;
+                for (int _i405 = 0; _i405 < _map402.size; ++_i405)
                 {
-                  _key413 = iprot.readString();
+                  _key403 = iprot.readString();
                   {
-                    org.apache.thrift.protocol.TList _list416 = iprot.readListBegin();
-                    _val414 = new ArrayList<ColumnStatisticsObj>(_list416.size);
-                    ColumnStatisticsObj _elem417;
-                    for (int _i418 = 0; _i418 < _list416.size; ++_i418)
+                    org.apache.thrift.protocol.TList _list406 = iprot.readListBegin();
+                    _val404 = new ArrayList<ColumnStatisticsObj>(_list406.size);
+                    ColumnStatisticsObj _elem407;
+                    for (int _i408 = 0; _i408 < _list406.size; ++_i408)
                     {
-                      _elem417 = new ColumnStatisticsObj();
-                      _elem417.read(iprot);
-                      _val414.add(_elem417);
+                      _elem407 = new ColumnStatisticsObj();
+                      _elem407.read(iprot);
+                      _val404.add(_elem407);
                     }
                     iprot.readListEnd();
                   }
-                  struct.partStats.put(_key413, _val414);
+                  struct.partStats.put(_key403, _val404);
                 }
                 iprot.readMapEnd();
               }
@@ -408,14 +408,14 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PART_STATS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, struct.partStats.size()));
-          for (Map.Entry<String, List<ColumnStatisticsObj>> _iter419 : struct.partStats.entrySet())
+          for (Map.Entry<String, List<ColumnStatisticsObj>> _iter409 : struct.partStats.entrySet())
           {
-            oprot.writeString(_iter419.getKey());
+            oprot.writeString(_iter409.getKey());
             {
-              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter419.getValue().size()));
-              for (ColumnStatisticsObj _iter420 : _iter419.getValue())
+              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter409.getValue().size()));
+              for (ColumnStatisticsObj _iter410 : _iter409.getValue())
               {
-                _iter420.write(oprot);
+                _iter410.write(oprot);
               }
               oprot.writeListEnd();
             }
@@ -443,14 +443,14 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.partStats.size());
-        for (Map.Entry<String, List<ColumnStatisticsObj>> _iter421 : struct.partStats.entrySet())
+        for (Map.Entry<String, List<ColumnStatisticsObj>> _iter411 : struct.partStats.entrySet())
         {
-          oprot.writeString(_iter421.getKey());
+          oprot.writeString(_iter411.getKey());
           {
-            oprot.writeI32(_iter421.getValue().size());
-            for (ColumnStatisticsObj _iter422 : _iter421.getValue())
+            oprot.writeI32(_iter411.getValue().size());
+            for (ColumnStatisticsObj _iter412 : _iter411.getValue())
             {
-              _iter422.write(oprot);
+              _iter412.write(oprot);
             }
           }
         }
@@ -461,25 +461,25 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, PartitionsStatsResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map423 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
-        struct.partStats = new HashMap<String,List<ColumnStatisticsObj>>(2*_map423.size);
-        String _key424;
-        List<ColumnStatisticsObj> _val425;
-        for (int _i426 = 0; _i426 < _map423.size; ++_i426)
+        org.apache.thrift.protocol.TMap _map413 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
+        struct.partStats = new HashMap<String,List<ColumnStatisticsObj>>(2*_map413.size);
+        String _key414;
+        List<ColumnStatisticsObj> _val415;
+        for (int _i416 = 0; _i416 < _map413.size; ++_i416)
         {
-          _key424 = iprot.readString();
+          _key414 = iprot.readString();
           {
-            org.apache.thrift.protocol.TList _list427 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            _val425 = new ArrayList<ColumnStatisticsObj>(_list427.size);
-            ColumnStatisticsObj _elem428;
-            for (int _i429 = 0; _i429 < _list427.size; ++_i429)
+            org.apache.thrift.protocol.TList _list417 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            _val415 = new ArrayList<ColumnStatisticsObj>(_list417.size);
+            ColumnStatisticsObj _elem418;
+            for (int _i419 = 0; _i419 < _list417.size; ++_i419)
             {
-              _elem428 = new ColumnStatisticsObj();
-              _elem428.read(iprot);
-              _val425.add(_elem428);
+              _elem418 = new ColumnStatisticsObj();
+              _elem418.read(iprot);
+              _val415.add(_elem418);
             }
           }
-          struct.partStats.put(_key424, _val425);
+          struct.partStats.put(_key414, _val415);
         }
       }
       struct.setPartStatsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrimaryKeysResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrimaryKeysResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrimaryKeysResponse.java
index 8005270..43f070c 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrimaryKeysResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrimaryKeysResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // PRIMARY_KEYS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list332 = iprot.readListBegin();
-                struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list332.size);
-                SQLPrimaryKey _elem333;
-                for (int _i334 = 0; _i334 < _list332.size; ++_i334)
+                org.apache.thrift.protocol.TList _list322 = iprot.readListBegin();
+                struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list322.size);
+                SQLPrimaryKey _elem323;
+                for (int _i324 = 0; _i324 < _list322.size; ++_i324)
                 {
-                  _elem333 = new SQLPrimaryKey();
-                  _elem333.read(iprot);
-                  struct.primaryKeys.add(_elem333);
+                  _elem323 = new SQLPrimaryKey();
+                  _elem323.read(iprot);
+                  struct.primaryKeys.add(_elem323);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PRIMARY_KEYS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.primaryKeys.size()));
-          for (SQLPrimaryKey _iter335 : struct.primaryKeys)
+          for (SQLPrimaryKey _iter325 : struct.primaryKeys)
           {
-            _iter335.write(oprot);
+            _iter325.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.primaryKeys.size());
-        for (SQLPrimaryKey _iter336 : struct.primaryKeys)
+        for (SQLPrimaryKey _iter326 : struct.primaryKeys)
         {
-          _iter336.write(oprot);
+          _iter326.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, PrimaryKeysResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list337 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list337.size);
-        SQLPrimaryKey _elem338;
-        for (int _i339 = 0; _i339 < _list337.size; ++_i339)
+        org.apache.thrift.protocol.TList _list327 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list327.size);
+        SQLPrimaryKey _elem328;
+        for (int _i329 = 0; _i329 < _list327.size; ++_i329)
         {
-          _elem338 = new SQLPrimaryKey();
-          _elem338.read(iprot);
-          struct.primaryKeys.add(_elem338);
+          _elem328 = new SQLPrimaryKey();
+          _elem328.read(iprot);
+          struct.primaryKeys.add(_elem328);
         }
       }
       struct.setPrimaryKeysIsSet(true);


[04/16] hive git commit: HIVE-18546: Remove unnecessary code introduced in HIVE-14498 (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
index a8fcea6..9b2aaff 100755
--- a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
+++ b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
@@ -176,8 +176,6 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  CompactionResponse compact2(CompactionRequest rqst)')
   print('  ShowCompactResponse show_compact(ShowCompactRequest rqst)')
   print('  void add_dynamic_partitions(AddDynamicPartitions rqst)')
-  print('   get_last_completed_transaction_for_tables( db_names,  table_names, TxnsSnapshot txns_snapshot)')
-  print('  BasicTxnInfo get_last_completed_transaction_for_table(string db_name, string table_name, TxnsSnapshot txns_snapshot)')
   print('  NotificationEventResponse get_next_notification(NotificationEventRequest rqst)')
   print('  CurrentNotificationEventId get_current_notificationEventId()')
   print('  NotificationEventsCountResponse get_notification_events_count(NotificationEventsCountRequest rqst)')
@@ -1188,18 +1186,6 @@ elif cmd == 'add_dynamic_partitions':
     sys.exit(1)
   pp.pprint(client.add_dynamic_partitions(eval(args[0]),))
 
-elif cmd == 'get_last_completed_transaction_for_tables':
-  if len(args) != 3:
-    print('get_last_completed_transaction_for_tables requires 3 args')
-    sys.exit(1)
-  pp.pprint(client.get_last_completed_transaction_for_tables(eval(args[0]),eval(args[1]),eval(args[2]),))
-
-elif cmd == 'get_last_completed_transaction_for_table':
-  if len(args) != 3:
-    print('get_last_completed_transaction_for_table requires 3 args')
-    sys.exit(1)
-  pp.pprint(client.get_last_completed_transaction_for_table(args[0],args[1],eval(args[2]),))
-
 elif cmd == 'get_next_notification':
   if len(args) != 1:
     print('get_next_notification requires 1 args')


[07/16] hive git commit: HIVE-18546: Remove unnecessary code introduced in HIVE-14498 (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnsSnapshot.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnsSnapshot.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnsSnapshot.java
deleted file mode 100644
index 5600bda..0000000
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnsSnapshot.java
+++ /dev/null
@@ -1,537 +0,0 @@
-/**
- * Autogenerated by Thrift Compiler (0.9.3)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package org.apache.hadoop.hive.metastore.api;
-
-import org.apache.thrift.scheme.IScheme;
-import org.apache.thrift.scheme.SchemeFactory;
-import org.apache.thrift.scheme.StandardScheme;
-
-import org.apache.thrift.scheme.TupleScheme;
-import org.apache.thrift.protocol.TTupleProtocol;
-import org.apache.thrift.protocol.TProtocolException;
-import org.apache.thrift.EncodingUtils;
-import org.apache.thrift.TException;
-import org.apache.thrift.async.AsyncMethodCallback;
-import org.apache.thrift.server.AbstractNonblockingServer.*;
-import java.util.List;
-import java.util.ArrayList;
-import java.util.Map;
-import java.util.HashMap;
-import java.util.EnumMap;
-import java.util.Set;
-import java.util.HashSet;
-import java.util.EnumSet;
-import java.util.Collections;
-import java.util.BitSet;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import javax.annotation.Generated;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
-@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class TxnsSnapshot implements org.apache.thrift.TBase<TxnsSnapshot, TxnsSnapshot._Fields>, java.io.Serializable, Cloneable, Comparable<TxnsSnapshot> {
-  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TxnsSnapshot");
-
-  private static final org.apache.thrift.protocol.TField TXN_HIGH_WATER_MARK_FIELD_DESC = new org.apache.thrift.protocol.TField("txn_high_water_mark", org.apache.thrift.protocol.TType.I64, (short)1);
-  private static final org.apache.thrift.protocol.TField OPEN_TXNS_FIELD_DESC = new org.apache.thrift.protocol.TField("open_txns", org.apache.thrift.protocol.TType.LIST, (short)2);
-
-  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
-  static {
-    schemes.put(StandardScheme.class, new TxnsSnapshotStandardSchemeFactory());
-    schemes.put(TupleScheme.class, new TxnsSnapshotTupleSchemeFactory());
-  }
-
-  private long txn_high_water_mark; // required
-  private List<Long> open_txns; // required
-
-  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
-  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-    TXN_HIGH_WATER_MARK((short)1, "txn_high_water_mark"),
-    OPEN_TXNS((short)2, "open_txns");
-
-    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
-
-    static {
-      for (_Fields field : EnumSet.allOf(_Fields.class)) {
-        byName.put(field.getFieldName(), field);
-      }
-    }
-
-    /**
-     * Find the _Fields constant that matches fieldId, or null if its not found.
-     */
-    public static _Fields findByThriftId(int fieldId) {
-      switch(fieldId) {
-        case 1: // TXN_HIGH_WATER_MARK
-          return TXN_HIGH_WATER_MARK;
-        case 2: // OPEN_TXNS
-          return OPEN_TXNS;
-        default:
-          return null;
-      }
-    }
-
-    /**
-     * Find the _Fields constant that matches fieldId, throwing an exception
-     * if it is not found.
-     */
-    public static _Fields findByThriftIdOrThrow(int fieldId) {
-      _Fields fields = findByThriftId(fieldId);
-      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
-      return fields;
-    }
-
-    /**
-     * Find the _Fields constant that matches name, or null if its not found.
-     */
-    public static _Fields findByName(String name) {
-      return byName.get(name);
-    }
-
-    private final short _thriftId;
-    private final String _fieldName;
-
-    _Fields(short thriftId, String fieldName) {
-      _thriftId = thriftId;
-      _fieldName = fieldName;
-    }
-
-    public short getThriftFieldId() {
-      return _thriftId;
-    }
-
-    public String getFieldName() {
-      return _fieldName;
-    }
-  }
-
-  // isset id assignments
-  private static final int __TXN_HIGH_WATER_MARK_ISSET_ID = 0;
-  private byte __isset_bitfield = 0;
-  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-  static {
-    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-    tmpMap.put(_Fields.TXN_HIGH_WATER_MARK, new org.apache.thrift.meta_data.FieldMetaData("txn_high_water_mark", org.apache.thrift.TFieldRequirementType.REQUIRED, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
-    tmpMap.put(_Fields.OPEN_TXNS, new org.apache.thrift.meta_data.FieldMetaData("open_txns", org.apache.thrift.TFieldRequirementType.REQUIRED, 
-        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
-            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
-    metaDataMap = Collections.unmodifiableMap(tmpMap);
-    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TxnsSnapshot.class, metaDataMap);
-  }
-
-  public TxnsSnapshot() {
-  }
-
-  public TxnsSnapshot(
-    long txn_high_water_mark,
-    List<Long> open_txns)
-  {
-    this();
-    this.txn_high_water_mark = txn_high_water_mark;
-    setTxn_high_water_markIsSet(true);
-    this.open_txns = open_txns;
-  }
-
-  /**
-   * Performs a deep copy on <i>other</i>.
-   */
-  public TxnsSnapshot(TxnsSnapshot other) {
-    __isset_bitfield = other.__isset_bitfield;
-    this.txn_high_water_mark = other.txn_high_water_mark;
-    if (other.isSetOpen_txns()) {
-      List<Long> __this__open_txns = new ArrayList<Long>(other.open_txns);
-      this.open_txns = __this__open_txns;
-    }
-  }
-
-  public TxnsSnapshot deepCopy() {
-    return new TxnsSnapshot(this);
-  }
-
-  @Override
-  public void clear() {
-    setTxn_high_water_markIsSet(false);
-    this.txn_high_water_mark = 0;
-    this.open_txns = null;
-  }
-
-  public long getTxn_high_water_mark() {
-    return this.txn_high_water_mark;
-  }
-
-  public void setTxn_high_water_mark(long txn_high_water_mark) {
-    this.txn_high_water_mark = txn_high_water_mark;
-    setTxn_high_water_markIsSet(true);
-  }
-
-  public void unsetTxn_high_water_mark() {
-    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TXN_HIGH_WATER_MARK_ISSET_ID);
-  }
-
-  /** Returns true if field txn_high_water_mark is set (has been assigned a value) and false otherwise */
-  public boolean isSetTxn_high_water_mark() {
-    return EncodingUtils.testBit(__isset_bitfield, __TXN_HIGH_WATER_MARK_ISSET_ID);
-  }
-
-  public void setTxn_high_water_markIsSet(boolean value) {
-    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TXN_HIGH_WATER_MARK_ISSET_ID, value);
-  }
-
-  public int getOpen_txnsSize() {
-    return (this.open_txns == null) ? 0 : this.open_txns.size();
-  }
-
-  public java.util.Iterator<Long> getOpen_txnsIterator() {
-    return (this.open_txns == null) ? null : this.open_txns.iterator();
-  }
-
-  public void addToOpen_txns(long elem) {
-    if (this.open_txns == null) {
-      this.open_txns = new ArrayList<Long>();
-    }
-    this.open_txns.add(elem);
-  }
-
-  public List<Long> getOpen_txns() {
-    return this.open_txns;
-  }
-
-  public void setOpen_txns(List<Long> open_txns) {
-    this.open_txns = open_txns;
-  }
-
-  public void unsetOpen_txns() {
-    this.open_txns = null;
-  }
-
-  /** Returns true if field open_txns is set (has been assigned a value) and false otherwise */
-  public boolean isSetOpen_txns() {
-    return this.open_txns != null;
-  }
-
-  public void setOpen_txnsIsSet(boolean value) {
-    if (!value) {
-      this.open_txns = null;
-    }
-  }
-
-  public void setFieldValue(_Fields field, Object value) {
-    switch (field) {
-    case TXN_HIGH_WATER_MARK:
-      if (value == null) {
-        unsetTxn_high_water_mark();
-      } else {
-        setTxn_high_water_mark((Long)value);
-      }
-      break;
-
-    case OPEN_TXNS:
-      if (value == null) {
-        unsetOpen_txns();
-      } else {
-        setOpen_txns((List<Long>)value);
-      }
-      break;
-
-    }
-  }
-
-  public Object getFieldValue(_Fields field) {
-    switch (field) {
-    case TXN_HIGH_WATER_MARK:
-      return getTxn_high_water_mark();
-
-    case OPEN_TXNS:
-      return getOpen_txns();
-
-    }
-    throw new IllegalStateException();
-  }
-
-  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-  public boolean isSet(_Fields field) {
-    if (field == null) {
-      throw new IllegalArgumentException();
-    }
-
-    switch (field) {
-    case TXN_HIGH_WATER_MARK:
-      return isSetTxn_high_water_mark();
-    case OPEN_TXNS:
-      return isSetOpen_txns();
-    }
-    throw new IllegalStateException();
-  }
-
-  @Override
-  public boolean equals(Object that) {
-    if (that == null)
-      return false;
-    if (that instanceof TxnsSnapshot)
-      return this.equals((TxnsSnapshot)that);
-    return false;
-  }
-
-  public boolean equals(TxnsSnapshot that) {
-    if (that == null)
-      return false;
-
-    boolean this_present_txn_high_water_mark = true;
-    boolean that_present_txn_high_water_mark = true;
-    if (this_present_txn_high_water_mark || that_present_txn_high_water_mark) {
-      if (!(this_present_txn_high_water_mark && that_present_txn_high_water_mark))
-        return false;
-      if (this.txn_high_water_mark != that.txn_high_water_mark)
-        return false;
-    }
-
-    boolean this_present_open_txns = true && this.isSetOpen_txns();
-    boolean that_present_open_txns = true && that.isSetOpen_txns();
-    if (this_present_open_txns || that_present_open_txns) {
-      if (!(this_present_open_txns && that_present_open_txns))
-        return false;
-      if (!this.open_txns.equals(that.open_txns))
-        return false;
-    }
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    List<Object> list = new ArrayList<Object>();
-
-    boolean present_txn_high_water_mark = true;
-    list.add(present_txn_high_water_mark);
-    if (present_txn_high_water_mark)
-      list.add(txn_high_water_mark);
-
-    boolean present_open_txns = true && (isSetOpen_txns());
-    list.add(present_open_txns);
-    if (present_open_txns)
-      list.add(open_txns);
-
-    return list.hashCode();
-  }
-
-  @Override
-  public int compareTo(TxnsSnapshot other) {
-    if (!getClass().equals(other.getClass())) {
-      return getClass().getName().compareTo(other.getClass().getName());
-    }
-
-    int lastComparison = 0;
-
-    lastComparison = Boolean.valueOf(isSetTxn_high_water_mark()).compareTo(other.isSetTxn_high_water_mark());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetTxn_high_water_mark()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.txn_high_water_mark, other.txn_high_water_mark);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    lastComparison = Boolean.valueOf(isSetOpen_txns()).compareTo(other.isSetOpen_txns());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetOpen_txns()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.open_txns, other.open_txns);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    return 0;
-  }
-
-  public _Fields fieldForId(int fieldId) {
-    return _Fields.findByThriftId(fieldId);
-  }
-
-  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
-  }
-
-  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
-    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder("TxnsSnapshot(");
-    boolean first = true;
-
-    sb.append("txn_high_water_mark:");
-    sb.append(this.txn_high_water_mark);
-    first = false;
-    if (!first) sb.append(", ");
-    sb.append("open_txns:");
-    if (this.open_txns == null) {
-      sb.append("null");
-    } else {
-      sb.append(this.open_txns);
-    }
-    first = false;
-    sb.append(")");
-    return sb.toString();
-  }
-
-  public void validate() throws org.apache.thrift.TException {
-    // check for required fields
-    if (!isSetTxn_high_water_mark()) {
-      throw new org.apache.thrift.protocol.TProtocolException("Required field 'txn_high_water_mark' is unset! Struct:" + toString());
-    }
-
-    if (!isSetOpen_txns()) {
-      throw new org.apache.thrift.protocol.TProtocolException("Required field 'open_txns' is unset! Struct:" + toString());
-    }
-
-    // check for sub-struct validity
-  }
-
-  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
-    try {
-      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
-    } catch (org.apache.thrift.TException te) {
-      throw new java.io.IOException(te);
-    }
-  }
-
-  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
-    try {
-      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
-      __isset_bitfield = 0;
-      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
-    } catch (org.apache.thrift.TException te) {
-      throw new java.io.IOException(te);
-    }
-  }
-
-  private static class TxnsSnapshotStandardSchemeFactory implements SchemeFactory {
-    public TxnsSnapshotStandardScheme getScheme() {
-      return new TxnsSnapshotStandardScheme();
-    }
-  }
-
-  private static class TxnsSnapshotStandardScheme extends StandardScheme<TxnsSnapshot> {
-
-    public void read(org.apache.thrift.protocol.TProtocol iprot, TxnsSnapshot struct) throws org.apache.thrift.TException {
-      org.apache.thrift.protocol.TField schemeField;
-      iprot.readStructBegin();
-      while (true)
-      {
-        schemeField = iprot.readFieldBegin();
-        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-          break;
-        }
-        switch (schemeField.id) {
-          case 1: // TXN_HIGH_WATER_MARK
-            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
-              struct.txn_high_water_mark = iprot.readI64();
-              struct.setTxn_high_water_markIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          case 2: // OPEN_TXNS
-            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
-              {
-                org.apache.thrift.protocol.TList _list624 = iprot.readListBegin();
-                struct.open_txns = new ArrayList<Long>(_list624.size);
-                long _elem625;
-                for (int _i626 = 0; _i626 < _list624.size; ++_i626)
-                {
-                  _elem625 = iprot.readI64();
-                  struct.open_txns.add(_elem625);
-                }
-                iprot.readListEnd();
-              }
-              struct.setOpen_txnsIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          default:
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-        }
-        iprot.readFieldEnd();
-      }
-      iprot.readStructEnd();
-      struct.validate();
-    }
-
-    public void write(org.apache.thrift.protocol.TProtocol oprot, TxnsSnapshot struct) throws org.apache.thrift.TException {
-      struct.validate();
-
-      oprot.writeStructBegin(STRUCT_DESC);
-      oprot.writeFieldBegin(TXN_HIGH_WATER_MARK_FIELD_DESC);
-      oprot.writeI64(struct.txn_high_water_mark);
-      oprot.writeFieldEnd();
-      if (struct.open_txns != null) {
-        oprot.writeFieldBegin(OPEN_TXNS_FIELD_DESC);
-        {
-          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.open_txns.size()));
-          for (long _iter627 : struct.open_txns)
-          {
-            oprot.writeI64(_iter627);
-          }
-          oprot.writeListEnd();
-        }
-        oprot.writeFieldEnd();
-      }
-      oprot.writeFieldStop();
-      oprot.writeStructEnd();
-    }
-
-  }
-
-  private static class TxnsSnapshotTupleSchemeFactory implements SchemeFactory {
-    public TxnsSnapshotTupleScheme getScheme() {
-      return new TxnsSnapshotTupleScheme();
-    }
-  }
-
-  private static class TxnsSnapshotTupleScheme extends TupleScheme<TxnsSnapshot> {
-
-    @Override
-    public void write(org.apache.thrift.protocol.TProtocol prot, TxnsSnapshot struct) throws org.apache.thrift.TException {
-      TTupleProtocol oprot = (TTupleProtocol) prot;
-      oprot.writeI64(struct.txn_high_water_mark);
-      {
-        oprot.writeI32(struct.open_txns.size());
-        for (long _iter628 : struct.open_txns)
-        {
-          oprot.writeI64(_iter628);
-        }
-      }
-    }
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol prot, TxnsSnapshot struct) throws org.apache.thrift.TException {
-      TTupleProtocol iprot = (TTupleProtocol) prot;
-      struct.txn_high_water_mark = iprot.readI64();
-      struct.setTxn_high_water_markIsSet(true);
-      {
-        org.apache.thrift.protocol.TList _list629 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.open_txns = new ArrayList<Long>(_list629.size);
-        long _elem630;
-        for (int _i631 = 0; _i631 < _list629.size; ++_i631)
-        {
-          _elem630 = iprot.readI64();
-          struct.open_txns.add(_elem630);
-        }
-      }
-      struct.setOpen_txnsIsSet(true);
-    }
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UniqueConstraintsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UniqueConstraintsResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UniqueConstraintsResponse.java
index 342e6b0..d545e66 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UniqueConstraintsResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UniqueConstraintsResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // UNIQUE_CONSTRAINTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list348 = iprot.readListBegin();
-                struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list348.size);
-                SQLUniqueConstraint _elem349;
-                for (int _i350 = 0; _i350 < _list348.size; ++_i350)
+                org.apache.thrift.protocol.TList _list338 = iprot.readListBegin();
+                struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list338.size);
+                SQLUniqueConstraint _elem339;
+                for (int _i340 = 0; _i340 < _list338.size; ++_i340)
                 {
-                  _elem349 = new SQLUniqueConstraint();
-                  _elem349.read(iprot);
-                  struct.uniqueConstraints.add(_elem349);
+                  _elem339 = new SQLUniqueConstraint();
+                  _elem339.read(iprot);
+                  struct.uniqueConstraints.add(_elem339);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(UNIQUE_CONSTRAINTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.uniqueConstraints.size()));
-          for (SQLUniqueConstraint _iter351 : struct.uniqueConstraints)
+          for (SQLUniqueConstraint _iter341 : struct.uniqueConstraints)
           {
-            _iter351.write(oprot);
+            _iter341.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.uniqueConstraints.size());
-        for (SQLUniqueConstraint _iter352 : struct.uniqueConstraints)
+        for (SQLUniqueConstraint _iter342 : struct.uniqueConstraints)
         {
-          _iter352.write(oprot);
+          _iter342.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, UniqueConstraintsResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list353 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list353.size);
-        SQLUniqueConstraint _elem354;
-        for (int _i355 = 0; _i355 < _list353.size; ++_i355)
+        org.apache.thrift.protocol.TList _list343 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list343.size);
+        SQLUniqueConstraint _elem344;
+        for (int _i345 = 0; _i345 < _list343.size; ++_i345)
         {
-          _elem354 = new SQLUniqueConstraint();
-          _elem354.read(iprot);
-          struct.uniqueConstraints.add(_elem354);
+          _elem344 = new SQLUniqueConstraint();
+          _elem344.read(iprot);
+          struct.uniqueConstraints.add(_elem344);
         }
       }
       struct.setUniqueConstraintsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java
index c9988c0..01654c7 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java
@@ -755,14 +755,14 @@ import org.slf4j.LoggerFactory;
           case 2: // POOLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list764 = iprot.readListBegin();
-                struct.pools = new ArrayList<WMPool>(_list764.size);
-                WMPool _elem765;
-                for (int _i766 = 0; _i766 < _list764.size; ++_i766)
+                org.apache.thrift.protocol.TList _list754 = iprot.readListBegin();
+                struct.pools = new ArrayList<WMPool>(_list754.size);
+                WMPool _elem755;
+                for (int _i756 = 0; _i756 < _list754.size; ++_i756)
                 {
-                  _elem765 = new WMPool();
-                  _elem765.read(iprot);
-                  struct.pools.add(_elem765);
+                  _elem755 = new WMPool();
+                  _elem755.read(iprot);
+                  struct.pools.add(_elem755);
                 }
                 iprot.readListEnd();
               }
@@ -774,14 +774,14 @@ import org.slf4j.LoggerFactory;
           case 3: // MAPPINGS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list767 = iprot.readListBegin();
-                struct.mappings = new ArrayList<WMMapping>(_list767.size);
-                WMMapping _elem768;
-                for (int _i769 = 0; _i769 < _list767.size; ++_i769)
+                org.apache.thrift.protocol.TList _list757 = iprot.readListBegin();
+                struct.mappings = new ArrayList<WMMapping>(_list757.size);
+                WMMapping _elem758;
+                for (int _i759 = 0; _i759 < _list757.size; ++_i759)
                 {
-                  _elem768 = new WMMapping();
-                  _elem768.read(iprot);
-                  struct.mappings.add(_elem768);
+                  _elem758 = new WMMapping();
+                  _elem758.read(iprot);
+                  struct.mappings.add(_elem758);
                 }
                 iprot.readListEnd();
               }
@@ -793,14 +793,14 @@ import org.slf4j.LoggerFactory;
           case 4: // TRIGGERS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list770 = iprot.readListBegin();
-                struct.triggers = new ArrayList<WMTrigger>(_list770.size);
-                WMTrigger _elem771;
-                for (int _i772 = 0; _i772 < _list770.size; ++_i772)
+                org.apache.thrift.protocol.TList _list760 = iprot.readListBegin();
+                struct.triggers = new ArrayList<WMTrigger>(_list760.size);
+                WMTrigger _elem761;
+                for (int _i762 = 0; _i762 < _list760.size; ++_i762)
                 {
-                  _elem771 = new WMTrigger();
-                  _elem771.read(iprot);
-                  struct.triggers.add(_elem771);
+                  _elem761 = new WMTrigger();
+                  _elem761.read(iprot);
+                  struct.triggers.add(_elem761);
                 }
                 iprot.readListEnd();
               }
@@ -812,14 +812,14 @@ import org.slf4j.LoggerFactory;
           case 5: // POOL_TRIGGERS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list773 = iprot.readListBegin();
-                struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list773.size);
-                WMPoolTrigger _elem774;
-                for (int _i775 = 0; _i775 < _list773.size; ++_i775)
+                org.apache.thrift.protocol.TList _list763 = iprot.readListBegin();
+                struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list763.size);
+                WMPoolTrigger _elem764;
+                for (int _i765 = 0; _i765 < _list763.size; ++_i765)
                 {
-                  _elem774 = new WMPoolTrigger();
-                  _elem774.read(iprot);
-                  struct.poolTriggers.add(_elem774);
+                  _elem764 = new WMPoolTrigger();
+                  _elem764.read(iprot);
+                  struct.poolTriggers.add(_elem764);
                 }
                 iprot.readListEnd();
               }
@@ -850,9 +850,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(POOLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.pools.size()));
-          for (WMPool _iter776 : struct.pools)
+          for (WMPool _iter766 : struct.pools)
           {
-            _iter776.write(oprot);
+            _iter766.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -863,9 +863,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(MAPPINGS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.mappings.size()));
-            for (WMMapping _iter777 : struct.mappings)
+            for (WMMapping _iter767 : struct.mappings)
             {
-              _iter777.write(oprot);
+              _iter767.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -877,9 +877,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TRIGGERS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.triggers.size()));
-            for (WMTrigger _iter778 : struct.triggers)
+            for (WMTrigger _iter768 : struct.triggers)
             {
-              _iter778.write(oprot);
+              _iter768.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -891,9 +891,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(POOL_TRIGGERS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.poolTriggers.size()));
-            for (WMPoolTrigger _iter779 : struct.poolTriggers)
+            for (WMPoolTrigger _iter769 : struct.poolTriggers)
             {
-              _iter779.write(oprot);
+              _iter769.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -920,9 +920,9 @@ import org.slf4j.LoggerFactory;
       struct.plan.write(oprot);
       {
         oprot.writeI32(struct.pools.size());
-        for (WMPool _iter780 : struct.pools)
+        for (WMPool _iter770 : struct.pools)
         {
-          _iter780.write(oprot);
+          _iter770.write(oprot);
         }
       }
       BitSet optionals = new BitSet();
@@ -939,27 +939,27 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetMappings()) {
         {
           oprot.writeI32(struct.mappings.size());
-          for (WMMapping _iter781 : struct.mappings)
+          for (WMMapping _iter771 : struct.mappings)
           {
-            _iter781.write(oprot);
+            _iter771.write(oprot);
           }
         }
       }
       if (struct.isSetTriggers()) {
         {
           oprot.writeI32(struct.triggers.size());
-          for (WMTrigger _iter782 : struct.triggers)
+          for (WMTrigger _iter772 : struct.triggers)
           {
-            _iter782.write(oprot);
+            _iter772.write(oprot);
           }
         }
       }
       if (struct.isSetPoolTriggers()) {
         {
           oprot.writeI32(struct.poolTriggers.size());
-          for (WMPoolTrigger _iter783 : struct.poolTriggers)
+          for (WMPoolTrigger _iter773 : struct.poolTriggers)
           {
-            _iter783.write(oprot);
+            _iter773.write(oprot);
           }
         }
       }
@@ -972,56 +972,56 @@ import org.slf4j.LoggerFactory;
       struct.plan.read(iprot);
       struct.setPlanIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list784 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.pools = new ArrayList<WMPool>(_list784.size);
-        WMPool _elem785;
-        for (int _i786 = 0; _i786 < _list784.size; ++_i786)
+        org.apache.thrift.protocol.TList _list774 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.pools = new ArrayList<WMPool>(_list774.size);
+        WMPool _elem775;
+        for (int _i776 = 0; _i776 < _list774.size; ++_i776)
         {
-          _elem785 = new WMPool();
-          _elem785.read(iprot);
-          struct.pools.add(_elem785);
+          _elem775 = new WMPool();
+          _elem775.read(iprot);
+          struct.pools.add(_elem775);
         }
       }
       struct.setPoolsIsSet(true);
       BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list787 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.mappings = new ArrayList<WMMapping>(_list787.size);
-          WMMapping _elem788;
-          for (int _i789 = 0; _i789 < _list787.size; ++_i789)
+          org.apache.thrift.protocol.TList _list777 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.mappings = new ArrayList<WMMapping>(_list777.size);
+          WMMapping _elem778;
+          for (int _i779 = 0; _i779 < _list777.size; ++_i779)
           {
-            _elem788 = new WMMapping();
-            _elem788.read(iprot);
-            struct.mappings.add(_elem788);
+            _elem778 = new WMMapping();
+            _elem778.read(iprot);
+            struct.mappings.add(_elem778);
           }
         }
         struct.setMappingsIsSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list790 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.triggers = new ArrayList<WMTrigger>(_list790.size);
-          WMTrigger _elem791;
-          for (int _i792 = 0; _i792 < _list790.size; ++_i792)
+          org.apache.thrift.protocol.TList _list780 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.triggers = new ArrayList<WMTrigger>(_list780.size);
+          WMTrigger _elem781;
+          for (int _i782 = 0; _i782 < _list780.size; ++_i782)
           {
-            _elem791 = new WMTrigger();
-            _elem791.read(iprot);
-            struct.triggers.add(_elem791);
+            _elem781 = new WMTrigger();
+            _elem781.read(iprot);
+            struct.triggers.add(_elem781);
           }
         }
         struct.setTriggersIsSet(true);
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list793 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list793.size);
-          WMPoolTrigger _elem794;
-          for (int _i795 = 0; _i795 < _list793.size; ++_i795)
+          org.apache.thrift.protocol.TList _list783 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list783.size);
+          WMPoolTrigger _elem784;
+          for (int _i785 = 0; _i785 < _list783.size; ++_i785)
           {
-            _elem794 = new WMPoolTrigger();
-            _elem794.read(iprot);
-            struct.poolTriggers.add(_elem794);
+            _elem784 = new WMPoolTrigger();
+            _elem784.read(iprot);
+            struct.poolTriggers.add(_elem784);
           }
         }
         struct.setPoolTriggersIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java
index fb96ad9..69ccf53 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java
@@ -346,14 +346,14 @@ import org.slf4j.LoggerFactory;
           case 1: // RESOURCE_PLANS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list796 = iprot.readListBegin();
-                struct.resourcePlans = new ArrayList<WMResourcePlan>(_list796.size);
-                WMResourcePlan _elem797;
-                for (int _i798 = 0; _i798 < _list796.size; ++_i798)
+                org.apache.thrift.protocol.TList _list786 = iprot.readListBegin();
+                struct.resourcePlans = new ArrayList<WMResourcePlan>(_list786.size);
+                WMResourcePlan _elem787;
+                for (int _i788 = 0; _i788 < _list786.size; ++_i788)
                 {
-                  _elem797 = new WMResourcePlan();
-                  _elem797.read(iprot);
-                  struct.resourcePlans.add(_elem797);
+                  _elem787 = new WMResourcePlan();
+                  _elem787.read(iprot);
+                  struct.resourcePlans.add(_elem787);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(RESOURCE_PLANS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.resourcePlans.size()));
-            for (WMResourcePlan _iter799 : struct.resourcePlans)
+            for (WMResourcePlan _iter789 : struct.resourcePlans)
             {
-              _iter799.write(oprot);
+              _iter789.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetResourcePlans()) {
         {
           oprot.writeI32(struct.resourcePlans.size());
-          for (WMResourcePlan _iter800 : struct.resourcePlans)
+          for (WMResourcePlan _iter790 : struct.resourcePlans)
           {
-            _iter800.write(oprot);
+            _iter790.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list801 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.resourcePlans = new ArrayList<WMResourcePlan>(_list801.size);
-          WMResourcePlan _elem802;
-          for (int _i803 = 0; _i803 < _list801.size; ++_i803)
+          org.apache.thrift.protocol.TList _list791 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.resourcePlans = new ArrayList<WMResourcePlan>(_list791.size);
+          WMResourcePlan _elem792;
+          for (int _i793 = 0; _i793 < _list791.size; ++_i793)
           {
-            _elem802 = new WMResourcePlan();
-            _elem802.read(iprot);
-            struct.resourcePlans.add(_elem802);
+            _elem792 = new WMResourcePlan();
+            _elem792.read(iprot);
+            struct.resourcePlans.add(_elem792);
           }
         }
         struct.setResourcePlansIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
index 4d4894a..ee30063 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
@@ -346,14 +346,14 @@ import org.slf4j.LoggerFactory;
           case 1: // TRIGGERS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list820 = iprot.readListBegin();
-                struct.triggers = new ArrayList<WMTrigger>(_list820.size);
-                WMTrigger _elem821;
-                for (int _i822 = 0; _i822 < _list820.size; ++_i822)
+                org.apache.thrift.protocol.TList _list810 = iprot.readListBegin();
+                struct.triggers = new ArrayList<WMTrigger>(_list810.size);
+                WMTrigger _elem811;
+                for (int _i812 = 0; _i812 < _list810.size; ++_i812)
                 {
-                  _elem821 = new WMTrigger();
-                  _elem821.read(iprot);
-                  struct.triggers.add(_elem821);
+                  _elem811 = new WMTrigger();
+                  _elem811.read(iprot);
+                  struct.triggers.add(_elem811);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TRIGGERS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.triggers.size()));
-            for (WMTrigger _iter823 : struct.triggers)
+            for (WMTrigger _iter813 : struct.triggers)
             {
-              _iter823.write(oprot);
+              _iter813.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetTriggers()) {
         {
           oprot.writeI32(struct.triggers.size());
-          for (WMTrigger _iter824 : struct.triggers)
+          for (WMTrigger _iter814 : struct.triggers)
           {
-            _iter824.write(oprot);
+            _iter814.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list825 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.triggers = new ArrayList<WMTrigger>(_list825.size);
-          WMTrigger _elem826;
-          for (int _i827 = 0; _i827 < _list825.size; ++_i827)
+          org.apache.thrift.protocol.TList _list815 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.triggers = new ArrayList<WMTrigger>(_list815.size);
+          WMTrigger _elem816;
+          for (int _i817 = 0; _i817 < _list815.size; ++_i817)
           {
-            _elem826 = new WMTrigger();
-            _elem826.read(iprot);
-            struct.triggers.add(_elem826);
+            _elem816 = new WMTrigger();
+            _elem816.read(iprot);
+            struct.triggers.add(_elem816);
           }
         }
         struct.setTriggersIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
index ea8f3aa..5caf586 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
@@ -441,13 +441,13 @@ import org.slf4j.LoggerFactory;
           case 1: // ERRORS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list804 = iprot.readListBegin();
-                struct.errors = new ArrayList<String>(_list804.size);
-                String _elem805;
-                for (int _i806 = 0; _i806 < _list804.size; ++_i806)
+                org.apache.thrift.protocol.TList _list794 = iprot.readListBegin();
+                struct.errors = new ArrayList<String>(_list794.size);
+                String _elem795;
+                for (int _i796 = 0; _i796 < _list794.size; ++_i796)
                 {
-                  _elem805 = iprot.readString();
-                  struct.errors.add(_elem805);
+                  _elem795 = iprot.readString();
+                  struct.errors.add(_elem795);
                 }
                 iprot.readListEnd();
               }
@@ -459,13 +459,13 @@ import org.slf4j.LoggerFactory;
           case 2: // WARNINGS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list807 = iprot.readListBegin();
-                struct.warnings = new ArrayList<String>(_list807.size);
-                String _elem808;
-                for (int _i809 = 0; _i809 < _list807.size; ++_i809)
+                org.apache.thrift.protocol.TList _list797 = iprot.readListBegin();
+                struct.warnings = new ArrayList<String>(_list797.size);
+                String _elem798;
+                for (int _i799 = 0; _i799 < _list797.size; ++_i799)
                 {
-                  _elem808 = iprot.readString();
-                  struct.warnings.add(_elem808);
+                  _elem798 = iprot.readString();
+                  struct.warnings.add(_elem798);
                 }
                 iprot.readListEnd();
               }
@@ -492,9 +492,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(ERRORS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.errors.size()));
-            for (String _iter810 : struct.errors)
+            for (String _iter800 : struct.errors)
             {
-              oprot.writeString(_iter810);
+              oprot.writeString(_iter800);
             }
             oprot.writeListEnd();
           }
@@ -506,9 +506,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(WARNINGS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.warnings.size()));
-            for (String _iter811 : struct.warnings)
+            for (String _iter801 : struct.warnings)
             {
-              oprot.writeString(_iter811);
+              oprot.writeString(_iter801);
             }
             oprot.writeListEnd();
           }
@@ -543,18 +543,18 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetErrors()) {
         {
           oprot.writeI32(struct.errors.size());
-          for (String _iter812 : struct.errors)
+          for (String _iter802 : struct.errors)
           {
-            oprot.writeString(_iter812);
+            oprot.writeString(_iter802);
           }
         }
       }
       if (struct.isSetWarnings()) {
         {
           oprot.writeI32(struct.warnings.size());
-          for (String _iter813 : struct.warnings)
+          for (String _iter803 : struct.warnings)
           {
-            oprot.writeString(_iter813);
+            oprot.writeString(_iter803);
           }
         }
       }
@@ -566,26 +566,26 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list814 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.errors = new ArrayList<String>(_list814.size);
-          String _elem815;
-          for (int _i816 = 0; _i816 < _list814.size; ++_i816)
+          org.apache.thrift.protocol.TList _list804 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.errors = new ArrayList<String>(_list804.size);
+          String _elem805;
+          for (int _i806 = 0; _i806 < _list804.size; ++_i806)
           {
-            _elem815 = iprot.readString();
-            struct.errors.add(_elem815);
+            _elem805 = iprot.readString();
+            struct.errors.add(_elem805);
           }
         }
         struct.setErrorsIsSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list817 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.warnings = new ArrayList<String>(_list817.size);
-          String _elem818;
-          for (int _i819 = 0; _i819 < _list817.size; ++_i819)
+          org.apache.thrift.protocol.TList _list807 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.warnings = new ArrayList<String>(_list807.size);
+          String _elem808;
+          for (int _i809 = 0; _i809 < _list807.size; ++_i809)
           {
-            _elem818 = iprot.readString();
-            struct.warnings.add(_elem818);
+            _elem808 = iprot.readString();
+            struct.warnings.add(_elem808);
           }
         }
         struct.setWarningsIsSet(true);


[05/16] hive git commit: HIVE-18546: Remove unnecessary code introduced in HIVE-14498 (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php b/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
index 6878ee1..a5b578e 100644
--- a/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -5039,7 +5039,7 @@ class Table {
    */
   public $rewriteEnabled = null;
   /**
-   * @var array
+   * @var \metastore\CreationMetadata
    */
   public $creationMetadata = null;
 
@@ -5123,16 +5123,8 @@ class Table {
           ),
         16 => array(
           'var' => 'creationMetadata',
-          'type' => TType::MAP,
-          'ktype' => TType::STRING,
-          'vtype' => TType::STRUCT,
-          'key' => array(
-            'type' => TType::STRING,
-          ),
-          'val' => array(
-            'type' => TType::STRUCT,
-            'class' => '\metastore\BasicTxnInfo',
-            ),
+          'type' => TType::STRUCT,
+          'class' => '\metastore\CreationMetadata',
           ),
         );
     }
@@ -5339,22 +5331,9 @@ class Table {
           }
           break;
         case 16:
-          if ($ftype == TType::MAP) {
-            $this->creationMetadata = array();
-            $_size181 = 0;
-            $_ktype182 = 0;
-            $_vtype183 = 0;
-            $xfer += $input->readMapBegin($_ktype182, $_vtype183, $_size181);
-            for ($_i185 = 0; $_i185 < $_size181; ++$_i185)
-            {
-              $key186 = '';
-              $val187 = new \metastore\BasicTxnInfo();
-              $xfer += $input->readString($key186);
-              $val187 = new \metastore\BasicTxnInfo();
-              $xfer += $val187->read($input);
-              $this->creationMetadata[$key186] = $val187;
-            }
-            $xfer += $input->readMapEnd();
+          if ($ftype == TType::STRUCT) {
+            $this->creationMetadata = new \metastore\CreationMetadata();
+            $xfer += $this->creationMetadata->read($input);
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -5418,9 +5397,9 @@ class Table {
       {
         $output->writeListBegin(TType::STRUCT, count($this->partitionKeys));
         {
-          foreach ($this->partitionKeys as $iter188)
+          foreach ($this->partitionKeys as $iter181)
           {
-            $xfer += $iter188->write($output);
+            $xfer += $iter181->write($output);
           }
         }
         $output->writeListEnd();
@@ -5435,10 +5414,10 @@ class Table {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->parameters));
         {
-          foreach ($this->parameters as $kiter189 => $viter190)
+          foreach ($this->parameters as $kiter182 => $viter183)
           {
-            $xfer += $output->writeString($kiter189);
-            $xfer += $output->writeString($viter190);
+            $xfer += $output->writeString($kiter182);
+            $xfer += $output->writeString($viter183);
           }
         }
         $output->writeMapEnd();
@@ -5479,21 +5458,11 @@ class Table {
       $xfer += $output->writeFieldEnd();
     }
     if ($this->creationMetadata !== null) {
-      if (!is_array($this->creationMetadata)) {
+      if (!is_object($this->creationMetadata)) {
         throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
       }
-      $xfer += $output->writeFieldBegin('creationMetadata', TType::MAP, 16);
-      {
-        $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->creationMetadata));
-        {
-          foreach ($this->creationMetadata as $kiter191 => $viter192)
-          {
-            $xfer += $output->writeString($kiter191);
-            $xfer += $viter192->write($output);
-          }
-        }
-        $output->writeMapEnd();
-      }
+      $xfer += $output->writeFieldBegin('creationMetadata', TType::STRUCT, 16);
+      $xfer += $this->creationMetadata->write($output);
       $xfer += $output->writeFieldEnd();
     }
     $xfer += $output->writeFieldStop();
@@ -5640,14 +5609,14 @@ class Partition {
         case 1:
           if ($ftype == TType::LST) {
             $this->values = array();
-            $_size193 = 0;
-            $_etype196 = 0;
-            $xfer += $input->readListBegin($_etype196, $_size193);
-            for ($_i197 = 0; $_i197 < $_size193; ++$_i197)
+            $_size184 = 0;
+            $_etype187 = 0;
+            $xfer += $input->readListBegin($_etype187, $_size184);
+            for ($_i188 = 0; $_i188 < $_size184; ++$_i188)
             {
-              $elem198 = null;
-              $xfer += $input->readString($elem198);
-              $this->values []= $elem198;
+              $elem189 = null;
+              $xfer += $input->readString($elem189);
+              $this->values []= $elem189;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -5693,17 +5662,17 @@ class Partition {
         case 7:
           if ($ftype == TType::MAP) {
             $this->parameters = array();
-            $_size199 = 0;
-            $_ktype200 = 0;
-            $_vtype201 = 0;
-            $xfer += $input->readMapBegin($_ktype200, $_vtype201, $_size199);
-            for ($_i203 = 0; $_i203 < $_size199; ++$_i203)
+            $_size190 = 0;
+            $_ktype191 = 0;
+            $_vtype192 = 0;
+            $xfer += $input->readMapBegin($_ktype191, $_vtype192, $_size190);
+            for ($_i194 = 0; $_i194 < $_size190; ++$_i194)
             {
-              $key204 = '';
-              $val205 = '';
-              $xfer += $input->readString($key204);
-              $xfer += $input->readString($val205);
-              $this->parameters[$key204] = $val205;
+              $key195 = '';
+              $val196 = '';
+              $xfer += $input->readString($key195);
+              $xfer += $input->readString($val196);
+              $this->parameters[$key195] = $val196;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -5739,9 +5708,9 @@ class Partition {
       {
         $output->writeListBegin(TType::STRING, count($this->values));
         {
-          foreach ($this->values as $iter206)
+          foreach ($this->values as $iter197)
           {
-            $xfer += $output->writeString($iter206);
+            $xfer += $output->writeString($iter197);
           }
         }
         $output->writeListEnd();
@@ -5784,10 +5753,10 @@ class Partition {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->parameters));
         {
-          foreach ($this->parameters as $kiter207 => $viter208)
+          foreach ($this->parameters as $kiter198 => $viter199)
           {
-            $xfer += $output->writeString($kiter207);
-            $xfer += $output->writeString($viter208);
+            $xfer += $output->writeString($kiter198);
+            $xfer += $output->writeString($viter199);
           }
         }
         $output->writeMapEnd();
@@ -5923,14 +5892,14 @@ class PartitionWithoutSD {
         case 1:
           if ($ftype == TType::LST) {
             $this->values = array();
-            $_size209 = 0;
-            $_etype212 = 0;
-            $xfer += $input->readListBegin($_etype212, $_size209);
-            for ($_i213 = 0; $_i213 < $_size209; ++$_i213)
+            $_size200 = 0;
+            $_etype203 = 0;
+            $xfer += $input->readListBegin($_etype203, $_size200);
+            for ($_i204 = 0; $_i204 < $_size200; ++$_i204)
             {
-              $elem214 = null;
-              $xfer += $input->readString($elem214);
-              $this->values []= $elem214;
+              $elem205 = null;
+              $xfer += $input->readString($elem205);
+              $this->values []= $elem205;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -5961,17 +5930,17 @@ class PartitionWithoutSD {
         case 5:
           if ($ftype == TType::MAP) {
             $this->parameters = array();
-            $_size215 = 0;
-            $_ktype216 = 0;
-            $_vtype217 = 0;
-            $xfer += $input->readMapBegin($_ktype216, $_vtype217, $_size215);
-            for ($_i219 = 0; $_i219 < $_size215; ++$_i219)
+            $_size206 = 0;
+            $_ktype207 = 0;
+            $_vtype208 = 0;
+            $xfer += $input->readMapBegin($_ktype207, $_vtype208, $_size206);
+            for ($_i210 = 0; $_i210 < $_size206; ++$_i210)
             {
-              $key220 = '';
-              $val221 = '';
-              $xfer += $input->readString($key220);
-              $xfer += $input->readString($val221);
-              $this->parameters[$key220] = $val221;
+              $key211 = '';
+              $val212 = '';
+              $xfer += $input->readString($key211);
+              $xfer += $input->readString($val212);
+              $this->parameters[$key211] = $val212;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -6007,9 +5976,9 @@ class PartitionWithoutSD {
       {
         $output->writeListBegin(TType::STRING, count($this->values));
         {
-          foreach ($this->values as $iter222)
+          foreach ($this->values as $iter213)
           {
-            $xfer += $output->writeString($iter222);
+            $xfer += $output->writeString($iter213);
           }
         }
         $output->writeListEnd();
@@ -6039,10 +6008,10 @@ class PartitionWithoutSD {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->parameters));
         {
-          foreach ($this->parameters as $kiter223 => $viter224)
+          foreach ($this->parameters as $kiter214 => $viter215)
           {
-            $xfer += $output->writeString($kiter223);
-            $xfer += $output->writeString($viter224);
+            $xfer += $output->writeString($kiter214);
+            $xfer += $output->writeString($viter215);
           }
         }
         $output->writeMapEnd();
@@ -6127,15 +6096,15 @@ class PartitionSpecWithSharedSD {
         case 1:
           if ($ftype == TType::LST) {
             $this->partitions = array();
-            $_size225 = 0;
-            $_etype228 = 0;
-            $xfer += $input->readListBegin($_etype228, $_size225);
-            for ($_i229 = 0; $_i229 < $_size225; ++$_i229)
+            $_size216 = 0;
+            $_etype219 = 0;
+            $xfer += $input->readListBegin($_etype219, $_size216);
+            for ($_i220 = 0; $_i220 < $_size216; ++$_i220)
             {
-              $elem230 = null;
-              $elem230 = new \metastore\PartitionWithoutSD();
-              $xfer += $elem230->read($input);
-              $this->partitions []= $elem230;
+              $elem221 = null;
+              $elem221 = new \metastore\PartitionWithoutSD();
+              $xfer += $elem221->read($input);
+              $this->partitions []= $elem221;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -6171,9 +6140,9 @@ class PartitionSpecWithSharedSD {
       {
         $output->writeListBegin(TType::STRUCT, count($this->partitions));
         {
-          foreach ($this->partitions as $iter231)
+          foreach ($this->partitions as $iter222)
           {
-            $xfer += $iter231->write($output);
+            $xfer += $iter222->write($output);
           }
         }
         $output->writeListEnd();
@@ -6246,15 +6215,15 @@ class PartitionListComposingSpec {
         case 1:
           if ($ftype == TType::LST) {
             $this->partitions = array();
-            $_size232 = 0;
-            $_etype235 = 0;
-            $xfer += $input->readListBegin($_etype235, $_size232);
-            for ($_i236 = 0; $_i236 < $_size232; ++$_i236)
+            $_size223 = 0;
+            $_etype226 = 0;
+            $xfer += $input->readListBegin($_etype226, $_size223);
+            for ($_i227 = 0; $_i227 < $_size223; ++$_i227)
             {
-              $elem237 = null;
-              $elem237 = new \metastore\Partition();
-              $xfer += $elem237->read($input);
-              $this->partitions []= $elem237;
+              $elem228 = null;
+              $elem228 = new \metastore\Partition();
+              $xfer += $elem228->read($input);
+              $this->partitions []= $elem228;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -6282,9 +6251,9 @@ class PartitionListComposingSpec {
       {
         $output->writeListBegin(TType::STRUCT, count($this->partitions));
         {
-          foreach ($this->partitions as $iter238)
+          foreach ($this->partitions as $iter229)
           {
-            $xfer += $iter238->write($output);
+            $xfer += $iter229->write($output);
           }
         }
         $output->writeListEnd();
@@ -6686,17 +6655,17 @@ class Index {
         case 9:
           if ($ftype == TType::MAP) {
             $this->parameters = array();
-            $_size239 = 0;
-            $_ktype240 = 0;
-            $_vtype241 = 0;
-            $xfer += $input->readMapBegin($_ktype240, $_vtype241, $_size239);
-            for ($_i243 = 0; $_i243 < $_size239; ++$_i243)
+            $_size230 = 0;
+            $_ktype231 = 0;
+            $_vtype232 = 0;
+            $xfer += $input->readMapBegin($_ktype231, $_vtype232, $_size230);
+            for ($_i234 = 0; $_i234 < $_size230; ++$_i234)
             {
-              $key244 = '';
-              $val245 = '';
-              $xfer += $input->readString($key244);
-              $xfer += $input->readString($val245);
-              $this->parameters[$key244] = $val245;
+              $key235 = '';
+              $val236 = '';
+              $xfer += $input->readString($key235);
+              $xfer += $input->readString($val236);
+              $this->parameters[$key235] = $val236;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -6774,10 +6743,10 @@ class Index {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->parameters));
         {
-          foreach ($this->parameters as $kiter246 => $viter247)
+          foreach ($this->parameters as $kiter237 => $viter238)
           {
-            $xfer += $output->writeString($kiter246);
-            $xfer += $output->writeString($viter247);
+            $xfer += $output->writeString($kiter237);
+            $xfer += $output->writeString($viter238);
           }
         }
         $output->writeMapEnd();
@@ -8724,15 +8693,15 @@ class ColumnStatistics {
         case 2:
           if ($ftype == TType::LST) {
             $this->statsObj = array();
-            $_size248 = 0;
-            $_etype251 = 0;
-            $xfer += $input->readListBegin($_etype251, $_size248);
-            for ($_i252 = 0; $_i252 < $_size248; ++$_i252)
+            $_size239 = 0;
+            $_etype242 = 0;
+            $xfer += $input->readListBegin($_etype242, $_size239);
+            for ($_i243 = 0; $_i243 < $_size239; ++$_i243)
             {
-              $elem253 = null;
-              $elem253 = new \metastore\ColumnStatisticsObj();
-              $xfer += $elem253->read($input);
-              $this->statsObj []= $elem253;
+              $elem244 = null;
+              $elem244 = new \metastore\ColumnStatisticsObj();
+              $xfer += $elem244->read($input);
+              $this->statsObj []= $elem244;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -8768,9 +8737,9 @@ class ColumnStatistics {
       {
         $output->writeListBegin(TType::STRUCT, count($this->statsObj));
         {
-          foreach ($this->statsObj as $iter254)
+          foreach ($this->statsObj as $iter245)
           {
-            $xfer += $iter254->write($output);
+            $xfer += $iter245->write($output);
           }
         }
         $output->writeListEnd();
@@ -8846,15 +8815,15 @@ class AggrStats {
         case 1:
           if ($ftype == TType::LST) {
             $this->colStats = array();
-            $_size255 = 0;
-            $_etype258 = 0;
-            $xfer += $input->readListBegin($_etype258, $_size255);
-            for ($_i259 = 0; $_i259 < $_size255; ++$_i259)
+            $_size246 = 0;
+            $_etype249 = 0;
+            $xfer += $input->readListBegin($_etype249, $_size246);
+            for ($_i250 = 0; $_i250 < $_size246; ++$_i250)
             {
-              $elem260 = null;
-              $elem260 = new \metastore\ColumnStatisticsObj();
-              $xfer += $elem260->read($input);
-              $this->colStats []= $elem260;
+              $elem251 = null;
+              $elem251 = new \metastore\ColumnStatisticsObj();
+              $xfer += $elem251->read($input);
+              $this->colStats []= $elem251;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -8889,9 +8858,9 @@ class AggrStats {
       {
         $output->writeListBegin(TType::STRUCT, count($this->colStats));
         {
-          foreach ($this->colStats as $iter261)
+          foreach ($this->colStats as $iter252)
           {
-            $xfer += $iter261->write($output);
+            $xfer += $iter252->write($output);
           }
         }
         $output->writeListEnd();
@@ -8972,15 +8941,15 @@ class SetPartitionsStatsRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->colStats = array();
-            $_size262 = 0;
-            $_etype265 = 0;
-            $xfer += $input->readListBegin($_etype265, $_size262);
-            for ($_i266 = 0; $_i266 < $_size262; ++$_i266)
+            $_size253 = 0;
+            $_etype256 = 0;
+            $xfer += $input->readListBegin($_etype256, $_size253);
+            for ($_i257 = 0; $_i257 < $_size253; ++$_i257)
             {
-              $elem267 = null;
-              $elem267 = new \metastore\ColumnStatistics();
-              $xfer += $elem267->read($input);
-              $this->colStats []= $elem267;
+              $elem258 = null;
+              $elem258 = new \metastore\ColumnStatistics();
+              $xfer += $elem258->read($input);
+              $this->colStats []= $elem258;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -9015,9 +8984,9 @@ class SetPartitionsStatsRequest {
       {
         $output->writeListBegin(TType::STRUCT, count($this->colStats));
         {
-          foreach ($this->colStats as $iter268)
+          foreach ($this->colStats as $iter259)
           {
-            $xfer += $iter268->write($output);
+            $xfer += $iter259->write($output);
           }
         }
         $output->writeListEnd();
@@ -9106,15 +9075,15 @@ class Schema {
         case 1:
           if ($ftype == TType::LST) {
             $this->fieldSchemas = array();
-            $_size269 = 0;
-            $_etype272 = 0;
-            $xfer += $input->readListBegin($_etype272, $_size269);
-            for ($_i273 = 0; $_i273 < $_size269; ++$_i273)
+            $_size260 = 0;
+            $_etype263 = 0;
+            $xfer += $input->readListBegin($_etype263, $_size260);
+            for ($_i264 = 0; $_i264 < $_size260; ++$_i264)
             {
-              $elem274 = null;
-              $elem274 = new \metastore\FieldSchema();
-              $xfer += $elem274->read($input);
-              $this->fieldSchemas []= $elem274;
+              $elem265 = null;
+              $elem265 = new \metastore\FieldSchema();
+              $xfer += $elem265->read($input);
+              $this->fieldSchemas []= $elem265;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -9124,17 +9093,17 @@ class Schema {
         case 2:
           if ($ftype == TType::MAP) {
             $this->properties = array();
-            $_size275 = 0;
-            $_ktype276 = 0;
-            $_vtype277 = 0;
-            $xfer += $input->readMapBegin($_ktype276, $_vtype277, $_size275);
-            for ($_i279 = 0; $_i279 < $_size275; ++$_i279)
+            $_size266 = 0;
+            $_ktype267 = 0;
+            $_vtype268 = 0;
+            $xfer += $input->readMapBegin($_ktype267, $_vtype268, $_size266);
+            for ($_i270 = 0; $_i270 < $_size266; ++$_i270)
             {
-              $key280 = '';
-              $val281 = '';
-              $xfer += $input->readString($key280);
-              $xfer += $input->readString($val281);
-              $this->properties[$key280] = $val281;
+              $key271 = '';
+              $val272 = '';
+              $xfer += $input->readString($key271);
+              $xfer += $input->readString($val272);
+              $this->properties[$key271] = $val272;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -9162,9 +9131,9 @@ class Schema {
       {
         $output->writeListBegin(TType::STRUCT, count($this->fieldSchemas));
         {
-          foreach ($this->fieldSchemas as $iter282)
+          foreach ($this->fieldSchemas as $iter273)
           {
-            $xfer += $iter282->write($output);
+            $xfer += $iter273->write($output);
           }
         }
         $output->writeListEnd();
@@ -9179,10 +9148,10 @@ class Schema {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->properties));
         {
-          foreach ($this->properties as $kiter283 => $viter284)
+          foreach ($this->properties as $kiter274 => $viter275)
           {
-            $xfer += $output->writeString($kiter283);
-            $xfer += $output->writeString($viter284);
+            $xfer += $output->writeString($kiter274);
+            $xfer += $output->writeString($viter275);
           }
         }
         $output->writeMapEnd();
@@ -9250,17 +9219,17 @@ class EnvironmentContext {
         case 1:
           if ($ftype == TType::MAP) {
             $this->properties = array();
-            $_size285 = 0;
-            $_ktype286 = 0;
-            $_vtype287 = 0;
-            $xfer += $input->readMapBegin($_ktype286, $_vtype287, $_size285);
-            for ($_i289 = 0; $_i289 < $_size285; ++$_i289)
+            $_size276 = 0;
+            $_ktype277 = 0;
+            $_vtype278 = 0;
+            $xfer += $input->readMapBegin($_ktype277, $_vtype278, $_size276);
+            for ($_i280 = 0; $_i280 < $_size276; ++$_i280)
             {
-              $key290 = '';
-              $val291 = '';
-              $xfer += $input->readString($key290);
-              $xfer += $input->readString($val291);
-              $this->properties[$key290] = $val291;
+              $key281 = '';
+              $val282 = '';
+              $xfer += $input->readString($key281);
+              $xfer += $input->readString($val282);
+              $this->properties[$key281] = $val282;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -9288,10 +9257,10 @@ class EnvironmentContext {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->properties));
         {
-          foreach ($this->properties as $kiter292 => $viter293)
+          foreach ($this->properties as $kiter283 => $viter284)
           {
-            $xfer += $output->writeString($kiter292);
-            $xfer += $output->writeString($viter293);
+            $xfer += $output->writeString($kiter283);
+            $xfer += $output->writeString($viter284);
           }
         }
         $output->writeMapEnd();
@@ -9454,15 +9423,15 @@ class PrimaryKeysResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->primaryKeys = array();
-            $_size294 = 0;
-            $_etype297 = 0;
-            $xfer += $input->readListBegin($_etype297, $_size294);
-            for ($_i298 = 0; $_i298 < $_size294; ++$_i298)
+            $_size285 = 0;
+            $_etype288 = 0;
+            $xfer += $input->readListBegin($_etype288, $_size285);
+            for ($_i289 = 0; $_i289 < $_size285; ++$_i289)
             {
-              $elem299 = null;
-              $elem299 = new \metastore\SQLPrimaryKey();
-              $xfer += $elem299->read($input);
-              $this->primaryKeys []= $elem299;
+              $elem290 = null;
+              $elem290 = new \metastore\SQLPrimaryKey();
+              $xfer += $elem290->read($input);
+              $this->primaryKeys []= $elem290;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -9490,9 +9459,9 @@ class PrimaryKeysResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->primaryKeys));
         {
-          foreach ($this->primaryKeys as $iter300)
+          foreach ($this->primaryKeys as $iter291)
           {
-            $xfer += $iter300->write($output);
+            $xfer += $iter291->write($output);
           }
         }
         $output->writeListEnd();
@@ -9701,15 +9670,15 @@ class ForeignKeysResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->foreignKeys = array();
-            $_size301 = 0;
-            $_etype304 = 0;
-            $xfer += $input->readListBegin($_etype304, $_size301);
-            for ($_i305 = 0; $_i305 < $_size301; ++$_i305)
+            $_size292 = 0;
+            $_etype295 = 0;
+            $xfer += $input->readListBegin($_etype295, $_size292);
+            for ($_i296 = 0; $_i296 < $_size292; ++$_i296)
             {
-              $elem306 = null;
-              $elem306 = new \metastore\SQLForeignKey();
-              $xfer += $elem306->read($input);
-              $this->foreignKeys []= $elem306;
+              $elem297 = null;
+              $elem297 = new \metastore\SQLForeignKey();
+              $xfer += $elem297->read($input);
+              $this->foreignKeys []= $elem297;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -9737,9 +9706,9 @@ class ForeignKeysResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->foreignKeys));
         {
-          foreach ($this->foreignKeys as $iter307)
+          foreach ($this->foreignKeys as $iter298)
           {
-            $xfer += $iter307->write($output);
+            $xfer += $iter298->write($output);
           }
         }
         $output->writeListEnd();
@@ -9902,15 +9871,15 @@ class UniqueConstraintsResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->uniqueConstraints = array();
-            $_size308 = 0;
-            $_etype311 = 0;
-            $xfer += $input->readListBegin($_etype311, $_size308);
-            for ($_i312 = 0; $_i312 < $_size308; ++$_i312)
+            $_size299 = 0;
+            $_etype302 = 0;
+            $xfer += $input->readListBegin($_etype302, $_size299);
+            for ($_i303 = 0; $_i303 < $_size299; ++$_i303)
             {
-              $elem313 = null;
-              $elem313 = new \metastore\SQLUniqueConstraint();
-              $xfer += $elem313->read($input);
-              $this->uniqueConstraints []= $elem313;
+              $elem304 = null;
+              $elem304 = new \metastore\SQLUniqueConstraint();
+              $xfer += $elem304->read($input);
+              $this->uniqueConstraints []= $elem304;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -9938,9 +9907,9 @@ class UniqueConstraintsResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->uniqueConstraints));
         {
-          foreach ($this->uniqueConstraints as $iter314)
+          foreach ($this->uniqueConstraints as $iter305)
           {
-            $xfer += $iter314->write($output);
+            $xfer += $iter305->write($output);
           }
         }
         $output->writeListEnd();
@@ -10103,15 +10072,15 @@ class NotNullConstraintsResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->notNullConstraints = array();
-            $_size315 = 0;
-            $_etype318 = 0;
-            $xfer += $input->readListBegin($_etype318, $_size315);
-            for ($_i319 = 0; $_i319 < $_size315; ++$_i319)
+            $_size306 = 0;
+            $_etype309 = 0;
+            $xfer += $input->readListBegin($_etype309, $_size306);
+            for ($_i310 = 0; $_i310 < $_size306; ++$_i310)
             {
-              $elem320 = null;
-              $elem320 = new \metastore\SQLNotNullConstraint();
-              $xfer += $elem320->read($input);
-              $this->notNullConstraints []= $elem320;
+              $elem311 = null;
+              $elem311 = new \metastore\SQLNotNullConstraint();
+              $xfer += $elem311->read($input);
+              $this->notNullConstraints []= $elem311;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -10139,9 +10108,9 @@ class NotNullConstraintsResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->notNullConstraints));
         {
-          foreach ($this->notNullConstraints as $iter321)
+          foreach ($this->notNullConstraints as $iter312)
           {
-            $xfer += $iter321->write($output);
+            $xfer += $iter312->write($output);
           }
         }
         $output->writeListEnd();
@@ -10327,15 +10296,15 @@ class AddPrimaryKeyRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->primaryKeyCols = array();
-            $_size322 = 0;
-            $_etype325 = 0;
-            $xfer += $input->readListBegin($_etype325, $_size322);
-            for ($_i326 = 0; $_i326 < $_size322; ++$_i326)
+            $_size313 = 0;
+            $_etype316 = 0;
+            $xfer += $input->readListBegin($_etype316, $_size313);
+            for ($_i317 = 0; $_i317 < $_size313; ++$_i317)
             {
-              $elem327 = null;
-              $elem327 = new \metastore\SQLPrimaryKey();
-              $xfer += $elem327->read($input);
-              $this->primaryKeyCols []= $elem327;
+              $elem318 = null;
+              $elem318 = new \metastore\SQLPrimaryKey();
+              $xfer += $elem318->read($input);
+              $this->primaryKeyCols []= $elem318;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -10363,9 +10332,9 @@ class AddPrimaryKeyRequest {
       {
         $output->writeListBegin(TType::STRUCT, count($this->primaryKeyCols));
         {
-          foreach ($this->primaryKeyCols as $iter328)
+          foreach ($this->primaryKeyCols as $iter319)
           {
-            $xfer += $iter328->write($output);
+            $xfer += $iter319->write($output);
           }
         }
         $output->writeListEnd();
@@ -10430,15 +10399,15 @@ class AddForeignKeyRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->foreignKeyCols = array();
-            $_size329 = 0;
-            $_etype332 = 0;
-            $xfer += $input->readListBegin($_etype332, $_size329);
-            for ($_i333 = 0; $_i333 < $_size329; ++$_i333)
+            $_size320 = 0;
+            $_etype323 = 0;
+            $xfer += $input->readListBegin($_etype323, $_size320);
+            for ($_i324 = 0; $_i324 < $_size320; ++$_i324)
             {
-              $elem334 = null;
-              $elem334 = new \metastore\SQLForeignKey();
-              $xfer += $elem334->read($input);
-              $this->foreignKeyCols []= $elem334;
+              $elem325 = null;
+              $elem325 = new \metastore\SQLForeignKey();
+              $xfer += $elem325->read($input);
+              $this->foreignKeyCols []= $elem325;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -10466,9 +10435,9 @@ class AddForeignKeyRequest {
       {
         $output->writeListBegin(TType::STRUCT, count($this->foreignKeyCols));
         {
-          foreach ($this->foreignKeyCols as $iter335)
+          foreach ($this->foreignKeyCols as $iter326)
           {
-            $xfer += $iter335->write($output);
+            $xfer += $iter326->write($output);
           }
         }
         $output->writeListEnd();
@@ -10533,15 +10502,15 @@ class AddUniqueConstraintRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->uniqueConstraintCols = array();
-            $_size336 = 0;
-            $_etype339 = 0;
-            $xfer += $input->readListBegin($_etype339, $_size336);
-            for ($_i340 = 0; $_i340 < $_size336; ++$_i340)
+            $_size327 = 0;
+            $_etype330 = 0;
+            $xfer += $input->readListBegin($_etype330, $_size327);
+            for ($_i331 = 0; $_i331 < $_size327; ++$_i331)
             {
-              $elem341 = null;
-              $elem341 = new \metastore\SQLUniqueConstraint();
-              $xfer += $elem341->read($input);
-              $this->uniqueConstraintCols []= $elem341;
+              $elem332 = null;
+              $elem332 = new \metastore\SQLUniqueConstraint();
+              $xfer += $elem332->read($input);
+              $this->uniqueConstraintCols []= $elem332;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -10569,9 +10538,9 @@ class AddUniqueConstraintRequest {
       {
         $output->writeListBegin(TType::STRUCT, count($this->uniqueConstraintCols));
         {
-          foreach ($this->uniqueConstraintCols as $iter342)
+          foreach ($this->uniqueConstraintCols as $iter333)
           {
-            $xfer += $iter342->write($output);
+            $xfer += $iter333->write($output);
           }
         }
         $output->writeListEnd();
@@ -10636,15 +10605,15 @@ class AddNotNullConstraintRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->notNullConstraintCols = array();
-            $_size343 = 0;
-            $_etype346 = 0;
-            $xfer += $input->readListBegin($_etype346, $_size343);
-            for ($_i347 = 0; $_i347 < $_size343; ++$_i347)
+            $_size334 = 0;
+            $_etype337 = 0;
+            $xfer += $input->readListBegin($_etype337, $_size334);
+            for ($_i338 = 0; $_i338 < $_size334; ++$_i338)
             {
-              $elem348 = null;
-              $elem348 = new \metastore\SQLNotNullConstraint();
-              $xfer += $elem348->read($input);
-              $this->notNullConstraintCols []= $elem348;
+              $elem339 = null;
+              $elem339 = new \metastore\SQLNotNullConstraint();
+              $xfer += $elem339->read($input);
+              $this->notNullConstraintCols []= $elem339;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -10672,9 +10641,9 @@ class AddNotNullConstraintRequest {
       {
         $output->writeListBegin(TType::STRUCT, count($this->notNullConstraintCols));
         {
-          foreach ($this->notNullConstraintCols as $iter349)
+          foreach ($this->notNullConstraintCols as $iter340)
           {
-            $xfer += $iter349->write($output);
+            $xfer += $iter340->write($output);
           }
         }
         $output->writeListEnd();
@@ -10750,15 +10719,15 @@ class PartitionsByExprResult {
         case 1:
           if ($ftype == TType::LST) {
             $this->partitions = array();
-            $_size350 = 0;
-            $_etype353 = 0;
-            $xfer += $input->readListBegin($_etype353, $_size350);
-            for ($_i354 = 0; $_i354 < $_size350; ++$_i354)
+            $_size341 = 0;
+            $_etype344 = 0;
+            $xfer += $input->readListBegin($_etype344, $_size341);
+            for ($_i345 = 0; $_i345 < $_size341; ++$_i345)
             {
-              $elem355 = null;
-              $elem355 = new \metastore\Partition();
-              $xfer += $elem355->read($input);
-              $this->partitions []= $elem355;
+              $elem346 = null;
+              $elem346 = new \metastore\Partition();
+              $xfer += $elem346->read($input);
+              $this->partitions []= $elem346;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -10793,9 +10762,9 @@ class PartitionsByExprResult {
       {
         $output->writeListBegin(TType::STRUCT, count($this->partitions));
         {
-          foreach ($this->partitions as $iter356)
+          foreach ($this->partitions as $iter347)
           {
-            $xfer += $iter356->write($output);
+            $xfer += $iter347->write($output);
           }
         }
         $output->writeListEnd();
@@ -11032,15 +11001,15 @@ class TableStatsResult {
         case 1:
           if ($ftype == TType::LST) {
             $this->tableStats = array();
-            $_size357 = 0;
-            $_etype360 = 0;
-            $xfer += $input->readListBegin($_etype360, $_size357);
-            for ($_i361 = 0; $_i361 < $_size357; ++$_i361)
+            $_size348 = 0;
+            $_etype351 = 0;
+            $xfer += $input->readListBegin($_etype351, $_size348);
+            for ($_i352 = 0; $_i352 < $_size348; ++$_i352)
             {
-              $elem362 = null;
-              $elem362 = new \metastore\ColumnStatisticsObj();
-              $xfer += $elem362->read($input);
-              $this->tableStats []= $elem362;
+              $elem353 = null;
+              $elem353 = new \metastore\ColumnStatisticsObj();
+              $xfer += $elem353->read($input);
+              $this->tableStats []= $elem353;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11068,9 +11037,9 @@ class TableStatsResult {
       {
         $output->writeListBegin(TType::STRUCT, count($this->tableStats));
         {
-          foreach ($this->tableStats as $iter363)
+          foreach ($this->tableStats as $iter354)
           {
-            $xfer += $iter363->write($output);
+            $xfer += $iter354->write($output);
           }
         }
         $output->writeListEnd();
@@ -11143,28 +11112,28 @@ class PartitionsStatsResult {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partStats = array();
-            $_size364 = 0;
-            $_ktype365 = 0;
-            $_vtype366 = 0;
-            $xfer += $input->readMapBegin($_ktype365, $_vtype366, $_size364);
-            for ($_i368 = 0; $_i368 < $_size364; ++$_i368)
+            $_size355 = 0;
+            $_ktype356 = 0;
+            $_vtype357 = 0;
+            $xfer += $input->readMapBegin($_ktype356, $_vtype357, $_size355);
+            for ($_i359 = 0; $_i359 < $_size355; ++$_i359)
             {
-              $key369 = '';
-              $val370 = array();
-              $xfer += $input->readString($key369);
-              $val370 = array();
-              $_size371 = 0;
-              $_etype374 = 0;
-              $xfer += $input->readListBegin($_etype374, $_size371);
-              for ($_i375 = 0; $_i375 < $_size371; ++$_i375)
+              $key360 = '';
+              $val361 = array();
+              $xfer += $input->readString($key360);
+              $val361 = array();
+              $_size362 = 0;
+              $_etype365 = 0;
+              $xfer += $input->readListBegin($_etype365, $_size362);
+              for ($_i366 = 0; $_i366 < $_size362; ++$_i366)
               {
-                $elem376 = null;
-                $elem376 = new \metastore\ColumnStatisticsObj();
-                $xfer += $elem376->read($input);
-                $val370 []= $elem376;
+                $elem367 = null;
+                $elem367 = new \metastore\ColumnStatisticsObj();
+                $xfer += $elem367->read($input);
+                $val361 []= $elem367;
               }
               $xfer += $input->readListEnd();
-              $this->partStats[$key369] = $val370;
+              $this->partStats[$key360] = $val361;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -11192,15 +11161,15 @@ class PartitionsStatsResult {
       {
         $output->writeMapBegin(TType::STRING, TType::LST, count($this->partStats));
         {
-          foreach ($this->partStats as $kiter377 => $viter378)
+          foreach ($this->partStats as $kiter368 => $viter369)
           {
-            $xfer += $output->writeString($kiter377);
+            $xfer += $output->writeString($kiter368);
             {
-              $output->writeListBegin(TType::STRUCT, count($viter378));
+              $output->writeListBegin(TType::STRUCT, count($viter369));
               {
-                foreach ($viter378 as $iter379)
+                foreach ($viter369 as $iter370)
                 {
-                  $xfer += $iter379->write($output);
+                  $xfer += $iter370->write($output);
                 }
               }
               $output->writeListEnd();
@@ -11304,14 +11273,14 @@ class TableStatsRequest {
         case 3:
           if ($ftype == TType::LST) {
             $this->colNames = array();
-            $_size380 = 0;
-            $_etype383 = 0;
-            $xfer += $input->readListBegin($_etype383, $_size380);
-            for ($_i384 = 0; $_i384 < $_size380; ++$_i384)
+            $_size371 = 0;
+            $_etype374 = 0;
+            $xfer += $input->readListBegin($_etype374, $_size371);
+            for ($_i375 = 0; $_i375 < $_size371; ++$_i375)
             {
-              $elem385 = null;
-              $xfer += $input->readString($elem385);
-              $this->colNames []= $elem385;
+              $elem376 = null;
+              $xfer += $input->readString($elem376);
+              $this->colNames []= $elem376;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11349,9 +11318,9 @@ class TableStatsRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->colNames));
         {
-          foreach ($this->colNames as $iter386)
+          foreach ($this->colNames as $iter377)
           {
-            $xfer += $output->writeString($iter386);
+            $xfer += $output->writeString($iter377);
           }
         }
         $output->writeListEnd();
@@ -11466,14 +11435,14 @@ class PartitionsStatsRequest {
         case 3:
           if ($ftype == TType::LST) {
             $this->colNames = array();
-            $_size387 = 0;
-            $_etype390 = 0;
-            $xfer += $input->readListBegin($_etype390, $_size387);
-            for ($_i391 = 0; $_i391 < $_size387; ++$_i391)
+            $_size378 = 0;
+            $_etype381 = 0;
+            $xfer += $input->readListBegin($_etype381, $_size378);
+            for ($_i382 = 0; $_i382 < $_size378; ++$_i382)
             {
-              $elem392 = null;
-              $xfer += $input->readString($elem392);
-              $this->colNames []= $elem392;
+              $elem383 = null;
+              $xfer += $input->readString($elem383);
+              $this->colNames []= $elem383;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11483,14 +11452,14 @@ class PartitionsStatsRequest {
         case 4:
           if ($ftype == TType::LST) {
             $this->partNames = array();
-            $_size393 = 0;
-            $_etype396 = 0;
-            $xfer += $input->readListBegin($_etype396, $_size393);
-            for ($_i397 = 0; $_i397 < $_size393; ++$_i397)
+            $_size384 = 0;
+            $_etype387 = 0;
+            $xfer += $input->readListBegin($_etype387, $_size384);
+            for ($_i388 = 0; $_i388 < $_size384; ++$_i388)
             {
-              $elem398 = null;
-              $xfer += $input->readString($elem398);
-              $this->partNames []= $elem398;
+              $elem389 = null;
+              $xfer += $input->readString($elem389);
+              $this->partNames []= $elem389;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11528,9 +11497,9 @@ class PartitionsStatsRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->colNames));
         {
-          foreach ($this->colNames as $iter399)
+          foreach ($this->colNames as $iter390)
           {
-            $xfer += $output->writeString($iter399);
+            $xfer += $output->writeString($iter390);
           }
         }
         $output->writeListEnd();
@@ -11545,9 +11514,9 @@ class PartitionsStatsRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->partNames));
         {
-          foreach ($this->partNames as $iter400)
+          foreach ($this->partNames as $iter391)
           {
-            $xfer += $output->writeString($iter400);
+            $xfer += $output->writeString($iter391);
           }
         }
         $output->writeListEnd();
@@ -11612,15 +11581,15 @@ class AddPartitionsResult {
         case 1:
           if ($ftype == TType::LST) {
             $this->partitions = array();
-            $_size401 = 0;
-            $_etype404 = 0;
-            $xfer += $input->readListBegin($_etype404, $_size401);
-            for ($_i405 = 0; $_i405 < $_size401; ++$_i405)
+            $_size392 = 0;
+            $_etype395 = 0;
+            $xfer += $input->readListBegin($_etype395, $_size392);
+            for ($_i396 = 0; $_i396 < $_size392; ++$_i396)
             {
-              $elem406 = null;
-              $elem406 = new \metastore\Partition();
-              $xfer += $elem406->read($input);
-              $this->partitions []= $elem406;
+              $elem397 = null;
+              $elem397 = new \metastore\Partition();
+              $xfer += $elem397->read($input);
+              $this->partitions []= $elem397;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11648,9 +11617,9 @@ class AddPartitionsResult {
       {
         $output->writeListBegin(TType::STRUCT, count($this->partitions));
         {
-          foreach ($this->partitions as $iter407)
+          foreach ($this->partitions as $iter398)
           {
-            $xfer += $iter407->write($output);
+            $xfer += $iter398->write($output);
           }
         }
         $output->writeListEnd();
@@ -11773,15 +11742,15 @@ class AddPartitionsRequest {
         case 3:
           if ($ftype == TType::LST) {
             $this->parts = array();
-            $_size408 = 0;
-            $_etype411 = 0;
-            $xfer += $input->readListBegin($_etype411, $_size408);
-            for ($_i412 = 0; $_i412 < $_size408; ++$_i412)
+            $_size399 = 0;
+            $_etype402 = 0;
+            $xfer += $input->readListBegin($_etype402, $_size399);
+            for ($_i403 = 0; $_i403 < $_size399; ++$_i403)
             {
-              $elem413 = null;
-              $elem413 = new \metastore\Partition();
-              $xfer += $elem413->read($input);
-              $this->parts []= $elem413;
+              $elem404 = null;
+              $elem404 = new \metastore\Partition();
+              $xfer += $elem404->read($input);
+              $this->parts []= $elem404;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11833,9 +11802,9 @@ class AddPartitionsRequest {
       {
         $output->writeListBegin(TType::STRUCT, count($this->parts));
         {
-          foreach ($this->parts as $iter414)
+          foreach ($this->parts as $iter405)
           {
-            $xfer += $iter414->write($output);
+            $xfer += $iter405->write($output);
           }
         }
         $output->writeListEnd();
@@ -11910,15 +11879,15 @@ class DropPartitionsResult {
         case 1:
           if ($ftype == TType::LST) {
             $this->partitions = array();
-            $_size415 = 0;
-            $_etype418 = 0;
-            $xfer += $input->readListBegin($_etype418, $_size415);
-            for ($_i419 = 0; $_i419 < $_size415; ++$_i419)
+            $_size406 = 0;
+            $_etype409 = 0;
+            $xfer += $input->readListBegin($_etype409, $_size406);
+            for ($_i410 = 0; $_i410 < $_size406; ++$_i410)
             {
-              $elem420 = null;
-              $elem420 = new \metastore\Partition();
-              $xfer += $elem420->read($input);
-              $this->partitions []= $elem420;
+              $elem411 = null;
+              $elem411 = new \metastore\Partition();
+              $xfer += $elem411->read($input);
+              $this->partitions []= $elem411;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11946,9 +11915,9 @@ class DropPartitionsResult {
       {
         $output->writeListBegin(TType::STRUCT, count($this->partitions));
         {
-          foreach ($this->partitions as $iter421)
+          foreach ($this->partitions as $iter412)
           {
-            $xfer += $iter421->write($output);
+            $xfer += $iter412->write($output);
           }
         }
         $output->writeListEnd();
@@ -12126,14 +12095,14 @@ class RequestPartsSpec {
         case 1:
           if ($ftype == TType::LST) {
             $this->names = array();
-            $_size422 = 0;
-            $_etype425 = 0;
-            $xfer += $input->readListBegin($_etype425, $_size422);
-            for ($_i426 = 0; $_i426 < $_size422; ++$_i426)
+            $_size413 = 0;
+            $_etype416 = 0;
+            $xfer += $input->readListBegin($_etype416, $_size413);
+            for ($_i417 = 0; $_i417 < $_size413; ++$_i417)
             {
-              $elem427 = null;
-              $xfer += $input->readString($elem427);
-              $this->names []= $elem427;
+              $elem418 = null;
+              $xfer += $input->readString($elem418);
+              $this->names []= $elem418;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -12143,15 +12112,15 @@ class RequestPartsSpec {
         case 2:
           if ($ftype == TType::LST) {
             $this->exprs = array();
-            $_size428 = 0;
-            $_etype431 = 0;
-            $xfer += $input->readListBegin($_etype431, $_size428);
-            for ($_i432 = 0; $_i432 < $_size428; ++$_i432)
+            $_size419 = 0;
+            $_etype422 = 0;
+            $xfer += $input->readListBegin($_etype422, $_size419);
+            for ($_i423 = 0; $_i423 < $_size419; ++$_i423)
             {
-              $elem433 = null;
-              $elem433 = new \metastore\DropPartitionsExpr();
-              $xfer += $elem433->read($input);
-              $this->exprs []= $elem433;
+              $elem424 = null;
+              $elem424 = new \metastore\DropPartitionsExpr();
+              $xfer += $elem424->read($input);
+              $this->exprs []= $elem424;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -12179,9 +12148,9 @@ class RequestPartsSpec {
       {
         $output->writeListBegin(TType::STRING, count($this->names));
         {
-          foreach ($this->names as $iter434)
+          foreach ($this->names as $iter425)
           {
-            $xfer += $output->writeString($iter434);
+            $xfer += $output->writeString($iter425);
           }
         }
         $output->writeListEnd();
@@ -12196,9 +12165,9 @@ class RequestPartsSpec {
       {
         $output->writeListBegin(TType::STRUCT, count($this->exprs));
         {
-          foreach ($this->exprs as $iter435)
+          foreach ($this->exprs as $iter426)
           {
-            $xfer += $iter435->write($output);
+            $xfer += $iter426->write($output);
           }
         }
         $output->writeListEnd();
@@ -12605,15 +12574,15 @@ class PartitionValuesRequest {
         case 3:
           if ($ftype == TType::LST) {
             $this->partitionKeys = array();
-            $_size436 = 0;
-            $_etype439 = 0;
-            $xfer += $input->readListBegin($_etype439, $_size436);
-            for ($_i440 = 0; $_i440 < $_size436; ++$_i440)
+            $_size427 = 0;
+            $_etype430 = 0;
+            $xfer += $input->readListBegin($_etype430, $_size427);
+            for ($_i431 = 0; $_i431 < $_size427; ++$_i431)
             {
-              $elem441 = null;
-              $elem441 = new \metastore\FieldSchema();
-              $xfer += $elem441->read($input);
-              $this->partitionKeys []= $elem441;
+              $elem432 = null;
+              $elem432 = new \metastore\FieldSchema();
+              $xfer += $elem432->read($input);
+              $this->partitionKeys []= $elem432;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -12637,15 +12606,15 @@ class PartitionValuesRequest {
         case 6:
           if ($ftype == TType::LST) {
             $this->partitionOrder = array();
-            $_size442 = 0;
-            $_etype445 = 0;
-            $xfer += $input->readListBegin($_etype445, $_size442);
-            for ($_i446 = 0; $_i446 < $_size442; ++$_i446)
+            $_size433 = 0;
+            $_etype436 = 0;
+            $xfer += $input->readListBegin($_etype436, $_size433);
+            for ($_i437 = 0; $_i437 < $_size433; ++$_i437)
             {
-              $elem447 = null;
-              $elem447 = new \metastore\FieldSchema();
-              $xfer += $elem447->read($input);
-              $this->partitionOrder []= $elem447;
+              $elem438 = null;
+              $elem438 = new \metastore\FieldSchema();
+              $xfer += $elem438->read($input);
+              $this->partitionOrder []= $elem438;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -12697,9 +12666,9 @@ class PartitionValuesRequest {
       {
         $output->writeListBegin(TType::STRUCT, count($this->partitionKeys));
         {
-          foreach ($this->partitionKeys as $iter448)
+          foreach ($this->partitionKeys as $iter439)
           {
-            $xfer += $iter448->write($output);
+            $xfer += $iter439->write($output);
           }
         }
         $output->writeListEnd();
@@ -12724,9 +12693,9 @@ class PartitionValuesRequest {
       {
         $output->writeListBegin(TType::STRUCT, count($this->partitionOrder));
         {
-          foreach ($this->partitionOrder as $iter449)
+          foreach ($this->partitionOrder as $iter440)
           {
-            $xfer += $iter449->write($output);
+            $xfer += $iter440->write($output);
           }
         }
         $output->writeListEnd();
@@ -12800,14 +12769,14 @@ class PartitionValuesRow {
         case 1:
           if ($ftype == TType::LST) {
             $this->row = array();
-            $_size450 = 0;
-            $_etype453 = 0;
-            $xfer += $input->readListBegin($_etype453, $_size450);
-            for ($_i454 = 0; $_i454 < $_size450; ++$_i454)
+            $_size441 = 0;
+            $_etype444 = 0;
+            $xfer += $input->readListBegin($_etype444, $_size441);
+            for ($_i445 = 0; $_i445 < $_size441; ++$_i445)
             {
-              $elem455 = null;
-              $xfer += $input->readString($elem455);
-              $this->row []= $elem455;
+              $elem446 = null;
+              $xfer += $input->readString($elem446);
+              $this->row []= $elem446;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -12835,9 +12804,9 @@ class PartitionValuesRow {
       {
         $output->writeListBegin(TType::STRING, count($this->row));
         {
-          foreach ($this->row as $iter456)
+          foreach ($this->row as $iter447)
           {
-            $xfer += $output->writeString($iter456);
+            $xfer += $output->writeString($iter447);
           }
         }
         $output->writeListEnd();
@@ -12902,15 +12871,15 @@ class PartitionValuesResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->partitionValues = array();
-            $_size457 = 0;
-            $_etype460 = 0;
-            $xfer += $input->readListBegin($_etype460, $_size457);
-            for ($_i461 = 0; $_i461 < $_size457; ++$_i461)
+            $_size448 = 0;
+            $_etype451 = 0;
+            $xfer += $input->readListBegin($_etype451, $_size448);
+            for ($_i452 = 0; $_i452 < $_size448; ++$_i452)
             {
-              $elem462 = null;
-              $elem462 = new \metastore\PartitionValuesRow();
-              $xfer += $elem462->read($input);
-              $this->partitionValues []= $elem462;
+              $elem453 = null;
+              $elem453 = new \metastore\PartitionValuesRow();
+              $xfer += $elem453->read($input);
+              $this->partitionValues []= $elem453;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -12938,9 +12907,9 @@ class PartitionValuesResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->partitionValues));
         {
-          foreach ($this->partitionValues as $iter463)
+          foreach ($this->partitionValues as $iter454)
           {
-            $xfer += $iter463->write($output);
+            $xfer += $iter454->write($output);
           }
         }
         $output->writeListEnd();
@@ -13229,15 +13198,15 @@ class Function {
         case 8:
           if ($ftype == TType::LST) {
             $this->resourceUris = array();
-            $_size464 = 0;
-            $_etype467 = 0;
-            $xfer += $input->readListBegin($_etype467, $_size464);
-            for ($_i468 = 0; $_i468 < $_size464; ++$_i468)
+            $_size455 = 0;
+            $_etype458 = 0;
+            $xfer += $input->readListBegin($_etype458, $_size455);
+            for ($_i459 = 0; $_i459 < $_size455; ++$_i459)
             {
-              $elem469 = null;
-              $elem469 = new \metastore\ResourceUri();
-              $xfer += $elem469->read($input);
-              $this->resourceUris []= $elem469;
+              $elem460 = null;
+              $elem460 = new \metastore\ResourceUri();
+              $xfer += $elem460->read($input);
+              $this->resourceUris []= $elem460;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13300,9 +13269,9 @@ class Function {
       {
         $output->writeListBegin(TType::STRUCT, count($this->resourceUris));
         {
-          foreach ($this->resourceUris as $iter470)
+          foreach ($this->resourceUris as $iter461)
           {
-            $xfer += $iter470->write($output);
+            $xfer += $iter461->write($output);
           }
         }
         $output->writeListEnd();
@@ -13644,15 +13613,15 @@ class GetOpenTxnsInfoResponse {
         case 2:
           if ($ftype == TType::LST) {
             $this->open_txns = array();
-            $_size471 = 0;
-            $_etype474 = 0;
-            $xfer += $input->readListBegin($_etype474, $_size471);
-            for ($_i475 = 0; $_i475 < $_size471; ++$_i475)
+            $_size462 = 0;
+            $_etype465 = 0;
+            $xfer += $input->readListBegin($_etype465, $_size462);
+            for ($_i466 = 0; $_i466 < $_size462; ++$_i466)
             {
-              $elem476 = null;
-              $elem476 = new \metastore\TxnInfo();
-              $xfer += $elem476->read($input);
-              $this->open_txns []= $elem476;
+              $elem467 = null;
+              $elem467 = new \metastore\TxnInfo();
+              $xfer += $elem467->read($input);
+              $this->open_txns []= $elem467;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13685,9 +13654,9 @@ class GetOpenTxnsInfoResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->open_txns));
         {
-          foreach ($this->open_txns as $iter477)
+          foreach ($this->open_txns as $iter468)
           {
-            $xfer += $iter477->write($output);
+            $xfer += $iter468->write($output);
           }
         }
         $output->writeListEnd();
@@ -13791,14 +13760,14 @@ class GetOpenTxnsResponse {
         case 2:
           if ($ftype == TType::LST) {
             $this->open_txns = array();
-            $_size478 = 0;
-            $_etype481 = 0;
-            $xfer += $input->readListBegin($_etype481, $_size478);
-            for ($_i482 = 0; $_i482 < $_size478; ++$_i482)
+            $_size469 = 0;
+            $_etype472 = 0;
+            $xfer += $input->readListBegin($_etype472, $_size469);
+            for ($_i473 = 0; $_i473 < $_size469; ++$_i473)
             {
-              $elem483 = null;
-              $xfer += $input->readI64($elem483);
-              $this->open_txns []= $elem483;
+              $elem474 = null;
+              $xfer += $input->readI64($elem474);
+              $this->open_txns []= $elem474;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13845,9 +13814,9 @@ class GetOpenTxnsResponse {
       {
         $output->writeListBegin(TType::I64, count($this->open_txns));
         {
-          foreach ($this->open_txns as $iter484)
+          foreach ($this->open_txns as $iter475)
           {
-            $xfer += $output->writeI64($iter484);
+            $xfer += $output->writeI64($iter475);
           }
         }
         $output->writeListEnd();
@@ -14065,14 +14034,14 @@ class OpenTxnsResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->txn_ids = array();
-            $_size485 = 0;
-            $_etype488 = 0;
-            $xfer += $input->readListBegin($_etype488, $_size485);
-            for ($_i489 = 0; $_i489 < $_size485; ++$_i489)
+            $_size476 = 0;
+            $_etype479 = 0;
+            $xfer += $input->readListBegin($_etype479, $_size476);
+            for ($_i480 = 0; $_i480 < $_size476; ++$_i480)
             {
-              $elem490 = null;
-              $xfer += $input->readI64($elem490);
-              $this->txn_ids []= $elem490;
+              $elem481 = null;
+              $xfer += $input->readI64($elem481);
+              $this->txn_ids []= $elem481;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14100,9 +14069,9 @@ class OpenTxnsResponse {
       {
         $output->writeListBegin(TType::I64, count($this->txn_ids));
         {
-          foreach ($this->txn_ids as $iter491)
+          foreach ($this->txn_ids as $iter482)
           {
-            $xfer += $output->writeI64($iter491);
+            $xfer += $output->writeI64($iter482);
           }
         }
         $output->writeListEnd();
@@ -14241,14 +14210,14 @@ class AbortTxnsRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->txn_ids = array();
-            $_size492 = 0;
-            $_etype495 = 0;
-            $xfer += $input->readListBegin($_etype495, $_size492);
-            for ($_i496 = 0; $_i496 < $_size492; ++$_i496)
+            $_size483 = 0;
+            $_etype486 = 0;
+            $xfer += $input->readListBegin($_etype486, $_size483);
+            for ($_i487 = 0; $_i487 < $_size483; ++$_i487)
             {
-              $elem497 = null;
-              $xfer += $input->readI64($elem497);
-              $this->txn_ids []= $elem497;
+              $elem488 = null;
+              $xfer += $input->readI64($elem488);
+              $this->txn_ids []= $elem488;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14276,9 +14245,9 @@ class AbortTxnsRequest {
       {
         $output->writeListBegin(TType::I64, count($this->txn_ids));
         {
-          foreach ($this->txn_ids as $iter498)
+          foreach ($this->txn_ids as $iter489)
           {
-            $xfer += $output->writeI64($iter498);
+            $xfer += $output->writeI64($iter489);
           }
         }
         $output->writeListEnd();
@@ -14698,15 +14667,15 @@ class LockRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->component = array();
-            $_size499 = 0;
-            $_etype502 = 0;
-            $xfer += $input->readListBegin($_etype502, $_size499);
-            for ($_i503 = 0; $_i503 < $_size499; ++$_i503)
+            $_size490 = 0;
+            $_etype493 = 0;
+            $xfer += $input->readListBegin($_etype493, $_size490);
+            for ($_i494 = 0; $_i494 < $_size490; ++$_i494)
             {
-              $elem504 = null;
-              $elem504 = new \metastore\LockComponent();
-              $xfer += $elem504->read($input);
-              $this->component []= $elem504;
+              $elem495 = null;
+              $elem495 = new \metastore\LockComponent();
+              $xfer += $elem495->read($input);
+              $this->component []= $elem495;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14762,9 +14731,9 @@ class LockRequest {
       {
         $output->writeListBegin(TType::STRUCT, count($this->component));
         {
-          foreach ($this->component as $iter505)
+          foreach ($this->component as $iter496)
           {
-            $xfer += $iter505->write($output);
+            $xfer += $iter496->write($output);
           }
         }
         $output->writeListEnd();
@@ -15707,15 +15676,15 @@ class ShowLocksResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->locks = array();
-            $_size506 = 0;
-            $_etype509 = 0;
-            $xfer += $input->readListBegin($_etype509, $_size506);
-            for ($_i510 = 0; $_i510 < $_size506; ++$_i510)
+            $_size497 = 0;
+            $_etype500 = 0;
+            $xfer += $input->readListBegin($_etype500, $_size497);
+            for ($_i501 = 0; $_i501 < $_size497; ++$_i501)
             {
-              $elem511 = null;
-              $elem511 = new \metastore\ShowLocksResponseElement();
-              $xfer += $elem511->read($input);
-              $this->locks []= $elem511;
+              $elem502 = null;
+              $elem502 = new \metastore\ShowLocksResponseElement();
+              $xfer += $elem502->read($input);
+              $this->locks []= $elem502;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15743,9 +15712,9 @@ class ShowLocksResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->locks));
         {
-          foreach ($this->locks as $iter512)
+          foreach ($this->locks as $iter503)
           {
-            $xfer += $iter512->write($output);
+            $xfer += $iter503->write($output);
           }
         }
         $output->writeListEnd();
@@ -16020,17 +15989,17 @@ class HeartbeatTxnRangeResponse {
         case 1:
           if ($ftype == TType::SET) {
             $this->aborted = array();
-            $_size513 = 0;
-            $_etype516 = 0;
-            $xfer += $input->readSetBegin($_etype516, $_size513);
-            for ($_i517 = 0; $_i517 < $_size513; ++$_i517)
+            $_size504 = 0;
+            $_etype507 = 0;
+            $xfer += $input->readSetBegin($_etype507, $_size504);
+            for ($_i508 = 0; $_i508 < $_size504; ++$_i508)
             {
-              $elem518 = null;
-              $xfer += $input->readI64($elem518);
-              if (is_scalar($elem518)) {
-                $this->aborted[$elem518] = true;
+              $elem509 = null;
+              $xfer += $input->readI64($elem509);
+              if (is_scalar($elem509)) {
+                $this->aborted[$elem509] = true;
               } else {
-                $this->aborted []= $elem518;
+                $this->aborted []= $elem509;
               }
             }
             $xfer += $input->readSetEnd();
@@ -16041,17 +16010,17 @@ class HeartbeatTxnRangeResponse {
         case 2:
           if ($ftype == TType::SET) {
             $this->nosuch = array();
-            $_size519 = 0;
-            $_etype522 = 0;
-            $xfer += $input->readSetBegin($_etype522, $_size519);
-            for ($_i523 = 0; $_i523 < $_size519; ++$_i523)
+            $_size510 = 0;
+            $_etype513 = 0;
+            $xfer += $input->readSetBegin($_etype513, $_size510);
+            for ($_i514 = 0; $_i514 < $_size510; ++$_i514)
             {
-              $elem524 = null;
-              $xfer += $input->readI64($elem524);
-              if (is_scalar($elem524)) {
-                $this->nosuch[$elem524] = true;
+              $elem515 = null;
+              $xfer += $input->readI64($elem515);
+              if (is_scalar($elem515)) {
+                $this->nosuch[$elem515] = true;
               } else {
-                $this->nosuch []= $elem524;
+                $this->nosuch []= $elem515;
               }
             }
             $xfer += $input->readSetEnd();
@@ -16080,12 +16049,12 @@ class HeartbeatTxnRangeResponse {
       {
         $output->writeSetBegin(TType::I64, count($this->aborted));
         {
-          foreach ($this->aborted as $iter525 => $iter526)
+          foreach ($this->aborted as $iter516 => $iter517)
           {
-            if (is_scalar($iter526)) {
-            $xfer += $output->writeI64($iter525);
+            if (is_scalar($iter517)) {
+            $xfer += $output->writeI64($iter516);
             } else {
-            $xfer += $output->writeI64($iter526);
+            $xfer += $output->writeI64($iter517);
             }
           }
         }
@@ -16101,12 +16070,12 @@ class HeartbeatTxnRangeResponse {
       {
         $output->writeSetBegin(TType::I64, count($this->nosuch));
         {
-          foreach ($this->nosuch as $iter527 => $iter528)
+          foreach ($this->nosuch as $iter518 => $iter519)
           {
-            if (is_scalar($iter528)) {
-            $xfer += $output->writeI64($iter527);
+            if (is_scalar($iter519)) {
+            $xfer += $output->writeI64($iter518);
             } else {
-            $xfer += $output->writeI64($iter528);
+            $xfer += $output->writeI64($iter519);
             }
           }
         }
@@ -16265,17 +16234,17 @@ class CompactionRequest {
         case 6:
           if ($ftype == TType::MAP) {
             $this->properties = array();
-            $_size529 = 0;
-            $_ktype530 = 0;
-            $_vtype531 = 0;
-            $xfer += $input->readMapBegin($_ktype530, $_vtype531, $_size529);
-            for ($_i533 = 0; $_i533 < $_size529; ++$_i533)
+            $_size520 = 0;
+            $_ktype521 = 0;
+            $_vtype522 = 0;
+            $xfer += $input->readMapBegin($_ktype521, $_vtype522, $_size520);
+            for ($_i524 = 0; $_i524 < $_size520; ++$_i524)
             {
-              $key534 = '';
-              $val535 = '';
-              $xfer += $input->readString($key534);
-              $xfer += $input->readString($val535);
-              $this->properties[$key534] = $val535;
+              $key525 = '';
+              $val526 = '';
+              $xfer += $input->readString($key525);
+              $xfer += $input->readString($val526);
+              $this->properties[$key525] = $val526;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -16328,10 +16297,10 @@ class CompactionRequest {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->properties));
         {
-          foreach ($this->properties as $kiter536 => $viter537)
+          foreach ($this->properties as $kiter527 => $viter528)
           {
-            $xfer += $output->writeString($kiter536);
-            $xfer += $output->writeString($viter537);
+            $xfer += $output->writeString($kiter527);
+            $xfer += $output->writeString($viter528);
           }
         }
         $output->writeMapEnd();
@@ -16918,15 +16887,15 @@ class ShowCompactResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->compacts = array();
-            $_size538 = 0;
-            $_etype541 = 0;
-            $xfer += $input->readListBegin($_etype541, $_size538);
-            for ($_i542 = 0; $_i542 < $_size538; ++$_i542)
+            $_size529 = 0;
+            $_etype532 = 0;
+            $xfer += $input->readListBegin($_etype532, $_size529);
+            for ($_i533 = 0; $_i533 < $_size529; ++$_i533)
             {
-              $elem543 = null;
-              $elem543 = new \metastore\ShowCompactResponseElement();
-              $xfer += $elem543->read($input);
-              $this->compacts []= $elem543;
+              $elem534 = null;
+              $elem534 = new \metastore\ShowCompactResponseElement();
+              $xfer += $elem534->read($input);
+              $this->compacts []= $elem534;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16954,9 +16923,9 @@ class ShowCompactResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->compacts));
         {
-          foreach ($this->compacts as $iter544)
+          foreach ($this->compacts as $iter535)
           {
-            $xfer += $iter544->write($output);
+            $xfer += $iter535->write($output);
           }
         }
         $output->writeListEnd();
@@ -17085,14 +17054,14 @@ class AddDynamicPartitions {
         case 4:
           if ($ftype == TType::LST) {
             $this->partitionnames = array();
-            $_size545 = 0;
-            $_etype548 = 0;
-            $xfer += $input->readListBegin($_etype548, $_size545);
-            for ($_i549 = 0; $_i549 < $_size545; ++$_i549)
+            $_size536 = 0;
+            $_etype539 = 0;
+            $xfer += $input->readListBegin($_etype539, $_size536);
+            for ($_i540 = 0; $_i540 < $_size536; ++$_i540)
             {
-              $elem550 = null;
-              $xfer += $input->readString($elem550);
-              $this->partitionnames []= $elem550;
+              $elem541 = null;
+              $xfer += $input->readString($elem541);
+              $this->partitionnames []= $elem541;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17142,9 +17111,9 @@ class AddDynamicPartitions {
       {
         $output->writeListBegin(TType::STRING, count($this->partitionnames));
         {
-          foreach ($this->partitionnames as $iter551)
+          foreach ($this->partitionnames as $iter542)
           {
-            $xfer += $output->writeString($iter551);
+            $xfer += $output->writeString($iter542);
           }
         }
         $output->writeListEnd();
@@ -17173,10 +17142,6 @@ class BasicTxnInfo {
   /**
    * @var int
    */
-  public $id = null;
-  /**
-   * @var int
-   */
   public $time = null;
   /**
    * @var int
@@ -17203,26 +17168,22 @@ class BasicTxnInfo {
           'type' => TType::BOOL,
           ),
         2 => array(
-          'var' => 'id',
-          'type' => TType::I64,
-          ),
-        3 => array(
           'var' => 'time',
           'type' => TType::I64,
           ),
-        4 => array(
+        3 => array(
           'var' => 'txnid',
           'type' => TType::I64,
           ),
-        5 => array(
+        4 => array(
           'var' => 'dbname',
           'type' => TType::STRING,
           ),
-        6 => array(
+        5 => array(
           'var' => 'tablename',
           'type' => TType::STRING,
           ),
-        7 => array(
+        6 => array(
           'var' => 'partitionname',
           'type' => TType::STRING,
           ),
@@ -17232,9 +17193,6 @@ class BasicTxnInfo {
       if (isset($vals['isnull'])) {
         $this->isnull = $vals['isnull'];
       }
-      if (isset($vals['id'])) {
-        $this->id = $vals['id'];
-      }
       if (isset($vals['time'])) {
         $this->time = $vals['time'];
       }
@@ -17281,40 +17239,33 @@ class BasicTxnInfo {
           break;
         case 2:
           if ($ftype == TType::I64) {
-            $xfer += $input->readI64($this->id);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 3:
-          if ($ftype == TType::I64) {
             $xfer += $input->readI64($this->time);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 4:
+        case 3:
           if ($ftype == TType::I64) {
             $xfer += $input->readI64($this->txnid);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 5:
+        case 4:
           if ($ftype == TType::STRING) {
             $xfer += $input->readString($this->dbname);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 6:
+        case 5:
           if ($ftype == TType::STRING) {
             $xfer += $input->readString($this->tablename);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 7:
+        case 6:
           if ($ftype == TType::STRING) {
             $xfer += $input->readString($this->partitionname);
           } else {
@@ -17339,33 +17290,28 @@ class BasicTxnInfo {
       $xfer += $output->writeBool($this->isnull);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->id !== null) {
-      $xfer += $output->writeFieldBegin('id', TType::I64, 2);
-      $xfer += $output->writeI64($this->id);
-      $xfer += $output->writeFieldEnd();
-    }
     if ($this->time !== null) {
-      $xfer += $output->writeFieldBegin('time', TType::I64, 3);
+      $xfer += $output->writeFieldBegin('time', TType::I64, 2);
       $xfer += $output->writeI64($this->time);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->txnid !== null) {
-      $xfer += $output->writeFieldBegin('txnid', TType::I64, 4);
+      $xfer += $output->writeFieldBegin('txnid', TType::I64, 3);
       $xfer += $output->writeI64($this->txnid);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->dbname !== null) {
-      $xfer += $output->writeFieldBegin('dbname', TType::STRING, 5);
+      $xfer += $output->writeFieldBegin('dbname', TType::STRING, 4);
       $xfer += $output->writeString($this->dbname);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->tablename !== null) {
-      $xfer += $output->writeFieldBegin('tablename', TType::STRING, 6);
+      $xfer += $output->writeFieldBegin('tablename', TType::STRING, 5);
       $xfer += $output->writeString($this->tablename);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->partitionname !== null) {
-      $xfer += $output->writeFieldBegin('partitionname', TType::STRING, 7);
+      $xfer += $output->writeFieldBegin('partitionname', TType::STRING, 6);
       $xfer += $output->writeString($this->partitionname);
       $xfer += $output->writeFieldEnd();
     }
@@ -17376,47 +17322,69 @@ class BasicTxnInfo {
 
 }
 
-class TxnsSnapshot {
+class CreationMetadata {
   static $_TSPEC;
 
   /**
-   * @var int
+   * @var string
    */
-  public $txn_high_water_mark = null;
+  public $dbName = null;
   /**
-   * @var int[]
+   * @var string
    */
-  public $open_txns = null;
+  public $tblName = null;
+  /**
+   * @var string[]
+   */
+  public $tablesUsed = null;
+  /**
+   * @var string
+   */
+  public $validTxnList = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'txn_high_water_mark',
-          'type' => TType::I64,
+          'var' => 'dbName',
+          'type' => TType::STRING,
           ),
         2 => array(
-          'var' => 'open_txns',
-          'type' => TType::LST,
-          'etype' => TType::I64,
+          'var' => 'tblName',
+          'type' => TType::STRING,
+          ),
+        3 => array(
+          'var' => 'tablesUsed',
+          'type' => TType::SET,
+          'etype' => TType::STRING,
           'elem' => array(
-            'type' => TType::I64,
+            'type' => TType::STRING,
             ),
           ),
+        4 => array(
+          'var' => 'validTxnList',
+          'type' => TType::STRING,
+          ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['txn_high_water_mark'])) {
-        $this->txn_high_water_mark = $vals['txn_high_water_mark'];
+      if (isset($vals['dbName'])) {
+        $this->dbName = $vals['dbName'];
       }
-      if (isset($vals['open_txns'])) {
-        $this->open_txns = $vals['open_txns'];
+      if (isset($vals['tblName'])) {
+        $this->tblName = $vals['tblName'];
+      }
+      if (isset($vals['tablesUsed'])) {
+        $this->tablesUsed = $vals['tablesUsed'];
+      }
+      if (isset($vals['validTxnList'])) {
+        $this->validTxnList = $vals['validTxnList'];
       }
     }
   }
 
   public function getName() {
-    return 'TxnsSnapshot';
+    return 'CreationMetadata';
   }
 
   public function read($input)
@@ -17435,25 +17403,43 @@ class TxnsSnapshot {
       switch ($fid)
       {
         case 1:
-          if ($ftype == TType::I64) {
-            $xfer += $input->readI64($this->txn_high_water_mark);
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->dbName);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 2:
-          if ($ftype == TType::LST) {
-            $this->open_txns = array();
-            $_size552 = 0;
-            $_etype555 = 0;
-            $xfer += $input->readListBegin($_etype555, $_size552);
-            for ($_i556 = 0; $_i556 < $_size552; ++$_i556)
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->tblName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::SET) {
+            $this->tablesUsed = array();
+            $_size543 = 0;
+            $_etype546 = 0;
+            $xfer += $input->readSetBegin($_etype546, $_size543);
+            for ($_i547 = 0; $_i547 < $_size543; ++$_i547)
             {
-              $elem557 = null;
-              $xfer += $input->readI64($elem557);
-              $this->open_txns []= $elem557;
+              $elem548 = null;
+              $xfer += $input->readString($elem548);
+              if (is_scalar($elem548)) {
+                $this->tablesUsed[$elem548] = true;
+              } else {
+                $this->tablesUsed []= $elem548;
+              }
             }
-            $xfer += $input->readListEnd();
+            $xfer += $input->readSetEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 4:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->validTxnList);
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -17470,29 +17456,43 @@ class TxnsSnapshot {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('TxnsSnapshot');
-    if ($this->txn_high_water_mark !== null) {
-      $xfer += $output->writeFieldBegin('txn_high_water_mark', TType::I64, 1);
-      $xfer += $output->writeI64($this->txn_high_water_mark);
+    $xfer += $output->writeStructBegin('CreationMetadata');
+    if ($this->dbName !== null) {
+      $xfer += $output->writeFieldBegin('dbName', TType::STRING, 1);
+      $xfer += $output->writeString($this->dbName);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->open_txns !== null) {
-      if (!is_array($this->open_txns)) {
+    if ($this->tblName !== null) {
+      $xfer += $output->writeFieldBegin('tblName', TType::STRING, 2);
+      $xfer += $output->writeString($this->tblName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->tablesUsed !== null) {
+      if (!is_array($this->tablesUsed)) {
         throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
       }
-      $xfer += $output->writeFieldBegin('open_txns', TType::LST, 2);
+      $xfer += $output->writeFieldBegin('tablesUsed', TType::SET, 3);
       {
-        $output->writeListBegin(TType::I64, count($this->open_txns));
+        $output->writeSetBegin(TType::STRING, count($this->tablesUsed));
         {
-          foreach ($this->open_txns as $iter558)
+          foreach ($this->tablesUsed as $iter549 => $iter550)
           {
-            $xfer += $output->writeI64($iter558);
+            if (is_scalar($iter550)) {
+            $xfer += $output->writeString($iter549);
+            } else {
+            $xfer += $output->writeString($iter550);
+            }
           }
         }
-        $output->writeListEnd();
+        $output->writeSetEnd();
       }
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->validTxnList !== null) {
+      $xfer += $output->writeFieldBegin('validTxnList', TType::STRING, 4);
+      $xfer += $output->writeString($this->validTxnList);
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;
@@ -17862,15 +17862,15 @@ class NotificationEventResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->events = array();
-            $_size559 = 0;
-            $_etype562 = 0;
-            $xfer += $input->readListBegin($_etype562, $_size559);
-            for ($_i563 = 0; $_i563 < $_size559; ++$_i563)
+            $_size551 = 0;
+            $_etype554 = 0;
+            $xfer += $input->readListBegin($_etype554, $_size551);
+            for ($_i555 = 0; $_i555 < $_size551; ++$_i555)
             {
-              $elem564 = null;
-              $elem564 = new \metastore\NotificationEvent();
-              $xfer += $elem564->read($input);
-              $this->events []= $elem564;
+              $elem556 = null;
+              $elem556 = new \metastore\NotificationEvent();
+              $xfer += $elem556->read($input);
+              $this->events []= $elem556;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17898,9 +17898,9 @@ class NotificationEventResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->events));
         {
-          foreach ($this->events as $iter565)
+          foreach ($this->events as $iter557)
           {
-            $xfer += $iter565->write($output);
+            $xfer += $iter557->write($output);
           }
         }
         $output->writeListEnd();
@@ -18245,14 +18245,14 @@ class InsertEventRequestData {
         case 2:
           if ($ftype == TType::LST) {
             $this->filesAdded = array();
-            $_size566 = 0;
-            $_etype569 = 0;
-            $xfer += $input->readListBegin($_etype569, $_size566);
-            for ($_i570 = 0; $_i570 < $_size566; ++$_i570)
+            $_size558 = 0;
+            $_etype561 = 0;
+            $xfer += $input->readListBegin($_etype561, $_size558);
+            for ($_i562 = 0; $_i562 < $_size558; ++$_i562)
             {
-              $elem571 = null;
-              $xfer += $input->readString($elem571);
-              $this->filesAdded []= $elem571;
+              $elem563 = null;
+              $xfer += $input->readString($elem563);
+              $this->filesAdded []= $elem563;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18262,14 +18262,14 @@ class InsertEventRequestData {
         case 3:
           if ($ftype == TType::LST) {
             $this->filesAddedChecksum = array();
-            $_size572 = 0;
-            $_etype575 = 0;
-            $xfer += $input->readListBegin($_etype575, $_size572);
-            for ($_i576 = 0; $_i576 < $_size572; ++$_i576)
+            $_size564 = 0;
+            $_etype567 = 0;
+            $xfer += $input->readListBegin($_etype567, $_size564);
+            for ($_i568 = 0; $_i568 < $_size564; ++$_i568)
             {
-              $elem577 = null;
-              $xfer += $input->readString($elem577);
-              $this->filesAddedChecksum []= $elem577;
+              $elem569 = null;
+              $xfer += $input->readString($elem569);
+              $this->filesAddedChecksum []= $elem569;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18302,9 +18302,9 @@ class InsertEventRequestData {
       {
         $output->writeListBegin(TType::STRING, count($this->filesAdded));
         {
-          foreach ($this->filesAdded as $iter578)
+          foreach ($this->filesAdded as $iter570)
           {
-            $xfer += $output->writeString($iter578);
+            $xfer += $output->writeString($iter570);
           }
         }
         $output->writeListEnd();
@@ -18319,9 +18319,9 @@ class InsertEventRequestData {
       {
         $output->writeListBegin(TType::STRING, count($this->filesAddedChecksum));
         {
-          foreach ($this->filesAddedChecksum as $iter579)
+          foreach ($this->filesAddedChecksum as $iter571)
           {
-            $xfer += $output->writeString($iter579);
+            $xfer += $output->writeString($iter571);
           }
         }
         $output->writeListEnd();
@@ -18539,14 +18539,14 @@ class FireEventRequest {
         case 5:
           if ($ftype == TType::LST) {
             $this->partitionVals = array();
-            $_size580 = 0;
-            $_etype583 = 0;
-            $xfer += $input->readListBegin($_etype583, $_size580);
-            for ($_i584 = 0; $_i584 < $_size580; ++$_i584)
+            $_size572 = 0;
+ 

<TRUNCATED>

[15/16] hive git commit: HIVE-18546: Remove unnecessary code introduced in HIVE-14498 (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
index 8b78230..af0fd6b 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@ -1240,14 +1240,14 @@ uint32_t ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1056;
-            ::apache::thrift::protocol::TType _etype1059;
-            xfer += iprot->readListBegin(_etype1059, _size1056);
-            this->success.resize(_size1056);
-            uint32_t _i1060;
-            for (_i1060 = 0; _i1060 < _size1056; ++_i1060)
+            uint32_t _size1049;
+            ::apache::thrift::protocol::TType _etype1052;
+            xfer += iprot->readListBegin(_etype1052, _size1049);
+            this->success.resize(_size1049);
+            uint32_t _i1053;
+            for (_i1053 = 0; _i1053 < _size1049; ++_i1053)
             {
-              xfer += iprot->readString(this->success[_i1060]);
+              xfer += iprot->readString(this->success[_i1053]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1286,10 +1286,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 _iter1061;
-      for (_iter1061 = this->success.begin(); _iter1061 != this->success.end(); ++_iter1061)
+      std::vector<std::string> ::const_iterator _iter1054;
+      for (_iter1054 = this->success.begin(); _iter1054 != this->success.end(); ++_iter1054)
       {
-        xfer += oprot->writeString((*_iter1061));
+        xfer += oprot->writeString((*_iter1054));
       }
       xfer += oprot->writeListEnd();
     }
@@ -1334,14 +1334,14 @@ uint32_t ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1062;
-            ::apache::thrift::protocol::TType _etype1065;
-            xfer += iprot->readListBegin(_etype1065, _size1062);
-            (*(this->success)).resize(_size1062);
-            uint32_t _i1066;
-            for (_i1066 = 0; _i1066 < _size1062; ++_i1066)
+            uint32_t _size1055;
+            ::apache::thrift::protocol::TType _etype1058;
+            xfer += iprot->readListBegin(_etype1058, _size1055);
+            (*(this->success)).resize(_size1055);
+            uint32_t _i1059;
+            for (_i1059 = 0; _i1059 < _size1055; ++_i1059)
             {
-              xfer += iprot->readString((*(this->success))[_i1066]);
+              xfer += iprot->readString((*(this->success))[_i1059]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1458,14 +1458,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1067;
-            ::apache::thrift::protocol::TType _etype1070;
-            xfer += iprot->readListBegin(_etype1070, _size1067);
-            this->success.resize(_size1067);
-            uint32_t _i1071;
-            for (_i1071 = 0; _i1071 < _size1067; ++_i1071)
+            uint32_t _size1060;
+            ::apache::thrift::protocol::TType _etype1063;
+            xfer += iprot->readListBegin(_etype1063, _size1060);
+            this->success.resize(_size1060);
+            uint32_t _i1064;
+            for (_i1064 = 0; _i1064 < _size1060; ++_i1064)
             {
-              xfer += iprot->readString(this->success[_i1071]);
+              xfer += iprot->readString(this->success[_i1064]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1504,10 +1504,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 _iter1072;
-      for (_iter1072 = this->success.begin(); _iter1072 != this->success.end(); ++_iter1072)
+      std::vector<std::string> ::const_iterator _iter1065;
+      for (_iter1065 = this->success.begin(); _iter1065 != this->success.end(); ++_iter1065)
       {
-        xfer += oprot->writeString((*_iter1072));
+        xfer += oprot->writeString((*_iter1065));
       }
       xfer += oprot->writeListEnd();
     }
@@ -1552,14 +1552,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1073;
-            ::apache::thrift::protocol::TType _etype1076;
-            xfer += iprot->readListBegin(_etype1076, _size1073);
-            (*(this->success)).resize(_size1073);
-            uint32_t _i1077;
-            for (_i1077 = 0; _i1077 < _size1073; ++_i1077)
+            uint32_t _size1066;
+            ::apache::thrift::protocol::TType _etype1069;
+            xfer += iprot->readListBegin(_etype1069, _size1066);
+            (*(this->success)).resize(_size1066);
+            uint32_t _i1070;
+            for (_i1070 = 0; _i1070 < _size1066; ++_i1070)
             {
-              xfer += iprot->readString((*(this->success))[_i1077]);
+              xfer += iprot->readString((*(this->success))[_i1070]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2621,17 +2621,17 @@ uint32_t ThriftHiveMetastore_get_type_all_result::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size1078;
-            ::apache::thrift::protocol::TType _ktype1079;
-            ::apache::thrift::protocol::TType _vtype1080;
-            xfer += iprot->readMapBegin(_ktype1079, _vtype1080, _size1078);
-            uint32_t _i1082;
-            for (_i1082 = 0; _i1082 < _size1078; ++_i1082)
+            uint32_t _size1071;
+            ::apache::thrift::protocol::TType _ktype1072;
+            ::apache::thrift::protocol::TType _vtype1073;
+            xfer += iprot->readMapBegin(_ktype1072, _vtype1073, _size1071);
+            uint32_t _i1075;
+            for (_i1075 = 0; _i1075 < _size1071; ++_i1075)
             {
-              std::string _key1083;
-              xfer += iprot->readString(_key1083);
-              Type& _val1084 = this->success[_key1083];
-              xfer += _val1084.read(iprot);
+              std::string _key1076;
+              xfer += iprot->readString(_key1076);
+              Type& _val1077 = this->success[_key1076];
+              xfer += _val1077.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -2670,11 +2670,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 _iter1085;
-      for (_iter1085 = this->success.begin(); _iter1085 != this->success.end(); ++_iter1085)
+      std::map<std::string, Type> ::const_iterator _iter1078;
+      for (_iter1078 = this->success.begin(); _iter1078 != this->success.end(); ++_iter1078)
       {
-        xfer += oprot->writeString(_iter1085->first);
-        xfer += _iter1085->second.write(oprot);
+        xfer += oprot->writeString(_iter1078->first);
+        xfer += _iter1078->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -2719,17 +2719,17 @@ uint32_t ThriftHiveMetastore_get_type_all_presult::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size1086;
-            ::apache::thrift::protocol::TType _ktype1087;
-            ::apache::thrift::protocol::TType _vtype1088;
-            xfer += iprot->readMapBegin(_ktype1087, _vtype1088, _size1086);
-            uint32_t _i1090;
-            for (_i1090 = 0; _i1090 < _size1086; ++_i1090)
+            uint32_t _size1079;
+            ::apache::thrift::protocol::TType _ktype1080;
+            ::apache::thrift::protocol::TType _vtype1081;
+            xfer += iprot->readMapBegin(_ktype1080, _vtype1081, _size1079);
+            uint32_t _i1083;
+            for (_i1083 = 0; _i1083 < _size1079; ++_i1083)
             {
-              std::string _key1091;
-              xfer += iprot->readString(_key1091);
-              Type& _val1092 = (*(this->success))[_key1091];
-              xfer += _val1092.read(iprot);
+              std::string _key1084;
+              xfer += iprot->readString(_key1084);
+              Type& _val1085 = (*(this->success))[_key1084];
+              xfer += _val1085.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -2883,14 +2883,14 @@ uint32_t ThriftHiveMetastore_get_fields_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1093;
-            ::apache::thrift::protocol::TType _etype1096;
-            xfer += iprot->readListBegin(_etype1096, _size1093);
-            this->success.resize(_size1093);
-            uint32_t _i1097;
-            for (_i1097 = 0; _i1097 < _size1093; ++_i1097)
+            uint32_t _size1086;
+            ::apache::thrift::protocol::TType _etype1089;
+            xfer += iprot->readListBegin(_etype1089, _size1086);
+            this->success.resize(_size1086);
+            uint32_t _i1090;
+            for (_i1090 = 0; _i1090 < _size1086; ++_i1090)
             {
-              xfer += this->success[_i1097].read(iprot);
+              xfer += this->success[_i1090].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -2945,10 +2945,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 _iter1098;
-      for (_iter1098 = this->success.begin(); _iter1098 != this->success.end(); ++_iter1098)
+      std::vector<FieldSchema> ::const_iterator _iter1091;
+      for (_iter1091 = this->success.begin(); _iter1091 != this->success.end(); ++_iter1091)
       {
-        xfer += (*_iter1098).write(oprot);
+        xfer += (*_iter1091).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3001,14 +3001,14 @@ uint32_t ThriftHiveMetastore_get_fields_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1099;
-            ::apache::thrift::protocol::TType _etype1102;
-            xfer += iprot->readListBegin(_etype1102, _size1099);
-            (*(this->success)).resize(_size1099);
-            uint32_t _i1103;
-            for (_i1103 = 0; _i1103 < _size1099; ++_i1103)
+            uint32_t _size1092;
+            ::apache::thrift::protocol::TType _etype1095;
+            xfer += iprot->readListBegin(_etype1095, _size1092);
+            (*(this->success)).resize(_size1092);
+            uint32_t _i1096;
+            for (_i1096 = 0; _i1096 < _size1092; ++_i1096)
             {
-              xfer += (*(this->success))[_i1103].read(iprot);
+              xfer += (*(this->success))[_i1096].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3194,14 +3194,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1104;
-            ::apache::thrift::protocol::TType _etype1107;
-            xfer += iprot->readListBegin(_etype1107, _size1104);
-            this->success.resize(_size1104);
-            uint32_t _i1108;
-            for (_i1108 = 0; _i1108 < _size1104; ++_i1108)
+            uint32_t _size1097;
+            ::apache::thrift::protocol::TType _etype1100;
+            xfer += iprot->readListBegin(_etype1100, _size1097);
+            this->success.resize(_size1097);
+            uint32_t _i1101;
+            for (_i1101 = 0; _i1101 < _size1097; ++_i1101)
             {
-              xfer += this->success[_i1108].read(iprot);
+              xfer += this->success[_i1101].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3256,10 +3256,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 _iter1109;
-      for (_iter1109 = this->success.begin(); _iter1109 != this->success.end(); ++_iter1109)
+      std::vector<FieldSchema> ::const_iterator _iter1102;
+      for (_iter1102 = this->success.begin(); _iter1102 != this->success.end(); ++_iter1102)
       {
-        xfer += (*_iter1109).write(oprot);
+        xfer += (*_iter1102).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3312,14 +3312,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1110;
-            ::apache::thrift::protocol::TType _etype1113;
-            xfer += iprot->readListBegin(_etype1113, _size1110);
-            (*(this->success)).resize(_size1110);
-            uint32_t _i1114;
-            for (_i1114 = 0; _i1114 < _size1110; ++_i1114)
+            uint32_t _size1103;
+            ::apache::thrift::protocol::TType _etype1106;
+            xfer += iprot->readListBegin(_etype1106, _size1103);
+            (*(this->success)).resize(_size1103);
+            uint32_t _i1107;
+            for (_i1107 = 0; _i1107 < _size1103; ++_i1107)
             {
-              xfer += (*(this->success))[_i1114].read(iprot);
+              xfer += (*(this->success))[_i1107].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3489,14 +3489,14 @@ uint32_t ThriftHiveMetastore_get_schema_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1115;
-            ::apache::thrift::protocol::TType _etype1118;
-            xfer += iprot->readListBegin(_etype1118, _size1115);
-            this->success.resize(_size1115);
-            uint32_t _i1119;
-            for (_i1119 = 0; _i1119 < _size1115; ++_i1119)
+            uint32_t _size1108;
+            ::apache::thrift::protocol::TType _etype1111;
+            xfer += iprot->readListBegin(_etype1111, _size1108);
+            this->success.resize(_size1108);
+            uint32_t _i1112;
+            for (_i1112 = 0; _i1112 < _size1108; ++_i1112)
             {
-              xfer += this->success[_i1119].read(iprot);
+              xfer += this->success[_i1112].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3551,10 +3551,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 _iter1120;
-      for (_iter1120 = this->success.begin(); _iter1120 != this->success.end(); ++_iter1120)
+      std::vector<FieldSchema> ::const_iterator _iter1113;
+      for (_iter1113 = this->success.begin(); _iter1113 != this->success.end(); ++_iter1113)
       {
-        xfer += (*_iter1120).write(oprot);
+        xfer += (*_iter1113).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3607,14 +3607,14 @@ uint32_t ThriftHiveMetastore_get_schema_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1121;
-            ::apache::thrift::protocol::TType _etype1124;
-            xfer += iprot->readListBegin(_etype1124, _size1121);
-            (*(this->success)).resize(_size1121);
-            uint32_t _i1125;
-            for (_i1125 = 0; _i1125 < _size1121; ++_i1125)
+            uint32_t _size1114;
+            ::apache::thrift::protocol::TType _etype1117;
+            xfer += iprot->readListBegin(_etype1117, _size1114);
+            (*(this->success)).resize(_size1114);
+            uint32_t _i1118;
+            for (_i1118 = 0; _i1118 < _size1114; ++_i1118)
             {
-              xfer += (*(this->success))[_i1125].read(iprot);
+              xfer += (*(this->success))[_i1118].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3800,14 +3800,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1126;
-            ::apache::thrift::protocol::TType _etype1129;
-            xfer += iprot->readListBegin(_etype1129, _size1126);
-            this->success.resize(_size1126);
-            uint32_t _i1130;
-            for (_i1130 = 0; _i1130 < _size1126; ++_i1130)
+            uint32_t _size1119;
+            ::apache::thrift::protocol::TType _etype1122;
+            xfer += iprot->readListBegin(_etype1122, _size1119);
+            this->success.resize(_size1119);
+            uint32_t _i1123;
+            for (_i1123 = 0; _i1123 < _size1119; ++_i1123)
             {
-              xfer += this->success[_i1130].read(iprot);
+              xfer += this->success[_i1123].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3862,10 +3862,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 _iter1131;
-      for (_iter1131 = this->success.begin(); _iter1131 != this->success.end(); ++_iter1131)
+      std::vector<FieldSchema> ::const_iterator _iter1124;
+      for (_iter1124 = this->success.begin(); _iter1124 != this->success.end(); ++_iter1124)
       {
-        xfer += (*_iter1131).write(oprot);
+        xfer += (*_iter1124).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3918,14 +3918,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1132;
-            ::apache::thrift::protocol::TType _etype1135;
-            xfer += iprot->readListBegin(_etype1135, _size1132);
-            (*(this->success)).resize(_size1132);
-            uint32_t _i1136;
-            for (_i1136 = 0; _i1136 < _size1132; ++_i1136)
+            uint32_t _size1125;
+            ::apache::thrift::protocol::TType _etype1128;
+            xfer += iprot->readListBegin(_etype1128, _size1125);
+            (*(this->success)).resize(_size1125);
+            uint32_t _i1129;
+            for (_i1129 = 0; _i1129 < _size1125; ++_i1129)
             {
-              xfer += (*(this->success))[_i1136].read(iprot);
+              xfer += (*(this->success))[_i1129].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4518,14 +4518,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->primaryKeys.clear();
-            uint32_t _size1137;
-            ::apache::thrift::protocol::TType _etype1140;
-            xfer += iprot->readListBegin(_etype1140, _size1137);
-            this->primaryKeys.resize(_size1137);
-            uint32_t _i1141;
-            for (_i1141 = 0; _i1141 < _size1137; ++_i1141)
+            uint32_t _size1130;
+            ::apache::thrift::protocol::TType _etype1133;
+            xfer += iprot->readListBegin(_etype1133, _size1130);
+            this->primaryKeys.resize(_size1130);
+            uint32_t _i1134;
+            for (_i1134 = 0; _i1134 < _size1130; ++_i1134)
             {
-              xfer += this->primaryKeys[_i1141].read(iprot);
+              xfer += this->primaryKeys[_i1134].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4538,14 +4538,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->foreignKeys.clear();
-            uint32_t _size1142;
-            ::apache::thrift::protocol::TType _etype1145;
-            xfer += iprot->readListBegin(_etype1145, _size1142);
-            this->foreignKeys.resize(_size1142);
-            uint32_t _i1146;
-            for (_i1146 = 0; _i1146 < _size1142; ++_i1146)
+            uint32_t _size1135;
+            ::apache::thrift::protocol::TType _etype1138;
+            xfer += iprot->readListBegin(_etype1138, _size1135);
+            this->foreignKeys.resize(_size1135);
+            uint32_t _i1139;
+            for (_i1139 = 0; _i1139 < _size1135; ++_i1139)
             {
-              xfer += this->foreignKeys[_i1146].read(iprot);
+              xfer += this->foreignKeys[_i1139].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4558,14 +4558,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->uniqueConstraints.clear();
-            uint32_t _size1147;
-            ::apache::thrift::protocol::TType _etype1150;
-            xfer += iprot->readListBegin(_etype1150, _size1147);
-            this->uniqueConstraints.resize(_size1147);
-            uint32_t _i1151;
-            for (_i1151 = 0; _i1151 < _size1147; ++_i1151)
+            uint32_t _size1140;
+            ::apache::thrift::protocol::TType _etype1143;
+            xfer += iprot->readListBegin(_etype1143, _size1140);
+            this->uniqueConstraints.resize(_size1140);
+            uint32_t _i1144;
+            for (_i1144 = 0; _i1144 < _size1140; ++_i1144)
             {
-              xfer += this->uniqueConstraints[_i1151].read(iprot);
+              xfer += this->uniqueConstraints[_i1144].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4578,14 +4578,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->notNullConstraints.clear();
-            uint32_t _size1152;
-            ::apache::thrift::protocol::TType _etype1155;
-            xfer += iprot->readListBegin(_etype1155, _size1152);
-            this->notNullConstraints.resize(_size1152);
-            uint32_t _i1156;
-            for (_i1156 = 0; _i1156 < _size1152; ++_i1156)
+            uint32_t _size1145;
+            ::apache::thrift::protocol::TType _etype1148;
+            xfer += iprot->readListBegin(_etype1148, _size1145);
+            this->notNullConstraints.resize(_size1145);
+            uint32_t _i1149;
+            for (_i1149 = 0; _i1149 < _size1145; ++_i1149)
             {
-              xfer += this->notNullConstraints[_i1156].read(iprot);
+              xfer += this->notNullConstraints[_i1149].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4618,10 +4618,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 _iter1157;
-    for (_iter1157 = this->primaryKeys.begin(); _iter1157 != this->primaryKeys.end(); ++_iter1157)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter1150;
+    for (_iter1150 = this->primaryKeys.begin(); _iter1150 != this->primaryKeys.end(); ++_iter1150)
     {
-      xfer += (*_iter1157).write(oprot);
+      xfer += (*_iter1150).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4630,10 +4630,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 _iter1158;
-    for (_iter1158 = this->foreignKeys.begin(); _iter1158 != this->foreignKeys.end(); ++_iter1158)
+    std::vector<SQLForeignKey> ::const_iterator _iter1151;
+    for (_iter1151 = this->foreignKeys.begin(); _iter1151 != this->foreignKeys.end(); ++_iter1151)
     {
-      xfer += (*_iter1158).write(oprot);
+      xfer += (*_iter1151).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4642,10 +4642,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 _iter1159;
-    for (_iter1159 = this->uniqueConstraints.begin(); _iter1159 != this->uniqueConstraints.end(); ++_iter1159)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter1152;
+    for (_iter1152 = this->uniqueConstraints.begin(); _iter1152 != this->uniqueConstraints.end(); ++_iter1152)
     {
-      xfer += (*_iter1159).write(oprot);
+      xfer += (*_iter1152).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4654,10 +4654,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 _iter1160;
-    for (_iter1160 = this->notNullConstraints.begin(); _iter1160 != this->notNullConstraints.end(); ++_iter1160)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter1153;
+    for (_iter1153 = this->notNullConstraints.begin(); _iter1153 != this->notNullConstraints.end(); ++_iter1153)
     {
-      xfer += (*_iter1160).write(oprot);
+      xfer += (*_iter1153).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4685,10 +4685,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 _iter1161;
-    for (_iter1161 = (*(this->primaryKeys)).begin(); _iter1161 != (*(this->primaryKeys)).end(); ++_iter1161)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter1154;
+    for (_iter1154 = (*(this->primaryKeys)).begin(); _iter1154 != (*(this->primaryKeys)).end(); ++_iter1154)
     {
-      xfer += (*_iter1161).write(oprot);
+      xfer += (*_iter1154).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4697,10 +4697,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 _iter1162;
-    for (_iter1162 = (*(this->foreignKeys)).begin(); _iter1162 != (*(this->foreignKeys)).end(); ++_iter1162)
+    std::vector<SQLForeignKey> ::const_iterator _iter1155;
+    for (_iter1155 = (*(this->foreignKeys)).begin(); _iter1155 != (*(this->foreignKeys)).end(); ++_iter1155)
     {
-      xfer += (*_iter1162).write(oprot);
+      xfer += (*_iter1155).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4709,10 +4709,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 _iter1163;
-    for (_iter1163 = (*(this->uniqueConstraints)).begin(); _iter1163 != (*(this->uniqueConstraints)).end(); ++_iter1163)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter1156;
+    for (_iter1156 = (*(this->uniqueConstraints)).begin(); _iter1156 != (*(this->uniqueConstraints)).end(); ++_iter1156)
     {
-      xfer += (*_iter1163).write(oprot);
+      xfer += (*_iter1156).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4721,10 +4721,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 _iter1164;
-    for (_iter1164 = (*(this->notNullConstraints)).begin(); _iter1164 != (*(this->notNullConstraints)).end(); ++_iter1164)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter1157;
+    for (_iter1157 = (*(this->notNullConstraints)).begin(); _iter1157 != (*(this->notNullConstraints)).end(); ++_iter1157)
     {
-      xfer += (*_iter1164).write(oprot);
+      xfer += (*_iter1157).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -6478,14 +6478,14 @@ uint32_t ThriftHiveMetastore_truncate_table_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partNames.clear();
-            uint32_t _size1165;
-            ::apache::thrift::protocol::TType _etype1168;
-            xfer += iprot->readListBegin(_etype1168, _size1165);
-            this->partNames.resize(_size1165);
-            uint32_t _i1169;
-            for (_i1169 = 0; _i1169 < _size1165; ++_i1169)
+            uint32_t _size1158;
+            ::apache::thrift::protocol::TType _etype1161;
+            xfer += iprot->readListBegin(_etype1161, _size1158);
+            this->partNames.resize(_size1158);
+            uint32_t _i1162;
+            for (_i1162 = 0; _i1162 < _size1158; ++_i1162)
             {
-              xfer += iprot->readString(this->partNames[_i1169]);
+              xfer += iprot->readString(this->partNames[_i1162]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6522,10 +6522,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 _iter1170;
-    for (_iter1170 = this->partNames.begin(); _iter1170 != this->partNames.end(); ++_iter1170)
+    std::vector<std::string> ::const_iterator _iter1163;
+    for (_iter1163 = this->partNames.begin(); _iter1163 != this->partNames.end(); ++_iter1163)
     {
-      xfer += oprot->writeString((*_iter1170));
+      xfer += oprot->writeString((*_iter1163));
     }
     xfer += oprot->writeListEnd();
   }
@@ -6557,10 +6557,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 _iter1171;
-    for (_iter1171 = (*(this->partNames)).begin(); _iter1171 != (*(this->partNames)).end(); ++_iter1171)
+    std::vector<std::string> ::const_iterator _iter1164;
+    for (_iter1164 = (*(this->partNames)).begin(); _iter1164 != (*(this->partNames)).end(); ++_iter1164)
     {
-      xfer += oprot->writeString((*_iter1171));
+      xfer += oprot->writeString((*_iter1164));
     }
     xfer += oprot->writeListEnd();
   }
@@ -6804,14 +6804,14 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1172;
-            ::apache::thrift::protocol::TType _etype1175;
-            xfer += iprot->readListBegin(_etype1175, _size1172);
-            this->success.resize(_size1172);
-            uint32_t _i1176;
-            for (_i1176 = 0; _i1176 < _size1172; ++_i1176)
+            uint32_t _size1165;
+            ::apache::thrift::protocol::TType _etype1168;
+            xfer += iprot->readListBegin(_etype1168, _size1165);
+            this->success.resize(_size1165);
+            uint32_t _i1169;
+            for (_i1169 = 0; _i1169 < _size1165; ++_i1169)
             {
-              xfer += iprot->readString(this->success[_i1176]);
+              xfer += iprot->readString(this->success[_i1169]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6850,10 +6850,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 _iter1177;
-      for (_iter1177 = this->success.begin(); _iter1177 != this->success.end(); ++_iter1177)
+      std::vector<std::string> ::const_iterator _iter1170;
+      for (_iter1170 = this->success.begin(); _iter1170 != this->success.end(); ++_iter1170)
       {
-        xfer += oprot->writeString((*_iter1177));
+        xfer += oprot->writeString((*_iter1170));
       }
       xfer += oprot->writeListEnd();
     }
@@ -6898,14 +6898,14 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1178;
-            ::apache::thrift::protocol::TType _etype1181;
-            xfer += iprot->readListBegin(_etype1181, _size1178);
-            (*(this->success)).resize(_size1178);
-            uint32_t _i1182;
-            for (_i1182 = 0; _i1182 < _size1178; ++_i1182)
+            uint32_t _size1171;
+            ::apache::thrift::protocol::TType _etype1174;
+            xfer += iprot->readListBegin(_etype1174, _size1171);
+            (*(this->success)).resize(_size1171);
+            uint32_t _i1175;
+            for (_i1175 = 0; _i1175 < _size1171; ++_i1175)
             {
-              xfer += iprot->readString((*(this->success))[_i1182]);
+              xfer += iprot->readString((*(this->success))[_i1175]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7075,14 +7075,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 _size1183;
-            ::apache::thrift::protocol::TType _etype1186;
-            xfer += iprot->readListBegin(_etype1186, _size1183);
-            this->success.resize(_size1183);
-            uint32_t _i1187;
-            for (_i1187 = 0; _i1187 < _size1183; ++_i1187)
+            uint32_t _size1176;
+            ::apache::thrift::protocol::TType _etype1179;
+            xfer += iprot->readListBegin(_etype1179, _size1176);
+            this->success.resize(_size1176);
+            uint32_t _i1180;
+            for (_i1180 = 0; _i1180 < _size1176; ++_i1180)
             {
-              xfer += iprot->readString(this->success[_i1187]);
+              xfer += iprot->readString(this->success[_i1180]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7121,10 +7121,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 _iter1188;
-      for (_iter1188 = this->success.begin(); _iter1188 != this->success.end(); ++_iter1188)
+      std::vector<std::string> ::const_iterator _iter1181;
+      for (_iter1181 = this->success.begin(); _iter1181 != this->success.end(); ++_iter1181)
       {
-        xfer += oprot->writeString((*_iter1188));
+        xfer += oprot->writeString((*_iter1181));
       }
       xfer += oprot->writeListEnd();
     }
@@ -7169,14 +7169,14 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_presult::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1189;
-            ::apache::thrift::protocol::TType _etype1192;
-            xfer += iprot->readListBegin(_etype1192, _size1189);
-            (*(this->success)).resize(_size1189);
-            uint32_t _i1193;
-            for (_i1193 = 0; _i1193 < _size1189; ++_i1193)
+            uint32_t _size1182;
+            ::apache::thrift::protocol::TType _etype1185;
+            xfer += iprot->readListBegin(_etype1185, _size1182);
+            (*(this->success)).resize(_size1182);
+            uint32_t _i1186;
+            for (_i1186 = 0; _i1186 < _size1182; ++_i1186)
             {
-              xfer += iprot->readString((*(this->success))[_i1193]);
+              xfer += iprot->readString((*(this->success))[_i1186]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7314,14 +7314,14 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_result::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1194;
-            ::apache::thrift::protocol::TType _etype1197;
-            xfer += iprot->readListBegin(_etype1197, _size1194);
-            this->success.resize(_size1194);
-            uint32_t _i1198;
-            for (_i1198 = 0; _i1198 < _size1194; ++_i1198)
+            uint32_t _size1187;
+            ::apache::thrift::protocol::TType _etype1190;
+            xfer += iprot->readListBegin(_etype1190, _size1187);
+            this->success.resize(_size1187);
+            uint32_t _i1191;
+            for (_i1191 = 0; _i1191 < _size1187; ++_i1191)
             {
-              xfer += iprot->readString(this->success[_i1198]);
+              xfer += iprot->readString(this->success[_i1191]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7360,10 +7360,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 _iter1199;
-      for (_iter1199 = this->success.begin(); _iter1199 != this->success.end(); ++_iter1199)
+      std::vector<std::string> ::const_iterator _iter1192;
+      for (_iter1192 = this->success.begin(); _iter1192 != this->success.end(); ++_iter1192)
       {
-        xfer += oprot->writeString((*_iter1199));
+        xfer += oprot->writeString((*_iter1192));
       }
       xfer += oprot->writeListEnd();
     }
@@ -7408,14 +7408,14 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_presult::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1200;
-            ::apache::thrift::protocol::TType _etype1203;
-            xfer += iprot->readListBegin(_etype1203, _size1200);
-            (*(this->success)).resize(_size1200);
-            uint32_t _i1204;
-            for (_i1204 = 0; _i1204 < _size1200; ++_i1204)
+            uint32_t _size1193;
+            ::apache::thrift::protocol::TType _etype1196;
+            xfer += iprot->readListBegin(_etype1196, _size1193);
+            (*(this->success)).resize(_size1193);
+            uint32_t _i1197;
+            for (_i1197 = 0; _i1197 < _size1193; ++_i1197)
             {
-              xfer += iprot->readString((*(this->success))[_i1204]);
+              xfer += iprot->readString((*(this->success))[_i1197]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7490,14 +7490,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 _size1205;
-            ::apache::thrift::protocol::TType _etype1208;
-            xfer += iprot->readListBegin(_etype1208, _size1205);
-            this->tbl_types.resize(_size1205);
-            uint32_t _i1209;
-            for (_i1209 = 0; _i1209 < _size1205; ++_i1209)
+            uint32_t _size1198;
+            ::apache::thrift::protocol::TType _etype1201;
+            xfer += iprot->readListBegin(_etype1201, _size1198);
+            this->tbl_types.resize(_size1198);
+            uint32_t _i1202;
+            for (_i1202 = 0; _i1202 < _size1198; ++_i1202)
             {
-              xfer += iprot->readString(this->tbl_types[_i1209]);
+              xfer += iprot->readString(this->tbl_types[_i1202]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7534,10 +7534,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 _iter1210;
-    for (_iter1210 = this->tbl_types.begin(); _iter1210 != this->tbl_types.end(); ++_iter1210)
+    std::vector<std::string> ::const_iterator _iter1203;
+    for (_iter1203 = this->tbl_types.begin(); _iter1203 != this->tbl_types.end(); ++_iter1203)
     {
-      xfer += oprot->writeString((*_iter1210));
+      xfer += oprot->writeString((*_iter1203));
     }
     xfer += oprot->writeListEnd();
   }
@@ -7569,10 +7569,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 _iter1211;
-    for (_iter1211 = (*(this->tbl_types)).begin(); _iter1211 != (*(this->tbl_types)).end(); ++_iter1211)
+    std::vector<std::string> ::const_iterator _iter1204;
+    for (_iter1204 = (*(this->tbl_types)).begin(); _iter1204 != (*(this->tbl_types)).end(); ++_iter1204)
     {
-      xfer += oprot->writeString((*_iter1211));
+      xfer += oprot->writeString((*_iter1204));
     }
     xfer += oprot->writeListEnd();
   }
@@ -7613,14 +7613,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1212;
-            ::apache::thrift::protocol::TType _etype1215;
-            xfer += iprot->readListBegin(_etype1215, _size1212);
-            this->success.resize(_size1212);
-            uint32_t _i1216;
-            for (_i1216 = 0; _i1216 < _size1212; ++_i1216)
+            uint32_t _size1205;
+            ::apache::thrift::protocol::TType _etype1208;
+            xfer += iprot->readListBegin(_etype1208, _size1205);
+            this->success.resize(_size1205);
+            uint32_t _i1209;
+            for (_i1209 = 0; _i1209 < _size1205; ++_i1209)
             {
-              xfer += this->success[_i1216].read(iprot);
+              xfer += this->success[_i1209].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7659,10 +7659,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 _iter1217;
-      for (_iter1217 = this->success.begin(); _iter1217 != this->success.end(); ++_iter1217)
+      std::vector<TableMeta> ::const_iterator _iter1210;
+      for (_iter1210 = this->success.begin(); _iter1210 != this->success.end(); ++_iter1210)
       {
-        xfer += (*_iter1217).write(oprot);
+        xfer += (*_iter1210).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -7707,14 +7707,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1218;
-            ::apache::thrift::protocol::TType _etype1221;
-            xfer += iprot->readListBegin(_etype1221, _size1218);
-            (*(this->success)).resize(_size1218);
-            uint32_t _i1222;
-            for (_i1222 = 0; _i1222 < _size1218; ++_i1222)
+            uint32_t _size1211;
+            ::apache::thrift::protocol::TType _etype1214;
+            xfer += iprot->readListBegin(_etype1214, _size1211);
+            (*(this->success)).resize(_size1211);
+            uint32_t _i1215;
+            for (_i1215 = 0; _i1215 < _size1211; ++_i1215)
             {
-              xfer += (*(this->success))[_i1222].read(iprot);
+              xfer += (*(this->success))[_i1215].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7852,14 +7852,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1223;
-            ::apache::thrift::protocol::TType _etype1226;
-            xfer += iprot->readListBegin(_etype1226, _size1223);
-            this->success.resize(_size1223);
-            uint32_t _i1227;
-            for (_i1227 = 0; _i1227 < _size1223; ++_i1227)
+            uint32_t _size1216;
+            ::apache::thrift::protocol::TType _etype1219;
+            xfer += iprot->readListBegin(_etype1219, _size1216);
+            this->success.resize(_size1216);
+            uint32_t _i1220;
+            for (_i1220 = 0; _i1220 < _size1216; ++_i1220)
             {
-              xfer += iprot->readString(this->success[_i1227]);
+              xfer += iprot->readString(this->success[_i1220]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7898,10 +7898,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 _iter1228;
-      for (_iter1228 = this->success.begin(); _iter1228 != this->success.end(); ++_iter1228)
+      std::vector<std::string> ::const_iterator _iter1221;
+      for (_iter1221 = this->success.begin(); _iter1221 != this->success.end(); ++_iter1221)
       {
-        xfer += oprot->writeString((*_iter1228));
+        xfer += oprot->writeString((*_iter1221));
       }
       xfer += oprot->writeListEnd();
     }
@@ -7946,14 +7946,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1229;
-            ::apache::thrift::protocol::TType _etype1232;
-            xfer += iprot->readListBegin(_etype1232, _size1229);
-            (*(this->success)).resize(_size1229);
-            uint32_t _i1233;
-            for (_i1233 = 0; _i1233 < _size1229; ++_i1233)
+            uint32_t _size1222;
+            ::apache::thrift::protocol::TType _etype1225;
+            xfer += iprot->readListBegin(_etype1225, _size1222);
+            (*(this->success)).resize(_size1222);
+            uint32_t _i1226;
+            for (_i1226 = 0; _i1226 < _size1222; ++_i1226)
             {
-              xfer += iprot->readString((*(this->success))[_i1233]);
+              xfer += iprot->readString((*(this->success))[_i1226]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8263,14 +8263,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 _size1234;
-            ::apache::thrift::protocol::TType _etype1237;
-            xfer += iprot->readListBegin(_etype1237, _size1234);
-            this->tbl_names.resize(_size1234);
-            uint32_t _i1238;
-            for (_i1238 = 0; _i1238 < _size1234; ++_i1238)
+            uint32_t _size1227;
+            ::apache::thrift::protocol::TType _etype1230;
+            xfer += iprot->readListBegin(_etype1230, _size1227);
+            this->tbl_names.resize(_size1227);
+            uint32_t _i1231;
+            for (_i1231 = 0; _i1231 < _size1227; ++_i1231)
             {
-              xfer += iprot->readString(this->tbl_names[_i1238]);
+              xfer += iprot->readString(this->tbl_names[_i1231]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8303,10 +8303,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 _iter1239;
-    for (_iter1239 = this->tbl_names.begin(); _iter1239 != this->tbl_names.end(); ++_iter1239)
+    std::vector<std::string> ::const_iterator _iter1232;
+    for (_iter1232 = this->tbl_names.begin(); _iter1232 != this->tbl_names.end(); ++_iter1232)
     {
-      xfer += oprot->writeString((*_iter1239));
+      xfer += oprot->writeString((*_iter1232));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8334,10 +8334,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 _iter1240;
-    for (_iter1240 = (*(this->tbl_names)).begin(); _iter1240 != (*(this->tbl_names)).end(); ++_iter1240)
+    std::vector<std::string> ::const_iterator _iter1233;
+    for (_iter1233 = (*(this->tbl_names)).begin(); _iter1233 != (*(this->tbl_names)).end(); ++_iter1233)
     {
-      xfer += oprot->writeString((*_iter1240));
+      xfer += oprot->writeString((*_iter1233));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8378,14 +8378,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 _size1241;
-            ::apache::thrift::protocol::TType _etype1244;
-            xfer += iprot->readListBegin(_etype1244, _size1241);
-            this->success.resize(_size1241);
-            uint32_t _i1245;
-            for (_i1245 = 0; _i1245 < _size1241; ++_i1245)
+            uint32_t _size1234;
+            ::apache::thrift::protocol::TType _etype1237;
+            xfer += iprot->readListBegin(_etype1237, _size1234);
+            this->success.resize(_size1234);
+            uint32_t _i1238;
+            for (_i1238 = 0; _i1238 < _size1234; ++_i1238)
             {
-              xfer += this->success[_i1245].read(iprot);
+              xfer += this->success[_i1238].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8416,10 +8416,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 _iter1246;
-      for (_iter1246 = this->success.begin(); _iter1246 != this->success.end(); ++_iter1246)
+      std::vector<Table> ::const_iterator _iter1239;
+      for (_iter1239 = this->success.begin(); _iter1239 != this->success.end(); ++_iter1239)
       {
-        xfer += (*_iter1246).write(oprot);
+        xfer += (*_iter1239).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -8460,14 +8460,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 _size1247;
-            ::apache::thrift::protocol::TType _etype1250;
-            xfer += iprot->readListBegin(_etype1250, _size1247);
-            (*(this->success)).resize(_size1247);
-            uint32_t _i1251;
-            for (_i1251 = 0; _i1251 < _size1247; ++_i1251)
+            uint32_t _size1240;
+            ::apache::thrift::protocol::TType _etype1243;
+            xfer += iprot->readListBegin(_etype1243, _size1240);
+            (*(this->success)).resize(_size1240);
+            uint32_t _i1244;
+            for (_i1244 = 0; _i1244 < _size1240; ++_i1244)
             {
-              xfer += (*(this->success))[_i1251].read(iprot);
+              xfer += (*(this->success))[_i1244].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9000,14 +9000,14 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_args::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_names.clear();
-            uint32_t _size1252;
-            ::apache::thrift::protocol::TType _etype1255;
-            xfer += iprot->readListBegin(_etype1255, _size1252);
-            this->tbl_names.resize(_size1252);
-            uint32_t _i1256;
-            for (_i1256 = 0; _i1256 < _size1252; ++_i1256)
+            uint32_t _size1245;
+            ::apache::thrift::protocol::TType _etype1248;
+            xfer += iprot->readListBegin(_etype1248, _size1245);
+            this->tbl_names.resize(_size1245);
+            uint32_t _i1249;
+            for (_i1249 = 0; _i1249 < _size1245; ++_i1249)
             {
-              xfer += iprot->readString(this->tbl_names[_i1256]);
+              xfer += iprot->readString(this->tbl_names[_i1249]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9040,10 +9040,10 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_args::write(:
   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 _iter1257;
-    for (_iter1257 = this->tbl_names.begin(); _iter1257 != this->tbl_names.end(); ++_iter1257)
+    std::vector<std::string> ::const_iterator _iter1250;
+    for (_iter1250 = this->tbl_names.begin(); _iter1250 != this->tbl_names.end(); ++_iter1250)
     {
-      xfer += oprot->writeString((*_iter1257));
+      xfer += oprot->writeString((*_iter1250));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9071,10 +9071,10 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_pargs::write(
   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 _iter1258;
-    for (_iter1258 = (*(this->tbl_names)).begin(); _iter1258 != (*(this->tbl_names)).end(); ++_iter1258)
+    std::vector<std::string> ::const_iterator _iter1251;
+    for (_iter1251 = (*(this->tbl_names)).begin(); _iter1251 != (*(this->tbl_names)).end(); ++_iter1251)
     {
-      xfer += oprot->writeString((*_iter1258));
+      xfer += oprot->writeString((*_iter1251));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9115,17 +9115,17 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_result::read(
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size1259;
-            ::apache::thrift::protocol::TType _ktype1260;
-            ::apache::thrift::protocol::TType _vtype1261;
-            xfer += iprot->readMapBegin(_ktype1260, _vtype1261, _size1259);
-            uint32_t _i1263;
-            for (_i1263 = 0; _i1263 < _size1259; ++_i1263)
+            uint32_t _size1252;
+            ::apache::thrift::protocol::TType _ktype1253;
+            ::apache::thrift::protocol::TType _vtype1254;
+            xfer += iprot->readMapBegin(_ktype1253, _vtype1254, _size1252);
+            uint32_t _i1256;
+            for (_i1256 = 0; _i1256 < _size1252; ++_i1256)
             {
-              std::string _key1264;
-              xfer += iprot->readString(_key1264);
-              Materialization& _val1265 = this->success[_key1264];
-              xfer += _val1265.read(iprot);
+              std::string _key1257;
+              xfer += iprot->readString(_key1257);
+              Materialization& _val1258 = this->success[_key1257];
+              xfer += _val1258.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -9180,11 +9180,11 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_result::write
     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, Materialization> ::const_iterator _iter1266;
-      for (_iter1266 = this->success.begin(); _iter1266 != this->success.end(); ++_iter1266)
+      std::map<std::string, Materialization> ::const_iterator _iter1259;
+      for (_iter1259 = this->success.begin(); _iter1259 != this->success.end(); ++_iter1259)
       {
-        xfer += oprot->writeString(_iter1266->first);
-        xfer += _iter1266->second.write(oprot);
+        xfer += oprot->writeString(_iter1259->first);
+        xfer += _iter1259->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -9237,17 +9237,17 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_presult::read
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size1267;
-            ::apache::thrift::protocol::TType _ktype1268;
-            ::apache::thrift::protocol::TType _vtype1269;
-            xfer += iprot->readMapBegin(_ktype1268, _vtype1269, _size1267);
-            uint32_t _i1271;
-            for (_i1271 = 0; _i1271 < _size1267; ++_i1271)
+            uint32_t _size1260;
+            ::apache::thrift::protocol::TType _ktype1261;
+            ::apache::thrift::protocol::TType _vtype1262;
+            xfer += iprot->readMapBegin(_ktype1261, _vtype1262, _size1260);
+            uint32_t _i1264;
+            for (_i1264 = 0; _i1264 < _size1260; ++_i1264)
             {
-              std::string _key1272;
-              xfer += iprot->readString(_key1272);
-              Materialization& _val1273 = (*(this->success))[_key1272];
-              xfer += _val1273.read(iprot);
+              std::string _key1265;
+              xfer += iprot->readString(_key1265);
+              Materialization& _val1266 = (*(this->success))[_key1265];
+              xfer += _val1266.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -9433,14 +9433,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 _size1274;
-            ::apache::thrift::protocol::TType _etype1277;
-            xfer += iprot->readListBegin(_etype1277, _size1274);
-            this->success.resize(_size1274);
-            uint32_t _i1278;
-            for (_i1278 = 0; _i1278 < _size1274; ++_i1278)
+            uint32_t _size1267;
+            ::apache::thrift::protocol::TType _etype1270;
+            xfer += iprot->readListBegin(_etype1270, _size1267);
+            this->success.resize(_size1267);
+            uint32_t _i1271;
+            for (_i1271 = 0; _i1271 < _size1267; ++_i1271)
             {
-              xfer += iprot->readString(this->success[_i1278]);
+              xfer += iprot->readString(this->success[_i1271]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9495,10 +9495,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 _iter1279;
-      for (_iter1279 = this->success.begin(); _iter1279 != this->success.end(); ++_iter1279)
+      std::vector<std::string> ::const_iterator _iter1272;
+      for (_iter1272 = this->success.begin(); _iter1272 != this->success.end(); ++_iter1272)
       {
-        xfer += oprot->writeString((*_iter1279));
+        xfer += oprot->writeString((*_iter1272));
       }
       xfer += oprot->writeListEnd();
     }
@@ -9551,14 +9551,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 _size1280;
-            ::apache::thrift::protocol::TType _etype1283;
-            xfer += iprot->readListBegin(_etype1283, _size1280);
-            (*(this->success)).resize(_size1280);
-            uint32_t _i1284;
-            for (_i1284 = 0; _i1284 < _size1280; ++_i1284)
+            uint32_t _size1273;
+            ::apache::thrift::protocol::TType _etype1276;
+            xfer += iprot->readListBegin(_etype1276, _size1273);
+            (*(this->success)).resize(_size1273);
+            uint32_t _i1277;
+            for (_i1277 = 0; _i1277 < _size1273; ++_i1277)
             {
-              xfer += iprot->readString((*(this->success))[_i1284]);
+              xfer += iprot->readString((*(this->success))[_i1277]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10892,14 +10892,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size1285;
-            ::apache::thrift::protocol::TType _etype1288;
-            xfer += iprot->readListBegin(_etype1288, _size1285);
-            this->new_parts.resize(_size1285);
-            uint32_t _i1289;
-            for (_i1289 = 0; _i1289 < _size1285; ++_i1289)
+            uint32_t _size1278;
+            ::apache::thrift::protocol::TType _etype1281;
+            xfer += iprot->readListBegin(_etype1281, _size1278);
+            this->new_parts.resize(_size1278);
+            uint32_t _i1282;
+            for (_i1282 = 0; _i1282 < _size1278; ++_i1282)
             {
-              xfer += this->new_parts[_i1289].read(iprot);
+              xfer += this->new_parts[_i1282].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -10928,10 +10928,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 _iter1290;
-    for (_iter1290 = this->new_parts.begin(); _iter1290 != this->new_parts.end(); ++_iter1290)
+    std::vector<Partition> ::const_iterator _iter1283;
+    for (_iter1283 = this->new_parts.begin(); _iter1283 != this->new_parts.end(); ++_iter1283)
     {
-      xfer += (*_iter1290).write(oprot);
+      xfer += (*_iter1283).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -10955,10 +10955,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 _iter1291;
-    for (_iter1291 = (*(this->new_parts)).begin(); _iter1291 != (*(this->new_parts)).end(); ++_iter1291)
+    std::vector<Partition> ::const_iterator _iter1284;
+    for (_iter1284 = (*(this->new_parts)).begin(); _iter1284 != (*(this->new_parts)).end(); ++_iter1284)
     {
-      xfer += (*_iter1291).write(oprot);
+      xfer += (*_iter1284).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -11167,14 +11167,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 _size1292;
-            ::apache::thrift::protocol::TType _etype1295;
-            xfer += iprot->readListBegin(_etype1295, _size1292);
-            this->new_parts.resize(_size1292);
-            uint32_t _i1296;
-            for (_i1296 = 0; _i1296 < _size1292; ++_i1296)
+            uint32_t _size1285;
+            ::apache::thrift::protocol::TType _etype1288;
+            xfer += iprot->readListBegin(_etype1288, _size1285);
+            this->new_parts.resize(_size1285);
+            uint32_t _i1289;
+            for (_i1289 = 0; _i1289 < _size1285; ++_i1289)
             {
-              xfer += this->new_parts[_i1296].read(iprot);
+              xfer += this->new_parts[_i1289].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -11203,10 +11203,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 _iter1297;
-    for (_iter1297 = this->new_parts.begin(); _iter1297 != this->new_parts.end(); ++_iter1297)
+    std::vector<PartitionSpec> ::const_iterator _iter1290;
+    for (_iter1290 = this->new_parts.begin(); _iter1290 != this->new_parts.end(); ++_iter1290)
     {
-      xfer += (*_iter1297).write(oprot);
+      xfer += (*_iter1290).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -11230,10 +11230,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 _iter1298;
-    for (_iter1298 = (*(this->new_parts)).begin(); _iter1298 != (*(this->new_parts)).end(); ++_iter1298)
+    std::vector<PartitionSpec> ::const_iterator _iter1291;
+    for (_iter1291 = (*(this->new_parts)).begin(); _iter1291 != (*(this->new_parts)).end(); ++_iter1291)
     {
-      xfer += (*_iter1298).write(oprot);
+      xfer += (*_iter1291).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -11458,14 +11458,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1299;
-            ::apache::thrift::protocol::TType _etype1302;
-            xfer += iprot->readListBegin(_etype1302, _size1299);
-            this->part_vals.resize(_size1299);
-            uint32_t _i1303;
-            for (_i1303 = 0; _i1303 < _size1299; ++_i1303)
+            uint32_t _size1292;
+            ::apache::thrift::protocol::TType _etype1295;
+            xfer += iprot->readListBegin(_etype1295, _size1292);
+            this->part_vals.resize(_size1292);
+            uint32_t _i1296;
+            for (_i1296 = 0; _i1296 < _size1292; ++_i1296)
             {
-              xfer += iprot->readString(this->part_vals[_i1303]);
+              xfer += iprot->readString(this->part_vals[_i1296]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11502,10 +11502,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 _iter1304;
-    for (_iter1304 = this->part_vals.begin(); _iter1304 != this->part_vals.end(); ++_iter1304)
+    std::vector<std::string> ::const_iterator _iter1297;
+    for (_iter1297 = this->part_vals.begin(); _iter1297 != this->part_vals.end(); ++_iter1297)
     {
-      xfer += oprot->writeString((*_iter1304));
+      xfer += oprot->writeString((*_iter1297));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11537,10 +11537,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 _iter1305;
-    for (_iter1305 = (*(this->part_vals)).begin(); _iter1305 != (*(this->part_vals)).end(); ++_iter1305)
+    std::vector<std::string> ::const_iterator _iter1298;
+    for (_iter1298 = (*(this->part_vals)).begin(); _iter1298 != (*(this->part_vals)).end(); ++_iter1298)
     {
-      xfer += oprot->writeString((*_iter1305));
+      xfer += oprot->writeString((*_iter1298));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12012,14 +12012,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1306;
-            ::apache::thrift::protocol::TType _etype1309;
-            xfer += iprot->readListBegin(_etype1309, _size1306);
-            this->part_vals.resize(_size1306);
-            uint32_t _i1310;
-            for (_i1310 = 0; _i1310 < _size1306; ++_i1310)
+            uint32_t _size1299;
+            ::apache::thrift::protocol::TType _etype1302;
+            xfer += iprot->readListBegin(_etype1302, _size1299);
+            this->part_vals.resize(_size1299);
+            uint32_t _i1303;
+            for (_i1303 = 0; _i1303 < _size1299; ++_i1303)
             {
-              xfer += iprot->readString(this->part_vals[_i1310]);
+              xfer += iprot->readString(this->part_vals[_i1303]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12064,10 +12064,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 _iter1311;
-    for (_iter1311 = this->part_vals.begin(); _iter1311 != this->part_vals.end(); ++_iter1311)
+    std::vector<std::string> ::const_iterator _iter1304;
+    for (_iter1304 = this->part_vals.begin(); _iter1304 != this->part_vals.end(); ++_iter1304)
     {
-      xfer += oprot->writeString((*_iter1311));
+      xfer += oprot->writeString((*_iter1304));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12103,10 +12103,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 _iter1312;
-    for (_iter1312 = (*(this->part_vals)).begin(); _iter1312 != (*(this->part_vals)).end(); ++_iter1312)
+    std::vector<std::string> ::const_iterator _iter1305;
+    for (_iter1305 = (*(this->part_vals)).begin(); _iter1305 != (*(this->part_vals)).end(); ++_iter1305)
     {
-      xfer += oprot->writeString((*_iter1312));
+      xfer += oprot->writeString((*_iter1305));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12909,14 +12909,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1313;
-            ::apache::thrift::protocol::TType _etype1316;
-            xfer += iprot->readListBegin(_etype1316, _size1313);
-            this->part_vals.resize(_size1313);
-            uint32_t _i1317;
-            for (_i1317 = 0; _i1317 < _size1313; ++_i1317)
+            uint32_t _size1306;
+            ::apache::thrift::protocol::TType _etype1309;
+            xfer += iprot->readListBegin(_etype1309, _size1306);
+            this->part_vals.resize(_size1306);
+            uint32_t _i1310;
+            for (_i1310 = 0; _i1310 < _size1306; ++_i1310)
             {
-              xfer += iprot->readString(this->part_vals[_i1317]);
+              xfer += iprot->readString(this->part_vals[_i1310]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12961,10 +12961,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 _iter1318;
-    for (_iter1318 = this->part_vals.begin(); _iter1318 != this->part_vals.end(); ++_iter1318)
+    std::vector<std::string> ::const_iterator _iter1311;
+    for (_iter1311 = this->part_vals.begin(); _iter1311 != this->part_vals.end(); ++_iter1311)
     {
-      xfer += oprot->writeString((*_iter1318));
+      xfer += oprot->writeString((*_iter1311));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13000,10 +13000,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 _iter1319;
-    for (_iter1319 = (*(this->part_vals)).begin(); _iter1319 != (*(this->part_vals)).end(); ++_iter1319)
+    std::vector<std::string> ::const_iterator _iter1312;
+    for (_iter1312 = (*(this->part_vals)).begin(); _iter1312 != (*(this->part_vals)).end(); ++_iter1312)
     {
-      xfer += oprot->writeString((*_iter1319));
+      xfer += oprot->writeString((*_iter1312));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13212,14 +13212,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1320;
-            ::apache::thrift::protocol::TType _etype1323;
-            xfer += iprot->readListBegin(_etype1323, _size1320);
-            this->part_vals.resize(_size1320);
-            uint32_t _i1324;
-            for (_i1324 = 0; _i1324 < _size1320; ++_i1324)
+            uint32_t _size1313;
+            ::apache::thrift::protocol::TType _etype1316;
+            xfer += iprot->readListBegin(_etype1316, _size1313);
+            this->part_vals.resize(_size1313);
+            uint32_t _i1317;
+            for (_i1317 = 0; _i1317 < _size1313; ++_i1317)
             {
-              xfer += iprot->readString(this->part_vals[_i1324]);
+              xfer += iprot->readString(this->part_vals[_i1317]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13272,10 +13272,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 _iter1325;
-    for (_iter1325 = this->part_vals.begin(); _iter1325 != this->part_vals.end(); ++_iter1325)
+    std::vector<std::string> ::const_iterator _iter1318;
+    for (_iter1318 = this->part_vals.begin(); _iter1318 != this->part_vals.end(); ++_iter1318)
     {
-      xfer += oprot->writeString((*_iter1325));
+      xfer += oprot->writeString((*_iter1318));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13315,10 +13315,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 _iter1326;
-    for (_iter1326 = (*(this->part_vals)).begin(); _iter1326 != (*(this->part_vals)).end(); ++_iter1326)
+    std::vector<std::string> ::const_iterator _iter1319;
+    for (_iter1319 = (*(this->part_vals)).begin(); _iter1319 != (*(this->part_vals)).end(); ++_iter1319)
     {
-      xfer += oprot->writeString((*_iter1326));
+      xfer += oprot->writeString((*_iter1319));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14324,14 +14324,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1327;
-            ::apache::thrift::protocol::TType _etype1330;
-            xfer += iprot->readListBegin(_etype1330, _size1327);
-            this->part_vals.resize(_size1327);
-            uint32_t _i1331;
-            for (_i1331 = 0; _i1331 < _size1327; ++_i1331)
+            uint32_t _size1320;
+            ::apache::thrift::protocol::TType _etype1323;
+            xfer += iprot->readListBegin(_etype1323, _size1320);
+            this->part_vals.resize(_size1320);
+            uint32_t _i1324;
+            for (_i1324 = 0; _i1324 < _size1320; ++_i1324)
             {
-              xfer += iprot->readString(this->part_vals[_i1331]);
+              xfer += iprot->readString(this->part_vals[_i1324]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14368,10 +14368,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 _iter1332;
-    for (_iter1332 = this->part_vals.begin(); _iter1332 != this->part_vals.end(); ++_iter1332)
+    std::vector<std::string> ::const_iterator _iter1325;
+    for (_iter1325 = this->part_vals.begin(); _iter1325 != this->part_vals.end(); ++_iter1325)
     {
-      xfer += oprot->writeString((*_iter1332));
+      xfer += oprot->writeString((*_iter1325));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14403,10 +14403,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 _iter1333;
-    for (_iter1333 = (*(this->part_vals)).begin(); _iter1333 != (*(this->part_vals)).end(); ++_iter1333)
+    std::vector<std::string> ::const_iterator _iter1326;
+    for (_iter1326 = (*(this->part_vals)).begin(); _iter1326 != (*(this->part_vals)).end(); ++_iter1326)
     {
-      xfer += oprot->writeString((*_iter1333));
+      xfer += oprot->writeString((*_iter1326));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14595,17 +14595,17 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size1334;
-            ::apache::thrift::protocol::TType _ktype1335;
-            ::apache::thrift::protocol::TType _vtype1336;
-            xfer += iprot->readMapBegin(_ktype1335, _vtype1336, _size1334);
-            uint32_t _i1338;
-            for (_i1338 = 0; _i1338 < _size1334; ++_i1338)
+            uint32_t _size1327;
+            ::apache::thrift::protocol::TType _ktype1328;
+            ::apache::thrift::protocol::TType _vtype1329;
+            xfer += iprot->readMapBegin(_ktype1328, _vtype1329, _size1327);
+            uint32_t _i1331;
+            for (_i1331 = 0; _i1331 < _size1327; ++_i1331)
             {
-              std::string _key1339;
-              xfer += iprot->readString(_key1339);
-              std::string& _val1340 = this->partitionSpecs[_key1339];
-              xfer += iprot->readString(_val1340);
+              std::string _key1332;
+              xfer += iprot->readString(_key1332);
+              std::string& _val1333 = this->partitionSpecs[_key1332];
+              xfer += iprot->readString(_val1333);
             }
             xfer += iprot->readMapEnd();
           }
@@ -14666,11 +14666,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 _iter1341;
-    for (_iter1341 = this->partitionSpecs.begin(); _iter1341 != this->partitionSpecs.end(); ++_iter1341)
+    std::map<std::string, std::string> ::const_iterator _iter1334;
+    for (_iter1334 = this->partitionSpecs.begin(); _iter1334 != this->partitionSpecs.end(); ++_iter1334)
     {
-      xfer += oprot->writeString(_iter1341->first);
-      xfer += oprot->writeString(_iter1341->second);
+      xfer += oprot->writeString(_iter1334->first);
+      xfer += oprot->writeString(_iter1334->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -14710,11 +14710,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 _iter1342;
-    for (_iter1342 = (*(this->partitionSpecs)).begin(); _iter1342 != (*(this->partitionSpecs)).end(); ++_iter1342)
+    std::map<std::string, std::string> ::const_iterator _iter1335;
+    for (_iter1335 = (*(this->partitionSpecs)).begin(); _iter1335 != (*(this->partitionSpecs)).end(); ++_iter1335)
     {
-      xfer += oprot->writeString(_iter1342->first);
-      xfer += oprot->writeString(_iter1342->second);
+      xfer += oprot->writeString(_iter1335->first);
+      xfer += oprot->writeString(_iter1335->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -14959,17 +14959,17 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size1343;
-            ::apache::thrift::protocol::TType _ktype1344;
-            ::apache::thrift::protocol::TType _vtype1345;
-            xfer += iprot->readMapBegin(_ktype1344, _vtype1345, _size1343);
-            uint32_t _i1347;
-            for (_i1347 = 0; _i1347 < _size1343; ++_i1347)
+            uint32_t _size1336;
+            ::apache::thrift::protocol::TType _ktype1337;
+            ::apache::thrift::protocol::TType _vtype1338;
+            xfer += iprot->readMapBegin(_ktype1337, _vtype1338, _size1336);
+            uint32_t _i1340;
+            for (_i1340 = 0; _i1340 < _size1336; ++_i1340)
             {
-              std::string _key1348;
-              xfer += iprot->readString(_key1348);
-              std::string& _val1349 = this->partitionSpecs[_key1348];
-              xfer += iprot->readString(_val1349);
+              std::string _key1341;
+              xfer += iprot->readString(_key1341);
+              std::string& _val1342 = this->partitionSpecs[_key1341];
+              xfer += iprot->readString(_val1342);
             }
             xfer += iprot->readMapEnd();
           }
@@ -15030,11 +15030,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 _iter1350;
-    for (_iter1350 = this->partitionSpecs.begin(); _iter1350 != this->partitionSpecs.end(); ++_iter1350)
+    std::map<std::string, std::string> ::const_iterator _iter1343;
+    for (_iter1343 = this->partitionSpecs.begin(); _iter1343 != this->partitionSpecs.end(); ++_iter1343)
     {
-      xfer += oprot->writeString(_iter1350->fir

<TRUNCATED>

[08/16] hive git commit: HIVE-18546: Remove unnecessary code introduced in HIVE-14498 (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index aaca408..d5e3527 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@ -346,10 +346,6 @@ import org.slf4j.LoggerFactory;
 
     public void add_dynamic_partitions(AddDynamicPartitions rqst) throws NoSuchTxnException, TxnAbortedException, org.apache.thrift.TException;
 
-    public List<BasicTxnInfo> get_last_completed_transaction_for_tables(List<String> db_names, List<String> table_names, TxnsSnapshot txns_snapshot) throws org.apache.thrift.TException;
-
-    public BasicTxnInfo get_last_completed_transaction_for_table(String db_name, String table_name, TxnsSnapshot txns_snapshot) throws org.apache.thrift.TException;
-
     public NotificationEventResponse get_next_notification(NotificationEventRequest rqst) throws org.apache.thrift.TException;
 
     public CurrentNotificationEventId get_current_notificationEventId() throws org.apache.thrift.TException;
@@ -716,10 +712,6 @@ import org.slf4j.LoggerFactory;
 
     public void add_dynamic_partitions(AddDynamicPartitions rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
-    public void get_last_completed_transaction_for_tables(List<String> db_names, List<String> table_names, TxnsSnapshot txns_snapshot, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
-
-    public void get_last_completed_transaction_for_table(String db_name, String table_name, TxnsSnapshot txns_snapshot, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
-
     public void get_next_notification(NotificationEventRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
     public void get_current_notificationEventId(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
@@ -5228,56 +5220,6 @@ import org.slf4j.LoggerFactory;
       return;
     }
 
-    public List<BasicTxnInfo> get_last_completed_transaction_for_tables(List<String> db_names, List<String> table_names, TxnsSnapshot txns_snapshot) throws org.apache.thrift.TException
-    {
-      send_get_last_completed_transaction_for_tables(db_names, table_names, txns_snapshot);
-      return recv_get_last_completed_transaction_for_tables();
-    }
-
-    public void send_get_last_completed_transaction_for_tables(List<String> db_names, List<String> table_names, TxnsSnapshot txns_snapshot) throws org.apache.thrift.TException
-    {
-      get_last_completed_transaction_for_tables_args args = new get_last_completed_transaction_for_tables_args();
-      args.setDb_names(db_names);
-      args.setTable_names(table_names);
-      args.setTxns_snapshot(txns_snapshot);
-      sendBase("get_last_completed_transaction_for_tables", args);
-    }
-
-    public List<BasicTxnInfo> recv_get_last_completed_transaction_for_tables() throws org.apache.thrift.TException
-    {
-      get_last_completed_transaction_for_tables_result result = new get_last_completed_transaction_for_tables_result();
-      receiveBase(result, "get_last_completed_transaction_for_tables");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_last_completed_transaction_for_tables failed: unknown result");
-    }
-
-    public BasicTxnInfo get_last_completed_transaction_for_table(String db_name, String table_name, TxnsSnapshot txns_snapshot) throws org.apache.thrift.TException
-    {
-      send_get_last_completed_transaction_for_table(db_name, table_name, txns_snapshot);
-      return recv_get_last_completed_transaction_for_table();
-    }
-
-    public void send_get_last_completed_transaction_for_table(String db_name, String table_name, TxnsSnapshot txns_snapshot) throws org.apache.thrift.TException
-    {
-      get_last_completed_transaction_for_table_args args = new get_last_completed_transaction_for_table_args();
-      args.setDb_name(db_name);
-      args.setTable_name(table_name);
-      args.setTxns_snapshot(txns_snapshot);
-      sendBase("get_last_completed_transaction_for_table", args);
-    }
-
-    public BasicTxnInfo recv_get_last_completed_transaction_for_table() throws org.apache.thrift.TException
-    {
-      get_last_completed_transaction_for_table_result result = new get_last_completed_transaction_for_table_result();
-      receiveBase(result, "get_last_completed_transaction_for_table");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_last_completed_transaction_for_table failed: unknown result");
-    }
-
     public NotificationEventResponse get_next_notification(NotificationEventRequest rqst) throws org.apache.thrift.TException
     {
       send_get_next_notification(rqst);
@@ -11484,82 +11426,6 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public void get_last_completed_transaction_for_tables(List<String> db_names, List<String> table_names, TxnsSnapshot txns_snapshot, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      get_last_completed_transaction_for_tables_call method_call = new get_last_completed_transaction_for_tables_call(db_names, table_names, txns_snapshot, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_last_completed_transaction_for_tables_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private List<String> db_names;
-      private List<String> table_names;
-      private TxnsSnapshot txns_snapshot;
-      public get_last_completed_transaction_for_tables_call(List<String> db_names, List<String> table_names, TxnsSnapshot txns_snapshot, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
-        super(client, protocolFactory, transport, resultHandler, false);
-        this.db_names = db_names;
-        this.table_names = table_names;
-        this.txns_snapshot = txns_snapshot;
-      }
-
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_last_completed_transaction_for_tables", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        get_last_completed_transaction_for_tables_args args = new get_last_completed_transaction_for_tables_args();
-        args.setDb_names(db_names);
-        args.setTable_names(table_names);
-        args.setTxns_snapshot(txns_snapshot);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      public List<BasicTxnInfo> getResult() throws org.apache.thrift.TException {
-        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
-          throw new IllegalStateException("Method call not finished!");
-        }
-        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
-        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_get_last_completed_transaction_for_tables();
-      }
-    }
-
-    public void get_last_completed_transaction_for_table(String db_name, String table_name, TxnsSnapshot txns_snapshot, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      get_last_completed_transaction_for_table_call method_call = new get_last_completed_transaction_for_table_call(db_name, table_name, txns_snapshot, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_last_completed_transaction_for_table_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private String db_name;
-      private String table_name;
-      private TxnsSnapshot txns_snapshot;
-      public get_last_completed_transaction_for_table_call(String db_name, String table_name, TxnsSnapshot txns_snapshot, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
-        super(client, protocolFactory, transport, resultHandler, false);
-        this.db_name = db_name;
-        this.table_name = table_name;
-        this.txns_snapshot = txns_snapshot;
-      }
-
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_last_completed_transaction_for_table", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        get_last_completed_transaction_for_table_args args = new get_last_completed_transaction_for_table_args();
-        args.setDb_name(db_name);
-        args.setTable_name(table_name);
-        args.setTxns_snapshot(txns_snapshot);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      public BasicTxnInfo getResult() throws org.apache.thrift.TException {
-        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
-          throw new IllegalStateException("Method call not finished!");
-        }
-        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
-        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_get_last_completed_transaction_for_table();
-      }
-    }
-
     public void get_next_notification(NotificationEventRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
       get_next_notification_call method_call = new get_next_notification_call(rqst, resultHandler, this, ___protocolFactory, ___transport);
@@ -12644,8 +12510,6 @@ import org.slf4j.LoggerFactory;
       processMap.put("compact2", new compact2());
       processMap.put("show_compact", new show_compact());
       processMap.put("add_dynamic_partitions", new add_dynamic_partitions());
-      processMap.put("get_last_completed_transaction_for_tables", new get_last_completed_transaction_for_tables());
-      processMap.put("get_last_completed_transaction_for_table", new get_last_completed_transaction_for_table());
       processMap.put("get_next_notification", new get_next_notification());
       processMap.put("get_current_notificationEventId", new get_current_notificationEventId());
       processMap.put("get_notification_events_count", new get_notification_events_count());
@@ -16602,46 +16466,6 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_last_completed_transaction_for_tables<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_last_completed_transaction_for_tables_args> {
-      public get_last_completed_transaction_for_tables() {
-        super("get_last_completed_transaction_for_tables");
-      }
-
-      public get_last_completed_transaction_for_tables_args getEmptyArgsInstance() {
-        return new get_last_completed_transaction_for_tables_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public get_last_completed_transaction_for_tables_result getResult(I iface, get_last_completed_transaction_for_tables_args args) throws org.apache.thrift.TException {
-        get_last_completed_transaction_for_tables_result result = new get_last_completed_transaction_for_tables_result();
-        result.success = iface.get_last_completed_transaction_for_tables(args.db_names, args.table_names, args.txns_snapshot);
-        return result;
-      }
-    }
-
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_last_completed_transaction_for_table<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_last_completed_transaction_for_table_args> {
-      public get_last_completed_transaction_for_table() {
-        super("get_last_completed_transaction_for_table");
-      }
-
-      public get_last_completed_transaction_for_table_args getEmptyArgsInstance() {
-        return new get_last_completed_transaction_for_table_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public get_last_completed_transaction_for_table_result getResult(I iface, get_last_completed_transaction_for_table_args args) throws org.apache.thrift.TException {
-        get_last_completed_transaction_for_table_result result = new get_last_completed_transaction_for_table_result();
-        result.success = iface.get_last_completed_transaction_for_table(args.db_name, args.table_name, args.txns_snapshot);
-        return result;
-      }
-    }
-
     @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_next_notification<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_next_notification_args> {
       public get_next_notification() {
         super("get_next_notification");
@@ -17527,8 +17351,6 @@ import org.slf4j.LoggerFactory;
       processMap.put("compact2", new compact2());
       processMap.put("show_compact", new show_compact());
       processMap.put("add_dynamic_partitions", new add_dynamic_partitions());
-      processMap.put("get_last_completed_transaction_for_tables", new get_last_completed_transaction_for_tables());
-      processMap.put("get_last_completed_transaction_for_table", new get_last_completed_transaction_for_table());
       processMap.put("get_next_notification", new get_next_notification());
       processMap.put("get_current_notificationEventId", new get_current_notificationEventId());
       processMap.put("get_notification_events_count", new get_notification_events_count());
@@ -26896,108 +26718,6 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_last_completed_transaction_for_tables<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_last_completed_transaction_for_tables_args, List<BasicTxnInfo>> {
-      public get_last_completed_transaction_for_tables() {
-        super("get_last_completed_transaction_for_tables");
-      }
-
-      public get_last_completed_transaction_for_tables_args getEmptyArgsInstance() {
-        return new get_last_completed_transaction_for_tables_args();
-      }
-
-      public AsyncMethodCallback<List<BasicTxnInfo>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<List<BasicTxnInfo>>() { 
-          public void onComplete(List<BasicTxnInfo> o) {
-            get_last_completed_transaction_for_tables_result result = new get_last_completed_transaction_for_tables_result();
-            result.success = o;
-            try {
-              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
-              return;
-            } catch (Exception e) {
-              LOGGER.error("Exception writing to internal frame buffer", e);
-            }
-            fb.close();
-          }
-          public void onError(Exception e) {
-            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
-            org.apache.thrift.TBase msg;
-            get_last_completed_transaction_for_tables_result result = new get_last_completed_transaction_for_tables_result();
-            {
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
-            }
-            try {
-              fcall.sendResponse(fb,msg,msgType,seqid);
-              return;
-            } catch (Exception ex) {
-              LOGGER.error("Exception writing to internal frame buffer", ex);
-            }
-            fb.close();
-          }
-        };
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public void start(I iface, get_last_completed_transaction_for_tables_args args, org.apache.thrift.async.AsyncMethodCallback<List<BasicTxnInfo>> resultHandler) throws TException {
-        iface.get_last_completed_transaction_for_tables(args.db_names, args.table_names, args.txns_snapshot,resultHandler);
-      }
-    }
-
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_last_completed_transaction_for_table<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_last_completed_transaction_for_table_args, BasicTxnInfo> {
-      public get_last_completed_transaction_for_table() {
-        super("get_last_completed_transaction_for_table");
-      }
-
-      public get_last_completed_transaction_for_table_args getEmptyArgsInstance() {
-        return new get_last_completed_transaction_for_table_args();
-      }
-
-      public AsyncMethodCallback<BasicTxnInfo> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<BasicTxnInfo>() { 
-          public void onComplete(BasicTxnInfo o) {
-            get_last_completed_transaction_for_table_result result = new get_last_completed_transaction_for_table_result();
-            result.success = o;
-            try {
-              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
-              return;
-            } catch (Exception e) {
-              LOGGER.error("Exception writing to internal frame buffer", e);
-            }
-            fb.close();
-          }
-          public void onError(Exception e) {
-            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
-            org.apache.thrift.TBase msg;
-            get_last_completed_transaction_for_table_result result = new get_last_completed_transaction_for_table_result();
-            {
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
-            }
-            try {
-              fcall.sendResponse(fb,msg,msgType,seqid);
-              return;
-            } catch (Exception ex) {
-              LOGGER.error("Exception writing to internal frame buffer", ex);
-            }
-            fb.close();
-          }
-        };
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public void start(I iface, get_last_completed_transaction_for_table_args args, org.apache.thrift.async.AsyncMethodCallback<BasicTxnInfo> resultHandler) throws TException {
-        iface.get_last_completed_transaction_for_table(args.db_name, args.table_name, args.txns_snapshot,resultHandler);
-      }
-    }
-
     @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_next_notification<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_next_notification_args, NotificationEventResponse> {
       public get_next_notification() {
         super("get_next_notification");
@@ -34145,13 +33865,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list828 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list828.size);
-                  String _elem829;
-                  for (int _i830 = 0; _i830 < _list828.size; ++_i830)
+                  org.apache.thrift.protocol.TList _list818 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list818.size);
+                  String _elem819;
+                  for (int _i820 = 0; _i820 < _list818.size; ++_i820)
                   {
-                    _elem829 = iprot.readString();
-                    struct.success.add(_elem829);
+                    _elem819 = iprot.readString();
+                    struct.success.add(_elem819);
                   }
                   iprot.readListEnd();
                 }
@@ -34186,9 +33906,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter831 : struct.success)
+            for (String _iter821 : struct.success)
             {
-              oprot.writeString(_iter831);
+              oprot.writeString(_iter821);
             }
             oprot.writeListEnd();
           }
@@ -34227,9 +33947,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter832 : struct.success)
+            for (String _iter822 : struct.success)
             {
-              oprot.writeString(_iter832);
+              oprot.writeString(_iter822);
             }
           }
         }
@@ -34244,13 +33964,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list833 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list833.size);
-            String _elem834;
-            for (int _i835 = 0; _i835 < _list833.size; ++_i835)
+            org.apache.thrift.protocol.TList _list823 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list823.size);
+            String _elem824;
+            for (int _i825 = 0; _i825 < _list823.size; ++_i825)
             {
-              _elem834 = iprot.readString();
-              struct.success.add(_elem834);
+              _elem824 = iprot.readString();
+              struct.success.add(_elem824);
             }
           }
           struct.setSuccessIsSet(true);
@@ -34904,13 +34624,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list836 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list836.size);
-                  String _elem837;
-                  for (int _i838 = 0; _i838 < _list836.size; ++_i838)
+                  org.apache.thrift.protocol.TList _list826 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list826.size);
+                  String _elem827;
+                  for (int _i828 = 0; _i828 < _list826.size; ++_i828)
                   {
-                    _elem837 = iprot.readString();
-                    struct.success.add(_elem837);
+                    _elem827 = iprot.readString();
+                    struct.success.add(_elem827);
                   }
                   iprot.readListEnd();
                 }
@@ -34945,9 +34665,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter839 : struct.success)
+            for (String _iter829 : struct.success)
             {
-              oprot.writeString(_iter839);
+              oprot.writeString(_iter829);
             }
             oprot.writeListEnd();
           }
@@ -34986,9 +34706,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter840 : struct.success)
+            for (String _iter830 : struct.success)
             {
-              oprot.writeString(_iter840);
+              oprot.writeString(_iter830);
             }
           }
         }
@@ -35003,13 +34723,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list841 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list841.size);
-            String _elem842;
-            for (int _i843 = 0; _i843 < _list841.size; ++_i843)
+            org.apache.thrift.protocol.TList _list831 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list831.size);
+            String _elem832;
+            for (int _i833 = 0; _i833 < _list831.size; ++_i833)
             {
-              _elem842 = iprot.readString();
-              struct.success.add(_elem842);
+              _elem832 = iprot.readString();
+              struct.success.add(_elem832);
             }
           }
           struct.setSuccessIsSet(true);
@@ -39616,16 +39336,16 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map844 = iprot.readMapBegin();
-                  struct.success = new HashMap<String,Type>(2*_map844.size);
-                  String _key845;
-                  Type _val846;
-                  for (int _i847 = 0; _i847 < _map844.size; ++_i847)
+                  org.apache.thrift.protocol.TMap _map834 = iprot.readMapBegin();
+                  struct.success = new HashMap<String,Type>(2*_map834.size);
+                  String _key835;
+                  Type _val836;
+                  for (int _i837 = 0; _i837 < _map834.size; ++_i837)
                   {
-                    _key845 = iprot.readString();
-                    _val846 = new Type();
-                    _val846.read(iprot);
-                    struct.success.put(_key845, _val846);
+                    _key835 = iprot.readString();
+                    _val836 = new Type();
+                    _val836.read(iprot);
+                    struct.success.put(_key835, _val836);
                   }
                   iprot.readMapEnd();
                 }
@@ -39660,10 +39380,10 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Map.Entry<String, Type> _iter848 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter838 : struct.success.entrySet())
             {
-              oprot.writeString(_iter848.getKey());
-              _iter848.getValue().write(oprot);
+              oprot.writeString(_iter838.getKey());
+              _iter838.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -39702,10 +39422,10 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Map.Entry<String, Type> _iter849 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter839 : struct.success.entrySet())
             {
-              oprot.writeString(_iter849.getKey());
-              _iter849.getValue().write(oprot);
+              oprot.writeString(_iter839.getKey());
+              _iter839.getValue().write(oprot);
             }
           }
         }
@@ -39720,16 +39440,16 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map850 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new HashMap<String,Type>(2*_map850.size);
-            String _key851;
-            Type _val852;
-            for (int _i853 = 0; _i853 < _map850.size; ++_i853)
+            org.apache.thrift.protocol.TMap _map840 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new HashMap<String,Type>(2*_map840.size);
+            String _key841;
+            Type _val842;
+            for (int _i843 = 0; _i843 < _map840.size; ++_i843)
             {
-              _key851 = iprot.readString();
-              _val852 = new Type();
-              _val852.read(iprot);
-              struct.success.put(_key851, _val852);
+              _key841 = iprot.readString();
+              _val842 = new Type();
+              _val842.read(iprot);
+              struct.success.put(_key841, _val842);
             }
           }
           struct.setSuccessIsSet(true);
@@ -40764,14 +40484,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list854 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list854.size);
-                  FieldSchema _elem855;
-                  for (int _i856 = 0; _i856 < _list854.size; ++_i856)
+                  org.apache.thrift.protocol.TList _list844 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list844.size);
+                  FieldSchema _elem845;
+                  for (int _i846 = 0; _i846 < _list844.size; ++_i846)
                   {
-                    _elem855 = new FieldSchema();
-                    _elem855.read(iprot);
-                    struct.success.add(_elem855);
+                    _elem845 = new FieldSchema();
+                    _elem845.read(iprot);
+                    struct.success.add(_elem845);
                   }
                   iprot.readListEnd();
                 }
@@ -40824,9 +40544,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter857 : struct.success)
+            for (FieldSchema _iter847 : struct.success)
             {
-              _iter857.write(oprot);
+              _iter847.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -40881,9 +40601,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter858 : struct.success)
+            for (FieldSchema _iter848 : struct.success)
             {
-              _iter858.write(oprot);
+              _iter848.write(oprot);
             }
           }
         }
@@ -40904,14 +40624,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list859 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list859.size);
-            FieldSchema _elem860;
-            for (int _i861 = 0; _i861 < _list859.size; ++_i861)
+            org.apache.thrift.protocol.TList _list849 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list849.size);
+            FieldSchema _elem850;
+            for (int _i851 = 0; _i851 < _list849.size; ++_i851)
             {
-              _elem860 = new FieldSchema();
-              _elem860.read(iprot);
-              struct.success.add(_elem860);
+              _elem850 = new FieldSchema();
+              _elem850.read(iprot);
+              struct.success.add(_elem850);
             }
           }
           struct.setSuccessIsSet(true);
@@ -42065,14 +41785,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list862 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list862.size);
-                  FieldSchema _elem863;
-                  for (int _i864 = 0; _i864 < _list862.size; ++_i864)
+                  org.apache.thrift.protocol.TList _list852 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list852.size);
+                  FieldSchema _elem853;
+                  for (int _i854 = 0; _i854 < _list852.size; ++_i854)
                   {
-                    _elem863 = new FieldSchema();
-                    _elem863.read(iprot);
-                    struct.success.add(_elem863);
+                    _elem853 = new FieldSchema();
+                    _elem853.read(iprot);
+                    struct.success.add(_elem853);
                   }
                   iprot.readListEnd();
                 }
@@ -42125,9 +41845,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter865 : struct.success)
+            for (FieldSchema _iter855 : struct.success)
             {
-              _iter865.write(oprot);
+              _iter855.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -42182,9 +41902,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter866 : struct.success)
+            for (FieldSchema _iter856 : struct.success)
             {
-              _iter866.write(oprot);
+              _iter856.write(oprot);
             }
           }
         }
@@ -42205,14 +41925,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list867 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list867.size);
-            FieldSchema _elem868;
-            for (int _i869 = 0; _i869 < _list867.size; ++_i869)
+            org.apache.thrift.protocol.TList _list857 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list857.size);
+            FieldSchema _elem858;
+            for (int _i859 = 0; _i859 < _list857.size; ++_i859)
             {
-              _elem868 = new FieldSchema();
-              _elem868.read(iprot);
-              struct.success.add(_elem868);
+              _elem858 = new FieldSchema();
+              _elem858.read(iprot);
+              struct.success.add(_elem858);
             }
           }
           struct.setSuccessIsSet(true);
@@ -43257,14 +42977,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list870 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list870.size);
-                  FieldSchema _elem871;
-                  for (int _i872 = 0; _i872 < _list870.size; ++_i872)
+                  org.apache.thrift.protocol.TList _list860 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list860.size);
+                  FieldSchema _elem861;
+                  for (int _i862 = 0; _i862 < _list860.size; ++_i862)
                   {
-                    _elem871 = new FieldSchema();
-                    _elem871.read(iprot);
-                    struct.success.add(_elem871);
+                    _elem861 = new FieldSchema();
+                    _elem861.read(iprot);
+                    struct.success.add(_elem861);
                   }
                   iprot.readListEnd();
                 }
@@ -43317,9 +43037,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter873 : struct.success)
+            for (FieldSchema _iter863 : struct.success)
             {
-              _iter873.write(oprot);
+              _iter863.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -43374,9 +43094,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter874 : struct.success)
+            for (FieldSchema _iter864 : struct.success)
             {
-              _iter874.write(oprot);
+              _iter864.write(oprot);
             }
           }
         }
@@ -43397,14 +43117,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list875 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list875.size);
-            FieldSchema _elem876;
-            for (int _i877 = 0; _i877 < _list875.size; ++_i877)
+            org.apache.thrift.protocol.TList _list865 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list865.size);
+            FieldSchema _elem866;
+            for (int _i867 = 0; _i867 < _list865.size; ++_i867)
             {
-              _elem876 = new FieldSchema();
-              _elem876.read(iprot);
-              struct.success.add(_elem876);
+              _elem866 = new FieldSchema();
+              _elem866.read(iprot);
+              struct.success.add(_elem866);
             }
           }
           struct.setSuccessIsSet(true);
@@ -44558,14 +44278,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list878 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list878.size);
-                  FieldSchema _elem879;
-                  for (int _i880 = 0; _i880 < _list878.size; ++_i880)
+                  org.apache.thrift.protocol.TList _list868 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list868.size);
+                  FieldSchema _elem869;
+                  for (int _i870 = 0; _i870 < _list868.size; ++_i870)
                   {
-                    _elem879 = new FieldSchema();
-                    _elem879.read(iprot);
-                    struct.success.add(_elem879);
+                    _elem869 = new FieldSchema();
+                    _elem869.read(iprot);
+                    struct.success.add(_elem869);
                   }
                   iprot.readListEnd();
                 }
@@ -44618,9 +44338,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter881 : struct.success)
+            for (FieldSchema _iter871 : struct.success)
             {
-              _iter881.write(oprot);
+              _iter871.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -44675,9 +44395,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter882 : struct.success)
+            for (FieldSchema _iter872 : struct.success)
             {
-              _iter882.write(oprot);
+              _iter872.write(oprot);
             }
           }
         }
@@ -44698,14 +44418,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list883 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list883.size);
-            FieldSchema _elem884;
-            for (int _i885 = 0; _i885 < _list883.size; ++_i885)
+            org.apache.thrift.protocol.TList _list873 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list873.size);
+            FieldSchema _elem874;
+            for (int _i875 = 0; _i875 < _list873.size; ++_i875)
             {
-              _elem884 = new FieldSchema();
-              _elem884.read(iprot);
-              struct.success.add(_elem884);
+              _elem874 = new FieldSchema();
+              _elem874.read(iprot);
+              struct.success.add(_elem874);
             }
           }
           struct.setSuccessIsSet(true);
@@ -47632,14 +47352,14 @@ import org.slf4j.LoggerFactory;
             case 2: // PRIMARY_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list886 = iprot.readListBegin();
-                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list886.size);
-                  SQLPrimaryKey _elem887;
-                  for (int _i888 = 0; _i888 < _list886.size; ++_i888)
+                  org.apache.thrift.protocol.TList _list876 = iprot.readListBegin();
+                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list876.size);
+                  SQLPrimaryKey _elem877;
+                  for (int _i878 = 0; _i878 < _list876.size; ++_i878)
                   {
-                    _elem887 = new SQLPrimaryKey();
-                    _elem887.read(iprot);
-                    struct.primaryKeys.add(_elem887);
+                    _elem877 = new SQLPrimaryKey();
+                    _elem877.read(iprot);
+                    struct.primaryKeys.add(_elem877);
                   }
                   iprot.readListEnd();
                 }
@@ -47651,14 +47371,14 @@ import org.slf4j.LoggerFactory;
             case 3: // FOREIGN_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list889 = iprot.readListBegin();
-                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list889.size);
-                  SQLForeignKey _elem890;
-                  for (int _i891 = 0; _i891 < _list889.size; ++_i891)
+                  org.apache.thrift.protocol.TList _list879 = iprot.readListBegin();
+                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list879.size);
+                  SQLForeignKey _elem880;
+                  for (int _i881 = 0; _i881 < _list879.size; ++_i881)
                   {
-                    _elem890 = new SQLForeignKey();
-                    _elem890.read(iprot);
-                    struct.foreignKeys.add(_elem890);
+                    _elem880 = new SQLForeignKey();
+                    _elem880.read(iprot);
+                    struct.foreignKeys.add(_elem880);
                   }
                   iprot.readListEnd();
                 }
@@ -47670,14 +47390,14 @@ import org.slf4j.LoggerFactory;
             case 4: // UNIQUE_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list892 = iprot.readListBegin();
-                  struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list892.size);
-                  SQLUniqueConstraint _elem893;
-                  for (int _i894 = 0; _i894 < _list892.size; ++_i894)
+                  org.apache.thrift.protocol.TList _list882 = iprot.readListBegin();
+                  struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list882.size);
+                  SQLUniqueConstraint _elem883;
+                  for (int _i884 = 0; _i884 < _list882.size; ++_i884)
                   {
-                    _elem893 = new SQLUniqueConstraint();
-                    _elem893.read(iprot);
-                    struct.uniqueConstraints.add(_elem893);
+                    _elem883 = new SQLUniqueConstraint();
+                    _elem883.read(iprot);
+                    struct.uniqueConstraints.add(_elem883);
                   }
                   iprot.readListEnd();
                 }
@@ -47689,14 +47409,14 @@ import org.slf4j.LoggerFactory;
             case 5: // NOT_NULL_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list895 = iprot.readListBegin();
-                  struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list895.size);
-                  SQLNotNullConstraint _elem896;
-                  for (int _i897 = 0; _i897 < _list895.size; ++_i897)
+                  org.apache.thrift.protocol.TList _list885 = iprot.readListBegin();
+                  struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list885.size);
+                  SQLNotNullConstraint _elem886;
+                  for (int _i887 = 0; _i887 < _list885.size; ++_i887)
                   {
-                    _elem896 = new SQLNotNullConstraint();
-                    _elem896.read(iprot);
-                    struct.notNullConstraints.add(_elem896);
+                    _elem886 = new SQLNotNullConstraint();
+                    _elem886.read(iprot);
+                    struct.notNullConstraints.add(_elem886);
                   }
                   iprot.readListEnd();
                 }
@@ -47727,9 +47447,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PRIMARY_KEYS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.primaryKeys.size()));
-            for (SQLPrimaryKey _iter898 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter888 : struct.primaryKeys)
             {
-              _iter898.write(oprot);
+              _iter888.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -47739,9 +47459,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(FOREIGN_KEYS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.foreignKeys.size()));
-            for (SQLForeignKey _iter899 : struct.foreignKeys)
+            for (SQLForeignKey _iter889 : struct.foreignKeys)
             {
-              _iter899.write(oprot);
+              _iter889.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -47751,9 +47471,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(UNIQUE_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.uniqueConstraints.size()));
-            for (SQLUniqueConstraint _iter900 : struct.uniqueConstraints)
+            for (SQLUniqueConstraint _iter890 : struct.uniqueConstraints)
             {
-              _iter900.write(oprot);
+              _iter890.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -47763,9 +47483,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(NOT_NULL_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.notNullConstraints.size()));
-            for (SQLNotNullConstraint _iter901 : struct.notNullConstraints)
+            for (SQLNotNullConstraint _iter891 : struct.notNullConstraints)
             {
-              _iter901.write(oprot);
+              _iter891.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -47811,36 +47531,36 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPrimaryKeys()) {
           {
             oprot.writeI32(struct.primaryKeys.size());
-            for (SQLPrimaryKey _iter902 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter892 : struct.primaryKeys)
             {
-              _iter902.write(oprot);
+              _iter892.write(oprot);
             }
           }
         }
         if (struct.isSetForeignKeys()) {
           {
             oprot.writeI32(struct.foreignKeys.size());
-            for (SQLForeignKey _iter903 : struct.foreignKeys)
+            for (SQLForeignKey _iter893 : struct.foreignKeys)
             {
-              _iter903.write(oprot);
+              _iter893.write(oprot);
             }
           }
         }
         if (struct.isSetUniqueConstraints()) {
           {
             oprot.writeI32(struct.uniqueConstraints.size());
-            for (SQLUniqueConstraint _iter904 : struct.uniqueConstraints)
+            for (SQLUniqueConstraint _iter894 : struct.uniqueConstraints)
             {
-              _iter904.write(oprot);
+              _iter894.write(oprot);
             }
           }
         }
         if (struct.isSetNotNullConstraints()) {
           {
             oprot.writeI32(struct.notNullConstraints.size());
-            for (SQLNotNullConstraint _iter905 : struct.notNullConstraints)
+            for (SQLNotNullConstraint _iter895 : struct.notNullConstraints)
             {
-              _iter905.write(oprot);
+              _iter895.write(oprot);
             }
           }
         }
@@ -47857,56 +47577,56 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list906 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list906.size);
-            SQLPrimaryKey _elem907;
-            for (int _i908 = 0; _i908 < _list906.size; ++_i908)
+            org.apache.thrift.protocol.TList _list896 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list896.size);
+            SQLPrimaryKey _elem897;
+            for (int _i898 = 0; _i898 < _list896.size; ++_i898)
             {
-              _elem907 = new SQLPrimaryKey();
-              _elem907.read(iprot);
-              struct.primaryKeys.add(_elem907);
+              _elem897 = new SQLPrimaryKey();
+              _elem897.read(iprot);
+              struct.primaryKeys.add(_elem897);
             }
           }
           struct.setPrimaryKeysIsSet(true);
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list909 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list909.size);
-            SQLForeignKey _elem910;
-            for (int _i911 = 0; _i911 < _list909.size; ++_i911)
+            org.apache.thrift.protocol.TList _list899 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list899.size);
+            SQLForeignKey _elem900;
+            for (int _i901 = 0; _i901 < _list899.size; ++_i901)
             {
-              _elem910 = new SQLForeignKey();
-              _elem910.read(iprot);
-              struct.foreignKeys.add(_elem910);
+              _elem900 = new SQLForeignKey();
+              _elem900.read(iprot);
+              struct.foreignKeys.add(_elem900);
             }
           }
           struct.setForeignKeysIsSet(true);
         }
         if (incoming.get(3)) {
           {
-            org.apache.thrift.protocol.TList _list912 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list912.size);
-            SQLUniqueConstraint _elem913;
-            for (int _i914 = 0; _i914 < _list912.size; ++_i914)
+            org.apache.thrift.protocol.TList _list902 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list902.size);
+            SQLUniqueConstraint _elem903;
+            for (int _i904 = 0; _i904 < _list902.size; ++_i904)
             {
-              _elem913 = new SQLUniqueConstraint();
-              _elem913.read(iprot);
-              struct.uniqueConstraints.add(_elem913);
+              _elem903 = new SQLUniqueConstraint();
+              _elem903.read(iprot);
+              struct.uniqueConstraints.add(_elem903);
             }
           }
           struct.setUniqueConstraintsIsSet(true);
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol.TList _list915 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list915.size);
-            SQLNotNullConstraint _elem916;
-            for (int _i917 = 0; _i917 < _list915.size; ++_i917)
+            org.apache.thrift.protocol.TList _list905 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list905.size);
+            SQLNotNullConstraint _elem906;
+            for (int _i907 = 0; _i907 < _list905.size; ++_i907)
             {
-              _elem916 = new SQLNotNullConstraint();
-              _elem916.read(iprot);
-              struct.notNullConstraints.add(_elem916);
+              _elem906 = new SQLNotNullConstraint();
+              _elem906.read(iprot);
+              struct.notNullConstraints.add(_elem906);
             }
           }
           struct.setNotNullConstraintsIsSet(true);
@@ -55398,13 +55118,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list918 = iprot.readListBegin();
-                  struct.partNames = new ArrayList<String>(_list918.size);
-                  String _elem919;
-                  for (int _i920 = 0; _i920 < _list918.size; ++_i920)
+                  org.apache.thrift.protocol.TList _list908 = iprot.readListBegin();
+                  struct.partNames = new ArrayList<String>(_list908.size);
+                  String _elem909;
+                  for (int _i910 = 0; _i910 < _list908.size; ++_i910)
                   {
-                    _elem919 = iprot.readString();
-                    struct.partNames.add(_elem919);
+                    _elem909 = iprot.readString();
+                    struct.partNames.add(_elem909);
                   }
                   iprot.readListEnd();
                 }
@@ -55440,9 +55160,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PART_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partNames.size()));
-            for (String _iter921 : struct.partNames)
+            for (String _iter911 : struct.partNames)
             {
-              oprot.writeString(_iter921);
+              oprot.writeString(_iter911);
             }
             oprot.writeListEnd();
           }
@@ -55485,9 +55205,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPartNames()) {
           {
             oprot.writeI32(struct.partNames.size());
-            for (String _iter922 : struct.partNames)
+            for (String _iter912 : struct.partNames)
             {
-              oprot.writeString(_iter922);
+              oprot.writeString(_iter912);
             }
           }
         }
@@ -55507,13 +55227,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list923 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.partNames = new ArrayList<String>(_list923.size);
-            String _elem924;
-            for (int _i925 = 0; _i925 < _list923.size; ++_i925)
+            org.apache.thrift.protocol.TList _list913 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.partNames = new ArrayList<String>(_list913.size);
+            String _elem914;
+            for (int _i915 = 0; _i915 < _list913.size; ++_i915)
             {
-              _elem924 = iprot.readString();
-              struct.partNames.add(_elem924);
+              _elem914 = iprot.readString();
+              struct.partNames.add(_elem914);
             }
           }
           struct.setPartNamesIsSet(true);
@@ -56738,13 +56458,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list926 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list926.size);
-                  String _elem927;
-                  for (int _i928 = 0; _i928 < _list926.size; ++_i928)
+                  org.apache.thrift.protocol.TList _list916 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list916.size);
+                  String _elem917;
+                  for (int _i918 = 0; _i918 < _list916.size; ++_i918)
                   {
-                    _elem927 = iprot.readString();
-                    struct.success.add(_elem927);
+                    _elem917 = iprot.readString();
+                    struct.success.add(_elem917);
                   }
                   iprot.readListEnd();
                 }
@@ -56779,9 +56499,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter929 : struct.success)
+            for (String _iter919 : struct.success)
             {
-              oprot.writeString(_iter929);
+              oprot.writeString(_iter919);
             }
             oprot.writeListEnd();
           }
@@ -56820,9 +56540,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter930 : struct.success)
+            for (String _iter920 : struct.success)
             {
-              oprot.writeString(_iter930);
+              oprot.writeString(_iter920);
             }
           }
         }
@@ -56837,13 +56557,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list931 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list931.size);
-            String _elem932;
-            for (int _i933 = 0; _i933 < _list931.size; ++_i933)
+            org.apache.thrift.protocol.TList _list921 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list921.size);
+            String _elem922;
+            for (int _i923 = 0; _i923 < _list921.size; ++_i923)
             {
-              _elem932 = iprot.readString();
-              struct.success.add(_elem932);
+              _elem922 = iprot.readString();
+              struct.success.add(_elem922);
             }
           }
           struct.setSuccessIsSet(true);
@@ -57817,13 +57537,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list934 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list934.size);
-                  String _elem935;
-                  for (int _i936 = 0; _i936 < _list934.size; ++_i936)
+                  org.apache.thrift.protocol.TList _list924 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list924.size);
+                  String _elem925;
+                  for (int _i926 = 0; _i926 < _list924.size; ++_i926)
                   {
-                    _elem935 = iprot.readString();
-                    struct.success.add(_elem935);
+                    _elem925 = iprot.readString();
+                    struct.success.add(_elem925);
                   }
                   iprot.readListEnd();
                 }
@@ -57858,9 +57578,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter937 : struct.success)
+            for (String _iter927 : struct.success)
             {
-              oprot.writeString(_iter937);
+              oprot.writeString(_iter927);
             }
             oprot.writeListEnd();
           }
@@ -57899,9 +57619,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter938 : struct.success)
+            for (String _iter928 : struct.success)
             {
-              oprot.writeString(_iter938);
+              oprot.writeString(_iter928);
             }
           }
         }
@@ -57916,13 +57636,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list939 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list939.size);
-            String _elem940;
-            for (int _i941 = 0; _i941 < _list939.size; ++_i941)
+            org.apache.thrift.protocol.TList _list929 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list929.size);
+            String _elem930;
+            for (int _i931 = 0; _i931 < _list929.size; ++_i931)
             {
-              _elem940 = iprot.readString();
-              struct.success.add(_elem940);
+              _elem930 = iprot.readString();
+              struct.success.add(_elem930);
             }
           }
           struct.setSuccessIsSet(true);
@@ -58688,13 +58408,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list942 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list942.size);
-                  String _elem943;
-                  for (int _i944 = 0; _i944 < _list942.size; ++_i944)
+                  org.apache.thrift.protocol.TList _list932 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list932.size);
+                  String _elem933;
+                  for (int _i934 = 0; _i934 < _list932.size; ++_i934)
                   {
-                    _elem943 = iprot.readString();
-                    struct.success.add(_elem943);
+                    _elem933 = iprot.readString();
+                    struct.success.add(_elem933);
                   }
                   iprot.readListEnd();
                 }
@@ -58729,9 +58449,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter945 : struct.success)
+            for (String _iter935 : struct.success)
             {
-              oprot.writeString(_iter945);
+              oprot.writeString(_iter935);
             }
             oprot.writeListEnd();
           }
@@ -58770,9 +58490,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter946 : struct.success)
+            for (String _iter936 : struct.success)
             {
-              oprot.writeString(_iter946);
+              oprot.writeString(_iter936);
             }
           }
         }
@@ -58787,13 +58507,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list947 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list947.size);
-            String _elem948;
-            for (int _i949 = 0; _i949 < _list947.size; ++_i949)
+            org.apache.thrift.protocol.TList _list937 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list937.size);
+            String _elem938;
+            for (int _i939 = 0; _i939 < _list937.size; ++_i939)
             {
-              _elem948 = iprot.readString();
-              struct.success.add(_elem948);
+              _elem938 = iprot.readString();
+              struct.success.add(_elem938);
             }
           }
           struct.setSuccessIsSet(true);
@@ -59298,13 +59018,13 @@ import org.slf4j.LoggerFactory;
             case 3: // TBL_TYPES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list950 = iprot.readListBegin();
-                  struct.tbl_types = new ArrayList<String>(_list950.size);
-                  String _elem951;
-                  for (int _i952 = 0; _i952 < _list950.size; ++_i952)
+                  org.apache.thrift.protocol.TList _list940 = iprot.readListBegin();
+                  struct.tbl_types = new ArrayList<String>(_list940.size);
+                  String _elem941;
+                  for (int _i942 = 0; _i942 < _list940.size; ++_i942)
                   {
-                    _elem951 = iprot.readString();
-                    struct.tbl_types.add(_elem951);
+                    _elem941 = iprot.readString();
+                    struct.tbl_types.add(_elem941);
                   }
                   iprot.readListEnd();
                 }
@@ -59340,9 +59060,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TBL_TYPES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_types.size()));
-            for (String _iter953 : struct.tbl_types)
+            for (String _iter943 : struct.tbl_types)
             {
-              oprot.writeString(_iter953);
+              oprot.writeString(_iter943);
             }
             oprot.writeListEnd();
           }
@@ -59385,9 +59105,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetTbl_types()) {
           {
             oprot.writeI32(struct.tbl_types.size());
-            for (String _iter954 : struct.tbl_types)
+            for (String _iter944 : struct.tbl_types)
             {
-              oprot.writeString(_iter954);
+              oprot.writeString(_iter944);
             }
           }
         }
@@ -59407,13 +59127,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list955 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_types = new ArrayList<String>(_list955.size);
-            String _elem956;
-            for (int _i957 = 0; _i957 < _list955.size; ++_i957)
+            org.apache.thrift.protocol.TList _list945 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_types = new ArrayList<String>(_list945.size);
+            String _elem946;
+            for (int _i947 = 0; _i947 < _list945.size; ++_i947)
             {
-              _elem956 = iprot.readString();
-              struct.tbl_types.add(_elem956);
+              _elem946 = iprot.readString();
+              struct.tbl_types.add(_elem946);
             }
           }
           struct.setTbl_typesIsSet(true);
@@ -59819,14 +59539,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list958 = iprot.readListBegin();
-                  struct.success = new ArrayList<TableMeta>(_list958.size);
-                  TableMeta _elem959;
-                  for (int _i960 = 0; _i960 < _list958.size; ++_i960)
+                  org.apache.thrift.protocol.TList _list948 = iprot.readListBegin();
+                  struct.success = new ArrayList<TableMeta>(_list948.size);
+                  TableMeta _elem949;
+                  for (int _i950 = 0; _i950 < _list948.size; ++_i950)
                   {
-                    _elem959 = new TableMeta();
-                    _elem959.read(iprot);
-                    struct.success.add(_elem959);
+                    _elem949 = new TableMeta();
+                    _elem949.read(iprot);
+                    struct.success.add(_elem949);
                   }
                   iprot.readListEnd();
                 }
@@ -59861,9 +59581,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (TableMeta _iter961 : struct.success)
+            for (TableMeta _iter951 : struct.success)
             {
-              _iter961.write(oprot);
+              _iter951.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -59902,9 +59622,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (TableMeta _iter962 : struct.success)
+            for (TableMeta _iter952 : struct.success)
             {
-              _iter962.write(oprot);
+              _iter952.write(oprot);
             }
           }
         }
@@ -59919,14 +59639,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list963 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<TableMeta>(_list963.size);
-            TableMeta _elem964;
-            for (int _i965 = 0; _i965 < _list963.size; ++_i965)
+            org.apache.thrift.protocol.TList _list953 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<TableMeta>(_list953.size);
+            TableMeta _elem954;
+            for (int _i955 = 0; _i955 < _list953.size; ++_i955)
             {
-              _elem964 = new TableMeta();
-              _elem964.read(iprot);
-              struct.success.add(_elem964);
+              _elem954 = new TableMeta();
+              _elem954.read(iprot);
+              struct.success.add(_elem954);
             }
           }
           struct.setSuccessIsSet(true);
@@ -60692,13 +60412,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list966 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list966.size);
-                  String _elem967;
-                  for (int _i968 = 0; _i968 < _list966.size; ++_i968)
+                  org.apache.thrift.protocol.TList _list956 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list956.size);
+                  String _elem957;
+                  for (int _i958 = 0; _i958 < _list956.size; ++_i958)
                   {
-                    _elem967 = iprot.readString();
-                    struct.success.add(_elem967);
+                    _elem957 = iprot.readString();
+                    struct.success.add(_elem957);
                   }
                   iprot.readListEnd();
                 }
@@ -60733,9 +60453,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter969 : struct.success)
+            for (String _iter959 : struct.success)
             {
-              oprot.writeString(_iter969);
+              oprot.writeString(_iter959);
             }
             oprot.writeListEnd();
           }
@@ -60774,9 +60494,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter970 : struct.success)
+            for (String _iter960 : struct.success)
             {
-              oprot.writeString(_iter970);
+              oprot.writeString(_iter960);
             }
           }
         }
@@ -60791,13 +60511,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list971 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list971.size);
-            String _elem972;
-            for (int _i973 = 0; _i973 < _list971.size; ++_i973)
+            org.apache.thrift.protocol.TList _list961 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list961.size);
+            String _elem962;
+            for (int _i963 = 0; _i963 < _list961.size; ++_i963)
             {
-              _elem972 = iprot.readString();
-              struct.success.add(_elem972);
+              _elem962 = iprot.readString();
+              struct.success.add(_elem962);
             }
           }
           struct.setSuccessIsSet(true);
@@ -62250,13 +61970,13 @@ import org.slf4j.LoggerFactory;
             case 2: // TBL_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list974 = iprot.readListBegin();
-                  struct.tbl_names = new ArrayList<String>(_list974.size);
-                  String _elem975;
-                  for (int _i976 = 0; _i976 < _list974.size; ++_i976)
+                  org.apache.thrift.protocol.TList _list964 = iprot.readListBegin();
+                  struct.tbl_names = new ArrayList<String>(_list964.size);
+                  String _elem965;
+                  for (int _i966 = 0; _i966 < _list964.size; ++_i966)
                   {
-                    _elem975 = iprot.readString();
-                    struct.tbl_names.add(_elem975);
+                    _elem965 = iprot.readString();
+                    struct.tbl_names.add(_elem965);
                   }
                   iprot.readListEnd();
                 }
@@ -62287,9 +62007,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_names.size()));
-            for (String _iter977 : struct.tbl_names)
+            for (String _iter967 : struct.tbl_names)
             {
-              oprot.writeString(_iter977);
+              oprot.writeString(_iter967);
             }
             oprot.writeListEnd();
           }
@@ -62326,9 +62046,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetTbl_names()) {
           {
             oprot.writeI32(struct.tbl_names.size());
-            for (String _iter978 : struct.tbl_names)
+            for (String _iter968 : struct.tbl_names)
             {
-              oprot.writeString(_iter978);
+              oprot.writeString(_iter968);
             }
           }
         }
@@ -62344,13 +62064,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list979 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_names = new ArrayList<String>(_list979.size);
-            String _elem980;
-            for (int _i981 = 0; _i981 < _list979.size; ++_i981)
+            org.apache.thrift.protocol.TList _list969 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_names = new ArrayList<String>(_list969.size);
+            String _elem970;
+            for (int _i971 = 0; _i971 < _list969.size; ++_i971)
             {
-              _elem980 = iprot.readString();
-              struct.tbl_names.add(_elem980);
+              _elem970 = iprot.readString();
+              struct.tbl_names.add(_elem970);
             }
           }
           struct.setTbl_namesIsSet(true);
@@ -62675,14 +62395,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list982 = iprot.readListBegin();
-                  struct.success = new ArrayList<Table>(_list982.size);
-                  Table _elem983;
-                  for (int _i984 = 0; _i984 < _list982.size; ++_i984)
+                  org.apache.thrift.protocol.TList _list972 = iprot.readListBegin();
+                  struct.success = new ArrayList<Table>(_list972.size);
+                  Table _elem973;
+                  for (int _i974 = 0; _i974 < _list972.size; ++_i974)
                   {
-                    _elem983 = new Table();
-                    _elem983.read(iprot);
-                    struct.success.add(_elem983);
+                    _elem973 = new Table();
+                    _elem973.read(iprot);
+                    struct.success.add(_elem973);
                   }
                   iprot.readListEnd();
                 }
@@ -62708,9 +62428,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Table _iter985 : struct.success)
+            for (Table _iter975 : struct.success)
             {
-              _iter985.write(oprot);
+              _iter975.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -62741,9 +62461,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Table _iter986 : struct.success)
+            for (Table _iter976 : struct.success)
             {
-              _iter986.write(oprot);
+              _iter976.write(oprot);
             }
           }
         }
@@ -62755,14 +62475,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list987 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Table>(_list987.size);
-            Table _elem988;
-            for (int _i989 = 0; _i989 < _list987.size; ++_i989)
+            org.apache.thrift.protocol.TList _list977 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Table>(_list977.size);
+            Table _elem978;
+            for (int _i979 = 0; _i979 < _list977.size; ++_i979)
             {
-              _elem988 = new Table();
-              _elem988.read(iprot);
-              struct.success.add(_elem988);
+              _elem978 = new Table();
+              _elem978.read(iprot);
+              struct.success.add(_elem978);
             }
           }
           struct.setSuccessIsSet(true);
@@ -65155,13 +64875,13 @@ import org.slf4j.LoggerFactory;
             case 2: // TBL_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list990 = iprot.readListBegin();
-                  struct.tbl_names = new ArrayList<String>(_list990.size);
-                  String _elem991;
-                  for (int _i992 = 0; _i992 < _list990.size; ++_i992)
+                  org.apache.thrift.protocol.TList _list980 = iprot.readListBegin();
+                  struct.tbl_names = new ArrayList<String>(_list980.size);
+                  String _elem981;
+                  for (int _i982 = 0; _i982 < _list980.size; ++_i982)
                   {
-                    _elem991 = iprot.readString();
-                    struct.tbl_names.add(_elem991);
+                    _elem981 = iprot.readString();
+                    struct.tbl_names.add(_elem981);
                   }
                   iprot.readListEnd();
                 }
@@ -65192,9 +64912,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_names.size()));
-            for (String _iter993 : struct.tbl_names)
+            for (String _iter983 : struct.tbl_names)
             {
-              oprot.writeString(_iter993);
+              oprot.writeString(_iter983);
             }
             oprot.writeListEnd();
           }
@@ -65231,9 +64951,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetTbl_names()) {
           {
             oprot.writeI32(struct.tbl_names.size());
-            for (String _iter994 : struct.tbl_names)
+            for (String _iter984 : struct.tbl_names)
             {
-              oprot.writeString(_iter994);
+              oprot.writeString(_iter984);
             }
           }
         }
@@ -65249,13 +64969,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list995 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_names = new ArrayList<String>(_list995.size);
-            String _elem996;
-            for (int _i997 = 0; _i997 < _list995.size; ++_i997)
+            org.apache.thrift.protocol.TList _list985 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_names = new ArrayList<String>(_list985.size);
+            String _elem986;
+            for (int _i987 = 0; _i987 < _list985.size; ++_i987)
             {
-              _elem996 = iprot.readString();
-              struct.tbl_names.add(_elem996);
+              _elem986 = iprot.readString();
+              struct.tbl_names.add(_elem986);
             }
           }
           struct.setTbl_namesIsSet(true);
@@ -65828,16 +65548,16 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map998 = iprot.readMapBegin();
-                  struct.success = new HashMap<String,Materialization>(2*_map998.size);
-                  String _key999;
-                  Materialization _val1000;
-                  for (int _i1001 = 0; _i1001 < _map998.size; ++_i1001)
+                  org.apache.thrift.protocol.TMap _map988 = iprot.readMapBegin();
+                  struct.success = new HashMap<String,Materialization>(2*_map988.size);
+                  String _key989;
+                  Materialization _val990;
+                  for (int _i991 = 0; _i991 < _map988.size; ++_i991)
                   {
-                    _key999 = iprot.readString();
-                    _val1000 = new Materialization();
-                    _val1000.read(iprot);
-                    struct.success.put(_key999, _val1000);
+                    _key989 = iprot.readString();
+                    _val990 = new Materialization();
+                    _val990.read(iprot);
+                    struct.success.put(_key989, _val990);
                   }
                   iprot.readMapEnd();
                 }
@@ -65890,10 +65610,10 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Map.Entry<String, Materialization> _iter1002 : struct.success.entrySet())
+            for (Map.Entry<St

<TRUNCATED>

[13/16] hive git commit: HIVE-18546: Remove unnecessary code introduced in HIVE-14498 (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
index 27f8c0f..aadf8f1 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -4940,7 +4940,7 @@ void Table::__set_rewriteEnabled(const bool val) {
 __isset.rewriteEnabled = true;
 }
 
-void Table::__set_creationMetadata(const std::map<std::string, BasicTxnInfo> & val) {
+void Table::__set_creationMetadata(const CreationMetadata& val) {
   this->creationMetadata = val;
 __isset.creationMetadata = true;
 }
@@ -5114,23 +5114,8 @@ uint32_t Table::read(::apache::thrift::protocol::TProtocol* iprot) {
         }
         break;
       case 16:
-        if (ftype == ::apache::thrift::protocol::T_MAP) {
-          {
-            this->creationMetadata.clear();
-            uint32_t _size223;
-            ::apache::thrift::protocol::TType _ktype224;
-            ::apache::thrift::protocol::TType _vtype225;
-            xfer += iprot->readMapBegin(_ktype224, _vtype225, _size223);
-            uint32_t _i227;
-            for (_i227 = 0; _i227 < _size223; ++_i227)
-            {
-              std::string _key228;
-              xfer += iprot->readString(_key228);
-              BasicTxnInfo& _val229 = this->creationMetadata[_key228];
-              xfer += _val229.read(iprot);
-            }
-            xfer += iprot->readMapEnd();
-          }
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->creationMetadata.read(iprot);
           this->__isset.creationMetadata = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -5184,10 +5169,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 _iter230;
-    for (_iter230 = this->partitionKeys.begin(); _iter230 != this->partitionKeys.end(); ++_iter230)
+    std::vector<FieldSchema> ::const_iterator _iter223;
+    for (_iter223 = this->partitionKeys.begin(); _iter223 != this->partitionKeys.end(); ++_iter223)
     {
-      xfer += (*_iter230).write(oprot);
+      xfer += (*_iter223).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5196,11 +5181,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 _iter231;
-    for (_iter231 = this->parameters.begin(); _iter231 != this->parameters.end(); ++_iter231)
+    std::map<std::string, std::string> ::const_iterator _iter224;
+    for (_iter224 = this->parameters.begin(); _iter224 != this->parameters.end(); ++_iter224)
     {
-      xfer += oprot->writeString(_iter231->first);
-      xfer += oprot->writeString(_iter231->second);
+      xfer += oprot->writeString(_iter224->first);
+      xfer += oprot->writeString(_iter224->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -5234,17 +5219,8 @@ uint32_t Table::write(::apache::thrift::protocol::TProtocol* oprot) const {
     xfer += oprot->writeFieldEnd();
   }
   if (this->__isset.creationMetadata) {
-    xfer += oprot->writeFieldBegin("creationMetadata", ::apache::thrift::protocol::T_MAP, 16);
-    {
-      xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->creationMetadata.size()));
-      std::map<std::string, BasicTxnInfo> ::const_iterator _iter232;
-      for (_iter232 = this->creationMetadata.begin(); _iter232 != this->creationMetadata.end(); ++_iter232)
-      {
-        xfer += oprot->writeString(_iter232->first);
-        xfer += _iter232->second.write(oprot);
-      }
-      xfer += oprot->writeMapEnd();
-    }
+    xfer += oprot->writeFieldBegin("creationMetadata", ::apache::thrift::protocol::T_STRUCT, 16);
+    xfer += this->creationMetadata.write(oprot);
     xfer += oprot->writeFieldEnd();
   }
   xfer += oprot->writeFieldStop();
@@ -5273,43 +5249,43 @@ void swap(Table &a, Table &b) {
   swap(a.__isset, b.__isset);
 }
 
-Table::Table(const Table& other233) {
-  tableName = other233.tableName;
-  dbName = other233.dbName;
-  owner = other233.owner;
-  createTime = other233.createTime;
-  lastAccessTime = other233.lastAccessTime;
-  retention = other233.retention;
-  sd = other233.sd;
-  partitionKeys = other233.partitionKeys;
-  parameters = other233.parameters;
-  viewOriginalText = other233.viewOriginalText;
-  viewExpandedText = other233.viewExpandedText;
-  tableType = other233.tableType;
-  privileges = other233.privileges;
-  temporary = other233.temporary;
-  rewriteEnabled = other233.rewriteEnabled;
-  creationMetadata = other233.creationMetadata;
-  __isset = other233.__isset;
-}
-Table& Table::operator=(const Table& other234) {
-  tableName = other234.tableName;
-  dbName = other234.dbName;
-  owner = other234.owner;
-  createTime = other234.createTime;
-  lastAccessTime = other234.lastAccessTime;
-  retention = other234.retention;
-  sd = other234.sd;
-  partitionKeys = other234.partitionKeys;
-  parameters = other234.parameters;
-  viewOriginalText = other234.viewOriginalText;
-  viewExpandedText = other234.viewExpandedText;
-  tableType = other234.tableType;
-  privileges = other234.privileges;
-  temporary = other234.temporary;
-  rewriteEnabled = other234.rewriteEnabled;
-  creationMetadata = other234.creationMetadata;
-  __isset = other234.__isset;
+Table::Table(const Table& other225) {
+  tableName = other225.tableName;
+  dbName = other225.dbName;
+  owner = other225.owner;
+  createTime = other225.createTime;
+  lastAccessTime = other225.lastAccessTime;
+  retention = other225.retention;
+  sd = other225.sd;
+  partitionKeys = other225.partitionKeys;
+  parameters = other225.parameters;
+  viewOriginalText = other225.viewOriginalText;
+  viewExpandedText = other225.viewExpandedText;
+  tableType = other225.tableType;
+  privileges = other225.privileges;
+  temporary = other225.temporary;
+  rewriteEnabled = other225.rewriteEnabled;
+  creationMetadata = other225.creationMetadata;
+  __isset = other225.__isset;
+}
+Table& Table::operator=(const Table& other226) {
+  tableName = other226.tableName;
+  dbName = other226.dbName;
+  owner = other226.owner;
+  createTime = other226.createTime;
+  lastAccessTime = other226.lastAccessTime;
+  retention = other226.retention;
+  sd = other226.sd;
+  partitionKeys = other226.partitionKeys;
+  parameters = other226.parameters;
+  viewOriginalText = other226.viewOriginalText;
+  viewExpandedText = other226.viewExpandedText;
+  tableType = other226.tableType;
+  privileges = other226.privileges;
+  temporary = other226.temporary;
+  rewriteEnabled = other226.rewriteEnabled;
+  creationMetadata = other226.creationMetadata;
+  __isset = other226.__isset;
   return *this;
 }
 void Table::printTo(std::ostream& out) const {
@@ -5397,14 +5373,14 @@ uint32_t Partition::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->values.clear();
-            uint32_t _size235;
-            ::apache::thrift::protocol::TType _etype238;
-            xfer += iprot->readListBegin(_etype238, _size235);
-            this->values.resize(_size235);
-            uint32_t _i239;
-            for (_i239 = 0; _i239 < _size235; ++_i239)
+            uint32_t _size227;
+            ::apache::thrift::protocol::TType _etype230;
+            xfer += iprot->readListBegin(_etype230, _size227);
+            this->values.resize(_size227);
+            uint32_t _i231;
+            for (_i231 = 0; _i231 < _size227; ++_i231)
             {
-              xfer += iprot->readString(this->values[_i239]);
+              xfer += iprot->readString(this->values[_i231]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5457,17 +5433,17 @@ uint32_t Partition::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->parameters.clear();
-            uint32_t _size240;
-            ::apache::thrift::protocol::TType _ktype241;
-            ::apache::thrift::protocol::TType _vtype242;
-            xfer += iprot->readMapBegin(_ktype241, _vtype242, _size240);
-            uint32_t _i244;
-            for (_i244 = 0; _i244 < _size240; ++_i244)
+            uint32_t _size232;
+            ::apache::thrift::protocol::TType _ktype233;
+            ::apache::thrift::protocol::TType _vtype234;
+            xfer += iprot->readMapBegin(_ktype233, _vtype234, _size232);
+            uint32_t _i236;
+            for (_i236 = 0; _i236 < _size232; ++_i236)
             {
-              std::string _key245;
-              xfer += iprot->readString(_key245);
-              std::string& _val246 = this->parameters[_key245];
-              xfer += iprot->readString(_val246);
+              std::string _key237;
+              xfer += iprot->readString(_key237);
+              std::string& _val238 = this->parameters[_key237];
+              xfer += iprot->readString(_val238);
             }
             xfer += iprot->readMapEnd();
           }
@@ -5504,10 +5480,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 _iter247;
-    for (_iter247 = this->values.begin(); _iter247 != this->values.end(); ++_iter247)
+    std::vector<std::string> ::const_iterator _iter239;
+    for (_iter239 = this->values.begin(); _iter239 != this->values.end(); ++_iter239)
     {
-      xfer += oprot->writeString((*_iter247));
+      xfer += oprot->writeString((*_iter239));
     }
     xfer += oprot->writeListEnd();
   }
@@ -5536,11 +5512,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 _iter248;
-    for (_iter248 = this->parameters.begin(); _iter248 != this->parameters.end(); ++_iter248)
+    std::map<std::string, std::string> ::const_iterator _iter240;
+    for (_iter240 = this->parameters.begin(); _iter240 != this->parameters.end(); ++_iter240)
     {
-      xfer += oprot->writeString(_iter248->first);
-      xfer += oprot->writeString(_iter248->second);
+      xfer += oprot->writeString(_iter240->first);
+      xfer += oprot->writeString(_iter240->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -5569,27 +5545,27 @@ void swap(Partition &a, Partition &b) {
   swap(a.__isset, b.__isset);
 }
 
-Partition::Partition(const Partition& other249) {
-  values = other249.values;
-  dbName = other249.dbName;
-  tableName = other249.tableName;
-  createTime = other249.createTime;
-  lastAccessTime = other249.lastAccessTime;
-  sd = other249.sd;
-  parameters = other249.parameters;
-  privileges = other249.privileges;
-  __isset = other249.__isset;
-}
-Partition& Partition::operator=(const Partition& other250) {
-  values = other250.values;
-  dbName = other250.dbName;
-  tableName = other250.tableName;
-  createTime = other250.createTime;
-  lastAccessTime = other250.lastAccessTime;
-  sd = other250.sd;
-  parameters = other250.parameters;
-  privileges = other250.privileges;
-  __isset = other250.__isset;
+Partition::Partition(const Partition& other241) {
+  values = other241.values;
+  dbName = other241.dbName;
+  tableName = other241.tableName;
+  createTime = other241.createTime;
+  lastAccessTime = other241.lastAccessTime;
+  sd = other241.sd;
+  parameters = other241.parameters;
+  privileges = other241.privileges;
+  __isset = other241.__isset;
+}
+Partition& Partition::operator=(const Partition& other242) {
+  values = other242.values;
+  dbName = other242.dbName;
+  tableName = other242.tableName;
+  createTime = other242.createTime;
+  lastAccessTime = other242.lastAccessTime;
+  sd = other242.sd;
+  parameters = other242.parameters;
+  privileges = other242.privileges;
+  __isset = other242.__isset;
   return *this;
 }
 void Partition::printTo(std::ostream& out) const {
@@ -5661,14 +5637,14 @@ uint32_t PartitionWithoutSD::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->values.clear();
-            uint32_t _size251;
-            ::apache::thrift::protocol::TType _etype254;
-            xfer += iprot->readListBegin(_etype254, _size251);
-            this->values.resize(_size251);
-            uint32_t _i255;
-            for (_i255 = 0; _i255 < _size251; ++_i255)
+            uint32_t _size243;
+            ::apache::thrift::protocol::TType _etype246;
+            xfer += iprot->readListBegin(_etype246, _size243);
+            this->values.resize(_size243);
+            uint32_t _i247;
+            for (_i247 = 0; _i247 < _size243; ++_i247)
             {
-              xfer += iprot->readString(this->values[_i255]);
+              xfer += iprot->readString(this->values[_i247]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5705,17 +5681,17 @@ uint32_t PartitionWithoutSD::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->parameters.clear();
-            uint32_t _size256;
-            ::apache::thrift::protocol::TType _ktype257;
-            ::apache::thrift::protocol::TType _vtype258;
-            xfer += iprot->readMapBegin(_ktype257, _vtype258, _size256);
-            uint32_t _i260;
-            for (_i260 = 0; _i260 < _size256; ++_i260)
+            uint32_t _size248;
+            ::apache::thrift::protocol::TType _ktype249;
+            ::apache::thrift::protocol::TType _vtype250;
+            xfer += iprot->readMapBegin(_ktype249, _vtype250, _size248);
+            uint32_t _i252;
+            for (_i252 = 0; _i252 < _size248; ++_i252)
             {
-              std::string _key261;
-              xfer += iprot->readString(_key261);
-              std::string& _val262 = this->parameters[_key261];
-              xfer += iprot->readString(_val262);
+              std::string _key253;
+              xfer += iprot->readString(_key253);
+              std::string& _val254 = this->parameters[_key253];
+              xfer += iprot->readString(_val254);
             }
             xfer += iprot->readMapEnd();
           }
@@ -5752,10 +5728,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 _iter263;
-    for (_iter263 = this->values.begin(); _iter263 != this->values.end(); ++_iter263)
+    std::vector<std::string> ::const_iterator _iter255;
+    for (_iter255 = this->values.begin(); _iter255 != this->values.end(); ++_iter255)
     {
-      xfer += oprot->writeString((*_iter263));
+      xfer += oprot->writeString((*_iter255));
     }
     xfer += oprot->writeListEnd();
   }
@@ -5776,11 +5752,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 _iter264;
-    for (_iter264 = this->parameters.begin(); _iter264 != this->parameters.end(); ++_iter264)
+    std::map<std::string, std::string> ::const_iterator _iter256;
+    for (_iter256 = this->parameters.begin(); _iter256 != this->parameters.end(); ++_iter256)
     {
-      xfer += oprot->writeString(_iter264->first);
-      xfer += oprot->writeString(_iter264->second);
+      xfer += oprot->writeString(_iter256->first);
+      xfer += oprot->writeString(_iter256->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -5807,23 +5783,23 @@ void swap(PartitionWithoutSD &a, PartitionWithoutSD &b) {
   swap(a.__isset, b.__isset);
 }
 
-PartitionWithoutSD::PartitionWithoutSD(const PartitionWithoutSD& other265) {
-  values = other265.values;
-  createTime = other265.createTime;
-  lastAccessTime = other265.lastAccessTime;
-  relativePath = other265.relativePath;
-  parameters = other265.parameters;
-  privileges = other265.privileges;
-  __isset = other265.__isset;
-}
-PartitionWithoutSD& PartitionWithoutSD::operator=(const PartitionWithoutSD& other266) {
-  values = other266.values;
-  createTime = other266.createTime;
-  lastAccessTime = other266.lastAccessTime;
-  relativePath = other266.relativePath;
-  parameters = other266.parameters;
-  privileges = other266.privileges;
-  __isset = other266.__isset;
+PartitionWithoutSD::PartitionWithoutSD(const PartitionWithoutSD& other257) {
+  values = other257.values;
+  createTime = other257.createTime;
+  lastAccessTime = other257.lastAccessTime;
+  relativePath = other257.relativePath;
+  parameters = other257.parameters;
+  privileges = other257.privileges;
+  __isset = other257.__isset;
+}
+PartitionWithoutSD& PartitionWithoutSD::operator=(const PartitionWithoutSD& other258) {
+  values = other258.values;
+  createTime = other258.createTime;
+  lastAccessTime = other258.lastAccessTime;
+  relativePath = other258.relativePath;
+  parameters = other258.parameters;
+  privileges = other258.privileges;
+  __isset = other258.__isset;
   return *this;
 }
 void PartitionWithoutSD::printTo(std::ostream& out) const {
@@ -5876,14 +5852,14 @@ uint32_t PartitionSpecWithSharedSD::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitions.clear();
-            uint32_t _size267;
-            ::apache::thrift::protocol::TType _etype270;
-            xfer += iprot->readListBegin(_etype270, _size267);
-            this->partitions.resize(_size267);
-            uint32_t _i271;
-            for (_i271 = 0; _i271 < _size267; ++_i271)
+            uint32_t _size259;
+            ::apache::thrift::protocol::TType _etype262;
+            xfer += iprot->readListBegin(_etype262, _size259);
+            this->partitions.resize(_size259);
+            uint32_t _i263;
+            for (_i263 = 0; _i263 < _size259; ++_i263)
             {
-              xfer += this->partitions[_i271].read(iprot);
+              xfer += this->partitions[_i263].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5920,10 +5896,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 _iter272;
-    for (_iter272 = this->partitions.begin(); _iter272 != this->partitions.end(); ++_iter272)
+    std::vector<PartitionWithoutSD> ::const_iterator _iter264;
+    for (_iter264 = this->partitions.begin(); _iter264 != this->partitions.end(); ++_iter264)
     {
-      xfer += (*_iter272).write(oprot);
+      xfer += (*_iter264).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5945,15 +5921,15 @@ void swap(PartitionSpecWithSharedSD &a, PartitionSpecWithSharedSD &b) {
   swap(a.__isset, b.__isset);
 }
 
-PartitionSpecWithSharedSD::PartitionSpecWithSharedSD(const PartitionSpecWithSharedSD& other273) {
-  partitions = other273.partitions;
-  sd = other273.sd;
-  __isset = other273.__isset;
+PartitionSpecWithSharedSD::PartitionSpecWithSharedSD(const PartitionSpecWithSharedSD& other265) {
+  partitions = other265.partitions;
+  sd = other265.sd;
+  __isset = other265.__isset;
 }
-PartitionSpecWithSharedSD& PartitionSpecWithSharedSD::operator=(const PartitionSpecWithSharedSD& other274) {
-  partitions = other274.partitions;
-  sd = other274.sd;
-  __isset = other274.__isset;
+PartitionSpecWithSharedSD& PartitionSpecWithSharedSD::operator=(const PartitionSpecWithSharedSD& other266) {
+  partitions = other266.partitions;
+  sd = other266.sd;
+  __isset = other266.__isset;
   return *this;
 }
 void PartitionSpecWithSharedSD::printTo(std::ostream& out) const {
@@ -5998,14 +5974,14 @@ uint32_t PartitionListComposingSpec::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitions.clear();
-            uint32_t _size275;
-            ::apache::thrift::protocol::TType _etype278;
-            xfer += iprot->readListBegin(_etype278, _size275);
-            this->partitions.resize(_size275);
-            uint32_t _i279;
-            for (_i279 = 0; _i279 < _size275; ++_i279)
+            uint32_t _size267;
+            ::apache::thrift::protocol::TType _etype270;
+            xfer += iprot->readListBegin(_etype270, _size267);
+            this->partitions.resize(_size267);
+            uint32_t _i271;
+            for (_i271 = 0; _i271 < _size267; ++_i271)
             {
-              xfer += this->partitions[_i279].read(iprot);
+              xfer += this->partitions[_i271].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6034,10 +6010,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 _iter280;
-    for (_iter280 = this->partitions.begin(); _iter280 != this->partitions.end(); ++_iter280)
+    std::vector<Partition> ::const_iterator _iter272;
+    for (_iter272 = this->partitions.begin(); _iter272 != this->partitions.end(); ++_iter272)
     {
-      xfer += (*_iter280).write(oprot);
+      xfer += (*_iter272).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -6054,13 +6030,13 @@ void swap(PartitionListComposingSpec &a, PartitionListComposingSpec &b) {
   swap(a.__isset, b.__isset);
 }
 
-PartitionListComposingSpec::PartitionListComposingSpec(const PartitionListComposingSpec& other281) {
-  partitions = other281.partitions;
-  __isset = other281.__isset;
+PartitionListComposingSpec::PartitionListComposingSpec(const PartitionListComposingSpec& other273) {
+  partitions = other273.partitions;
+  __isset = other273.__isset;
 }
-PartitionListComposingSpec& PartitionListComposingSpec::operator=(const PartitionListComposingSpec& other282) {
-  partitions = other282.partitions;
-  __isset = other282.__isset;
+PartitionListComposingSpec& PartitionListComposingSpec::operator=(const PartitionListComposingSpec& other274) {
+  partitions = other274.partitions;
+  __isset = other274.__isset;
   return *this;
 }
 void PartitionListComposingSpec::printTo(std::ostream& out) const {
@@ -6212,21 +6188,21 @@ void swap(PartitionSpec &a, PartitionSpec &b) {
   swap(a.__isset, b.__isset);
 }
 
-PartitionSpec::PartitionSpec(const PartitionSpec& other283) {
-  dbName = other283.dbName;
-  tableName = other283.tableName;
-  rootPath = other283.rootPath;
-  sharedSDPartitionSpec = other283.sharedSDPartitionSpec;
-  partitionList = other283.partitionList;
-  __isset = other283.__isset;
-}
-PartitionSpec& PartitionSpec::operator=(const PartitionSpec& other284) {
-  dbName = other284.dbName;
-  tableName = other284.tableName;
-  rootPath = other284.rootPath;
-  sharedSDPartitionSpec = other284.sharedSDPartitionSpec;
-  partitionList = other284.partitionList;
-  __isset = other284.__isset;
+PartitionSpec::PartitionSpec(const PartitionSpec& other275) {
+  dbName = other275.dbName;
+  tableName = other275.tableName;
+  rootPath = other275.rootPath;
+  sharedSDPartitionSpec = other275.sharedSDPartitionSpec;
+  partitionList = other275.partitionList;
+  __isset = other275.__isset;
+}
+PartitionSpec& PartitionSpec::operator=(const PartitionSpec& other276) {
+  dbName = other276.dbName;
+  tableName = other276.tableName;
+  rootPath = other276.rootPath;
+  sharedSDPartitionSpec = other276.sharedSDPartitionSpec;
+  partitionList = other276.partitionList;
+  __isset = other276.__isset;
   return *this;
 }
 void PartitionSpec::printTo(std::ostream& out) const {
@@ -6374,17 +6350,17 @@ uint32_t Index::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->parameters.clear();
-            uint32_t _size285;
-            ::apache::thrift::protocol::TType _ktype286;
-            ::apache::thrift::protocol::TType _vtype287;
-            xfer += iprot->readMapBegin(_ktype286, _vtype287, _size285);
-            uint32_t _i289;
-            for (_i289 = 0; _i289 < _size285; ++_i289)
+            uint32_t _size277;
+            ::apache::thrift::protocol::TType _ktype278;
+            ::apache::thrift::protocol::TType _vtype279;
+            xfer += iprot->readMapBegin(_ktype278, _vtype279, _size277);
+            uint32_t _i281;
+            for (_i281 = 0; _i281 < _size277; ++_i281)
             {
-              std::string _key290;
-              xfer += iprot->readString(_key290);
-              std::string& _val291 = this->parameters[_key290];
-              xfer += iprot->readString(_val291);
+              std::string _key282;
+              xfer += iprot->readString(_key282);
+              std::string& _val283 = this->parameters[_key282];
+              xfer += iprot->readString(_val283);
             }
             xfer += iprot->readMapEnd();
           }
@@ -6453,11 +6429,11 @@ uint32_t Index::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 _iter292;
-    for (_iter292 = this->parameters.begin(); _iter292 != this->parameters.end(); ++_iter292)
+    std::map<std::string, std::string> ::const_iterator _iter284;
+    for (_iter284 = this->parameters.begin(); _iter284 != this->parameters.end(); ++_iter284)
     {
-      xfer += oprot->writeString(_iter292->first);
-      xfer += oprot->writeString(_iter292->second);
+      xfer += oprot->writeString(_iter284->first);
+      xfer += oprot->writeString(_iter284->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -6487,31 +6463,31 @@ void swap(Index &a, Index &b) {
   swap(a.__isset, b.__isset);
 }
 
-Index::Index(const Index& other293) {
-  indexName = other293.indexName;
-  indexHandlerClass = other293.indexHandlerClass;
-  dbName = other293.dbName;
-  origTableName = other293.origTableName;
-  createTime = other293.createTime;
-  lastAccessTime = other293.lastAccessTime;
-  indexTableName = other293.indexTableName;
-  sd = other293.sd;
-  parameters = other293.parameters;
-  deferredRebuild = other293.deferredRebuild;
-  __isset = other293.__isset;
-}
-Index& Index::operator=(const Index& other294) {
-  indexName = other294.indexName;
-  indexHandlerClass = other294.indexHandlerClass;
-  dbName = other294.dbName;
-  origTableName = other294.origTableName;
-  createTime = other294.createTime;
-  lastAccessTime = other294.lastAccessTime;
-  indexTableName = other294.indexTableName;
-  sd = other294.sd;
-  parameters = other294.parameters;
-  deferredRebuild = other294.deferredRebuild;
-  __isset = other294.__isset;
+Index::Index(const Index& other285) {
+  indexName = other285.indexName;
+  indexHandlerClass = other285.indexHandlerClass;
+  dbName = other285.dbName;
+  origTableName = other285.origTableName;
+  createTime = other285.createTime;
+  lastAccessTime = other285.lastAccessTime;
+  indexTableName = other285.indexTableName;
+  sd = other285.sd;
+  parameters = other285.parameters;
+  deferredRebuild = other285.deferredRebuild;
+  __isset = other285.__isset;
+}
+Index& Index::operator=(const Index& other286) {
+  indexName = other286.indexName;
+  indexHandlerClass = other286.indexHandlerClass;
+  dbName = other286.dbName;
+  origTableName = other286.origTableName;
+  createTime = other286.createTime;
+  lastAccessTime = other286.lastAccessTime;
+  indexTableName = other286.indexTableName;
+  sd = other286.sd;
+  parameters = other286.parameters;
+  deferredRebuild = other286.deferredRebuild;
+  __isset = other286.__isset;
   return *this;
 }
 void Index::printTo(std::ostream& out) const {
@@ -6662,19 +6638,19 @@ void swap(BooleanColumnStatsData &a, BooleanColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-BooleanColumnStatsData::BooleanColumnStatsData(const BooleanColumnStatsData& other295) {
-  numTrues = other295.numTrues;
-  numFalses = other295.numFalses;
-  numNulls = other295.numNulls;
-  bitVectors = other295.bitVectors;
-  __isset = other295.__isset;
+BooleanColumnStatsData::BooleanColumnStatsData(const BooleanColumnStatsData& other287) {
+  numTrues = other287.numTrues;
+  numFalses = other287.numFalses;
+  numNulls = other287.numNulls;
+  bitVectors = other287.bitVectors;
+  __isset = other287.__isset;
 }
-BooleanColumnStatsData& BooleanColumnStatsData::operator=(const BooleanColumnStatsData& other296) {
-  numTrues = other296.numTrues;
-  numFalses = other296.numFalses;
-  numNulls = other296.numNulls;
-  bitVectors = other296.bitVectors;
-  __isset = other296.__isset;
+BooleanColumnStatsData& BooleanColumnStatsData::operator=(const BooleanColumnStatsData& other288) {
+  numTrues = other288.numTrues;
+  numFalses = other288.numFalses;
+  numNulls = other288.numNulls;
+  bitVectors = other288.bitVectors;
+  __isset = other288.__isset;
   return *this;
 }
 void BooleanColumnStatsData::printTo(std::ostream& out) const {
@@ -6837,21 +6813,21 @@ void swap(DoubleColumnStatsData &a, DoubleColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-DoubleColumnStatsData::DoubleColumnStatsData(const DoubleColumnStatsData& other297) {
-  lowValue = other297.lowValue;
-  highValue = other297.highValue;
-  numNulls = other297.numNulls;
-  numDVs = other297.numDVs;
-  bitVectors = other297.bitVectors;
-  __isset = other297.__isset;
-}
-DoubleColumnStatsData& DoubleColumnStatsData::operator=(const DoubleColumnStatsData& other298) {
-  lowValue = other298.lowValue;
-  highValue = other298.highValue;
-  numNulls = other298.numNulls;
-  numDVs = other298.numDVs;
-  bitVectors = other298.bitVectors;
-  __isset = other298.__isset;
+DoubleColumnStatsData::DoubleColumnStatsData(const DoubleColumnStatsData& other289) {
+  lowValue = other289.lowValue;
+  highValue = other289.highValue;
+  numNulls = other289.numNulls;
+  numDVs = other289.numDVs;
+  bitVectors = other289.bitVectors;
+  __isset = other289.__isset;
+}
+DoubleColumnStatsData& DoubleColumnStatsData::operator=(const DoubleColumnStatsData& other290) {
+  lowValue = other290.lowValue;
+  highValue = other290.highValue;
+  numNulls = other290.numNulls;
+  numDVs = other290.numDVs;
+  bitVectors = other290.bitVectors;
+  __isset = other290.__isset;
   return *this;
 }
 void DoubleColumnStatsData::printTo(std::ostream& out) const {
@@ -7015,21 +6991,21 @@ void swap(LongColumnStatsData &a, LongColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-LongColumnStatsData::LongColumnStatsData(const LongColumnStatsData& other299) {
-  lowValue = other299.lowValue;
-  highValue = other299.highValue;
-  numNulls = other299.numNulls;
-  numDVs = other299.numDVs;
-  bitVectors = other299.bitVectors;
-  __isset = other299.__isset;
-}
-LongColumnStatsData& LongColumnStatsData::operator=(const LongColumnStatsData& other300) {
-  lowValue = other300.lowValue;
-  highValue = other300.highValue;
-  numNulls = other300.numNulls;
-  numDVs = other300.numDVs;
-  bitVectors = other300.bitVectors;
-  __isset = other300.__isset;
+LongColumnStatsData::LongColumnStatsData(const LongColumnStatsData& other291) {
+  lowValue = other291.lowValue;
+  highValue = other291.highValue;
+  numNulls = other291.numNulls;
+  numDVs = other291.numDVs;
+  bitVectors = other291.bitVectors;
+  __isset = other291.__isset;
+}
+LongColumnStatsData& LongColumnStatsData::operator=(const LongColumnStatsData& other292) {
+  lowValue = other292.lowValue;
+  highValue = other292.highValue;
+  numNulls = other292.numNulls;
+  numDVs = other292.numDVs;
+  bitVectors = other292.bitVectors;
+  __isset = other292.__isset;
   return *this;
 }
 void LongColumnStatsData::printTo(std::ostream& out) const {
@@ -7195,21 +7171,21 @@ void swap(StringColumnStatsData &a, StringColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-StringColumnStatsData::StringColumnStatsData(const StringColumnStatsData& other301) {
-  maxColLen = other301.maxColLen;
-  avgColLen = other301.avgColLen;
-  numNulls = other301.numNulls;
-  numDVs = other301.numDVs;
-  bitVectors = other301.bitVectors;
-  __isset = other301.__isset;
-}
-StringColumnStatsData& StringColumnStatsData::operator=(const StringColumnStatsData& other302) {
-  maxColLen = other302.maxColLen;
-  avgColLen = other302.avgColLen;
-  numNulls = other302.numNulls;
-  numDVs = other302.numDVs;
-  bitVectors = other302.bitVectors;
-  __isset = other302.__isset;
+StringColumnStatsData::StringColumnStatsData(const StringColumnStatsData& other293) {
+  maxColLen = other293.maxColLen;
+  avgColLen = other293.avgColLen;
+  numNulls = other293.numNulls;
+  numDVs = other293.numDVs;
+  bitVectors = other293.bitVectors;
+  __isset = other293.__isset;
+}
+StringColumnStatsData& StringColumnStatsData::operator=(const StringColumnStatsData& other294) {
+  maxColLen = other294.maxColLen;
+  avgColLen = other294.avgColLen;
+  numNulls = other294.numNulls;
+  numDVs = other294.numDVs;
+  bitVectors = other294.bitVectors;
+  __isset = other294.__isset;
   return *this;
 }
 void StringColumnStatsData::printTo(std::ostream& out) const {
@@ -7355,19 +7331,19 @@ void swap(BinaryColumnStatsData &a, BinaryColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-BinaryColumnStatsData::BinaryColumnStatsData(const BinaryColumnStatsData& other303) {
-  maxColLen = other303.maxColLen;
-  avgColLen = other303.avgColLen;
-  numNulls = other303.numNulls;
-  bitVectors = other303.bitVectors;
-  __isset = other303.__isset;
+BinaryColumnStatsData::BinaryColumnStatsData(const BinaryColumnStatsData& other295) {
+  maxColLen = other295.maxColLen;
+  avgColLen = other295.avgColLen;
+  numNulls = other295.numNulls;
+  bitVectors = other295.bitVectors;
+  __isset = other295.__isset;
 }
-BinaryColumnStatsData& BinaryColumnStatsData::operator=(const BinaryColumnStatsData& other304) {
-  maxColLen = other304.maxColLen;
-  avgColLen = other304.avgColLen;
-  numNulls = other304.numNulls;
-  bitVectors = other304.bitVectors;
-  __isset = other304.__isset;
+BinaryColumnStatsData& BinaryColumnStatsData::operator=(const BinaryColumnStatsData& other296) {
+  maxColLen = other296.maxColLen;
+  avgColLen = other296.avgColLen;
+  numNulls = other296.numNulls;
+  bitVectors = other296.bitVectors;
+  __isset = other296.__isset;
   return *this;
 }
 void BinaryColumnStatsData::printTo(std::ostream& out) const {
@@ -7472,13 +7448,13 @@ void swap(Decimal &a, Decimal &b) {
   swap(a.scale, b.scale);
 }
 
-Decimal::Decimal(const Decimal& other305) {
-  unscaled = other305.unscaled;
-  scale = other305.scale;
+Decimal::Decimal(const Decimal& other297) {
+  unscaled = other297.unscaled;
+  scale = other297.scale;
 }
-Decimal& Decimal::operator=(const Decimal& other306) {
-  unscaled = other306.unscaled;
-  scale = other306.scale;
+Decimal& Decimal::operator=(const Decimal& other298) {
+  unscaled = other298.unscaled;
+  scale = other298.scale;
   return *this;
 }
 void Decimal::printTo(std::ostream& out) const {
@@ -7639,21 +7615,21 @@ void swap(DecimalColumnStatsData &a, DecimalColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-DecimalColumnStatsData::DecimalColumnStatsData(const DecimalColumnStatsData& other307) {
-  lowValue = other307.lowValue;
-  highValue = other307.highValue;
-  numNulls = other307.numNulls;
-  numDVs = other307.numDVs;
-  bitVectors = other307.bitVectors;
-  __isset = other307.__isset;
-}
-DecimalColumnStatsData& DecimalColumnStatsData::operator=(const DecimalColumnStatsData& other308) {
-  lowValue = other308.lowValue;
-  highValue = other308.highValue;
-  numNulls = other308.numNulls;
-  numDVs = other308.numDVs;
-  bitVectors = other308.bitVectors;
-  __isset = other308.__isset;
+DecimalColumnStatsData::DecimalColumnStatsData(const DecimalColumnStatsData& other299) {
+  lowValue = other299.lowValue;
+  highValue = other299.highValue;
+  numNulls = other299.numNulls;
+  numDVs = other299.numDVs;
+  bitVectors = other299.bitVectors;
+  __isset = other299.__isset;
+}
+DecimalColumnStatsData& DecimalColumnStatsData::operator=(const DecimalColumnStatsData& other300) {
+  lowValue = other300.lowValue;
+  highValue = other300.highValue;
+  numNulls = other300.numNulls;
+  numDVs = other300.numDVs;
+  bitVectors = other300.bitVectors;
+  __isset = other300.__isset;
   return *this;
 }
 void DecimalColumnStatsData::printTo(std::ostream& out) const {
@@ -7739,11 +7715,11 @@ void swap(Date &a, Date &b) {
   swap(a.daysSinceEpoch, b.daysSinceEpoch);
 }
 
-Date::Date(const Date& other309) {
-  daysSinceEpoch = other309.daysSinceEpoch;
+Date::Date(const Date& other301) {
+  daysSinceEpoch = other301.daysSinceEpoch;
 }
-Date& Date::operator=(const Date& other310) {
-  daysSinceEpoch = other310.daysSinceEpoch;
+Date& Date::operator=(const Date& other302) {
+  daysSinceEpoch = other302.daysSinceEpoch;
   return *this;
 }
 void Date::printTo(std::ostream& out) const {
@@ -7903,21 +7879,21 @@ void swap(DateColumnStatsData &a, DateColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-DateColumnStatsData::DateColumnStatsData(const DateColumnStatsData& other311) {
-  lowValue = other311.lowValue;
-  highValue = other311.highValue;
-  numNulls = other311.numNulls;
-  numDVs = other311.numDVs;
-  bitVectors = other311.bitVectors;
-  __isset = other311.__isset;
-}
-DateColumnStatsData& DateColumnStatsData::operator=(const DateColumnStatsData& other312) {
-  lowValue = other312.lowValue;
-  highValue = other312.highValue;
-  numNulls = other312.numNulls;
-  numDVs = other312.numDVs;
-  bitVectors = other312.bitVectors;
-  __isset = other312.__isset;
+DateColumnStatsData::DateColumnStatsData(const DateColumnStatsData& other303) {
+  lowValue = other303.lowValue;
+  highValue = other303.highValue;
+  numNulls = other303.numNulls;
+  numDVs = other303.numDVs;
+  bitVectors = other303.bitVectors;
+  __isset = other303.__isset;
+}
+DateColumnStatsData& DateColumnStatsData::operator=(const DateColumnStatsData& other304) {
+  lowValue = other304.lowValue;
+  highValue = other304.highValue;
+  numNulls = other304.numNulls;
+  numDVs = other304.numDVs;
+  bitVectors = other304.bitVectors;
+  __isset = other304.__isset;
   return *this;
 }
 void DateColumnStatsData::printTo(std::ostream& out) const {
@@ -8103,25 +8079,25 @@ void swap(ColumnStatisticsData &a, ColumnStatisticsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-ColumnStatisticsData::ColumnStatisticsData(const ColumnStatisticsData& other313) {
-  booleanStats = other313.booleanStats;
-  longStats = other313.longStats;
-  doubleStats = other313.doubleStats;
-  stringStats = other313.stringStats;
-  binaryStats = other313.binaryStats;
-  decimalStats = other313.decimalStats;
-  dateStats = other313.dateStats;
-  __isset = other313.__isset;
-}
-ColumnStatisticsData& ColumnStatisticsData::operator=(const ColumnStatisticsData& other314) {
-  booleanStats = other314.booleanStats;
-  longStats = other314.longStats;
-  doubleStats = other314.doubleStats;
-  stringStats = other314.stringStats;
-  binaryStats = other314.binaryStats;
-  decimalStats = other314.decimalStats;
-  dateStats = other314.dateStats;
-  __isset = other314.__isset;
+ColumnStatisticsData::ColumnStatisticsData(const ColumnStatisticsData& other305) {
+  booleanStats = other305.booleanStats;
+  longStats = other305.longStats;
+  doubleStats = other305.doubleStats;
+  stringStats = other305.stringStats;
+  binaryStats = other305.binaryStats;
+  decimalStats = other305.decimalStats;
+  dateStats = other305.dateStats;
+  __isset = other305.__isset;
+}
+ColumnStatisticsData& ColumnStatisticsData::operator=(const ColumnStatisticsData& other306) {
+  booleanStats = other306.booleanStats;
+  longStats = other306.longStats;
+  doubleStats = other306.doubleStats;
+  stringStats = other306.stringStats;
+  binaryStats = other306.binaryStats;
+  decimalStats = other306.decimalStats;
+  dateStats = other306.dateStats;
+  __isset = other306.__isset;
   return *this;
 }
 void ColumnStatisticsData::printTo(std::ostream& out) const {
@@ -8249,15 +8225,15 @@ void swap(ColumnStatisticsObj &a, ColumnStatisticsObj &b) {
   swap(a.statsData, b.statsData);
 }
 
-ColumnStatisticsObj::ColumnStatisticsObj(const ColumnStatisticsObj& other315) {
-  colName = other315.colName;
-  colType = other315.colType;
-  statsData = other315.statsData;
+ColumnStatisticsObj::ColumnStatisticsObj(const ColumnStatisticsObj& other307) {
+  colName = other307.colName;
+  colType = other307.colType;
+  statsData = other307.statsData;
 }
-ColumnStatisticsObj& ColumnStatisticsObj::operator=(const ColumnStatisticsObj& other316) {
-  colName = other316.colName;
-  colType = other316.colType;
-  statsData = other316.statsData;
+ColumnStatisticsObj& ColumnStatisticsObj::operator=(const ColumnStatisticsObj& other308) {
+  colName = other308.colName;
+  colType = other308.colType;
+  statsData = other308.statsData;
   return *this;
 }
 void ColumnStatisticsObj::printTo(std::ostream& out) const {
@@ -8420,21 +8396,21 @@ void swap(ColumnStatisticsDesc &a, ColumnStatisticsDesc &b) {
   swap(a.__isset, b.__isset);
 }
 
-ColumnStatisticsDesc::ColumnStatisticsDesc(const ColumnStatisticsDesc& other317) {
-  isTblLevel = other317.isTblLevel;
-  dbName = other317.dbName;
-  tableName = other317.tableName;
-  partName = other317.partName;
-  lastAnalyzed = other317.lastAnalyzed;
-  __isset = other317.__isset;
-}
-ColumnStatisticsDesc& ColumnStatisticsDesc::operator=(const ColumnStatisticsDesc& other318) {
-  isTblLevel = other318.isTblLevel;
-  dbName = other318.dbName;
-  tableName = other318.tableName;
-  partName = other318.partName;
-  lastAnalyzed = other318.lastAnalyzed;
-  __isset = other318.__isset;
+ColumnStatisticsDesc::ColumnStatisticsDesc(const ColumnStatisticsDesc& other309) {
+  isTblLevel = other309.isTblLevel;
+  dbName = other309.dbName;
+  tableName = other309.tableName;
+  partName = other309.partName;
+  lastAnalyzed = other309.lastAnalyzed;
+  __isset = other309.__isset;
+}
+ColumnStatisticsDesc& ColumnStatisticsDesc::operator=(const ColumnStatisticsDesc& other310) {
+  isTblLevel = other310.isTblLevel;
+  dbName = other310.dbName;
+  tableName = other310.tableName;
+  partName = other310.partName;
+  lastAnalyzed = other310.lastAnalyzed;
+  __isset = other310.__isset;
   return *this;
 }
 void ColumnStatisticsDesc::printTo(std::ostream& out) const {
@@ -8496,14 +8472,14 @@ uint32_t ColumnStatistics::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->statsObj.clear();
-            uint32_t _size319;
-            ::apache::thrift::protocol::TType _etype322;
-            xfer += iprot->readListBegin(_etype322, _size319);
-            this->statsObj.resize(_size319);
-            uint32_t _i323;
-            for (_i323 = 0; _i323 < _size319; ++_i323)
+            uint32_t _size311;
+            ::apache::thrift::protocol::TType _etype314;
+            xfer += iprot->readListBegin(_etype314, _size311);
+            this->statsObj.resize(_size311);
+            uint32_t _i315;
+            for (_i315 = 0; _i315 < _size311; ++_i315)
             {
-              xfer += this->statsObj[_i323].read(iprot);
+              xfer += this->statsObj[_i315].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8540,10 +8516,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 _iter324;
-    for (_iter324 = this->statsObj.begin(); _iter324 != this->statsObj.end(); ++_iter324)
+    std::vector<ColumnStatisticsObj> ::const_iterator _iter316;
+    for (_iter316 = this->statsObj.begin(); _iter316 != this->statsObj.end(); ++_iter316)
     {
-      xfer += (*_iter324).write(oprot);
+      xfer += (*_iter316).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8560,13 +8536,13 @@ void swap(ColumnStatistics &a, ColumnStatistics &b) {
   swap(a.statsObj, b.statsObj);
 }
 
-ColumnStatistics::ColumnStatistics(const ColumnStatistics& other325) {
-  statsDesc = other325.statsDesc;
-  statsObj = other325.statsObj;
+ColumnStatistics::ColumnStatistics(const ColumnStatistics& other317) {
+  statsDesc = other317.statsDesc;
+  statsObj = other317.statsObj;
 }
-ColumnStatistics& ColumnStatistics::operator=(const ColumnStatistics& other326) {
-  statsDesc = other326.statsDesc;
-  statsObj = other326.statsObj;
+ColumnStatistics& ColumnStatistics::operator=(const ColumnStatistics& other318) {
+  statsDesc = other318.statsDesc;
+  statsObj = other318.statsObj;
   return *this;
 }
 void ColumnStatistics::printTo(std::ostream& out) const {
@@ -8617,14 +8593,14 @@ uint32_t AggrStats::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->colStats.clear();
-            uint32_t _size327;
-            ::apache::thrift::protocol::TType _etype330;
-            xfer += iprot->readListBegin(_etype330, _size327);
-            this->colStats.resize(_size327);
-            uint32_t _i331;
-            for (_i331 = 0; _i331 < _size327; ++_i331)
+            uint32_t _size319;
+            ::apache::thrift::protocol::TType _etype322;
+            xfer += iprot->readListBegin(_etype322, _size319);
+            this->colStats.resize(_size319);
+            uint32_t _i323;
+            for (_i323 = 0; _i323 < _size319; ++_i323)
             {
-              xfer += this->colStats[_i331].read(iprot);
+              xfer += this->colStats[_i323].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8665,10 +8641,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 _iter332;
-    for (_iter332 = this->colStats.begin(); _iter332 != this->colStats.end(); ++_iter332)
+    std::vector<ColumnStatisticsObj> ::const_iterator _iter324;
+    for (_iter324 = this->colStats.begin(); _iter324 != this->colStats.end(); ++_iter324)
     {
-      xfer += (*_iter332).write(oprot);
+      xfer += (*_iter324).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8689,13 +8665,13 @@ void swap(AggrStats &a, AggrStats &b) {
   swap(a.partsFound, b.partsFound);
 }
 
-AggrStats::AggrStats(const AggrStats& other333) {
-  colStats = other333.colStats;
-  partsFound = other333.partsFound;
+AggrStats::AggrStats(const AggrStats& other325) {
+  colStats = other325.colStats;
+  partsFound = other325.partsFound;
 }
-AggrStats& AggrStats::operator=(const AggrStats& other334) {
-  colStats = other334.colStats;
-  partsFound = other334.partsFound;
+AggrStats& AggrStats::operator=(const AggrStats& other326) {
+  colStats = other326.colStats;
+  partsFound = other326.partsFound;
   return *this;
 }
 void AggrStats::printTo(std::ostream& out) const {
@@ -8746,14 +8722,14 @@ uint32_t SetPartitionsStatsRequest::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->colStats.clear();
-            uint32_t _size335;
-            ::apache::thrift::protocol::TType _etype338;
-            xfer += iprot->readListBegin(_etype338, _size335);
-            this->colStats.resize(_size335);
-            uint32_t _i339;
-            for (_i339 = 0; _i339 < _size335; ++_i339)
+            uint32_t _size327;
+            ::apache::thrift::protocol::TType _etype330;
+            xfer += iprot->readListBegin(_etype330, _size327);
+            this->colStats.resize(_size327);
+            uint32_t _i331;
+            for (_i331 = 0; _i331 < _size327; ++_i331)
             {
-              xfer += this->colStats[_i339].read(iprot);
+              xfer += this->colStats[_i331].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8792,10 +8768,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 _iter340;
-    for (_iter340 = this->colStats.begin(); _iter340 != this->colStats.end(); ++_iter340)
+    std::vector<ColumnStatistics> ::const_iterator _iter332;
+    for (_iter332 = this->colStats.begin(); _iter332 != this->colStats.end(); ++_iter332)
     {
-      xfer += (*_iter340).write(oprot);
+      xfer += (*_iter332).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8818,15 +8794,15 @@ void swap(SetPartitionsStatsRequest &a, SetPartitionsStatsRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-SetPartitionsStatsRequest::SetPartitionsStatsRequest(const SetPartitionsStatsRequest& other341) {
-  colStats = other341.colStats;
-  needMerge = other341.needMerge;
-  __isset = other341.__isset;
+SetPartitionsStatsRequest::SetPartitionsStatsRequest(const SetPartitionsStatsRequest& other333) {
+  colStats = other333.colStats;
+  needMerge = other333.needMerge;
+  __isset = other333.__isset;
 }
-SetPartitionsStatsRequest& SetPartitionsStatsRequest::operator=(const SetPartitionsStatsRequest& other342) {
-  colStats = other342.colStats;
-  needMerge = other342.needMerge;
-  __isset = other342.__isset;
+SetPartitionsStatsRequest& SetPartitionsStatsRequest::operator=(const SetPartitionsStatsRequest& other334) {
+  colStats = other334.colStats;
+  needMerge = other334.needMerge;
+  __isset = other334.__isset;
   return *this;
 }
 void SetPartitionsStatsRequest::printTo(std::ostream& out) const {
@@ -8875,14 +8851,14 @@ uint32_t Schema::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->fieldSchemas.clear();
-            uint32_t _size343;
-            ::apache::thrift::protocol::TType _etype346;
-            xfer += iprot->readListBegin(_etype346, _size343);
-            this->fieldSchemas.resize(_size343);
-            uint32_t _i347;
-            for (_i347 = 0; _i347 < _size343; ++_i347)
+            uint32_t _size335;
+            ::apache::thrift::protocol::TType _etype338;
+            xfer += iprot->readListBegin(_etype338, _size335);
+            this->fieldSchemas.resize(_size335);
+            uint32_t _i339;
+            for (_i339 = 0; _i339 < _size335; ++_i339)
             {
-              xfer += this->fieldSchemas[_i347].read(iprot);
+              xfer += this->fieldSchemas[_i339].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8895,17 +8871,17 @@ uint32_t Schema::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->properties.clear();
-            uint32_t _size348;
-            ::apache::thrift::protocol::TType _ktype349;
-            ::apache::thrift::protocol::TType _vtype350;
-            xfer += iprot->readMapBegin(_ktype349, _vtype350, _size348);
-            uint32_t _i352;
-            for (_i352 = 0; _i352 < _size348; ++_i352)
+            uint32_t _size340;
+            ::apache::thrift::protocol::TType _ktype341;
+            ::apache::thrift::protocol::TType _vtype342;
+            xfer += iprot->readMapBegin(_ktype341, _vtype342, _size340);
+            uint32_t _i344;
+            for (_i344 = 0; _i344 < _size340; ++_i344)
             {
-              std::string _key353;
-              xfer += iprot->readString(_key353);
-              std::string& _val354 = this->properties[_key353];
-              xfer += iprot->readString(_val354);
+              std::string _key345;
+              xfer += iprot->readString(_key345);
+              std::string& _val346 = this->properties[_key345];
+              xfer += iprot->readString(_val346);
             }
             xfer += iprot->readMapEnd();
           }
@@ -8934,10 +8910,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 _iter355;
-    for (_iter355 = this->fieldSchemas.begin(); _iter355 != this->fieldSchemas.end(); ++_iter355)
+    std::vector<FieldSchema> ::const_iterator _iter347;
+    for (_iter347 = this->fieldSchemas.begin(); _iter347 != this->fieldSchemas.end(); ++_iter347)
     {
-      xfer += (*_iter355).write(oprot);
+      xfer += (*_iter347).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8946,11 +8922,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 _iter356;
-    for (_iter356 = this->properties.begin(); _iter356 != this->properties.end(); ++_iter356)
+    std::map<std::string, std::string> ::const_iterator _iter348;
+    for (_iter348 = this->properties.begin(); _iter348 != this->properties.end(); ++_iter348)
     {
-      xfer += oprot->writeString(_iter356->first);
-      xfer += oprot->writeString(_iter356->second);
+      xfer += oprot->writeString(_iter348->first);
+      xfer += oprot->writeString(_iter348->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -8968,15 +8944,15 @@ void swap(Schema &a, Schema &b) {
   swap(a.__isset, b.__isset);
 }
 
-Schema::Schema(const Schema& other357) {
-  fieldSchemas = other357.fieldSchemas;
-  properties = other357.properties;
-  __isset = other357.__isset;
+Schema::Schema(const Schema& other349) {
+  fieldSchemas = other349.fieldSchemas;
+  properties = other349.properties;
+  __isset = other349.__isset;
 }
-Schema& Schema::operator=(const Schema& other358) {
-  fieldSchemas = other358.fieldSchemas;
-  properties = other358.properties;
-  __isset = other358.__isset;
+Schema& Schema::operator=(const Schema& other350) {
+  fieldSchemas = other350.fieldSchemas;
+  properties = other350.properties;
+  __isset = other350.__isset;
   return *this;
 }
 void Schema::printTo(std::ostream& out) const {
@@ -9021,17 +8997,17 @@ uint32_t EnvironmentContext::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->properties.clear();
-            uint32_t _size359;
-            ::apache::thrift::protocol::TType _ktype360;
-            ::apache::thrift::protocol::TType _vtype361;
-            xfer += iprot->readMapBegin(_ktype360, _vtype361, _size359);
-            uint32_t _i363;
-            for (_i363 = 0; _i363 < _size359; ++_i363)
+            uint32_t _size351;
+            ::apache::thrift::protocol::TType _ktype352;
+            ::apache::thrift::protocol::TType _vtype353;
+            xfer += iprot->readMapBegin(_ktype352, _vtype353, _size351);
+            uint32_t _i355;
+            for (_i355 = 0; _i355 < _size351; ++_i355)
             {
-              std::string _key364;
-              xfer += iprot->readString(_key364);
-              std::string& _val365 = this->properties[_key364];
-              xfer += iprot->readString(_val365);
+              std::string _key356;
+              xfer += iprot->readString(_key356);
+              std::string& _val357 = this->properties[_key356];
+              xfer += iprot->readString(_val357);
             }
             xfer += iprot->readMapEnd();
           }
@@ -9060,11 +9036,11 @@ uint32_t EnvironmentContext::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("properties", ::apache::thrift::protocol::T_MAP, 1);
   {
     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 _iter366;
-    for (_iter366 = this->properties.begin(); _iter366 != this->properties.end(); ++_iter366)
+    std::map<std::string, std::string> ::const_iterator _iter358;
+    for (_iter358 = this->properties.begin(); _iter358 != this->properties.end(); ++_iter358)
     {
-      xfer += oprot->writeString(_iter366->first);
-      xfer += oprot->writeString(_iter366->second);
+      xfer += oprot->writeString(_iter358->first);
+      xfer += oprot->writeString(_iter358->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -9081,13 +9057,13 @@ void swap(EnvironmentContext &a, EnvironmentContext &b) {
   swap(a.__isset, b.__isset);
 }
 
-EnvironmentContext::EnvironmentContext(const EnvironmentContext& other367) {
-  properties = other367.properties;
-  __isset = other367.__isset;
+EnvironmentContext::EnvironmentContext(const EnvironmentContext& other359) {
+  properties = other359.properties;
+  __isset = other359.__isset;
 }
-EnvironmentContext& EnvironmentContext::operator=(const EnvironmentContext& other368) {
-  properties = other368.properties;
-  __isset = other368.__isset;
+EnvironmentContext& EnvironmentContext::operator=(const EnvironmentContext& other360) {
+  properties = other360.properties;
+  __isset = other360.__isset;
   return *this;
 }
 void EnvironmentContext::printTo(std::ostream& out) const {
@@ -9189,13 +9165,13 @@ void swap(PrimaryKeysRequest &a, PrimaryKeysRequest &b) {
   swap(a.tbl_name, b.tbl_name);
 }
 
-PrimaryKeysRequest::PrimaryKeysRequest(const PrimaryKeysRequest& other369) {
-  db_name = other369.db_name;
-  tbl_name = other369.tbl_name;
+PrimaryKeysRequest::PrimaryKeysRequest(const PrimaryKeysRequest& other361) {
+  db_name = other361.db_name;
+  tbl_name = other361.tbl_name;
 }
-PrimaryKeysRequest& PrimaryKeysRequest::operator=(const PrimaryKeysRequest& other370) {
-  db_name = other370.db_name;
-  tbl_name = other370.tbl_name;
+PrimaryKeysRequest& PrimaryKeysRequest::operator=(const PrimaryKeysRequest& other362) {
+  db_name = other362.db_name;
+  tbl_name = other362.tbl_name;
   return *this;
 }
 void PrimaryKeysRequest::printTo(std::ostream& out) const {
@@ -9241,14 +9217,14 @@ uint32_t PrimaryKeysResponse::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->primaryKeys.clear();
-            uint32_t _size371;
-            ::apache::thrift::protocol::TType _etype374;
-            xfer += iprot->readListBegin(_etype374, _size371);
-            this->primaryKeys.resize(_size371);
-            uint32_t _i375;
-            for (_i375 = 0; _i375 < _size371; ++_i375)
+            uint32_t _size363;
+            ::apache::thrift::protocol::TType _etype366;
+            xfer += iprot->readListBegin(_etype366, _size363);
+            this->primaryKeys.resize(_size363);
+            uint32_t _i367;
+            for (_i367 = 0; _i367 < _size363; ++_i367)
             {
-              xfer += this->primaryKeys[_i375].read(iprot);
+              xfer += this->primaryKeys[_i367].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9279,10 +9255,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 _iter376;
-    for (_iter376 = this->primaryKeys.begin(); _iter376 != this->primaryKeys.end(); ++_iter376)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter368;
+    for (_iter368 = this->primaryKeys.begin(); _iter368 != this->primaryKeys.end(); ++_iter368)
     {
-      xfer += (*_iter376).write(oprot);
+      xfer += (*_iter368).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9298,11 +9274,11 @@ void swap(PrimaryKeysResponse &a, PrimaryKeysResponse &b) {
   swap(a.primaryKeys, b.primaryKeys);
 }
 
-PrimaryKeysResponse::PrimaryKeysResponse(const PrimaryKeysResponse& other377) {
-  primaryKeys = other377.primaryKeys;
+PrimaryKeysResponse::PrimaryKeysResponse(const PrimaryKeysResponse& other369) {
+  primaryKeys = other369.primaryKeys;
 }
-PrimaryKeysResponse& PrimaryKeysResponse::operator=(const PrimaryKeysResponse& other378) {
-  primaryKeys = other378.primaryKeys;
+PrimaryKeysResponse& PrimaryKeysResponse::operator=(const PrimaryKeysResponse& other370) {
+  primaryKeys = other370.primaryKeys;
   return *this;
 }
 void PrimaryKeysResponse::printTo(std::ostream& out) const {
@@ -9433,19 +9409,19 @@ void swap(ForeignKeysRequest &a, ForeignKeysRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-ForeignKeysRequest::ForeignKeysRequest(const ForeignKeysRequest& other379) {
-  parent_db_name = other379.parent_db_name;
-  parent_tbl_name = other379.parent_tbl_name;
-  foreign_db_name = other379.foreign_db_name;
-  foreign_tbl_name = other379.foreign_tbl_name;
-  __isset = other379.__isset;
+ForeignKeysRequest::ForeignKeysRequest(const ForeignKeysRequest& other371) {
+  parent_db_name = other371.parent_db_name;
+  parent_tbl_name = other371.parent_tbl_name;
+  foreign_db_name = other371.foreign_db_name;
+  foreign_tbl_name = other371.foreign_tbl_name;
+  __isset = other371.__isset;
 }
-ForeignKeysRequest& ForeignKeysRequest::operator=(const ForeignKeysRequest& other380) {
-  parent_db_name = other380.parent_db_name;
-  parent_tbl_name = other380.parent_tbl_name;
-  foreign_db_name = other380.foreign_db_name;
-  foreign_tbl_name = other380.foreign_tbl_name;
-  __isset = other380.__isset;
+ForeignKeysRequest& ForeignKeysRequest::operator=(const ForeignKeysRequest& other372) {
+  parent_db_name = other372.parent_db_name;
+  parent_tbl_name = other372.parent_tbl_name;
+  foreign_db_name = other372.foreign_db_name;
+  foreign_tbl_name = other372.foreign_tbl_name;
+  __isset = other372.__isset;
   return *this;
 }
 void ForeignKeysRequest::printTo(std::ostream& out) const {
@@ -9493,14 +9469,14 @@ uint32_t ForeignKeysResponse::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->foreignKeys.clear();
-            uint32_t _size381;
-            ::apache::thrift::protocol::TType _etype384;
-            xfer += iprot->readListBegin(_etype384, _size381);
-            this->foreignKeys.resize(_size381);
-            uint32_t _i385;
-            for (_i385 = 0; _i385 < _size381; ++_i385)
+            uint32_t _size373;
+            ::apache::thrift::protocol::TType _etype376;
+            xfer += iprot->readListBegin(_etype376, _size373);
+            this->foreignKeys.resize(_size373);
+            uint32_t _i377;
+            for (_i377 = 0; _i377 < _size373; ++_i377)
             {
-              xfer += this->foreignKeys[_i385].read(iprot);
+              xfer += this->foreignKeys[_i377].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9531,10 +9507,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 _iter386;
-    for (_iter386 = this->foreignKeys.begin(); _iter386 != this->foreignKeys.end(); ++_iter386)
+    std::vector<SQLForeignKey> ::const_iterator _iter378;
+    for (_iter378 = this->foreignKeys.begin(); _iter378 != this->foreignKeys.end(); ++_iter378)
     {
-      xfer += (*_iter386).write(oprot);
+      xfer += (*_iter378).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9550,11 +9526,11 @@ void swap(ForeignKeysResponse &a, ForeignKeysResponse &b) {
   swap(a.foreignKeys, b.foreignKeys);
 }
 
-ForeignKeysResponse::ForeignKeysResponse(const ForeignKeysResponse& other387) {
-  foreignKeys = other387.foreignKeys;
+ForeignKeysResponse::ForeignKeysResponse(const ForeignKeysResponse& other379) {
+  foreignKeys = other379.foreignKeys;
 }
-ForeignKeysResponse& ForeignKeysResponse::operator=(const ForeignKeysResponse& other388) {
-  foreignKeys = other388.foreignKeys;
+ForeignKeysResponse& ForeignKeysResponse::operator=(const ForeignKeysResponse& other380) {
+  foreignKeys = other380.foreignKeys;
   return *this;
 }
 void ForeignKeysResponse::printTo(std::ostream& out) const {
@@ -9656,13 +9632,13 @@ void swap(UniqueConstraintsRequest &a, UniqueConstraintsRequest &b) {
   swap(a.tbl_name, b.tbl_name);
 }
 
-UniqueConstraintsRequest::UniqueConstraintsRequest(const UniqueConstraintsRequest& other389) {
-  db_name = other389.db_name;
-  tbl_name = other389.tbl_name;
+UniqueConstraintsRequest::UniqueConstraintsRequest(const UniqueConstraintsRequest& other381) {
+  db_name = other381.db_name;
+  tbl_name = other381.tbl_name;
 }
-UniqueConstraintsRequest& UniqueConstraintsRequest::operator=(const UniqueConstraintsRequest& other390) {
-  db_name = other390.db_name;
-  tbl_name = other390.tbl_name;
+UniqueConstraintsRequest& UniqueConstraintsRequest::operator=(const UniqueConstraintsRequest& other382) {
+  db_name = other382.db_name;
+  tbl_name = other382.tbl_name;
   return *this;
 }
 void UniqueConstraintsRequest::printTo(std::ostream& out) const {
@@ -9708,14 +9684,14 @@ uint32_t UniqueConstraintsResponse::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->uniqueConstraints.clear();
-            uint32_t _size391;
-            ::apache::thrift::protocol::TType _etype394;
-            xfer += iprot->readListBegin(_etype394, _size391);
-            this->uniqueConstraints.resize(_size391);
-            uint32_t _i395;
-            for (_i395 = 0; _i395 < _size391; ++_i395)
+            uint32_t _size383;
+            ::apache::thrift::protocol::TType _etype386;
+            xfer += iprot->readListBegin(_etype386, _size383);
+            this->uniqueConstraints.resize(_size383);
+            uint32_t _i387;
+            for (_i387 = 0; _i387 < _size383; ++_i387)
             {
-              xfer += this->uniqueConstraints[_i395].read(iprot);
+              xfer += this->uniqueConstraints[_i387].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9746,10 +9722,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 _iter396;
-    for (_iter396 = this->uniqueConstraints.begin(); _iter396 != this->uniqueConstraints.end(); ++_iter396)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter388;
+    for (_iter388 = this->uniqueConstraints.begin(); _iter388 != this->uniqueConstraints.end(); ++_iter388)
     {
-      xfer += (*_iter396).write(oprot);
+      xfer += (*_iter388).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9765,11 +9741,11 @@ void swap(UniqueConstraintsResponse &a, UniqueConstraintsResponse &b) {
   swap(a.uniqueConstraints, b.uniqueConstraints);
 }
 
-UniqueConstraintsResponse::UniqueConstraintsResponse(const UniqueConstraintsResponse& other397) {
-  uniqueConstraints = other397.uniqueConstraints;
+UniqueConstraintsResponse::UniqueConstraintsResponse(const UniqueConstraintsResponse& other389) {
+  uniqueConstraints = other389.uniqueConstraints;
 }
-UniqueConstraintsResponse& UniqueConstraintsResponse::operator=(const UniqueConstraintsResponse& other398) {
-  uniqueConstraints = other398.uniqueConstraints;
+UniqueConstraintsResponse& UniqueConstraintsResponse::operator=(const UniqueConstraintsResponse& other390) {
+  uniqueConstraints = other390.uniqueConstraints;
   return *this;
 }
 void UniqueConstraintsResponse::printTo(std::ostream& out) const {
@@ -9871,13 +9847,13 @@ void swap(NotNullConstraintsRequest &a, NotNullConstraintsRequest &b) {
   swap(a.tbl_name, b.tbl_name);
 }
 
-NotNullConstraintsRequest::NotNullConstraintsRequest(const NotNullConstraintsRequest& other399) {
-  db_name = other399.db_name;
-  tbl_name = other399.tbl_name;
+NotNullConstraintsRequest::NotNullConstraintsRequest(const NotNullConstraintsRequest& other391) {
+  db_name = other391.db_name;
+  tbl_name = other391.tbl_name;
 }
-NotNullConstraintsRequest& NotNullConstraintsRequest::operator=(const NotNullConstraintsRequest& other400) {
-  db_name = other400.db_name;
-  tbl_name = other400.tbl_name;
+NotNullConstraintsRequest& NotNullConstraintsRequest::operator=(const NotNullConstraintsRequest& other392) {
+  db_name = other392.db_name;
+  tbl_name = other392.tbl_name;
   return *this;
 }
 void NotNullConstraintsRequest::printTo(std::ostream& out) const {
@@ -9923,14 +9899,14 @@ uint32_t NotNullConstraintsResponse::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->notNullConstraints.clear();
-            uint32_t _size401;
-            ::apache::thrift::protocol::TType _etype404;
-            xfer += iprot->readListBegin(_etype404, _size401);
-            this->notNullConstraints.resize(_size401);
-            uint32_t _i405;
-            for (_i405 = 0; _i405 < _size401; ++_i405)
+            uint32_t _size393;
+            ::apache::thrift::protocol::TType _etype396;
+            xfer += iprot->readListBegin(_etype396, _size393);
+            this->notNullConstraints.resize(_size393);
+            uint32_t _i397;
+            for (_i397 = 0; _i397 < _size393; ++_i397)
             {
-              xfer += this->notNullConstraints[_i405].read(iprot);
+              xfer += this->notNullConstraints[_i397].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9961,10 +9937,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 _iter406;
-    for (_iter406 = this->notNullConstraints.begin(); _iter406 != this->notNullConstraints.end(); ++_iter406)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter398;
+    for (_iter398 = this->notNullConstraints.begin(); _iter398 != this->notNullConstraints.end(); ++_iter398)
     {
-      xfer += (*_iter406).write(oprot);
+      xfer += (*_iter398).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9980,11 +9956,11 @@ void swap(NotNullConstraintsResponse &a, NotNullConstraintsResponse &b) {
   swap(a.notNullConstraints, b.notNullConstraints);
 }
 
-NotNullConstraintsResponse::NotNullConstraintsResponse(const NotNullConstraintsResponse& other407) {
-  notNullConstraints = other407.notNullConstraints;
+NotNullConstraintsResponse::NotNullConstraintsResponse(const NotNullConstraintsResponse& other399) {
+  notNullConstraints = other399.notNullConstraints;
 }
-NotNullConstraintsResponse& NotNullConstraintsResponse::operator=(const NotNullConstraintsResponse& other408) {
-  notNullConstraints = other408.notNullConstraints;
+NotNullConstraintsResponse& NotNullConstraintsResponse::operator=(const NotNullConstraintsResponse& other400) {
+  notNullConstraints = other400.notNullConstraints;
   return *this;
 }
 void NotNullConstraintsResponse::printTo(std::ostream& out) const {
@@ -10106,15 +10082,15 @@ void swap(DropConstraintRequest &a, DropConstraintRequest &b) {
   swap(a.constraintname, b.constraintname);
 }
 
-DropConstraintRequest::DropConstraintRequest(const DropConstraintRequest& other409) {
-  dbname = other409.dbname;
-  tablename = other409.tablename;
-  constraintname = other409.constraintname;
+DropConstraintRequest::DropConstraintRequest(const DropConstraintRequest& other401) {
+  dbname = other401.dbname;
+  tablename = other401.tablename;
+  constraintname = other401.constraintname;
 }
-DropConstraintRequest& DropConstraintRequest::operator=(const DropConstraintRequest& other410) {
-  dbname = other410.dbname;
-  tablename = other410.tablename;
-  constraintname = other410.constraintname;
+DropConstraintRequest& DropConstraintRequest::operator=(const DropConstraintRequest& other402) {
+  dbname = other402.dbname;
+  tablename = other402.tablename;
+  constraintname = other402.constraintname;
   return *this;
 }
 void DropConstraintRequest::printTo(std::ostream& out) const {
@@ -10161,14 +10137,14 @@ uint32_t AddPrimaryKeyRequest::read(::apache::thrift::protocol::TProtocol* iprot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->primaryKeyCols.clear();
-            uint32_t _size411;
-            ::apache::thrift::protocol::TType _etype414;
-            xfer += iprot->readListBegin(_etype414, _size411);
-            this->primaryKeyCols.resize(_size411);
-            uint32_t _i415;
-            for (_i415 = 0; _i415 < _size411; ++_i415)
+            uint32_t _size403;
+            ::apache::thrift::protocol::TType _etype406;
+            xfer += iprot->readListBegin(_etype406, _size403);
+            this->primaryKeyCols.resize(_size403);
+            uint32_t _i407;
+            for (_i407 = 0; _i407 < _size403; ++_i407)
             {
-              xfer += this->primaryKeyCols[_i415].read(iprot);
+              xfer += this->primaryKeyCols[_i407].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -10199,10 +10175,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 _iter416;
-    for (_iter416 = this->primaryKeyCols.begin(); _iter416 != this->primaryKeyCols.end(); ++_iter416)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter408;
+    for (_iter408 = this->primaryKeyCols.begin(); _iter408 != this->primaryKeyCols.end(); ++_iter408)
     {
-      xfer += (*_iter416).write(oprot);
+      xfer += (*_iter408).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -10218,11 +10194,11 @@ void swap(AddPrimaryKeyRequest &a, AddPrimaryKeyRequest &b) {
   swap(a.primaryKeyCols, b.primaryKeyCols);
 }
 
-AddPrimaryKeyRequest::AddPrimaryKeyRequest(const AddPrimaryKeyRequest& other417) {
-  primaryKeyCols = other417.primaryKeyCols;
+AddPrimaryKeyRequest::AddPrimaryKeyRequest(const AddPrimaryKeyRequest& other409) {
+  primaryKeyCols = other409.primaryKeyCols;
 }
-AddPrimaryKeyRequest& AddPrimaryKeyRequest::operator=(const AddPrimaryKeyRequest& other418) {
-  primaryKeyCols = other418.primaryKeyCols;
+AddPrimaryKeyRequest& AddPrimaryKeyRequest::operator=(const AddPrimaryKeyRequest& other410) {
+  primaryKeyCols = other410.primaryKeyCols;
   return *this;
 }
 void AddPrimaryKeyRequest::printTo(std::ostream& out) const {
@@ -10267,14 +10243,14 @@ uint32_t AddForeignKeyRequest::read(::apache::thrift::protocol::TProtocol* iprot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->foreignKeyCols.clear();
-            uint32_t _size419;
-            ::apache::thrift::protocol::TType _etype422;
-            xfer += iprot->readListBegin(_etype422, _size419);
-            this->foreignKeyCols.resize(_size419);
-            uint32_t _i423;
-            for (_i423 = 0; _i423 < _size419; ++_i423)
+            uint32_t _size411;
+            ::apache::thrift::protocol::TType _etype414;
+            xfer += iprot->readListBegin(_etype414, _size411);
+            this->foreignKeyCols.resize(_size411);
+            uint32_t _i415;
+            for (_i415 = 0; _i415 < _size411; ++_i415)
             {
-              xfer += this->foreignKeyCols[_i423].read(iprot);
+              xfer += this->foreignKeyCols[_i415].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -10305,10 +10281,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 _iter424;
-    for (_iter424 = this->foreignKeyCols.begin(); _iter424 != this->foreignKeyCols.end(); ++_iter424)
+    std::vector<SQLForeignKey> ::const_iterator _iter416;
+    for (_iter416 = this->foreignKeyCols.begin(); _iter416 != this->foreignKeyCols.end(); ++_iter416)
     {
-      xfer += (*_iter424).write(oprot);
+      xfer += (*_iter416).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -10324,11 +10300,11 @@ void swap(AddForeignKeyRequest &a, AddForeignKeyRequest &b) {
   swap(a.foreignKeyCols, b.foreignKeyCols);
 }
 
-AddForeignKeyRequest::AddForeignKeyRequest(const AddForeignKeyRequest& other425) {
-  foreignKeyCols = other425.foreignKeyCols;
+AddForeignKeyRequest::AddForeignKeyRequest(const AddForeignKeyRequest& other417) {
+  foreignKeyCols = other417.foreignKeyCols;
 }
-AddForeignKeyRequest& AddForeignKeyRequest::operator=(const AddForeignKeyRequest& other426) {
-  foreignKeyCols = other426.foreignKeyCols;
+AddForeignKeyRequest& AddForeignKeyRequest::operator=(const AddForeignKeyRequest& other418) {
+  foreignKeyCols = other418.foreignKeyCols;
   return *this;
 }
 void AddForeignKeyRequest::printTo(std::ostream& out) const {
@@ -10373,14 +10349,14 @@ uint32_t AddUniqueConstraintRequest::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->uniqueConstraintCols.clear();
-            uint32_t _size427;
-            ::apache::thrift::protocol::TType _etype430;
-            xfer += iprot->readListBegin(_etype430, _size427);
-            this->uniqueConstraintCols.resize(_size427);
-            uint32_t _i431;
-            for (_i431 = 0; _i431 < _size427; ++_i431)
+            uint32_t _size419;
+            ::apache::thrift::protocol::TType _etype422;
+            xfer += iprot->readListBegin(_etype422, _size419);
+            this->uniqueConstraintCols.resize(_size419);
+            uint32_t _i423;
+            for (_i423 = 0; _i423 < _size419; ++_i423)
             {
-              xfer += this->uniqueConstraintCols[_i431].read(iprot);
+              xfer += this->uniqueConstraintCols[_i423].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -10411,10 +10387,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 _iter432;
-    for (_iter432 = this->uniqueConstraintCols.begin(); _iter432 != this->uniqueConstraintCols.end(); ++_iter432)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter424;
+    for (_iter424 = this->uniqueConstraintCols.begin(); _iter424 != this->uniqueConstraintCols.end(); ++_iter424)
     {
-      xfer += (*_iter432).write(oprot);
+      xfer += (*_iter424).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -10430,11 +10406,11 @@ void swap(AddUniqueConstraintRequest &a, AddUniqueConstraintRequest &b) {
   swap(a.uniqueConstraintCols, b.uniqueConstraintCols);
 }
 
-AddUniqueConstraintRequest::AddUniqueConstraintRequest(const AddUniqueConstraintRequest& other433) {
-  uniqueConstraintCols = other433.uniqueConstraintCols;
+AddUniqueConstraintRequest::AddUniqueConstraintRequest(const AddUniqueConstraintRequest& other425) {
+  uniqueConstraintCols = other425.uniqueConstraintCols;
 }
-AddUniqueConstraintRequest& AddUniqueConstraintRequest::operator=(const AddUniqueConstraintRequest& other434) {
-  uniqueConstraintCols = other434.uniqueConstraintCols;
+AddUniqueConstraintRequest& AddUniqueConstraintRequest::operator=(const AddUniqueConstraintRequest& other426) {
+  uniqueConstraintCols = other426.uniqueConstraintCols;
   return *this;
 }
 void AddUniqueConstraintRequest::printTo(std::ostream& out) const {
@@ -10479,14 +10455,14 @@ uint32_t AddNotNullConstraintRequest::read(::apache::thrift::protocol::TProtocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->notNullConstraintCols.clear();
-            uint32_t _size435;
-            ::apache::thrift::protocol::TType _etype438;
-            xfer += iprot->readListBegin(_etype438, _size435);
-            this->notNullConstraintCols.resize(_size435);
-            uint32_t _i439;
-            for (_i439 = 0; _i439 < _size435; ++_i439)
+            uint32_t _size427;
+            ::apache::thrift::protocol::TType _etype430;
+            xfer += iprot->readListBegin(_etype430, _size427);
+            this->notNullConstraintCols.resize(_size427);
+            uint32_t _i431;
+            for (_i431 = 0; _i431 < _size427; ++_i431)
             {
-              xfer += this->notNullConstraintCols[_i439].read(iprot);
+              xfer += this->notNullConstraintCols[_i431].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -10517,10 +10493,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 _iter440;
-    for (_iter440 = this->notNullConstraintCols.begin(); _iter440 != this->notNullConstraintCols.end(); ++_iter440)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter432;
+    for (_iter432 = this->notNullConstraintCols.begin(); _iter432 != this->notNullConstraintCols.end(); ++_iter432)
     {
-      xfer += (*_iter440).write(oprot);
+      xfer += (*_iter432).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -10536,11 +10512,11 @@ void swap(AddNotNullConstraintRequest &a, AddNotNullConstraintRequest &b) {
   swap(a.notNullConstraintCols, b.notNullConstraintCols);
 }
 
-AddNotNullConstraintRequest::AddNotNullConstraintRequest(const AddNotNullConstraintRequest& other441) {
-  notNullConstraintCols = other441.notNullConstraintCols;
+AddNotNullConstraintRequest::AddNotNullConstraintRequest(const AddNotNullConstraintRequest& other433) {
+  notNullConstraintCols = other433.notNullConstraintCols;
 }
-AddNotNullConstraintRequest& AddNotNullConstraintRequest::operator=(const AddNotNullConstraintRequest& other442) {
-  notNullConstraintCols = other442.notNullConstraintCols;
+AddNotNullConstraintRequest& AddNotNullConstraintRequest::operator=(const AddNotNullConstraintRequest& other434) {
+  notNullConstraintCols = other434.notNullConstraintCols;
   return *this;
 }
 void AddNotNullConstraintRequest::printTo(std::ostream& out) const {
@@ -10590,14 +10566,14 @@ uint32_t PartitionsByExprResult::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitions.clear();
-            uint32_t _size443;
-            ::apache::thrift::protocol::TType _etype446;
-            xfer += iprot->readListBegin(_etype446, _size443);
-            this->partitions.resize(_size443);
-            uint32_t _i447;
-            for (_i447 = 0; _i447 < _size443; ++_i447)
+            uint32_t _size435;
+            ::apache::thrift::protocol::TType _etype438;
+            xfer += iprot->readListBegin(_etype438, _size435);
+            this->partitions.resize(_size435);
+            uint32_t _i439;
+            for (_i439 = 0; _i439 < _size435; ++_i439)
             {
-              xfer += this->partitions[_i447].read(iprot);
+              xfer += this->partitions[_i439].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -10638,10 +10614,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 _iter448;
-    for (_iter448 = this->partitions.begin(); _iter448 != this->partitions.end(); ++_iter448)
+    std::vector<Partition> ::const_iterator _iter440;
+    for (_iter440 = this->partitions.begin(); _iter440 != this->partitions.end(); ++_iter440)
     {
-      xfer += (*_iter448).write(oprot);
+      xfer += (*_iter440).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -10662,13 +10638,13 @@ void swap(PartitionsByExprResult &a, PartitionsByExprResult &b) {
   swap(a.hasUnknownPartitions, b.hasUnknownPartitions);
 }
 
-PartitionsByExprResult::PartitionsByExprResult(const PartitionsByExprResult& other449) {
-  partitions = other449.partitions;
-  hasUnknownPartitions = other449.hasUnknownPartitions;
+PartitionsByExprResult::PartitionsByExprResult(const PartitionsByExprResult& other441) {
+  partitions = other441.partitions;
+  hasUnknownPartitions = other441.hasUnknownPartitions;
 }
-PartitionsByExprResult& PartitionsByExprResult::operator=(const PartitionsByExprResult& other450) {
-  partitions = other450.partitions;
-  hasUnknownPartitions = other450.hasUnknownPartitions;
+PartitionsByExprResult& PartitionsByExprResult::operator=(const PartitionsByExprResult& other442) {
+  partitions = other442.partitions;
+  hasUnknownPartitions = other442.hasUnknownPartitions;
   return *this;
 }
 void PartitionsByExprResult::printTo(std::ostream& out) const {
@@ -10830,21 +10806,21 @@ void swap(PartitionsByExprRequest &a, PartitionsByExprRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-PartitionsByExprRequest::PartitionsByExprRequest(const PartitionsByExprRequest& other451) {
-  dbName = other451.dbName;
-  tblName = other451.tblName;
-  expr = other451.expr;
-  defaultPartitionName = other451.defaultPartitionName;
-  maxParts = other451.maxParts;

<TRUNCATED>

[11/16] hive git commit: HIVE-18546: Remove unnecessary code introduced in HIVE-14498 (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java
new file mode 100644
index 0000000..74cfce6
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java
@@ -0,0 +1,750 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class CreationMetadata implements org.apache.thrift.TBase<CreationMetadata, CreationMetadata._Fields>, java.io.Serializable, Cloneable, Comparable<CreationMetadata> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CreationMetadata");
+
+  private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tblName", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField TABLES_USED_FIELD_DESC = new org.apache.thrift.protocol.TField("tablesUsed", org.apache.thrift.protocol.TType.SET, (short)3);
+  private static final org.apache.thrift.protocol.TField VALID_TXN_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("validTxnList", org.apache.thrift.protocol.TType.STRING, (short)4);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new CreationMetadataStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new CreationMetadataTupleSchemeFactory());
+  }
+
+  private String dbName; // required
+  private String tblName; // required
+  private Set<String> tablesUsed; // required
+  private String validTxnList; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    DB_NAME((short)1, "dbName"),
+    TBL_NAME((short)2, "tblName"),
+    TABLES_USED((short)3, "tablesUsed"),
+    VALID_TXN_LIST((short)4, "validTxnList");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // DB_NAME
+          return DB_NAME;
+        case 2: // TBL_NAME
+          return TBL_NAME;
+        case 3: // TABLES_USED
+          return TABLES_USED;
+        case 4: // VALID_TXN_LIST
+          return VALID_TXN_LIST;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final _Fields optionals[] = {_Fields.VALID_TXN_LIST};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("dbName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TBL_NAME, new org.apache.thrift.meta_data.FieldMetaData("tblName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TABLES_USED, new org.apache.thrift.meta_data.FieldMetaData("tablesUsed", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.SetMetaData(org.apache.thrift.protocol.TType.SET, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.VALID_TXN_LIST, new org.apache.thrift.meta_data.FieldMetaData("validTxnList", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CreationMetadata.class, metaDataMap);
+  }
+
+  public CreationMetadata() {
+  }
+
+  public CreationMetadata(
+    String dbName,
+    String tblName,
+    Set<String> tablesUsed)
+  {
+    this();
+    this.dbName = dbName;
+    this.tblName = tblName;
+    this.tablesUsed = tablesUsed;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public CreationMetadata(CreationMetadata other) {
+    if (other.isSetDbName()) {
+      this.dbName = other.dbName;
+    }
+    if (other.isSetTblName()) {
+      this.tblName = other.tblName;
+    }
+    if (other.isSetTablesUsed()) {
+      Set<String> __this__tablesUsed = new HashSet<String>(other.tablesUsed);
+      this.tablesUsed = __this__tablesUsed;
+    }
+    if (other.isSetValidTxnList()) {
+      this.validTxnList = other.validTxnList;
+    }
+  }
+
+  public CreationMetadata deepCopy() {
+    return new CreationMetadata(this);
+  }
+
+  @Override
+  public void clear() {
+    this.dbName = null;
+    this.tblName = null;
+    this.tablesUsed = null;
+    this.validTxnList = null;
+  }
+
+  public String getDbName() {
+    return this.dbName;
+  }
+
+  public void setDbName(String dbName) {
+    this.dbName = dbName;
+  }
+
+  public void unsetDbName() {
+    this.dbName = null;
+  }
+
+  /** Returns true if field dbName is set (has been assigned a value) and false otherwise */
+  public boolean isSetDbName() {
+    return this.dbName != null;
+  }
+
+  public void setDbNameIsSet(boolean value) {
+    if (!value) {
+      this.dbName = null;
+    }
+  }
+
+  public String getTblName() {
+    return this.tblName;
+  }
+
+  public void setTblName(String tblName) {
+    this.tblName = tblName;
+  }
+
+  public void unsetTblName() {
+    this.tblName = null;
+  }
+
+  /** Returns true if field tblName is set (has been assigned a value) and false otherwise */
+  public boolean isSetTblName() {
+    return this.tblName != null;
+  }
+
+  public void setTblNameIsSet(boolean value) {
+    if (!value) {
+      this.tblName = null;
+    }
+  }
+
+  public int getTablesUsedSize() {
+    return (this.tablesUsed == null) ? 0 : this.tablesUsed.size();
+  }
+
+  public java.util.Iterator<String> getTablesUsedIterator() {
+    return (this.tablesUsed == null) ? null : this.tablesUsed.iterator();
+  }
+
+  public void addToTablesUsed(String elem) {
+    if (this.tablesUsed == null) {
+      this.tablesUsed = new HashSet<String>();
+    }
+    this.tablesUsed.add(elem);
+  }
+
+  public Set<String> getTablesUsed() {
+    return this.tablesUsed;
+  }
+
+  public void setTablesUsed(Set<String> tablesUsed) {
+    this.tablesUsed = tablesUsed;
+  }
+
+  public void unsetTablesUsed() {
+    this.tablesUsed = null;
+  }
+
+  /** Returns true if field tablesUsed is set (has been assigned a value) and false otherwise */
+  public boolean isSetTablesUsed() {
+    return this.tablesUsed != null;
+  }
+
+  public void setTablesUsedIsSet(boolean value) {
+    if (!value) {
+      this.tablesUsed = null;
+    }
+  }
+
+  public String getValidTxnList() {
+    return this.validTxnList;
+  }
+
+  public void setValidTxnList(String validTxnList) {
+    this.validTxnList = validTxnList;
+  }
+
+  public void unsetValidTxnList() {
+    this.validTxnList = null;
+  }
+
+  /** Returns true if field validTxnList is set (has been assigned a value) and false otherwise */
+  public boolean isSetValidTxnList() {
+    return this.validTxnList != null;
+  }
+
+  public void setValidTxnListIsSet(boolean value) {
+    if (!value) {
+      this.validTxnList = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case DB_NAME:
+      if (value == null) {
+        unsetDbName();
+      } else {
+        setDbName((String)value);
+      }
+      break;
+
+    case TBL_NAME:
+      if (value == null) {
+        unsetTblName();
+      } else {
+        setTblName((String)value);
+      }
+      break;
+
+    case TABLES_USED:
+      if (value == null) {
+        unsetTablesUsed();
+      } else {
+        setTablesUsed((Set<String>)value);
+      }
+      break;
+
+    case VALID_TXN_LIST:
+      if (value == null) {
+        unsetValidTxnList();
+      } else {
+        setValidTxnList((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case DB_NAME:
+      return getDbName();
+
+    case TBL_NAME:
+      return getTblName();
+
+    case TABLES_USED:
+      return getTablesUsed();
+
+    case VALID_TXN_LIST:
+      return getValidTxnList();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case DB_NAME:
+      return isSetDbName();
+    case TBL_NAME:
+      return isSetTblName();
+    case TABLES_USED:
+      return isSetTablesUsed();
+    case VALID_TXN_LIST:
+      return isSetValidTxnList();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof CreationMetadata)
+      return this.equals((CreationMetadata)that);
+    return false;
+  }
+
+  public boolean equals(CreationMetadata that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_dbName = true && this.isSetDbName();
+    boolean that_present_dbName = true && that.isSetDbName();
+    if (this_present_dbName || that_present_dbName) {
+      if (!(this_present_dbName && that_present_dbName))
+        return false;
+      if (!this.dbName.equals(that.dbName))
+        return false;
+    }
+
+    boolean this_present_tblName = true && this.isSetTblName();
+    boolean that_present_tblName = true && that.isSetTblName();
+    if (this_present_tblName || that_present_tblName) {
+      if (!(this_present_tblName && that_present_tblName))
+        return false;
+      if (!this.tblName.equals(that.tblName))
+        return false;
+    }
+
+    boolean this_present_tablesUsed = true && this.isSetTablesUsed();
+    boolean that_present_tablesUsed = true && that.isSetTablesUsed();
+    if (this_present_tablesUsed || that_present_tablesUsed) {
+      if (!(this_present_tablesUsed && that_present_tablesUsed))
+        return false;
+      if (!this.tablesUsed.equals(that.tablesUsed))
+        return false;
+    }
+
+    boolean this_present_validTxnList = true && this.isSetValidTxnList();
+    boolean that_present_validTxnList = true && that.isSetValidTxnList();
+    if (this_present_validTxnList || that_present_validTxnList) {
+      if (!(this_present_validTxnList && that_present_validTxnList))
+        return false;
+      if (!this.validTxnList.equals(that.validTxnList))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_dbName = true && (isSetDbName());
+    list.add(present_dbName);
+    if (present_dbName)
+      list.add(dbName);
+
+    boolean present_tblName = true && (isSetTblName());
+    list.add(present_tblName);
+    if (present_tblName)
+      list.add(tblName);
+
+    boolean present_tablesUsed = true && (isSetTablesUsed());
+    list.add(present_tablesUsed);
+    if (present_tablesUsed)
+      list.add(tablesUsed);
+
+    boolean present_validTxnList = true && (isSetValidTxnList());
+    list.add(present_validTxnList);
+    if (present_validTxnList)
+      list.add(validTxnList);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(CreationMetadata other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetDbName()).compareTo(other.isSetDbName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDbName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.dbName, other.dbName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetTblName()).compareTo(other.isSetTblName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTblName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tblName, other.tblName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetTablesUsed()).compareTo(other.isSetTablesUsed());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTablesUsed()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tablesUsed, other.tablesUsed);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetValidTxnList()).compareTo(other.isSetValidTxnList());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetValidTxnList()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.validTxnList, other.validTxnList);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("CreationMetadata(");
+    boolean first = true;
+
+    sb.append("dbName:");
+    if (this.dbName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.dbName);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("tblName:");
+    if (this.tblName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.tblName);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("tablesUsed:");
+    if (this.tablesUsed == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.tablesUsed);
+    }
+    first = false;
+    if (isSetValidTxnList()) {
+      if (!first) sb.append(", ");
+      sb.append("validTxnList:");
+      if (this.validTxnList == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.validTxnList);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetDbName()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'dbName' is unset! Struct:" + toString());
+    }
+
+    if (!isSetTblName()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'tblName' is unset! Struct:" + toString());
+    }
+
+    if (!isSetTablesUsed()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'tablesUsed' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class CreationMetadataStandardSchemeFactory implements SchemeFactory {
+    public CreationMetadataStandardScheme getScheme() {
+      return new CreationMetadataStandardScheme();
+    }
+  }
+
+  private static class CreationMetadataStandardScheme extends StandardScheme<CreationMetadata> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, CreationMetadata struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // DB_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.dbName = iprot.readString();
+              struct.setDbNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // TBL_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.tblName = iprot.readString();
+              struct.setTblNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // TABLES_USED
+            if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
+              {
+                org.apache.thrift.protocol.TSet _set614 = iprot.readSetBegin();
+                struct.tablesUsed = new HashSet<String>(2*_set614.size);
+                String _elem615;
+                for (int _i616 = 0; _i616 < _set614.size; ++_i616)
+                {
+                  _elem615 = iprot.readString();
+                  struct.tablesUsed.add(_elem615);
+                }
+                iprot.readSetEnd();
+              }
+              struct.setTablesUsedIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // VALID_TXN_LIST
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.validTxnList = iprot.readString();
+              struct.setValidTxnListIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, CreationMetadata struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.dbName != null) {
+        oprot.writeFieldBegin(DB_NAME_FIELD_DESC);
+        oprot.writeString(struct.dbName);
+        oprot.writeFieldEnd();
+      }
+      if (struct.tblName != null) {
+        oprot.writeFieldBegin(TBL_NAME_FIELD_DESC);
+        oprot.writeString(struct.tblName);
+        oprot.writeFieldEnd();
+      }
+      if (struct.tablesUsed != null) {
+        oprot.writeFieldBegin(TABLES_USED_FIELD_DESC);
+        {
+          oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, struct.tablesUsed.size()));
+          for (String _iter617 : struct.tablesUsed)
+          {
+            oprot.writeString(_iter617);
+          }
+          oprot.writeSetEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.validTxnList != null) {
+        if (struct.isSetValidTxnList()) {
+          oprot.writeFieldBegin(VALID_TXN_LIST_FIELD_DESC);
+          oprot.writeString(struct.validTxnList);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class CreationMetadataTupleSchemeFactory implements SchemeFactory {
+    public CreationMetadataTupleScheme getScheme() {
+      return new CreationMetadataTupleScheme();
+    }
+  }
+
+  private static class CreationMetadataTupleScheme extends TupleScheme<CreationMetadata> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, CreationMetadata struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.dbName);
+      oprot.writeString(struct.tblName);
+      {
+        oprot.writeI32(struct.tablesUsed.size());
+        for (String _iter618 : struct.tablesUsed)
+        {
+          oprot.writeString(_iter618);
+        }
+      }
+      BitSet optionals = new BitSet();
+      if (struct.isSetValidTxnList()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetValidTxnList()) {
+        oprot.writeString(struct.validTxnList);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, CreationMetadata struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.dbName = iprot.readString();
+      struct.setDbNameIsSet(true);
+      struct.tblName = iprot.readString();
+      struct.setTblNameIsSet(true);
+      {
+        org.apache.thrift.protocol.TSet _set619 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.tablesUsed = new HashSet<String>(2*_set619.size);
+        String _elem620;
+        for (int _i621 = 0; _i621 < _set619.size; ++_i621)
+        {
+          _elem620 = iprot.readString();
+          struct.tablesUsed.add(_elem620);
+        }
+      }
+      struct.setTablesUsedIsSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.validTxnList = iprot.readString();
+        struct.setValidTxnListIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java
index 83ff494..b9dc04a 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java
@@ -346,14 +346,14 @@ import org.slf4j.LoggerFactory;
           case 1: // PARTITIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list470 = iprot.readListBegin();
-                struct.partitions = new ArrayList<Partition>(_list470.size);
-                Partition _elem471;
-                for (int _i472 = 0; _i472 < _list470.size; ++_i472)
+                org.apache.thrift.protocol.TList _list460 = iprot.readListBegin();
+                struct.partitions = new ArrayList<Partition>(_list460.size);
+                Partition _elem461;
+                for (int _i462 = 0; _i462 < _list460.size; ++_i462)
                 {
-                  _elem471 = new Partition();
-                  _elem471.read(iprot);
-                  struct.partitions.add(_elem471);
+                  _elem461 = new Partition();
+                  _elem461.read(iprot);
+                  struct.partitions.add(_elem461);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PARTITIONS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitions.size()));
-            for (Partition _iter473 : struct.partitions)
+            for (Partition _iter463 : struct.partitions)
             {
-              _iter473.write(oprot);
+              _iter463.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitions()) {
         {
           oprot.writeI32(struct.partitions.size());
-          for (Partition _iter474 : struct.partitions)
+          for (Partition _iter464 : struct.partitions)
           {
-            _iter474.write(oprot);
+            _iter464.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list475 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.partitions = new ArrayList<Partition>(_list475.size);
-          Partition _elem476;
-          for (int _i477 = 0; _i477 < _list475.size; ++_i477)
+          org.apache.thrift.protocol.TList _list465 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.partitions = new ArrayList<Partition>(_list465.size);
+          Partition _elem466;
+          for (int _i467 = 0; _i467 < _list465.size; ++_i467)
           {
-            _elem476 = new Partition();
-            _elem476.read(iprot);
-            struct.partitions.add(_elem476);
+            _elem466 = new Partition();
+            _elem466.read(iprot);
+            struct.partitions.add(_elem466);
           }
         }
         struct.setPartitionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/EnvironmentContext.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/EnvironmentContext.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/EnvironmentContext.java
index d7c4feb..6829cfe 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/EnvironmentContext.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/EnvironmentContext.java
@@ -344,15 +344,15 @@ import org.slf4j.LoggerFactory;
           case 1: // PROPERTIES
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map322 = iprot.readMapBegin();
-                struct.properties = new HashMap<String,String>(2*_map322.size);
-                String _key323;
-                String _val324;
-                for (int _i325 = 0; _i325 < _map322.size; ++_i325)
+                org.apache.thrift.protocol.TMap _map312 = iprot.readMapBegin();
+                struct.properties = new HashMap<String,String>(2*_map312.size);
+                String _key313;
+                String _val314;
+                for (int _i315 = 0; _i315 < _map312.size; ++_i315)
                 {
-                  _key323 = iprot.readString();
-                  _val324 = iprot.readString();
-                  struct.properties.put(_key323, _val324);
+                  _key313 = iprot.readString();
+                  _val314 = iprot.readString();
+                  struct.properties.put(_key313, _val314);
                 }
                 iprot.readMapEnd();
               }
@@ -378,10 +378,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PROPERTIES_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.properties.size()));
-          for (Map.Entry<String, String> _iter326 : struct.properties.entrySet())
+          for (Map.Entry<String, String> _iter316 : struct.properties.entrySet())
           {
-            oprot.writeString(_iter326.getKey());
-            oprot.writeString(_iter326.getValue());
+            oprot.writeString(_iter316.getKey());
+            oprot.writeString(_iter316.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -412,10 +412,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetProperties()) {
         {
           oprot.writeI32(struct.properties.size());
-          for (Map.Entry<String, String> _iter327 : struct.properties.entrySet())
+          for (Map.Entry<String, String> _iter317 : struct.properties.entrySet())
           {
-            oprot.writeString(_iter327.getKey());
-            oprot.writeString(_iter327.getValue());
+            oprot.writeString(_iter317.getKey());
+            oprot.writeString(_iter317.getValue());
           }
         }
       }
@@ -427,15 +427,15 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TMap _map328 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.properties = new HashMap<String,String>(2*_map328.size);
-          String _key329;
-          String _val330;
-          for (int _i331 = 0; _i331 < _map328.size; ++_i331)
+          org.apache.thrift.protocol.TMap _map318 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.properties = new HashMap<String,String>(2*_map318.size);
+          String _key319;
+          String _val320;
+          for (int _i321 = 0; _i321 < _map318.size; ++_i321)
           {
-            _key329 = iprot.readString();
-            _val330 = iprot.readString();
-            struct.properties.put(_key329, _val330);
+            _key319 = iprot.readString();
+            _val320 = iprot.readString();
+            struct.properties.put(_key319, _val320);
           }
         }
         struct.setPropertiesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
index 4efec9d..7cc201b 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
@@ -713,13 +713,13 @@ import org.slf4j.LoggerFactory;
           case 5: // PARTITION_VALS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list656 = iprot.readListBegin();
-                struct.partitionVals = new ArrayList<String>(_list656.size);
-                String _elem657;
-                for (int _i658 = 0; _i658 < _list656.size; ++_i658)
+                org.apache.thrift.protocol.TList _list646 = iprot.readListBegin();
+                struct.partitionVals = new ArrayList<String>(_list646.size);
+                String _elem647;
+                for (int _i648 = 0; _i648 < _list646.size; ++_i648)
                 {
-                  _elem657 = iprot.readString();
-                  struct.partitionVals.add(_elem657);
+                  _elem647 = iprot.readString();
+                  struct.partitionVals.add(_elem647);
                 }
                 iprot.readListEnd();
               }
@@ -768,9 +768,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PARTITION_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partitionVals.size()));
-            for (String _iter659 : struct.partitionVals)
+            for (String _iter649 : struct.partitionVals)
             {
-              oprot.writeString(_iter659);
+              oprot.writeString(_iter649);
             }
             oprot.writeListEnd();
           }
@@ -816,9 +816,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitionVals()) {
         {
           oprot.writeI32(struct.partitionVals.size());
-          for (String _iter660 : struct.partitionVals)
+          for (String _iter650 : struct.partitionVals)
           {
-            oprot.writeString(_iter660);
+            oprot.writeString(_iter650);
           }
         }
       }
@@ -843,13 +843,13 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list661 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.partitionVals = new ArrayList<String>(_list661.size);
-          String _elem662;
-          for (int _i663 = 0; _i663 < _list661.size; ++_i663)
+          org.apache.thrift.protocol.TList _list651 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.partitionVals = new ArrayList<String>(_list651.size);
+          String _elem652;
+          for (int _i653 = 0; _i653 < _list651.size; ++_i653)
           {
-            _elem662 = iprot.readString();
-            struct.partitionVals.add(_elem662);
+            _elem652 = iprot.readString();
+            struct.partitionVals.add(_elem652);
           }
         }
         struct.setPartitionValsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ForeignKeysResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ForeignKeysResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ForeignKeysResponse.java
index 2b921c5..75b2404 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ForeignKeysResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ForeignKeysResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // FOREIGN_KEYS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list340 = iprot.readListBegin();
-                struct.foreignKeys = new ArrayList<SQLForeignKey>(_list340.size);
-                SQLForeignKey _elem341;
-                for (int _i342 = 0; _i342 < _list340.size; ++_i342)
+                org.apache.thrift.protocol.TList _list330 = iprot.readListBegin();
+                struct.foreignKeys = new ArrayList<SQLForeignKey>(_list330.size);
+                SQLForeignKey _elem331;
+                for (int _i332 = 0; _i332 < _list330.size; ++_i332)
                 {
-                  _elem341 = new SQLForeignKey();
-                  _elem341.read(iprot);
-                  struct.foreignKeys.add(_elem341);
+                  _elem331 = new SQLForeignKey();
+                  _elem331.read(iprot);
+                  struct.foreignKeys.add(_elem331);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FOREIGN_KEYS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.foreignKeys.size()));
-          for (SQLForeignKey _iter343 : struct.foreignKeys)
+          for (SQLForeignKey _iter333 : struct.foreignKeys)
           {
-            _iter343.write(oprot);
+            _iter333.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.foreignKeys.size());
-        for (SQLForeignKey _iter344 : struct.foreignKeys)
+        for (SQLForeignKey _iter334 : struct.foreignKeys)
         {
-          _iter344.write(oprot);
+          _iter334.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, ForeignKeysResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list345 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.foreignKeys = new ArrayList<SQLForeignKey>(_list345.size);
-        SQLForeignKey _elem346;
-        for (int _i347 = 0; _i347 < _list345.size; ++_i347)
+        org.apache.thrift.protocol.TList _list335 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.foreignKeys = new ArrayList<SQLForeignKey>(_list335.size);
+        SQLForeignKey _elem336;
+        for (int _i337 = 0; _i337 < _list335.size; ++_i337)
         {
-          _elem346 = new SQLForeignKey();
-          _elem346.read(iprot);
-          struct.foreignKeys.add(_elem346);
+          _elem336 = new SQLForeignKey();
+          _elem336.read(iprot);
+          struct.foreignKeys.add(_elem336);
         }
       }
       struct.setForeignKeysIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
index 0f5d5eb..ca62b88 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
@@ -997,14 +997,14 @@ import org.slf4j.LoggerFactory;
           case 8: // RESOURCE_URIS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list526 = iprot.readListBegin();
-                struct.resourceUris = new ArrayList<ResourceUri>(_list526.size);
-                ResourceUri _elem527;
-                for (int _i528 = 0; _i528 < _list526.size; ++_i528)
+                org.apache.thrift.protocol.TList _list516 = iprot.readListBegin();
+                struct.resourceUris = new ArrayList<ResourceUri>(_list516.size);
+                ResourceUri _elem517;
+                for (int _i518 = 0; _i518 < _list516.size; ++_i518)
                 {
-                  _elem527 = new ResourceUri();
-                  _elem527.read(iprot);
-                  struct.resourceUris.add(_elem527);
+                  _elem517 = new ResourceUri();
+                  _elem517.read(iprot);
+                  struct.resourceUris.add(_elem517);
                 }
                 iprot.readListEnd();
               }
@@ -1063,9 +1063,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(RESOURCE_URIS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.resourceUris.size()));
-          for (ResourceUri _iter529 : struct.resourceUris)
+          for (ResourceUri _iter519 : struct.resourceUris)
           {
-            _iter529.write(oprot);
+            _iter519.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1138,9 +1138,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetResourceUris()) {
         {
           oprot.writeI32(struct.resourceUris.size());
-          for (ResourceUri _iter530 : struct.resourceUris)
+          for (ResourceUri _iter520 : struct.resourceUris)
           {
-            _iter530.write(oprot);
+            _iter520.write(oprot);
           }
         }
       }
@@ -1180,14 +1180,14 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(7)) {
         {
-          org.apache.thrift.protocol.TList _list531 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.resourceUris = new ArrayList<ResourceUri>(_list531.size);
-          ResourceUri _elem532;
-          for (int _i533 = 0; _i533 < _list531.size; ++_i533)
+          org.apache.thrift.protocol.TList _list521 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.resourceUris = new ArrayList<ResourceUri>(_list521.size);
+          ResourceUri _elem522;
+          for (int _i523 = 0; _i523 < _list521.size; ++_i523)
           {
-            _elem532 = new ResourceUri();
-            _elem532.read(iprot);
-            struct.resourceUris.add(_elem532);
+            _elem522 = new ResourceUri();
+            _elem522.read(iprot);
+            struct.resourceUris.add(_elem522);
           }
         }
         struct.setResourceUrisIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
index bff424f..2b4883d 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
@@ -346,14 +346,14 @@ import org.slf4j.LoggerFactory;
           case 1: // FUNCTIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list724 = iprot.readListBegin();
-                struct.functions = new ArrayList<Function>(_list724.size);
-                Function _elem725;
-                for (int _i726 = 0; _i726 < _list724.size; ++_i726)
+                org.apache.thrift.protocol.TList _list714 = iprot.readListBegin();
+                struct.functions = new ArrayList<Function>(_list714.size);
+                Function _elem715;
+                for (int _i716 = 0; _i716 < _list714.size; ++_i716)
                 {
-                  _elem725 = new Function();
-                  _elem725.read(iprot);
-                  struct.functions.add(_elem725);
+                  _elem715 = new Function();
+                  _elem715.read(iprot);
+                  struct.functions.add(_elem715);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(FUNCTIONS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.functions.size()));
-            for (Function _iter727 : struct.functions)
+            for (Function _iter717 : struct.functions)
             {
-              _iter727.write(oprot);
+              _iter717.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetFunctions()) {
         {
           oprot.writeI32(struct.functions.size());
-          for (Function _iter728 : struct.functions)
+          for (Function _iter718 : struct.functions)
           {
-            _iter728.write(oprot);
+            _iter718.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list729 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.functions = new ArrayList<Function>(_list729.size);
-          Function _elem730;
-          for (int _i731 = 0; _i731 < _list729.size; ++_i731)
+          org.apache.thrift.protocol.TList _list719 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.functions = new ArrayList<Function>(_list719.size);
+          Function _elem720;
+          for (int _i721 = 0; _i721 < _list719.size; ++_i721)
           {
-            _elem730 = new Function();
-            _elem730.read(iprot);
-            struct.functions.add(_elem730);
+            _elem720 = new Function();
+            _elem720.read(iprot);
+            struct.functions.add(_elem720);
           }
         }
         struct.setFunctionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
index 38a5ed9..5a371c4 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
@@ -619,13 +619,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list674 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list674.size);
-                long _elem675;
-                for (int _i676 = 0; _i676 < _list674.size; ++_i676)
+                org.apache.thrift.protocol.TList _list664 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list664.size);
+                long _elem665;
+                for (int _i666 = 0; _i666 < _list664.size; ++_i666)
                 {
-                  _elem675 = iprot.readI64();
-                  struct.fileIds.add(_elem675);
+                  _elem665 = iprot.readI64();
+                  struct.fileIds.add(_elem665);
                 }
                 iprot.readListEnd();
               }
@@ -675,9 +675,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size()));
-          for (long _iter677 : struct.fileIds)
+          for (long _iter667 : struct.fileIds)
           {
-            oprot.writeI64(_iter677);
+            oprot.writeI64(_iter667);
           }
           oprot.writeListEnd();
         }
@@ -719,9 +719,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter678 : struct.fileIds)
+        for (long _iter668 : struct.fileIds)
         {
-          oprot.writeI64(_iter678);
+          oprot.writeI64(_iter668);
         }
       }
       oprot.writeBinary(struct.expr);
@@ -745,13 +745,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataByExprRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list679 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list679.size);
-        long _elem680;
-        for (int _i681 = 0; _i681 < _list679.size; ++_i681)
+        org.apache.thrift.protocol.TList _list669 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list669.size);
+        long _elem670;
+        for (int _i671 = 0; _i671 < _list669.size; ++_i671)
         {
-          _elem680 = iprot.readI64();
-          struct.fileIds.add(_elem680);
+          _elem670 = iprot.readI64();
+          struct.fileIds.add(_elem670);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
index a3dc743..6eb6eee 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
@@ -444,16 +444,16 @@ import org.slf4j.LoggerFactory;
           case 1: // METADATA
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map664 = iprot.readMapBegin();
-                struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map664.size);
-                long _key665;
-                MetadataPpdResult _val666;
-                for (int _i667 = 0; _i667 < _map664.size; ++_i667)
+                org.apache.thrift.protocol.TMap _map654 = iprot.readMapBegin();
+                struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map654.size);
+                long _key655;
+                MetadataPpdResult _val656;
+                for (int _i657 = 0; _i657 < _map654.size; ++_i657)
                 {
-                  _key665 = iprot.readI64();
-                  _val666 = new MetadataPpdResult();
-                  _val666.read(iprot);
-                  struct.metadata.put(_key665, _val666);
+                  _key655 = iprot.readI64();
+                  _val656 = new MetadataPpdResult();
+                  _val656.read(iprot);
+                  struct.metadata.put(_key655, _val656);
                 }
                 iprot.readMapEnd();
               }
@@ -487,10 +487,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(METADATA_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRUCT, struct.metadata.size()));
-          for (Map.Entry<Long, MetadataPpdResult> _iter668 : struct.metadata.entrySet())
+          for (Map.Entry<Long, MetadataPpdResult> _iter658 : struct.metadata.entrySet())
           {
-            oprot.writeI64(_iter668.getKey());
-            _iter668.getValue().write(oprot);
+            oprot.writeI64(_iter658.getKey());
+            _iter658.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -518,10 +518,10 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.metadata.size());
-        for (Map.Entry<Long, MetadataPpdResult> _iter669 : struct.metadata.entrySet())
+        for (Map.Entry<Long, MetadataPpdResult> _iter659 : struct.metadata.entrySet())
         {
-          oprot.writeI64(_iter669.getKey());
-          _iter669.getValue().write(oprot);
+          oprot.writeI64(_iter659.getKey());
+          _iter659.getValue().write(oprot);
         }
       }
       oprot.writeBool(struct.isSupported);
@@ -531,16 +531,16 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataByExprResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map670 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map670.size);
-        long _key671;
-        MetadataPpdResult _val672;
-        for (int _i673 = 0; _i673 < _map670.size; ++_i673)
+        org.apache.thrift.protocol.TMap _map660 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map660.size);
+        long _key661;
+        MetadataPpdResult _val662;
+        for (int _i663 = 0; _i663 < _map660.size; ++_i663)
         {
-          _key671 = iprot.readI64();
-          _val672 = new MetadataPpdResult();
-          _val672.read(iprot);
-          struct.metadata.put(_key671, _val672);
+          _key661 = iprot.readI64();
+          _val662 = new MetadataPpdResult();
+          _val662.read(iprot);
+          struct.metadata.put(_key661, _val662);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
index 53603af..0404358 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
@@ -351,13 +351,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list692 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list692.size);
-                long _elem693;
-                for (int _i694 = 0; _i694 < _list692.size; ++_i694)
+                org.apache.thrift.protocol.TList _list682 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list682.size);
+                long _elem683;
+                for (int _i684 = 0; _i684 < _list682.size; ++_i684)
                 {
-                  _elem693 = iprot.readI64();
-                  struct.fileIds.add(_elem693);
+                  _elem683 = iprot.readI64();
+                  struct.fileIds.add(_elem683);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size()));
-          for (long _iter695 : struct.fileIds)
+          for (long _iter685 : struct.fileIds)
           {
-            oprot.writeI64(_iter695);
+            oprot.writeI64(_iter685);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter696 : struct.fileIds)
+        for (long _iter686 : struct.fileIds)
         {
-          oprot.writeI64(_iter696);
+          oprot.writeI64(_iter686);
         }
       }
     }
@@ -421,13 +421,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list697 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list697.size);
-        long _elem698;
-        for (int _i699 = 0; _i699 < _list697.size; ++_i699)
+        org.apache.thrift.protocol.TList _list687 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list687.size);
+        long _elem688;
+        for (int _i689 = 0; _i689 < _list687.size; ++_i689)
         {
-          _elem698 = iprot.readI64();
-          struct.fileIds.add(_elem698);
+          _elem688 = iprot.readI64();
+          struct.fileIds.add(_elem688);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
index 440965e..3858890 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
@@ -433,15 +433,15 @@ import org.slf4j.LoggerFactory;
           case 1: // METADATA
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map682 = iprot.readMapBegin();
-                struct.metadata = new HashMap<Long,ByteBuffer>(2*_map682.size);
-                long _key683;
-                ByteBuffer _val684;
-                for (int _i685 = 0; _i685 < _map682.size; ++_i685)
+                org.apache.thrift.protocol.TMap _map672 = iprot.readMapBegin();
+                struct.metadata = new HashMap<Long,ByteBuffer>(2*_map672.size);
+                long _key673;
+                ByteBuffer _val674;
+                for (int _i675 = 0; _i675 < _map672.size; ++_i675)
                 {
-                  _key683 = iprot.readI64();
-                  _val684 = iprot.readBinary();
-                  struct.metadata.put(_key683, _val684);
+                  _key673 = iprot.readI64();
+                  _val674 = iprot.readBinary();
+                  struct.metadata.put(_key673, _val674);
                 }
                 iprot.readMapEnd();
               }
@@ -475,10 +475,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(METADATA_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRING, struct.metadata.size()));
-          for (Map.Entry<Long, ByteBuffer> _iter686 : struct.metadata.entrySet())
+          for (Map.Entry<Long, ByteBuffer> _iter676 : struct.metadata.entrySet())
           {
-            oprot.writeI64(_iter686.getKey());
-            oprot.writeBinary(_iter686.getValue());
+            oprot.writeI64(_iter676.getKey());
+            oprot.writeBinary(_iter676.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -506,10 +506,10 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.metadata.size());
-        for (Map.Entry<Long, ByteBuffer> _iter687 : struct.metadata.entrySet())
+        for (Map.Entry<Long, ByteBuffer> _iter677 : struct.metadata.entrySet())
         {
-          oprot.writeI64(_iter687.getKey());
-          oprot.writeBinary(_iter687.getValue());
+          oprot.writeI64(_iter677.getKey());
+          oprot.writeBinary(_iter677.getValue());
         }
       }
       oprot.writeBool(struct.isSupported);
@@ -519,15 +519,15 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map688 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.metadata = new HashMap<Long,ByteBuffer>(2*_map688.size);
-        long _key689;
-        ByteBuffer _val690;
-        for (int _i691 = 0; _i691 < _map688.size; ++_i691)
+        org.apache.thrift.protocol.TMap _map678 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.metadata = new HashMap<Long,ByteBuffer>(2*_map678.size);
+        long _key679;
+        ByteBuffer _val680;
+        for (int _i681 = 0; _i681 < _map678.size; ++_i681)
         {
-          _key689 = iprot.readI64();
-          _val690 = iprot.readBinary();
-          struct.metadata.put(_key689, _val690);
+          _key679 = iprot.readI64();
+          _val680 = iprot.readBinary();
+          struct.metadata.put(_key679, _val680);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java
index a77f661..ae644df 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java
@@ -447,14 +447,14 @@ import org.slf4j.LoggerFactory;
           case 2: // OPEN_TXNS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list534 = iprot.readListBegin();
-                struct.open_txns = new ArrayList<TxnInfo>(_list534.size);
-                TxnInfo _elem535;
-                for (int _i536 = 0; _i536 < _list534.size; ++_i536)
+                org.apache.thrift.protocol.TList _list524 = iprot.readListBegin();
+                struct.open_txns = new ArrayList<TxnInfo>(_list524.size);
+                TxnInfo _elem525;
+                for (int _i526 = 0; _i526 < _list524.size; ++_i526)
                 {
-                  _elem535 = new TxnInfo();
-                  _elem535.read(iprot);
-                  struct.open_txns.add(_elem535);
+                  _elem525 = new TxnInfo();
+                  _elem525.read(iprot);
+                  struct.open_txns.add(_elem525);
                 }
                 iprot.readListEnd();
               }
@@ -483,9 +483,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(OPEN_TXNS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.open_txns.size()));
-          for (TxnInfo _iter537 : struct.open_txns)
+          for (TxnInfo _iter527 : struct.open_txns)
           {
-            _iter537.write(oprot);
+            _iter527.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -511,9 +511,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeI64(struct.txn_high_water_mark);
       {
         oprot.writeI32(struct.open_txns.size());
-        for (TxnInfo _iter538 : struct.open_txns)
+        for (TxnInfo _iter528 : struct.open_txns)
         {
-          _iter538.write(oprot);
+          _iter528.write(oprot);
         }
       }
     }
@@ -524,14 +524,14 @@ import org.slf4j.LoggerFactory;
       struct.txn_high_water_mark = iprot.readI64();
       struct.setTxn_high_water_markIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list539 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.open_txns = new ArrayList<TxnInfo>(_list539.size);
-        TxnInfo _elem540;
-        for (int _i541 = 0; _i541 < _list539.size; ++_i541)
+        org.apache.thrift.protocol.TList _list529 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.open_txns = new ArrayList<TxnInfo>(_list529.size);
+        TxnInfo _elem530;
+        for (int _i531 = 0; _i531 < _list529.size; ++_i531)
         {
-          _elem540 = new TxnInfo();
-          _elem540.read(iprot);
-          struct.open_txns.add(_elem540);
+          _elem530 = new TxnInfo();
+          _elem530.read(iprot);
+          struct.open_txns.add(_elem530);
         }
       }
       struct.setOpen_txnsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java
index 70ea1de..662c093 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java
@@ -615,13 +615,13 @@ import org.slf4j.LoggerFactory;
           case 2: // OPEN_TXNS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list542 = iprot.readListBegin();
-                struct.open_txns = new ArrayList<Long>(_list542.size);
-                long _elem543;
-                for (int _i544 = 0; _i544 < _list542.size; ++_i544)
+                org.apache.thrift.protocol.TList _list532 = iprot.readListBegin();
+                struct.open_txns = new ArrayList<Long>(_list532.size);
+                long _elem533;
+                for (int _i534 = 0; _i534 < _list532.size; ++_i534)
                 {
-                  _elem543 = iprot.readI64();
-                  struct.open_txns.add(_elem543);
+                  _elem533 = iprot.readI64();
+                  struct.open_txns.add(_elem533);
                 }
                 iprot.readListEnd();
               }
@@ -666,9 +666,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(OPEN_TXNS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.open_txns.size()));
-          for (long _iter545 : struct.open_txns)
+          for (long _iter535 : struct.open_txns)
           {
-            oprot.writeI64(_iter545);
+            oprot.writeI64(_iter535);
           }
           oprot.writeListEnd();
         }
@@ -704,9 +704,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeI64(struct.txn_high_water_mark);
       {
         oprot.writeI32(struct.open_txns.size());
-        for (long _iter546 : struct.open_txns)
+        for (long _iter536 : struct.open_txns)
         {
-          oprot.writeI64(_iter546);
+          oprot.writeI64(_iter536);
         }
       }
       oprot.writeBinary(struct.abortedBits);
@@ -726,13 +726,13 @@ import org.slf4j.LoggerFactory;
       struct.txn_high_water_mark = iprot.readI64();
       struct.setTxn_high_water_markIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list547 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.open_txns = new ArrayList<Long>(_list547.size);
-        long _elem548;
-        for (int _i549 = 0; _i549 < _list547.size; ++_i549)
+        org.apache.thrift.protocol.TList _list537 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.open_txns = new ArrayList<Long>(_list537.size);
+        long _elem538;
+        for (int _i539 = 0; _i539 < _list537.size; ++_i539)
         {
-          _elem548 = iprot.readI64();
-          struct.open_txns.add(_elem548);
+          _elem538 = iprot.readI64();
+          struct.open_txns.add(_elem538);
         }
       }
       struct.setOpen_txnsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
index 575737d..680ce86 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
@@ -525,13 +525,13 @@ import org.slf4j.LoggerFactory;
           case 2: // TBL_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list740 = iprot.readListBegin();
-                struct.tblNames = new ArrayList<String>(_list740.size);
-                String _elem741;
-                for (int _i742 = 0; _i742 < _list740.size; ++_i742)
+                org.apache.thrift.protocol.TList _list730 = iprot.readListBegin();
+                struct.tblNames = new ArrayList<String>(_list730.size);
+                String _elem731;
+                for (int _i732 = 0; _i732 < _list730.size; ++_i732)
                 {
-                  _elem741 = iprot.readString();
-                  struct.tblNames.add(_elem741);
+                  _elem731 = iprot.readString();
+                  struct.tblNames.add(_elem731);
                 }
                 iprot.readListEnd();
               }
@@ -572,9 +572,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tblNames.size()));
-            for (String _iter743 : struct.tblNames)
+            for (String _iter733 : struct.tblNames)
             {
-              oprot.writeString(_iter743);
+              oprot.writeString(_iter733);
             }
             oprot.writeListEnd();
           }
@@ -617,9 +617,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetTblNames()) {
         {
           oprot.writeI32(struct.tblNames.size());
-          for (String _iter744 : struct.tblNames)
+          for (String _iter734 : struct.tblNames)
           {
-            oprot.writeString(_iter744);
+            oprot.writeString(_iter734);
           }
         }
       }
@@ -636,13 +636,13 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list745 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.tblNames = new ArrayList<String>(_list745.size);
-          String _elem746;
-          for (int _i747 = 0; _i747 < _list745.size; ++_i747)
+          org.apache.thrift.protocol.TList _list735 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.tblNames = new ArrayList<String>(_list735.size);
+          String _elem736;
+          for (int _i737 = 0; _i737 < _list735.size; ++_i737)
           {
-            _elem746 = iprot.readString();
-            struct.tblNames.add(_elem746);
+            _elem736 = iprot.readString();
+            struct.tblNames.add(_elem736);
           }
         }
         struct.setTblNamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
index 050d093..ccd85c4 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // TABLES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list748 = iprot.readListBegin();
-                struct.tables = new ArrayList<Table>(_list748.size);
-                Table _elem749;
-                for (int _i750 = 0; _i750 < _list748.size; ++_i750)
+                org.apache.thrift.protocol.TList _list738 = iprot.readListBegin();
+                struct.tables = new ArrayList<Table>(_list738.size);
+                Table _elem739;
+                for (int _i740 = 0; _i740 < _list738.size; ++_i740)
                 {
-                  _elem749 = new Table();
-                  _elem749.read(iprot);
-                  struct.tables.add(_elem749);
+                  _elem739 = new Table();
+                  _elem739.read(iprot);
+                  struct.tables.add(_elem739);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TABLES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.tables.size()));
-          for (Table _iter751 : struct.tables)
+          for (Table _iter741 : struct.tables)
           {
-            _iter751.write(oprot);
+            _iter741.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.tables.size());
-        for (Table _iter752 : struct.tables)
+        for (Table _iter742 : struct.tables)
         {
-          _iter752.write(oprot);
+          _iter742.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetTablesResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list753 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.tables = new ArrayList<Table>(_list753.size);
-        Table _elem754;
-        for (int _i755 = 0; _i755 < _list753.size; ++_i755)
+        org.apache.thrift.protocol.TList _list743 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.tables = new ArrayList<Table>(_list743.size);
+        Table _elem744;
+        for (int _i745 = 0; _i745 < _list743.size; ++_i745)
         {
-          _elem754 = new Table();
-          _elem754.read(iprot);
-          struct.tables.add(_elem754);
+          _elem744 = new Table();
+          _elem744.read(iprot);
+          struct.tables.add(_elem744);
         }
       }
       struct.setTablesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
index 828e94e..762f465 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
@@ -453,13 +453,13 @@ import org.slf4j.LoggerFactory;
           case 1: // ABORTED
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set582 = iprot.readSetBegin();
-                struct.aborted = new HashSet<Long>(2*_set582.size);
-                long _elem583;
-                for (int _i584 = 0; _i584 < _set582.size; ++_i584)
+                org.apache.thrift.protocol.TSet _set572 = iprot.readSetBegin();
+                struct.aborted = new HashSet<Long>(2*_set572.size);
+                long _elem573;
+                for (int _i574 = 0; _i574 < _set572.size; ++_i574)
                 {
-                  _elem583 = iprot.readI64();
-                  struct.aborted.add(_elem583);
+                  _elem573 = iprot.readI64();
+                  struct.aborted.add(_elem573);
                 }
                 iprot.readSetEnd();
               }
@@ -471,13 +471,13 @@ import org.slf4j.LoggerFactory;
           case 2: // NOSUCH
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set585 = iprot.readSetBegin();
-                struct.nosuch = new HashSet<Long>(2*_set585.size);
-                long _elem586;
-                for (int _i587 = 0; _i587 < _set585.size; ++_i587)
+                org.apache.thrift.protocol.TSet _set575 = iprot.readSetBegin();
+                struct.nosuch = new HashSet<Long>(2*_set575.size);
+                long _elem576;
+                for (int _i577 = 0; _i577 < _set575.size; ++_i577)
                 {
-                  _elem586 = iprot.readI64();
-                  struct.nosuch.add(_elem586);
+                  _elem576 = iprot.readI64();
+                  struct.nosuch.add(_elem576);
                 }
                 iprot.readSetEnd();
               }
@@ -503,9 +503,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(ABORTED_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.aborted.size()));
-          for (long _iter588 : struct.aborted)
+          for (long _iter578 : struct.aborted)
           {
-            oprot.writeI64(_iter588);
+            oprot.writeI64(_iter578);
           }
           oprot.writeSetEnd();
         }
@@ -515,9 +515,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(NOSUCH_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.nosuch.size()));
-          for (long _iter589 : struct.nosuch)
+          for (long _iter579 : struct.nosuch)
           {
-            oprot.writeI64(_iter589);
+            oprot.writeI64(_iter579);
           }
           oprot.writeSetEnd();
         }
@@ -542,16 +542,16 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.aborted.size());
-        for (long _iter590 : struct.aborted)
+        for (long _iter580 : struct.aborted)
         {
-          oprot.writeI64(_iter590);
+          oprot.writeI64(_iter580);
         }
       }
       {
         oprot.writeI32(struct.nosuch.size());
-        for (long _iter591 : struct.nosuch)
+        for (long _iter581 : struct.nosuch)
         {
-          oprot.writeI64(_iter591);
+          oprot.writeI64(_iter581);
         }
       }
     }
@@ -560,24 +560,24 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, HeartbeatTxnRangeResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TSet _set592 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.aborted = new HashSet<Long>(2*_set592.size);
-        long _elem593;
-        for (int _i594 = 0; _i594 < _set592.size; ++_i594)
+        org.apache.thrift.protocol.TSet _set582 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.aborted = new HashSet<Long>(2*_set582.size);
+        long _elem583;
+        for (int _i584 = 0; _i584 < _set582.size; ++_i584)
         {
-          _elem593 = iprot.readI64();
-          struct.aborted.add(_elem593);
+          _elem583 = iprot.readI64();
+          struct.aborted.add(_elem583);
         }
       }
       struct.setAbortedIsSet(true);
       {
-        org.apache.thrift.protocol.TSet _set595 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.nosuch = new HashSet<Long>(2*_set595.size);
-        long _elem596;
-        for (int _i597 = 0; _i597 < _set595.size; ++_i597)
+        org.apache.thrift.protocol.TSet _set585 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.nosuch = new HashSet<Long>(2*_set585.size);
+        long _elem586;
+        for (int _i587 = 0; _i587 < _set585.size; ++_i587)
         {
-          _elem596 = iprot.readI64();
-          struct.nosuch.add(_elem596);
+          _elem586 = iprot.readI64();
+          struct.nosuch.add(_elem586);
         }
       }
       struct.setNosuchIsSet(true);


[16/16] hive git commit: HIVE-18546: Remove unnecessary code introduced in HIVE-14498 (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
HIVE-18546: Remove unnecessary code introduced in HIVE-14498 (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/0a328f03
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/0a328f03
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/0a328f03

Branch: refs/heads/master
Commit: 0a328f030b9e0ee4d87dd7242106a0c4ed820cea
Parents: 4a33ec8
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Thu Jan 25 09:47:13 2018 -0800
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Sun Feb 4 14:06:39 2018 -0800

----------------------------------------------------------------------
 .../hive/ql/parse/TestReplicationScenarios.java |   17 +-
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |   20 +
 .../upgrade/derby/048-HIVE-14498.derby.sql      |   28 +-
 .../upgrade/derby/hive-schema-3.0.0.derby.sql   |   16 +-
 .../derby/hive-txn-schema-3.0.0.derby.sql       |    4 +-
 .../upgrade/hive/hive-schema-3.0.0.hive.sql     |   51 +-
 .../upgrade/mssql/033-HIVE-14498.mssql.sql      |   32 +-
 .../upgrade/mssql/hive-schema-3.0.0.mssql.sql   |   30 +-
 .../upgrade/mysql/048-HIVE-14498.mysql.sql      |   33 +-
 .../upgrade/mysql/hive-schema-3.0.0.mysql.sql   |   40 +-
 .../mysql/hive-txn-schema-3.0.0.mysql.sql       |    4 +-
 .../upgrade/oracle/048-HIVE-14498.oracle.sql    |   36 +-
 .../upgrade/oracle/hive-schema-3.0.0.oracle.sql |   31 +-
 .../oracle/hive-txn-schema-3.0.0.oracle.sql     |    4 +-
 .../postgres/047-HIVE-14498.postgres.sql        |   37 +-
 .../postgres/hive-schema-3.0.0.postgres.sql     |   36 +-
 .../postgres/hive-txn-schema-3.0.0.postgres.sql |    4 +-
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |   52 +-
 .../apache/hadoop/hive/ql/metadata/Table.java   |    5 +-
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |    2 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |    4 +-
 .../hadoop/hive/ql/plan/CreateViewDesc.java     |    7 +-
 .../hadoop/hive/ql/plan/ImportTableDesc.java    |    3 +
 .../hadoop/hive/ql/metadata/TestHive.java       |    4 -
 .../results/clientpositive/llap/sysdb.q.out     |  110 +-
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 3656 +++++--------
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |  280 -
 .../ThriftHiveMetastore_server.skeleton.cpp     |   10 -
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp | 4841 +++++++++---------
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |   68 +-
 .../hive/metastore/api/AbortTxnsRequest.java    |   32 +-
 .../metastore/api/AddDynamicPartitions.java     |   32 +-
 .../metastore/api/AddForeignKeyRequest.java     |   36 +-
 .../api/AddNotNullConstraintRequest.java        |   36 +-
 .../metastore/api/AddPartitionsRequest.java     |   36 +-
 .../hive/metastore/api/AddPartitionsResult.java |   36 +-
 .../metastore/api/AddPrimaryKeyRequest.java     |   36 +-
 .../api/AddUniqueConstraintRequest.java         |   36 +-
 .../hadoop/hive/metastore/api/AggrStats.java    |   36 +-
 .../hadoop/hive/metastore/api/BasicTxnInfo.java |  167 +-
 .../metastore/api/ClearFileMetadataRequest.java |   32 +-
 .../hive/metastore/api/ClientCapabilities.java  |   32 +-
 .../hive/metastore/api/ColumnStatistics.java    |   36 +-
 .../hive/metastore/api/CompactionRequest.java   |   44 +-
 .../hive/metastore/api/CreationMetadata.java    |  750 +++
 .../metastore/api/DropPartitionsResult.java     |   36 +-
 .../hive/metastore/api/EnvironmentContext.java  |   44 +-
 .../hive/metastore/api/FireEventRequest.java    |   32 +-
 .../hive/metastore/api/ForeignKeysResponse.java |   36 +-
 .../hadoop/hive/metastore/api/Function.java     |   36 +-
 .../metastore/api/GetAllFunctionsResponse.java  |   36 +-
 .../api/GetFileMetadataByExprRequest.java       |   32 +-
 .../api/GetFileMetadataByExprResult.java        |   48 +-
 .../metastore/api/GetFileMetadataRequest.java   |   32 +-
 .../metastore/api/GetFileMetadataResult.java    |   44 +-
 .../metastore/api/GetOpenTxnsInfoResponse.java  |   36 +-
 .../hive/metastore/api/GetOpenTxnsResponse.java |   32 +-
 .../hive/metastore/api/GetTablesRequest.java    |   32 +-
 .../hive/metastore/api/GetTablesResult.java     |   36 +-
 .../api/HeartbeatTxnRangeResponse.java          |   64 +-
 .../apache/hadoop/hive/metastore/api/Index.java |   44 +-
 .../metastore/api/InsertEventRequestData.java   |   64 +-
 .../hadoop/hive/metastore/api/LockRequest.java  |   36 +-
 .../hive/metastore/api/Materialization.java     |   32 +-
 .../api/NotNullConstraintsResponse.java         |   36 +-
 .../api/NotificationEventResponse.java          |   36 +-
 .../hive/metastore/api/OpenTxnsResponse.java    |   32 +-
 .../hadoop/hive/metastore/api/Partition.java    |   76 +-
 .../api/PartitionListComposingSpec.java         |   36 +-
 .../api/PartitionSpecWithSharedSD.java          |   36 +-
 .../metastore/api/PartitionValuesRequest.java   |   72 +-
 .../metastore/api/PartitionValuesResponse.java  |   36 +-
 .../hive/metastore/api/PartitionValuesRow.java  |   32 +-
 .../hive/metastore/api/PartitionWithoutSD.java  |   76 +-
 .../metastore/api/PartitionsByExprResult.java   |   36 +-
 .../metastore/api/PartitionsStatsRequest.java   |   64 +-
 .../metastore/api/PartitionsStatsResult.java    |   76 +-
 .../hive/metastore/api/PrimaryKeysResponse.java |   36 +-
 .../metastore/api/PutFileMetadataRequest.java   |   64 +-
 .../hive/metastore/api/RequestPartsSpec.java    |   68 +-
 .../hadoop/hive/metastore/api/Schema.java       |   80 +-
 .../api/SetPartitionsStatsRequest.java          |   36 +-
 .../hive/metastore/api/ShowCompactResponse.java |   36 +-
 .../hive/metastore/api/ShowLocksResponse.java   |   36 +-
 .../apache/hadoop/hive/metastore/api/Table.java |  141 +-
 .../hive/metastore/api/TableStatsRequest.java   |   32 +-
 .../hive/metastore/api/TableStatsResult.java    |   36 +-
 .../hive/metastore/api/ThriftHiveMetastore.java | 4731 +++++------------
 .../hadoop/hive/metastore/api/TxnsSnapshot.java |  537 --
 .../api/UniqueConstraintsResponse.java          |   36 +-
 .../hive/metastore/api/WMFullResourcePlan.java  |  144 +-
 .../api/WMGetAllResourcePlanResponse.java       |   36 +-
 .../WMGetTriggersForResourePlanResponse.java    |   36 +-
 .../api/WMValidateResourcePlanResponse.java     |   64 +-
 .../gen-php/metastore/ThriftHiveMetastore.php   | 1979 +++----
 .../src/gen/thrift/gen-php/metastore/Types.php  | 1728 +++----
 .../hive_metastore/ThriftHiveMetastore-remote   |   14 -
 .../hive_metastore/ThriftHiveMetastore.py       | 1390 ++---
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  | 1159 ++---
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |   33 +-
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |  116 -
 .../hadoop/hive/metastore/HiveMetaStore.java    |   13 -
 .../hive/metastore/HiveMetaStoreClient.java     |   19 -
 .../hadoop/hive/metastore/IMetaStoreClient.java |   18 -
 .../MaterializationsInvalidationCache.java      |  178 +-
 .../hadoop/hive/metastore/ObjectStore.java      |  109 +-
 .../hive/metastore/model/MCreationMetadata.java |   77 +
 .../hadoop/hive/metastore/model/MTable.java     |    8 +-
 .../hadoop/hive/metastore/txn/TxnHandler.java   |   90 +-
 .../hadoop/hive/metastore/txn/TxnStore.java     |   26 +-
 .../src/main/resources/package.jdo              |   32 +-
 .../src/main/thrift/hive_metastore.thrift       |   23 +-
 .../hive/metastore/cache/TestCachedStore.java   |    1 -
 .../hive/metastore/client/TestGetTableMeta.java |    6 +-
 .../TestTablesCreateDropAlterTruncate.java      |    2 +-
 115 files changed, 10434 insertions(+), 14974 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
index 39d077a..d1d2d1f 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
@@ -2239,8 +2239,11 @@ public class TestReplicationScenarios {
     run("LOAD DATA LOCAL INPATH '" + ptn_locn_2 + "' OVERWRITE INTO TABLE " + dbName + ".ptned PARTITION(b=2)", driver);
     verifySetup("SELECT a from " + dbName + ".ptned WHERE b=2", ptn_data_2, driver);
 
-    run("CREATE MATERIALIZED VIEW " + dbName + ".mat_view AS SELECT a FROM " + dbName + ".ptned where b=1", driver);
-    verifySetup("SELECT a from " + dbName + ".mat_view", ptn_data_1, driver);
+    // TODO: Enable back when HIVE-18387 goes in, as it fixes the issue.
+    // The problem is that alter for stats is removing the metadata information.
+    // HIVE-18387 rewrites that logic and will fix the issue.
+    //run("CREATE MATERIALIZED VIEW " + dbName + ".mat_view AS SELECT a FROM " + dbName + ".ptned where b=1", driver);
+    //verifySetup("SELECT a from " + dbName + ".mat_view", ptn_data_1, driver);
 
     advanceDumpDir();
     run("REPL DUMP " + dbName, driver);
@@ -2251,7 +2254,7 @@ public class TestReplicationScenarios {
 
     // view is referring to old database, so no data
     verifyRun("SELECT * from " + dbName + "_dupe.virtual_view", empty, driverMirror);
-    verifyRun("SELECT a from " + dbName + "_dupe.mat_view", ptn_data_1, driverMirror);
+    //verifyRun("SELECT a from " + dbName + "_dupe.mat_view", ptn_data_1, driverMirror);
 
     run("CREATE VIEW " + dbName + ".virtual_view2 AS SELECT a FROM " + dbName + ".ptned where b=2", driver);
     verifySetup("SELECT a from " + dbName + ".virtual_view2", ptn_data_2, driver);
@@ -2259,8 +2262,8 @@ public class TestReplicationScenarios {
     // Create a view with name already exist. Just to verify if failure flow clears the added create_table event.
     run("CREATE VIEW " + dbName + ".virtual_view2 AS SELECT a FROM " + dbName + ".ptned where b=2", driver);
 
-    run("CREATE MATERIALIZED VIEW " + dbName + ".mat_view2 AS SELECT * FROM " + dbName + ".unptned", driver);
-    verifySetup("SELECT * from " + dbName + ".mat_view2", unptn_data, driver);
+    //run("CREATE MATERIALIZED VIEW " + dbName + ".mat_view2 AS SELECT * FROM " + dbName + ".unptned", driver);
+    //verifySetup("SELECT * from " + dbName + ".mat_view2", unptn_data, driver);
 
     // Perform REPL-DUMP/LOAD
     advanceDumpDir();
@@ -2277,10 +2280,10 @@ public class TestReplicationScenarios {
     verifyRun("SELECT a from " + dbName + "_dupe.ptned where b=1", ptn_data_1, driverMirror);
     // view is referring to old database, so no data
     verifyRun("SELECT * from " + dbName + "_dupe.virtual_view", empty, driverMirror);
-    verifyRun("SELECT a from " + dbName + "_dupe.mat_view", ptn_data_1, driverMirror);
+    //verifyRun("SELECT a from " + dbName + "_dupe.mat_view", ptn_data_1, driverMirror);
     // view is referring to old database, so no data
     verifyRun("SELECT * from " + dbName + "_dupe.virtual_view2", empty, driverMirror);
-    verifyRun("SELECT * from " + dbName + "_dupe.mat_view2", unptn_data, driverMirror);
+    //verifyRun("SELECT * from " + dbName + "_dupe.mat_view2", unptn_data, driverMirror);
 
     // Test "alter table" with rename
     run("ALTER VIEW " + dbName + ".virtual_view RENAME TO " + dbName + ".virtual_view_rename", driver);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index 4432aca..2bc33bd 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -926,6 +926,26 @@ public class QTestUtil {
     conf.set("hive.metastore.filter.hook",
         "org.apache.hadoop.hive.metastore.DefaultMetaStoreFilterHookImpl");
     db = Hive.get(conf);
+
+    // First delete any MVs to avoid race conditions
+    for (String dbName : db.getAllDatabases()) {
+      SessionState.get().setCurrentDatabase(dbName);
+      for (String tblName : db.getAllTables()) {
+        Table tblObj = null;
+        try {
+          tblObj = db.getTable(tblName);
+        } catch (InvalidTableException e) {
+          LOG.warn("Trying to drop table " + e.getTableName() + ". But it does not exist.");
+          continue;
+        }
+        // only remove MVs first
+        if (!tblObj.isMaterializedView()) {
+          continue;
+        }
+        db.dropTable(dbName, tblName, true, true, fsType == FsType.encrypted_hdfs);
+      }
+    }
+
     // Delete any tables other than the source tables
     // and any databases other than the default database.
     for (String dbName : db.getAllDatabases()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/metastore/scripts/upgrade/derby/048-HIVE-14498.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/048-HIVE-14498.derby.sql b/metastore/scripts/upgrade/derby/048-HIVE-14498.derby.sql
index 4ffd054..e28d002 100644
--- a/metastore/scripts/upgrade/derby/048-HIVE-14498.derby.sql
+++ b/metastore/scripts/upgrade/derby/048-HIVE-14498.derby.sql
@@ -1,20 +1,14 @@
 -- create mv_creation_metadata table
-CREATE TABLE "APP"."MV_CREATION_METADATA" ("TBL_ID" BIGINT NOT NULL, "TBL_NAME" VARCHAR(256) NOT NULL, "LAST_TRANSACTION_INFO" LONG VARCHAR NOT NULL);
-ALTER TABLE "APP"."MV_CREATION_METADATA" ADD CONSTRAINT "MV_CREATION_METADATA_FK" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION;
+CREATE TABLE "APP"."MV_CREATION_METADATA" ("MV_CREATION_METADATA_ID" BIGINT NOT NULL, "DB_NAME" VARCHAR(128) NOT NULL, "TBL_NAME" VARCHAR(256) NOT NULL, "TXN_LIST" CLOB);
+CREATE TABLE "APP"."MV_TABLES_USED" ("MV_CREATION_METADATA_ID" BIGINT NOT NULL, "TBL_ID" BIGINT NOT NULL);
+ALTER TABLE "APP"."MV_CREATION_METADATA" ADD CONSTRAINT "MV_CREATION_METADATA_PK" PRIMARY KEY ("MV_CREATION_METADATA_ID");
+CREATE UNIQUE INDEX "APP"."MV_UNIQUE_TABLE" ON "APP"."MV_CREATION_METADATA" ("TBL_NAME", "DB_NAME");
+ALTER TABLE "APP"."MV_TABLES_USED" ADD CONSTRAINT "MV_TABLES_USED_FK1" FOREIGN KEY ("MV_CREATION_METADATA_ID") REFERENCES "APP"."MV_CREATION_METADATA" ("MV_CREATION_METADATA_ID") ON DELETE NO ACTION ON UPDATE NO ACTION;
+ALTER TABLE "APP"."MV_TABLES_USED" ADD CONSTRAINT "MV_TABLES_USED_FK2" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION;
 
 -- modify completed_txn_components table
-CREATE TABLE "COMPLETED_TXN_COMPONENTS_NEW" (
-  "CTC_TXNID" bigint,
-  "CTC_DATABASE" varchar(128) NOT NULL,
-  "CTC_TABLE" varchar(256),
-  "CTC_PARTITION" varchar(767),
-  "CTC_ID" bigint GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1) NOT NULL,
-  "CTC_TIMESTAMP" timestamp DEFAULT CURRENT_TIMESTAMP NOT NULL
-);
-CREATE INDEX "APP"."COMPLETED_TXN_COMPONENTS_IDX" ON "APP"."COMPLETED_TXN_COMPONENTS_NEW" ("CTC_ID");
-CREATE INDEX "APP"."COMPLETED_TXN_COMPONENTS_IDX2" ON "APP"."COMPLETED_TXN_COMPONENTS_NEW" ("CTC_DATABASE", "CTC_TABLE", "CTC_PARTITION");
-INSERT INTO "COMPLETED_TXN_COMPONENTS_NEW" ("CTC_TXNID", "CTC_DATABASE", "CTC_TABLE", "CTC_PARTITION")
-SELECT "CTC_TXNID", "CTC_DATABASE", "CTC_TABLE", "CTC_PARTITION" FROM "COMPLETED_TXN_COMPONENTS";
-RENAME TABLE "COMPLETED_TXN_COMPONENTS" TO "COMPLETED_TXN_COMPONENTS_BACKUP";
-RENAME TABLE "COMPLETED_TXN_COMPONENTS_NEW" TO "COMPLETED_TXN_COMPONENTS";
-DROP TABLE "APP"."COMPLETED_TXN_COMPONENTS_BACKUP";
+ALTER TABLE "APP"."COMPLETED_TXN_COMPONENTS" ADD "CTC_TIMESTAMP" timestamp;
+UPDATE "APP"."TBLS" SET "IS_REWRITE_ENABLED" = CURRENT_TIMESTAMP;
+ALTER TABLE "APP"."COMPLETED_TXN_COMPONENTS" ALTER COLUMN "CTC_TIMESTAMP" SET DEFAULT CURRENT_TIMESTAMP;
+ALTER TABLE "APP"."COMPLETED_TXN_COMPONENTS" ALTER COLUMN "CTC_TIMESTAMP" NOT NULL;
+CREATE INDEX "APP"."COMPLETED_TXN_COMPONENTS_IDX" ON "APP"."COMPLETED_TXN_COMPONENTS" ("CTC_DATABASE", "CTC_TABLE", "CTC_PARTITION");

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/metastore/scripts/upgrade/derby/hive-schema-3.0.0.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/hive-schema-3.0.0.derby.sql b/metastore/scripts/upgrade/derby/hive-schema-3.0.0.derby.sql
index 6a59b0d..a8f227b 100644
--- a/metastore/scripts/upgrade/derby/hive-schema-3.0.0.derby.sql
+++ b/metastore/scripts/upgrade/derby/hive-schema-3.0.0.derby.sql
@@ -62,6 +62,10 @@ CREATE TABLE "APP"."ROLES" ("ROLE_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT
 
 CREATE TABLE "APP"."TBLS" ("TBL_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "DB_ID" BIGINT, "LAST_ACCESS_TIME" INTEGER NOT NULL, "OWNER" VARCHAR(767), "RETENTION" INTEGER NOT NULL, "SD_ID" BIGINT, "TBL_NAME" VARCHAR(256), "TBL_TYPE" VARCHAR(128), "VIEW_EXPANDED_TEXT" LONG VARCHAR, "VIEW_ORIGINAL_TEXT" LONG VARCHAR, "IS_REWRITE_ENABLED" CHAR(1) NOT NULL DEFAULT 'N');
 
+CREATE TABLE "APP"."MV_CREATION_METADATA" ("MV_CREATION_METADATA_ID" BIGINT NOT NULL, "DB_NAME" VARCHAR(128) NOT NULL, "TBL_NAME" VARCHAR(256) NOT NULL, "TXN_LIST" CLOB);
+
+CREATE TABLE "APP"."MV_TABLES_USED" ("MV_CREATION_METADATA_ID" BIGINT NOT NULL, "TBL_ID" BIGINT NOT NULL);
+
 CREATE TABLE "APP"."PARTITION_KEYS" ("TBL_ID" BIGINT NOT NULL, "PKEY_COMMENT" VARCHAR(4000), "PKEY_NAME" VARCHAR(128) NOT NULL, "PKEY_TYPE" VARCHAR(767) NOT NULL, "INTEGER_IDX" INTEGER NOT NULL);
 
 CREATE TABLE "APP"."PART_COL_PRIVS" ("PART_COLUMN_GRANT_ID" BIGINT NOT NULL, "COLUMN_NAME" VARCHAR(767), "CREATE_TIME" INTEGER NOT NULL, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PART_ID" BIGINT, "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "PART_COL_PRIV" VARCHAR(128));
@@ -120,8 +124,6 @@ CREATE TABLE "APP"."WM_POOL_TO_TRIGGER"  (POOL_ID BIGINT NOT NULL, TRIGGER_ID BI
 
 CREATE TABLE "APP"."WM_MAPPING" (MAPPING_ID BIGINT NOT NULL, RP_ID BIGINT NOT NULL, ENTITY_TYPE VARCHAR(128) NOT NULL, ENTITY_NAME VARCHAR(128) NOT NULL, POOL_ID BIGINT, ORDERING INTEGER);
 
-CREATE TABLE "APP"."MV_CREATION_METADATA" ("TBL_ID" BIGINT NOT NULL, "TBL_NAME" VARCHAR(256) NOT NULL, "LAST_TRANSACTION_INFO" LONG VARCHAR NOT NULL);
-
 -- ----------------------------------------------
 -- DML Statements
 -- ----------------------------------------------
@@ -178,6 +180,8 @@ CREATE UNIQUE INDEX "APP"."UNIQUE_WM_TRIGGER" ON "APP"."WM_TRIGGER" ("RP_ID", "N
 
 CREATE UNIQUE INDEX "APP"."UNIQUE_WM_MAPPING" ON "APP"."WM_MAPPING" ("RP_ID", "ENTITY_TYPE", "ENTITY_NAME");
 
+CREATE UNIQUE INDEX "APP"."MV_UNIQUE_TABLE" ON "APP"."MV_CREATION_METADATA" ("TBL_NAME", "DB_NAME");
+
 -- ----------------------------------------------
 -- DDL Statements for keys
 -- ----------------------------------------------
@@ -219,6 +223,8 @@ ALTER TABLE "APP"."NUCLEUS_TABLES" ADD CONSTRAINT "NUCLEUS_TABLES_PK" PRIMARY KE
 
 ALTER TABLE "APP"."TBLS" ADD CONSTRAINT "TBLS_PK" PRIMARY KEY ("TBL_ID");
 
+ALTER TABLE "APP"."MV_CREATION_METADATA" ADD CONSTRAINT "MV_CREATION_METADATA_PK" PRIMARY KEY ("MV_CREATION_METADATA_ID");
+
 ALTER TABLE "APP"."SD_PARAMS" ADD CONSTRAINT "SD_PARAMS_PK" PRIMARY KEY ("SD_ID", "PARAM_KEY");
 
 ALTER TABLE "APP"."DATABASE_PARAMS" ADD CONSTRAINT "DATABASE_PARAMS_PK" PRIMARY KEY ("DB_ID", "PARAM_KEY");
@@ -306,6 +312,10 @@ ALTER TABLE "APP"."TBLS" ADD CONSTRAINT "TBLS_FK2" FOREIGN KEY ("SD_ID") REFEREN
 
 ALTER TABLE "APP"."TBLS" ADD CONSTRAINT "TBLS_FK1" FOREIGN KEY ("DB_ID") REFERENCES "APP"."DBS" ("DB_ID") ON DELETE NO ACTION ON UPDATE NO ACTION;
 
+ALTER TABLE "APP"."MV_TABLES_USED" ADD CONSTRAINT "MV_TABLES_USED_FK1" FOREIGN KEY ("MV_CREATION_METADATA_ID") REFERENCES "APP"."MV_CREATION_METADATA" ("MV_CREATION_METADATA_ID") ON DELETE NO ACTION ON UPDATE NO ACTION;
+
+ALTER TABLE "APP"."MV_TABLES_USED" ADD CONSTRAINT "MV_TABLES_USED_FK2" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION;
+
 ALTER TABLE "APP"."SD_PARAMS" ADD CONSTRAINT "SD_PARAMS_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION;
 
 ALTER TABLE "APP"."DATABASE_PARAMS" ADD CONSTRAINT "DATABASE_PARAMS_FK1" FOREIGN KEY ("DB_ID") REFERENCES "APP"."DBS" ("DB_ID") ON DELETE NO ACTION ON UPDATE NO ACTION;
@@ -374,8 +384,6 @@ ALTER TABLE "APP"."WM_MAPPING" ADD CONSTRAINT "WM_MAPPING_FK1" FOREIGN KEY ("RP_
 
 ALTER TABLE "APP"."WM_MAPPING" ADD CONSTRAINT "WM_MAPPING_FK2" FOREIGN KEY ("POOL_ID") REFERENCES "APP"."WM_POOL" ("POOL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION;
 
-ALTER TABLE "APP"."MV_CREATION_METADATA" ADD CONSTRAINT "MV_CREATION_METADATA_FK" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION;
-
 -- ----------------------------------------------
 -- DDL Statements for checks
 -- ----------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/metastore/scripts/upgrade/derby/hive-txn-schema-3.0.0.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/hive-txn-schema-3.0.0.derby.sql b/metastore/scripts/upgrade/derby/hive-txn-schema-3.0.0.derby.sql
index d72b06c..85d593f 100644
--- a/metastore/scripts/upgrade/derby/hive-txn-schema-3.0.0.derby.sql
+++ b/metastore/scripts/upgrade/derby/hive-txn-schema-3.0.0.derby.sql
@@ -43,12 +43,10 @@ CREATE TABLE COMPLETED_TXN_COMPONENTS (
   CTC_DATABASE varchar(128) NOT NULL,
   CTC_TABLE varchar(256),
   CTC_PARTITION varchar(767),
-  CTC_ID bigint GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1) NOT NULL,
   CTC_TIMESTAMP timestamp DEFAULT CURRENT_TIMESTAMP NOT NULL
 );
 
-CREATE INDEX COMPLETED_TXN_COMPONENTS_IDX ON COMPLETED_TXN_COMPONENTS (CTC_ID);
-CREATE INDEX COMPLETED_TXN_COMPONENTS_IDX2 ON COMPLETED_TXN_COMPONENTS (CTC_DATABASE, CTC_TABLE, CTC_PARTITION);
+CREATE INDEX COMPLETED_TXN_COMPONENTS_IDX ON COMPLETED_TXN_COMPONENTS (CTC_DATABASE, CTC_TABLE, CTC_PARTITION);
 
 CREATE TABLE NEXT_TXN_ID (
   NTXN_NEXT bigint NOT NULL

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/metastore/scripts/upgrade/hive/hive-schema-3.0.0.hive.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/hive/hive-schema-3.0.0.hive.sql b/metastore/scripts/upgrade/hive/hive-schema-3.0.0.hive.sql
index eb4f012..d6e0c5c 100644
--- a/metastore/scripts/upgrade/hive/hive-schema-3.0.0.hive.sql
+++ b/metastore/scripts/upgrade/hive/hive-schema-3.0.0.hive.sql
@@ -656,6 +656,39 @@ TBLPROPERTIES (
 FROM TBLS"
 );
 
+CREATE TABLE IF NOT EXISTS `MV_CREATION_METADATA` (
+  `MV_CREATION_METADATA_ID` bigint,
+  `DB_NAME` string,
+  `TBL_NAME` string,
+  `TXN_LIST` string,
+  CONSTRAINT `SYS_PK_MV_CREATION_METADATA` PRIMARY KEY (`MV_CREATION_METADATA_ID`) DISABLE
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+"hive.sql.database.type" = "METASTORE",
+"hive.sql.query" =
+"SELECT
+  \"MV_CREATION_METADATA_ID\",
+  \"DB_NAME\",
+  \"TBL_NAME\",
+  \"TXN_LIST\"
+FROM MV_CREATION_METADATA"
+);
+
+CREATE TABLE IF NOT EXISTS `MV_TABLES_USED` (
+  `MV_CREATION_METADATA_ID` bigint,
+  `TBL_ID` bigint
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+"hive.sql.database.type" = "METASTORE",
+"hive.sql.query" =
+"SELECT
+  \"MV_CREATION_METADATA_ID\",
+  \"TBL_ID\"
+FROM MV_TABLES_USED"
+);
+
 CREATE TABLE IF NOT EXISTS `TBL_COL_PRIVS` (
   `TBL_COLUMN_GRANT_ID` bigint,
   `COLUMN_NAME` string,
@@ -1065,24 +1098,6 @@ LEFT OUTER JOIN WM_POOL ON WM_POOL.POOL_ID = WM_MAPPING.POOL_ID
 "
 );
 
-CREATE TABLE IF NOT EXISTS `MV_CREATION_METADATA` (
-  TBL_ID bigint,
-  TBL_NAME string,
-  LAST_TRANSACTION_INFO string
-)
-STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
-TBLPROPERTIES (
-"hive.sql.database.type" = "METASTORE",
-"hive.sql.query" =
-"SELECT
-  \"TBL_ID\",
-  \"TBL_NAME\",
-  \"LAST_TRANSACTION_INFO\"
-FROM
-  \"MV_CREATION_METADATA\""
-);
-
-
 DROP DATABASE IF EXISTS INFORMATION_SCHEMA;
 CREATE DATABASE INFORMATION_SCHEMA;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/metastore/scripts/upgrade/mssql/033-HIVE-14498.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/033-HIVE-14498.mssql.sql b/metastore/scripts/upgrade/mssql/033-HIVE-14498.mssql.sql
index 3a47600..cb41b99 100644
--- a/metastore/scripts/upgrade/mssql/033-HIVE-14498.mssql.sql
+++ b/metastore/scripts/upgrade/mssql/033-HIVE-14498.mssql.sql
@@ -1,23 +1,19 @@
 CREATE TABLE MV_CREATION_METADATA
 (
-    TBL_ID bigint NOT NULL,
+    MV_CREATION_METADATA_ID bigint NOT NULL,
+    DB_NAME nvarchar(128) NOT NULL,
     TBL_NAME nvarchar(256) NOT NULL,
-    LAST_TRANSACTION_INFO text NOT NULL
+    TXN_LIST text NULL
 );
-ALTER TABLE MV_CREATION_METADATA ADD CONSTRAINT MV_CREATION_METADATA_FK FOREIGN KEY (TBL_ID) REFERENCES TBLS (TBL_ID);
-
-CREATE TABLE COMPLETED_TXN_COMPONENTS_NEW(
-	CTC_TXNID bigint NULL,
-	CTC_DATABASE varchar(128) NOT NULL,
-	CTC_TABLE varchar(256) NULL,
-	CTC_PARTITION varchar(767) NULL,
-	CTC_ID bigint GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1) NOT NULL,
-	CTC_TIMESTAMP timestamp DEFAULT CURRENT_TIMESTAMP NOT NULL
+CREATE TABLE MV_TABLES_USED
+(
+    MV_CREATION_METADATA_ID bigint NOT NULL,
+    TBL_ID bigint NOT NULL
 );
-CREATE INDEX COMPLETED_TXN_COMPONENTS_IDX ON COMPLETED_TXN_COMPONENTS_NEW (CTC_ID);
-CREATE INDEX COMPLETED_TXN_COMPONENTS_IDX2 ON COMPLETED_TXN_COMPONENTS_NEW (CTC_DATABASE, CTC_TABLE, CTC_PARTITION);
-INSERT INTO COMPLETED_TXN_COMPONENTS_NEW (CTC_TXNID, CTC_DATABASE, CTC_TABLE, CTC_PARTITION)
-SELECT CTC_TXNID, CTC_DATABASE, CTC_TABLE, CTC_PARTITION FROM COMPLETED_TXN_COMPONENTS;
-exec sp_rename 'COMPLETED_TXN_COMPONENTS', 'COMPLETED_TXN_COMPONENTS_BACKUP';
-exec sp_rename 'COMPLETED_TXN_COMPONENTS_NEW', 'COMPLETED_TXN_COMPONENTS';
-DROP TABLE COMPLETED_TXN_COMPONENTS_BACKUP;
+ALTER TABLE MV_CREATION_METADATA ADD CONSTRAINT MV_CREATION_METADATA_PK PRIMARY KEY (MV_CREATION_METADATA_ID);
+CREATE INDEX MV_UNIQUE_TABLE ON MV_CREATION_METADATA (TBL_NAME,DB_NAME);
+ALTER TABLE MV_TABLES_USED ADD FOREIGN KEY(MV_CREATION_METADATA_ID) REFERENCES MV_CREATION_METADATA (MV_CREATION_METADATA_ID);
+ALTER TABLE MV_TABLES_USED ADD FOREIGN KEY(TBL_ID) REFERENCES TBLS (TBL_ID);
+
+ALTER TABLE COMPLETED_TXN_COMPONENTS ADD CTC_TIMESTAMP timestamp NOT NULL DEFAULT(CURRENT_TIMESTAMP);
+CREATE INDEX COMPLETED_TXN_COMPONENTS_IDX ON COMPLETED_TXN_COMPONENTS (CTC_DATABASE, CTC_TABLE, CTC_PARTITION);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/metastore/scripts/upgrade/mssql/hive-schema-3.0.0.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/hive-schema-3.0.0.mssql.sql b/metastore/scripts/upgrade/mssql/hive-schema-3.0.0.mssql.sql
index c45bb3e..448086e 100644
--- a/metastore/scripts/upgrade/mssql/hive-schema-3.0.0.mssql.sql
+++ b/metastore/scripts/upgrade/mssql/hive-schema-3.0.0.mssql.sql
@@ -366,6 +366,27 @@ CREATE TABLE TBLS
 
 ALTER TABLE TBLS ADD CONSTRAINT TBLS_PK PRIMARY KEY (TBL_ID);
 
+-- Table MV_CREATION_METADATA for classes [org.apache.hadoop.hive.metastore.model.MCreationMetadata]
+CREATE TABLE MV_CREATION_METADATA
+(
+    MV_CREATION_METADATA_ID bigint NOT NULL,
+    DB_NAME nvarchar(128) NOT NULL,
+    TBL_NAME nvarchar(256) NOT NULL,
+    TXN_LIST text NULL
+);
+
+ALTER TABLE MV_CREATION_METADATA ADD CONSTRAINT MV_CREATION_METADATA_PK PRIMARY KEY (MV_CREATION_METADATA_ID);
+CREATE INDEX MV_UNIQUE_TABLE ON MV_CREATION_METADATA (TBL_NAME,DB_NAME);
+
+CREATE TABLE MV_TABLES_USED
+(
+    MV_CREATION_METADATA_ID bigint NOT NULL,
+    TBL_ID bigint NOT NULL
+);
+
+ALTER TABLE MV_TABLES_USED WITH CHECK ADD FOREIGN KEY(MV_CREATION_METADATA_ID) REFERENCES MV_CREATION_METADATA (MV_CREATION_METADATA_ID);
+ALTER TABLE MV_TABLES_USED WITH CHECK ADD FOREIGN KEY(TBL_ID) REFERENCES TBLS (TBL_ID);
+
 -- Table SDS for classes [org.apache.hadoop.hive.metastore.model.MStorageDescriptor]
 CREATE TABLE SDS
 (
@@ -652,15 +673,6 @@ CREATE TABLE WM_MAPPING
 
 ALTER TABLE WM_MAPPING ADD CONSTRAINT WM_MAPPING_PK PRIMARY KEY (MAPPING_ID);
 
-CREATE TABLE MV_CREATION_METADATA
-(
-    TBL_ID bigint NOT NULL,
-    TBL_NAME nvarchar(256) NOT NULL,
-    LAST_TRANSACTION_INFO text NOT NULL
-);
-
-ALTER TABLE MV_CREATION_METADATA ADD CONSTRAINT MV_CREATION_METADATA_FK FOREIGN KEY (TBL_ID) REFERENCES TBLS (TBL_ID) ;
-
 -- Constraints for table MASTER_KEYS for class(es) [org.apache.hadoop.hive.metastore.model.MMasterKey]
 
 -- Constraints for table IDXS for class(es) [org.apache.hadoop.hive.metastore.model.MIndex]

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/metastore/scripts/upgrade/mysql/048-HIVE-14498.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/048-HIVE-14498.mysql.sql b/metastore/scripts/upgrade/mysql/048-HIVE-14498.mysql.sql
index 986eaf5..1a791a6 100644
--- a/metastore/scripts/upgrade/mysql/048-HIVE-14498.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/048-HIVE-14498.mysql.sql
@@ -1,22 +1,19 @@
 CREATE TABLE IF NOT EXISTS `MV_CREATION_METADATA` (
+  `MV_CREATION_METADATA_ID` bigint(20) NOT NULL,
+  `DB_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `TBL_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `TXN_LIST` TEXT DEFAULT NULL,
+  PRIMARY KEY (`MV_CREATION_METADATA_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+CREATE INDEX MV_UNIQUE_TABLE ON MV_CREATION_METADATA (TBL_NAME, DB_NAME) USING BTREE;
+CREATE TABLE IF NOT EXISTS `MV_TABLES_USED` (
+  `MV_CREATION_METADATA_ID` bigint(20) NOT NULL,
   `TBL_ID` bigint(20) NOT NULL,
-  `TBL_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
-  `LAST_TRANSACTION_INFO` mediumtext NOT NULL,
-  CONSTRAINT `MV_CREATION_METADATA_FK` FOREIGN KEY (`TBL_ID`) REFERENCES `TBLS` (`TBL_ID`)
+  CONSTRAINT `MV_TABLES_USED_FK1` FOREIGN KEY (`MV_CREATION_METADATA_ID`) REFERENCES `MV_CREATION_METADATA` (`MV_CREATION_METADATA_ID`),
+  CONSTRAINT `MV_TABLES_USED_FK2` FOREIGN KEY (`TBL_ID`) REFERENCES `TBLS` (`TBL_ID`)
 ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
 
-CREATE TABLE COMPLETED_TXN_COMPONENTS_NEW (
-  CTC_TXNID bigint NOT NULL,
-  CTC_DATABASE varchar(128) NOT NULL,
-  CTC_TABLE varchar(256),
-  CTC_PARTITION varchar(767),
-  CTC_ID bigint NOT NULL AUTO_INCREMENT,
-  CTC_TIMESTAMP timestamp DEFAULT CURRENT_TIMESTAMP NOT NULL,
-  PRIMARY KEY(CTC_ID)
-) ENGINE=InnoDB DEFAULT CHARSET=latin1;
-CREATE INDEX COMPLETED_TXN_COMPONENTS_IDX2 ON COMPLETED_TXN_COMPONENTS_NEW (CTC_DATABASE, CTC_TABLE, CTC_PARTITION) USING BTREE;
-INSERT INTO COMPLETED_TXN_COMPONENTS_NEW (CTC_TXNID, CTC_DATABASE, CTC_TABLE, CTC_PARTITION)
-SELECT CTC_TXNID, CTC_DATABASE, CTC_TABLE, CTC_PARTITION FROM COMPLETED_TXN_COMPONENTS;
-RENAME TABLE COMPLETED_TXN_COMPONENTS TO COMPLETED_TXN_COMPONENTS_BACKUP;
-RENAME TABLE COMPLETED_TXN_COMPONENTS_NEW TO COMPLETED_TXN_COMPONENTS;
-DROP TABLE COMPLETED_TXN_COMPONENTS_BACKUP;
+ALTER TABLE `COMPLETED_TXN_COMPONENTS` ADD `CTC_TIMESTAMP` timestamp;
+UPDATE `COMPLETED_TXN_COMPONENTS` SET `CTC_TIMESTAMP` = CURRENT_TIMESTAMP;
+ALTER TABLE `COMPLETED_TXN_COMPONENTS` MODIFY COLUMN `CTC_TIMESTAMP` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP;
+CREATE INDEX COMPLETED_TXN_COMPONENTS_IDX ON COMPLETED_TXN_COMPONENTS (CTC_DATABASE, CTC_TABLE, CTC_PARTITION) USING BTREE;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/metastore/scripts/upgrade/mysql/hive-schema-3.0.0.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/hive-schema-3.0.0.mysql.sql b/metastore/scripts/upgrade/mysql/hive-schema-3.0.0.mysql.sql
index 01c995d..947f2e1 100644
--- a/metastore/scripts/upgrade/mysql/hive-schema-3.0.0.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/hive-schema-3.0.0.mysql.sql
@@ -570,6 +570,23 @@ CREATE TABLE IF NOT EXISTS `TABLE_PARAMS` (
 /*!40101 SET character_set_client = @saved_cs_client */;
 
 --
+-- Table structure for table `MV_CREATION_METADATA`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `MV_CREATION_METADATA` (
+  `MV_CREATION_METADATA_ID` bigint(20) NOT NULL,
+  `DB_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `TBL_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+  `TXN_LIST` TEXT DEFAULT NULL,
+  PRIMARY KEY (`MV_CREATION_METADATA_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+CREATE INDEX MV_UNIQUE_TABLE ON MV_CREATION_METADATA (TBL_NAME, DB_NAME) USING BTREE;
+
+--
 -- Table structure for table `TBLS`
 --
 
@@ -587,7 +604,7 @@ CREATE TABLE IF NOT EXISTS `TBLS` (
   `TBL_TYPE` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
   `VIEW_EXPANDED_TEXT` mediumtext,
   `VIEW_ORIGINAL_TEXT` mediumtext,
-  `IS_REWRITE_ENABLED` bit(1) NOT NULL DEFAULT 0,
+  `IS_REWRITE_ENABLED` bit(1) NOT NULL DEFAULT 0
   PRIMARY KEY (`TBL_ID`),
   UNIQUE KEY `UNIQUETABLE` (`TBL_NAME`,`DB_ID`),
   KEY `TBLS_N50` (`SD_ID`),
@@ -598,6 +615,20 @@ CREATE TABLE IF NOT EXISTS `TBLS` (
 /*!40101 SET character_set_client = @saved_cs_client */;
 
 --
+-- Table structure for table `MV_TABLES_USED`
+--
+
+/*!40101 SET @saved_cs_client     = @@character_set_client */;
+/*!40101 SET character_set_client = utf8 */;
+CREATE TABLE IF NOT EXISTS `MV_TABLES_USED` (
+  `MV_CREATION_METADATA_ID` bigint(20) NOT NULL,
+  `TBL_ID` bigint(20) NOT NULL,
+  CONSTRAINT `MV_TABLES_USED_FK1` FOREIGN KEY (`MV_CREATION_METADATA_ID`) REFERENCES `MV_CREATION_METADATA` (`MV_CREATION_METADATA_ID`),
+  CONSTRAINT `MV_TABLES_USED_FK2` FOREIGN KEY (`TBL_ID`) REFERENCES `TBLS` (`TBL_ID`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+/*!40101 SET character_set_client = @saved_cs_client */;
+
+--
 -- Table structure for table `TBL_COL_PRIVS`
 --
 
@@ -910,13 +941,6 @@ CREATE TABLE IF NOT EXISTS WM_MAPPING
     CONSTRAINT `WM_MAPPING_FK2` FOREIGN KEY (`POOL_ID`) REFERENCES `WM_POOL` (`POOL_ID`)
 ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
 
-CREATE TABLE IF NOT EXISTS `MV_CREATION_METADATA` (
-  `TBL_ID` bigint(20) NOT NULL,
-  `TBL_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
-  `LAST_TRANSACTION_INFO` mediumtext NOT NULL,
-  CONSTRAINT `MV_CREATION_METADATA_FK` FOREIGN KEY (`TBL_ID`) REFERENCES `TBLS` (`TBL_ID`)
-) ENGINE=InnoDB DEFAULT CHARSET=latin1;
-
 -- ----------------------------
 -- Transaction and Lock Tables
 -- ----------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/metastore/scripts/upgrade/mysql/hive-txn-schema-3.0.0.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/hive-txn-schema-3.0.0.mysql.sql b/metastore/scripts/upgrade/mysql/hive-txn-schema-3.0.0.mysql.sql
index 497846f..41da503 100644
--- a/metastore/scripts/upgrade/mysql/hive-txn-schema-3.0.0.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/hive-txn-schema-3.0.0.mysql.sql
@@ -45,9 +45,7 @@ CREATE TABLE COMPLETED_TXN_COMPONENTS (
   CTC_DATABASE varchar(128) NOT NULL,
   CTC_TABLE varchar(256),
   CTC_PARTITION varchar(767),
-  CTC_ID bigint NOT NULL AUTO_INCREMENT,
-  CTC_TIMESTAMP timestamp DEFAULT CURRENT_TIMESTAMP NOT NULL,
-  PRIMARY KEY(CTC_ID)
+  CTC_TIMESTAMP timestamp DEFAULT CURRENT_TIMESTAMP NOT NULL
 ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
 
 CREATE INDEX COMPLETED_TXN_COMPONENTS_IDX2 ON COMPLETED_TXN_COMPONENTS (CTC_DATABASE, CTC_TABLE, CTC_PARTITION) USING BTREE;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/metastore/scripts/upgrade/oracle/048-HIVE-14498.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/048-HIVE-14498.oracle.sql b/metastore/scripts/upgrade/oracle/048-HIVE-14498.oracle.sql
index 0b01e89..9c3e3cc 100644
--- a/metastore/scripts/upgrade/oracle/048-HIVE-14498.oracle.sql
+++ b/metastore/scripts/upgrade/oracle/048-HIVE-14498.oracle.sql
@@ -1,23 +1,21 @@
 CREATE TABLE MV_CREATION_METADATA
 (
-    TBL_ID BIGINT NOT NULL,
-    TBL_NAME nvarchar(256) NOT NULL,
-    LAST_TRANSACTION_INFO CLOB NOT NULL
+    MV_CREATION_METADATA_ID NUMBER NOT NULL,
+    DB_NAME VARCHAR2(128) NOT NULL,
+    TBL_NAME VARCHAR2(256) NOT NULL,
+    TXN_LIST CLOB NULL
 );
-ALTER TABLE MV_CREATION_METADATA ADD CONSTRAINT MV_CREATION_METADATA_FK FOREIGN KEY (TBL_ID) REFERENCES TBLS (TBL_ID);
+CREATE TABLE MV_TABLES_USED
+(
+    MV_CREATION_METADATA_ID NUMBER NOT NULL,
+    TBL_ID NUMBER NOT NULL
+);
+ALTER TABLE MV_CREATION_METADATA ADD CONSTRAINT MV_CREATION_METADATA_PK PRIMARY KEY (MV_CREATION_METADATA_ID);
+ALTER TABLE MV_TABLES_USED ADD CONSTRAINT MV_TABLES_USED_FK1 FOREIGN KEY (MV_CREATION_METADATA_ID) REFERENCES MV_CREATION_METADATA (MV_CREATION_METADATA_ID);
+ALTER TABLE MV_TABLES_USED ADD CONSTRAINT MV_TABLES_USED_FK2 FOREIGN KEY (TBL_ID) REFERENCES TBLS (TBL_ID);
 
-CREATE TABLE COMPLETED_TXN_COMPONENTS_NEW (
-  CTC_TXNID NUMBER(19),
-  CTC_DATABASE varchar(128) NOT NULL,
-  CTC_TABLE varchar(128),
-  CTC_PARTITION varchar(767),
-  CTC_ID bigint GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1) NOT NULL,
-  CTC_TIMESTAMP timestamp DEFAULT CURRENT_TIMESTAMP NOT NULL
-) ROWDEPENDENCIES;
-CREATE INDEX COMPLETED_TXN_COMPONENTS_INDEX ON COMPLETED_TXN_COMPONENTS_NEW (CTC_ID);
-CREATE INDEX COMPLETED_TXN_COMPONENTS_INDEX2 ON COMPLETED_TXN_COMPONENTS_NEW (CTC_DATABASE, CTC_TABLE, CTC_PARTITION);
-INSERT INTO COMPLETED_TXN_COMPONENTS_NEW (CTC_TXNID, CTC_DATABASE, CTC_TABLE, CTC_PARTITION)
-SELECT CTC_TXNID, CTC_DATABASE, CTC_TABLE, CTC_PARTITION FROM COMPLETED_TXN_COMPONENTS;
-RENAME TABLE COMPLETED_TXN_COMPONENTS TO COMPLETED_TXN_COMPONENTS_BACKUP;
-RENAME TABLE COMPLETED_TXN_COMPONENTS_NEW TO COMPLETED_TXN_COMPONENTS;
-DROP TABLE COMPLETED_TXN_COMPONENTS_BACKUP;
+ALTER TABLE COMPLETED_TXN_COMPONENTS ADD CTC_TIMESTAMP timestamp NULL;
+UPDATE COMPLETED_TXN_COMPONENTS SET CTC_TIMESTAMP = CURRENT_TIMESTAMP;
+ALTER TABLE COMPLETED_TXN_COMPONENTS MODIFY(CTC_TIMESTAMP DEFAULT CURRENT_TIMESTAMP);
+ALTER TABLE COMPLETED_TXN_COMPONENTS MODIFY(CTC_TIMESTAMP NOT NULL);
+CREATE INDEX COMPLETED_TXN_COMPONENTS_INDEX ON COMPLETED_TXN_COMPONENTS (CTC_DATABASE, CTC_TABLE, CTC_PARTITION);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/metastore/scripts/upgrade/oracle/hive-schema-3.0.0.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/hive-schema-3.0.0.oracle.sql b/metastore/scripts/upgrade/oracle/hive-schema-3.0.0.oracle.sql
index e1aee6f..481d413 100644
--- a/metastore/scripts/upgrade/oracle/hive-schema-3.0.0.oracle.sql
+++ b/metastore/scripts/upgrade/oracle/hive-schema-3.0.0.oracle.sql
@@ -381,6 +381,24 @@ CREATE TABLE TBLS
 
 ALTER TABLE TBLS ADD CONSTRAINT TBLS_PK PRIMARY KEY (TBL_ID);
 
+-- Table MV_CREATION_METADATA for classes [org.apache.hadoop.hive.metastore.model.MCreationMetadata]
+CREATE TABLE MV_CREATION_METADATA
+(
+    MV_CREATION_METADATA_ID NUMBER NOT NULL,
+    DB_NAME VARCHAR2(128) NOT NULL,
+    TBL_NAME VARCHAR2(256) NOT NULL,
+    TXN_LIST CLOB NULL
+);
+
+ALTER TABLE MV_CREATION_METADATA ADD CONSTRAINT MV_CREATION_METADATA_PK PRIMARY KEY (MV_CREATION_METADATA_ID);
+
+-- Table MV_CREATION_METADATA for classes [org.apache.hadoop.hive.metastore.model.MCreationMetadata]
+CREATE TABLE MV_TABLES_USED
+(
+    MV_CREATION_METADATA_ID NUMBER NOT NULL,
+    TBL_ID NUMBER NOT NULL
+);
+
 -- Table PARTITION_EVENTS for classes [org.apache.hadoop.hive.metastore.model.MPartitionEvent]
 CREATE TABLE PARTITION_EVENTS
 (
@@ -633,15 +651,6 @@ CREATE TABLE WM_MAPPING
 
 ALTER TABLE WM_MAPPING ADD CONSTRAINT WM_MAPPING_PK PRIMARY KEY (MAPPING_ID);
 
-CREATE TABLE MV_CREATION_METADATA
-(
-    TBL_ID BIGINT NOT NULL,
-    TBL_NAME nvarchar(256) NOT NULL,
-    LAST_TRANSACTION_INFO CLOB NOT NULL
-);
-
-ALTER TABLE MV_CREATION_METADATA ADD CONSTRAINT MV_CREATION_METADATA_FK FOREIGN KEY (TBL_ID) REFERENCES TBLS (TBL_ID);
-
 -- Constraints for table PART_COL_PRIVS for class(es) [org.apache.hadoop.hive.metastore.model.MPartitionColumnPrivilege]
 ALTER TABLE PART_COL_PRIVS ADD CONSTRAINT PART_COL_PRIVS_FK1 FOREIGN KEY (PART_ID) REFERENCES PARTITIONS (PART_ID) INITIALLY DEFERRED ;
 
@@ -901,6 +910,10 @@ ALTER TABLE WM_MAPPING ADD CONSTRAINT WM_MAPPING_FK1 FOREIGN KEY (RP_ID) REFEREN
 
 ALTER TABLE WM_MAPPING ADD CONSTRAINT WM_MAPPING_FK2 FOREIGN KEY (POOL_ID) REFERENCES WM_POOL (POOL_ID);
 
+ALTER TABLE MV_TABLES_USED ADD CONSTRAINT MV_TABLES_USED_FK1 FOREIGN KEY (MV_CREATION_METADATA_ID) REFERENCES MV_CREATION_METADATA (MV_CREATION_METADATA_ID);
+
+ALTER TABLE MV_TABLES_USED ADD CONSTRAINT MV_TABLES_USED_FK2 FOREIGN KEY (TBL_ID) REFERENCES TBLS (TBL_ID);
+
 ------------------------------
 -- Transaction and lock tables
 ------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/metastore/scripts/upgrade/oracle/hive-txn-schema-3.0.0.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/hive-txn-schema-3.0.0.oracle.sql b/metastore/scripts/upgrade/oracle/hive-txn-schema-3.0.0.oracle.sql
index 5411bc4..5fcf037 100644
--- a/metastore/scripts/upgrade/oracle/hive-txn-schema-3.0.0.oracle.sql
+++ b/metastore/scripts/upgrade/oracle/hive-txn-schema-3.0.0.oracle.sql
@@ -44,12 +44,10 @@ CREATE TABLE COMPLETED_TXN_COMPONENTS (
   CTC_DATABASE varchar(128) NOT NULL,
   CTC_TABLE varchar(128),
   CTC_PARTITION varchar(767),
-  CTC_ID bigint GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1) NOT NULL,
   CTC_TIMESTAMP timestamp DEFAULT CURRENT_TIMESTAMP NOT NULL
 ) ROWDEPENDENCIES;
 
-CREATE INDEX COMPLETED_TXN_COMPONENTS_INDEX ON COMPLETED_TXN_COMPONENTS (CTC_ID);
-CREATE INDEX COMPLETED_TXN_COMPONENTS_INDEX2 ON COMPLETED_TXN_COMPONENTS (CTC_DATABASE, CTC_TABLE, CTC_PARTITION);
+CREATE INDEX COMPLETED_TXN_COMPONENTS_INDEX ON COMPLETED_TXN_COMPONENTS (CTC_DATABASE, CTC_TABLE, CTC_PARTITION);
 
 CREATE TABLE NEXT_TXN_ID (
   NTXN_NEXT NUMBER(19) NOT NULL

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/metastore/scripts/upgrade/postgres/047-HIVE-14498.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/047-HIVE-14498.postgres.sql b/metastore/scripts/upgrade/postgres/047-HIVE-14498.postgres.sql
index 8d4de88..8d1c5b2 100644
--- a/metastore/scripts/upgrade/postgres/047-HIVE-14498.postgres.sql
+++ b/metastore/scripts/upgrade/postgres/047-HIVE-14498.postgres.sql
@@ -1,23 +1,24 @@
 CREATE TABLE "MV_CREATION_METADATA" (
-    "TBL_ID" BIGINT NOT NULL,
+    "MV_CREATION_METADATA_ID" bigint NOT NULL,
+    "DB_NAME" character varying(128) NOT NULL,
     "TBL_NAME" character varying(256) NOT NULL,
-    "LAST_TRANSACTION_INFO" TEXT NOT NULL
+    "TXN_LIST" text
+);
+CREATE TABLE "MV_TABLES_USED" (
+    "MV_CREATION_METADATA_ID" bigint NOT NULL,
+    "TBL_ID" bigint NOT NULL
 );
 ALTER TABLE ONLY "MV_CREATION_METADATA"
-    ADD CONSTRAINT "MV_CREATION_METADATA_FK" FOREIGN KEY ("TBL_ID") REFERENCES "TBLS"("TBL_ID") DEFERRABLE;
+    ADD CONSTRAINT "MV_CREATION_METADATA_PK" PRIMARY KEY ("MV_CREATION_METADATA_ID");
+CREATE INDEX "MV_UNIQUE_TABLE"
+    ON "MV_CREATION_METADATA" USING btree ("TBL_NAME", "DB_NAME");
+ALTER TABLE ONLY "MV_TABLES_USED"
+    ADD CONSTRAINT "MV_TABLES_USED_FK1" FOREIGN KEY ("MV_CREATION_METADATA_ID") REFERENCES "MV_CREATION_METADATA" ("MV_CREATION_METADATA_ID") DEFERRABLE;
+ALTER TABLE ONLY "MV_TABLES_USED"
+    ADD CONSTRAINT "MV_TABLES_USED_FK2" FOREIGN KEY ("TBL_ID") REFERENCES "TBLS" ("TBL_ID") DEFERRABLE;
 
-CREATE TABLE COMPLETED_TXN_COMPONENTS_NEW (
-  CTC_TXNID bigint,
-  CTC_DATABASE varchar(128) NOT NULL,
-  CTC_TABLE varchar(256),
-  CTC_PARTITION varchar(767),
-  CTC_ID serial UNIQUE,
-  CTC_TIMESTAMP timestamp DEFAULT CURRENT_TIMESTAMP NOT NULL
-);
-CREATE INDEX COMPLETED_TXN_COMPONENTS_INDEX ON COMPLETED_TXN_COMPONENTS_NEW USING btree (CTC_ID);
-CREATE INDEX COMPLETED_TXN_COMPONENTS_INDEX2 ON COMPLETED_TXN_COMPONENTS_NEW USING btree (CTC_DATABASE, CTC_TABLE, CTC_PARTITION);
-INSERT INTO COMPLETED_TXN_COMPONENTS_NEW (CTC_TXNID, CTC_DATABASE, CTC_TABLE, CTC_PARTITION)
-SELECT CTC_TXNID, CTC_DATABASE, CTC_TABLE, CTC_PARTITION FROM COMPLETED_TXN_COMPONENTS;
-ALTER TABLE COMPLETED_TXN_COMPONENTS RENAME TO COMPLETED_TXN_COMPONENTS_BACKUP;
-ALTER TABLE COMPLETED_TXN_COMPONENTS_NEW RENAME TO COMPLETED_TXN_COMPONENTS;
-DROP TABLE COMPLETED_TXN_COMPONENTS_BACKUP;
+ALTER TABLE COMPLETED_TXN_COMPONENTS ADD COLUMN CTC_TIMESTAMP timestamp NULL;
+UPDATE COMPLETED_TXN_COMPONENTS SET CTC_TIMESTAMP = CURRENT_TIMESTAMP;
+ALTER TABLE COMPLETED_TXN_COMPONENTS ALTER COLUMN CTC_TIMESTAMP SET NOT NULL;
+ALTER TABLE COMPLETED_TXN_COMPONENTS ALTER COLUMN CTC_TIMESTAMP SET DEFAULT CURRENT_TIMESTAMP;
+CREATE INDEX COMPLETED_TXN_COMPONENTS_INDEX ON COMPLETED_TXN_COMPONENTS USING btree (CTC_DATABASE, CTC_TABLE, CTC_PARTITION);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/metastore/scripts/upgrade/postgres/hive-schema-3.0.0.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/hive-schema-3.0.0.postgres.sql b/metastore/scripts/upgrade/postgres/hive-schema-3.0.0.postgres.sql
index 28cb016..af71ed3 100644
--- a/metastore/scripts/upgrade/postgres/hive-schema-3.0.0.postgres.sql
+++ b/metastore/scripts/upgrade/postgres/hive-schema-3.0.0.postgres.sql
@@ -376,6 +376,25 @@ CREATE TABLE "TBLS" (
     "IS_REWRITE_ENABLED" boolean NOT NULL DEFAULT false
 );
 
+--
+-- Name: MV_CREATION_METADATA; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "MV_CREATION_METADATA" (
+    "MV_CREATION_METADATA_ID" bigint NOT NULL,
+    "DB_NAME" character varying(128) NOT NULL,
+    "TBL_NAME" character varying(256) NOT NULL,
+    "TXN_LIST" text
+);
+
+--
+-- Name: MV_TABLES_USED; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+--
+
+CREATE TABLE "MV_TABLES_USED" (
+    "MV_CREATION_METADATA_ID" bigint NOT NULL,
+    "TBL_ID" bigint NOT NULL
+);
 
 --
 -- Name: TBL_COL_PRIVS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
@@ -659,12 +678,6 @@ CREATE TABLE "WM_MAPPING" (
     "ORDERING" integer
 );
 
-CREATE TABLE "MV_CREATION_METADATA" (
-    "TBL_ID" BIGINT NOT NULL,
-    "TBL_NAME" character varying(256) NOT NULL,
-    "LAST_TRANSACTION_INFO" TEXT NOT NULL
-);
-
 --
 -- Name: BUCKETING_COLS_pkey; Type: CONSTRAINT; Schema: public; Owner: hiveuser; Tablespace:
 --
@@ -1573,7 +1586,16 @@ ALTER TABLE ONLY "WM_MAPPING"
     ADD CONSTRAINT "WM_MAPPING_FK2" FOREIGN KEY ("POOL_ID") REFERENCES "WM_POOL" ("POOL_ID") DEFERRABLE;
 
 ALTER TABLE ONLY "MV_CREATION_METADATA"
-    ADD CONSTRAINT "MV_CREATION_METADATA_FK" FOREIGN KEY ("TBL_ID") REFERENCES "TBLS"("TBL_ID") DEFERRABLE;
+    ADD CONSTRAINT "MV_CREATION_METADATA_PK" PRIMARY KEY ("MV_CREATION_METADATA_ID");
+
+CREATE INDEX "MV_UNIQUE_TABLE"
+    ON "MV_CREATION_METADATA" USING btree ("TBL_NAME", "DB_NAME");
+
+ALTER TABLE ONLY "MV_TABLES_USED"
+    ADD CONSTRAINT "MV_TABLES_USED_FK1" FOREIGN KEY ("MV_CREATION_METADATA_ID") REFERENCES "MV_CREATION_METADATA" ("MV_CREATION_METADATA_ID") DEFERRABLE;
+
+ALTER TABLE ONLY "MV_TABLES_USED"
+    ADD CONSTRAINT "MV_TABLES_USED_FK2" FOREIGN KEY ("TBL_ID") REFERENCES "TBLS" ("TBL_ID") DEFERRABLE;
 
 --
 -- Name: public; Type: ACL; Schema: -; Owner: hiveuser

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/metastore/scripts/upgrade/postgres/hive-txn-schema-3.0.0.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/hive-txn-schema-3.0.0.postgres.sql b/metastore/scripts/upgrade/postgres/hive-txn-schema-3.0.0.postgres.sql
index a81d6ee..3eb0730 100644
--- a/metastore/scripts/upgrade/postgres/hive-txn-schema-3.0.0.postgres.sql
+++ b/metastore/scripts/upgrade/postgres/hive-txn-schema-3.0.0.postgres.sql
@@ -44,12 +44,10 @@ CREATE TABLE COMPLETED_TXN_COMPONENTS (
   CTC_DATABASE varchar(128) NOT NULL,
   CTC_TABLE varchar(256),
   CTC_PARTITION varchar(767),
-  CTC_ID serial UNIQUE,
   CTC_TIMESTAMP timestamp DEFAULT CURRENT_TIMESTAMP NOT NULL
 );
 
-CREATE INDEX COMPLETED_TXN_COMPONENTS_INDEX ON COMPLETED_TXN_COMPONENTS USING btree (CTC_ID);
-CREATE INDEX COMPLETED_TXN_COMPONENTS_INDEX2 ON COMPLETED_TXN_COMPONENTS USING btree (CTC_DATABASE, CTC_TABLE, CTC_PARTITION);
+CREATE INDEX COMPLETED_TXN_COMPONENTS_INDEX ON COMPLETED_TXN_COMPONENTS USING btree (CTC_DATABASE, CTC_TABLE, CTC_PARTITION);
 
 CREATE TABLE NEXT_TXN_ID (
   NTXN_NEXT bigint NOT NULL

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index 51ef390..d3aa571 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -50,6 +50,7 @@ import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.concurrent.ExecutionException;
 
+import com.google.common.collect.ImmutableSet;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -79,6 +80,7 @@ import org.apache.hadoop.hive.metastore.api.BasicTxnInfo;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 import org.apache.hadoop.hive.metastore.api.CompactionResponse;
+import org.apache.hadoop.hive.metastore.api.CreationMetadata;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -5121,10 +5123,11 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
 
       if (crtView.isMaterialized()) {
         // We need to update the status of the creation signature
-        String txnString = conf.get(ValidTxnList.VALID_TXNS_KEY);
-        oldview.getTTable().setCreationMetadata(
-            generateCreationMetadata(db, crtView.getTablesUsed(),
-                txnString == null ? null : new ValidReadTxnList(txnString)));
+        CreationMetadata cm =
+            new CreationMetadata(oldview.getDbName(), oldview.getTableName(),
+                ImmutableSet.copyOf(crtView.getTablesUsed()));
+        cm.setValidTxnList(conf.get(ValidTxnList.VALID_TXNS_KEY));
+        oldview.getTTable().setCreationMetadata(cm);
         db.alterTable(crtView.getViewName(), oldview, null);
         // This is a replace/rebuild, so we need an exclusive lock
         addIfAbsentByName(new WriteEntity(oldview, WriteEntity.WriteType.DDL_EXCLUSIVE));
@@ -5156,10 +5159,11 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       Table tbl = crtView.toTable(conf);
       // We set the signature for the view if it is a materialized view
       if (tbl.isMaterializedView()) {
-        String txnString = conf.get(ValidTxnList.VALID_TXNS_KEY);
-        tbl.getTTable().setCreationMetadata(
-            generateCreationMetadata(db, crtView.getTablesUsed(),
-                txnString == null ? null : new ValidReadTxnList(txnString)));
+        CreationMetadata cm =
+            new CreationMetadata(tbl.getDbName(), tbl.getTableName(),
+                ImmutableSet.copyOf(crtView.getTablesUsed()));
+        cm.setValidTxnList(conf.get(ValidTxnList.VALID_TXNS_KEY));
+        tbl.getTTable().setCreationMetadata(cm);
       }
       db.createTable(tbl, crtView.getIfNotExists());
       addIfAbsentByName(new WriteEntity(tbl, WriteEntity.WriteType.DDL_NO_LOCK));
@@ -5171,38 +5175,6 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     return 0;
   }
 
-  private Map<String, BasicTxnInfo> generateCreationMetadata(
-      Hive db, List<String> tablesUsed, ValidReadTxnList txnList)
-          throws SemanticException {
-    Map<String, BasicTxnInfo> signature = new HashMap<>();
-    try {
-      if (!CollectionUtils.isEmpty(tablesUsed)) {
-        if (txnList == null) {
-          for (String fullyQualifiedName : tablesUsed) {
-            signature.put(fullyQualifiedName, new BasicTxnInfo(true));
-          }
-        } else {
-          List<String> dbNames = new ArrayList<>();
-          List<String> tableNames = new ArrayList<>();
-          for (String fullyQualifiedName : tablesUsed) {
-            // Add to creation metadata
-            String[] names =  fullyQualifiedName.split("\\.");
-            dbNames.add(names[0]);
-            tableNames.add(names[1]);
-          }
-          List<BasicTxnInfo> txnInfos =
-              db.getMSC().getLastCompletedTransactionForTables(dbNames, tableNames, txnList);
-          for (int i = 0; i < tablesUsed.size(); i++) {
-            signature.put(tablesUsed.get(i), txnInfos.get(i));
-          }
-        }
-      }
-    } catch (Exception ex) {
-      throw new SemanticException(ex);
-    }
-    return signature;
-  }
-
   private int truncateTable(Hive db, TruncateTableDesc truncateTableDesc) throws HiveException {
 
     if (truncateTableDesc.getColumnIndexes() != null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
index 9b0ffe0..632a213 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hive.metastore.HiveMetaStoreUtils;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.BasicTxnInfo;
+import org.apache.hadoop.hive.metastore.api.CreationMetadata;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Order;
@@ -846,7 +847,7 @@ public class Table implements Serializable {
   /**
    * @return the creation metadata (only for materialized views)
    */
-  public Map<String, BasicTxnInfo> getCreationMetadata() {
+  public CreationMetadata getCreationMetadata() {
     return tTable.getCreationMetadata();
   }
 
@@ -854,7 +855,7 @@ public class Table implements Serializable {
    * @param creationMetadata
    *          the creation metadata (only for materialized views)
    */
-  public void setCreationMetadata(Map<String, BasicTxnInfo> creationMetadata) {
+  public void setCreationMetadata(CreationMetadata creationMetadata) {
     tTable.setCreationMetadata(creationMetadata);
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
index d159e4b..3eb869d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
@@ -4350,7 +4350,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     // One last test: if we are enabling the rewrite, we need to check that query
     // only uses transactional (MM and ACID) tables
     if (enableFlag) {
-      for (String tableName : materializedViewTable.getCreationMetadata().keySet()) {
+      for (String tableName : materializedViewTable.getCreationMetadata().getTablesUsed()) {
         Table table = getTable(tableName, true);
         if (!AcidUtils.isAcidTable(table)) {
           throw new SemanticException("Automatic rewriting for materialized view cannot "

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index c2e2499..b67a03f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -11726,8 +11726,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     createVwDesc.setViewExpandedText(expandedText);
   }
 
-  private List<String> getTablesUsed(ParseContext parseCtx) throws SemanticException {
-    List<String> tablesUsed = new ArrayList<>();
+  private Set<String> getTablesUsed(ParseContext parseCtx) throws SemanticException {
+    Set<String> tablesUsed = new HashSet<>();
     for (TableScanOperator topOp : parseCtx.getTopOps().values()) {
       Table table = topOp.getConf().getTableMetadata();
       if (!table.isMaterializedTable() && !table.isView()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateViewDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateViewDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateViewDesc.java
index 97baf25..f0f7b18 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateViewDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateViewDesc.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.plan;
 import java.io.Serializable;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -65,7 +66,7 @@ public class CreateViewDesc extends DDLDesc implements Serializable {
   private String serde; // only used for materialized views
   private String storageHandler; // only used for materialized views
   private Map<String, String> serdeProps; // only used for materialized views
-  private List<String> tablesUsed;  // only used for materialized views
+  private Set<String> tablesUsed;  // only used for materialized views
   private ReplicationSpec replicationSpec = null;
 
   /**
@@ -245,11 +246,11 @@ public class CreateViewDesc extends DDLDesc implements Serializable {
     this.ifNotExists = ifNotExists;
   }
 
-  public List<String> getTablesUsed() {
+  public Set<String> getTablesUsed() {
     return tablesUsed;
   }
 
-  public void setTablesUsed(List<String> tablesUsed) {
+  public void setTablesUsed(Set<String> tablesUsed) {
     this.tablesUsed = tablesUsed;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/ql/src/java/org/apache/hadoop/hive/ql/plan/ImportTableDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ImportTableDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ImportTableDesc.java
index 3535fa4..bdfb632 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ImportTableDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ImportTableDesc.java
@@ -23,6 +23,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 
+import com.google.common.collect.ImmutableSet;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -100,6 +101,8 @@ public class ImportTableDesc {
                   table.getSd().getSerdeInfo().getSerializationLib(),
                   null, // storagehandler passed as table params
                   table.getSd().getSerdeInfo().getParameters());
+          this.createViewDesc.setTablesUsed(table.getCreationMetadata() != null ?
+              table.getCreationMetadata().getTablesUsed() : ImmutableSet.of());
         } else {
           this.createViewDesc = new CreateViewDesc(dbDotView,
                   table.getAllCols(),

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
index aa95d2f..b5b478f 100755
--- a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
@@ -172,8 +172,6 @@ public class TestHive extends TestCase {
 
       tbl.setRewriteEnabled(false);
 
-      tbl.setCreationMetadata(new HashMap<String, BasicTxnInfo>());
-
       // create table
       setNullCreateTableGrants();
       try {
@@ -235,8 +233,6 @@ public class TestHive extends TestCase {
 
       tbl.setRewriteEnabled(false);
 
-      tbl.setCreationMetadata(new HashMap<String, BasicTxnInfo>());
-
       setNullCreateTableGrants();
       try {
         hm.createTable(tbl);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/ql/src/test/results/clientpositive/llap/sysdb.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/sysdb.q.out b/ql/src/test/results/clientpositive/llap/sysdb.q.out
index 5ed427f..4ff9875 100644
--- a/ql/src/test/results/clientpositive/llap/sysdb.q.out
+++ b/ql/src/test/results/clientpositive/llap/sysdb.q.out
@@ -1572,6 +1572,80 @@ FROM TBLS"
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: SYS@TBLS
 POSTHOOK: Output: database:sys
+PREHOOK: query: CREATE TABLE IF NOT EXISTS `MV_CREATION_METADATA` (
+  `MV_CREATION_METADATA_ID` bigint,
+  `DB_NAME` string,
+  `TBL_NAME` string,
+  `TXN_LIST` string,
+  CONSTRAINT `SYS_PK_MV_CREATION_METADATA` PRIMARY KEY (`MV_CREATION_METADATA_ID`) DISABLE
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+"hive.sql.database.type" = "METASTORE",
+"hive.sql.query" =
+"SELECT
+  \"MV_CREATION_METADATA_ID\",
+  \"DB_NAME\",
+  \"TBL_NAME\",
+  \"TXN_LIST\"
+FROM MV_CREATION_METADATA"
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: SYS@MV_CREATION_METADATA
+PREHOOK: Output: database:sys
+POSTHOOK: query: CREATE TABLE IF NOT EXISTS `MV_CREATION_METADATA` (
+  `MV_CREATION_METADATA_ID` bigint,
+  `DB_NAME` string,
+  `TBL_NAME` string,
+  `TXN_LIST` string,
+  CONSTRAINT `SYS_PK_MV_CREATION_METADATA` PRIMARY KEY (`MV_CREATION_METADATA_ID`) DISABLE
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+"hive.sql.database.type" = "METASTORE",
+"hive.sql.query" =
+"SELECT
+  \"MV_CREATION_METADATA_ID\",
+  \"DB_NAME\",
+  \"TBL_NAME\",
+  \"TXN_LIST\"
+FROM MV_CREATION_METADATA"
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: SYS@MV_CREATION_METADATA
+POSTHOOK: Output: database:sys
+PREHOOK: query: CREATE TABLE IF NOT EXISTS `MV_TABLES_USED` (
+  `MV_CREATION_METADATA_ID` bigint,
+  `TBL_ID` bigint
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+"hive.sql.database.type" = "METASTORE",
+"hive.sql.query" =
+"SELECT
+  \"MV_CREATION_METADATA_ID\",
+  \"TBL_ID\"
+FROM MV_TABLES_USED"
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: SYS@MV_TABLES_USED
+PREHOOK: Output: database:sys
+POSTHOOK: query: CREATE TABLE IF NOT EXISTS `MV_TABLES_USED` (
+  `MV_CREATION_METADATA_ID` bigint,
+  `TBL_ID` bigint
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+"hive.sql.database.type" = "METASTORE",
+"hive.sql.query" =
+"SELECT
+  \"MV_CREATION_METADATA_ID\",
+  \"TBL_ID\"
+FROM MV_TABLES_USED"
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: SYS@MV_TABLES_USED
+POSTHOOK: Output: database:sys
 PREHOOK: query: CREATE TABLE IF NOT EXISTS `TBL_COL_PRIVS` (
   `TBL_COLUMN_GRANT_ID` bigint,
   `COLUMN_NAME` string,
@@ -1936,13 +2010,15 @@ POSTHOOK: Output: SYS@VERSION
 POSTHOOK: Output: database:sys
 PREHOOK: query: INSERT INTO `VERSION` VALUES (1, '3.0.0', 'Hive release version 3.0.0')
 PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
 PREHOOK: Output: sys@version
 POSTHOOK: query: INSERT INTO `VERSION` VALUES (1, '3.0.0', 'Hive release version 3.0.0')
 POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
 POSTHOOK: Output: sys@version
-POSTHOOK: Lineage: version.schema_version SIMPLE [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
-POSTHOOK: Lineage: version.ver_id EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
-POSTHOOK: Lineage: version.version_comment SIMPLE [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+POSTHOOK: Lineage: version.schema_version SCRIPT []
+POSTHOOK: Lineage: version.ver_id SCRIPT []
+POSTHOOK: Lineage: version.version_comment SCRIPT []
 PREHOOK: query: CREATE TABLE IF NOT EXISTS `DB_VERSION` (
   `VER_ID` BIGINT,
   `SCHEMA_VERSION` string,
@@ -3227,7 +3303,7 @@ POSTHOOK: query: select count(*) from cds
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@cds
 #### A masked pattern was here ####
-71
+73
 PREHOOK: query: select column_name, type_name, integer_idx from columns_v2 order by column_name, integer_idx limit 5
 PREHOOK: type: QUERY
 PREHOOK: Input: sys@columns_v2
@@ -3381,7 +3457,7 @@ POSTHOOK: query: select count(*) from sds
 POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@sds
 #### A masked pattern was here ####
-77
+79
 PREHOOK: query: select param_key, param_value from sd_params order by param_key, param_value limit 5
 PREHOOK: type: QUERY
 PREHOOK: Input: sys@sd_params
@@ -3606,11 +3682,11 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: sys@table_params
 POSTHOOK: Input: sys@table_stats_view
 #### A masked pattern was here ####
-{"BASIC_STATS":"true","COLUMN_STATS":{"key":"true","value":"true"}}	0	0	0	0
-{"BASIC_STATS":"true","COLUMN_STATS":{"entity_name":"true","entity_type":"true","ordering":"true","pool_path":"true","rp_name":"true"}}	0	0	0	0
-{"BASIC_STATS":"true","COLUMN_STATS":{"next_val":"true","sequence_name":"true"}}	0	0	0	0
-{"BASIC_STATS":"true","COLUMN_STATS":{"key":"true","value":"true"}}	0	0	0	0
+{"BASIC_STATS":"true","COLUMN_STATS":{"key":"true"}}	0	0	0	0
+{"BASIC_STATS":"true","COLUMN_STATS":{"alloc_fraction":"true","path":"true","query_parallelism":"true","rp_name":"true","scheduling_policy":"true"}}	0	0	0	0
+{"BASIC_STATS":"true","COLUMN_STATS":{"name":"true","serde_id":"true","slib":"true"}}	0	0	0	0
 #### A masked pattern was here ####
+{"BASIC_STATS":"true","COLUMN_STATS":{"db_id":"true","param_key":"true","param_value":"true"}}	0	0	0	0
 PREHOOK: query: select COLUMN_STATS_ACCURATE, NUM_FILES, NUM_ROWS, RAW_DATA_SIZE, TOTAL_SIZE FROM PARTITION_STATS_VIEW where COLUMN_STATS_ACCURATE is not null order by NUM_FILES, NUM_ROWS, RAW_DATA_SIZE limit 5
 PREHOOK: type: QUERY
 PREHOOK: Input: sys@partition_params
@@ -3785,6 +3861,8 @@ default	sys	global_privs	BASE_TABLE	NULL	NULL	NULL	NULL	NULL	YES	NO	NULL
 default	sys	idxs	BASE_TABLE	NULL	NULL	NULL	NULL	NULL	YES	NO	NULL
 default	sys	index_params	BASE_TABLE	NULL	NULL	NULL	NULL	NULL	YES	NO	NULL
 default	sys	key_constraints	BASE_TABLE	NULL	NULL	NULL	NULL	NULL	YES	NO	NULL
+default	sys	mv_creation_metadata	BASE_TABLE	NULL	NULL	NULL	NULL	NULL	YES	NO	NULL
+default	sys	mv_tables_used	BASE_TABLE	NULL	NULL	NULL	NULL	NULL	YES	NO	NULL
 default	sys	part_col_privs	BASE_TABLE	NULL	NULL	NULL	NULL	NULL	YES	NO	NULL
 default	sys	part_col_stats	BASE_TABLE	NULL	NULL	NULL	NULL	NULL	YES	NO	NULL
 default	sys	part_privs	BASE_TABLE	NULL	NULL	NULL	NULL	NULL	YES	NO	NULL
@@ -3870,13 +3948,13 @@ default	default	alltypesorc	ctimestamp1	8	NULL	YES	timestamp	NULL	NULL	NULL	NULL
 default	default	alltypesorc	ctimestamp2	9	NULL	YES	timestamp	NULL	NULL	NULL	NULL	NULL	9	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	11	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	timestamp	NULL	NULL
 default	default	alltypesorc	cboolean1	10	NULL	YES	boolean	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	11	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	boolean	NULL	NULL
 default	default	alltypesorc	cboolean2	11	NULL	YES	boolean	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	11	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	boolean	NULL	NULL
-default	default	moretypes	a	0	NULL	YES	decimal(10,2)	NULL	NULL	10	10	2	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	85	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	decimal(10,2)	10	10
-default	default	moretypes	b	1	NULL	YES	tinyint	NULL	NULL	3	10	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	85	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	tinyint	3	10
-default	default	moretypes	c	2	NULL	YES	smallint	NULL	NULL	5	10	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	85	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	smallint	5	10
-default	default	moretypes	d	3	NULL	YES	int	NULL	NULL	10	10	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	85	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	int	10	10
-default	default	moretypes	e	4	NULL	YES	bigint	NULL	NULL	19	10	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	85	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	bigint	19	10
-default	default	moretypes	f	5	NULL	YES	varchar(10)	10	10	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	85	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	varchar(10)	NULL	NULL
-default	default	moretypes	g	6	NULL	YES	char(3)	3	3	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	85	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	char(3)	NULL	NULL
+default	default	moretypes	a	0	NULL	YES	decimal(10,2)	NULL	NULL	10	10	2	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	28	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	decimal(10,2)	10	10
+default	default	moretypes	b	1	NULL	YES	tinyint	NULL	NULL	3	10	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	28	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	tinyint	3	10
+default	default	moretypes	c	2	NULL	YES	smallint	NULL	NULL	5	10	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	28	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	smallint	5	10
+default	default	moretypes	d	3	NULL	YES	int	NULL	NULL	10	10	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	28	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	int	10	10
+default	default	moretypes	e	4	NULL	YES	bigint	NULL	NULL	19	10	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	28	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	bigint	19	10
+default	default	moretypes	f	5	NULL	YES	varchar(10)	10	10	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	28	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	varchar(10)	NULL	NULL
+default	default	moretypes	g	6	NULL	YES	char(3)	3	3	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	28	NO	NO	NULL	NULL	NULL	NULL	NULL	NULL	NEVER	NULL	NO	NO	NULL	YES	char(3)	NULL	NULL
 PREHOOK: query: select * from COLUMN_PRIVILEGES order by GRANTOR, GRANTEE, TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME limit 10
 PREHOOK: type: QUERY
 PREHOOK: Input: information_schema@column_privileges


[09/16] hive git commit: HIVE-18546: Remove unnecessary code introduced in HIVE-14498 (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
index 5896fd9..4bdca8c 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
@@ -547,13 +547,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list700 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list700.size);
-                long _elem701;
-                for (int _i702 = 0; _i702 < _list700.size; ++_i702)
+                org.apache.thrift.protocol.TList _list690 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list690.size);
+                long _elem691;
+                for (int _i692 = 0; _i692 < _list690.size; ++_i692)
                 {
-                  _elem701 = iprot.readI64();
-                  struct.fileIds.add(_elem701);
+                  _elem691 = iprot.readI64();
+                  struct.fileIds.add(_elem691);
                 }
                 iprot.readListEnd();
               }
@@ -565,13 +565,13 @@ import org.slf4j.LoggerFactory;
           case 2: // METADATA
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list703 = iprot.readListBegin();
-                struct.metadata = new ArrayList<ByteBuffer>(_list703.size);
-                ByteBuffer _elem704;
-                for (int _i705 = 0; _i705 < _list703.size; ++_i705)
+                org.apache.thrift.protocol.TList _list693 = iprot.readListBegin();
+                struct.metadata = new ArrayList<ByteBuffer>(_list693.size);
+                ByteBuffer _elem694;
+                for (int _i695 = 0; _i695 < _list693.size; ++_i695)
                 {
-                  _elem704 = iprot.readBinary();
-                  struct.metadata.add(_elem704);
+                  _elem694 = iprot.readBinary();
+                  struct.metadata.add(_elem694);
                 }
                 iprot.readListEnd();
               }
@@ -605,9 +605,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size()));
-          for (long _iter706 : struct.fileIds)
+          for (long _iter696 : struct.fileIds)
           {
-            oprot.writeI64(_iter706);
+            oprot.writeI64(_iter696);
           }
           oprot.writeListEnd();
         }
@@ -617,9 +617,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(METADATA_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.metadata.size()));
-          for (ByteBuffer _iter707 : struct.metadata)
+          for (ByteBuffer _iter697 : struct.metadata)
           {
-            oprot.writeBinary(_iter707);
+            oprot.writeBinary(_iter697);
           }
           oprot.writeListEnd();
         }
@@ -651,16 +651,16 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter708 : struct.fileIds)
+        for (long _iter698 : struct.fileIds)
         {
-          oprot.writeI64(_iter708);
+          oprot.writeI64(_iter698);
         }
       }
       {
         oprot.writeI32(struct.metadata.size());
-        for (ByteBuffer _iter709 : struct.metadata)
+        for (ByteBuffer _iter699 : struct.metadata)
         {
-          oprot.writeBinary(_iter709);
+          oprot.writeBinary(_iter699);
         }
       }
       BitSet optionals = new BitSet();
@@ -677,24 +677,24 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, PutFileMetadataRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list710 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list710.size);
-        long _elem711;
-        for (int _i712 = 0; _i712 < _list710.size; ++_i712)
+        org.apache.thrift.protocol.TList _list700 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list700.size);
+        long _elem701;
+        for (int _i702 = 0; _i702 < _list700.size; ++_i702)
         {
-          _elem711 = iprot.readI64();
-          struct.fileIds.add(_elem711);
+          _elem701 = iprot.readI64();
+          struct.fileIds.add(_elem701);
         }
       }
       struct.setFileIdsIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list713 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.metadata = new ArrayList<ByteBuffer>(_list713.size);
-        ByteBuffer _elem714;
-        for (int _i715 = 0; _i715 < _list713.size; ++_i715)
+        org.apache.thrift.protocol.TList _list703 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.metadata = new ArrayList<ByteBuffer>(_list703.size);
+        ByteBuffer _elem704;
+        for (int _i705 = 0; _i705 < _list703.size; ++_i705)
         {
-          _elem714 = iprot.readBinary();
-          struct.metadata.add(_elem714);
+          _elem704 = iprot.readBinary();
+          struct.metadata.add(_elem704);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java
index f9c2370..d1b5247 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java
@@ -168,13 +168,13 @@ import org.slf4j.LoggerFactory;
           if (field.type == NAMES_FIELD_DESC.type) {
             List<String> names;
             {
-              org.apache.thrift.protocol.TList _list478 = iprot.readListBegin();
-              names = new ArrayList<String>(_list478.size);
-              String _elem479;
-              for (int _i480 = 0; _i480 < _list478.size; ++_i480)
+              org.apache.thrift.protocol.TList _list468 = iprot.readListBegin();
+              names = new ArrayList<String>(_list468.size);
+              String _elem469;
+              for (int _i470 = 0; _i470 < _list468.size; ++_i470)
               {
-                _elem479 = iprot.readString();
-                names.add(_elem479);
+                _elem469 = iprot.readString();
+                names.add(_elem469);
               }
               iprot.readListEnd();
             }
@@ -187,14 +187,14 @@ import org.slf4j.LoggerFactory;
           if (field.type == EXPRS_FIELD_DESC.type) {
             List<DropPartitionsExpr> exprs;
             {
-              org.apache.thrift.protocol.TList _list481 = iprot.readListBegin();
-              exprs = new ArrayList<DropPartitionsExpr>(_list481.size);
-              DropPartitionsExpr _elem482;
-              for (int _i483 = 0; _i483 < _list481.size; ++_i483)
+              org.apache.thrift.protocol.TList _list471 = iprot.readListBegin();
+              exprs = new ArrayList<DropPartitionsExpr>(_list471.size);
+              DropPartitionsExpr _elem472;
+              for (int _i473 = 0; _i473 < _list471.size; ++_i473)
               {
-                _elem482 = new DropPartitionsExpr();
-                _elem482.read(iprot);
-                exprs.add(_elem482);
+                _elem472 = new DropPartitionsExpr();
+                _elem472.read(iprot);
+                exprs.add(_elem472);
               }
               iprot.readListEnd();
             }
@@ -219,9 +219,9 @@ import org.slf4j.LoggerFactory;
         List<String> names = (List<String>)value_;
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, names.size()));
-          for (String _iter484 : names)
+          for (String _iter474 : names)
           {
-            oprot.writeString(_iter484);
+            oprot.writeString(_iter474);
           }
           oprot.writeListEnd();
         }
@@ -230,9 +230,9 @@ import org.slf4j.LoggerFactory;
         List<DropPartitionsExpr> exprs = (List<DropPartitionsExpr>)value_;
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, exprs.size()));
-          for (DropPartitionsExpr _iter485 : exprs)
+          for (DropPartitionsExpr _iter475 : exprs)
           {
-            _iter485.write(oprot);
+            _iter475.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -250,13 +250,13 @@ import org.slf4j.LoggerFactory;
         case NAMES:
           List<String> names;
           {
-            org.apache.thrift.protocol.TList _list486 = iprot.readListBegin();
-            names = new ArrayList<String>(_list486.size);
-            String _elem487;
-            for (int _i488 = 0; _i488 < _list486.size; ++_i488)
+            org.apache.thrift.protocol.TList _list476 = iprot.readListBegin();
+            names = new ArrayList<String>(_list476.size);
+            String _elem477;
+            for (int _i478 = 0; _i478 < _list476.size; ++_i478)
             {
-              _elem487 = iprot.readString();
-              names.add(_elem487);
+              _elem477 = iprot.readString();
+              names.add(_elem477);
             }
             iprot.readListEnd();
           }
@@ -264,14 +264,14 @@ import org.slf4j.LoggerFactory;
         case EXPRS:
           List<DropPartitionsExpr> exprs;
           {
-            org.apache.thrift.protocol.TList _list489 = iprot.readListBegin();
-            exprs = new ArrayList<DropPartitionsExpr>(_list489.size);
-            DropPartitionsExpr _elem490;
-            for (int _i491 = 0; _i491 < _list489.size; ++_i491)
+            org.apache.thrift.protocol.TList _list479 = iprot.readListBegin();
+            exprs = new ArrayList<DropPartitionsExpr>(_list479.size);
+            DropPartitionsExpr _elem480;
+            for (int _i481 = 0; _i481 < _list479.size; ++_i481)
             {
-              _elem490 = new DropPartitionsExpr();
-              _elem490.read(iprot);
-              exprs.add(_elem490);
+              _elem480 = new DropPartitionsExpr();
+              _elem480.read(iprot);
+              exprs.add(_elem480);
             }
             iprot.readListEnd();
           }
@@ -291,9 +291,9 @@ import org.slf4j.LoggerFactory;
         List<String> names = (List<String>)value_;
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, names.size()));
-          for (String _iter492 : names)
+          for (String _iter482 : names)
           {
-            oprot.writeString(_iter492);
+            oprot.writeString(_iter482);
           }
           oprot.writeListEnd();
         }
@@ -302,9 +302,9 @@ import org.slf4j.LoggerFactory;
         List<DropPartitionsExpr> exprs = (List<DropPartitionsExpr>)value_;
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, exprs.size()));
-          for (DropPartitionsExpr _iter493 : exprs)
+          for (DropPartitionsExpr _iter483 : exprs)
           {
-            _iter493.write(oprot);
+            _iter483.write(oprot);
           }
           oprot.writeListEnd();
         }

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Schema.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Schema.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Schema.java
index c95216f..eaae445 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Schema.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Schema.java
@@ -445,14 +445,14 @@ import org.slf4j.LoggerFactory;
           case 1: // FIELD_SCHEMAS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list304 = iprot.readListBegin();
-                struct.fieldSchemas = new ArrayList<FieldSchema>(_list304.size);
-                FieldSchema _elem305;
-                for (int _i306 = 0; _i306 < _list304.size; ++_i306)
+                org.apache.thrift.protocol.TList _list294 = iprot.readListBegin();
+                struct.fieldSchemas = new ArrayList<FieldSchema>(_list294.size);
+                FieldSchema _elem295;
+                for (int _i296 = 0; _i296 < _list294.size; ++_i296)
                 {
-                  _elem305 = new FieldSchema();
-                  _elem305.read(iprot);
-                  struct.fieldSchemas.add(_elem305);
+                  _elem295 = new FieldSchema();
+                  _elem295.read(iprot);
+                  struct.fieldSchemas.add(_elem295);
                 }
                 iprot.readListEnd();
               }
@@ -464,15 +464,15 @@ import org.slf4j.LoggerFactory;
           case 2: // PROPERTIES
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map307 = iprot.readMapBegin();
-                struct.properties = new HashMap<String,String>(2*_map307.size);
-                String _key308;
-                String _val309;
-                for (int _i310 = 0; _i310 < _map307.size; ++_i310)
+                org.apache.thrift.protocol.TMap _map297 = iprot.readMapBegin();
+                struct.properties = new HashMap<String,String>(2*_map297.size);
+                String _key298;
+                String _val299;
+                for (int _i300 = 0; _i300 < _map297.size; ++_i300)
                 {
-                  _key308 = iprot.readString();
-                  _val309 = iprot.readString();
-                  struct.properties.put(_key308, _val309);
+                  _key298 = iprot.readString();
+                  _val299 = iprot.readString();
+                  struct.properties.put(_key298, _val299);
                 }
                 iprot.readMapEnd();
               }
@@ -498,9 +498,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FIELD_SCHEMAS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.fieldSchemas.size()));
-          for (FieldSchema _iter311 : struct.fieldSchemas)
+          for (FieldSchema _iter301 : struct.fieldSchemas)
           {
-            _iter311.write(oprot);
+            _iter301.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -510,10 +510,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PROPERTIES_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.properties.size()));
-          for (Map.Entry<String, String> _iter312 : struct.properties.entrySet())
+          for (Map.Entry<String, String> _iter302 : struct.properties.entrySet())
           {
-            oprot.writeString(_iter312.getKey());
-            oprot.writeString(_iter312.getValue());
+            oprot.writeString(_iter302.getKey());
+            oprot.writeString(_iter302.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -547,19 +547,19 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetFieldSchemas()) {
         {
           oprot.writeI32(struct.fieldSchemas.size());
-          for (FieldSchema _iter313 : struct.fieldSchemas)
+          for (FieldSchema _iter303 : struct.fieldSchemas)
           {
-            _iter313.write(oprot);
+            _iter303.write(oprot);
           }
         }
       }
       if (struct.isSetProperties()) {
         {
           oprot.writeI32(struct.properties.size());
-          for (Map.Entry<String, String> _iter314 : struct.properties.entrySet())
+          for (Map.Entry<String, String> _iter304 : struct.properties.entrySet())
           {
-            oprot.writeString(_iter314.getKey());
-            oprot.writeString(_iter314.getValue());
+            oprot.writeString(_iter304.getKey());
+            oprot.writeString(_iter304.getValue());
           }
         }
       }
@@ -571,29 +571,29 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list315 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.fieldSchemas = new ArrayList<FieldSchema>(_list315.size);
-          FieldSchema _elem316;
-          for (int _i317 = 0; _i317 < _list315.size; ++_i317)
+          org.apache.thrift.protocol.TList _list305 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.fieldSchemas = new ArrayList<FieldSchema>(_list305.size);
+          FieldSchema _elem306;
+          for (int _i307 = 0; _i307 < _list305.size; ++_i307)
           {
-            _elem316 = new FieldSchema();
-            _elem316.read(iprot);
-            struct.fieldSchemas.add(_elem316);
+            _elem306 = new FieldSchema();
+            _elem306.read(iprot);
+            struct.fieldSchemas.add(_elem306);
           }
         }
         struct.setFieldSchemasIsSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TMap _map318 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.properties = new HashMap<String,String>(2*_map318.size);
-          String _key319;
-          String _val320;
-          for (int _i321 = 0; _i321 < _map318.size; ++_i321)
+          org.apache.thrift.protocol.TMap _map308 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.properties = new HashMap<String,String>(2*_map308.size);
+          String _key309;
+          String _val310;
+          for (int _i311 = 0; _i311 < _map308.size; ++_i311)
           {
-            _key319 = iprot.readString();
-            _val320 = iprot.readString();
-            struct.properties.put(_key319, _val320);
+            _key309 = iprot.readString();
+            _val310 = iprot.readString();
+            struct.properties.put(_key309, _val310);
           }
         }
         struct.setPropertiesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetPartitionsStatsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetPartitionsStatsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetPartitionsStatsRequest.java
index f4a66ed..ae0fbb4 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetPartitionsStatsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetPartitionsStatsRequest.java
@@ -435,14 +435,14 @@ import org.slf4j.LoggerFactory;
           case 1: // COL_STATS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list296 = iprot.readListBegin();
-                struct.colStats = new ArrayList<ColumnStatistics>(_list296.size);
-                ColumnStatistics _elem297;
-                for (int _i298 = 0; _i298 < _list296.size; ++_i298)
+                org.apache.thrift.protocol.TList _list286 = iprot.readListBegin();
+                struct.colStats = new ArrayList<ColumnStatistics>(_list286.size);
+                ColumnStatistics _elem287;
+                for (int _i288 = 0; _i288 < _list286.size; ++_i288)
                 {
-                  _elem297 = new ColumnStatistics();
-                  _elem297.read(iprot);
-                  struct.colStats.add(_elem297);
+                  _elem287 = new ColumnStatistics();
+                  _elem287.read(iprot);
+                  struct.colStats.add(_elem287);
                 }
                 iprot.readListEnd();
               }
@@ -476,9 +476,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COL_STATS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.colStats.size()));
-          for (ColumnStatistics _iter299 : struct.colStats)
+          for (ColumnStatistics _iter289 : struct.colStats)
           {
-            _iter299.write(oprot);
+            _iter289.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -508,9 +508,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.colStats.size());
-        for (ColumnStatistics _iter300 : struct.colStats)
+        for (ColumnStatistics _iter290 : struct.colStats)
         {
-          _iter300.write(oprot);
+          _iter290.write(oprot);
         }
       }
       BitSet optionals = new BitSet();
@@ -527,14 +527,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, SetPartitionsStatsRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list301 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.colStats = new ArrayList<ColumnStatistics>(_list301.size);
-        ColumnStatistics _elem302;
-        for (int _i303 = 0; _i303 < _list301.size; ++_i303)
+        org.apache.thrift.protocol.TList _list291 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.colStats = new ArrayList<ColumnStatistics>(_list291.size);
+        ColumnStatistics _elem292;
+        for (int _i293 = 0; _i293 < _list291.size; ++_i293)
         {
-          _elem302 = new ColumnStatistics();
-          _elem302.read(iprot);
-          struct.colStats.add(_elem302);
+          _elem292 = new ColumnStatistics();
+          _elem292.read(iprot);
+          struct.colStats.add(_elem292);
         }
       }
       struct.setColStatsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
index 1b50602..5687b19 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // COMPACTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list608 = iprot.readListBegin();
-                struct.compacts = new ArrayList<ShowCompactResponseElement>(_list608.size);
-                ShowCompactResponseElement _elem609;
-                for (int _i610 = 0; _i610 < _list608.size; ++_i610)
+                org.apache.thrift.protocol.TList _list598 = iprot.readListBegin();
+                struct.compacts = new ArrayList<ShowCompactResponseElement>(_list598.size);
+                ShowCompactResponseElement _elem599;
+                for (int _i600 = 0; _i600 < _list598.size; ++_i600)
                 {
-                  _elem609 = new ShowCompactResponseElement();
-                  _elem609.read(iprot);
-                  struct.compacts.add(_elem609);
+                  _elem599 = new ShowCompactResponseElement();
+                  _elem599.read(iprot);
+                  struct.compacts.add(_elem599);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COMPACTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.compacts.size()));
-          for (ShowCompactResponseElement _iter611 : struct.compacts)
+          for (ShowCompactResponseElement _iter601 : struct.compacts)
           {
-            _iter611.write(oprot);
+            _iter601.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.compacts.size());
-        for (ShowCompactResponseElement _iter612 : struct.compacts)
+        for (ShowCompactResponseElement _iter602 : struct.compacts)
         {
-          _iter612.write(oprot);
+          _iter602.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, ShowCompactResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list613 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.compacts = new ArrayList<ShowCompactResponseElement>(_list613.size);
-        ShowCompactResponseElement _elem614;
-        for (int _i615 = 0; _i615 < _list613.size; ++_i615)
+        org.apache.thrift.protocol.TList _list603 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.compacts = new ArrayList<ShowCompactResponseElement>(_list603.size);
+        ShowCompactResponseElement _elem604;
+        for (int _i605 = 0; _i605 < _list603.size; ++_i605)
         {
-          _elem614 = new ShowCompactResponseElement();
-          _elem614.read(iprot);
-          struct.compacts.add(_elem614);
+          _elem604 = new ShowCompactResponseElement();
+          _elem604.read(iprot);
+          struct.compacts.add(_elem604);
         }
       }
       struct.setCompactsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
index a21a191..f22deb2 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
@@ -350,14 +350,14 @@ import org.slf4j.LoggerFactory;
           case 1: // LOCKS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list574 = iprot.readListBegin();
-                struct.locks = new ArrayList<ShowLocksResponseElement>(_list574.size);
-                ShowLocksResponseElement _elem575;
-                for (int _i576 = 0; _i576 < _list574.size; ++_i576)
+                org.apache.thrift.protocol.TList _list564 = iprot.readListBegin();
+                struct.locks = new ArrayList<ShowLocksResponseElement>(_list564.size);
+                ShowLocksResponseElement _elem565;
+                for (int _i566 = 0; _i566 < _list564.size; ++_i566)
                 {
-                  _elem575 = new ShowLocksResponseElement();
-                  _elem575.read(iprot);
-                  struct.locks.add(_elem575);
+                  _elem565 = new ShowLocksResponseElement();
+                  _elem565.read(iprot);
+                  struct.locks.add(_elem565);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(LOCKS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.locks.size()));
-          for (ShowLocksResponseElement _iter577 : struct.locks)
+          for (ShowLocksResponseElement _iter567 : struct.locks)
           {
-            _iter577.write(oprot);
+            _iter567.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -416,9 +416,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetLocks()) {
         {
           oprot.writeI32(struct.locks.size());
-          for (ShowLocksResponseElement _iter578 : struct.locks)
+          for (ShowLocksResponseElement _iter568 : struct.locks)
           {
-            _iter578.write(oprot);
+            _iter568.write(oprot);
           }
         }
       }
@@ -430,14 +430,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list579 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.locks = new ArrayList<ShowLocksResponseElement>(_list579.size);
-          ShowLocksResponseElement _elem580;
-          for (int _i581 = 0; _i581 < _list579.size; ++_i581)
+          org.apache.thrift.protocol.TList _list569 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.locks = new ArrayList<ShowLocksResponseElement>(_list569.size);
+          ShowLocksResponseElement _elem570;
+          for (int _i571 = 0; _i571 < _list569.size; ++_i571)
           {
-            _elem580 = new ShowLocksResponseElement();
-            _elem580.read(iprot);
-            struct.locks.add(_elem580);
+            _elem570 = new ShowLocksResponseElement();
+            _elem570.read(iprot);
+            struct.locks.add(_elem570);
           }
         }
         struct.setLocksIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
index f317b03..a132e5e 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
@@ -53,7 +53,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField PRIVILEGES_FIELD_DESC = new org.apache.thrift.protocol.TField("privileges", org.apache.thrift.protocol.TType.STRUCT, (short)13);
   private static final org.apache.thrift.protocol.TField TEMPORARY_FIELD_DESC = new org.apache.thrift.protocol.TField("temporary", org.apache.thrift.protocol.TType.BOOL, (short)14);
   private static final org.apache.thrift.protocol.TField REWRITE_ENABLED_FIELD_DESC = new org.apache.thrift.protocol.TField("rewriteEnabled", org.apache.thrift.protocol.TType.BOOL, (short)15);
-  private static final org.apache.thrift.protocol.TField CREATION_METADATA_FIELD_DESC = new org.apache.thrift.protocol.TField("creationMetadata", org.apache.thrift.protocol.TType.MAP, (short)16);
+  private static final org.apache.thrift.protocol.TField CREATION_METADATA_FIELD_DESC = new org.apache.thrift.protocol.TField("creationMetadata", org.apache.thrift.protocol.TType.STRUCT, (short)16);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -76,7 +76,7 @@ import org.slf4j.LoggerFactory;
   private PrincipalPrivilegeSet privileges; // optional
   private boolean temporary; // optional
   private boolean rewriteEnabled; // optional
-  private Map<String,BasicTxnInfo> creationMetadata; // optional
+  private CreationMetadata creationMetadata; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -226,9 +226,7 @@ import org.slf4j.LoggerFactory;
     tmpMap.put(_Fields.REWRITE_ENABLED, new org.apache.thrift.meta_data.FieldMetaData("rewriteEnabled", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
     tmpMap.put(_Fields.CREATION_METADATA, new org.apache.thrift.meta_data.FieldMetaData("creationMetadata", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
-        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
-            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
-            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT            , "BasicTxnInfo"))));
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT        , "CreationMetadata")));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Table.class, metaDataMap);
   }
@@ -316,19 +314,7 @@ import org.slf4j.LoggerFactory;
     this.temporary = other.temporary;
     this.rewriteEnabled = other.rewriteEnabled;
     if (other.isSetCreationMetadata()) {
-      Map<String,BasicTxnInfo> __this__creationMetadata = new HashMap<String,BasicTxnInfo>(other.creationMetadata.size());
-      for (Map.Entry<String, BasicTxnInfo> other_element : other.creationMetadata.entrySet()) {
-
-        String other_element_key = other_element.getKey();
-        BasicTxnInfo other_element_value = other_element.getValue();
-
-        String __this__creationMetadata_copy_key = other_element_key;
-
-        BasicTxnInfo __this__creationMetadata_copy_value = other_element_value;
-
-        __this__creationMetadata.put(__this__creationMetadata_copy_key, __this__creationMetadata_copy_value);
-      }
-      this.creationMetadata = __this__creationMetadata;
+      this.creationMetadata = other.creationMetadata;
     }
   }
 
@@ -727,22 +713,11 @@ import org.slf4j.LoggerFactory;
     __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REWRITEENABLED_ISSET_ID, value);
   }
 
-  public int getCreationMetadataSize() {
-    return (this.creationMetadata == null) ? 0 : this.creationMetadata.size();
-  }
-
-  public void putToCreationMetadata(String key, BasicTxnInfo val) {
-    if (this.creationMetadata == null) {
-      this.creationMetadata = new HashMap<String,BasicTxnInfo>();
-    }
-    this.creationMetadata.put(key, val);
-  }
-
-  public Map<String,BasicTxnInfo> getCreationMetadata() {
+  public CreationMetadata getCreationMetadata() {
     return this.creationMetadata;
   }
 
-  public void setCreationMetadata(Map<String,BasicTxnInfo> creationMetadata) {
+  public void setCreationMetadata(CreationMetadata creationMetadata) {
     this.creationMetadata = creationMetadata;
   }
 
@@ -887,7 +862,7 @@ import org.slf4j.LoggerFactory;
       if (value == null) {
         unsetCreationMetadata();
       } else {
-        setCreationMetadata((Map<String,BasicTxnInfo>)value);
+        setCreationMetadata((CreationMetadata)value);
       }
       break;
 
@@ -1738,21 +1713,9 @@ import org.slf4j.LoggerFactory;
             }
             break;
           case 16: // CREATION_METADATA
-            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
-              {
-                org.apache.thrift.protocol.TMap _map197 = iprot.readMapBegin();
-                struct.creationMetadata = new HashMap<String,BasicTxnInfo>(2*_map197.size);
-                String _key198;
-                BasicTxnInfo _val199;
-                for (int _i200 = 0; _i200 < _map197.size; ++_i200)
-                {
-                  _key198 = iprot.readString();
-                  _val199 = new BasicTxnInfo();
-                  _val199.read(iprot);
-                  struct.creationMetadata.put(_key198, _val199);
-                }
-                iprot.readMapEnd();
-              }
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.creationMetadata = new CreationMetadata();
+              struct.creationMetadata.read(iprot);
               struct.setCreationMetadataIsSet(true);
             } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
@@ -1804,9 +1767,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARTITION_KEYS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitionKeys.size()));
-          for (FieldSchema _iter201 : struct.partitionKeys)
+          for (FieldSchema _iter197 : struct.partitionKeys)
           {
-            _iter201.write(oprot);
+            _iter197.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1816,10 +1779,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARAMETERS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.parameters.size()));
-          for (Map.Entry<String, String> _iter202 : struct.parameters.entrySet())
+          for (Map.Entry<String, String> _iter198 : struct.parameters.entrySet())
           {
-            oprot.writeString(_iter202.getKey());
-            oprot.writeString(_iter202.getValue());
+            oprot.writeString(_iter198.getKey());
+            oprot.writeString(_iter198.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -1860,15 +1823,7 @@ import org.slf4j.LoggerFactory;
       if (struct.creationMetadata != null) {
         if (struct.isSetCreationMetadata()) {
           oprot.writeFieldBegin(CREATION_METADATA_FIELD_DESC);
-          {
-            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.creationMetadata.size()));
-            for (Map.Entry<String, BasicTxnInfo> _iter203 : struct.creationMetadata.entrySet())
-            {
-              oprot.writeString(_iter203.getKey());
-              _iter203.getValue().write(oprot);
-            }
-            oprot.writeMapEnd();
-          }
+          struct.creationMetadata.write(oprot);
           oprot.writeFieldEnd();
         }
       }
@@ -1963,19 +1918,19 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitionKeys()) {
         {
           oprot.writeI32(struct.partitionKeys.size());
-          for (FieldSchema _iter204 : struct.partitionKeys)
+          for (FieldSchema _iter199 : struct.partitionKeys)
           {
-            _iter204.write(oprot);
+            _iter199.write(oprot);
           }
         }
       }
       if (struct.isSetParameters()) {
         {
           oprot.writeI32(struct.parameters.size());
-          for (Map.Entry<String, String> _iter205 : struct.parameters.entrySet())
+          for (Map.Entry<String, String> _iter200 : struct.parameters.entrySet())
           {
-            oprot.writeString(_iter205.getKey());
-            oprot.writeString(_iter205.getValue());
+            oprot.writeString(_iter200.getKey());
+            oprot.writeString(_iter200.getValue());
           }
         }
       }
@@ -1998,14 +1953,7 @@ import org.slf4j.LoggerFactory;
         oprot.writeBool(struct.rewriteEnabled);
       }
       if (struct.isSetCreationMetadata()) {
-        {
-          oprot.writeI32(struct.creationMetadata.size());
-          for (Map.Entry<String, BasicTxnInfo> _iter206 : struct.creationMetadata.entrySet())
-          {
-            oprot.writeString(_iter206.getKey());
-            _iter206.getValue().write(oprot);
-          }
-        }
+        struct.creationMetadata.write(oprot);
       }
     }
 
@@ -2044,29 +1992,29 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(7)) {
         {
-          org.apache.thrift.protocol.TList _list207 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.partitionKeys = new ArrayList<FieldSchema>(_list207.size);
-          FieldSchema _elem208;
-          for (int _i209 = 0; _i209 < _list207.size; ++_i209)
+          org.apache.thrift.protocol.TList _list201 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.partitionKeys = new ArrayList<FieldSchema>(_list201.size);
+          FieldSchema _elem202;
+          for (int _i203 = 0; _i203 < _list201.size; ++_i203)
           {
-            _elem208 = new FieldSchema();
-            _elem208.read(iprot);
-            struct.partitionKeys.add(_elem208);
+            _elem202 = new FieldSchema();
+            _elem202.read(iprot);
+            struct.partitionKeys.add(_elem202);
           }
         }
         struct.setPartitionKeysIsSet(true);
       }
       if (incoming.get(8)) {
         {
-          org.apache.thrift.protocol.TMap _map210 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.parameters = new HashMap<String,String>(2*_map210.size);
-          String _key211;
-          String _val212;
-          for (int _i213 = 0; _i213 < _map210.size; ++_i213)
+          org.apache.thrift.protocol.TMap _map204 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.parameters = new HashMap<String,String>(2*_map204.size);
+          String _key205;
+          String _val206;
+          for (int _i207 = 0; _i207 < _map204.size; ++_i207)
           {
-            _key211 = iprot.readString();
-            _val212 = iprot.readString();
-            struct.parameters.put(_key211, _val212);
+            _key205 = iprot.readString();
+            _val206 = iprot.readString();
+            struct.parameters.put(_key205, _val206);
           }
         }
         struct.setParametersIsSet(true);
@@ -2097,19 +2045,8 @@ import org.slf4j.LoggerFactory;
         struct.setRewriteEnabledIsSet(true);
       }
       if (incoming.get(15)) {
-        {
-          org.apache.thrift.protocol.TMap _map214 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.creationMetadata = new HashMap<String,BasicTxnInfo>(2*_map214.size);
-          String _key215;
-          BasicTxnInfo _val216;
-          for (int _i217 = 0; _i217 < _map214.size; ++_i217)
-          {
-            _key215 = iprot.readString();
-            _val216 = new BasicTxnInfo();
-            _val216.read(iprot);
-            struct.creationMetadata.put(_key215, _val216);
-          }
-        }
+        struct.creationMetadata = new CreationMetadata();
+        struct.creationMetadata.read(iprot);
         struct.setCreationMetadataIsSet(true);
       }
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java
index 575fa9f..69be837 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java
@@ -537,13 +537,13 @@ import org.slf4j.LoggerFactory;
           case 3: // COL_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list430 = iprot.readListBegin();
-                struct.colNames = new ArrayList<String>(_list430.size);
-                String _elem431;
-                for (int _i432 = 0; _i432 < _list430.size; ++_i432)
+                org.apache.thrift.protocol.TList _list420 = iprot.readListBegin();
+                struct.colNames = new ArrayList<String>(_list420.size);
+                String _elem421;
+                for (int _i422 = 0; _i422 < _list420.size; ++_i422)
                 {
-                  _elem431 = iprot.readString();
-                  struct.colNames.add(_elem431);
+                  _elem421 = iprot.readString();
+                  struct.colNames.add(_elem421);
                 }
                 iprot.readListEnd();
               }
@@ -579,9 +579,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COL_NAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.colNames.size()));
-          for (String _iter433 : struct.colNames)
+          for (String _iter423 : struct.colNames)
           {
-            oprot.writeString(_iter433);
+            oprot.writeString(_iter423);
           }
           oprot.writeListEnd();
         }
@@ -608,9 +608,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeString(struct.tblName);
       {
         oprot.writeI32(struct.colNames.size());
-        for (String _iter434 : struct.colNames)
+        for (String _iter424 : struct.colNames)
         {
-          oprot.writeString(_iter434);
+          oprot.writeString(_iter424);
         }
       }
     }
@@ -623,13 +623,13 @@ import org.slf4j.LoggerFactory;
       struct.tblName = iprot.readString();
       struct.setTblNameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list435 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.colNames = new ArrayList<String>(_list435.size);
-        String _elem436;
-        for (int _i437 = 0; _i437 < _list435.size; ++_i437)
+        org.apache.thrift.protocol.TList _list425 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.colNames = new ArrayList<String>(_list425.size);
+        String _elem426;
+        for (int _i427 = 0; _i427 < _list425.size; ++_i427)
         {
-          _elem436 = iprot.readString();
-          struct.colNames.add(_elem436);
+          _elem426 = iprot.readString();
+          struct.colNames.add(_elem426);
         }
       }
       struct.setColNamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a328f03/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java
index 32aeb9d..e65166e 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // TABLE_STATS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list404 = iprot.readListBegin();
-                struct.tableStats = new ArrayList<ColumnStatisticsObj>(_list404.size);
-                ColumnStatisticsObj _elem405;
-                for (int _i406 = 0; _i406 < _list404.size; ++_i406)
+                org.apache.thrift.protocol.TList _list394 = iprot.readListBegin();
+                struct.tableStats = new ArrayList<ColumnStatisticsObj>(_list394.size);
+                ColumnStatisticsObj _elem395;
+                for (int _i396 = 0; _i396 < _list394.size; ++_i396)
                 {
-                  _elem405 = new ColumnStatisticsObj();
-                  _elem405.read(iprot);
-                  struct.tableStats.add(_elem405);
+                  _elem395 = new ColumnStatisticsObj();
+                  _elem395.read(iprot);
+                  struct.tableStats.add(_elem395);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TABLE_STATS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.tableStats.size()));
-          for (ColumnStatisticsObj _iter407 : struct.tableStats)
+          for (ColumnStatisticsObj _iter397 : struct.tableStats)
           {
-            _iter407.write(oprot);
+            _iter397.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.tableStats.size());
-        for (ColumnStatisticsObj _iter408 : struct.tableStats)
+        for (ColumnStatisticsObj _iter398 : struct.tableStats)
         {
-          _iter408.write(oprot);
+          _iter398.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, TableStatsResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list409 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.tableStats = new ArrayList<ColumnStatisticsObj>(_list409.size);
-        ColumnStatisticsObj _elem410;
-        for (int _i411 = 0; _i411 < _list409.size; ++_i411)
+        org.apache.thrift.protocol.TList _list399 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.tableStats = new ArrayList<ColumnStatisticsObj>(_list399.size);
+        ColumnStatisticsObj _elem400;
+        for (int _i401 = 0; _i401 < _list399.size; ++_i401)
         {
-          _elem410 = new ColumnStatisticsObj();
-          _elem410.read(iprot);
-          struct.tableStats.add(_elem410);
+          _elem400 = new ColumnStatisticsObj();
+          _elem400.read(iprot);
+          struct.tableStats.add(_elem400);
         }
       }
       struct.setTableStatsIsSet(true);