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

[01/12] hive git commit: HIVE-19089: Create/Replicate Allocate write-id event (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Repository: hive
Updated Branches:
  refs/heads/master 7fb088b7a -> fa9e743e7


http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 9256b7a..39a0f31 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
@@ -124,6 +124,7 @@ import org.apache.hadoop.hive.metastore.datasource.BoneCPDataSourceProvider;
 import org.apache.hadoop.hive.metastore.datasource.DataSourceProvider;
 import org.apache.hadoop.hive.metastore.datasource.HikariCPDataSourceProvider;
 import org.apache.hadoop.hive.metastore.events.AbortTxnEvent;
+import org.apache.hadoop.hive.metastore.events.AllocWriteIdEvent;
 import org.apache.hadoop.hive.metastore.events.CommitTxnEvent;
 import org.apache.hadoop.hive.metastore.events.OpenTxnEvent;
 import org.apache.hadoop.hive.metastore.messaging.EventMessage;
@@ -584,24 +585,15 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         stmt = dbConn.createStatement();
 
         if (rqst.isSetReplPolicy()) {
-          List<String> inQueries = new ArrayList<>();
-          StringBuilder prefix = new StringBuilder();
-          StringBuilder suffix = new StringBuilder();
-
-          prefix.append("select RTM_TARGET_TXN_ID from REPL_TXN_MAP where ");
-          suffix.append(" and RTM_REPL_POLICY = " + quoteString(rqst.getReplPolicy()));
-
-          TxnUtils.buildQueryWithINClause(conf, inQueries, prefix, suffix, rqst.getReplSrcTxnIds(),
-                  "RTM_SRC_TXN_ID", false, false);
-
-          for (String query : inQueries) {
-            LOG.debug("Going to execute select <" + query + ">");
-            rs = stmt.executeQuery(query);
-            if (rs.next()) {
-              LOG.info("Transactions " + rqst.getReplSrcTxnIds().toString() +
-                      " are already present for repl policy " + rqst.getReplPolicy());
-              return new OpenTxnsResponse(new ArrayList<>());
+          List<Long> targetTxnIdList = getTargetTxnIdList(rqst.getReplPolicy(), rqst.getReplSrcTxnIds(), stmt);
+          if (!targetTxnIdList.isEmpty()) {
+            if (targetTxnIdList.size() != rqst.getReplSrcTxnIds().size()) {
+              LOG.warn("target txn id number " + targetTxnIdList.toString() +
+                      " is not matching with source txn id number " + rqst.getReplSrcTxnIds().toString());
             }
+            LOG.info("Target transactions " + targetTxnIdList.toString() + " are present for repl policy :" +
+              rqst.getReplPolicy() + " and Source transaction id : " + rqst.getReplSrcTxnIds().toString());
+            return new OpenTxnsResponse(targetTxnIdList);
           }
         }
 
@@ -677,8 +669,8 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         }
 
         if (transactionalListeners != null) {
-          MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                  EventMessage.EventType.OPEN_TXN, new OpenTxnEvent(txnIds, dbConn, sqlGenerator));
+          MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners,
+                  EventMessage.EventType.OPEN_TXN, new OpenTxnEvent(txnIds, null), dbConn, sqlGenerator);
         }
 
         LOG.debug("Going to commit");
@@ -699,21 +691,27 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
     }
   }
 
-  private Long getTargetTxnId(String replPolicy, long sourceTxnId, Statement stmt) throws SQLException {
+  private List<Long> getTargetTxnIdList(String replPolicy, List<Long> sourceTxnIdList, Statement stmt)
+          throws SQLException {
     ResultSet rs = null;
     try {
-      String s = "select RTM_TARGET_TXN_ID from REPL_TXN_MAP where RTM_SRC_TXN_ID = " + sourceTxnId +
-
-              " and RTM_REPL_POLICY = " + quoteString(replPolicy);
-      LOG.debug("Going to execute query <" + s + ">");
-      rs = stmt.executeQuery(s);
-      if (!rs.next()) {
-        LOG.info("Target txn is missing for the input source txn for ReplPolicy: " +
-                quoteString(replPolicy) + " , srcTxnId: " + sourceTxnId);
-        return -1L;
+      List<String> inQueries = new ArrayList<>();
+      StringBuilder prefix = new StringBuilder();
+      StringBuilder suffix = new StringBuilder();
+      List<Long> targetTxnIdList = new ArrayList<>();
+      prefix.append("select RTM_TARGET_TXN_ID from REPL_TXN_MAP where ");
+      suffix.append(" and RTM_REPL_POLICY = " + quoteString(replPolicy));
+      TxnUtils.buildQueryWithINClause(conf, inQueries, prefix, suffix, sourceTxnIdList,
+              "RTM_SRC_TXN_ID", false, false);
+      for (String query : inQueries) {
+        LOG.debug("Going to execute select <" + query + ">");
+        rs = stmt.executeQuery(query);
+        while (rs.next()) {
+          targetTxnIdList.add(rs.getLong(1));
+        }
       }
-      LOG.debug("targetTxnid for srcTxnId " + sourceTxnId + " is " + rs.getLong(1));
-      return rs.getLong(1);
+      LOG.debug("targetTxnid for srcTxnId " + sourceTxnIdList.toString() + " is " + targetTxnIdList.toString());
+      return targetTxnIdList;
     }  catch (SQLException e) {
       LOG.warn("failed to get target txn ids " + e.getMessage());
       throw e;
@@ -737,10 +735,15 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
 
         if (rqst.isSetReplPolicy()) {
           sourceTxnId = rqst.getTxnid();
-          txnid = getTargetTxnId(rqst.getReplPolicy(), sourceTxnId, stmt);
-          if (txnid == -1) {
+          List<Long> targetTxnIds = getTargetTxnIdList(rqst.getReplPolicy(),
+                  Collections.singletonList(sourceTxnId), stmt);
+          if (targetTxnIds.isEmpty()) {
+            LOG.info("Target txn id is missing for source txn id : " + sourceTxnId +
+                    " and repl policy " + rqst.getReplPolicy());
             return;
           }
+          assert targetTxnIds.size() == 1;
+          txnid = targetTxnIds.get(0);
         }
 
         if (abortTxns(dbConn, Collections.singletonList(txnid), true) != 1) {
@@ -769,8 +772,8 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         }
 
         if (transactionalListeners != null) {
-          MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                  EventMessage.EventType.ABORT_TXN, new AbortTxnEvent(txnid, dbConn, sqlGenerator));
+          MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners,
+                  EventMessage.EventType.ABORT_TXN, new AbortTxnEvent(txnid, null), dbConn, sqlGenerator);
         }
 
         LOG.debug("Going to commit");
@@ -808,8 +811,8 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
 
         for (Long txnId : txnids) {
           if (transactionalListeners != null) {
-            MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                    EventMessage.EventType.ABORT_TXN, new AbortTxnEvent(txnId, dbConn, sqlGenerator));
+            MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners,
+                    EventMessage.EventType.ABORT_TXN, new AbortTxnEvent(txnId, null), dbConn, sqlGenerator);
           }
         }
         LOG.debug("Going to commit");
@@ -880,10 +883,15 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
 
         if (rqst.isSetReplPolicy()) {
           sourceTxnId = rqst.getTxnid();
-          txnid = getTargetTxnId(rqst.getReplPolicy(), sourceTxnId, stmt);
-          if (txnid == -1) {
+          List<Long> targetTxnIds = getTargetTxnIdList(rqst.getReplPolicy(),
+                  Collections.singletonList(sourceTxnId), stmt);
+          if (targetTxnIds.isEmpty()) {
+            LOG.info("Target txn id is missing for source txn id : " + sourceTxnId +
+                    " and repl policy " + rqst.getReplPolicy());
             return;
           }
+          assert targetTxnIds.size() == 1;
+          txnid = targetTxnIds.get(0);
         }
 
         /**
@@ -1052,8 +1060,8 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         }
 
         if (transactionalListeners != null) {
-          MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                  EventMessage.EventType.COMMIT_TXN, new CommitTxnEvent(txnid, dbConn, sqlGenerator));
+          MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners,
+                  EventMessage.EventType.COMMIT_TXN, new CommitTxnEvent(txnid, null), dbConn, sqlGenerator);
         }
 
         MaterializationsInvalidationCache materializationsInvalidationCache =
@@ -1225,9 +1233,10 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
   }
 
   @Override
+  @RetrySemantics.Idempotent
   public AllocateTableWriteIdsResponse allocateTableWriteIds(AllocateTableWriteIdsRequest rqst)
           throws NoSuchTxnException, TxnAbortedException, MetaException {
-    List<Long> txnIds = rqst.getTxnIds();
+    List<Long> txnIds;
     String dbName = rqst.getDbName().toLowerCase();
     String tblName = rqst.getTableName().toLowerCase();
     try {
@@ -1235,12 +1244,36 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
       Statement stmt = null;
       ResultSet rs = null;
       TxnStore.MutexAPI.LockHandle handle = null;
+      List<TxnToWriteId> txnToWriteIds = new ArrayList<>();
+      List<TxnToWriteId> srcTxnToWriteIds = null;
       try {
         lockInternal();
         dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
         stmt = dbConn.createStatement();
 
-        Collections.sort(txnIds); //easier to read logs
+        if (rqst.isSetReplPolicy()) {
+          srcTxnToWriteIds = rqst.getSrcTxnToWriteIdList();
+          List<Long> srcTxnIds = new ArrayList<>();
+          assert (rqst.isSetSrcTxnToWriteIdList());
+          assert (!rqst.isSetTxnIds());
+          assert (!srcTxnToWriteIds.isEmpty());
+
+          for (TxnToWriteId txnToWriteId :  srcTxnToWriteIds) {
+            srcTxnIds.add(txnToWriteId.getTxnId());
+          }
+          txnIds = getTargetTxnIdList(rqst.getReplPolicy(), srcTxnIds, stmt);
+          if (srcTxnIds.size() != txnIds.size()) {
+            LOG.warn("Target txn id is missing for source txn id : " + srcTxnIds.toString() +
+                    " and repl policy " + rqst.getReplPolicy());
+            throw new RuntimeException("This should never happen for txnIds: " + txnIds);
+          }
+        } else {
+          assert (!rqst.isSetSrcTxnToWriteIdList());
+          assert (rqst.isSetTxnIds());
+          txnIds = rqst.getTxnIds();
+        }
+
+        Collections.sort(txnIds); //easier to read logs and for assumption done in replication flow
 
         // Check if all the input txns are in open state. Write ID should be allocated only for open transactions.
         if (!isTxnsInOpenState(txnIds, stmt)) {
@@ -1248,10 +1281,10 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
           throw new RuntimeException("This should never happen for txnIds: " + txnIds);
         }
 
-        List<TxnToWriteId> txnToWriteIds = new ArrayList<>();
-        List<Long> allocatedTxns = new ArrayList<>();
-        long txnId;
         long writeId;
+        String s;
+        long allocatedTxnsCount = 0;
+        long txnId;
         List<String> queries = new ArrayList<>();
         StringBuilder prefix = new StringBuilder();
         StringBuilder suffix = new StringBuilder();
@@ -1274,25 +1307,26 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
             txnId = rs.getLong(1);
             writeId = rs.getLong(2);
             txnToWriteIds.add(new TxnToWriteId(txnId, writeId));
-            allocatedTxns.add(txnId);
+            allocatedTxnsCount++;
             LOG.info("Reused already allocated writeID: " + writeId + " for txnId: " + txnId);
           }
         }
 
-        // If all the txns in the list have already allocated write ids, then just skip new allocations
-        long numOfWriteIds = txnIds.size() - allocatedTxns.size();
-        assert(numOfWriteIds >= 0);
-        if (0 == numOfWriteIds) {
-          // If all the txns in the list have pre-allocated write ids for the given table, then just return
+        // Batch allocation should always happen atomically. Either write ids for all txns is allocated or none.
+        long numOfWriteIds = txnIds.size();
+        assert ((allocatedTxnsCount == 0) || (numOfWriteIds == allocatedTxnsCount));
+        if (allocatedTxnsCount == numOfWriteIds) {
+          // If all the txns in the list have pre-allocated write ids for the given table, then just return.
+          // This is for idempotent case.
           return new AllocateTableWriteIdsResponse(txnToWriteIds);
         }
 
         handle = getMutexAPI().acquireLock(MUTEX_KEY.WriteIdAllocator.name());
 
-        // There are some txns in the list which has no write id allocated and hence go ahead and do it.
+        // There are some txns in the list which does not have write id allocated and hence go ahead and do it.
         // Get the next write id for the given table and update it with new next write id.
         // This is select for update query which takes a lock if the table entry is already there in NEXT_WRITE_ID
-        String s = sqlGenerator.addForUpdateClause(
+        s = sqlGenerator.addForUpdateClause(
                 "select nwi_next from NEXT_WRITE_ID where nwi_database = " + quoteString(dbName)
                         + " and nwi_table = " + quoteString(tblName));
         LOG.debug("Going to execute query <" + s + ">");
@@ -1300,14 +1334,14 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         if (!rs.next()) {
           // First allocation of write id should add the table to the next_write_id meta table
           // The initial value for write id should be 1 and hence we add 1 with number of write ids allocated here
+          writeId = 1;
           s = "insert into NEXT_WRITE_ID (nwi_database, nwi_table, nwi_next) values ("
                   + quoteString(dbName) + "," + quoteString(tblName) + "," + String.valueOf(numOfWriteIds + 1) + ")";
           LOG.debug("Going to execute insert <" + s + ">");
           stmt.execute(s);
-          writeId = 1;
         } else {
-          // Update the NEXT_WRITE_ID for the given table after incrementing by number of write ids allocated
           writeId = rs.getLong(1);
+          // Update the NEXT_WRITE_ID for the given table after incrementing by number of write ids allocated
           s = "update NEXT_WRITE_ID set nwi_next = " + (writeId + numOfWriteIds)
                   + " where nwi_database = " + quoteString(dbName)
                   + " and nwi_table = " + quoteString(tblName);
@@ -1319,15 +1353,22 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         // write ids
         List<String> rows = new ArrayList<>();
         for (long txn : txnIds) {
-          if (allocatedTxns.contains(txn)) {
-            continue;
-          }
           rows.add(txn + ", " + quoteString(dbName) + ", " + quoteString(tblName) + ", " + writeId);
           txnToWriteIds.add(new TxnToWriteId(txn, writeId));
           LOG.info("Allocated writeID: " + writeId + " for txnId: " + txn);
           writeId++;
         }
 
+        if (rqst.isSetReplPolicy()) {
+          int lastIdx = txnToWriteIds.size()-1;
+          if ((txnToWriteIds.get(0).getWriteId() != srcTxnToWriteIds.get(0).getWriteId()) ||
+              (txnToWriteIds.get(lastIdx).getWriteId() != srcTxnToWriteIds.get(lastIdx).getWriteId())) {
+            LOG.error("Allocated write id range {} is not matching with the input write id range {}.",
+                    txnToWriteIds, srcTxnToWriteIds);
+            throw new IllegalStateException("Write id allocation failed for: " + srcTxnToWriteIds);
+          }
+        }
+
         // Insert entries to TXN_TO_WRITE_ID for newly allocated write ids
         List<String> inserts = sqlGenerator.createInsertValuesStmt(
                 "TXN_TO_WRITE_ID (t2w_txnid, t2w_database, t2w_table, t2w_writeid)", rows);
@@ -1336,6 +1377,13 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
           stmt.execute(insert);
         }
 
+        if (transactionalListeners != null) {
+          MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners,
+                  EventMessage.EventType.ALLOC_WRITE_ID,
+                  new AllocWriteIdEvent(txnToWriteIds, rqst.getDbName(), rqst.getTableName(), null),
+                  dbConn, sqlGenerator);
+        }
+
         LOG.debug("Going to commit");
         dbConn.commit();
         return new AllocateTableWriteIdsResponse(txnToWriteIds);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 612afe1..5bba329 100644
--- a/standalone-metastore/src/main/thrift/hive_metastore.thrift
+++ b/standalone-metastore/src/main/thrift/hive_metastore.thrift
@@ -884,9 +884,14 @@ struct GetValidWriteIdsResponse {
 
 // Request msg to allocate table write ids for the given list of txns
 struct AllocateTableWriteIdsRequest {
-    1: required list<i64> txnIds,
-    2: required string dbName,
-    3: required string tableName,
+    1: required string dbName,
+    2: required string tableName,
+    // Either txnIds or replPolicy+srcTxnToWriteIdList can exist in a call. txnIds is used by normal flow and
+    // replPolicy+srcTxnToWriteIdList is used by replication task.
+    3: optional list<i64> txnIds,
+    4: optional string replPolicy,
+    // The list is assumed to be sorted by both txnids and write ids. The write id list is assumed to be contiguous.
+    5: optional list<TxnToWriteId> srcTxnToWriteIdList,
 }
 
 // Map for allocated write id against the txn for which it is allocated

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
index ecddc7b..74c057b 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
@@ -2231,9 +2231,22 @@ public class HiveMetaStoreClientPreCatalog implements IMetaStoreClient, AutoClos
   @Override
   public List<TxnToWriteId> allocateTableWriteIdsBatch(List<Long> txnIds, String dbName, String tableName)
           throws TException {
-    AllocateTableWriteIdsRequest rqst = new AllocateTableWriteIdsRequest(txnIds, dbName, tableName);
-    AllocateTableWriteIdsResponse writeIds = client.allocate_table_write_ids(rqst);
-    return writeIds.getTxnToWriteIds();
+    AllocateTableWriteIdsRequest rqst = new AllocateTableWriteIdsRequest(dbName, tableName);
+    rqst.setTxnIds(txnIds);
+    return allocateTableWriteIdsBatchIntr(rqst);
+  }
+
+  @Override
+  public List<TxnToWriteId> replAllocateTableWriteIdsBatch(String dbName, String tableName,
+                                         String replPolicy, List<TxnToWriteId> srcTxnToWriteIdList) throws TException {
+    AllocateTableWriteIdsRequest rqst = new AllocateTableWriteIdsRequest(dbName, tableName);
+    rqst.setReplPolicy(replPolicy);
+    rqst.setSrcTxnToWriteIdList(srcTxnToWriteIdList);
+    return allocateTableWriteIdsBatchIntr(rqst);
+  }
+
+  private List<TxnToWriteId> allocateTableWriteIdsBatchIntr(AllocateTableWriteIdsRequest rqst) throws TException {
+    return client.allocate_table_write_ids(rqst).getTxnToWriteIds();
   }
 
   @Override


[03/12] hive git commit: HIVE-19089: Create/Replicate Allocate write-id event (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 b0e64d8..d241414 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
@@ -15507,10 +15507,10 @@ class get_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype805, _size802) = iprot.readListBegin()
-          for _i806 in xrange(_size802):
-            _elem807 = iprot.readString()
-            self.success.append(_elem807)
+          (_etype812, _size809) = iprot.readListBegin()
+          for _i813 in xrange(_size809):
+            _elem814 = iprot.readString()
+            self.success.append(_elem814)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15533,8 +15533,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 iter808 in self.success:
-        oprot.writeString(iter808)
+      for iter815 in self.success:
+        oprot.writeString(iter815)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15639,10 +15639,10 @@ class get_all_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype812, _size809) = iprot.readListBegin()
-          for _i813 in xrange(_size809):
-            _elem814 = iprot.readString()
-            self.success.append(_elem814)
+          (_etype819, _size816) = iprot.readListBegin()
+          for _i820 in xrange(_size816):
+            _elem821 = iprot.readString()
+            self.success.append(_elem821)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15665,8 +15665,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 iter815 in self.success:
-        oprot.writeString(iter815)
+      for iter822 in self.success:
+        oprot.writeString(iter822)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -16436,12 +16436,12 @@ class get_type_all_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype817, _vtype818, _size816 ) = iprot.readMapBegin()
-          for _i820 in xrange(_size816):
-            _key821 = iprot.readString()
-            _val822 = Type()
-            _val822.read(iprot)
-            self.success[_key821] = _val822
+          (_ktype824, _vtype825, _size823 ) = iprot.readMapBegin()
+          for _i827 in xrange(_size823):
+            _key828 = iprot.readString()
+            _val829 = Type()
+            _val829.read(iprot)
+            self.success[_key828] = _val829
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -16464,9 +16464,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 kiter823,viter824 in self.success.items():
-        oprot.writeString(kiter823)
-        viter824.write(oprot)
+      for kiter830,viter831 in self.success.items():
+        oprot.writeString(kiter830)
+        viter831.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -16609,11 +16609,11 @@ class get_fields_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype828, _size825) = iprot.readListBegin()
-          for _i829 in xrange(_size825):
-            _elem830 = FieldSchema()
-            _elem830.read(iprot)
-            self.success.append(_elem830)
+          (_etype835, _size832) = iprot.readListBegin()
+          for _i836 in xrange(_size832):
+            _elem837 = FieldSchema()
+            _elem837.read(iprot)
+            self.success.append(_elem837)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16648,8 +16648,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 iter831 in self.success:
-        iter831.write(oprot)
+      for iter838 in self.success:
+        iter838.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -16816,11 +16816,11 @@ class get_fields_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype835, _size832) = iprot.readListBegin()
-          for _i836 in xrange(_size832):
-            _elem837 = FieldSchema()
-            _elem837.read(iprot)
-            self.success.append(_elem837)
+          (_etype842, _size839) = iprot.readListBegin()
+          for _i843 in xrange(_size839):
+            _elem844 = FieldSchema()
+            _elem844.read(iprot)
+            self.success.append(_elem844)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16855,8 +16855,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 iter838 in self.success:
-        iter838.write(oprot)
+      for iter845 in self.success:
+        iter845.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17009,11 +17009,11 @@ class get_schema_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype842, _size839) = iprot.readListBegin()
-          for _i843 in xrange(_size839):
-            _elem844 = FieldSchema()
-            _elem844.read(iprot)
-            self.success.append(_elem844)
+          (_etype849, _size846) = iprot.readListBegin()
+          for _i850 in xrange(_size846):
+            _elem851 = FieldSchema()
+            _elem851.read(iprot)
+            self.success.append(_elem851)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17048,8 +17048,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 iter845 in self.success:
-        iter845.write(oprot)
+      for iter852 in self.success:
+        iter852.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17216,11 +17216,11 @@ class get_schema_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype849, _size846) = iprot.readListBegin()
-          for _i850 in xrange(_size846):
-            _elem851 = FieldSchema()
-            _elem851.read(iprot)
-            self.success.append(_elem851)
+          (_etype856, _size853) = iprot.readListBegin()
+          for _i857 in xrange(_size853):
+            _elem858 = FieldSchema()
+            _elem858.read(iprot)
+            self.success.append(_elem858)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17255,8 +17255,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 iter852 in self.success:
-        iter852.write(oprot)
+      for iter859 in self.success:
+        iter859.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17709,66 +17709,66 @@ class create_table_with_constraints_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.primaryKeys = []
-          (_etype856, _size853) = iprot.readListBegin()
-          for _i857 in xrange(_size853):
-            _elem858 = SQLPrimaryKey()
-            _elem858.read(iprot)
-            self.primaryKeys.append(_elem858)
+          (_etype863, _size860) = iprot.readListBegin()
+          for _i864 in xrange(_size860):
+            _elem865 = SQLPrimaryKey()
+            _elem865.read(iprot)
+            self.primaryKeys.append(_elem865)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.foreignKeys = []
-          (_etype862, _size859) = iprot.readListBegin()
-          for _i863 in xrange(_size859):
-            _elem864 = SQLForeignKey()
-            _elem864.read(iprot)
-            self.foreignKeys.append(_elem864)
+          (_etype869, _size866) = iprot.readListBegin()
+          for _i870 in xrange(_size866):
+            _elem871 = SQLForeignKey()
+            _elem871.read(iprot)
+            self.foreignKeys.append(_elem871)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.LIST:
           self.uniqueConstraints = []
-          (_etype868, _size865) = iprot.readListBegin()
-          for _i869 in xrange(_size865):
-            _elem870 = SQLUniqueConstraint()
-            _elem870.read(iprot)
-            self.uniqueConstraints.append(_elem870)
+          (_etype875, _size872) = iprot.readListBegin()
+          for _i876 in xrange(_size872):
+            _elem877 = SQLUniqueConstraint()
+            _elem877.read(iprot)
+            self.uniqueConstraints.append(_elem877)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.LIST:
           self.notNullConstraints = []
-          (_etype874, _size871) = iprot.readListBegin()
-          for _i875 in xrange(_size871):
-            _elem876 = SQLNotNullConstraint()
-            _elem876.read(iprot)
-            self.notNullConstraints.append(_elem876)
+          (_etype881, _size878) = iprot.readListBegin()
+          for _i882 in xrange(_size878):
+            _elem883 = SQLNotNullConstraint()
+            _elem883.read(iprot)
+            self.notNullConstraints.append(_elem883)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 6:
         if ftype == TType.LIST:
           self.defaultConstraints = []
-          (_etype880, _size877) = iprot.readListBegin()
-          for _i881 in xrange(_size877):
-            _elem882 = SQLDefaultConstraint()
-            _elem882.read(iprot)
-            self.defaultConstraints.append(_elem882)
+          (_etype887, _size884) = iprot.readListBegin()
+          for _i888 in xrange(_size884):
+            _elem889 = SQLDefaultConstraint()
+            _elem889.read(iprot)
+            self.defaultConstraints.append(_elem889)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 7:
         if ftype == TType.LIST:
           self.checkConstraints = []
-          (_etype886, _size883) = iprot.readListBegin()
-          for _i887 in xrange(_size883):
-            _elem888 = SQLCheckConstraint()
-            _elem888.read(iprot)
-            self.checkConstraints.append(_elem888)
+          (_etype893, _size890) = iprot.readListBegin()
+          for _i894 in xrange(_size890):
+            _elem895 = SQLCheckConstraint()
+            _elem895.read(iprot)
+            self.checkConstraints.append(_elem895)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17789,43 +17789,43 @@ 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 iter889 in self.primaryKeys:
-        iter889.write(oprot)
+      for iter896 in self.primaryKeys:
+        iter896.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 iter890 in self.foreignKeys:
-        iter890.write(oprot)
+      for iter897 in self.foreignKeys:
+        iter897.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 iter891 in self.uniqueConstraints:
-        iter891.write(oprot)
+      for iter898 in self.uniqueConstraints:
+        iter898.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 iter892 in self.notNullConstraints:
-        iter892.write(oprot)
+      for iter899 in self.notNullConstraints:
+        iter899.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.defaultConstraints is not None:
       oprot.writeFieldBegin('defaultConstraints', TType.LIST, 6)
       oprot.writeListBegin(TType.STRUCT, len(self.defaultConstraints))
-      for iter893 in self.defaultConstraints:
-        iter893.write(oprot)
+      for iter900 in self.defaultConstraints:
+        iter900.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.checkConstraints is not None:
       oprot.writeFieldBegin('checkConstraints', TType.LIST, 7)
       oprot.writeListBegin(TType.STRUCT, len(self.checkConstraints))
-      for iter894 in self.checkConstraints:
-        iter894.write(oprot)
+      for iter901 in self.checkConstraints:
+        iter901.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -19385,10 +19385,10 @@ class truncate_table_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.partNames = []
-          (_etype898, _size895) = iprot.readListBegin()
-          for _i899 in xrange(_size895):
-            _elem900 = iprot.readString()
-            self.partNames.append(_elem900)
+          (_etype905, _size902) = iprot.readListBegin()
+          for _i906 in xrange(_size902):
+            _elem907 = iprot.readString()
+            self.partNames.append(_elem907)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19413,8 +19413,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 iter901 in self.partNames:
-        oprot.writeString(iter901)
+      for iter908 in self.partNames:
+        oprot.writeString(iter908)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -19614,10 +19614,10 @@ class get_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype905, _size902) = iprot.readListBegin()
-          for _i906 in xrange(_size902):
-            _elem907 = iprot.readString()
-            self.success.append(_elem907)
+          (_etype912, _size909) = iprot.readListBegin()
+          for _i913 in xrange(_size909):
+            _elem914 = iprot.readString()
+            self.success.append(_elem914)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19640,8 +19640,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 iter908 in self.success:
-        oprot.writeString(iter908)
+      for iter915 in self.success:
+        oprot.writeString(iter915)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19791,10 +19791,10 @@ class get_tables_by_type_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype912, _size909) = iprot.readListBegin()
-          for _i913 in xrange(_size909):
-            _elem914 = iprot.readString()
-            self.success.append(_elem914)
+          (_etype919, _size916) = iprot.readListBegin()
+          for _i920 in xrange(_size916):
+            _elem921 = iprot.readString()
+            self.success.append(_elem921)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19817,8 +19817,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 iter915 in self.success:
-        oprot.writeString(iter915)
+      for iter922 in self.success:
+        oprot.writeString(iter922)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19942,10 +19942,10 @@ class get_materialized_views_for_rewriting_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype919, _size916) = iprot.readListBegin()
-          for _i920 in xrange(_size916):
-            _elem921 = iprot.readString()
-            self.success.append(_elem921)
+          (_etype926, _size923) = iprot.readListBegin()
+          for _i927 in xrange(_size923):
+            _elem928 = iprot.readString()
+            self.success.append(_elem928)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19968,8 +19968,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 iter922 in self.success:
-        oprot.writeString(iter922)
+      for iter929 in self.success:
+        oprot.writeString(iter929)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20042,10 +20042,10 @@ class get_table_meta_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.tbl_types = []
-          (_etype926, _size923) = iprot.readListBegin()
-          for _i927 in xrange(_size923):
-            _elem928 = iprot.readString()
-            self.tbl_types.append(_elem928)
+          (_etype933, _size930) = iprot.readListBegin()
+          for _i934 in xrange(_size930):
+            _elem935 = iprot.readString()
+            self.tbl_types.append(_elem935)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20070,8 +20070,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 iter929 in self.tbl_types:
-        oprot.writeString(iter929)
+      for iter936 in self.tbl_types:
+        oprot.writeString(iter936)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20127,11 +20127,11 @@ class get_table_meta_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype933, _size930) = iprot.readListBegin()
-          for _i934 in xrange(_size930):
-            _elem935 = TableMeta()
-            _elem935.read(iprot)
-            self.success.append(_elem935)
+          (_etype940, _size937) = iprot.readListBegin()
+          for _i941 in xrange(_size937):
+            _elem942 = TableMeta()
+            _elem942.read(iprot)
+            self.success.append(_elem942)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20154,8 +20154,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 iter936 in self.success:
-        iter936.write(oprot)
+      for iter943 in self.success:
+        iter943.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20279,10 +20279,10 @@ class get_all_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype940, _size937) = iprot.readListBegin()
-          for _i941 in xrange(_size937):
-            _elem942 = iprot.readString()
-            self.success.append(_elem942)
+          (_etype947, _size944) = iprot.readListBegin()
+          for _i948 in xrange(_size944):
+            _elem949 = iprot.readString()
+            self.success.append(_elem949)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20305,8 +20305,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 iter943 in self.success:
-        oprot.writeString(iter943)
+      for iter950 in self.success:
+        oprot.writeString(iter950)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20542,10 +20542,10 @@ class get_table_objects_by_name_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype947, _size944) = iprot.readListBegin()
-          for _i948 in xrange(_size944):
-            _elem949 = iprot.readString()
-            self.tbl_names.append(_elem949)
+          (_etype954, _size951) = iprot.readListBegin()
+          for _i955 in xrange(_size951):
+            _elem956 = iprot.readString()
+            self.tbl_names.append(_elem956)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20566,8 +20566,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 iter950 in self.tbl_names:
-        oprot.writeString(iter950)
+      for iter957 in self.tbl_names:
+        oprot.writeString(iter957)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20619,11 +20619,11 @@ class get_table_objects_by_name_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype954, _size951) = iprot.readListBegin()
-          for _i955 in xrange(_size951):
-            _elem956 = Table()
-            _elem956.read(iprot)
-            self.success.append(_elem956)
+          (_etype961, _size958) = iprot.readListBegin()
+          for _i962 in xrange(_size958):
+            _elem963 = Table()
+            _elem963.read(iprot)
+            self.success.append(_elem963)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20640,8 +20640,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 iter957 in self.success:
-        iter957.write(oprot)
+      for iter964 in self.success:
+        iter964.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21033,10 +21033,10 @@ class get_materialization_invalidation_info_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype961, _size958) = iprot.readListBegin()
-          for _i962 in xrange(_size958):
-            _elem963 = iprot.readString()
-            self.tbl_names.append(_elem963)
+          (_etype968, _size965) = iprot.readListBegin()
+          for _i969 in xrange(_size965):
+            _elem970 = iprot.readString()
+            self.tbl_names.append(_elem970)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21057,8 +21057,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 iter964 in self.tbl_names:
-        oprot.writeString(iter964)
+      for iter971 in self.tbl_names:
+        oprot.writeString(iter971)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21119,12 +21119,12 @@ class get_materialization_invalidation_info_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype966, _vtype967, _size965 ) = iprot.readMapBegin()
-          for _i969 in xrange(_size965):
-            _key970 = iprot.readString()
-            _val971 = Materialization()
-            _val971.read(iprot)
-            self.success[_key970] = _val971
+          (_ktype973, _vtype974, _size972 ) = iprot.readMapBegin()
+          for _i976 in xrange(_size972):
+            _key977 = iprot.readString()
+            _val978 = Materialization()
+            _val978.read(iprot)
+            self.success[_key977] = _val978
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -21159,9 +21159,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 kiter972,viter973 in self.success.items():
-        oprot.writeString(kiter972)
-        viter973.write(oprot)
+      for kiter979,viter980 in self.success.items():
+        oprot.writeString(kiter979)
+        viter980.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21526,10 +21526,10 @@ class get_table_names_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype977, _size974) = iprot.readListBegin()
-          for _i978 in xrange(_size974):
-            _elem979 = iprot.readString()
-            self.success.append(_elem979)
+          (_etype984, _size981) = iprot.readListBegin()
+          for _i985 in xrange(_size981):
+            _elem986 = iprot.readString()
+            self.success.append(_elem986)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21564,8 +21564,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 iter980 in self.success:
-        oprot.writeString(iter980)
+      for iter987 in self.success:
+        oprot.writeString(iter987)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -22535,11 +22535,11 @@ class add_partitions_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype984, _size981) = iprot.readListBegin()
-          for _i985 in xrange(_size981):
-            _elem986 = Partition()
-            _elem986.read(iprot)
-            self.new_parts.append(_elem986)
+          (_etype991, _size988) = iprot.readListBegin()
+          for _i992 in xrange(_size988):
+            _elem993 = Partition()
+            _elem993.read(iprot)
+            self.new_parts.append(_elem993)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22556,8 +22556,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 iter987 in self.new_parts:
-        iter987.write(oprot)
+      for iter994 in self.new_parts:
+        iter994.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -22715,11 +22715,11 @@ class add_partitions_pspec_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype991, _size988) = iprot.readListBegin()
-          for _i992 in xrange(_size988):
-            _elem993 = PartitionSpec()
-            _elem993.read(iprot)
-            self.new_parts.append(_elem993)
+          (_etype998, _size995) = iprot.readListBegin()
+          for _i999 in xrange(_size995):
+            _elem1000 = PartitionSpec()
+            _elem1000.read(iprot)
+            self.new_parts.append(_elem1000)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22736,8 +22736,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 iter994 in self.new_parts:
-        iter994.write(oprot)
+      for iter1001 in self.new_parts:
+        iter1001.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -22911,10 +22911,10 @@ class append_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype998, _size995) = iprot.readListBegin()
-          for _i999 in xrange(_size995):
-            _elem1000 = iprot.readString()
-            self.part_vals.append(_elem1000)
+          (_etype1005, _size1002) = iprot.readListBegin()
+          for _i1006 in xrange(_size1002):
+            _elem1007 = iprot.readString()
+            self.part_vals.append(_elem1007)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22939,8 +22939,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 iter1001 in self.part_vals:
-        oprot.writeString(iter1001)
+      for iter1008 in self.part_vals:
+        oprot.writeString(iter1008)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -23293,10 +23293,10 @@ class append_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1005, _size1002) = iprot.readListBegin()
-          for _i1006 in xrange(_size1002):
-            _elem1007 = iprot.readString()
-            self.part_vals.append(_elem1007)
+          (_etype1012, _size1009) = iprot.readListBegin()
+          for _i1013 in xrange(_size1009):
+            _elem1014 = iprot.readString()
+            self.part_vals.append(_elem1014)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23327,8 +23327,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 iter1008 in self.part_vals:
-        oprot.writeString(iter1008)
+      for iter1015 in self.part_vals:
+        oprot.writeString(iter1015)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -23923,10 +23923,10 @@ class drop_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1012, _size1009) = iprot.readListBegin()
-          for _i1013 in xrange(_size1009):
-            _elem1014 = iprot.readString()
-            self.part_vals.append(_elem1014)
+          (_etype1019, _size1016) = iprot.readListBegin()
+          for _i1020 in xrange(_size1016):
+            _elem1021 = iprot.readString()
+            self.part_vals.append(_elem1021)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23956,8 +23956,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 iter1015 in self.part_vals:
-        oprot.writeString(iter1015)
+      for iter1022 in self.part_vals:
+        oprot.writeString(iter1022)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -24130,10 +24130,10 @@ class drop_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1019, _size1016) = iprot.readListBegin()
-          for _i1020 in xrange(_size1016):
-            _elem1021 = iprot.readString()
-            self.part_vals.append(_elem1021)
+          (_etype1026, _size1023) = iprot.readListBegin()
+          for _i1027 in xrange(_size1023):
+            _elem1028 = iprot.readString()
+            self.part_vals.append(_elem1028)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24169,8 +24169,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 iter1022 in self.part_vals:
-        oprot.writeString(iter1022)
+      for iter1029 in self.part_vals:
+        oprot.writeString(iter1029)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -24907,10 +24907,10 @@ class get_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1026, _size1023) = iprot.readListBegin()
-          for _i1027 in xrange(_size1023):
-            _elem1028 = iprot.readString()
-            self.part_vals.append(_elem1028)
+          (_etype1033, _size1030) = iprot.readListBegin()
+          for _i1034 in xrange(_size1030):
+            _elem1035 = iprot.readString()
+            self.part_vals.append(_elem1035)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24935,8 +24935,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 iter1029 in self.part_vals:
-        oprot.writeString(iter1029)
+      for iter1036 in self.part_vals:
+        oprot.writeString(iter1036)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -25095,11 +25095,11 @@ class exchange_partition_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype1031, _vtype1032, _size1030 ) = iprot.readMapBegin()
-          for _i1034 in xrange(_size1030):
-            _key1035 = iprot.readString()
-            _val1036 = iprot.readString()
-            self.partitionSpecs[_key1035] = _val1036
+          (_ktype1038, _vtype1039, _size1037 ) = iprot.readMapBegin()
+          for _i1041 in xrange(_size1037):
+            _key1042 = iprot.readString()
+            _val1043 = iprot.readString()
+            self.partitionSpecs[_key1042] = _val1043
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -25136,9 +25136,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 kiter1037,viter1038 in self.partitionSpecs.items():
-        oprot.writeString(kiter1037)
-        oprot.writeString(viter1038)
+      for kiter1044,viter1045 in self.partitionSpecs.items():
+        oprot.writeString(kiter1044)
+        oprot.writeString(viter1045)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -25343,11 +25343,11 @@ class exchange_partitions_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype1040, _vtype1041, _size1039 ) = iprot.readMapBegin()
-          for _i1043 in xrange(_size1039):
-            _key1044 = iprot.readString()
-            _val1045 = iprot.readString()
-            self.partitionSpecs[_key1044] = _val1045
+          (_ktype1047, _vtype1048, _size1046 ) = iprot.readMapBegin()
+          for _i1050 in xrange(_size1046):
+            _key1051 = iprot.readString()
+            _val1052 = iprot.readString()
+            self.partitionSpecs[_key1051] = _val1052
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -25384,9 +25384,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 kiter1046,viter1047 in self.partitionSpecs.items():
-        oprot.writeString(kiter1046)
-        oprot.writeString(viter1047)
+      for kiter1053,viter1054 in self.partitionSpecs.items():
+        oprot.writeString(kiter1053)
+        oprot.writeString(viter1054)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -25469,11 +25469,11 @@ class exchange_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1051, _size1048) = iprot.readListBegin()
-          for _i1052 in xrange(_size1048):
-            _elem1053 = Partition()
-            _elem1053.read(iprot)
-            self.success.append(_elem1053)
+          (_etype1058, _size1055) = iprot.readListBegin()
+          for _i1059 in xrange(_size1055):
+            _elem1060 = Partition()
+            _elem1060.read(iprot)
+            self.success.append(_elem1060)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25514,8 +25514,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 iter1054 in self.success:
-        iter1054.write(oprot)
+      for iter1061 in self.success:
+        iter1061.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -25609,10 +25609,10 @@ class get_partition_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1058, _size1055) = iprot.readListBegin()
-          for _i1059 in xrange(_size1055):
-            _elem1060 = iprot.readString()
-            self.part_vals.append(_elem1060)
+          (_etype1065, _size1062) = iprot.readListBegin()
+          for _i1066 in xrange(_size1062):
+            _elem1067 = iprot.readString()
+            self.part_vals.append(_elem1067)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25624,10 +25624,10 @@ class get_partition_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1064, _size1061) = iprot.readListBegin()
-          for _i1065 in xrange(_size1061):
-            _elem1066 = iprot.readString()
-            self.group_names.append(_elem1066)
+          (_etype1071, _size1068) = iprot.readListBegin()
+          for _i1072 in xrange(_size1068):
+            _elem1073 = iprot.readString()
+            self.group_names.append(_elem1073)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25652,8 +25652,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 iter1067 in self.part_vals:
-        oprot.writeString(iter1067)
+      for iter1074 in self.part_vals:
+        oprot.writeString(iter1074)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.user_name is not None:
@@ -25663,8 +25663,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 iter1068 in self.group_names:
-        oprot.writeString(iter1068)
+      for iter1075 in self.group_names:
+        oprot.writeString(iter1075)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -26093,11 +26093,11 @@ class get_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1072, _size1069) = iprot.readListBegin()
-          for _i1073 in xrange(_size1069):
-            _elem1074 = Partition()
-            _elem1074.read(iprot)
-            self.success.append(_elem1074)
+          (_etype1079, _size1076) = iprot.readListBegin()
+          for _i1080 in xrange(_size1076):
+            _elem1081 = Partition()
+            _elem1081.read(iprot)
+            self.success.append(_elem1081)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26126,8 +26126,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 iter1075 in self.success:
-        iter1075.write(oprot)
+      for iter1082 in self.success:
+        iter1082.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26221,10 +26221,10 @@ class get_partitions_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1079, _size1076) = iprot.readListBegin()
-          for _i1080 in xrange(_size1076):
-            _elem1081 = iprot.readString()
-            self.group_names.append(_elem1081)
+          (_etype1086, _size1083) = iprot.readListBegin()
+          for _i1087 in xrange(_size1083):
+            _elem1088 = iprot.readString()
+            self.group_names.append(_elem1088)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26257,8 +26257,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 iter1082 in self.group_names:
-        oprot.writeString(iter1082)
+      for iter1089 in self.group_names:
+        oprot.writeString(iter1089)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -26319,11 +26319,11 @@ class get_partitions_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1086, _size1083) = iprot.readListBegin()
-          for _i1087 in xrange(_size1083):
-            _elem1088 = Partition()
-            _elem1088.read(iprot)
-            self.success.append(_elem1088)
+          (_etype1093, _size1090) = iprot.readListBegin()
+          for _i1094 in xrange(_size1090):
+            _elem1095 = Partition()
+            _elem1095.read(iprot)
+            self.success.append(_elem1095)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26352,8 +26352,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 iter1089 in self.success:
-        iter1089.write(oprot)
+      for iter1096 in self.success:
+        iter1096.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26511,11 +26511,11 @@ class get_partitions_pspec_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1093, _size1090) = iprot.readListBegin()
-          for _i1094 in xrange(_size1090):
-            _elem1095 = PartitionSpec()
-            _elem1095.read(iprot)
-            self.success.append(_elem1095)
+          (_etype1100, _size1097) = iprot.readListBegin()
+          for _i1101 in xrange(_size1097):
+            _elem1102 = PartitionSpec()
+            _elem1102.read(iprot)
+            self.success.append(_elem1102)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26544,8 +26544,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 iter1096 in self.success:
-        iter1096.write(oprot)
+      for iter1103 in self.success:
+        iter1103.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26703,10 +26703,10 @@ class get_partition_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1100, _size1097) = iprot.readListBegin()
-          for _i1101 in xrange(_size1097):
-            _elem1102 = iprot.readString()
-            self.success.append(_elem1102)
+          (_etype1107, _size1104) = iprot.readListBegin()
+          for _i1108 in xrange(_size1104):
+            _elem1109 = iprot.readString()
+            self.success.append(_elem1109)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26735,8 +26735,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 iter1103 in self.success:
-        oprot.writeString(iter1103)
+      for iter1110 in self.success:
+        oprot.writeString(iter1110)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26976,10 +26976,10 @@ class get_partitions_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1107, _size1104) = iprot.readListBegin()
-          for _i1108 in xrange(_size1104):
-            _elem1109 = iprot.readString()
-            self.part_vals.append(_elem1109)
+          (_etype1114, _size1111) = iprot.readListBegin()
+          for _i1115 in xrange(_size1111):
+            _elem1116 = iprot.readString()
+            self.part_vals.append(_elem1116)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27009,8 +27009,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 iter1110 in self.part_vals:
-        oprot.writeString(iter1110)
+      for iter1117 in self.part_vals:
+        oprot.writeString(iter1117)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -27074,11 +27074,11 @@ class get_partitions_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1114, _size1111) = iprot.readListBegin()
-          for _i1115 in xrange(_size1111):
-            _elem1116 = Partition()
-            _elem1116.read(iprot)
-            self.success.append(_elem1116)
+          (_etype1121, _size1118) = iprot.readListBegin()
+          for _i1122 in xrange(_size1118):
+            _elem1123 = Partition()
+            _elem1123.read(iprot)
+            self.success.append(_elem1123)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27107,8 +27107,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 iter1117 in self.success:
-        iter1117.write(oprot)
+      for iter1124 in self.success:
+        iter1124.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27195,10 +27195,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1121, _size1118) = iprot.readListBegin()
-          for _i1122 in xrange(_size1118):
-            _elem1123 = iprot.readString()
-            self.part_vals.append(_elem1123)
+          (_etype1128, _size1125) = iprot.readListBegin()
+          for _i1129 in xrange(_size1125):
+            _elem1130 = iprot.readString()
+            self.part_vals.append(_elem1130)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27215,10 +27215,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 6:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1127, _size1124) = iprot.readListBegin()
-          for _i1128 in xrange(_size1124):
-            _elem1129 = iprot.readString()
-            self.group_names.append(_elem1129)
+          (_etype1134, _size1131) = iprot.readListBegin()
+          for _i1135 in xrange(_size1131):
+            _elem1136 = iprot.readString()
+            self.group_names.append(_elem1136)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27243,8 +27243,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 iter1130 in self.part_vals:
-        oprot.writeString(iter1130)
+      for iter1137 in self.part_vals:
+        oprot.writeString(iter1137)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -27258,8 +27258,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 iter1131 in self.group_names:
-        oprot.writeString(iter1131)
+      for iter1138 in self.group_names:
+        oprot.writeString(iter1138)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -27321,11 +27321,11 @@ class get_partitions_ps_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1135, _size1132) = iprot.readListBegin()
-          for _i1136 in xrange(_size1132):
-            _elem1137 = Partition()
-            _elem1137.read(iprot)
-            self.success.append(_elem1137)
+          (_etype1142, _size1139) = iprot.readListBegin()
+          for _i1143 in xrange(_size1139):
+            _elem1144 = Partition()
+            _elem1144.read(iprot)
+            self.success.append(_elem1144)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27354,8 +27354,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 iter1138 in self.success:
-        iter1138.write(oprot)
+      for iter1145 in self.success:
+        iter1145.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27436,10 +27436,10 @@ class get_partition_names_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1142, _size1139) = iprot.readListBegin()
-          for _i1143 in xrange(_size1139):
-            _elem1144 = iprot.readString()
-            self.part_vals.append(_elem1144)
+          (_etype1149, _size1146) = iprot.readListBegin()
+          for _i1150 in xrange(_size1146):
+            _elem1151 = iprot.readString()
+            self.part_vals.append(_elem1151)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27469,8 +27469,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 iter1145 in self.part_vals:
-        oprot.writeString(iter1145)
+      for iter1152 in self.part_vals:
+        oprot.writeString(iter1152)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -27534,10 +27534,10 @@ class get_partition_names_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1149, _size1146) = iprot.readListBegin()
-          for _i1150 in xrange(_size1146):
-            _elem1151 = iprot.readString()
-            self.success.append(_elem1151)
+          (_etype1156, _size1153) = iprot.readListBegin()
+          for _i1157 in xrange(_size1153):
+            _elem1158 = iprot.readString()
+            self.success.append(_elem1158)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27566,8 +27566,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 iter1152 in self.success:
-        oprot.writeString(iter1152)
+      for iter1159 in self.success:
+        oprot.writeString(iter1159)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27738,11 +27738,11 @@ class get_partitions_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1156, _size1153) = iprot.readListBegin()
-          for _i1157 in xrange(_size1153):
-            _elem1158 = Partition()
-            _elem1158.read(iprot)
-            self.success.append(_elem1158)
+          (_etype1163, _size1160) = iprot.readListBegin()
+          for _i1164 in xrange(_size1160):
+            _elem1165 = Partition()
+            _elem1165.read(iprot)
+            self.success.append(_elem1165)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27771,8 +27771,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 iter1159 in self.success:
-        iter1159.write(oprot)
+      for iter1166 in self.success:
+        iter1166.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27943,11 +27943,11 @@ class get_part_specs_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1163, _size1160) = iprot.readListBegin()
-          for _i1164 in xrange(_size1160):
-            _elem1165 = PartitionSpec()
-            _elem1165.read(iprot)
-            self.success.append(_elem1165)
+          (_etype1170, _size1167) = iprot.readListBegin()
+          for _i1171 in xrange(_size1167):
+            _elem1172 = PartitionSpec()
+            _elem1172.read(iprot)
+            self.success.append(_elem1172)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27976,8 +27976,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 iter1166 in self.success:
-        iter1166.write(oprot)
+      for iter1173 in self.success:
+        iter1173.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -28397,10 +28397,10 @@ class get_partitions_by_names_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.names = []
-          (_etype1170, _size1167) = iprot.readListBegin()
-          for _i1171 in xrange(_size1167):
-            _elem1172 = iprot.readString()
-            self.names.append(_elem1172)
+          (_etype1177, _size1174) = iprot.readListBegin()
+          for _i1178 in xrange(_size1174):
+            _elem1179 = iprot.readString()
+            self.names.append(_elem1179)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28425,8 +28425,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 iter1173 in self.names:
-        oprot.writeString(iter1173)
+      for iter1180 in self.names:
+        oprot.writeString(iter1180)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -28485,11 +28485,11 @@ class get_partitions_by_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1177, _size1174) = iprot.readListBegin()
-          for _i1178 in xrange(_size1174):
-            _elem1179 = Partition()
-            _elem1179.read(iprot)
-            self.success.append(_elem1179)
+          (_etype1184, _size1181) = iprot.readListBegin()
+          for _i1185 in xrange(_size1181):
+            _elem1186 = Partition()
+            _elem1186.read(iprot)
+            self.success.append(_elem1186)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28518,8 +28518,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 iter1180 in self.success:
-        iter1180.write(oprot)
+      for iter1187 in self.success:
+        iter1187.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -28769,11 +28769,11 @@ class alter_partitions_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1184, _size1181) = iprot.readListBegin()
-          for _i1185 in xrange(_size1181):
-            _elem1186 = Partition()
-            _elem1186.read(iprot)
-            self.new_parts.append(_elem1186)
+          (_etype1191, _size1188) = iprot.readListBegin()
+          for _i1192 in xrange(_size1188):
+            _elem1193 = Partition()
+            _elem1193.read(iprot)
+            self.new_parts.append(_elem1193)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28798,8 +28798,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 iter1187 in self.new_parts:
-        iter1187.write(oprot)
+      for iter1194 in self.new_parts:
+        iter1194.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -28952,11 +28952,11 @@ class alter_partitions_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1191, _size1188) = iprot.readListBegin()
-          for _i1192 in xrange(_size1188):
-            _elem1193 = Partition()
-            _elem1193.read(iprot)
-            self.new_parts.append(_elem1193)
+          (_etype1198, _size1195) = iprot.readListBegin()
+          for _i1199 in xrange(_size1195):
+            _elem1200 = Partition()
+            _elem1200.read(iprot)
+            self.new_parts.append(_elem1200)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28987,8 +28987,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 iter1194 in self.new_parts:
-        iter1194.write(oprot)
+      for iter1201 in self.new_parts:
+        iter1201.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -29332,10 +29332,10 @@ class rename_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1198, _size1195) = iprot.readListBegin()
-          for _i1199 in xrange(_size1195):
-            _elem1200 = iprot.readString()
-            self.part_vals.append(_elem1200)
+          (_etype1205, _size1202) = iprot.readListBegin()
+          for _i1206 in xrange(_size1202):
+            _elem1207 = iprot.readString()
+            self.part_vals.append(_elem1207)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29366,8 +29366,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 iter1201 in self.part_vals:
-        oprot.writeString(iter1201)
+      for iter1208 in self.part_vals:
+        oprot.writeString(iter1208)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.new_part is not None:
@@ -29509,10 +29509,10 @@ class partition_name_has_valid_characters_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1205, _size1202) = iprot.readListBegin()
-          for _i1206 in xrange(_size1202):
-            _elem1207 = iprot.readString()
-            self.part_vals.append(_elem1207)
+          (_etype1212, _size1209) = iprot.readListBegin()
+          for _i1213 in xrange(_size1209):
+            _elem1214 = iprot.readString()
+            self.part_vals.append(_elem1214)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29534,8 +29534,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 iter1208 in self.part_vals:
-        oprot.writeString(iter1208)
+      for iter1215 in self.part_vals:
+        oprot.writeString(iter1215)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.throw_exception is not None:
@@ -29893,10 +29893,10 @@ class partition_name_to_vals_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1212, _size1209) = iprot.readListBegin()
-          for _i1213 in xrange(_size1209):
-            _elem1214 = iprot.readString()
-            self.success.append(_elem1214)
+          (_etype1219, _size1216) = iprot.readListBegin()
+          for _i1220 in xrange(_size1216):
+            _elem1221 = iprot.readString()
+            self.success.append(_elem1221)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29919,8 +29919,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 iter1215 in self.success:
-        oprot.writeString(iter1215)
+      for iter1222 in self.success:
+        oprot.writeString(iter1222)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -30044,11 +30044,11 @@ class partition_name_to_spec_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype1217, _vtype1218, _size1216 ) = iprot.readMapBegin()
-          for _i1220 in xrange(_size1216):
-            _key1221 = iprot.readString()
-            _val1222 = iprot.readString()
-            self.success[_key1221] = _val1222
+          (_ktype1224, _vtype1225, _size1223 ) = iprot.readMapBegin()
+          for _i1227 in xrange(_size1223):
+            _key1228 = iprot.readString()
+            _val1229 = iprot.readString()
+            self.success[_key1228] = _val1229
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -30071,9 +30071,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 kiter1223,viter1224 in self.success.items():
-        oprot.writeString(kiter1223)
-        oprot.writeString(viter1224)
+      for kiter1230,viter1231 in self.success.items():
+        oprot.writeString(kiter1230)
+        oprot.writeString(viter1231)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -30149,11 +30149,11 @@ class markPartitionForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype1226, _vtype1227, _size1225 ) = iprot.readMapBegin()
-          for _i1229 in xrange(_size1225):
-            _key1230 = iprot.readString()
-            _val1231 = iprot.readString()
-            self.part_vals[_key1230] = _val1231
+          (_ktype1233, _vtype1234, _size1232 ) = iprot.readMapBegin()
+          for _i1236 in xrange(_size1232):
+            _key1237 = iprot.readString()
+            _val1238 = iprot.readString()
+            self.part_vals[_key1237] = _val1238
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -30183,9 +30183,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 kiter1232,viter1233 in self.part_vals.items():
-        oprot.writeString(kiter1232)
-        oprot.writeString(viter1233)
+      for kiter1239,viter1240 in self.part_vals.items():
+        oprot.writeString(kiter1239)
+        oprot.writeString(viter1240)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -30399,11 +30399,11 @@ class isPartitionMarkedForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype1235, _vtype1236, _size1234 ) = iprot.readMapBegin()
-          for _i1238 in xrange(_size1234):
-            _key1239 = iprot.readString()
-            _val1240 = iprot.readString()
-            self.part_vals[_key1239] = _val1240
+          (_ktype1242, _vtype1243, _size1241 ) = iprot.readMapBegin()
+          for _i1245 in xrange(_size1241):
+            _key1246 = iprot.readString()
+            _val1247 = iprot.readString()
+            self.part_vals[_key1246] = _val1247
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -30433,9 +30433,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 kiter1241,viter1242 in self.part_vals.items():
-        oprot.writeString(kiter1241)
-        oprot.writeString(viter1242)
+      for kiter1248,viter1249 in self.part_vals.items():
+        oprot.writeString(kiter1248)
+        oprot.writeString(viter1249)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -34087,10 +34087,10 @@ class get_functions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1246, _size1243) = iprot.readListBegin()
-          for _i1247 in xrange(_size1243):
-            _elem1248 = iprot.readString()
-            self.success.append(_elem1248)
+          (_etype1253, _size1250) = iprot.readListBegin()
+          for _i1254 in xrange(_size1250):
+            _elem1255 = iprot.readString()
+            self.success.append(_elem1255)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -34113,8 +34113,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 iter1249 in self.success:
-        oprot.writeString(iter1249)
+      for iter1256 in self.success:
+        oprot.writeString(iter1256)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -34802,10 +34802,10 @@ class get_role_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1253, _size1250) = iprot.readListBegin()
-          for _i1254 in xrange(_size1250):
-            _elem1255 = iprot.readString()
-            self.success.append(_elem1255)
+          (_etype1260, _size1257) = iprot.readListBegin()
+          for _i1261 in xrange(_size1257):
+            _elem1262 = iprot.readString()
+            self.success.append(_elem1262)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -34828,8 +34828,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 iter1256 in self.success:
-        oprot.writeString(iter1256)
+      for iter1263 in self.success:
+        oprot.writeString(iter1263)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -35343,11 +35343,11 @@ class list_roles_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1260, _size1257) = iprot.readListBegin()
-          for _i1261 in xrange(_size1257):
-            _elem1262 = Role()
-            _elem1262.read(iprot)
-            self.success.append(_elem1262)
+          (_etype1267, _size1264) = iprot.readListBegin()
+          for _i1268 in xrange(_size1264):
+            _elem1269 = Role()
+            _elem1269.read(iprot)
+            self.success.append(_elem1269)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -35370,8 +35370,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 iter1263 in self.success:
-        iter1263.write(oprot)
+      for iter1270 in self.success:
+        iter1270.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -35880,10 +35880,10 @@ class get_privilege_set_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1267, _size1264) = iprot.readListBegin()
-          for _i1268 in xrange(_size1264):
-            _elem1269 = iprot.readString()
-            self.group_names.append(_elem1269)
+          (_etype1274, _size1271) = iprot.readListBegin()
+          for _i1275 in xrange(_size1271):
+            _elem1276 = iprot.readString()
+            self.group_names.append(_elem1276)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -35908,8 +35908,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 iter1270 in self.group_names:
-        oprot.writeString(iter1270)
+      for iter1277 in self.group_names:
+        oprot.writeString(iter1277)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -36136,11 +36136,11 @@ class list_privileges_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1274, _size1271) = iprot.readListBegin()
-          for _i1275 in xrange(_size1271):
-            _elem1276 = HiveObjectPrivilege()
-            _elem1276.read(iprot)
-            self.success.append(_elem1276)
+          (_etype1281, _size1278) = iprot.readListBegin()
+          for _i1282 in xrange(_size1278):
+            _elem1283 = HiveObjectPrivilege()
+            _elem1283.read(iprot)
+            self.success.append(_elem1283)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -36163,8 +36163,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 iter1277 in self.success:
-        iter1277.write(oprot)
+      for iter1284 in self.success:
+        iter1284.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -36662,10 +36662,10 @@ class set_ugi_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1281, _size1278) = iprot.readListBegin()
-          for _i1282 in xrange(_size1278):
-            _elem1283 = iprot.readString()
-            self.group_names.append(_elem1283)
+          (_etype1288, _size1285) = iprot.readListBegin()
+          for _i1289 in xrange(_size1285):
+            _elem1290 = iprot.readString()
+            self.group_names.append(_elem1290)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -36686,8 +36686,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 iter1284 in self.group_names:
-        oprot.writeString(iter1284)
+      for iter1291 in self.group_names:
+        oprot.writeString(iter1291)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -36742,10 +36742,10 @@ class set_ugi_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1288, _size1285) = iprot.readListBegin()
-          for _i1289 in xrange(_size1285):
-            _elem1290 = iprot.readString()
-            self.success.append(_elem1290)
+          (_etype1295, _size1292) = iprot.readListBegin()
+          for _i1296 in xrange(_size1292):
+            _elem1297 = iprot.readString()
+            self.success.append(_elem1297)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -36768,8 +36768,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 iter1291 in self.success:
-        oprot.writeString(iter1291)
+      for iter1298 in self.success:
+        oprot.writeString(iter1298)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -37701,10 +37701,10 @@ class get_all_token_identifiers_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1295, _size1292) = iprot.readListBegin()
-          for _i1296 in xrange(_size1292):
-            _elem1297 = iprot.readString()
-            self.success.append(_elem1297)
+          (_etype1302, _size1299) = iprot.readListBegin()
+          for _i1303 in xrange(_size1299):
+            _elem1304 = iprot.readString()
+            self.success.append(_elem1304)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -37721,8 +37721,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 iter1298 in self.success:
-        oprot.writeString(iter1298)
+      for iter1305 in self.success:
+        oprot.writeString(iter1305)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -38249,10 +38249,10 @@ class get_master_keys_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1302, _size1299) = iprot.readListBegin()
-          for _i1303 in xrange(_size1299):
-            _elem1304 = iprot.readString()
-            self.success.append(_elem1304)
+          (_etype1309, _size1306) = iprot.readListBegin()
+          for _i1310 in xrange(_size1306):
+            _elem1311 = iprot.readString()
+            self.success.append(_elem1311)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -38269,8 +38269,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 iter1305 in self.success:
-        oprot.writeString(iter1305)
+      for iter1312 in self.success:
+        oprot.writeString(iter1312)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -46438,11 +46438,11 @@ class get_schema_all_versions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1309, _size1306) = iprot.readListBegin()
-          for _i1310 in xrange(_size1306):
-            _elem1311 = SchemaVersion()
-            _elem1311.read(iprot)
-            self.success.append(_elem1311)
+          (_etype1316, _size1313) = iprot.readListBegin()
+          for _i1317 in xrange(_size1313):
+            _elem1318 = SchemaVersion()
+            _elem1318.read(iprot)
+            self.success.append(_elem1318)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -46471,8 +46471,8 @@ class get_schema_all_versions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1312 in self.success:
-        iter1312.write(oprot)
+      for iter1319 in self.success:
+        iter1319.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:


[02/12] hive git commit: HIVE-19089: Create/Replicate Allocate write-id event (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 f2f61e0..9bf9843 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
@@ -11834,22 +11834,28 @@ class GetValidWriteIdsResponse:
 class AllocateTableWriteIdsRequest:
   """
   Attributes:
-   - txnIds
    - dbName
    - tableName
+   - txnIds
+   - replPolicy
+   - srcTxnToWriteIdList
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.LIST, 'txnIds', (TType.I64,None), None, ), # 1
-    (2, TType.STRING, 'dbName', None, None, ), # 2
-    (3, TType.STRING, 'tableName', None, None, ), # 3
+    (1, TType.STRING, 'dbName', None, None, ), # 1
+    (2, TType.STRING, 'tableName', None, None, ), # 2
+    (3, TType.LIST, 'txnIds', (TType.I64,None), None, ), # 3
+    (4, TType.STRING, 'replPolicy', None, None, ), # 4
+    (5, TType.LIST, 'srcTxnToWriteIdList', (TType.STRUCT,(TxnToWriteId, TxnToWriteId.thrift_spec)), None, ), # 5
   )
 
-  def __init__(self, txnIds=None, dbName=None, tableName=None,):
-    self.txnIds = txnIds
+  def __init__(self, dbName=None, tableName=None, txnIds=None, replPolicy=None, srcTxnToWriteIdList=None,):
     self.dbName = dbName
     self.tableName = tableName
+    self.txnIds = txnIds
+    self.replPolicy = replPolicy
+    self.srcTxnToWriteIdList = srcTxnToWriteIdList
 
   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:
@@ -11861,6 +11867,16 @@ class AllocateTableWriteIdsRequest:
       if ftype == TType.STOP:
         break
       if fid == 1:
+        if ftype == TType.STRING:
+          self.dbName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.tableName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
         if ftype == TType.LIST:
           self.txnIds = []
           (_etype547, _size544) = iprot.readListBegin()
@@ -11870,14 +11886,20 @@ class AllocateTableWriteIdsRequest:
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
-      elif fid == 2:
+      elif fid == 4:
         if ftype == TType.STRING:
-          self.dbName = iprot.readString()
+          self.replPolicy = iprot.readString()
         else:
           iprot.skip(ftype)
-      elif fid == 3:
-        if ftype == TType.STRING:
-          self.tableName = iprot.readString()
+      elif fid == 5:
+        if ftype == TType.LIST:
+          self.srcTxnToWriteIdList = []
+          (_etype553, _size550) = iprot.readListBegin()
+          for _i554 in xrange(_size550):
+            _elem555 = TxnToWriteId()
+            _elem555.read(iprot)
+            self.srcTxnToWriteIdList.append(_elem555)
+          iprot.readListEnd()
         else:
           iprot.skip(ftype)
       else:
@@ -11890,27 +11912,36 @@ class AllocateTableWriteIdsRequest:
       oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
       return
     oprot.writeStructBegin('AllocateTableWriteIdsRequest')
-    if self.txnIds is not None:
-      oprot.writeFieldBegin('txnIds', TType.LIST, 1)
-      oprot.writeListBegin(TType.I64, len(self.txnIds))
-      for iter550 in self.txnIds:
-        oprot.writeI64(iter550)
-      oprot.writeListEnd()
-      oprot.writeFieldEnd()
     if self.dbName is not None:
-      oprot.writeFieldBegin('dbName', TType.STRING, 2)
+      oprot.writeFieldBegin('dbName', TType.STRING, 1)
       oprot.writeString(self.dbName)
       oprot.writeFieldEnd()
     if self.tableName is not None:
-      oprot.writeFieldBegin('tableName', TType.STRING, 3)
+      oprot.writeFieldBegin('tableName', TType.STRING, 2)
       oprot.writeString(self.tableName)
       oprot.writeFieldEnd()
+    if self.txnIds is not None:
+      oprot.writeFieldBegin('txnIds', TType.LIST, 3)
+      oprot.writeListBegin(TType.I64, len(self.txnIds))
+      for iter556 in self.txnIds:
+        oprot.writeI64(iter556)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.replPolicy is not None:
+      oprot.writeFieldBegin('replPolicy', TType.STRING, 4)
+      oprot.writeString(self.replPolicy)
+      oprot.writeFieldEnd()
+    if self.srcTxnToWriteIdList is not None:
+      oprot.writeFieldBegin('srcTxnToWriteIdList', TType.LIST, 5)
+      oprot.writeListBegin(TType.STRUCT, len(self.srcTxnToWriteIdList))
+      for iter557 in self.srcTxnToWriteIdList:
+        iter557.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
   def validate(self):
-    if self.txnIds is None:
-      raise TProtocol.TProtocolException(message='Required field txnIds is unset!')
     if self.dbName is None:
       raise TProtocol.TProtocolException(message='Required field dbName is unset!')
     if self.tableName is None:
@@ -11920,9 +11951,11 @@ class AllocateTableWriteIdsRequest:
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.txnIds)
     value = (value * 31) ^ hash(self.dbName)
     value = (value * 31) ^ hash(self.tableName)
+    value = (value * 31) ^ hash(self.txnIds)
+    value = (value * 31) ^ hash(self.replPolicy)
+    value = (value * 31) ^ hash(self.srcTxnToWriteIdList)
     return value
 
   def __repr__(self):
@@ -12044,11 +12077,11 @@ class AllocateTableWriteIdsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.txnToWriteIds = []
-          (_etype554, _size551) = iprot.readListBegin()
-          for _i555 in xrange(_size551):
-            _elem556 = TxnToWriteId()
-            _elem556.read(iprot)
-            self.txnToWriteIds.append(_elem556)
+          (_etype561, _size558) = iprot.readListBegin()
+          for _i562 in xrange(_size558):
+            _elem563 = TxnToWriteId()
+            _elem563.read(iprot)
+            self.txnToWriteIds.append(_elem563)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12065,8 +12098,8 @@ class AllocateTableWriteIdsResponse:
     if self.txnToWriteIds is not None:
       oprot.writeFieldBegin('txnToWriteIds', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.txnToWriteIds))
-      for iter557 in self.txnToWriteIds:
-        iter557.write(oprot)
+      for iter564 in self.txnToWriteIds:
+        iter564.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -12294,11 +12327,11 @@ class LockRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.component = []
-          (_etype561, _size558) = iprot.readListBegin()
-          for _i562 in xrange(_size558):
-            _elem563 = LockComponent()
-            _elem563.read(iprot)
-            self.component.append(_elem563)
+          (_etype568, _size565) = iprot.readListBegin()
+          for _i569 in xrange(_size565):
+            _elem570 = LockComponent()
+            _elem570.read(iprot)
+            self.component.append(_elem570)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12335,8 +12368,8 @@ class LockRequest:
     if self.component is not None:
       oprot.writeFieldBegin('component', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.component))
-      for iter564 in self.component:
-        iter564.write(oprot)
+      for iter571 in self.component:
+        iter571.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.txnid is not None:
@@ -13034,11 +13067,11 @@ class ShowLocksResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.locks = []
-          (_etype568, _size565) = iprot.readListBegin()
-          for _i569 in xrange(_size565):
-            _elem570 = ShowLocksResponseElement()
-            _elem570.read(iprot)
-            self.locks.append(_elem570)
+          (_etype575, _size572) = iprot.readListBegin()
+          for _i576 in xrange(_size572):
+            _elem577 = ShowLocksResponseElement()
+            _elem577.read(iprot)
+            self.locks.append(_elem577)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13055,8 +13088,8 @@ class ShowLocksResponse:
     if self.locks is not None:
       oprot.writeFieldBegin('locks', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.locks))
-      for iter571 in self.locks:
-        iter571.write(oprot)
+      for iter578 in self.locks:
+        iter578.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -13271,20 +13304,20 @@ class HeartbeatTxnRangeResponse:
       if fid == 1:
         if ftype == TType.SET:
           self.aborted = set()
-          (_etype575, _size572) = iprot.readSetBegin()
-          for _i576 in xrange(_size572):
-            _elem577 = iprot.readI64()
-            self.aborted.add(_elem577)
+          (_etype582, _size579) = iprot.readSetBegin()
+          for _i583 in xrange(_size579):
+            _elem584 = iprot.readI64()
+            self.aborted.add(_elem584)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.SET:
           self.nosuch = set()
-          (_etype581, _size578) = iprot.readSetBegin()
-          for _i582 in xrange(_size578):
-            _elem583 = iprot.readI64()
-            self.nosuch.add(_elem583)
+          (_etype588, _size585) = iprot.readSetBegin()
+          for _i589 in xrange(_size585):
+            _elem590 = iprot.readI64()
+            self.nosuch.add(_elem590)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
@@ -13301,15 +13334,15 @@ class HeartbeatTxnRangeResponse:
     if self.aborted is not None:
       oprot.writeFieldBegin('aborted', TType.SET, 1)
       oprot.writeSetBegin(TType.I64, len(self.aborted))
-      for iter584 in self.aborted:
-        oprot.writeI64(iter584)
+      for iter591 in self.aborted:
+        oprot.writeI64(iter591)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     if self.nosuch is not None:
       oprot.writeFieldBegin('nosuch', TType.SET, 2)
       oprot.writeSetBegin(TType.I64, len(self.nosuch))
-      for iter585 in self.nosuch:
-        oprot.writeI64(iter585)
+      for iter592 in self.nosuch:
+        oprot.writeI64(iter592)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -13406,11 +13439,11 @@ class CompactionRequest:
       elif fid == 6:
         if ftype == TType.MAP:
           self.properties = {}
-          (_ktype587, _vtype588, _size586 ) = iprot.readMapBegin()
-          for _i590 in xrange(_size586):
-            _key591 = iprot.readString()
-            _val592 = iprot.readString()
-            self.properties[_key591] = _val592
+          (_ktype594, _vtype595, _size593 ) = iprot.readMapBegin()
+          for _i597 in xrange(_size593):
+            _key598 = iprot.readString()
+            _val599 = iprot.readString()
+            self.properties[_key598] = _val599
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -13447,9 +13480,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 kiter593,viter594 in self.properties.items():
-        oprot.writeString(kiter593)
-        oprot.writeString(viter594)
+      for kiter600,viter601 in self.properties.items():
+        oprot.writeString(kiter600)
+        oprot.writeString(viter601)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -13884,11 +13917,11 @@ class ShowCompactResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.compacts = []
-          (_etype598, _size595) = iprot.readListBegin()
-          for _i599 in xrange(_size595):
-            _elem600 = ShowCompactResponseElement()
-            _elem600.read(iprot)
-            self.compacts.append(_elem600)
+          (_etype605, _size602) = iprot.readListBegin()
+          for _i606 in xrange(_size602):
+            _elem607 = ShowCompactResponseElement()
+            _elem607.read(iprot)
+            self.compacts.append(_elem607)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13905,8 +13938,8 @@ class ShowCompactResponse:
     if self.compacts is not None:
       oprot.writeFieldBegin('compacts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.compacts))
-      for iter601 in self.compacts:
-        iter601.write(oprot)
+      for iter608 in self.compacts:
+        iter608.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -13995,10 +14028,10 @@ class AddDynamicPartitions:
       elif fid == 5:
         if ftype == TType.LIST:
           self.partitionnames = []
-          (_etype605, _size602) = iprot.readListBegin()
-          for _i606 in xrange(_size602):
-            _elem607 = iprot.readString()
-            self.partitionnames.append(_elem607)
+          (_etype612, _size609) = iprot.readListBegin()
+          for _i613 in xrange(_size609):
+            _elem614 = iprot.readString()
+            self.partitionnames.append(_elem614)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14036,8 +14069,8 @@ class AddDynamicPartitions:
     if self.partitionnames is not None:
       oprot.writeFieldBegin('partitionnames', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.partitionnames))
-      for iter608 in self.partitionnames:
-        oprot.writeString(iter608)
+      for iter615 in self.partitionnames:
+        oprot.writeString(iter615)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.operationType is not None:
@@ -14267,10 +14300,10 @@ class CreationMetadata:
       elif fid == 4:
         if ftype == TType.SET:
           self.tablesUsed = set()
-          (_etype612, _size609) = iprot.readSetBegin()
-          for _i613 in xrange(_size609):
-            _elem614 = iprot.readString()
-            self.tablesUsed.add(_elem614)
+          (_etype619, _size616) = iprot.readSetBegin()
+          for _i620 in xrange(_size616):
+            _elem621 = iprot.readString()
+            self.tablesUsed.add(_elem621)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
@@ -14304,8 +14337,8 @@ class CreationMetadata:
     if self.tablesUsed is not None:
       oprot.writeFieldBegin('tablesUsed', TType.SET, 4)
       oprot.writeSetBegin(TType.STRING, len(self.tablesUsed))
-      for iter615 in self.tablesUsed:
-        oprot.writeString(iter615)
+      for iter622 in self.tablesUsed:
+        oprot.writeString(iter622)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     if self.validTxnList is not None:
@@ -14617,11 +14650,11 @@ class NotificationEventResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.events = []
-          (_etype619, _size616) = iprot.readListBegin()
-          for _i620 in xrange(_size616):
-            _elem621 = NotificationEvent()
-            _elem621.read(iprot)
-            self.events.append(_elem621)
+          (_etype626, _size623) = iprot.readListBegin()
+          for _i627 in xrange(_size623):
+            _elem628 = NotificationEvent()
+            _elem628.read(iprot)
+            self.events.append(_elem628)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14638,8 +14671,8 @@ class NotificationEventResponse:
     if self.events is not None:
       oprot.writeFieldBegin('events', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.events))
-      for iter622 in self.events:
-        iter622.write(oprot)
+      for iter629 in self.events:
+        iter629.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14933,20 +14966,20 @@ class InsertEventRequestData:
       elif fid == 2:
         if ftype == TType.LIST:
           self.filesAdded = []
-          (_etype626, _size623) = iprot.readListBegin()
-          for _i627 in xrange(_size623):
-            _elem628 = iprot.readString()
-            self.filesAdded.append(_elem628)
+          (_etype633, _size630) = iprot.readListBegin()
+          for _i634 in xrange(_size630):
+            _elem635 = iprot.readString()
+            self.filesAdded.append(_elem635)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.filesAddedChecksum = []
-          (_etype632, _size629) = iprot.readListBegin()
-          for _i633 in xrange(_size629):
-            _elem634 = iprot.readString()
-            self.filesAddedChecksum.append(_elem634)
+          (_etype639, _size636) = iprot.readListBegin()
+          for _i640 in xrange(_size636):
+            _elem641 = iprot.readString()
+            self.filesAddedChecksum.append(_elem641)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14967,15 +15000,15 @@ class InsertEventRequestData:
     if self.filesAdded is not None:
       oprot.writeFieldBegin('filesAdded', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.filesAdded))
-      for iter635 in self.filesAdded:
-        oprot.writeString(iter635)
+      for iter642 in self.filesAdded:
+        oprot.writeString(iter642)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.filesAddedChecksum is not None:
       oprot.writeFieldBegin('filesAddedChecksum', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.filesAddedChecksum))
-      for iter636 in self.filesAddedChecksum:
-        oprot.writeString(iter636)
+      for iter643 in self.filesAddedChecksum:
+        oprot.writeString(iter643)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -15133,10 +15166,10 @@ class FireEventRequest:
       elif fid == 5:
         if ftype == TType.LIST:
           self.partitionVals = []
-          (_etype640, _size637) = iprot.readListBegin()
-          for _i641 in xrange(_size637):
-            _elem642 = iprot.readString()
-            self.partitionVals.append(_elem642)
+          (_etype647, _size644) = iprot.readListBegin()
+          for _i648 in xrange(_size644):
+            _elem649 = iprot.readString()
+            self.partitionVals.append(_elem649)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15174,8 +15207,8 @@ class FireEventRequest:
     if self.partitionVals is not None:
       oprot.writeFieldBegin('partitionVals', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.partitionVals))
-      for iter643 in self.partitionVals:
-        oprot.writeString(iter643)
+      for iter650 in self.partitionVals:
+        oprot.writeString(iter650)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.catName is not None:
@@ -15367,12 +15400,12 @@ class GetFileMetadataByExprResult:
       if fid == 1:
         if ftype == TType.MAP:
           self.metadata = {}
-          (_ktype645, _vtype646, _size644 ) = iprot.readMapBegin()
-          for _i648 in xrange(_size644):
-            _key649 = iprot.readI64()
-            _val650 = MetadataPpdResult()
-            _val650.read(iprot)
-            self.metadata[_key649] = _val650
+          (_ktype652, _vtype653, _size651 ) = iprot.readMapBegin()
+          for _i655 in xrange(_size651):
+            _key656 = iprot.readI64()
+            _val657 = MetadataPpdResult()
+            _val657.read(iprot)
+            self.metadata[_key656] = _val657
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -15394,9 +15427,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 kiter651,viter652 in self.metadata.items():
-        oprot.writeI64(kiter651)
-        viter652.write(oprot)
+      for kiter658,viter659 in self.metadata.items():
+        oprot.writeI64(kiter658)
+        viter659.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.isSupported is not None:
@@ -15466,10 +15499,10 @@ class GetFileMetadataByExprRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype656, _size653) = iprot.readListBegin()
-          for _i657 in xrange(_size653):
-            _elem658 = iprot.readI64()
-            self.fileIds.append(_elem658)
+          (_etype663, _size660) = iprot.readListBegin()
+          for _i664 in xrange(_size660):
+            _elem665 = iprot.readI64()
+            self.fileIds.append(_elem665)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15501,8 +15534,8 @@ class GetFileMetadataByExprRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter659 in self.fileIds:
-        oprot.writeI64(iter659)
+      for iter666 in self.fileIds:
+        oprot.writeI64(iter666)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.expr is not None:
@@ -15576,11 +15609,11 @@ class GetFileMetadataResult:
       if fid == 1:
         if ftype == TType.MAP:
           self.metadata = {}
-          (_ktype661, _vtype662, _size660 ) = iprot.readMapBegin()
-          for _i664 in xrange(_size660):
-            _key665 = iprot.readI64()
-            _val666 = iprot.readString()
-            self.metadata[_key665] = _val666
+          (_ktype668, _vtype669, _size667 ) = iprot.readMapBegin()
+          for _i671 in xrange(_size667):
+            _key672 = iprot.readI64()
+            _val673 = iprot.readString()
+            self.metadata[_key672] = _val673
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -15602,9 +15635,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 kiter667,viter668 in self.metadata.items():
-        oprot.writeI64(kiter667)
-        oprot.writeString(viter668)
+      for kiter674,viter675 in self.metadata.items():
+        oprot.writeI64(kiter674)
+        oprot.writeString(viter675)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.isSupported is not None:
@@ -15665,10 +15698,10 @@ class GetFileMetadataRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype672, _size669) = iprot.readListBegin()
-          for _i673 in xrange(_size669):
-            _elem674 = iprot.readI64()
-            self.fileIds.append(_elem674)
+          (_etype679, _size676) = iprot.readListBegin()
+          for _i680 in xrange(_size676):
+            _elem681 = iprot.readI64()
+            self.fileIds.append(_elem681)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15685,8 +15718,8 @@ class GetFileMetadataRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter675 in self.fileIds:
-        oprot.writeI64(iter675)
+      for iter682 in self.fileIds:
+        oprot.writeI64(iter682)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -15792,20 +15825,20 @@ class PutFileMetadataRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype679, _size676) = iprot.readListBegin()
-          for _i680 in xrange(_size676):
-            _elem681 = iprot.readI64()
-            self.fileIds.append(_elem681)
+          (_etype686, _size683) = iprot.readListBegin()
+          for _i687 in xrange(_size683):
+            _elem688 = iprot.readI64()
+            self.fileIds.append(_elem688)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.LIST:
           self.metadata = []
-          (_etype685, _size682) = iprot.readListBegin()
-          for _i686 in xrange(_size682):
-            _elem687 = iprot.readString()
-            self.metadata.append(_elem687)
+          (_etype692, _size689) = iprot.readListBegin()
+          for _i693 in xrange(_size689):
+            _elem694 = iprot.readString()
+            self.metadata.append(_elem694)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15827,15 +15860,15 @@ class PutFileMetadataRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter688 in self.fileIds:
-        oprot.writeI64(iter688)
+      for iter695 in self.fileIds:
+        oprot.writeI64(iter695)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.metadata is not None:
       oprot.writeFieldBegin('metadata', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.metadata))
-      for iter689 in self.metadata:
-        oprot.writeString(iter689)
+      for iter696 in self.metadata:
+        oprot.writeString(iter696)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.type is not None:
@@ -15943,10 +15976,10 @@ class ClearFileMetadataRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype693, _size690) = iprot.readListBegin()
-          for _i694 in xrange(_size690):
-            _elem695 = iprot.readI64()
-            self.fileIds.append(_elem695)
+          (_etype700, _size697) = iprot.readListBegin()
+          for _i701 in xrange(_size697):
+            _elem702 = iprot.readI64()
+            self.fileIds.append(_elem702)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15963,8 +15996,8 @@ class ClearFileMetadataRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter696 in self.fileIds:
-        oprot.writeI64(iter696)
+      for iter703 in self.fileIds:
+        oprot.writeI64(iter703)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16193,11 +16226,11 @@ class GetAllFunctionsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.functions = []
-          (_etype700, _size697) = iprot.readListBegin()
-          for _i701 in xrange(_size697):
-            _elem702 = Function()
-            _elem702.read(iprot)
-            self.functions.append(_elem702)
+          (_etype707, _size704) = iprot.readListBegin()
+          for _i708 in xrange(_size704):
+            _elem709 = Function()
+            _elem709.read(iprot)
+            self.functions.append(_elem709)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16214,8 +16247,8 @@ class GetAllFunctionsResponse:
     if self.functions is not None:
       oprot.writeFieldBegin('functions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.functions))
-      for iter703 in self.functions:
-        iter703.write(oprot)
+      for iter710 in self.functions:
+        iter710.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16267,10 +16300,10 @@ class ClientCapabilities:
       if fid == 1:
         if ftype == TType.LIST:
           self.values = []
-          (_etype707, _size704) = iprot.readListBegin()
-          for _i708 in xrange(_size704):
-            _elem709 = iprot.readI32()
-            self.values.append(_elem709)
+          (_etype714, _size711) = iprot.readListBegin()
+          for _i715 in xrange(_size711):
+            _elem716 = iprot.readI32()
+            self.values.append(_elem716)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16287,8 +16320,8 @@ class ClientCapabilities:
     if self.values is not None:
       oprot.writeFieldBegin('values', TType.LIST, 1)
       oprot.writeListBegin(TType.I32, len(self.values))
-      for iter710 in self.values:
-        oprot.writeI32(iter710)
+      for iter717 in self.values:
+        oprot.writeI32(iter717)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16533,10 +16566,10 @@ class GetTablesRequest:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tblNames = []
-          (_etype714, _size711) = iprot.readListBegin()
-          for _i715 in xrange(_size711):
-            _elem716 = iprot.readString()
-            self.tblNames.append(_elem716)
+          (_etype721, _size718) = iprot.readListBegin()
+          for _i722 in xrange(_size718):
+            _elem723 = iprot.readString()
+            self.tblNames.append(_elem723)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16568,8 +16601,8 @@ class GetTablesRequest:
     if self.tblNames is not None:
       oprot.writeFieldBegin('tblNames', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.tblNames))
-      for iter717 in self.tblNames:
-        oprot.writeString(iter717)
+      for iter724 in self.tblNames:
+        oprot.writeString(iter724)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.capabilities is not None:
@@ -16634,11 +16667,11 @@ class GetTablesResult:
       if fid == 1:
         if ftype == TType.LIST:
           self.tables = []
-          (_etype721, _size718) = iprot.readListBegin()
-          for _i722 in xrange(_size718):
-            _elem723 = Table()
-            _elem723.read(iprot)
-            self.tables.append(_elem723)
+          (_etype728, _size725) = iprot.readListBegin()
+          for _i729 in xrange(_size725):
+            _elem730 = Table()
+            _elem730.read(iprot)
+            self.tables.append(_elem730)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16655,8 +16688,8 @@ class GetTablesResult:
     if self.tables is not None:
       oprot.writeFieldBegin('tables', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.tables))
-      for iter724 in self.tables:
-        iter724.write(oprot)
+      for iter731 in self.tables:
+        iter731.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16970,10 +17003,10 @@ class Materialization:
       if fid == 1:
         if ftype == TType.SET:
           self.tablesUsed = set()
-          (_etype728, _size725) = iprot.readSetBegin()
-          for _i729 in xrange(_size725):
-            _elem730 = iprot.readString()
-            self.tablesUsed.add(_elem730)
+          (_etype735, _size732) = iprot.readSetBegin()
+          for _i736 in xrange(_size732):
+            _elem737 = iprot.readString()
+            self.tablesUsed.add(_elem737)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
@@ -17005,8 +17038,8 @@ class Materialization:
     if self.tablesUsed is not None:
       oprot.writeFieldBegin('tablesUsed', TType.SET, 1)
       oprot.writeSetBegin(TType.STRING, len(self.tablesUsed))
-      for iter731 in self.tablesUsed:
-        oprot.writeString(iter731)
+      for iter738 in self.tablesUsed:
+        oprot.writeString(iter738)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     if self.validTxnList is not None:
@@ -17911,44 +17944,44 @@ class WMFullResourcePlan:
       elif fid == 2:
         if ftype == TType.LIST:
           self.pools = []
-          (_etype735, _size732) = iprot.readListBegin()
-          for _i736 in xrange(_size732):
-            _elem737 = WMPool()
-            _elem737.read(iprot)
-            self.pools.append(_elem737)
+          (_etype742, _size739) = iprot.readListBegin()
+          for _i743 in xrange(_size739):
+            _elem744 = WMPool()
+            _elem744.read(iprot)
+            self.pools.append(_elem744)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.mappings = []
-          (_etype741, _size738) = iprot.readListBegin()
-          for _i742 in xrange(_size738):
-            _elem743 = WMMapping()
-            _elem743.read(iprot)
-            self.mappings.append(_elem743)
+          (_etype748, _size745) = iprot.readListBegin()
+          for _i749 in xrange(_size745):
+            _elem750 = WMMapping()
+            _elem750.read(iprot)
+            self.mappings.append(_elem750)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.LIST:
           self.triggers = []
-          (_etype747, _size744) = iprot.readListBegin()
-          for _i748 in xrange(_size744):
-            _elem749 = WMTrigger()
-            _elem749.read(iprot)
-            self.triggers.append(_elem749)
+          (_etype754, _size751) = iprot.readListBegin()
+          for _i755 in xrange(_size751):
+            _elem756 = WMTrigger()
+            _elem756.read(iprot)
+            self.triggers.append(_elem756)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.LIST:
           self.poolTriggers = []
-          (_etype753, _size750) = iprot.readListBegin()
-          for _i754 in xrange(_size750):
-            _elem755 = WMPoolTrigger()
-            _elem755.read(iprot)
-            self.poolTriggers.append(_elem755)
+          (_etype760, _size757) = iprot.readListBegin()
+          for _i761 in xrange(_size757):
+            _elem762 = WMPoolTrigger()
+            _elem762.read(iprot)
+            self.poolTriggers.append(_elem762)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17969,29 +18002,29 @@ class WMFullResourcePlan:
     if self.pools is not None:
       oprot.writeFieldBegin('pools', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.pools))
-      for iter756 in self.pools:
-        iter756.write(oprot)
+      for iter763 in self.pools:
+        iter763.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 iter757 in self.mappings:
-        iter757.write(oprot)
+      for iter764 in self.mappings:
+        iter764.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 iter758 in self.triggers:
-        iter758.write(oprot)
+      for iter765 in self.triggers:
+        iter765.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 iter759 in self.poolTriggers:
-        iter759.write(oprot)
+      for iter766 in self.poolTriggers:
+        iter766.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18465,11 +18498,11 @@ class WMGetAllResourcePlanResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.resourcePlans = []
-          (_etype763, _size760) = iprot.readListBegin()
-          for _i764 in xrange(_size760):
-            _elem765 = WMResourcePlan()
-            _elem765.read(iprot)
-            self.resourcePlans.append(_elem765)
+          (_etype770, _size767) = iprot.readListBegin()
+          for _i771 in xrange(_size767):
+            _elem772 = WMResourcePlan()
+            _elem772.read(iprot)
+            self.resourcePlans.append(_elem772)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18486,8 +18519,8 @@ class WMGetAllResourcePlanResponse:
     if self.resourcePlans is not None:
       oprot.writeFieldBegin('resourcePlans', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.resourcePlans))
-      for iter766 in self.resourcePlans:
-        iter766.write(oprot)
+      for iter773 in self.resourcePlans:
+        iter773.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18791,20 +18824,20 @@ class WMValidateResourcePlanResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.errors = []
-          (_etype770, _size767) = iprot.readListBegin()
-          for _i771 in xrange(_size767):
-            _elem772 = iprot.readString()
-            self.errors.append(_elem772)
+          (_etype777, _size774) = iprot.readListBegin()
+          for _i778 in xrange(_size774):
+            _elem779 = iprot.readString()
+            self.errors.append(_elem779)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.LIST:
           self.warnings = []
-          (_etype776, _size773) = iprot.readListBegin()
-          for _i777 in xrange(_size773):
-            _elem778 = iprot.readString()
-            self.warnings.append(_elem778)
+          (_etype783, _size780) = iprot.readListBegin()
+          for _i784 in xrange(_size780):
+            _elem785 = iprot.readString()
+            self.warnings.append(_elem785)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18821,15 +18854,15 @@ class WMValidateResourcePlanResponse:
     if self.errors is not None:
       oprot.writeFieldBegin('errors', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.errors))
-      for iter779 in self.errors:
-        oprot.writeString(iter779)
+      for iter786 in self.errors:
+        oprot.writeString(iter786)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.warnings is not None:
       oprot.writeFieldBegin('warnings', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.warnings))
-      for iter780 in self.warnings:
-        oprot.writeString(iter780)
+      for iter787 in self.warnings:
+        oprot.writeString(iter787)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -19406,11 +19439,11 @@ class WMGetTriggersForResourePlanResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.triggers = []
-          (_etype784, _size781) = iprot.readListBegin()
-          for _i785 in xrange(_size781):
-            _elem786 = WMTrigger()
-            _elem786.read(iprot)
-            self.triggers.append(_elem786)
+          (_etype791, _size788) = iprot.readListBegin()
+          for _i792 in xrange(_size788):
+            _elem793 = WMTrigger()
+            _elem793.read(iprot)
+            self.triggers.append(_elem793)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19427,8 +19460,8 @@ class WMGetTriggersForResourePlanResponse:
     if self.triggers is not None:
       oprot.writeFieldBegin('triggers', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.triggers))
-      for iter787 in self.triggers:
-        iter787.write(oprot)
+      for iter794 in self.triggers:
+        iter794.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20612,11 +20645,11 @@ class SchemaVersion:
       elif fid == 4:
         if ftype == TType.LIST:
           self.cols = []
-          (_etype791, _size788) = iprot.readListBegin()
-          for _i792 in xrange(_size788):
-            _elem793 = FieldSchema()
-            _elem793.read(iprot)
-            self.cols.append(_elem793)
+          (_etype798, _size795) = iprot.readListBegin()
+          for _i799 in xrange(_size795):
+            _elem800 = FieldSchema()
+            _elem800.read(iprot)
+            self.cols.append(_elem800)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20676,8 +20709,8 @@ class SchemaVersion:
     if self.cols is not None:
       oprot.writeFieldBegin('cols', TType.LIST, 4)
       oprot.writeListBegin(TType.STRUCT, len(self.cols))
-      for iter794 in self.cols:
-        iter794.write(oprot)
+      for iter801 in self.cols:
+        iter801.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.state is not None:
@@ -20932,11 +20965,11 @@ class FindSchemasByColsResp:
       if fid == 1:
         if ftype == TType.LIST:
           self.schemaVersions = []
-          (_etype798, _size795) = iprot.readListBegin()
-          for _i799 in xrange(_size795):
-            _elem800 = SchemaVersionDescriptor()
-            _elem800.read(iprot)
-            self.schemaVersions.append(_elem800)
+          (_etype805, _size802) = iprot.readListBegin()
+          for _i806 in xrange(_size802):
+            _elem807 = SchemaVersionDescriptor()
+            _elem807.read(iprot)
+            self.schemaVersions.append(_elem807)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20953,8 +20986,8 @@ class FindSchemasByColsResp:
     if self.schemaVersions is not None:
       oprot.writeFieldBegin('schemaVersions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.schemaVersions))
-      for iter801 in self.schemaVersions:
-        iter801.write(oprot)
+      for iter808 in self.schemaVersions:
+        iter808.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 0e70e89..3dbe4d8 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
@@ -2639,20 +2639,23 @@ end
 
 class AllocateTableWriteIdsRequest
   include ::Thrift::Struct, ::Thrift::Struct_Union
-  TXNIDS = 1
-  DBNAME = 2
-  TABLENAME = 3
+  DBNAME = 1
+  TABLENAME = 2
+  TXNIDS = 3
+  REPLPOLICY = 4
+  SRCTXNTOWRITEIDLIST = 5
 
   FIELDS = {
-    TXNIDS => {:type => ::Thrift::Types::LIST, :name => 'txnIds', :element => {:type => ::Thrift::Types::I64}},
     DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'},
-    TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tableName'}
+    TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tableName'},
+    TXNIDS => {:type => ::Thrift::Types::LIST, :name => 'txnIds', :element => {:type => ::Thrift::Types::I64}, :optional => true},
+    REPLPOLICY => {:type => ::Thrift::Types::STRING, :name => 'replPolicy', :optional => true},
+    SRCTXNTOWRITEIDLIST => {:type => ::Thrift::Types::LIST, :name => 'srcTxnToWriteIdList', :element => {:type => ::Thrift::Types::STRUCT, :class => ::TxnToWriteId}, :optional => true}
   }
 
   def struct_fields; FIELDS; end
 
   def validate
-    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field txnIds is unset!') unless @txnIds
     raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field dbName is unset!') unless @dbName
     raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field tableName is unset!') unless @tableName
   end

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 565549a..ae9ec5c 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
@@ -62,7 +62,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
-import java.util.function.Consumer;
 import java.util.regex.Pattern;
 
 import javax.jdo.JDOException;
@@ -88,6 +87,7 @@ import org.apache.hadoop.hive.metastore.events.AddNotNullConstraintEvent;
 import org.apache.hadoop.hive.metastore.events.AddPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.AddPrimaryKeyEvent;
 import org.apache.hadoop.hive.metastore.events.AddUniqueConstraintEvent;
+import org.apache.hadoop.hive.metastore.events.AllocWriteIdEvent;
 import org.apache.hadoop.hive.metastore.events.AlterDatabaseEvent;
 import org.apache.hadoop.hive.metastore.events.AlterISchemaEvent;
 import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent;
@@ -177,7 +177,6 @@ import org.apache.thrift.transport.TFramedTransport;
 import org.apache.thrift.transport.TServerSocket;
 import org.apache.thrift.transport.TTransport;
 import org.apache.thrift.transport.TTransportFactory;
-import org.iq80.leveldb.DB;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -7106,7 +7105,13 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     @Override
     public AllocateTableWriteIdsResponse allocate_table_write_ids(
             AllocateTableWriteIdsRequest rqst) throws TException {
-      return getTxnHandler().allocateTableWriteIds(rqst);
+      AllocateTableWriteIdsResponse response = getTxnHandler().allocateTableWriteIds(rqst);
+      if (listeners != null && !listeners.isEmpty()) {
+        MetaStoreListenerNotifier.notifyEvent(listeners, EventType.ALLOC_WRITE_ID,
+                new AllocWriteIdEvent(response.getTxnToWriteIds(), rqst.getDbName(),
+                        rqst.getTableName(), this));
+      }
+      return response;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 9a43b2c..108add0 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
@@ -72,7 +72,6 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.thrift.TApplicationException;
-import org.apache.thrift.TBase;
 import org.apache.thrift.TException;
 import org.apache.thrift.protocol.TBinaryProtocol;
 import org.apache.thrift.protocol.TCompactProtocol;
@@ -2436,9 +2435,22 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   @Override
   public List<TxnToWriteId> allocateTableWriteIdsBatch(List<Long> txnIds, String dbName, String tableName)
           throws TException {
-    AllocateTableWriteIdsRequest rqst = new AllocateTableWriteIdsRequest(txnIds, dbName, tableName);
-    AllocateTableWriteIdsResponse writeIds = client.allocate_table_write_ids(rqst);
-    return writeIds.getTxnToWriteIds();
+    AllocateTableWriteIdsRequest rqst = new AllocateTableWriteIdsRequest(dbName, tableName);
+    rqst.setTxnIds(txnIds);
+    return allocateTableWriteIdsBatchIntr(rqst);
+  }
+
+  @Override
+  public List<TxnToWriteId> replAllocateTableWriteIdsBatch(String dbName, String tableName,
+                                 String replPolicy, List<TxnToWriteId> srcTxnToWriteIdList) throws TException {
+    AllocateTableWriteIdsRequest rqst = new AllocateTableWriteIdsRequest(dbName, tableName);
+    rqst.setReplPolicy(replPolicy);
+    rqst.setSrcTxnToWriteIdList(srcTxnToWriteIdList);
+    return allocateTableWriteIdsBatchIntr(rqst);
+  }
+
+  private List<TxnToWriteId> allocateTableWriteIdsBatchIntr(AllocateTableWriteIdsRequest rqst) throws TException {
+    return client.allocate_table_write_ids(rqst).getTxnToWriteIds();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 98674cf..27f8775 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
@@ -27,7 +27,6 @@ import java.util.Map.Entry;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.common.ValidTxnWriteIdList;
@@ -36,7 +35,6 @@ import org.apache.hadoop.hive.common.classification.RetrySemantics;
 import org.apache.hadoop.hive.metastore.annotation.NoReconnect;
 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.CheckConstraintsRequest;
 import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.CmRecycleRequest;
@@ -127,7 +125,6 @@ import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMTrigger;
 import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.utils.ObjectPair;
 import org.apache.thrift.TException;
 
@@ -2884,6 +2881,16 @@ public interface IMetaStoreClient {
   List<TxnToWriteId> allocateTableWriteIdsBatch(List<Long> txnIds, String dbName, String tableName) throws TException;
 
   /**
+   * Allocate a per table write ID and associate it with the given transaction. Used by replication load task.
+   * @param dbName name of DB in which the table belongs.
+   * @param tableName table to which the write ID to be allocated
+   * @param replPolicy Used by replication task to identify the source cluster.
+   * @param srcTxnToWriteIdList List of txn to write id map sent from the source cluster.
+   * @throws TException
+   */
+  List<TxnToWriteId> replAllocateTableWriteIdsBatch(String dbName, String tableName, String replPolicy,
+                                                    List<TxnToWriteId> srcTxnToWriteIdList) throws TException;
+  /**
    * Show the list of currently open transactions.  This is for use by "show transactions" in the
    * grammar, not for applications that want to find a list of current transactions to work with.
    * Those wishing the latter should call {@link #getValidTxns()}.

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
index 7b3a80c..92505af 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
@@ -53,6 +53,9 @@ import org.apache.hadoop.hive.metastore.events.LoadPartitionDoneEvent;
 import org.apache.hadoop.hive.metastore.events.OpenTxnEvent;
 import org.apache.hadoop.hive.metastore.events.CommitTxnEvent;
 import org.apache.hadoop.hive.metastore.events.AbortTxnEvent;
+import org.apache.hadoop.hive.metastore.events.AllocWriteIdEvent;
+import org.apache.hadoop.hive.metastore.tools.SQLGenerator;
+import java.sql.Connection;
 
 /**
  * This abstract class needs to be extended to  provide implementation of actions that needs
@@ -234,26 +237,45 @@ public abstract class MetaStoreEventListener implements Configurable {
 
   /**
    * This will be called when a new transaction is started.
-   * @param openTxnEvent
+   * @param openTxnEvent event to be processed
+   * @param dbConn jdbc connection to remote meta store db.
+   * @param sqlGenerator helper class to generate db specific sql string.
    * @throws MetaException
    */
-  public void onOpenTxn(OpenTxnEvent openTxnEvent) throws MetaException {
+  public void onOpenTxn(OpenTxnEvent openTxnEvent, Connection dbConn, SQLGenerator sqlGenerator) throws MetaException {
   }
 
   /**
    * This will be called to commit a transaction.
-   * @param commitTxnEvent
+   * @param commitTxnEvent event to be processed
+   * @param dbConn jdbc connection to remote meta store db.
+   * @param sqlGenerator helper class to generate db specific sql string.
    * @throws MetaException
    */
-  public void onCommitTxn(CommitTxnEvent commitTxnEvent) throws MetaException {
+  public void onCommitTxn(CommitTxnEvent commitTxnEvent, Connection dbConn, SQLGenerator sqlGenerator) throws
+          MetaException {
   }
 
   /**
    * This will be called to abort a transaction.
-   * @param abortTxnEvent
+   * @param abortTxnEvent event to be processed
+   * @param dbConn jdbc connection to remote meta store db.
+   * @param sqlGenerator helper class to generate db specific sql string.
    * @throws MetaException
    */
-  public void onAbortTxn(AbortTxnEvent abortTxnEvent) throws MetaException {
+  public void onAbortTxn(AbortTxnEvent abortTxnEvent, Connection dbConn, SQLGenerator sqlGenerator)
+          throws MetaException {
+  }
+
+  /**
+   * This will be called to alloc a new write id.
+   * @param allocWriteIdEvent event to be processed
+   * @param dbConn jdbc connection to remote meta store db.
+   * @param sqlGenerator helper class to generate db specific sql string.
+   * @throws MetaException
+   */
+  public void onAllocWriteId(AllocWriteIdEvent allocWriteIdEvent, Connection dbConn, SQLGenerator sqlGenerator)
+          throws MetaException {
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
index e9bbfdc..b2856e2 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
@@ -52,10 +52,11 @@ import org.apache.hadoop.hive.metastore.events.ListenerEvent;
 import org.apache.hadoop.hive.metastore.events.OpenTxnEvent;
 import org.apache.hadoop.hive.metastore.events.CommitTxnEvent;
 import org.apache.hadoop.hive.metastore.events.AbortTxnEvent;
-
+import org.apache.hadoop.hive.metastore.events.AllocWriteIdEvent;
+import org.apache.hadoop.hive.metastore.tools.SQLGenerator;
+import java.sql.Connection;
 import java.util.List;
 import java.util.Map;
-
 import static org.apache.hadoop.hive.metastore.MetaStoreEventListenerConstants.HIVE_METASTORE_TRANSACTION_ACTIVE;
 import static org.apache.hadoop.hive.metastore.messaging.EventMessage.EventType;
 
@@ -209,29 +210,37 @@ public class MetaStoreListenerNotifier {
               (listener, event) -> listener.onCreateCatalog((CreateCatalogEvent)event))
           .put(EventType.DROP_CATALOG,
               (listener, event) -> listener.onDropCatalog((DropCatalogEvent)event))
-          .put(EventType.OPEN_TXN, new EventNotifier() {
-            @Override
-            public void notify(MetaStoreEventListener listener, ListenerEvent event) throws MetaException {
-              listener.onOpenTxn((OpenTxnEvent)event);
-            }
-          })
-          .put(EventType.COMMIT_TXN, new EventNotifier() {
-            @Override
-            public void notify(MetaStoreEventListener listener, ListenerEvent event)
-                throws MetaException {
-              listener.onCommitTxn((CommitTxnEvent) event);
-            }
-          })
-          .put(EventType.ABORT_TXN, new EventNotifier() {
-            @Override
-            public void notify(MetaStoreEventListener listener, ListenerEvent event)
-                throws MetaException {
-              listener.onAbortTxn((AbortTxnEvent) event);
-            }
-          })
+          .put(EventType.OPEN_TXN,
+              (listener, event) -> listener.onOpenTxn((OpenTxnEvent) event, null, null))
+          .put(EventType.COMMIT_TXN,
+              (listener, event) -> listener.onCommitTxn((CommitTxnEvent) event, null, null))
+          .put(EventType.ABORT_TXN,
+              (listener, event) -> listener.onAbortTxn((AbortTxnEvent) event, null, null))
+          .put(EventType.ALLOC_WRITE_ID,
+              (listener, event) -> listener.onAllocWriteId((AllocWriteIdEvent) event, null, null))
           .build()
   );
 
+
+  private interface TxnEventNotifier {
+    void notify(MetaStoreEventListener listener, ListenerEvent event, Connection dbConn, SQLGenerator sqlGenerator)
+            throws MetaException;
+  }
+
+  private static Map<EventType, TxnEventNotifier> txnNotificationEvents = Maps.newHashMap(
+    ImmutableMap.<EventType, TxnEventNotifier>builder()
+      .put(EventType.OPEN_TXN,
+        (listener, event, dbConn, sqlGenerator) -> listener.onOpenTxn((OpenTxnEvent) event, dbConn, sqlGenerator))
+      .put(EventType.COMMIT_TXN,
+        (listener, event, dbConn, sqlGenerator) -> listener.onCommitTxn((CommitTxnEvent) event, dbConn, sqlGenerator))
+      .put(EventType.ABORT_TXN,
+        (listener, event, dbConn, sqlGenerator) -> listener.onAbortTxn((AbortTxnEvent) event, dbConn, sqlGenerator))
+      .put(EventType.ALLOC_WRITE_ID,
+        (listener, event, dbConn, sqlGenerator) ->
+                listener.onAllocWriteId((AllocWriteIdEvent) event, dbConn, sqlGenerator))
+      .build()
+  );
+
   /**
    * Notify a list of listeners about a specific metastore event. Each listener notified might update
    * the (ListenerEvent) event by setting a parameter key/value pair. These updated parameters will
@@ -262,6 +271,38 @@ public class MetaStoreListenerNotifier {
   }
 
   /**
+   * Notify a list of listeners about a specific metastore event to be executed within a txn. Each listener notified
+   * might update the (ListenerEvent) event by setting a parameter key/value pair. These updated parameters will
+   * be returned to the caller.
+   *
+   * @param listeners List of MetaStoreEventListener listeners.
+   * @param eventType Type of the notification event.
+   * @param event The ListenerEvent with information about the event.
+   * @param dbConn The JDBC connection to the remote meta store db.
+   * @param sqlGenerator The helper class to generate db specific SQL string.
+   * @return A list of key/value pair parameters that the listeners set. The returned object will return an empty
+   *         map if no parameters were updated or if no listeners were notified.
+   * @throws MetaException If an error occurred while calling the listeners.
+   */
+  public static Map<String, String> notifyEventWithDirectSql(List<? extends MetaStoreEventListener> listeners,
+                                                             EventType eventType,
+                                                             ListenerEvent event,
+                                                   Connection dbConn, SQLGenerator sqlGenerator) throws MetaException {
+
+    Preconditions.checkNotNull(listeners, "Listeners must not be null.");
+    Preconditions.checkNotNull(event, "The event must not be null.");
+
+    for (MetaStoreEventListener listener : listeners) {
+      txnNotificationEvents.get(eventType).notify(listener, event, dbConn, sqlGenerator);
+    }
+
+    // Each listener called above might set a different parameter on the event.
+    // This write permission is allowed on the listener side to avoid breaking compatibility if we change the API
+    // method calls.
+    return event.getParameters();
+  }
+
+  /**
    * Notify a list of listeners about a specific metastore event. Each listener notified might update
    * the (ListenerEvent) event by setting a parameter key/value pair. These updated parameters will
    * be returned to the caller.

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AbortTxnEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AbortTxnEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AbortTxnEvent.java
index 062e719..fe4b974 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AbortTxnEvent.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AbortTxnEvent.java
@@ -21,8 +21,6 @@ package org.apache.hadoop.hive.metastore.events;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hive.metastore.IHMSHandler;
-import org.apache.hadoop.hive.metastore.tools.SQLGenerator;
-import java.sql.Connection;
 
 /**
  * AbortTxnEvent
@@ -33,8 +31,6 @@ import java.sql.Connection;
 public class AbortTxnEvent extends ListenerEvent {
 
   private final Long txnId;
-  Connection connection;
-  SQLGenerator sqlGenerator;
 
   /**
    *
@@ -44,20 +40,6 @@ public class AbortTxnEvent extends ListenerEvent {
   public AbortTxnEvent(Long transactionId, IHMSHandler handler) {
     super(true, handler);
     txnId = transactionId;
-    connection = null;
-    sqlGenerator = null;
-  }
-
-  /**
-   * @param transactionId Unique identification for the transaction just got aborted.
-   * @param connection connection to execute direct SQL statement within same transaction
-   * @param sqlGenerator generates db specific SQL query
-   */
-  public AbortTxnEvent(Long transactionId, Connection connection, SQLGenerator sqlGenerator) {
-    super(true, null);
-    this.txnId = transactionId;
-    this.connection = connection;
-    this.sqlGenerator = sqlGenerator;
   }
 
   /**
@@ -66,18 +48,4 @@ public class AbortTxnEvent extends ListenerEvent {
   public Long getTxnId() {
     return txnId;
   }
-
-  /**
-   * @return Connection connection - used only by DbNotificationListener
-   */
-  public Connection getConnection() {
-    return connection;
-  }
-
-  /**
-   * @return SQLGenerator sqlGenerator - used only by DbNotificationListener
-   */
-  public SQLGenerator getSqlGenerator() {
-    return sqlGenerator;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AllocWriteIdEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AllocWriteIdEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AllocWriteIdEvent.java
new file mode 100644
index 0000000..84a9a4e
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AllocWriteIdEvent.java
@@ -0,0 +1,57 @@
+/*
+ * 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.events;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
+import java.util.List;
+
+/**
+ * AllocWriteIdEvent.
+ * Event for allocating write id.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class AllocWriteIdEvent extends ListenerEvent {
+
+  private final List<TxnToWriteId> txnToWriteIdList;
+  private final String tableName;
+  private final String dbName;
+
+  public AllocWriteIdEvent(List<TxnToWriteId> txnToWriteIdList, String dbName, String tableName, IHMSHandler handler) {
+    super(true, handler);
+    this.txnToWriteIdList = txnToWriteIdList;
+    this.tableName = tableName;
+    this.dbName = dbName;
+  }
+
+  public List<TxnToWriteId> getTxnToWriteIdList() {
+    return txnToWriteIdList;
+  }
+
+  public String getTableName() {
+    return tableName;
+  }
+
+  public String getDbName() {
+    return dbName;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/CommitTxnEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/CommitTxnEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/CommitTxnEvent.java
index 7262e6b..ba382cd 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/CommitTxnEvent.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/CommitTxnEvent.java
@@ -21,8 +21,6 @@ package org.apache.hadoop.hive.metastore.events;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hive.metastore.IHMSHandler;
-import org.apache.hadoop.hive.metastore.tools.SQLGenerator;
-import java.sql.Connection;
 
 /**
  * CommitTxnEvent
@@ -33,8 +31,6 @@ import java.sql.Connection;
 public class CommitTxnEvent extends ListenerEvent {
 
   private final Long txnId;
-  Connection connection;
-  SQLGenerator sqlGenerator;
 
   /**
    *
@@ -44,20 +40,6 @@ public class CommitTxnEvent extends ListenerEvent {
   public CommitTxnEvent(Long transactionId, IHMSHandler handler) {
     super(true, handler);
     this.txnId = transactionId;
-    this.connection = null;
-    this.sqlGenerator = null;
-  }
-
-  /**
-   * @param transactionId Unique identification for the transaction just got committed.
-   * @param connection connection to execute direct SQL statement within same transaction
-   * @param sqlGenerator generates db specific SQL query
-   */
-  public CommitTxnEvent(Long transactionId, Connection connection, SQLGenerator sqlGenerator) {
-    super(true, null);
-    this.txnId = transactionId;
-    this.connection = connection;
-    this.sqlGenerator = sqlGenerator;
   }
 
   /**
@@ -66,18 +48,4 @@ public class CommitTxnEvent extends ListenerEvent {
   public Long getTxnId() {
     return txnId;
   }
-
-  /**
-   * @return Connection connection - used only by DbNotificationListener
-   */
-  public Connection getConnection() {
-    return connection;
-  }
-
-  /**
-   * @return SQLGenerator sqlGenerator - used only by DbNotificationListener
-   */
-  public SQLGenerator getSqlGenerator() {
-    return sqlGenerator;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/ListenerEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/ListenerEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/ListenerEvent.java
index b542afc..93f1c2b 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/ListenerEvent.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/ListenerEvent.java
@@ -23,9 +23,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hive.metastore.HiveMetaStore;
 import org.apache.hadoop.hive.metastore.IHMSHandler;
 import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
-import org.apache.hadoop.hive.metastore.tools.SQLGenerator;
 import javax.annotation.concurrent.NotThreadSafe;
-import java.sql.Connection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
@@ -163,20 +161,6 @@ public abstract class ListenerEvent {
   }
 
   /**
-   * Used by ACID/transaction related events for generating direct SQL in DBNotificationListener.
-   */
-  public Connection getConnection() {
-    return null;
-  }
-
-  /**
-   * Used by ACID/transaction related events for generating direct SQL in DBNotificationListener.
-   */
-  public SQLGenerator getSqlGenerator() {
-    return null;
-  }
-
-  /**
    * Put a parameter to the listener event only if the parameter is absent.
    *
    * Overridden parameters is not allowed, and an exception may be thrown to avoid a mis-configuration

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/OpenTxnEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/OpenTxnEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/OpenTxnEvent.java
index 088a6a1..547c43e 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/OpenTxnEvent.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/OpenTxnEvent.java
@@ -22,8 +22,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hive.metastore.IHMSHandler;
 import com.google.common.collect.Lists;
-import org.apache.hadoop.hive.metastore.tools.SQLGenerator;
-import java.sql.Connection;
 import java.util.List;
 
 /**
@@ -34,8 +32,6 @@ import java.util.List;
 @InterfaceStability.Stable
 public class OpenTxnEvent extends ListenerEvent {
   private List<Long> txnIds;
-  Connection connection;
-  SQLGenerator sqlGenerator;
 
   /**
    * @param txnIds List of unique identification for the transaction just opened.
@@ -44,20 +40,6 @@ public class OpenTxnEvent extends ListenerEvent {
   public OpenTxnEvent(List<Long> txnIds, IHMSHandler handler) {
     super(true, handler);
     this.txnIds = Lists.newArrayList(txnIds);
-    this.connection = null;
-    this.sqlGenerator = null;
-  }
-
-  /**
-   * @param txnIds List of unique identification for the transaction just opened.
-   * @param connection connection to execute direct SQL statement within same transaction
-   * @param sqlGenerator generates db specific SQL query
-   */
-  public OpenTxnEvent(List<Long> txnIds, Connection connection, SQLGenerator sqlGenerator) {
-    super(true, null);
-    this.txnIds = Lists.newArrayList(txnIds);
-    this.connection = connection;
-    this.sqlGenerator = sqlGenerator;
   }
 
   /**
@@ -66,18 +48,4 @@ public class OpenTxnEvent extends ListenerEvent {
   public List<Long> getTxnIds() {
     return txnIds;
   }
-
-  /**
-   * @return Connection connection - used only by DbNotificationListener
-   */
-  public Connection getConnection() {
-    return connection;
-  }
-
-  /**
-   * @return SQLGenerator sqlGenerator - used only by DbNotificationListener
-   */
-  public SQLGenerator getSqlGenerator() {
-    return sqlGenerator;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AllocWriteIdMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AllocWriteIdMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AllocWriteIdMessage.java
new file mode 100644
index 0000000..0766d02
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AllocWriteIdMessage.java
@@ -0,0 +1,36 @@
+/*
+ * 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.messaging;
+
+import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
+
+import java.util.List;
+
+/**
+ * HCat message sent when an alloc write id is done.
+ */
+public abstract class AllocWriteIdMessage extends EventMessage {
+  protected AllocWriteIdMessage() {
+    super(EventType.ALLOC_WRITE_ID);
+  }
+
+  public abstract String getTableName();
+
+  public abstract List<TxnToWriteId> getTxnToWriteIdList();
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java
index 5137c86..ffbce1d 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java
@@ -58,7 +58,8 @@ public abstract class EventMessage {
     DROP_CATALOG(MessageFactory.DROP_CATALOG_EVENT),
     OPEN_TXN(MessageFactory.OPEN_TXN_EVENT),
     COMMIT_TXN(MessageFactory.COMMIT_TXN_EVENT),
-    ABORT_TXN(MessageFactory.ABORT_TXN_EVENT);
+    ABORT_TXN(MessageFactory.ABORT_TXN_EVENT),
+    ALLOC_WRITE_ID(MessageFactory.ALLOC_WRITE_ID_EVENT);
 
     private String typeString;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageDeserializer.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageDeserializer.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageDeserializer.java
index 6583cc7..ca33579 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageDeserializer.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageDeserializer.java
@@ -181,6 +181,11 @@ public abstract class MessageDeserializer {
    */
   public abstract AbortTxnMessage getAbortTxnMessage(String messageBody);
 
+  /*
+   * Method to de-serialize AllocWriteIdMessage instance.
+   */
+  public abstract AllocWriteIdMessage getAllocWriteIdMessage(String messageBody);
+
   // Protection against construction.
   protected MessageDeserializer() {}
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
index dc4420e..75ca6ec 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
 import org.apache.hadoop.hive.metastore.utils.JavaUtils;
@@ -71,6 +72,7 @@ public abstract class MessageFactory {
   public static final String OPEN_TXN_EVENT = "OPEN_TXN";
   public static final String COMMIT_TXN_EVENT = "COMMIT_TXN";
   public static final String ABORT_TXN_EVENT = "ABORT_TXN";
+  public static final String ALLOC_WRITE_ID_EVENT = "ALLOC_WRITE_ID_EVENT";
 
   private static MessageFactory instance = null;
 
@@ -265,6 +267,17 @@ public abstract class MessageFactory {
    */
   public abstract AbortTxnMessage buildAbortTxnMessage(Long txnId);
 
+  /**
+   * Factory method for building alloc write id message
+   *
+   * @param txnToWriteIdList List of Txn Ids and write id map
+   * @param dbName db for which write ids to be allocated
+   * @param tableName table for which write ids to be allocated
+   * @return instance of AllocWriteIdMessage
+   */
+  public abstract AllocWriteIdMessage buildAllocWriteIdMessage(List<TxnToWriteId> txnToWriteIdList, String dbName,
+                                                               String tableName);
+
   /***
    * Factory method for building add primary key message
    *

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/event/filters/DatabaseAndTableFilter.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/event/filters/DatabaseAndTableFilter.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/event/filters/DatabaseAndTableFilter.java
index 09292a3..fdb6942 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/event/filters/DatabaseAndTableFilter.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/event/filters/DatabaseAndTableFilter.java
@@ -43,8 +43,7 @@ public class DatabaseAndTableFilter extends BasicFilter {
   private boolean isTxnRelatedEvent(final NotificationEvent event) {
     return ((event.getEventType().equals(MessageFactory.OPEN_TXN_EVENT)) ||
             (event.getEventType().equals(MessageFactory.COMMIT_TXN_EVENT)) ||
-            (event.getEventType().equals(MessageFactory.ABORT_TXN_EVENT))
-          );
+            (event.getEventType().equals(MessageFactory.ABORT_TXN_EVENT)));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAllocWriteIdMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAllocWriteIdMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAllocWriteIdMessage.java
new file mode 100644
index 0000000..d4aba11
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAllocWriteIdMessage.java
@@ -0,0 +1,111 @@
+/*
+ * 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.messaging.json;
+
+import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
+import org.apache.hadoop.hive.metastore.messaging.AllocWriteIdMessage;
+import org.codehaus.jackson.annotate.JsonProperty;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * JSON implementation of AllocWriteId
+ */
+public class JSONAllocWriteIdMessage extends AllocWriteIdMessage {
+
+  @JsonProperty
+  private String server, servicePrincipal, dbName, tableName;
+
+  @JsonProperty
+  private List<Long> txnIdList, writeIdList;
+
+  @JsonProperty
+  private Long timestamp;
+
+  private List<TxnToWriteId> txnToWriteIdList;
+
+  /**
+   * Default constructor, needed for Jackson.
+   */
+  public JSONAllocWriteIdMessage() {
+  }
+
+  public JSONAllocWriteIdMessage(String server, String servicePrincipal,
+                                 List<TxnToWriteId> txnToWriteIdList, String dbName, String tableName, Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.timestamp = timestamp;
+    this.txnIdList = new ArrayList<>();
+    this.writeIdList = new ArrayList<>();
+    for (TxnToWriteId txnToWriteId : txnToWriteIdList) {
+      this.txnIdList.add(txnToWriteId.getTxnId());
+      this.writeIdList.add(txnToWriteId.getWriteId());
+    }
+    this.tableName = tableName;
+    this.dbName = dbName;
+    this.txnToWriteIdList = txnToWriteIdList;
+  }
+
+  @Override
+  public String getServer() {
+    return server;
+  }
+
+  @Override
+  public String getServicePrincipal() {
+    return servicePrincipal;
+  }
+
+  @Override
+  public String getDB() {
+    return dbName;
+  }
+
+  @Override
+  public Long getTimestamp() {
+    return timestamp;
+  }
+
+  @Override
+  public String getTableName() {
+    return tableName;
+  }
+
+  @Override
+  public List<TxnToWriteId> getTxnToWriteIdList() {
+    // after deserialization, need to recreate the txnToWriteIdList as its not under JsonProperty.
+    if (txnToWriteIdList == null) {
+      txnToWriteIdList = new ArrayList<>();
+      for (int idx = 0; idx < txnIdList.size(); idx++) {
+        txnToWriteIdList.add(new TxnToWriteId(txnIdList.get(idx), writeIdList.get(idx)));
+      }
+    }
+    return txnToWriteIdList;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    } catch (Exception exception) {
+      throw new IllegalArgumentException("Could not serialize: ", exception);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageDeserializer.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageDeserializer.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageDeserializer.java
index d019ec1..f54e24d 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageDeserializer.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageDeserializer.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hive.metastore.messaging.MessageDeserializer;
 import org.apache.hadoop.hive.metastore.messaging.OpenTxnMessage;
 import org.apache.hadoop.hive.metastore.messaging.CommitTxnMessage;
 import org.apache.hadoop.hive.metastore.messaging.AbortTxnMessage;
+import org.apache.hadoop.hive.metastore.messaging.AllocWriteIdMessage;
 import org.codehaus.jackson.map.DeserializationConfig;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.map.SerializationConfig;
@@ -250,4 +251,12 @@ public class JSONMessageDeserializer extends MessageDeserializer {
       throw new IllegalArgumentException("Could not construct AbortTxnMessage", e);
     }
   }
+
+  public AllocWriteIdMessage getAllocWriteIdMessage(String messageBody) {
+    try {
+      return mapper.readValue(messageBody, JSONAllocWriteIdMessage.class);
+    } catch (Exception e) {
+      throw new IllegalArgumentException("Could not construct AllocWriteIdMessage", e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
index 65a6f78..f0c5f4f 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
 import org.apache.hadoop.hive.metastore.messaging.AddForeignKeyMessage;
 import org.apache.hadoop.hive.metastore.messaging.AddNotNullConstraintMessage;
 import org.apache.hadoop.hive.metastore.messaging.AddPartitionMessage;
@@ -63,6 +64,7 @@ import org.apache.hadoop.hive.metastore.messaging.PartitionFiles;
 import org.apache.hadoop.hive.metastore.messaging.OpenTxnMessage;
 import org.apache.hadoop.hive.metastore.messaging.CommitTxnMessage;
 import org.apache.hadoop.hive.metastore.messaging.AbortTxnMessage;
+import org.apache.hadoop.hive.metastore.messaging.AllocWriteIdMessage;
 import org.apache.thrift.TBase;
 import org.apache.thrift.TDeserializer;
 import org.apache.thrift.TException;
@@ -221,6 +223,11 @@ public class JSONMessageFactory extends MessageFactory {
     return new JSONAbortTxnMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, txnId, now());
   }
 
+  public AllocWriteIdMessage buildAllocWriteIdMessage(List<TxnToWriteId> txnToWriteIdList,
+                                                      String dbName, String tableName) {
+    return new JSONAllocWriteIdMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, txnToWriteIdList, dbName, tableName, now());
+  }
+
   private long now() {
     return System.currentTimeMillis() / 1000;
   }


[11/12] hive git commit: HIVE-19089: Create/Replicate Allocate write-id event (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 fd52f09..dfa13a0 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@ -2107,14 +2107,14 @@ uint32_t ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1181;
-            ::apache::thrift::protocol::TType _etype1184;
-            xfer += iprot->readListBegin(_etype1184, _size1181);
-            this->success.resize(_size1181);
-            uint32_t _i1185;
-            for (_i1185 = 0; _i1185 < _size1181; ++_i1185)
+            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[_i1185]);
+              xfer += iprot->readString(this->success[_i1191]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2153,10 +2153,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 _iter1186;
-      for (_iter1186 = this->success.begin(); _iter1186 != this->success.end(); ++_iter1186)
+      std::vector<std::string> ::const_iterator _iter1192;
+      for (_iter1192 = this->success.begin(); _iter1192 != this->success.end(); ++_iter1192)
       {
-        xfer += oprot->writeString((*_iter1186));
+        xfer += oprot->writeString((*_iter1192));
       }
       xfer += oprot->writeListEnd();
     }
@@ -2201,14 +2201,14 @@ uint32_t ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            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)
+            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))[_i1191]);
+              xfer += iprot->readString((*(this->success))[_i1197]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2325,14 +2325,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1192;
-            ::apache::thrift::protocol::TType _etype1195;
-            xfer += iprot->readListBegin(_etype1195, _size1192);
-            this->success.resize(_size1192);
-            uint32_t _i1196;
-            for (_i1196 = 0; _i1196 < _size1192; ++_i1196)
+            uint32_t _size1198;
+            ::apache::thrift::protocol::TType _etype1201;
+            xfer += iprot->readListBegin(_etype1201, _size1198);
+            this->success.resize(_size1198);
+            uint32_t _i1202;
+            for (_i1202 = 0; _i1202 < _size1198; ++_i1202)
             {
-              xfer += iprot->readString(this->success[_i1196]);
+              xfer += iprot->readString(this->success[_i1202]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2371,10 +2371,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 _iter1197;
-      for (_iter1197 = this->success.begin(); _iter1197 != this->success.end(); ++_iter1197)
+      std::vector<std::string> ::const_iterator _iter1203;
+      for (_iter1203 = this->success.begin(); _iter1203 != this->success.end(); ++_iter1203)
       {
-        xfer += oprot->writeString((*_iter1197));
+        xfer += oprot->writeString((*_iter1203));
       }
       xfer += oprot->writeListEnd();
     }
@@ -2419,14 +2419,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1198;
-            ::apache::thrift::protocol::TType _etype1201;
-            xfer += iprot->readListBegin(_etype1201, _size1198);
-            (*(this->success)).resize(_size1198);
-            uint32_t _i1202;
-            for (_i1202 = 0; _i1202 < _size1198; ++_i1202)
+            uint32_t _size1204;
+            ::apache::thrift::protocol::TType _etype1207;
+            xfer += iprot->readListBegin(_etype1207, _size1204);
+            (*(this->success)).resize(_size1204);
+            uint32_t _i1208;
+            for (_i1208 = 0; _i1208 < _size1204; ++_i1208)
             {
-              xfer += iprot->readString((*(this->success))[_i1202]);
+              xfer += iprot->readString((*(this->success))[_i1208]);
             }
             xfer += iprot->readListEnd();
           }
@@ -3488,17 +3488,17 @@ uint32_t ThriftHiveMetastore_get_type_all_result::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size1203;
-            ::apache::thrift::protocol::TType _ktype1204;
-            ::apache::thrift::protocol::TType _vtype1205;
-            xfer += iprot->readMapBegin(_ktype1204, _vtype1205, _size1203);
-            uint32_t _i1207;
-            for (_i1207 = 0; _i1207 < _size1203; ++_i1207)
+            uint32_t _size1209;
+            ::apache::thrift::protocol::TType _ktype1210;
+            ::apache::thrift::protocol::TType _vtype1211;
+            xfer += iprot->readMapBegin(_ktype1210, _vtype1211, _size1209);
+            uint32_t _i1213;
+            for (_i1213 = 0; _i1213 < _size1209; ++_i1213)
             {
-              std::string _key1208;
-              xfer += iprot->readString(_key1208);
-              Type& _val1209 = this->success[_key1208];
-              xfer += _val1209.read(iprot);
+              std::string _key1214;
+              xfer += iprot->readString(_key1214);
+              Type& _val1215 = this->success[_key1214];
+              xfer += _val1215.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -3537,11 +3537,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 _iter1210;
-      for (_iter1210 = this->success.begin(); _iter1210 != this->success.end(); ++_iter1210)
+      std::map<std::string, Type> ::const_iterator _iter1216;
+      for (_iter1216 = this->success.begin(); _iter1216 != this->success.end(); ++_iter1216)
       {
-        xfer += oprot->writeString(_iter1210->first);
-        xfer += _iter1210->second.write(oprot);
+        xfer += oprot->writeString(_iter1216->first);
+        xfer += _iter1216->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -3586,17 +3586,17 @@ uint32_t ThriftHiveMetastore_get_type_all_presult::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size1211;
-            ::apache::thrift::protocol::TType _ktype1212;
-            ::apache::thrift::protocol::TType _vtype1213;
-            xfer += iprot->readMapBegin(_ktype1212, _vtype1213, _size1211);
-            uint32_t _i1215;
-            for (_i1215 = 0; _i1215 < _size1211; ++_i1215)
+            uint32_t _size1217;
+            ::apache::thrift::protocol::TType _ktype1218;
+            ::apache::thrift::protocol::TType _vtype1219;
+            xfer += iprot->readMapBegin(_ktype1218, _vtype1219, _size1217);
+            uint32_t _i1221;
+            for (_i1221 = 0; _i1221 < _size1217; ++_i1221)
             {
-              std::string _key1216;
-              xfer += iprot->readString(_key1216);
-              Type& _val1217 = (*(this->success))[_key1216];
-              xfer += _val1217.read(iprot);
+              std::string _key1222;
+              xfer += iprot->readString(_key1222);
+              Type& _val1223 = (*(this->success))[_key1222];
+              xfer += _val1223.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -3750,14 +3750,14 @@ uint32_t ThriftHiveMetastore_get_fields_result::read(::apache::thrift::protocol:
         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 _size1224;
+            ::apache::thrift::protocol::TType _etype1227;
+            xfer += iprot->readListBegin(_etype1227, _size1224);
+            this->success.resize(_size1224);
+            uint32_t _i1228;
+            for (_i1228 = 0; _i1228 < _size1224; ++_i1228)
             {
-              xfer += this->success[_i1222].read(iprot);
+              xfer += this->success[_i1228].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3812,10 +3812,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 _iter1223;
-      for (_iter1223 = this->success.begin(); _iter1223 != this->success.end(); ++_iter1223)
+      std::vector<FieldSchema> ::const_iterator _iter1229;
+      for (_iter1229 = this->success.begin(); _iter1229 != this->success.end(); ++_iter1229)
       {
-        xfer += (*_iter1223).write(oprot);
+        xfer += (*_iter1229).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3868,14 +3868,14 @@ uint32_t ThriftHiveMetastore_get_fields_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1224;
-            ::apache::thrift::protocol::TType _etype1227;
-            xfer += iprot->readListBegin(_etype1227, _size1224);
-            (*(this->success)).resize(_size1224);
-            uint32_t _i1228;
-            for (_i1228 = 0; _i1228 < _size1224; ++_i1228)
+            uint32_t _size1230;
+            ::apache::thrift::protocol::TType _etype1233;
+            xfer += iprot->readListBegin(_etype1233, _size1230);
+            (*(this->success)).resize(_size1230);
+            uint32_t _i1234;
+            for (_i1234 = 0; _i1234 < _size1230; ++_i1234)
             {
-              xfer += (*(this->success))[_i1228].read(iprot);
+              xfer += (*(this->success))[_i1234].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4061,14 +4061,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::read(::
         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 _size1235;
+            ::apache::thrift::protocol::TType _etype1238;
+            xfer += iprot->readListBegin(_etype1238, _size1235);
+            this->success.resize(_size1235);
+            uint32_t _i1239;
+            for (_i1239 = 0; _i1239 < _size1235; ++_i1239)
             {
-              xfer += this->success[_i1233].read(iprot);
+              xfer += this->success[_i1239].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4123,10 +4123,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 _iter1234;
-      for (_iter1234 = this->success.begin(); _iter1234 != this->success.end(); ++_iter1234)
+      std::vector<FieldSchema> ::const_iterator _iter1240;
+      for (_iter1240 = this->success.begin(); _iter1240 != this->success.end(); ++_iter1240)
       {
-        xfer += (*_iter1234).write(oprot);
+        xfer += (*_iter1240).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -4179,14 +4179,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1235;
-            ::apache::thrift::protocol::TType _etype1238;
-            xfer += iprot->readListBegin(_etype1238, _size1235);
-            (*(this->success)).resize(_size1235);
-            uint32_t _i1239;
-            for (_i1239 = 0; _i1239 < _size1235; ++_i1239)
+            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)
             {
-              xfer += (*(this->success))[_i1239].read(iprot);
+              xfer += (*(this->success))[_i1245].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4356,14 +4356,14 @@ uint32_t ThriftHiveMetastore_get_schema_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            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)
+            uint32_t _size1246;
+            ::apache::thrift::protocol::TType _etype1249;
+            xfer += iprot->readListBegin(_etype1249, _size1246);
+            this->success.resize(_size1246);
+            uint32_t _i1250;
+            for (_i1250 = 0; _i1250 < _size1246; ++_i1250)
             {
-              xfer += this->success[_i1244].read(iprot);
+              xfer += this->success[_i1250].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4418,10 +4418,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 _iter1245;
-      for (_iter1245 = this->success.begin(); _iter1245 != this->success.end(); ++_iter1245)
+      std::vector<FieldSchema> ::const_iterator _iter1251;
+      for (_iter1251 = this->success.begin(); _iter1251 != this->success.end(); ++_iter1251)
       {
-        xfer += (*_iter1245).write(oprot);
+        xfer += (*_iter1251).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -4474,14 +4474,14 @@ uint32_t ThriftHiveMetastore_get_schema_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1246;
-            ::apache::thrift::protocol::TType _etype1249;
-            xfer += iprot->readListBegin(_etype1249, _size1246);
-            (*(this->success)).resize(_size1246);
-            uint32_t _i1250;
-            for (_i1250 = 0; _i1250 < _size1246; ++_i1250)
+            uint32_t _size1252;
+            ::apache::thrift::protocol::TType _etype1255;
+            xfer += iprot->readListBegin(_etype1255, _size1252);
+            (*(this->success)).resize(_size1252);
+            uint32_t _i1256;
+            for (_i1256 = 0; _i1256 < _size1252; ++_i1256)
             {
-              xfer += (*(this->success))[_i1250].read(iprot);
+              xfer += (*(this->success))[_i1256].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4667,14 +4667,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1251;
-            ::apache::thrift::protocol::TType _etype1254;
-            xfer += iprot->readListBegin(_etype1254, _size1251);
-            this->success.resize(_size1251);
-            uint32_t _i1255;
-            for (_i1255 = 0; _i1255 < _size1251; ++_i1255)
+            uint32_t _size1257;
+            ::apache::thrift::protocol::TType _etype1260;
+            xfer += iprot->readListBegin(_etype1260, _size1257);
+            this->success.resize(_size1257);
+            uint32_t _i1261;
+            for (_i1261 = 0; _i1261 < _size1257; ++_i1261)
             {
-              xfer += this->success[_i1255].read(iprot);
+              xfer += this->success[_i1261].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4729,10 +4729,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 _iter1256;
-      for (_iter1256 = this->success.begin(); _iter1256 != this->success.end(); ++_iter1256)
+      std::vector<FieldSchema> ::const_iterator _iter1262;
+      for (_iter1262 = this->success.begin(); _iter1262 != this->success.end(); ++_iter1262)
       {
-        xfer += (*_iter1256).write(oprot);
+        xfer += (*_iter1262).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -4785,14 +4785,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1257;
-            ::apache::thrift::protocol::TType _etype1260;
-            xfer += iprot->readListBegin(_etype1260, _size1257);
-            (*(this->success)).resize(_size1257);
-            uint32_t _i1261;
-            for (_i1261 = 0; _i1261 < _size1257; ++_i1261)
+            uint32_t _size1263;
+            ::apache::thrift::protocol::TType _etype1266;
+            xfer += iprot->readListBegin(_etype1266, _size1263);
+            (*(this->success)).resize(_size1263);
+            uint32_t _i1267;
+            for (_i1267 = 0; _i1267 < _size1263; ++_i1267)
             {
-              xfer += (*(this->success))[_i1261].read(iprot);
+              xfer += (*(this->success))[_i1267].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5385,14 +5385,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->primaryKeys.clear();
-            uint32_t _size1262;
-            ::apache::thrift::protocol::TType _etype1265;
-            xfer += iprot->readListBegin(_etype1265, _size1262);
-            this->primaryKeys.resize(_size1262);
-            uint32_t _i1266;
-            for (_i1266 = 0; _i1266 < _size1262; ++_i1266)
+            uint32_t _size1268;
+            ::apache::thrift::protocol::TType _etype1271;
+            xfer += iprot->readListBegin(_etype1271, _size1268);
+            this->primaryKeys.resize(_size1268);
+            uint32_t _i1272;
+            for (_i1272 = 0; _i1272 < _size1268; ++_i1272)
             {
-              xfer += this->primaryKeys[_i1266].read(iprot);
+              xfer += this->primaryKeys[_i1272].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5405,14 +5405,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->foreignKeys.clear();
-            uint32_t _size1267;
-            ::apache::thrift::protocol::TType _etype1270;
-            xfer += iprot->readListBegin(_etype1270, _size1267);
-            this->foreignKeys.resize(_size1267);
-            uint32_t _i1271;
-            for (_i1271 = 0; _i1271 < _size1267; ++_i1271)
+            uint32_t _size1273;
+            ::apache::thrift::protocol::TType _etype1276;
+            xfer += iprot->readListBegin(_etype1276, _size1273);
+            this->foreignKeys.resize(_size1273);
+            uint32_t _i1277;
+            for (_i1277 = 0; _i1277 < _size1273; ++_i1277)
             {
-              xfer += this->foreignKeys[_i1271].read(iprot);
+              xfer += this->foreignKeys[_i1277].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5425,14 +5425,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->uniqueConstraints.clear();
-            uint32_t _size1272;
-            ::apache::thrift::protocol::TType _etype1275;
-            xfer += iprot->readListBegin(_etype1275, _size1272);
-            this->uniqueConstraints.resize(_size1272);
-            uint32_t _i1276;
-            for (_i1276 = 0; _i1276 < _size1272; ++_i1276)
+            uint32_t _size1278;
+            ::apache::thrift::protocol::TType _etype1281;
+            xfer += iprot->readListBegin(_etype1281, _size1278);
+            this->uniqueConstraints.resize(_size1278);
+            uint32_t _i1282;
+            for (_i1282 = 0; _i1282 < _size1278; ++_i1282)
             {
-              xfer += this->uniqueConstraints[_i1276].read(iprot);
+              xfer += this->uniqueConstraints[_i1282].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5445,14 +5445,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->notNullConstraints.clear();
-            uint32_t _size1277;
-            ::apache::thrift::protocol::TType _etype1280;
-            xfer += iprot->readListBegin(_etype1280, _size1277);
-            this->notNullConstraints.resize(_size1277);
-            uint32_t _i1281;
-            for (_i1281 = 0; _i1281 < _size1277; ++_i1281)
+            uint32_t _size1283;
+            ::apache::thrift::protocol::TType _etype1286;
+            xfer += iprot->readListBegin(_etype1286, _size1283);
+            this->notNullConstraints.resize(_size1283);
+            uint32_t _i1287;
+            for (_i1287 = 0; _i1287 < _size1283; ++_i1287)
             {
-              xfer += this->notNullConstraints[_i1281].read(iprot);
+              xfer += this->notNullConstraints[_i1287].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5465,14 +5465,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->defaultConstraints.clear();
-            uint32_t _size1282;
-            ::apache::thrift::protocol::TType _etype1285;
-            xfer += iprot->readListBegin(_etype1285, _size1282);
-            this->defaultConstraints.resize(_size1282);
-            uint32_t _i1286;
-            for (_i1286 = 0; _i1286 < _size1282; ++_i1286)
+            uint32_t _size1288;
+            ::apache::thrift::protocol::TType _etype1291;
+            xfer += iprot->readListBegin(_etype1291, _size1288);
+            this->defaultConstraints.resize(_size1288);
+            uint32_t _i1292;
+            for (_i1292 = 0; _i1292 < _size1288; ++_i1292)
             {
-              xfer += this->defaultConstraints[_i1286].read(iprot);
+              xfer += this->defaultConstraints[_i1292].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5485,14 +5485,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->checkConstraints.clear();
-            uint32_t _size1287;
-            ::apache::thrift::protocol::TType _etype1290;
-            xfer += iprot->readListBegin(_etype1290, _size1287);
-            this->checkConstraints.resize(_size1287);
-            uint32_t _i1291;
-            for (_i1291 = 0; _i1291 < _size1287; ++_i1291)
+            uint32_t _size1293;
+            ::apache::thrift::protocol::TType _etype1296;
+            xfer += iprot->readListBegin(_etype1296, _size1293);
+            this->checkConstraints.resize(_size1293);
+            uint32_t _i1297;
+            for (_i1297 = 0; _i1297 < _size1293; ++_i1297)
             {
-              xfer += this->checkConstraints[_i1291].read(iprot);
+              xfer += this->checkConstraints[_i1297].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5525,10 +5525,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 _iter1292;
-    for (_iter1292 = this->primaryKeys.begin(); _iter1292 != this->primaryKeys.end(); ++_iter1292)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter1298;
+    for (_iter1298 = this->primaryKeys.begin(); _iter1298 != this->primaryKeys.end(); ++_iter1298)
     {
-      xfer += (*_iter1292).write(oprot);
+      xfer += (*_iter1298).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5537,10 +5537,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 _iter1293;
-    for (_iter1293 = this->foreignKeys.begin(); _iter1293 != this->foreignKeys.end(); ++_iter1293)
+    std::vector<SQLForeignKey> ::const_iterator _iter1299;
+    for (_iter1299 = this->foreignKeys.begin(); _iter1299 != this->foreignKeys.end(); ++_iter1299)
     {
-      xfer += (*_iter1293).write(oprot);
+      xfer += (*_iter1299).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5549,10 +5549,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 _iter1294;
-    for (_iter1294 = this->uniqueConstraints.begin(); _iter1294 != this->uniqueConstraints.end(); ++_iter1294)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter1300;
+    for (_iter1300 = this->uniqueConstraints.begin(); _iter1300 != this->uniqueConstraints.end(); ++_iter1300)
     {
-      xfer += (*_iter1294).write(oprot);
+      xfer += (*_iter1300).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5561,10 +5561,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 _iter1295;
-    for (_iter1295 = this->notNullConstraints.begin(); _iter1295 != this->notNullConstraints.end(); ++_iter1295)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter1301;
+    for (_iter1301 = this->notNullConstraints.begin(); _iter1301 != this->notNullConstraints.end(); ++_iter1301)
     {
-      xfer += (*_iter1295).write(oprot);
+      xfer += (*_iter1301).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5573,10 +5573,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("defaultConstraints", ::apache::thrift::protocol::T_LIST, 6);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->defaultConstraints.size()));
-    std::vector<SQLDefaultConstraint> ::const_iterator _iter1296;
-    for (_iter1296 = this->defaultConstraints.begin(); _iter1296 != this->defaultConstraints.end(); ++_iter1296)
+    std::vector<SQLDefaultConstraint> ::const_iterator _iter1302;
+    for (_iter1302 = this->defaultConstraints.begin(); _iter1302 != this->defaultConstraints.end(); ++_iter1302)
     {
-      xfer += (*_iter1296).write(oprot);
+      xfer += (*_iter1302).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5585,10 +5585,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("checkConstraints", ::apache::thrift::protocol::T_LIST, 7);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->checkConstraints.size()));
-    std::vector<SQLCheckConstraint> ::const_iterator _iter1297;
-    for (_iter1297 = this->checkConstraints.begin(); _iter1297 != this->checkConstraints.end(); ++_iter1297)
+    std::vector<SQLCheckConstraint> ::const_iterator _iter1303;
+    for (_iter1303 = this->checkConstraints.begin(); _iter1303 != this->checkConstraints.end(); ++_iter1303)
     {
-      xfer += (*_iter1297).write(oprot);
+      xfer += (*_iter1303).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5616,10 +5616,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 _iter1298;
-    for (_iter1298 = (*(this->primaryKeys)).begin(); _iter1298 != (*(this->primaryKeys)).end(); ++_iter1298)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter1304;
+    for (_iter1304 = (*(this->primaryKeys)).begin(); _iter1304 != (*(this->primaryKeys)).end(); ++_iter1304)
     {
-      xfer += (*_iter1298).write(oprot);
+      xfer += (*_iter1304).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5628,10 +5628,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 _iter1299;
-    for (_iter1299 = (*(this->foreignKeys)).begin(); _iter1299 != (*(this->foreignKeys)).end(); ++_iter1299)
+    std::vector<SQLForeignKey> ::const_iterator _iter1305;
+    for (_iter1305 = (*(this->foreignKeys)).begin(); _iter1305 != (*(this->foreignKeys)).end(); ++_iter1305)
     {
-      xfer += (*_iter1299).write(oprot);
+      xfer += (*_iter1305).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5640,10 +5640,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 _iter1300;
-    for (_iter1300 = (*(this->uniqueConstraints)).begin(); _iter1300 != (*(this->uniqueConstraints)).end(); ++_iter1300)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter1306;
+    for (_iter1306 = (*(this->uniqueConstraints)).begin(); _iter1306 != (*(this->uniqueConstraints)).end(); ++_iter1306)
     {
-      xfer += (*_iter1300).write(oprot);
+      xfer += (*_iter1306).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5652,10 +5652,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 _iter1301;
-    for (_iter1301 = (*(this->notNullConstraints)).begin(); _iter1301 != (*(this->notNullConstraints)).end(); ++_iter1301)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter1307;
+    for (_iter1307 = (*(this->notNullConstraints)).begin(); _iter1307 != (*(this->notNullConstraints)).end(); ++_iter1307)
     {
-      xfer += (*_iter1301).write(oprot);
+      xfer += (*_iter1307).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5664,10 +5664,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("defaultConstraints", ::apache::thrift::protocol::T_LIST, 6);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->defaultConstraints)).size()));
-    std::vector<SQLDefaultConstraint> ::const_iterator _iter1302;
-    for (_iter1302 = (*(this->defaultConstraints)).begin(); _iter1302 != (*(this->defaultConstraints)).end(); ++_iter1302)
+    std::vector<SQLDefaultConstraint> ::const_iterator _iter1308;
+    for (_iter1308 = (*(this->defaultConstraints)).begin(); _iter1308 != (*(this->defaultConstraints)).end(); ++_iter1308)
     {
-      xfer += (*_iter1302).write(oprot);
+      xfer += (*_iter1308).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5676,10 +5676,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("checkConstraints", ::apache::thrift::protocol::T_LIST, 7);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->checkConstraints)).size()));
-    std::vector<SQLCheckConstraint> ::const_iterator _iter1303;
-    for (_iter1303 = (*(this->checkConstraints)).begin(); _iter1303 != (*(this->checkConstraints)).end(); ++_iter1303)
+    std::vector<SQLCheckConstraint> ::const_iterator _iter1309;
+    for (_iter1309 = (*(this->checkConstraints)).begin(); _iter1309 != (*(this->checkConstraints)).end(); ++_iter1309)
     {
-      xfer += (*_iter1303).write(oprot);
+      xfer += (*_iter1309).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7847,14 +7847,14 @@ uint32_t ThriftHiveMetastore_truncate_table_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partNames.clear();
-            uint32_t _size1304;
-            ::apache::thrift::protocol::TType _etype1307;
-            xfer += iprot->readListBegin(_etype1307, _size1304);
-            this->partNames.resize(_size1304);
-            uint32_t _i1308;
-            for (_i1308 = 0; _i1308 < _size1304; ++_i1308)
+            uint32_t _size1310;
+            ::apache::thrift::protocol::TType _etype1313;
+            xfer += iprot->readListBegin(_etype1313, _size1310);
+            this->partNames.resize(_size1310);
+            uint32_t _i1314;
+            for (_i1314 = 0; _i1314 < _size1310; ++_i1314)
             {
-              xfer += iprot->readString(this->partNames[_i1308]);
+              xfer += iprot->readString(this->partNames[_i1314]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7891,10 +7891,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 _iter1309;
-    for (_iter1309 = this->partNames.begin(); _iter1309 != this->partNames.end(); ++_iter1309)
+    std::vector<std::string> ::const_iterator _iter1315;
+    for (_iter1315 = this->partNames.begin(); _iter1315 != this->partNames.end(); ++_iter1315)
     {
-      xfer += oprot->writeString((*_iter1309));
+      xfer += oprot->writeString((*_iter1315));
     }
     xfer += oprot->writeListEnd();
   }
@@ -7926,10 +7926,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 _iter1310;
-    for (_iter1310 = (*(this->partNames)).begin(); _iter1310 != (*(this->partNames)).end(); ++_iter1310)
+    std::vector<std::string> ::const_iterator _iter1316;
+    for (_iter1316 = (*(this->partNames)).begin(); _iter1316 != (*(this->partNames)).end(); ++_iter1316)
     {
-      xfer += oprot->writeString((*_iter1310));
+      xfer += oprot->writeString((*_iter1316));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8173,14 +8173,14 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1311;
-            ::apache::thrift::protocol::TType _etype1314;
-            xfer += iprot->readListBegin(_etype1314, _size1311);
-            this->success.resize(_size1311);
-            uint32_t _i1315;
-            for (_i1315 = 0; _i1315 < _size1311; ++_i1315)
+            uint32_t _size1317;
+            ::apache::thrift::protocol::TType _etype1320;
+            xfer += iprot->readListBegin(_etype1320, _size1317);
+            this->success.resize(_size1317);
+            uint32_t _i1321;
+            for (_i1321 = 0; _i1321 < _size1317; ++_i1321)
             {
-              xfer += iprot->readString(this->success[_i1315]);
+              xfer += iprot->readString(this->success[_i1321]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8219,10 +8219,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 _iter1316;
-      for (_iter1316 = this->success.begin(); _iter1316 != this->success.end(); ++_iter1316)
+      std::vector<std::string> ::const_iterator _iter1322;
+      for (_iter1322 = this->success.begin(); _iter1322 != this->success.end(); ++_iter1322)
       {
-        xfer += oprot->writeString((*_iter1316));
+        xfer += oprot->writeString((*_iter1322));
       }
       xfer += oprot->writeListEnd();
     }
@@ -8267,14 +8267,14 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1317;
-            ::apache::thrift::protocol::TType _etype1320;
-            xfer += iprot->readListBegin(_etype1320, _size1317);
-            (*(this->success)).resize(_size1317);
-            uint32_t _i1321;
-            for (_i1321 = 0; _i1321 < _size1317; ++_i1321)
+            uint32_t _size1323;
+            ::apache::thrift::protocol::TType _etype1326;
+            xfer += iprot->readListBegin(_etype1326, _size1323);
+            (*(this->success)).resize(_size1323);
+            uint32_t _i1327;
+            for (_i1327 = 0; _i1327 < _size1323; ++_i1327)
             {
-              xfer += iprot->readString((*(this->success))[_i1321]);
+              xfer += iprot->readString((*(this->success))[_i1327]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8444,14 +8444,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 _size1322;
-            ::apache::thrift::protocol::TType _etype1325;
-            xfer += iprot->readListBegin(_etype1325, _size1322);
-            this->success.resize(_size1322);
-            uint32_t _i1326;
-            for (_i1326 = 0; _i1326 < _size1322; ++_i1326)
+            uint32_t _size1328;
+            ::apache::thrift::protocol::TType _etype1331;
+            xfer += iprot->readListBegin(_etype1331, _size1328);
+            this->success.resize(_size1328);
+            uint32_t _i1332;
+            for (_i1332 = 0; _i1332 < _size1328; ++_i1332)
             {
-              xfer += iprot->readString(this->success[_i1326]);
+              xfer += iprot->readString(this->success[_i1332]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8490,10 +8490,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 _iter1327;
-      for (_iter1327 = this->success.begin(); _iter1327 != this->success.end(); ++_iter1327)
+      std::vector<std::string> ::const_iterator _iter1333;
+      for (_iter1333 = this->success.begin(); _iter1333 != this->success.end(); ++_iter1333)
       {
-        xfer += oprot->writeString((*_iter1327));
+        xfer += oprot->writeString((*_iter1333));
       }
       xfer += oprot->writeListEnd();
     }
@@ -8538,14 +8538,14 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_presult::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1328;
-            ::apache::thrift::protocol::TType _etype1331;
-            xfer += iprot->readListBegin(_etype1331, _size1328);
-            (*(this->success)).resize(_size1328);
-            uint32_t _i1332;
-            for (_i1332 = 0; _i1332 < _size1328; ++_i1332)
+            uint32_t _size1334;
+            ::apache::thrift::protocol::TType _etype1337;
+            xfer += iprot->readListBegin(_etype1337, _size1334);
+            (*(this->success)).resize(_size1334);
+            uint32_t _i1338;
+            for (_i1338 = 0; _i1338 < _size1334; ++_i1338)
             {
-              xfer += iprot->readString((*(this->success))[_i1332]);
+              xfer += iprot->readString((*(this->success))[_i1338]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8683,14 +8683,14 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_result::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1333;
-            ::apache::thrift::protocol::TType _etype1336;
-            xfer += iprot->readListBegin(_etype1336, _size1333);
-            this->success.resize(_size1333);
-            uint32_t _i1337;
-            for (_i1337 = 0; _i1337 < _size1333; ++_i1337)
+            uint32_t _size1339;
+            ::apache::thrift::protocol::TType _etype1342;
+            xfer += iprot->readListBegin(_etype1342, _size1339);
+            this->success.resize(_size1339);
+            uint32_t _i1343;
+            for (_i1343 = 0; _i1343 < _size1339; ++_i1343)
             {
-              xfer += iprot->readString(this->success[_i1337]);
+              xfer += iprot->readString(this->success[_i1343]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8729,10 +8729,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 _iter1338;
-      for (_iter1338 = this->success.begin(); _iter1338 != this->success.end(); ++_iter1338)
+      std::vector<std::string> ::const_iterator _iter1344;
+      for (_iter1344 = this->success.begin(); _iter1344 != this->success.end(); ++_iter1344)
       {
-        xfer += oprot->writeString((*_iter1338));
+        xfer += oprot->writeString((*_iter1344));
       }
       xfer += oprot->writeListEnd();
     }
@@ -8777,14 +8777,14 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_presult::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1339;
-            ::apache::thrift::protocol::TType _etype1342;
-            xfer += iprot->readListBegin(_etype1342, _size1339);
-            (*(this->success)).resize(_size1339);
-            uint32_t _i1343;
-            for (_i1343 = 0; _i1343 < _size1339; ++_i1343)
+            uint32_t _size1345;
+            ::apache::thrift::protocol::TType _etype1348;
+            xfer += iprot->readListBegin(_etype1348, _size1345);
+            (*(this->success)).resize(_size1345);
+            uint32_t _i1349;
+            for (_i1349 = 0; _i1349 < _size1345; ++_i1349)
             {
-              xfer += iprot->readString((*(this->success))[_i1343]);
+              xfer += iprot->readString((*(this->success))[_i1349]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8859,14 +8859,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 _size1344;
-            ::apache::thrift::protocol::TType _etype1347;
-            xfer += iprot->readListBegin(_etype1347, _size1344);
-            this->tbl_types.resize(_size1344);
-            uint32_t _i1348;
-            for (_i1348 = 0; _i1348 < _size1344; ++_i1348)
+            uint32_t _size1350;
+            ::apache::thrift::protocol::TType _etype1353;
+            xfer += iprot->readListBegin(_etype1353, _size1350);
+            this->tbl_types.resize(_size1350);
+            uint32_t _i1354;
+            for (_i1354 = 0; _i1354 < _size1350; ++_i1354)
             {
-              xfer += iprot->readString(this->tbl_types[_i1348]);
+              xfer += iprot->readString(this->tbl_types[_i1354]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8903,10 +8903,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 _iter1349;
-    for (_iter1349 = this->tbl_types.begin(); _iter1349 != this->tbl_types.end(); ++_iter1349)
+    std::vector<std::string> ::const_iterator _iter1355;
+    for (_iter1355 = this->tbl_types.begin(); _iter1355 != this->tbl_types.end(); ++_iter1355)
     {
-      xfer += oprot->writeString((*_iter1349));
+      xfer += oprot->writeString((*_iter1355));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8938,10 +8938,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 _iter1350;
-    for (_iter1350 = (*(this->tbl_types)).begin(); _iter1350 != (*(this->tbl_types)).end(); ++_iter1350)
+    std::vector<std::string> ::const_iterator _iter1356;
+    for (_iter1356 = (*(this->tbl_types)).begin(); _iter1356 != (*(this->tbl_types)).end(); ++_iter1356)
     {
-      xfer += oprot->writeString((*_iter1350));
+      xfer += oprot->writeString((*_iter1356));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8982,14 +8982,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1351;
-            ::apache::thrift::protocol::TType _etype1354;
-            xfer += iprot->readListBegin(_etype1354, _size1351);
-            this->success.resize(_size1351);
-            uint32_t _i1355;
-            for (_i1355 = 0; _i1355 < _size1351; ++_i1355)
+            uint32_t _size1357;
+            ::apache::thrift::protocol::TType _etype1360;
+            xfer += iprot->readListBegin(_etype1360, _size1357);
+            this->success.resize(_size1357);
+            uint32_t _i1361;
+            for (_i1361 = 0; _i1361 < _size1357; ++_i1361)
             {
-              xfer += this->success[_i1355].read(iprot);
+              xfer += this->success[_i1361].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9028,10 +9028,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 _iter1356;
-      for (_iter1356 = this->success.begin(); _iter1356 != this->success.end(); ++_iter1356)
+      std::vector<TableMeta> ::const_iterator _iter1362;
+      for (_iter1362 = this->success.begin(); _iter1362 != this->success.end(); ++_iter1362)
       {
-        xfer += (*_iter1356).write(oprot);
+        xfer += (*_iter1362).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -9076,14 +9076,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1357;
-            ::apache::thrift::protocol::TType _etype1360;
-            xfer += iprot->readListBegin(_etype1360, _size1357);
-            (*(this->success)).resize(_size1357);
-            uint32_t _i1361;
-            for (_i1361 = 0; _i1361 < _size1357; ++_i1361)
+            uint32_t _size1363;
+            ::apache::thrift::protocol::TType _etype1366;
+            xfer += iprot->readListBegin(_etype1366, _size1363);
+            (*(this->success)).resize(_size1363);
+            uint32_t _i1367;
+            for (_i1367 = 0; _i1367 < _size1363; ++_i1367)
             {
-              xfer += (*(this->success))[_i1361].read(iprot);
+              xfer += (*(this->success))[_i1367].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9221,14 +9221,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1362;
-            ::apache::thrift::protocol::TType _etype1365;
-            xfer += iprot->readListBegin(_etype1365, _size1362);
-            this->success.resize(_size1362);
-            uint32_t _i1366;
-            for (_i1366 = 0; _i1366 < _size1362; ++_i1366)
+            uint32_t _size1368;
+            ::apache::thrift::protocol::TType _etype1371;
+            xfer += iprot->readListBegin(_etype1371, _size1368);
+            this->success.resize(_size1368);
+            uint32_t _i1372;
+            for (_i1372 = 0; _i1372 < _size1368; ++_i1372)
             {
-              xfer += iprot->readString(this->success[_i1366]);
+              xfer += iprot->readString(this->success[_i1372]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9267,10 +9267,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 _iter1367;
-      for (_iter1367 = this->success.begin(); _iter1367 != this->success.end(); ++_iter1367)
+      std::vector<std::string> ::const_iterator _iter1373;
+      for (_iter1373 = this->success.begin(); _iter1373 != this->success.end(); ++_iter1373)
       {
-        xfer += oprot->writeString((*_iter1367));
+        xfer += oprot->writeString((*_iter1373));
       }
       xfer += oprot->writeListEnd();
     }
@@ -9315,14 +9315,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1368;
-            ::apache::thrift::protocol::TType _etype1371;
-            xfer += iprot->readListBegin(_etype1371, _size1368);
-            (*(this->success)).resize(_size1368);
-            uint32_t _i1372;
-            for (_i1372 = 0; _i1372 < _size1368; ++_i1372)
+            uint32_t _size1374;
+            ::apache::thrift::protocol::TType _etype1377;
+            xfer += iprot->readListBegin(_etype1377, _size1374);
+            (*(this->success)).resize(_size1374);
+            uint32_t _i1378;
+            for (_i1378 = 0; _i1378 < _size1374; ++_i1378)
             {
-              xfer += iprot->readString((*(this->success))[_i1372]);
+              xfer += iprot->readString((*(this->success))[_i1378]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9632,14 +9632,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 _size1373;
-            ::apache::thrift::protocol::TType _etype1376;
-            xfer += iprot->readListBegin(_etype1376, _size1373);
-            this->tbl_names.resize(_size1373);
-            uint32_t _i1377;
-            for (_i1377 = 0; _i1377 < _size1373; ++_i1377)
+            uint32_t _size1379;
+            ::apache::thrift::protocol::TType _etype1382;
+            xfer += iprot->readListBegin(_etype1382, _size1379);
+            this->tbl_names.resize(_size1379);
+            uint32_t _i1383;
+            for (_i1383 = 0; _i1383 < _size1379; ++_i1383)
             {
-              xfer += iprot->readString(this->tbl_names[_i1377]);
+              xfer += iprot->readString(this->tbl_names[_i1383]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9672,10 +9672,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 _iter1378;
-    for (_iter1378 = this->tbl_names.begin(); _iter1378 != this->tbl_names.end(); ++_iter1378)
+    std::vector<std::string> ::const_iterator _iter1384;
+    for (_iter1384 = this->tbl_names.begin(); _iter1384 != this->tbl_names.end(); ++_iter1384)
     {
-      xfer += oprot->writeString((*_iter1378));
+      xfer += oprot->writeString((*_iter1384));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9703,10 +9703,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 _iter1379;
-    for (_iter1379 = (*(this->tbl_names)).begin(); _iter1379 != (*(this->tbl_names)).end(); ++_iter1379)
+    std::vector<std::string> ::const_iterator _iter1385;
+    for (_iter1385 = (*(this->tbl_names)).begin(); _iter1385 != (*(this->tbl_names)).end(); ++_iter1385)
     {
-      xfer += oprot->writeString((*_iter1379));
+      xfer += oprot->writeString((*_iter1385));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9747,14 +9747,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 _size1380;
-            ::apache::thrift::protocol::TType _etype1383;
-            xfer += iprot->readListBegin(_etype1383, _size1380);
-            this->success.resize(_size1380);
-            uint32_t _i1384;
-            for (_i1384 = 0; _i1384 < _size1380; ++_i1384)
+            uint32_t _size1386;
+            ::apache::thrift::protocol::TType _etype1389;
+            xfer += iprot->readListBegin(_etype1389, _size1386);
+            this->success.resize(_size1386);
+            uint32_t _i1390;
+            for (_i1390 = 0; _i1390 < _size1386; ++_i1390)
             {
-              xfer += this->success[_i1384].read(iprot);
+              xfer += this->success[_i1390].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9785,10 +9785,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 _iter1385;
-      for (_iter1385 = this->success.begin(); _iter1385 != this->success.end(); ++_iter1385)
+      std::vector<Table> ::const_iterator _iter1391;
+      for (_iter1391 = this->success.begin(); _iter1391 != this->success.end(); ++_iter1391)
       {
-        xfer += (*_iter1385).write(oprot);
+        xfer += (*_iter1391).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -9829,14 +9829,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 _size1386;
-            ::apache::thrift::protocol::TType _etype1389;
-            xfer += iprot->readListBegin(_etype1389, _size1386);
-            (*(this->success)).resize(_size1386);
-            uint32_t _i1390;
-            for (_i1390 = 0; _i1390 < _size1386; ++_i1390)
+            uint32_t _size1392;
+            ::apache::thrift::protocol::TType _etype1395;
+            xfer += iprot->readListBegin(_etype1395, _size1392);
+            (*(this->success)).resize(_size1392);
+            uint32_t _i1396;
+            for (_i1396 = 0; _i1396 < _size1392; ++_i1396)
             {
-              xfer += (*(this->success))[_i1390].read(iprot);
+              xfer += (*(this->success))[_i1396].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -10369,14 +10369,14 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_args::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_names.clear();
-            uint32_t _size1391;
-            ::apache::thrift::protocol::TType _etype1394;
-            xfer += iprot->readListBegin(_etype1394, _size1391);
-            this->tbl_names.resize(_size1391);
-            uint32_t _i1395;
-            for (_i1395 = 0; _i1395 < _size1391; ++_i1395)
+            uint32_t _size1397;
+            ::apache::thrift::protocol::TType _etype1400;
+            xfer += iprot->readListBegin(_etype1400, _size1397);
+            this->tbl_names.resize(_size1397);
+            uint32_t _i1401;
+            for (_i1401 = 0; _i1401 < _size1397; ++_i1401)
             {
-              xfer += iprot->readString(this->tbl_names[_i1395]);
+              xfer += iprot->readString(this->tbl_names[_i1401]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10409,10 +10409,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 _iter1396;
-    for (_iter1396 = this->tbl_names.begin(); _iter1396 != this->tbl_names.end(); ++_iter1396)
+    std::vector<std::string> ::const_iterator _iter1402;
+    for (_iter1402 = this->tbl_names.begin(); _iter1402 != this->tbl_names.end(); ++_iter1402)
     {
-      xfer += oprot->writeString((*_iter1396));
+      xfer += oprot->writeString((*_iter1402));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10440,10 +10440,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 _iter1397;
-    for (_iter1397 = (*(this->tbl_names)).begin(); _iter1397 != (*(this->tbl_names)).end(); ++_iter1397)
+    std::vector<std::string> ::const_iterator _iter1403;
+    for (_iter1403 = (*(this->tbl_names)).begin(); _iter1403 != (*(this->tbl_names)).end(); ++_iter1403)
     {
-      xfer += oprot->writeString((*_iter1397));
+      xfer += oprot->writeString((*_iter1403));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10484,17 +10484,17 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_result::read(
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size1398;
-            ::apache::thrift::protocol::TType _ktype1399;
-            ::apache::thrift::protocol::TType _vtype1400;
-            xfer += iprot->readMapBegin(_ktype1399, _vtype1400, _size1398);
-            uint32_t _i1402;
-            for (_i1402 = 0; _i1402 < _size1398; ++_i1402)
+            uint32_t _size1404;
+            ::apache::thrift::protocol::TType _ktype1405;
+            ::apache::thrift::protocol::TType _vtype1406;
+            xfer += iprot->readMapBegin(_ktype1405, _vtype1406, _size1404);
+            uint32_t _i1408;
+            for (_i1408 = 0; _i1408 < _size1404; ++_i1408)
             {
-              std::string _key1403;
-              xfer += iprot->readString(_key1403);
-              Materialization& _val1404 = this->success[_key1403];
-              xfer += _val1404.read(iprot);
+              std::string _key1409;
+              xfer += iprot->readString(_key1409);
+              Materialization& _val1410 = this->success[_key1409];
+              xfer += _val1410.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -10549,11 +10549,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 _iter1405;
-      for (_iter1405 = this->success.begin(); _iter1405 != this->success.end(); ++_iter1405)
+      std::map<std::string, Materialization> ::const_iterator _iter1411;
+      for (_iter1411 = this->success.begin(); _iter1411 != this->success.end(); ++_iter1411)
       {
-        xfer += oprot->writeString(_iter1405->first);
-        xfer += _iter1405->second.write(oprot);
+        xfer += oprot->writeString(_iter1411->first);
+        xfer += _iter1411->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -10606,17 +10606,17 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_presult::read
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size1406;
-            ::apache::thrift::protocol::TType _ktype1407;
-            ::apache::thrift::protocol::TType _vtype1408;
-            xfer += iprot->readMapBegin(_ktype1407, _vtype1408, _size1406);
-            uint32_t _i1410;
-            for (_i1410 = 0; _i1410 < _size1406; ++_i1410)
+            uint32_t _size1412;
+            ::apache::thrift::protocol::TType _ktype1413;
+            ::apache::thrift::protocol::TType _vtype1414;
+            xfer += iprot->readMapBegin(_ktype1413, _vtype1414, _size1412);
+            uint32_t _i1416;
+            for (_i1416 = 0; _i1416 < _size1412; ++_i1416)
             {
-              std::string _key1411;
-              xfer += iprot->readString(_key1411);
-              Materialization& _val1412 = (*(this->success))[_key1411];
-              xfer += _val1412.read(iprot);
+              std::string _key1417;
+              xfer += iprot->readString(_key1417);
+              Materialization& _val1418 = (*(this->success))[_key1417];
+              xfer += _val1418.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -11077,14 +11077,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 _size1413;
-            ::apache::thrift::protocol::TType _etype1416;
-            xfer += iprot->readListBegin(_etype1416, _size1413);
-            this->success.resize(_size1413);
-            uint32_t _i1417;
-            for (_i1417 = 0; _i1417 < _size1413; ++_i1417)
+            uint32_t _size1419;
+            ::apache::thrift::protocol::TType _etype1422;
+            xfer += iprot->readListBegin(_etype1422, _size1419);
+            this->success.resize(_size1419);
+            uint32_t _i1423;
+            for (_i1423 = 0; _i1423 < _size1419; ++_i1423)
             {
-              xfer += iprot->readString(this->success[_i1417]);
+              xfer += iprot->readString(this->success[_i1423]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11139,10 +11139,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 _iter1418;
-      for (_iter1418 = this->success.begin(); _iter1418 != this->success.end(); ++_iter1418)
+      std::vector<std::string> ::const_iterator _iter1424;
+      for (_iter1424 = this->success.begin(); _iter1424 != this->success.end(); ++_iter1424)
       {
-        xfer += oprot->writeString((*_iter1418));
+        xfer += oprot->writeString((*_iter1424));
       }
       xfer += oprot->writeListEnd();
     }
@@ -11195,14 +11195,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 _size1419;
-            ::apache::thrift::protocol::TType _etype1422;
-            xfer += iprot->readListBegin(_etype1422, _size1419);
-            (*(this->success)).resize(_size1419);
-            uint32_t _i1423;
-            for (_i1423 = 0; _i1423 < _size1419; ++_i1423)
+            uint32_t _size1425;
+            ::apache::thrift::protocol::TType _etype1428;
+            xfer += iprot->readListBegin(_etype1428, _size1425);
+            (*(this->success)).resize(_size1425);
+            uint32_t _i1429;
+            for (_i1429 = 0; _i1429 < _size1425; ++_i1429)
             {
-              xfer += iprot->readString((*(this->success))[_i1423]);
+              xfer += iprot->readString((*(this->success))[_i1429]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12536,14 +12536,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size1424;
-            ::apache::thrift::protocol::TType _etype1427;
-            xfer += iprot->readListBegin(_etype1427, _size1424);
-            this->new_parts.resize(_size1424);
-            uint32_t _i1428;
-            for (_i1428 = 0; _i1428 < _size1424; ++_i1428)
+            uint32_t _size1430;
+            ::apache::thrift::protocol::TType _etype1433;
+            xfer += iprot->readListBegin(_etype1433, _size1430);
+            this->new_parts.resize(_size1430);
+            uint32_t _i1434;
+            for (_i1434 = 0; _i1434 < _size1430; ++_i1434)
             {
-              xfer += this->new_parts[_i1428].read(iprot);
+              xfer += this->new_parts[_i1434].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12572,10 +12572,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 _iter1429;
-    for (_iter1429 = this->new_parts.begin(); _iter1429 != this->new_parts.end(); ++_iter1429)
+    std::vector<Partition> ::const_iterator _iter1435;
+    for (_iter1435 = this->new_parts.begin(); _iter1435 != this->new_parts.end(); ++_iter1435)
     {
-      xfer += (*_iter1429).write(oprot);
+      xfer += (*_iter1435).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -12599,10 +12599,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 _iter1430;
-    for (_iter1430 = (*(this->new_parts)).begin(); _iter1430 != (*(this->new_parts)).end(); ++_iter1430)
+    std::vector<Partition> ::const_iterator _iter1436;
+    for (_iter1436 = (*(this->new_parts)).begin(); _iter1436 != (*(this->new_parts)).end(); ++_iter1436)
     {
-      xfer += (*_iter1430).write(oprot);
+      xfer += (*_iter1436).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -12811,14 +12811,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 _size1431;
-            ::apache::thrift::protocol::TType _etype1434;
-            xfer += iprot->readListBegin(_etype1434, _size1431);
-            this->new_parts.resize(_size1431);
-            uint32_t _i1435;
-            for (_i1435 = 0; _i1435 < _size1431; ++_i1435)
+            uint32_t _size1437;
+            ::apache::thrift::protocol::TType _etype1440;
+            xfer += iprot->readListBegin(_etype1440, _size1437);
+            this->new_parts.resize(_size1437);
+            uint32_t _i1441;
+            for (_i1441 = 0; _i1441 < _size1437; ++_i1441)
             {
-              xfer += this->new_parts[_i1435].read(iprot);
+              xfer += this->new_parts[_i1441].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12847,10 +12847,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 _iter1436;
-    for (_iter1436 = this->new_parts.begin(); _iter1436 != this->new_parts.end(); ++_iter1436)
+    std::vector<PartitionSpec> ::const_iterator _iter1442;
+    for (_iter1442 = this->new_parts.begin(); _iter1442 != this->new_parts.end(); ++_iter1442)
     {
-      xfer += (*_iter1436).write(oprot);
+      xfer += (*_iter1442).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -12874,10 +12874,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 _iter1437;
-    for (_iter1437 = (*(this->new_parts)).begin(); _iter1437 != (*(this->new_parts)).end(); ++_iter1437)
+    std::vector<PartitionSpec> ::const_iterator _iter1443;
+    for (_iter1443 = (*(this->new_parts)).begin(); _iter1443 != (*(this->new_parts)).end(); ++_iter1443)
     {
-      xfer += (*_iter1437).write(oprot);
+      xfer += (*_iter1443).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -13102,14 +13102,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1438;
-            ::apache::thrift::protocol::TType _etype1441;
-            xfer += iprot->readListBegin(_etype1441, _size1438);
-            this->part_vals.resize(_size1438);
-            uint32_t _i1442;
-            for (_i1442 = 0; _i1442 < _size1438; ++_i1442)
+            uint32_t _size1444;
+            ::apache::thrift::protocol::TType _etype1447;
+            xfer += iprot->readListBegin(_etype1447, _size1444);
+            this->part_vals.resize(_size1444);
+            uint32_t _i1448;
+            for (_i1448 = 0; _i1448 < _size1444; ++_i1448)
             {
-              xfer += iprot->readString(this->part_vals[_i1442]);
+              xfer += iprot->readString(this->part_vals[_i1448]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13146,10 +13146,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 _iter1443;
-    for (_iter1443 = this->part_vals.begin(); _iter1443 != this->part_vals.end(); ++_iter1443)
+    std::vector<std::string> ::const_iterator _iter1449;
+    for (_iter1449 = this->part_vals.begin(); _iter1449 != this->part_vals.end(); ++_iter1449)
     {
-      xfer += oprot->writeString((*_iter1443));
+      xfer += oprot->writeString((*_iter1449));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13181,10 +13181,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 _iter1444;
-    for (_iter1444 = (*(this->part_vals)).begin(); _iter1444 != (*(this->part_vals)).end(); ++_iter1444)
+    std::vector<std::string> ::const_iterator _iter1450;
+    for (_iter1450 = (*(this->part_vals)).begin(); _iter1450 != (*(this->part_vals)).end(); ++_iter1450)
     {
-      xfer += oprot->writeString((*_iter1444));
+      xfer += oprot->writeString((*_iter1450));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13656,14 +13656,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1445;
-            ::apache::thrift::protocol::TType _etype1448;
-            xfer += iprot->readListBegin(_etype1448, _size1445);
-            this->part_vals.resize(_size1445);
-            uint32_t _i1449;
-            for (_i1449 = 0; _i1449 < _size1445; ++_i1449)
+            uint32_t _size1451;
+            ::apache::thrift::protocol::TType _etype1454;
+            xfer += iprot->readListBegin(_etype1454, _size1451);
+            this->part_vals.resize(_size1451);
+            uint32_t _i1455;
+            for (_i1455 = 0; _i1455 < _size1451; ++_i1455)
             {
-              xfer += iprot->readString(this->part_vals[_i1449]);
+              xfer += iprot->readString(this->part_vals[_i1455]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13708,10 +13708,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 _iter1450;
-    for (_iter1450 = this->part_vals.begin(); _iter1450 != this->part_vals.end(); ++_iter1450)
+    std::vector<std::string> ::const_iterator _iter1456;
+    for (_iter1456 = this->part_vals.begin(); _iter1456 != this->part_vals.end(); ++_iter1456)
     {
-      xfer += oprot->writeString((*_iter1450));
+      xfer += oprot->writeString((*_iter1456));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13747,10 +13747,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 _iter1451;
-    for (_iter1451 = (*(this->part_vals)).begin(); _iter1451 != (*(this->part_vals)).end(); ++_iter1451)
+    std::vector<std::string> ::const_iterator _iter1457;
+    for (_iter1457 = (*(this->part_vals)).begin(); _iter1457 != (*(this->part_vals)).end(); ++_iter1457)
     {
-      xfer += oprot->writeString((*_iter1451));
+      xfer += oprot->writeString((*_iter1457));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14553,14 +14553,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1452;
-            ::apache::thrift::protocol::TType _etype1455;
-            xfer += iprot->readListBegin(_etype1455, _size1452);
-            this->part_vals.resize(_size1452);
-            uint32_t _i1456;
-            for (_i1456 = 0; _i1456 < _size1452; ++_i1456)
+            uint32_t _size1458;
+            ::apache::thrift::protocol::TType _etype1461;
+            xfer += iprot->readListBegin(_etype1461, _size1458);
+            this->part_vals.resize(_size1458);
+            uint32_t _i1462;
+            for (_i1462 = 0; _i1462 < _size1458; ++_i1462)
             {
-              xfer += iprot->readString(this->part_vals[_i1456]);
+              xfer += iprot->readString(this->part_vals[_i1462]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14605,10 +14605,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 _iter1457;
-    for (_iter1457 = this->part_vals.begin(); _iter1457 != this->part_vals.end(); ++_iter1457)
+    std::vector<std::string> ::const_iterator _iter1463;
+    for (_iter1463 = this->part_vals.begin(); _iter1463 != this->part_vals.end(); ++_iter1463)
     {
-      xfer += oprot->writeString((*_iter1457));
+      xfer += oprot->writeString((*_iter1463));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14644,10 +14644,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 _iter1458;
-    for (_iter1458 = (*(this->part_vals)).begin(); _iter1458 != (*(this->part_vals)).end(); ++_iter1458)
+    std::vector<std::string> ::const_iterator _iter1464;
+    for (_iter1464 = (*(this->part_vals)).begin(); _iter1464 != (*(this->part_vals)).end(); ++_iter1464)
     {
-      xfer += oprot->writeString((*_iter1458));
+      xfer += oprot->writeString((*_iter1464));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14856,14 +14856,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1459;
-            ::apache::thrift::protocol::TType _etype1462;
-            xfer += iprot->readListBegin(_etype1462, _size1459);
-            this->part_vals.resize(_size1459);
-            uint32_t _i1463;
-            for (_i1463 = 0; _i1463 < _size1459; ++_i1463)
+            uint32_t _size1465;
+            ::apache::thrift::protocol::TType _etype1468;
+            xfer += iprot->readListBegin(_etype1468, _size1465);
+            this->part_vals.resize(_size1465);
+            uint32_t _i1469;
+            for (_i1469 = 0; _i1469 < _size1465; ++_i1469)
             {
-              xfer += iprot->readString(this->part_vals[_i1463]);
+              xfer += iprot->readString(this->part_vals[_i1469]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14916,10 +14916,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 _iter1464;
-    for (_iter1464 = this->part_vals.begin(); _iter1464 != this->part_vals.end(); ++_iter1464)
+    std::vector<std::string> ::const_iterator _iter1470;
+    for (_iter1470 = this->part_vals.begin(); _iter1470 != this->part_vals.end(); ++_iter1470)
     {
-      xfer += oprot->writeString((*_iter1464));
+      xfer += oprot->writeString((*_iter1470));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14959,10 +14959,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 _iter1465;
-    for (_iter1465 = (*(this->part_vals)).begin(); _iter1465 != (*(this->part_vals)).end(); ++_iter1465)
+    std::vector<std::string> ::const_iterator _iter1471;
+    for (_iter1471 = (*(this->part_vals)).begin(); _iter1471 != (*(this->part_vals)).end(); ++_iter1471)
     {
-      xfer += oprot->writeString((*_iter1465));
+      xfer += oprot->writeString((*_iter1471));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15968,14 +15968,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1466;
-            ::apache::thrift::protocol::TType _etype1469;
-            xfer += iprot->readListBegin(_etype1469, _size1466);
-            this->part_vals.resize(_size1466);
-            uint32_t _i1470;
-            for (_i1470 = 0; _i1470 < _size1466; ++_i1470)
+            uint32_t _size1472;
+            ::apache::thrift::protocol::TType _etype1475;
+            xfer += iprot->readListBegin(_etype1475, _size1472);
+            this->part_vals.resize(_size1472);
+            uint32_t _i1476;
+            for (_i1476 = 0; _i1476 < _size1472; ++_i1476)
             {
-              xfer += iprot->readString(this->part_vals[_i1470]);
+              xfer += iprot->readString(this->part_vals[_i1476]);
             }
             xfer += iprot->readListEnd();
           }
@@ -16012,10 +16012,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 _iter1471;
-    for (_iter1471 = this->part_vals.begin(); _iter1471 != this->part_vals.end(); ++_iter1471)
+    std::vector<std::string> ::const_iterator _iter1477;
+    for (_iter1477 = this->part_vals.begin(); _iter1477 != this->part_vals.end(); ++_iter1477)
     {
-      xfer += oprot->writeString((*_iter1471));
+      xfer += oprot->writeString((*_iter1477));
     }
     xfer += oprot->writeListEnd();
   }
@@ -16047,10 +16047,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 _iter1472;
-    for (_iter1472 = (*(this->part_vals)).begin(); _iter1472 != (*(this->part_vals)).end(); ++_iter1472)
+    std::vector<std::string> ::const_iterator _iter1478;
+    for (_iter1478 = (*(this->part_vals)).begin(); _iter1478 != (*(this->part_vals)).end(); ++_iter1478)
     {
-      xfer += oprot->writeString((*_iter1472));
+      xfer += oprot->writeString((*_iter1478));
     }
     xfer += oprot->writeListEnd();
   }


<TRUNCATED>

[07/12] hive git commit: HIVE-19089: Create/Replicate Allocate write-id event (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 afe82e3..a354f27 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
@@ -40589,13 +40589,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list912 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list912.size);
-                  String _elem913;
-                  for (int _i914 = 0; _i914 < _list912.size; ++_i914)
+                  org.apache.thrift.protocol.TList _list920 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list920.size);
+                  String _elem921;
+                  for (int _i922 = 0; _i922 < _list920.size; ++_i922)
                   {
-                    _elem913 = iprot.readString();
-                    struct.success.add(_elem913);
+                    _elem921 = iprot.readString();
+                    struct.success.add(_elem921);
                   }
                   iprot.readListEnd();
                 }
@@ -40630,9 +40630,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 _iter915 : struct.success)
+            for (String _iter923 : struct.success)
             {
-              oprot.writeString(_iter915);
+              oprot.writeString(_iter923);
             }
             oprot.writeListEnd();
           }
@@ -40671,9 +40671,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter916 : struct.success)
+            for (String _iter924 : struct.success)
             {
-              oprot.writeString(_iter916);
+              oprot.writeString(_iter924);
             }
           }
         }
@@ -40688,13 +40688,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list917 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list917.size);
-            String _elem918;
-            for (int _i919 = 0; _i919 < _list917.size; ++_i919)
+            org.apache.thrift.protocol.TList _list925 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list925.size);
+            String _elem926;
+            for (int _i927 = 0; _i927 < _list925.size; ++_i927)
             {
-              _elem918 = iprot.readString();
-              struct.success.add(_elem918);
+              _elem926 = iprot.readString();
+              struct.success.add(_elem926);
             }
           }
           struct.setSuccessIsSet(true);
@@ -41348,13 +41348,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list920 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list920.size);
-                  String _elem921;
-                  for (int _i922 = 0; _i922 < _list920.size; ++_i922)
+                  org.apache.thrift.protocol.TList _list928 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list928.size);
+                  String _elem929;
+                  for (int _i930 = 0; _i930 < _list928.size; ++_i930)
                   {
-                    _elem921 = iprot.readString();
-                    struct.success.add(_elem921);
+                    _elem929 = iprot.readString();
+                    struct.success.add(_elem929);
                   }
                   iprot.readListEnd();
                 }
@@ -41389,9 +41389,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 _iter923 : struct.success)
+            for (String _iter931 : struct.success)
             {
-              oprot.writeString(_iter923);
+              oprot.writeString(_iter931);
             }
             oprot.writeListEnd();
           }
@@ -41430,9 +41430,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter924 : struct.success)
+            for (String _iter932 : struct.success)
             {
-              oprot.writeString(_iter924);
+              oprot.writeString(_iter932);
             }
           }
         }
@@ -41447,13 +41447,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list925 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list925.size);
-            String _elem926;
-            for (int _i927 = 0; _i927 < _list925.size; ++_i927)
+            org.apache.thrift.protocol.TList _list933 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list933.size);
+            String _elem934;
+            for (int _i935 = 0; _i935 < _list933.size; ++_i935)
             {
-              _elem926 = iprot.readString();
-              struct.success.add(_elem926);
+              _elem934 = iprot.readString();
+              struct.success.add(_elem934);
             }
           }
           struct.setSuccessIsSet(true);
@@ -46060,16 +46060,16 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map928 = iprot.readMapBegin();
-                  struct.success = new HashMap<String,Type>(2*_map928.size);
-                  String _key929;
-                  Type _val930;
-                  for (int _i931 = 0; _i931 < _map928.size; ++_i931)
+                  org.apache.thrift.protocol.TMap _map936 = iprot.readMapBegin();
+                  struct.success = new HashMap<String,Type>(2*_map936.size);
+                  String _key937;
+                  Type _val938;
+                  for (int _i939 = 0; _i939 < _map936.size; ++_i939)
                   {
-                    _key929 = iprot.readString();
-                    _val930 = new Type();
-                    _val930.read(iprot);
-                    struct.success.put(_key929, _val930);
+                    _key937 = iprot.readString();
+                    _val938 = new Type();
+                    _val938.read(iprot);
+                    struct.success.put(_key937, _val938);
                   }
                   iprot.readMapEnd();
                 }
@@ -46104,10 +46104,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> _iter932 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter940 : struct.success.entrySet())
             {
-              oprot.writeString(_iter932.getKey());
-              _iter932.getValue().write(oprot);
+              oprot.writeString(_iter940.getKey());
+              _iter940.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -46146,10 +46146,10 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Map.Entry<String, Type> _iter933 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter941 : struct.success.entrySet())
             {
-              oprot.writeString(_iter933.getKey());
-              _iter933.getValue().write(oprot);
+              oprot.writeString(_iter941.getKey());
+              _iter941.getValue().write(oprot);
             }
           }
         }
@@ -46164,16 +46164,16 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map934 = 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*_map934.size);
-            String _key935;
-            Type _val936;
-            for (int _i937 = 0; _i937 < _map934.size; ++_i937)
+            org.apache.thrift.protocol.TMap _map942 = 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*_map942.size);
+            String _key943;
+            Type _val944;
+            for (int _i945 = 0; _i945 < _map942.size; ++_i945)
             {
-              _key935 = iprot.readString();
-              _val936 = new Type();
-              _val936.read(iprot);
-              struct.success.put(_key935, _val936);
+              _key943 = iprot.readString();
+              _val944 = new Type();
+              _val944.read(iprot);
+              struct.success.put(_key943, _val944);
             }
           }
           struct.setSuccessIsSet(true);
@@ -47208,14 +47208,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list938 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list938.size);
-                  FieldSchema _elem939;
-                  for (int _i940 = 0; _i940 < _list938.size; ++_i940)
+                  org.apache.thrift.protocol.TList _list946 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list946.size);
+                  FieldSchema _elem947;
+                  for (int _i948 = 0; _i948 < _list946.size; ++_i948)
                   {
-                    _elem939 = new FieldSchema();
-                    _elem939.read(iprot);
-                    struct.success.add(_elem939);
+                    _elem947 = new FieldSchema();
+                    _elem947.read(iprot);
+                    struct.success.add(_elem947);
                   }
                   iprot.readListEnd();
                 }
@@ -47268,9 +47268,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 _iter941 : struct.success)
+            for (FieldSchema _iter949 : struct.success)
             {
-              _iter941.write(oprot);
+              _iter949.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -47325,9 +47325,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter942 : struct.success)
+            for (FieldSchema _iter950 : struct.success)
             {
-              _iter942.write(oprot);
+              _iter950.write(oprot);
             }
           }
         }
@@ -47348,14 +47348,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list943 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list943.size);
-            FieldSchema _elem944;
-            for (int _i945 = 0; _i945 < _list943.size; ++_i945)
+            org.apache.thrift.protocol.TList _list951 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list951.size);
+            FieldSchema _elem952;
+            for (int _i953 = 0; _i953 < _list951.size; ++_i953)
             {
-              _elem944 = new FieldSchema();
-              _elem944.read(iprot);
-              struct.success.add(_elem944);
+              _elem952 = new FieldSchema();
+              _elem952.read(iprot);
+              struct.success.add(_elem952);
             }
           }
           struct.setSuccessIsSet(true);
@@ -48509,14 +48509,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list946 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list946.size);
-                  FieldSchema _elem947;
-                  for (int _i948 = 0; _i948 < _list946.size; ++_i948)
+                  org.apache.thrift.protocol.TList _list954 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list954.size);
+                  FieldSchema _elem955;
+                  for (int _i956 = 0; _i956 < _list954.size; ++_i956)
                   {
-                    _elem947 = new FieldSchema();
-                    _elem947.read(iprot);
-                    struct.success.add(_elem947);
+                    _elem955 = new FieldSchema();
+                    _elem955.read(iprot);
+                    struct.success.add(_elem955);
                   }
                   iprot.readListEnd();
                 }
@@ -48569,9 +48569,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 _iter949 : struct.success)
+            for (FieldSchema _iter957 : struct.success)
             {
-              _iter949.write(oprot);
+              _iter957.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -48626,9 +48626,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter950 : struct.success)
+            for (FieldSchema _iter958 : struct.success)
             {
-              _iter950.write(oprot);
+              _iter958.write(oprot);
             }
           }
         }
@@ -48649,14 +48649,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list951 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list951.size);
-            FieldSchema _elem952;
-            for (int _i953 = 0; _i953 < _list951.size; ++_i953)
+            org.apache.thrift.protocol.TList _list959 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list959.size);
+            FieldSchema _elem960;
+            for (int _i961 = 0; _i961 < _list959.size; ++_i961)
             {
-              _elem952 = new FieldSchema();
-              _elem952.read(iprot);
-              struct.success.add(_elem952);
+              _elem960 = new FieldSchema();
+              _elem960.read(iprot);
+              struct.success.add(_elem960);
             }
           }
           struct.setSuccessIsSet(true);
@@ -49701,14 +49701,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list954 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list954.size);
-                  FieldSchema _elem955;
-                  for (int _i956 = 0; _i956 < _list954.size; ++_i956)
+                  org.apache.thrift.protocol.TList _list962 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list962.size);
+                  FieldSchema _elem963;
+                  for (int _i964 = 0; _i964 < _list962.size; ++_i964)
                   {
-                    _elem955 = new FieldSchema();
-                    _elem955.read(iprot);
-                    struct.success.add(_elem955);
+                    _elem963 = new FieldSchema();
+                    _elem963.read(iprot);
+                    struct.success.add(_elem963);
                   }
                   iprot.readListEnd();
                 }
@@ -49761,9 +49761,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 _iter957 : struct.success)
+            for (FieldSchema _iter965 : struct.success)
             {
-              _iter957.write(oprot);
+              _iter965.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -49818,9 +49818,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter958 : struct.success)
+            for (FieldSchema _iter966 : struct.success)
             {
-              _iter958.write(oprot);
+              _iter966.write(oprot);
             }
           }
         }
@@ -49841,14 +49841,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list959 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list959.size);
-            FieldSchema _elem960;
-            for (int _i961 = 0; _i961 < _list959.size; ++_i961)
+            org.apache.thrift.protocol.TList _list967 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list967.size);
+            FieldSchema _elem968;
+            for (int _i969 = 0; _i969 < _list967.size; ++_i969)
             {
-              _elem960 = new FieldSchema();
-              _elem960.read(iprot);
-              struct.success.add(_elem960);
+              _elem968 = new FieldSchema();
+              _elem968.read(iprot);
+              struct.success.add(_elem968);
             }
           }
           struct.setSuccessIsSet(true);
@@ -51002,14 +51002,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list962 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list962.size);
-                  FieldSchema _elem963;
-                  for (int _i964 = 0; _i964 < _list962.size; ++_i964)
+                  org.apache.thrift.protocol.TList _list970 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list970.size);
+                  FieldSchema _elem971;
+                  for (int _i972 = 0; _i972 < _list970.size; ++_i972)
                   {
-                    _elem963 = new FieldSchema();
-                    _elem963.read(iprot);
-                    struct.success.add(_elem963);
+                    _elem971 = new FieldSchema();
+                    _elem971.read(iprot);
+                    struct.success.add(_elem971);
                   }
                   iprot.readListEnd();
                 }
@@ -51062,9 +51062,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 _iter965 : struct.success)
+            for (FieldSchema _iter973 : struct.success)
             {
-              _iter965.write(oprot);
+              _iter973.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -51119,9 +51119,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter966 : struct.success)
+            for (FieldSchema _iter974 : struct.success)
             {
-              _iter966.write(oprot);
+              _iter974.write(oprot);
             }
           }
         }
@@ -51142,14 +51142,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list967 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list967.size);
-            FieldSchema _elem968;
-            for (int _i969 = 0; _i969 < _list967.size; ++_i969)
+            org.apache.thrift.protocol.TList _list975 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list975.size);
+            FieldSchema _elem976;
+            for (int _i977 = 0; _i977 < _list975.size; ++_i977)
             {
-              _elem968 = new FieldSchema();
-              _elem968.read(iprot);
-              struct.success.add(_elem968);
+              _elem976 = new FieldSchema();
+              _elem976.read(iprot);
+              struct.success.add(_elem976);
             }
           }
           struct.setSuccessIsSet(true);
@@ -54278,14 +54278,14 @@ import org.slf4j.LoggerFactory;
             case 2: // PRIMARY_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list970 = iprot.readListBegin();
-                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list970.size);
-                  SQLPrimaryKey _elem971;
-                  for (int _i972 = 0; _i972 < _list970.size; ++_i972)
+                  org.apache.thrift.protocol.TList _list978 = iprot.readListBegin();
+                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list978.size);
+                  SQLPrimaryKey _elem979;
+                  for (int _i980 = 0; _i980 < _list978.size; ++_i980)
                   {
-                    _elem971 = new SQLPrimaryKey();
-                    _elem971.read(iprot);
-                    struct.primaryKeys.add(_elem971);
+                    _elem979 = new SQLPrimaryKey();
+                    _elem979.read(iprot);
+                    struct.primaryKeys.add(_elem979);
                   }
                   iprot.readListEnd();
                 }
@@ -54297,14 +54297,14 @@ import org.slf4j.LoggerFactory;
             case 3: // FOREIGN_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list973 = iprot.readListBegin();
-                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list973.size);
-                  SQLForeignKey _elem974;
-                  for (int _i975 = 0; _i975 < _list973.size; ++_i975)
+                  org.apache.thrift.protocol.TList _list981 = iprot.readListBegin();
+                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list981.size);
+                  SQLForeignKey _elem982;
+                  for (int _i983 = 0; _i983 < _list981.size; ++_i983)
                   {
-                    _elem974 = new SQLForeignKey();
-                    _elem974.read(iprot);
-                    struct.foreignKeys.add(_elem974);
+                    _elem982 = new SQLForeignKey();
+                    _elem982.read(iprot);
+                    struct.foreignKeys.add(_elem982);
                   }
                   iprot.readListEnd();
                 }
@@ -54316,14 +54316,14 @@ import org.slf4j.LoggerFactory;
             case 4: // UNIQUE_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list976 = iprot.readListBegin();
-                  struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list976.size);
-                  SQLUniqueConstraint _elem977;
-                  for (int _i978 = 0; _i978 < _list976.size; ++_i978)
+                  org.apache.thrift.protocol.TList _list984 = iprot.readListBegin();
+                  struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list984.size);
+                  SQLUniqueConstraint _elem985;
+                  for (int _i986 = 0; _i986 < _list984.size; ++_i986)
                   {
-                    _elem977 = new SQLUniqueConstraint();
-                    _elem977.read(iprot);
-                    struct.uniqueConstraints.add(_elem977);
+                    _elem985 = new SQLUniqueConstraint();
+                    _elem985.read(iprot);
+                    struct.uniqueConstraints.add(_elem985);
                   }
                   iprot.readListEnd();
                 }
@@ -54335,14 +54335,14 @@ import org.slf4j.LoggerFactory;
             case 5: // NOT_NULL_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list979 = iprot.readListBegin();
-                  struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list979.size);
-                  SQLNotNullConstraint _elem980;
-                  for (int _i981 = 0; _i981 < _list979.size; ++_i981)
+                  org.apache.thrift.protocol.TList _list987 = iprot.readListBegin();
+                  struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list987.size);
+                  SQLNotNullConstraint _elem988;
+                  for (int _i989 = 0; _i989 < _list987.size; ++_i989)
                   {
-                    _elem980 = new SQLNotNullConstraint();
-                    _elem980.read(iprot);
-                    struct.notNullConstraints.add(_elem980);
+                    _elem988 = new SQLNotNullConstraint();
+                    _elem988.read(iprot);
+                    struct.notNullConstraints.add(_elem988);
                   }
                   iprot.readListEnd();
                 }
@@ -54354,14 +54354,14 @@ import org.slf4j.LoggerFactory;
             case 6: // DEFAULT_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list982 = iprot.readListBegin();
-                  struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list982.size);
-                  SQLDefaultConstraint _elem983;
-                  for (int _i984 = 0; _i984 < _list982.size; ++_i984)
+                  org.apache.thrift.protocol.TList _list990 = iprot.readListBegin();
+                  struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list990.size);
+                  SQLDefaultConstraint _elem991;
+                  for (int _i992 = 0; _i992 < _list990.size; ++_i992)
                   {
-                    _elem983 = new SQLDefaultConstraint();
-                    _elem983.read(iprot);
-                    struct.defaultConstraints.add(_elem983);
+                    _elem991 = new SQLDefaultConstraint();
+                    _elem991.read(iprot);
+                    struct.defaultConstraints.add(_elem991);
                   }
                   iprot.readListEnd();
                 }
@@ -54373,14 +54373,14 @@ import org.slf4j.LoggerFactory;
             case 7: // CHECK_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list985 = iprot.readListBegin();
-                  struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list985.size);
-                  SQLCheckConstraint _elem986;
-                  for (int _i987 = 0; _i987 < _list985.size; ++_i987)
+                  org.apache.thrift.protocol.TList _list993 = iprot.readListBegin();
+                  struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list993.size);
+                  SQLCheckConstraint _elem994;
+                  for (int _i995 = 0; _i995 < _list993.size; ++_i995)
                   {
-                    _elem986 = new SQLCheckConstraint();
-                    _elem986.read(iprot);
-                    struct.checkConstraints.add(_elem986);
+                    _elem994 = new SQLCheckConstraint();
+                    _elem994.read(iprot);
+                    struct.checkConstraints.add(_elem994);
                   }
                   iprot.readListEnd();
                 }
@@ -54411,9 +54411,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 _iter988 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter996 : struct.primaryKeys)
             {
-              _iter988.write(oprot);
+              _iter996.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -54423,9 +54423,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 _iter989 : struct.foreignKeys)
+            for (SQLForeignKey _iter997 : struct.foreignKeys)
             {
-              _iter989.write(oprot);
+              _iter997.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -54435,9 +54435,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 _iter990 : struct.uniqueConstraints)
+            for (SQLUniqueConstraint _iter998 : struct.uniqueConstraints)
             {
-              _iter990.write(oprot);
+              _iter998.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -54447,9 +54447,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 _iter991 : struct.notNullConstraints)
+            for (SQLNotNullConstraint _iter999 : struct.notNullConstraints)
             {
-              _iter991.write(oprot);
+              _iter999.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -54459,9 +54459,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(DEFAULT_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.defaultConstraints.size()));
-            for (SQLDefaultConstraint _iter992 : struct.defaultConstraints)
+            for (SQLDefaultConstraint _iter1000 : struct.defaultConstraints)
             {
-              _iter992.write(oprot);
+              _iter1000.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -54471,9 +54471,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(CHECK_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.checkConstraints.size()));
-            for (SQLCheckConstraint _iter993 : struct.checkConstraints)
+            for (SQLCheckConstraint _iter1001 : struct.checkConstraints)
             {
-              _iter993.write(oprot);
+              _iter1001.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -54525,54 +54525,54 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPrimaryKeys()) {
           {
             oprot.writeI32(struct.primaryKeys.size());
-            for (SQLPrimaryKey _iter994 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter1002 : struct.primaryKeys)
             {
-              _iter994.write(oprot);
+              _iter1002.write(oprot);
             }
           }
         }
         if (struct.isSetForeignKeys()) {
           {
             oprot.writeI32(struct.foreignKeys.size());
-            for (SQLForeignKey _iter995 : struct.foreignKeys)
+            for (SQLForeignKey _iter1003 : struct.foreignKeys)
             {
-              _iter995.write(oprot);
+              _iter1003.write(oprot);
             }
           }
         }
         if (struct.isSetUniqueConstraints()) {
           {
             oprot.writeI32(struct.uniqueConstraints.size());
-            for (SQLUniqueConstraint _iter996 : struct.uniqueConstraints)
+            for (SQLUniqueConstraint _iter1004 : struct.uniqueConstraints)
             {
-              _iter996.write(oprot);
+              _iter1004.write(oprot);
             }
           }
         }
         if (struct.isSetNotNullConstraints()) {
           {
             oprot.writeI32(struct.notNullConstraints.size());
-            for (SQLNotNullConstraint _iter997 : struct.notNullConstraints)
+            for (SQLNotNullConstraint _iter1005 : struct.notNullConstraints)
             {
-              _iter997.write(oprot);
+              _iter1005.write(oprot);
             }
           }
         }
         if (struct.isSetDefaultConstraints()) {
           {
             oprot.writeI32(struct.defaultConstraints.size());
-            for (SQLDefaultConstraint _iter998 : struct.defaultConstraints)
+            for (SQLDefaultConstraint _iter1006 : struct.defaultConstraints)
             {
-              _iter998.write(oprot);
+              _iter1006.write(oprot);
             }
           }
         }
         if (struct.isSetCheckConstraints()) {
           {
             oprot.writeI32(struct.checkConstraints.size());
-            for (SQLCheckConstraint _iter999 : struct.checkConstraints)
+            for (SQLCheckConstraint _iter1007 : struct.checkConstraints)
             {
-              _iter999.write(oprot);
+              _iter1007.write(oprot);
             }
           }
         }
@@ -54589,84 +54589,84 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list1000 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1000.size);
-            SQLPrimaryKey _elem1001;
-            for (int _i1002 = 0; _i1002 < _list1000.size; ++_i1002)
+            org.apache.thrift.protocol.TList _list1008 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1008.size);
+            SQLPrimaryKey _elem1009;
+            for (int _i1010 = 0; _i1010 < _list1008.size; ++_i1010)
             {
-              _elem1001 = new SQLPrimaryKey();
-              _elem1001.read(iprot);
-              struct.primaryKeys.add(_elem1001);
+              _elem1009 = new SQLPrimaryKey();
+              _elem1009.read(iprot);
+              struct.primaryKeys.add(_elem1009);
             }
           }
           struct.setPrimaryKeysIsSet(true);
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1003 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1003.size);
-            SQLForeignKey _elem1004;
-            for (int _i1005 = 0; _i1005 < _list1003.size; ++_i1005)
+            org.apache.thrift.protocol.TList _list1011 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1011.size);
+            SQLForeignKey _elem1012;
+            for (int _i1013 = 0; _i1013 < _list1011.size; ++_i1013)
             {
-              _elem1004 = new SQLForeignKey();
-              _elem1004.read(iprot);
-              struct.foreignKeys.add(_elem1004);
+              _elem1012 = new SQLForeignKey();
+              _elem1012.read(iprot);
+              struct.foreignKeys.add(_elem1012);
             }
           }
           struct.setForeignKeysIsSet(true);
         }
         if (incoming.get(3)) {
           {
-            org.apache.thrift.protocol.TList _list1006 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1006.size);
-            SQLUniqueConstraint _elem1007;
-            for (int _i1008 = 0; _i1008 < _list1006.size; ++_i1008)
+            org.apache.thrift.protocol.TList _list1014 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1014.size);
+            SQLUniqueConstraint _elem1015;
+            for (int _i1016 = 0; _i1016 < _list1014.size; ++_i1016)
             {
-              _elem1007 = new SQLUniqueConstraint();
-              _elem1007.read(iprot);
-              struct.uniqueConstraints.add(_elem1007);
+              _elem1015 = new SQLUniqueConstraint();
+              _elem1015.read(iprot);
+              struct.uniqueConstraints.add(_elem1015);
             }
           }
           struct.setUniqueConstraintsIsSet(true);
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol.TList _list1009 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1009.size);
-            SQLNotNullConstraint _elem1010;
-            for (int _i1011 = 0; _i1011 < _list1009.size; ++_i1011)
+            org.apache.thrift.protocol.TList _list1017 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1017.size);
+            SQLNotNullConstraint _elem1018;
+            for (int _i1019 = 0; _i1019 < _list1017.size; ++_i1019)
             {
-              _elem1010 = new SQLNotNullConstraint();
-              _elem1010.read(iprot);
-              struct.notNullConstraints.add(_elem1010);
+              _elem1018 = new SQLNotNullConstraint();
+              _elem1018.read(iprot);
+              struct.notNullConstraints.add(_elem1018);
             }
           }
           struct.setNotNullConstraintsIsSet(true);
         }
         if (incoming.get(5)) {
           {
-            org.apache.thrift.protocol.TList _list1012 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1012.size);
-            SQLDefaultConstraint _elem1013;
-            for (int _i1014 = 0; _i1014 < _list1012.size; ++_i1014)
+            org.apache.thrift.protocol.TList _list1020 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1020.size);
+            SQLDefaultConstraint _elem1021;
+            for (int _i1022 = 0; _i1022 < _list1020.size; ++_i1022)
             {
-              _elem1013 = new SQLDefaultConstraint();
-              _elem1013.read(iprot);
-              struct.defaultConstraints.add(_elem1013);
+              _elem1021 = new SQLDefaultConstraint();
+              _elem1021.read(iprot);
+              struct.defaultConstraints.add(_elem1021);
             }
           }
           struct.setDefaultConstraintsIsSet(true);
         }
         if (incoming.get(6)) {
           {
-            org.apache.thrift.protocol.TList _list1015 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1015.size);
-            SQLCheckConstraint _elem1016;
-            for (int _i1017 = 0; _i1017 < _list1015.size; ++_i1017)
+            org.apache.thrift.protocol.TList _list1023 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1023.size);
+            SQLCheckConstraint _elem1024;
+            for (int _i1025 = 0; _i1025 < _list1023.size; ++_i1025)
             {
-              _elem1016 = new SQLCheckConstraint();
-              _elem1016.read(iprot);
-              struct.checkConstraints.add(_elem1016);
+              _elem1024 = new SQLCheckConstraint();
+              _elem1024.read(iprot);
+              struct.checkConstraints.add(_elem1024);
             }
           }
           struct.setCheckConstraintsIsSet(true);
@@ -63816,13 +63816,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1018 = iprot.readListBegin();
-                  struct.partNames = new ArrayList<String>(_list1018.size);
-                  String _elem1019;
-                  for (int _i1020 = 0; _i1020 < _list1018.size; ++_i1020)
+                  org.apache.thrift.protocol.TList _list1026 = iprot.readListBegin();
+                  struct.partNames = new ArrayList<String>(_list1026.size);
+                  String _elem1027;
+                  for (int _i1028 = 0; _i1028 < _list1026.size; ++_i1028)
                   {
-                    _elem1019 = iprot.readString();
-                    struct.partNames.add(_elem1019);
+                    _elem1027 = iprot.readString();
+                    struct.partNames.add(_elem1027);
                   }
                   iprot.readListEnd();
                 }
@@ -63858,9 +63858,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 _iter1021 : struct.partNames)
+            for (String _iter1029 : struct.partNames)
             {
-              oprot.writeString(_iter1021);
+              oprot.writeString(_iter1029);
             }
             oprot.writeListEnd();
           }
@@ -63903,9 +63903,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPartNames()) {
           {
             oprot.writeI32(struct.partNames.size());
-            for (String _iter1022 : struct.partNames)
+            for (String _iter1030 : struct.partNames)
             {
-              oprot.writeString(_iter1022);
+              oprot.writeString(_iter1030);
             }
           }
         }
@@ -63925,13 +63925,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1023 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.partNames = new ArrayList<String>(_list1023.size);
-            String _elem1024;
-            for (int _i1025 = 0; _i1025 < _list1023.size; ++_i1025)
+            org.apache.thrift.protocol.TList _list1031 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.partNames = new ArrayList<String>(_list1031.size);
+            String _elem1032;
+            for (int _i1033 = 0; _i1033 < _list1031.size; ++_i1033)
             {
-              _elem1024 = iprot.readString();
-              struct.partNames.add(_elem1024);
+              _elem1032 = iprot.readString();
+              struct.partNames.add(_elem1032);
             }
           }
           struct.setPartNamesIsSet(true);
@@ -65156,13 +65156,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1026 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1026.size);
-                  String _elem1027;
-                  for (int _i1028 = 0; _i1028 < _list1026.size; ++_i1028)
+                  org.apache.thrift.protocol.TList _list1034 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1034.size);
+                  String _elem1035;
+                  for (int _i1036 = 0; _i1036 < _list1034.size; ++_i1036)
                   {
-                    _elem1027 = iprot.readString();
-                    struct.success.add(_elem1027);
+                    _elem1035 = iprot.readString();
+                    struct.success.add(_elem1035);
                   }
                   iprot.readListEnd();
                 }
@@ -65197,9 +65197,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 _iter1029 : struct.success)
+            for (String _iter1037 : struct.success)
             {
-              oprot.writeString(_iter1029);
+              oprot.writeString(_iter1037);
             }
             oprot.writeListEnd();
           }
@@ -65238,9 +65238,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1030 : struct.success)
+            for (String _iter1038 : struct.success)
             {
-              oprot.writeString(_iter1030);
+              oprot.writeString(_iter1038);
             }
           }
         }
@@ -65255,13 +65255,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1031 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1031.size);
-            String _elem1032;
-            for (int _i1033 = 0; _i1033 < _list1031.size; ++_i1033)
+            org.apache.thrift.protocol.TList _list1039 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1039.size);
+            String _elem1040;
+            for (int _i1041 = 0; _i1041 < _list1039.size; ++_i1041)
             {
-              _elem1032 = iprot.readString();
-              struct.success.add(_elem1032);
+              _elem1040 = iprot.readString();
+              struct.success.add(_elem1040);
             }
           }
           struct.setSuccessIsSet(true);
@@ -66235,13 +66235,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1034 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1034.size);
-                  String _elem1035;
-                  for (int _i1036 = 0; _i1036 < _list1034.size; ++_i1036)
+                  org.apache.thrift.protocol.TList _list1042 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1042.size);
+                  String _elem1043;
+                  for (int _i1044 = 0; _i1044 < _list1042.size; ++_i1044)
                   {
-                    _elem1035 = iprot.readString();
-                    struct.success.add(_elem1035);
+                    _elem1043 = iprot.readString();
+                    struct.success.add(_elem1043);
                   }
                   iprot.readListEnd();
                 }
@@ -66276,9 +66276,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 _iter1037 : struct.success)
+            for (String _iter1045 : struct.success)
             {
-              oprot.writeString(_iter1037);
+              oprot.writeString(_iter1045);
             }
             oprot.writeListEnd();
           }
@@ -66317,9 +66317,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1038 : struct.success)
+            for (String _iter1046 : struct.success)
             {
-              oprot.writeString(_iter1038);
+              oprot.writeString(_iter1046);
             }
           }
         }
@@ -66334,13 +66334,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1039 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1039.size);
-            String _elem1040;
-            for (int _i1041 = 0; _i1041 < _list1039.size; ++_i1041)
+            org.apache.thrift.protocol.TList _list1047 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1047.size);
+            String _elem1048;
+            for (int _i1049 = 0; _i1049 < _list1047.size; ++_i1049)
             {
-              _elem1040 = iprot.readString();
-              struct.success.add(_elem1040);
+              _elem1048 = iprot.readString();
+              struct.success.add(_elem1048);
             }
           }
           struct.setSuccessIsSet(true);
@@ -67106,13 +67106,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1042 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1042.size);
-                  String _elem1043;
-                  for (int _i1044 = 0; _i1044 < _list1042.size; ++_i1044)
+                  org.apache.thrift.protocol.TList _list1050 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1050.size);
+                  String _elem1051;
+                  for (int _i1052 = 0; _i1052 < _list1050.size; ++_i1052)
                   {
-                    _elem1043 = iprot.readString();
-                    struct.success.add(_elem1043);
+                    _elem1051 = iprot.readString();
+                    struct.success.add(_elem1051);
                   }
                   iprot.readListEnd();
                 }
@@ -67147,9 +67147,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 _iter1045 : struct.success)
+            for (String _iter1053 : struct.success)
             {
-              oprot.writeString(_iter1045);
+              oprot.writeString(_iter1053);
             }
             oprot.writeListEnd();
           }
@@ -67188,9 +67188,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1046 : struct.success)
+            for (String _iter1054 : struct.success)
             {
-              oprot.writeString(_iter1046);
+              oprot.writeString(_iter1054);
             }
           }
         }
@@ -67205,13 +67205,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1047 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1047.size);
-            String _elem1048;
-            for (int _i1049 = 0; _i1049 < _list1047.size; ++_i1049)
+            org.apache.thrift.protocol.TList _list1055 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1055.size);
+            String _elem1056;
+            for (int _i1057 = 0; _i1057 < _list1055.size; ++_i1057)
             {
-              _elem1048 = iprot.readString();
-              struct.success.add(_elem1048);
+              _elem1056 = iprot.readString();
+              struct.success.add(_elem1056);
             }
           }
           struct.setSuccessIsSet(true);
@@ -67716,13 +67716,13 @@ import org.slf4j.LoggerFactory;
             case 3: // TBL_TYPES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1050 = iprot.readListBegin();
-                  struct.tbl_types = new ArrayList<String>(_list1050.size);
-                  String _elem1051;
-                  for (int _i1052 = 0; _i1052 < _list1050.size; ++_i1052)
+                  org.apache.thrift.protocol.TList _list1058 = iprot.readListBegin();
+                  struct.tbl_types = new ArrayList<String>(_list1058.size);
+                  String _elem1059;
+                  for (int _i1060 = 0; _i1060 < _list1058.size; ++_i1060)
                   {
-                    _elem1051 = iprot.readString();
-                    struct.tbl_types.add(_elem1051);
+                    _elem1059 = iprot.readString();
+                    struct.tbl_types.add(_elem1059);
                   }
                   iprot.readListEnd();
                 }
@@ -67758,9 +67758,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 _iter1053 : struct.tbl_types)
+            for (String _iter1061 : struct.tbl_types)
             {
-              oprot.writeString(_iter1053);
+              oprot.writeString(_iter1061);
             }
             oprot.writeListEnd();
           }
@@ -67803,9 +67803,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetTbl_types()) {
           {
             oprot.writeI32(struct.tbl_types.size());
-            for (String _iter1054 : struct.tbl_types)
+            for (String _iter1062 : struct.tbl_types)
             {
-              oprot.writeString(_iter1054);
+              oprot.writeString(_iter1062);
             }
           }
         }
@@ -67825,13 +67825,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1055 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_types = new ArrayList<String>(_list1055.size);
-            String _elem1056;
-            for (int _i1057 = 0; _i1057 < _list1055.size; ++_i1057)
+            org.apache.thrift.protocol.TList _list1063 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_types = new ArrayList<String>(_list1063.size);
+            String _elem1064;
+            for (int _i1065 = 0; _i1065 < _list1063.size; ++_i1065)
             {
-              _elem1056 = iprot.readString();
-              struct.tbl_types.add(_elem1056);
+              _elem1064 = iprot.readString();
+              struct.tbl_types.add(_elem1064);
             }
           }
           struct.setTbl_typesIsSet(true);
@@ -68237,14 +68237,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1058 = iprot.readListBegin();
-                  struct.success = new ArrayList<TableMeta>(_list1058.size);
-                  TableMeta _elem1059;
-                  for (int _i1060 = 0; _i1060 < _list1058.size; ++_i1060)
+                  org.apache.thrift.protocol.TList _list1066 = iprot.readListBegin();
+                  struct.success = new ArrayList<TableMeta>(_list1066.size);
+                  TableMeta _elem1067;
+                  for (int _i1068 = 0; _i1068 < _list1066.size; ++_i1068)
                   {
-                    _elem1059 = new TableMeta();
-                    _elem1059.read(iprot);
-                    struct.success.add(_elem1059);
+                    _elem1067 = new TableMeta();
+                    _elem1067.read(iprot);
+                    struct.success.add(_elem1067);
                   }
                   iprot.readListEnd();
                 }
@@ -68279,9 +68279,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 _iter1061 : struct.success)
+            for (TableMeta _iter1069 : struct.success)
             {
-              _iter1061.write(oprot);
+              _iter1069.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -68320,9 +68320,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (TableMeta _iter1062 : struct.success)
+            for (TableMeta _iter1070 : struct.success)
             {
-              _iter1062.write(oprot);
+              _iter1070.write(oprot);
             }
           }
         }
@@ -68337,14 +68337,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1063 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<TableMeta>(_list1063.size);
-            TableMeta _elem1064;
-            for (int _i1065 = 0; _i1065 < _list1063.size; ++_i1065)
+            org.apache.thrift.protocol.TList _list1071 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<TableMeta>(_list1071.size);
+            TableMeta _elem1072;
+            for (int _i1073 = 0; _i1073 < _list1071.size; ++_i1073)
             {
-              _elem1064 = new TableMeta();
-              _elem1064.read(iprot);
-              struct.success.add(_elem1064);
+              _elem1072 = new TableMeta();
+              _elem1072.read(iprot);
+              struct.success.add(_elem1072);
             }
           }
           struct.setSuccessIsSet(true);
@@ -69110,13 +69110,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1066 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1066.size);
-                  String _elem1067;
-                  for (int _i1068 = 0; _i1068 < _list1066.size; ++_i1068)
+                  org.apache.thrift.protocol.TList _list1074 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1074.size);
+                  String _elem1075;
+                  for (int _i1076 = 0; _i1076 < _list1074.size; ++_i1076)
                   {
-                    _elem1067 = iprot.readString();
-                    struct.success.add(_elem1067);
+                    _elem1075 = iprot.readString();
+                    struct.success.add(_elem1075);
                   }
                   iprot.readListEnd();
                 }
@@ -69151,9 +69151,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 _iter1069 : struct.success)
+            for (String _iter1077 : struct.success)
             {
-              oprot.writeString(_iter1069);
+              oprot.writeString(_iter1077);
             }
             oprot.writeListEnd();
           }
@@ -69192,9 +69192,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1070 : struct.success)
+            for (String _iter1078 : struct.success)
             {
-              oprot.writeString(_iter1070);
+              oprot.writeString(_iter1078);
             }
           }
         }
@@ -69209,13 +69209,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1071 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1071.size);
-            String _elem1072;
-            for (int _i1073 = 0; _i1073 < _list1071.size; ++_i1073)
+            org.apache.thrift.protocol.TList _list1079 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1079.size);
+            String _elem1080;
+            for (int _i1081 = 0; _i1081 < _list1079.size; ++_i1081)
             {
-              _elem1072 = iprot.readString();
-              struct.success.add(_elem1072);
+              _elem1080 = iprot.readString();
+              struct.success.add(_elem1080);
             }
           }
           struct.setSuccessIsSet(true);
@@ -70668,13 +70668,13 @@ import org.slf4j.LoggerFactory;
             case 2: // TBL_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1074 = iprot.readListBegin();
-                  struct.tbl_names = new ArrayList<String>(_list1074.size);
-                  String _elem1075;
-                  for (int _i1076 = 0; _i1076 < _list1074.size; ++_i1076)
+                  org.apache.thrift.protocol.TList _list1082 = iprot.readListBegin();
+                  struct.tbl_names = new ArrayList<String>(_list1082.size);
+                  String _elem1083;
+                  for (int _i1084 = 0; _i1084 < _list1082.size; ++_i1084)
                   {
-                    _elem1075 = iprot.readString();
-                    struct.tbl_names.add(_elem1075);
+                    _elem1083 = iprot.readString();
+                    struct.tbl_names.add(_elem1083);
                   }
                   iprot.readListEnd();
                 }
@@ -70705,9 +70705,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 _iter1077 : struct.tbl_names)
+            for (String _iter1085 : struct.tbl_names)
             {
-              oprot.writeString(_iter1077);
+              oprot.writeString(_iter1085);
             }
             oprot.writeListEnd();
           }
@@ -70744,9 +70744,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetTbl_names()) {
           {
             oprot.writeI32(struct.tbl_names.size());
-            for (String _iter1078 : struct.tbl_names)
+            for (String _iter1086 : struct.tbl_names)
             {
-              oprot.writeString(_iter1078);
+              oprot.writeString(_iter1086);
             }
           }
         }
@@ -70762,13 +70762,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list1079 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_names = new ArrayList<String>(_list1079.size);
-            String _elem1080;
-            for (int _i1081 = 0; _i1081 < _list1079.size; ++_i1081)
+            org.apache.thrift.protocol.TList _list1087 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_names = new ArrayList<String>(_list1087.size);
+            String _elem1088;
+            for (int _i1089 = 0; _i1089 < _list1087.size; ++_i1089)
             {
-              _elem1080 = iprot.readString();
-              struct.tbl_names.add(_elem1080);
+              _elem1088 = iprot.readString();
+              struct.tbl_names.add(_elem1088);
             }
           }
           struct.setTbl_namesIsSet(true);
@@ -71093,14 +71093,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1082 = iprot.readListBegin();
-                  struct.success = new ArrayList<Table>(_list1082.size);
-                  Table _elem1083;
-                  for (int _i1084 = 0; _i1084 < _list1082.size; ++_i1084)
+                  org.apache.thrift.protocol.TList _list1090 = iprot.readListBegin();
+                  struct.success = new ArrayList<Table>(_list1090.size);
+                  Table _elem1091;
+                  for (int _i1092 = 0; _i1092 < _list1090.size; ++_i1092)
                   {
-                    _elem1083 = new Table();
-                    _elem1083.read(iprot);
-                    struct.success.add(_elem1083);
+                    _elem1091 = new Table();
+                    _elem1091.read(iprot);
+                    struct.success.add(_elem1091);
                   }
                   iprot.readListEnd();
                 }
@@ -71126,9 +71126,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 _iter1085 : struct.success)
+            for (Table _iter1093 : struct.success)
             {
-              _iter1085.write(oprot);
+              _iter1093.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -71159,9 +71159,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Table _iter1086 : struct.success)
+            for (Table _iter1094 : struct.success)
             {
-              _iter1086.write(oprot);
+              _iter1094.write(oprot);
             }
           }
         }
@@ -71173,14 +71173,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1087 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Table>(_list1087.size);
-            Table _elem1088;
-            for (int _i1089 = 0; _i1089 < _list1087.size; ++_i1089)
+            org.apache.thrift.protocol.TList _list1095 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Table>(_list1095.size);
+            Table _elem1096;
+            for (int _i1097 = 0; _i1097 < _list1095.size; ++_i1097)
             {
-              _elem1088 = new Table();
-              _elem1088.read(iprot);
-              struct.success.add(_elem1088);
+              _elem1096 = new Table();
+              _elem1096.read(iprot);
+              struct.success.add(_elem1096);
             }
           }
           struct.setSuccessIsSet(true);
@@ -73573,13 +73573,13 @@ import org.slf4j.LoggerFactory;
             case 2: // TBL_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1090 = iprot.readListBegin();
-                  struct.tbl_names = new ArrayList<String>(_list1090.size);
-                  String _elem1091;
-                  for (int _i1092 = 0; _i1092 < _list1090.size; ++_i1092)
+                  org.apache.thrift.protocol.TList _list1098 = iprot.readListBegin();
+                  struct.tbl_names = new ArrayList<String>(_list1098.size);
+                  String _elem1099;
+                  for (int _i1100 = 0; _i1100 < _list1098.size; ++_i1100)
                   {
-                    _elem1091 = iprot.readString();
-                    struct.tbl_names.add(_elem1091);
+                    _elem1099 = iprot.readString();
+                    struct.tbl_names.add(_elem1099);
                   }
                   iprot.readListEnd();
                 }
@@ -73610,9 +73610,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 _iter1093 : struct.tbl_names)
+            for (String _iter1101 : struct.tbl_names)
             {
-              oprot.writeString(_iter1093);
+              oprot.writeString(_iter1101);
             }
             oprot.writeListEnd();
           }
@@ -73649,9 +73649,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetTbl_names()) {
           {
             oprot.writeI32(struct.tbl_names.size());
-            for (String _iter1094 : struct.tbl_names)
+            for (String _iter1102 : struct.tbl_names)
             {
-              oprot.writeString(_iter1094);
+              oprot.writeString(_iter1102);
             }
           }
         }
@@ -73667,13 +73667,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list1095 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_names = new ArrayList<String>(_list1095.size);
-            String _elem1096;
-            for (int _i1097 = 0; _i1097 < _list1095.size; ++_i1097)
+            org.apache.thrift.protocol.TList _list1103 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_names = new ArrayList<String>(_list1103.size);
+            String _elem1104;
+            for (int _i1105 = 0; _i1105 < _list1103.size; ++_i1105)
             {
-              _elem1096 = iprot.readString();
-              struct.tbl_names.add(_elem1096);
+              _elem1104 = iprot.readString();
+              struct.tbl_names.add(_elem1104);
             }
           }
           struct.setTbl_namesIsSet(true);
@@ -74246,16 +74246,16 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map1098 = iprot.readMapBegin();
-                  struct.success = new HashMap<String,Materialization>(2*_map1098.size);
-                  String _key1099;
-                  Materialization _val1100;
-                  for (int _i1101 = 0; _i1101 < _map1098.size; ++_i1101)
+                  org.apache.thrift.protocol.TMap _map1106 = iprot.readMapBegin();
+                  struct.success = new HashMap<String,Materialization>(2*_map1106.size);
+                  String _key1107;
+                  Materialization _val1108;
+                  for (int _i1109 = 0; _i1109 < _map1106.size; ++_i1109)
                   {
-                    _key1099 = iprot.readString();
-                    _val1100 = new Materialization();
-                    _val1100.read(iprot);
-                    struct.success.put(_key1099, _val1100);
+                    _key1107 = iprot.readString();
+                    _val1108 = new Materialization();
+                    _val1108.read(iprot);
+                    struct.success.put(_key1107, _val1108);
                   }
                   iprot.readMapEnd();
                 }
@@ -74308,10 +74308,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> _iter1102 : struct.success.entrySet())
+            for (Map.Entry<String, Materialization> _iter1110 : struct.success.entrySet())
             {
-              oprot.writeString(_iter1102.getKey());
-              _iter1102.getValue().write(oprot);
+              oprot.writeString(_iter1110.getKey());
+              _iter1110.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -74366,10 +74366,10 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Map.Entry<String, Materialization> _iter1103 : struct.success.entrySet())
+            for (Map.Entry<String, Materialization> _iter1111 : struct.success.entrySet())
             {
-              oprot.writeString(_iter1103.getKey());
-              _iter1103.getValue().write(oprot);
+              oprot.writeString(_iter1111.getKey());
+              _iter1111.getValue().write(oprot);
             }
           }
         }
@@ -74390,16 +74390,16 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map1104 = 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,Materialization>(2*_map1104.size);
-            String _key1105;
-            Materialization _val1106;
-            for (int _i1107 = 0; _i1107 < _map1104.size; ++_i1107)
+            org.apache.thrift.protocol.TMap _map1112 = 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,Materialization>(2*_map1112.size);
+            String _key1113;
+            Materialization _val1114;
+            for (int _i1115 = 0; _i1115 < _map1112.size; ++_i1115)
             {
-              _key1105 = iprot.readString();
-              _val1106 = new Materialization();
-              _val1106.read(iprot);
-              struct.success.put(_key1105, _val1106);
+              _key1113 = iprot.readString();
+              _val1114 = new Materialization();
+              _val1114.read(iprot);
+              struct.success.put(_key1113, _val1114);
             }
           }
           struct.setSuccessIsSet(true);
@@ -76792,13 +76792,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1108 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1108.size);
-                  String _elem1109;
-                  for (int _i1110 = 0; _i1110 < _list1108.size; ++_i1110)
+                  org.apache.thrift.protocol.TList _list1116 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1116.size);
+                  String _elem1117;
+                  for (int _i1118 = 0; _i1118 < _list1116.size; ++_i1118)
                   {
-                    _elem1109 = iprot.readString();
-                    struct.success.add(_elem1109);
+                    _elem1117 = iprot.readString();
+                    struct.success.add(_elem1117);
                   }
                   iprot.readListEnd();
                 }
@@ -76851,9 +76851,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 _iter1111 : struct.success)
+            for (String _iter1119 : struct.success)
             {
-              oprot.writeString(_iter1111);
+              oprot.writeString(_iter1119);
             }
             oprot.writeListEnd();
           }
@@ -76908,9 +76908,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1112 : struct.success)
+            for (String _iter1120 : struct.success)
             {
-              oprot.writeString(_iter1112);
+              oprot.writeString(_iter1120);
             }
           }
         }
@@ -76931,13 +76931,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1113 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1113.size);
-            String _elem1114;
-            for (int _i1115 = 0; _i1115 < _list1113.size; ++_i1115)
+            org.apache.thrift.protocol.TList _list1121 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1121.size);
+            String _elem1122;
+            for (int _i1123 = 0; _i1123 < _list1121.size; ++_i1123)
             {
-              _elem1114 = iprot.readString();
-              struct.success.add(_elem1114);
+              _elem1122 = iprot.readString();
+              struct.success.add(_elem1122);
             }
           }
           struct.setSuccessIsSet(true);
@@ -82796,14 +82796,14 @@ import org.slf4j.LoggerFactory;
             case 1: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1116 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<Partition>(_list1116.size);
-                  Partition _elem1117;
-                  for (int _i1118 = 0; _i1118 < _list1116.size; ++_i1118)
+                  org.apache.thrift.protocol.TList _list1124 = iprot.readListBegin();
+                  struct.new_parts = new ArrayList<Partition>(_list1124.size);
+                  Partition _elem1125;
+                  for (int _i1126 = 0; _i1126 < _list1124.size; ++_i1126)
                   {
-                    _elem1117 = new Partition();
-                    _elem1117.read(iprot);
-                    struct.new_parts.add(_elem1117);
+                    _elem1125 = new Partition();
+                    _elem1125.read(iprot);
+                    struct.new_parts.add(_elem1125);
                   }
                   iprot.readListEnd();
                 }
@@ -82829,9 +82829,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size()));
-            for (Partition _iter1119 : struct.new_parts)
+            for (Partition _iter1127 : struct.new_parts)
             {
-              _iter1119.write(oprot);
+              _iter1127.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -82862,9 +82862,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (Partition _iter1120 : struct.new_parts)
+            for (Partition _iter1128 : struct.new_parts)
             {
-              _iter1120.write(oprot);
+              _iter1128.write(oprot);
             }
           }
         }
@@ -82876,14 +82876,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1121 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.new_parts = new ArrayList<Partition>(_list1121.size);
-            Partition _elem1122;
-            for (int _i1123 = 0; _i1123 < _list1121.size; ++_i1123)
+            org.apache.thrift.protocol.TList _list1129 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.new_parts = new ArrayList<Partition>(_list1129.size);
+            Partition _elem1130;
+            for (int _i1131 = 0; _i1131 < _list1129.size; ++_i1131)
             {
-              _elem1122 = new Partition();
-              _elem1122.read(iprot);
-              struct.new_parts.add(_elem1122);
+              _elem1130 = new Partition();
+              _elem1130.read(iprot);
+              struct.new_parts.add(_elem1130);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -83884,14 +83884,14 @@ import org.slf4j.LoggerFactory;
             case 1: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1124 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<PartitionSpec>(_list1124.size);
-                  PartitionSpec _elem1125;
-                  for (int _i1126 = 0; _i1126 < _list1124.size; ++_i1126)
+                  org.apache.thrift.protocol.TList _list1132 = iprot.readListBegin();
+                  struct.new_parts = new ArrayList<PartitionSpec>(_list1132.size);
+                  PartitionSpec _elem1133;
+                  for (int _i1134 = 0; _i1134 < _list1132.size; ++_i1134)
                   {
-                    _elem1125 = new PartitionSpec();
-                    _elem1125.read(iprot);
-                    struct.new_parts.add(_elem1125);
+                    _elem1133 = new PartitionSpec();
+                    _elem1133.read(iprot);
+                    struct.new_parts.add(_elem1133);
                   }
                   iprot.readListEnd();
                 }
@@ -83917,9 +83917,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size()));
-            for (PartitionSpec _iter1127 : struct.new_parts)
+            for (PartitionSpec _iter1135 : struct.new_parts)
             {
-              _iter1127.write(oprot);
+              _iter1135.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -83950,9 +83950,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (PartitionSpec _iter1128 : struct.new_parts)
+            for (PartitionSpec _iter1136 : struct.new_parts)
             {
-              _iter1128.write(oprot);
+              _iter1136.write(oprot);
             }
           }
         }
@@ -83964,14 +83964,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1129 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.new_parts = new ArrayList<PartitionSpec>(_list1129.size);
-            PartitionSpec _elem1130;
-            for (int _i1131 = 0; _i1131 < _list1129.size; ++_i1131)
+            org.apache.thrift.protocol.TList _list1137 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.new_parts = new ArrayList<PartitionSpec>(_list1137.size);
+            PartitionSpec _elem1138;
+            for (int _i1139 = 0; _i1139 < _list1137.size; ++_i1139)
             {
-              _elem1130 = new PartitionSpec();
-              _elem1130.read(iprot);
-              struct.new_parts.add(_elem1130);
+              _elem1138 = new PartitionSpec();
+              _elem1138.read(iprot);
+              struct.new_parts.add(_elem1138);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -85147,13 +85147,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1132 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list1132.size);
-                  String _elem1133;
-                  for (int _i1134 = 0; _i1134 < _list1132.size; ++_i1134)
+                  org.apache.thrift.protocol.TList _list1140 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list1140.size);
+                  String _elem1141;
+                  for (int _i1142 = 0; _i1142 < _list1140.size; ++_i1142)
                   {
-                    _elem1133 = iprot.readString();
-                    struct.part_vals.add(_elem1133);
+                    _elem1141 = iprot.readString();
+                    struct.part_vals.add(_elem1141);
                   }
                   iprot.readListEnd();
                 }
@@ -85189,9 +85189,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.

<TRUNCATED>

[10/12] hive git commit: HIVE-19089: Create/Replicate Allocate write-id event (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 1904047..9902427 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
@@ -16979,10 +16979,6 @@ AllocateTableWriteIdsRequest::~AllocateTableWriteIdsRequest() throw() {
 }
 
 
-void AllocateTableWriteIdsRequest::__set_txnIds(const std::vector<int64_t> & val) {
-  this->txnIds = val;
-}
-
 void AllocateTableWriteIdsRequest::__set_dbName(const std::string& val) {
   this->dbName = val;
 }
@@ -16991,6 +16987,21 @@ void AllocateTableWriteIdsRequest::__set_tableName(const std::string& val) {
   this->tableName = val;
 }
 
+void AllocateTableWriteIdsRequest::__set_txnIds(const std::vector<int64_t> & val) {
+  this->txnIds = val;
+__isset.txnIds = true;
+}
+
+void AllocateTableWriteIdsRequest::__set_replPolicy(const std::string& val) {
+  this->replPolicy = val;
+__isset.replPolicy = true;
+}
+
+void AllocateTableWriteIdsRequest::__set_srcTxnToWriteIdList(const std::vector<TxnToWriteId> & val) {
+  this->srcTxnToWriteIdList = val;
+__isset.srcTxnToWriteIdList = true;
+}
+
 uint32_t AllocateTableWriteIdsRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
@@ -17003,7 +17014,6 @@ uint32_t AllocateTableWriteIdsRequest::read(::apache::thrift::protocol::TProtoco
 
   using ::apache::thrift::protocol::TProtocolException;
 
-  bool isset_txnIds = false;
   bool isset_dbName = false;
   bool isset_tableName = false;
 
@@ -17016,6 +17026,22 @@ uint32_t AllocateTableWriteIdsRequest::read(::apache::thrift::protocol::TProtoco
     switch (fid)
     {
       case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->dbName);
+          isset_dbName = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->tableName);
+          isset_tableName = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->txnIds.clear();
@@ -17030,23 +17056,35 @@ uint32_t AllocateTableWriteIdsRequest::read(::apache::thrift::protocol::TProtoco
             }
             xfer += iprot->readListEnd();
           }
-          isset_txnIds = true;
+          this->__isset.txnIds = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
-      case 2:
+      case 4:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->dbName);
-          isset_dbName = true;
+          xfer += iprot->readString(this->replPolicy);
+          this->__isset.replPolicy = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
-      case 3:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->tableName);
-          isset_tableName = true;
+      case 5:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->srcTxnToWriteIdList.clear();
+            uint32_t _size701;
+            ::apache::thrift::protocol::TType _etype704;
+            xfer += iprot->readListBegin(_etype704, _size701);
+            this->srcTxnToWriteIdList.resize(_size701);
+            uint32_t _i705;
+            for (_i705 = 0; _i705 < _size701; ++_i705)
+            {
+              xfer += this->srcTxnToWriteIdList[_i705].read(iprot);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.srcTxnToWriteIdList = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -17060,8 +17098,6 @@ uint32_t AllocateTableWriteIdsRequest::read(::apache::thrift::protocol::TProtoco
 
   xfer += iprot->readStructEnd();
 
-  if (!isset_txnIds)
-    throw TProtocolException(TProtocolException::INVALID_DATA);
   if (!isset_dbName)
     throw TProtocolException(TProtocolException::INVALID_DATA);
   if (!isset_tableName)
@@ -17074,26 +17110,45 @@ uint32_t AllocateTableWriteIdsRequest::write(::apache::thrift::protocol::TProtoc
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
   xfer += oprot->writeStructBegin("AllocateTableWriteIdsRequest");
 
-  xfer += oprot->writeFieldBegin("txnIds", ::apache::thrift::protocol::T_LIST, 1);
-  {
-    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->txnIds.size()));
-    std::vector<int64_t> ::const_iterator _iter701;
-    for (_iter701 = this->txnIds.begin(); _iter701 != this->txnIds.end(); ++_iter701)
-    {
-      xfer += oprot->writeI64((*_iter701));
-    }
-    xfer += oprot->writeListEnd();
-  }
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1);
   xfer += oprot->writeString(this->dbName);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 3);
+  xfer += oprot->writeFieldBegin("tableName", ::apache::thrift::protocol::T_STRING, 2);
   xfer += oprot->writeString(this->tableName);
   xfer += oprot->writeFieldEnd();
 
+  if (this->__isset.txnIds) {
+    xfer += oprot->writeFieldBegin("txnIds", ::apache::thrift::protocol::T_LIST, 3);
+    {
+      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->txnIds.size()));
+      std::vector<int64_t> ::const_iterator _iter706;
+      for (_iter706 = this->txnIds.begin(); _iter706 != this->txnIds.end(); ++_iter706)
+      {
+        xfer += oprot->writeI64((*_iter706));
+      }
+      xfer += oprot->writeListEnd();
+    }
+    xfer += oprot->writeFieldEnd();
+  }
+  if (this->__isset.replPolicy) {
+    xfer += oprot->writeFieldBegin("replPolicy", ::apache::thrift::protocol::T_STRING, 4);
+    xfer += oprot->writeString(this->replPolicy);
+    xfer += oprot->writeFieldEnd();
+  }
+  if (this->__isset.srcTxnToWriteIdList) {
+    xfer += oprot->writeFieldBegin("srcTxnToWriteIdList", ::apache::thrift::protocol::T_LIST, 5);
+    {
+      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->srcTxnToWriteIdList.size()));
+      std::vector<TxnToWriteId> ::const_iterator _iter707;
+      for (_iter707 = this->srcTxnToWriteIdList.begin(); _iter707 != this->srcTxnToWriteIdList.end(); ++_iter707)
+      {
+        xfer += (*_iter707).write(oprot);
+      }
+      xfer += oprot->writeListEnd();
+    }
+    xfer += oprot->writeFieldEnd();
+  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
@@ -17101,28 +17156,39 @@ uint32_t AllocateTableWriteIdsRequest::write(::apache::thrift::protocol::TProtoc
 
 void swap(AllocateTableWriteIdsRequest &a, AllocateTableWriteIdsRequest &b) {
   using ::std::swap;
-  swap(a.txnIds, b.txnIds);
   swap(a.dbName, b.dbName);
   swap(a.tableName, b.tableName);
+  swap(a.txnIds, b.txnIds);
+  swap(a.replPolicy, b.replPolicy);
+  swap(a.srcTxnToWriteIdList, b.srcTxnToWriteIdList);
+  swap(a.__isset, b.__isset);
 }
 
-AllocateTableWriteIdsRequest::AllocateTableWriteIdsRequest(const AllocateTableWriteIdsRequest& other702) {
-  txnIds = other702.txnIds;
-  dbName = other702.dbName;
-  tableName = other702.tableName;
-}
-AllocateTableWriteIdsRequest& AllocateTableWriteIdsRequest::operator=(const AllocateTableWriteIdsRequest& other703) {
-  txnIds = other703.txnIds;
-  dbName = other703.dbName;
-  tableName = other703.tableName;
+AllocateTableWriteIdsRequest::AllocateTableWriteIdsRequest(const AllocateTableWriteIdsRequest& other708) {
+  dbName = other708.dbName;
+  tableName = other708.tableName;
+  txnIds = other708.txnIds;
+  replPolicy = other708.replPolicy;
+  srcTxnToWriteIdList = other708.srcTxnToWriteIdList;
+  __isset = other708.__isset;
+}
+AllocateTableWriteIdsRequest& AllocateTableWriteIdsRequest::operator=(const AllocateTableWriteIdsRequest& other709) {
+  dbName = other709.dbName;
+  tableName = other709.tableName;
+  txnIds = other709.txnIds;
+  replPolicy = other709.replPolicy;
+  srcTxnToWriteIdList = other709.srcTxnToWriteIdList;
+  __isset = other709.__isset;
   return *this;
 }
 void AllocateTableWriteIdsRequest::printTo(std::ostream& out) const {
   using ::apache::thrift::to_string;
   out << "AllocateTableWriteIdsRequest(";
-  out << "txnIds=" << to_string(txnIds);
-  out << ", " << "dbName=" << to_string(dbName);
+  out << "dbName=" << to_string(dbName);
   out << ", " << "tableName=" << to_string(tableName);
+  out << ", " << "txnIds="; (__isset.txnIds ? (out << to_string(txnIds)) : (out << "<null>"));
+  out << ", " << "replPolicy="; (__isset.replPolicy ? (out << to_string(replPolicy)) : (out << "<null>"));
+  out << ", " << "srcTxnToWriteIdList="; (__isset.srcTxnToWriteIdList ? (out << to_string(srcTxnToWriteIdList)) : (out << "<null>"));
   out << ")";
 }
 
@@ -17218,13 +17284,13 @@ void swap(TxnToWriteId &a, TxnToWriteId &b) {
   swap(a.writeId, b.writeId);
 }
 
-TxnToWriteId::TxnToWriteId(const TxnToWriteId& other704) {
-  txnId = other704.txnId;
-  writeId = other704.writeId;
+TxnToWriteId::TxnToWriteId(const TxnToWriteId& other710) {
+  txnId = other710.txnId;
+  writeId = other710.writeId;
 }
-TxnToWriteId& TxnToWriteId::operator=(const TxnToWriteId& other705) {
-  txnId = other705.txnId;
-  writeId = other705.writeId;
+TxnToWriteId& TxnToWriteId::operator=(const TxnToWriteId& other711) {
+  txnId = other711.txnId;
+  writeId = other711.writeId;
   return *this;
 }
 void TxnToWriteId::printTo(std::ostream& out) const {
@@ -17270,14 +17336,14 @@ uint32_t AllocateTableWriteIdsResponse::read(::apache::thrift::protocol::TProtoc
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->txnToWriteIds.clear();
-            uint32_t _size706;
-            ::apache::thrift::protocol::TType _etype709;
-            xfer += iprot->readListBegin(_etype709, _size706);
-            this->txnToWriteIds.resize(_size706);
-            uint32_t _i710;
-            for (_i710 = 0; _i710 < _size706; ++_i710)
+            uint32_t _size712;
+            ::apache::thrift::protocol::TType _etype715;
+            xfer += iprot->readListBegin(_etype715, _size712);
+            this->txnToWriteIds.resize(_size712);
+            uint32_t _i716;
+            for (_i716 = 0; _i716 < _size712; ++_i716)
             {
-              xfer += this->txnToWriteIds[_i710].read(iprot);
+              xfer += this->txnToWriteIds[_i716].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -17308,10 +17374,10 @@ uint32_t AllocateTableWriteIdsResponse::write(::apache::thrift::protocol::TProto
   xfer += oprot->writeFieldBegin("txnToWriteIds", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->txnToWriteIds.size()));
-    std::vector<TxnToWriteId> ::const_iterator _iter711;
-    for (_iter711 = this->txnToWriteIds.begin(); _iter711 != this->txnToWriteIds.end(); ++_iter711)
+    std::vector<TxnToWriteId> ::const_iterator _iter717;
+    for (_iter717 = this->txnToWriteIds.begin(); _iter717 != this->txnToWriteIds.end(); ++_iter717)
     {
-      xfer += (*_iter711).write(oprot);
+      xfer += (*_iter717).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -17327,11 +17393,11 @@ void swap(AllocateTableWriteIdsResponse &a, AllocateTableWriteIdsResponse &b) {
   swap(a.txnToWriteIds, b.txnToWriteIds);
 }
 
-AllocateTableWriteIdsResponse::AllocateTableWriteIdsResponse(const AllocateTableWriteIdsResponse& other712) {
-  txnToWriteIds = other712.txnToWriteIds;
+AllocateTableWriteIdsResponse::AllocateTableWriteIdsResponse(const AllocateTableWriteIdsResponse& other718) {
+  txnToWriteIds = other718.txnToWriteIds;
 }
-AllocateTableWriteIdsResponse& AllocateTableWriteIdsResponse::operator=(const AllocateTableWriteIdsResponse& other713) {
-  txnToWriteIds = other713.txnToWriteIds;
+AllocateTableWriteIdsResponse& AllocateTableWriteIdsResponse::operator=(const AllocateTableWriteIdsResponse& other719) {
+  txnToWriteIds = other719.txnToWriteIds;
   return *this;
 }
 void AllocateTableWriteIdsResponse::printTo(std::ostream& out) const {
@@ -17409,9 +17475,9 @@ uint32_t LockComponent::read(::apache::thrift::protocol::TProtocol* iprot) {
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast714;
-          xfer += iprot->readI32(ecast714);
-          this->type = (LockType::type)ecast714;
+          int32_t ecast720;
+          xfer += iprot->readI32(ecast720);
+          this->type = (LockType::type)ecast720;
           isset_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -17419,9 +17485,9 @@ uint32_t LockComponent::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast715;
-          xfer += iprot->readI32(ecast715);
-          this->level = (LockLevel::type)ecast715;
+          int32_t ecast721;
+          xfer += iprot->readI32(ecast721);
+          this->level = (LockLevel::type)ecast721;
           isset_level = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -17453,9 +17519,9 @@ uint32_t LockComponent::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 6:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast716;
-          xfer += iprot->readI32(ecast716);
-          this->operationType = (DataOperationType::type)ecast716;
+          int32_t ecast722;
+          xfer += iprot->readI32(ecast722);
+          this->operationType = (DataOperationType::type)ecast722;
           this->__isset.operationType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -17555,27 +17621,27 @@ void swap(LockComponent &a, LockComponent &b) {
   swap(a.__isset, b.__isset);
 }
 
-LockComponent::LockComponent(const LockComponent& other717) {
-  type = other717.type;
-  level = other717.level;
-  dbname = other717.dbname;
-  tablename = other717.tablename;
-  partitionname = other717.partitionname;
-  operationType = other717.operationType;
-  isTransactional = other717.isTransactional;
-  isDynamicPartitionWrite = other717.isDynamicPartitionWrite;
-  __isset = other717.__isset;
-}
-LockComponent& LockComponent::operator=(const LockComponent& other718) {
-  type = other718.type;
-  level = other718.level;
-  dbname = other718.dbname;
-  tablename = other718.tablename;
-  partitionname = other718.partitionname;
-  operationType = other718.operationType;
-  isTransactional = other718.isTransactional;
-  isDynamicPartitionWrite = other718.isDynamicPartitionWrite;
-  __isset = other718.__isset;
+LockComponent::LockComponent(const LockComponent& other723) {
+  type = other723.type;
+  level = other723.level;
+  dbname = other723.dbname;
+  tablename = other723.tablename;
+  partitionname = other723.partitionname;
+  operationType = other723.operationType;
+  isTransactional = other723.isTransactional;
+  isDynamicPartitionWrite = other723.isDynamicPartitionWrite;
+  __isset = other723.__isset;
+}
+LockComponent& LockComponent::operator=(const LockComponent& other724) {
+  type = other724.type;
+  level = other724.level;
+  dbname = other724.dbname;
+  tablename = other724.tablename;
+  partitionname = other724.partitionname;
+  operationType = other724.operationType;
+  isTransactional = other724.isTransactional;
+  isDynamicPartitionWrite = other724.isDynamicPartitionWrite;
+  __isset = other724.__isset;
   return *this;
 }
 void LockComponent::printTo(std::ostream& out) const {
@@ -17647,14 +17713,14 @@ uint32_t LockRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->component.clear();
-            uint32_t _size719;
-            ::apache::thrift::protocol::TType _etype722;
-            xfer += iprot->readListBegin(_etype722, _size719);
-            this->component.resize(_size719);
-            uint32_t _i723;
-            for (_i723 = 0; _i723 < _size719; ++_i723)
+            uint32_t _size725;
+            ::apache::thrift::protocol::TType _etype728;
+            xfer += iprot->readListBegin(_etype728, _size725);
+            this->component.resize(_size725);
+            uint32_t _i729;
+            for (_i729 = 0; _i729 < _size725; ++_i729)
             {
-              xfer += this->component[_i723].read(iprot);
+              xfer += this->component[_i729].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -17721,10 +17787,10 @@ uint32_t LockRequest::write(::apache::thrift::protocol::TProtocol* oprot) const
   xfer += oprot->writeFieldBegin("component", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->component.size()));
-    std::vector<LockComponent> ::const_iterator _iter724;
-    for (_iter724 = this->component.begin(); _iter724 != this->component.end(); ++_iter724)
+    std::vector<LockComponent> ::const_iterator _iter730;
+    for (_iter730 = this->component.begin(); _iter730 != this->component.end(); ++_iter730)
     {
-      xfer += (*_iter724).write(oprot);
+      xfer += (*_iter730).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -17763,21 +17829,21 @@ void swap(LockRequest &a, LockRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-LockRequest::LockRequest(const LockRequest& other725) {
-  component = other725.component;
-  txnid = other725.txnid;
-  user = other725.user;
-  hostname = other725.hostname;
-  agentInfo = other725.agentInfo;
-  __isset = other725.__isset;
-}
-LockRequest& LockRequest::operator=(const LockRequest& other726) {
-  component = other726.component;
-  txnid = other726.txnid;
-  user = other726.user;
-  hostname = other726.hostname;
-  agentInfo = other726.agentInfo;
-  __isset = other726.__isset;
+LockRequest::LockRequest(const LockRequest& other731) {
+  component = other731.component;
+  txnid = other731.txnid;
+  user = other731.user;
+  hostname = other731.hostname;
+  agentInfo = other731.agentInfo;
+  __isset = other731.__isset;
+}
+LockRequest& LockRequest::operator=(const LockRequest& other732) {
+  component = other732.component;
+  txnid = other732.txnid;
+  user = other732.user;
+  hostname = other732.hostname;
+  agentInfo = other732.agentInfo;
+  __isset = other732.__isset;
   return *this;
 }
 void LockRequest::printTo(std::ostream& out) const {
@@ -17837,9 +17903,9 @@ uint32_t LockResponse::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast727;
-          xfer += iprot->readI32(ecast727);
-          this->state = (LockState::type)ecast727;
+          int32_t ecast733;
+          xfer += iprot->readI32(ecast733);
+          this->state = (LockState::type)ecast733;
           isset_state = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -17885,13 +17951,13 @@ void swap(LockResponse &a, LockResponse &b) {
   swap(a.state, b.state);
 }
 
-LockResponse::LockResponse(const LockResponse& other728) {
-  lockid = other728.lockid;
-  state = other728.state;
+LockResponse::LockResponse(const LockResponse& other734) {
+  lockid = other734.lockid;
+  state = other734.state;
 }
-LockResponse& LockResponse::operator=(const LockResponse& other729) {
-  lockid = other729.lockid;
-  state = other729.state;
+LockResponse& LockResponse::operator=(const LockResponse& other735) {
+  lockid = other735.lockid;
+  state = other735.state;
   return *this;
 }
 void LockResponse::printTo(std::ostream& out) const {
@@ -18013,17 +18079,17 @@ void swap(CheckLockRequest &a, CheckLockRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-CheckLockRequest::CheckLockRequest(const CheckLockRequest& other730) {
-  lockid = other730.lockid;
-  txnid = other730.txnid;
-  elapsed_ms = other730.elapsed_ms;
-  __isset = other730.__isset;
+CheckLockRequest::CheckLockRequest(const CheckLockRequest& other736) {
+  lockid = other736.lockid;
+  txnid = other736.txnid;
+  elapsed_ms = other736.elapsed_ms;
+  __isset = other736.__isset;
 }
-CheckLockRequest& CheckLockRequest::operator=(const CheckLockRequest& other731) {
-  lockid = other731.lockid;
-  txnid = other731.txnid;
-  elapsed_ms = other731.elapsed_ms;
-  __isset = other731.__isset;
+CheckLockRequest& CheckLockRequest::operator=(const CheckLockRequest& other737) {
+  lockid = other737.lockid;
+  txnid = other737.txnid;
+  elapsed_ms = other737.elapsed_ms;
+  __isset = other737.__isset;
   return *this;
 }
 void CheckLockRequest::printTo(std::ostream& out) const {
@@ -18107,11 +18173,11 @@ void swap(UnlockRequest &a, UnlockRequest &b) {
   swap(a.lockid, b.lockid);
 }
 
-UnlockRequest::UnlockRequest(const UnlockRequest& other732) {
-  lockid = other732.lockid;
+UnlockRequest::UnlockRequest(const UnlockRequest& other738) {
+  lockid = other738.lockid;
 }
-UnlockRequest& UnlockRequest::operator=(const UnlockRequest& other733) {
-  lockid = other733.lockid;
+UnlockRequest& UnlockRequest::operator=(const UnlockRequest& other739) {
+  lockid = other739.lockid;
   return *this;
 }
 void UnlockRequest::printTo(std::ostream& out) const {
@@ -18250,19 +18316,19 @@ void swap(ShowLocksRequest &a, ShowLocksRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-ShowLocksRequest::ShowLocksRequest(const ShowLocksRequest& other734) {
-  dbname = other734.dbname;
-  tablename = other734.tablename;
-  partname = other734.partname;
-  isExtended = other734.isExtended;
-  __isset = other734.__isset;
+ShowLocksRequest::ShowLocksRequest(const ShowLocksRequest& other740) {
+  dbname = other740.dbname;
+  tablename = other740.tablename;
+  partname = other740.partname;
+  isExtended = other740.isExtended;
+  __isset = other740.__isset;
 }
-ShowLocksRequest& ShowLocksRequest::operator=(const ShowLocksRequest& other735) {
-  dbname = other735.dbname;
-  tablename = other735.tablename;
-  partname = other735.partname;
-  isExtended = other735.isExtended;
-  __isset = other735.__isset;
+ShowLocksRequest& ShowLocksRequest::operator=(const ShowLocksRequest& other741) {
+  dbname = other741.dbname;
+  tablename = other741.tablename;
+  partname = other741.partname;
+  isExtended = other741.isExtended;
+  __isset = other741.__isset;
   return *this;
 }
 void ShowLocksRequest::printTo(std::ostream& out) const {
@@ -18415,9 +18481,9 @@ uint32_t ShowLocksResponseElement::read(::apache::thrift::protocol::TProtocol* i
         break;
       case 5:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast736;
-          xfer += iprot->readI32(ecast736);
-          this->state = (LockState::type)ecast736;
+          int32_t ecast742;
+          xfer += iprot->readI32(ecast742);
+          this->state = (LockState::type)ecast742;
           isset_state = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -18425,9 +18491,9 @@ uint32_t ShowLocksResponseElement::read(::apache::thrift::protocol::TProtocol* i
         break;
       case 6:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast737;
-          xfer += iprot->readI32(ecast737);
-          this->type = (LockType::type)ecast737;
+          int32_t ecast743;
+          xfer += iprot->readI32(ecast743);
+          this->type = (LockType::type)ecast743;
           isset_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -18643,43 +18709,43 @@ void swap(ShowLocksResponseElement &a, ShowLocksResponseElement &b) {
   swap(a.__isset, b.__isset);
 }
 
-ShowLocksResponseElement::ShowLocksResponseElement(const ShowLocksResponseElement& other738) {
-  lockid = other738.lockid;
-  dbname = other738.dbname;
-  tablename = other738.tablename;
-  partname = other738.partname;
-  state = other738.state;
-  type = other738.type;
-  txnid = other738.txnid;
-  lastheartbeat = other738.lastheartbeat;
-  acquiredat = other738.acquiredat;
-  user = other738.user;
-  hostname = other738.hostname;
-  heartbeatCount = other738.heartbeatCount;
-  agentInfo = other738.agentInfo;
-  blockedByExtId = other738.blockedByExtId;
-  blockedByIntId = other738.blockedByIntId;
-  lockIdInternal = other738.lockIdInternal;
-  __isset = other738.__isset;
-}
-ShowLocksResponseElement& ShowLocksResponseElement::operator=(const ShowLocksResponseElement& other739) {
-  lockid = other739.lockid;
-  dbname = other739.dbname;
-  tablename = other739.tablename;
-  partname = other739.partname;
-  state = other739.state;
-  type = other739.type;
-  txnid = other739.txnid;
-  lastheartbeat = other739.lastheartbeat;
-  acquiredat = other739.acquiredat;
-  user = other739.user;
-  hostname = other739.hostname;
-  heartbeatCount = other739.heartbeatCount;
-  agentInfo = other739.agentInfo;
-  blockedByExtId = other739.blockedByExtId;
-  blockedByIntId = other739.blockedByIntId;
-  lockIdInternal = other739.lockIdInternal;
-  __isset = other739.__isset;
+ShowLocksResponseElement::ShowLocksResponseElement(const ShowLocksResponseElement& other744) {
+  lockid = other744.lockid;
+  dbname = other744.dbname;
+  tablename = other744.tablename;
+  partname = other744.partname;
+  state = other744.state;
+  type = other744.type;
+  txnid = other744.txnid;
+  lastheartbeat = other744.lastheartbeat;
+  acquiredat = other744.acquiredat;
+  user = other744.user;
+  hostname = other744.hostname;
+  heartbeatCount = other744.heartbeatCount;
+  agentInfo = other744.agentInfo;
+  blockedByExtId = other744.blockedByExtId;
+  blockedByIntId = other744.blockedByIntId;
+  lockIdInternal = other744.lockIdInternal;
+  __isset = other744.__isset;
+}
+ShowLocksResponseElement& ShowLocksResponseElement::operator=(const ShowLocksResponseElement& other745) {
+  lockid = other745.lockid;
+  dbname = other745.dbname;
+  tablename = other745.tablename;
+  partname = other745.partname;
+  state = other745.state;
+  type = other745.type;
+  txnid = other745.txnid;
+  lastheartbeat = other745.lastheartbeat;
+  acquiredat = other745.acquiredat;
+  user = other745.user;
+  hostname = other745.hostname;
+  heartbeatCount = other745.heartbeatCount;
+  agentInfo = other745.agentInfo;
+  blockedByExtId = other745.blockedByExtId;
+  blockedByIntId = other745.blockedByIntId;
+  lockIdInternal = other745.lockIdInternal;
+  __isset = other745.__isset;
   return *this;
 }
 void ShowLocksResponseElement::printTo(std::ostream& out) const {
@@ -18738,14 +18804,14 @@ uint32_t ShowLocksResponse::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->locks.clear();
-            uint32_t _size740;
-            ::apache::thrift::protocol::TType _etype743;
-            xfer += iprot->readListBegin(_etype743, _size740);
-            this->locks.resize(_size740);
-            uint32_t _i744;
-            for (_i744 = 0; _i744 < _size740; ++_i744)
+            uint32_t _size746;
+            ::apache::thrift::protocol::TType _etype749;
+            xfer += iprot->readListBegin(_etype749, _size746);
+            this->locks.resize(_size746);
+            uint32_t _i750;
+            for (_i750 = 0; _i750 < _size746; ++_i750)
             {
-              xfer += this->locks[_i744].read(iprot);
+              xfer += this->locks[_i750].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -18774,10 +18840,10 @@ uint32_t ShowLocksResponse::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("locks", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->locks.size()));
-    std::vector<ShowLocksResponseElement> ::const_iterator _iter745;
-    for (_iter745 = this->locks.begin(); _iter745 != this->locks.end(); ++_iter745)
+    std::vector<ShowLocksResponseElement> ::const_iterator _iter751;
+    for (_iter751 = this->locks.begin(); _iter751 != this->locks.end(); ++_iter751)
     {
-      xfer += (*_iter745).write(oprot);
+      xfer += (*_iter751).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -18794,13 +18860,13 @@ void swap(ShowLocksResponse &a, ShowLocksResponse &b) {
   swap(a.__isset, b.__isset);
 }
 
-ShowLocksResponse::ShowLocksResponse(const ShowLocksResponse& other746) {
-  locks = other746.locks;
-  __isset = other746.__isset;
+ShowLocksResponse::ShowLocksResponse(const ShowLocksResponse& other752) {
+  locks = other752.locks;
+  __isset = other752.__isset;
 }
-ShowLocksResponse& ShowLocksResponse::operator=(const ShowLocksResponse& other747) {
-  locks = other747.locks;
-  __isset = other747.__isset;
+ShowLocksResponse& ShowLocksResponse::operator=(const ShowLocksResponse& other753) {
+  locks = other753.locks;
+  __isset = other753.__isset;
   return *this;
 }
 void ShowLocksResponse::printTo(std::ostream& out) const {
@@ -18901,15 +18967,15 @@ void swap(HeartbeatRequest &a, HeartbeatRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-HeartbeatRequest::HeartbeatRequest(const HeartbeatRequest& other748) {
-  lockid = other748.lockid;
-  txnid = other748.txnid;
-  __isset = other748.__isset;
+HeartbeatRequest::HeartbeatRequest(const HeartbeatRequest& other754) {
+  lockid = other754.lockid;
+  txnid = other754.txnid;
+  __isset = other754.__isset;
 }
-HeartbeatRequest& HeartbeatRequest::operator=(const HeartbeatRequest& other749) {
-  lockid = other749.lockid;
-  txnid = other749.txnid;
-  __isset = other749.__isset;
+HeartbeatRequest& HeartbeatRequest::operator=(const HeartbeatRequest& other755) {
+  lockid = other755.lockid;
+  txnid = other755.txnid;
+  __isset = other755.__isset;
   return *this;
 }
 void HeartbeatRequest::printTo(std::ostream& out) const {
@@ -19012,13 +19078,13 @@ void swap(HeartbeatTxnRangeRequest &a, HeartbeatTxnRangeRequest &b) {
   swap(a.max, b.max);
 }
 
-HeartbeatTxnRangeRequest::HeartbeatTxnRangeRequest(const HeartbeatTxnRangeRequest& other750) {
-  min = other750.min;
-  max = other750.max;
+HeartbeatTxnRangeRequest::HeartbeatTxnRangeRequest(const HeartbeatTxnRangeRequest& other756) {
+  min = other756.min;
+  max = other756.max;
 }
-HeartbeatTxnRangeRequest& HeartbeatTxnRangeRequest::operator=(const HeartbeatTxnRangeRequest& other751) {
-  min = other751.min;
-  max = other751.max;
+HeartbeatTxnRangeRequest& HeartbeatTxnRangeRequest::operator=(const HeartbeatTxnRangeRequest& other757) {
+  min = other757.min;
+  max = other757.max;
   return *this;
 }
 void HeartbeatTxnRangeRequest::printTo(std::ostream& out) const {
@@ -19069,15 +19135,15 @@ uint32_t HeartbeatTxnRangeResponse::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             this->aborted.clear();
-            uint32_t _size752;
-            ::apache::thrift::protocol::TType _etype755;
-            xfer += iprot->readSetBegin(_etype755, _size752);
-            uint32_t _i756;
-            for (_i756 = 0; _i756 < _size752; ++_i756)
+            uint32_t _size758;
+            ::apache::thrift::protocol::TType _etype761;
+            xfer += iprot->readSetBegin(_etype761, _size758);
+            uint32_t _i762;
+            for (_i762 = 0; _i762 < _size758; ++_i762)
             {
-              int64_t _elem757;
-              xfer += iprot->readI64(_elem757);
-              this->aborted.insert(_elem757);
+              int64_t _elem763;
+              xfer += iprot->readI64(_elem763);
+              this->aborted.insert(_elem763);
             }
             xfer += iprot->readSetEnd();
           }
@@ -19090,15 +19156,15 @@ uint32_t HeartbeatTxnRangeResponse::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             this->nosuch.clear();
-            uint32_t _size758;
-            ::apache::thrift::protocol::TType _etype761;
-            xfer += iprot->readSetBegin(_etype761, _size758);
-            uint32_t _i762;
-            for (_i762 = 0; _i762 < _size758; ++_i762)
+            uint32_t _size764;
+            ::apache::thrift::protocol::TType _etype767;
+            xfer += iprot->readSetBegin(_etype767, _size764);
+            uint32_t _i768;
+            for (_i768 = 0; _i768 < _size764; ++_i768)
             {
-              int64_t _elem763;
-              xfer += iprot->readI64(_elem763);
-              this->nosuch.insert(_elem763);
+              int64_t _elem769;
+              xfer += iprot->readI64(_elem769);
+              this->nosuch.insert(_elem769);
             }
             xfer += iprot->readSetEnd();
           }
@@ -19131,10 +19197,10 @@ uint32_t HeartbeatTxnRangeResponse::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("aborted", ::apache::thrift::protocol::T_SET, 1);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->aborted.size()));
-    std::set<int64_t> ::const_iterator _iter764;
-    for (_iter764 = this->aborted.begin(); _iter764 != this->aborted.end(); ++_iter764)
+    std::set<int64_t> ::const_iterator _iter770;
+    for (_iter770 = this->aborted.begin(); _iter770 != this->aborted.end(); ++_iter770)
     {
-      xfer += oprot->writeI64((*_iter764));
+      xfer += oprot->writeI64((*_iter770));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -19143,10 +19209,10 @@ uint32_t HeartbeatTxnRangeResponse::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("nosuch", ::apache::thrift::protocol::T_SET, 2);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->nosuch.size()));
-    std::set<int64_t> ::const_iterator _iter765;
-    for (_iter765 = this->nosuch.begin(); _iter765 != this->nosuch.end(); ++_iter765)
+    std::set<int64_t> ::const_iterator _iter771;
+    for (_iter771 = this->nosuch.begin(); _iter771 != this->nosuch.end(); ++_iter771)
     {
-      xfer += oprot->writeI64((*_iter765));
+      xfer += oprot->writeI64((*_iter771));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -19163,13 +19229,13 @@ void swap(HeartbeatTxnRangeResponse &a, HeartbeatTxnRangeResponse &b) {
   swap(a.nosuch, b.nosuch);
 }
 
-HeartbeatTxnRangeResponse::HeartbeatTxnRangeResponse(const HeartbeatTxnRangeResponse& other766) {
-  aborted = other766.aborted;
-  nosuch = other766.nosuch;
+HeartbeatTxnRangeResponse::HeartbeatTxnRangeResponse(const HeartbeatTxnRangeResponse& other772) {
+  aborted = other772.aborted;
+  nosuch = other772.nosuch;
 }
-HeartbeatTxnRangeResponse& HeartbeatTxnRangeResponse::operator=(const HeartbeatTxnRangeResponse& other767) {
-  aborted = other767.aborted;
-  nosuch = other767.nosuch;
+HeartbeatTxnRangeResponse& HeartbeatTxnRangeResponse::operator=(const HeartbeatTxnRangeResponse& other773) {
+  aborted = other773.aborted;
+  nosuch = other773.nosuch;
   return *this;
 }
 void HeartbeatTxnRangeResponse::printTo(std::ostream& out) const {
@@ -19262,9 +19328,9 @@ uint32_t CompactionRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast768;
-          xfer += iprot->readI32(ecast768);
-          this->type = (CompactionType::type)ecast768;
+          int32_t ecast774;
+          xfer += iprot->readI32(ecast774);
+          this->type = (CompactionType::type)ecast774;
           isset_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -19282,17 +19348,17 @@ uint32_t CompactionRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->properties.clear();
-            uint32_t _size769;
-            ::apache::thrift::protocol::TType _ktype770;
-            ::apache::thrift::protocol::TType _vtype771;
-            xfer += iprot->readMapBegin(_ktype770, _vtype771, _size769);
-            uint32_t _i773;
-            for (_i773 = 0; _i773 < _size769; ++_i773)
+            uint32_t _size775;
+            ::apache::thrift::protocol::TType _ktype776;
+            ::apache::thrift::protocol::TType _vtype777;
+            xfer += iprot->readMapBegin(_ktype776, _vtype777, _size775);
+            uint32_t _i779;
+            for (_i779 = 0; _i779 < _size775; ++_i779)
             {
-              std::string _key774;
-              xfer += iprot->readString(_key774);
-              std::string& _val775 = this->properties[_key774];
-              xfer += iprot->readString(_val775);
+              std::string _key780;
+              xfer += iprot->readString(_key780);
+              std::string& _val781 = this->properties[_key780];
+              xfer += iprot->readString(_val781);
             }
             xfer += iprot->readMapEnd();
           }
@@ -19350,11 +19416,11 @@ uint32_t CompactionRequest::write(::apache::thrift::protocol::TProtocol* oprot)
     xfer += oprot->writeFieldBegin("properties", ::apache::thrift::protocol::T_MAP, 6);
     {
       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 _iter776;
-      for (_iter776 = this->properties.begin(); _iter776 != this->properties.end(); ++_iter776)
+      std::map<std::string, std::string> ::const_iterator _iter782;
+      for (_iter782 = this->properties.begin(); _iter782 != this->properties.end(); ++_iter782)
       {
-        xfer += oprot->writeString(_iter776->first);
-        xfer += oprot->writeString(_iter776->second);
+        xfer += oprot->writeString(_iter782->first);
+        xfer += oprot->writeString(_iter782->second);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -19376,23 +19442,23 @@ void swap(CompactionRequest &a, CompactionRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-CompactionRequest::CompactionRequest(const CompactionRequest& other777) {
-  dbname = other777.dbname;
-  tablename = other777.tablename;
-  partitionname = other777.partitionname;
-  type = other777.type;
-  runas = other777.runas;
-  properties = other777.properties;
-  __isset = other777.__isset;
-}
-CompactionRequest& CompactionRequest::operator=(const CompactionRequest& other778) {
-  dbname = other778.dbname;
-  tablename = other778.tablename;
-  partitionname = other778.partitionname;
-  type = other778.type;
-  runas = other778.runas;
-  properties = other778.properties;
-  __isset = other778.__isset;
+CompactionRequest::CompactionRequest(const CompactionRequest& other783) {
+  dbname = other783.dbname;
+  tablename = other783.tablename;
+  partitionname = other783.partitionname;
+  type = other783.type;
+  runas = other783.runas;
+  properties = other783.properties;
+  __isset = other783.__isset;
+}
+CompactionRequest& CompactionRequest::operator=(const CompactionRequest& other784) {
+  dbname = other784.dbname;
+  tablename = other784.tablename;
+  partitionname = other784.partitionname;
+  type = other784.type;
+  runas = other784.runas;
+  properties = other784.properties;
+  __isset = other784.__isset;
   return *this;
 }
 void CompactionRequest::printTo(std::ostream& out) const {
@@ -19519,15 +19585,15 @@ void swap(CompactionResponse &a, CompactionResponse &b) {
   swap(a.accepted, b.accepted);
 }
 
-CompactionResponse::CompactionResponse(const CompactionResponse& other779) {
-  id = other779.id;
-  state = other779.state;
-  accepted = other779.accepted;
+CompactionResponse::CompactionResponse(const CompactionResponse& other785) {
+  id = other785.id;
+  state = other785.state;
+  accepted = other785.accepted;
 }
-CompactionResponse& CompactionResponse::operator=(const CompactionResponse& other780) {
-  id = other780.id;
-  state = other780.state;
-  accepted = other780.accepted;
+CompactionResponse& CompactionResponse::operator=(const CompactionResponse& other786) {
+  id = other786.id;
+  state = other786.state;
+  accepted = other786.accepted;
   return *this;
 }
 void CompactionResponse::printTo(std::ostream& out) const {
@@ -19588,11 +19654,11 @@ void swap(ShowCompactRequest &a, ShowCompactRequest &b) {
   (void) b;
 }
 
-ShowCompactRequest::ShowCompactRequest(const ShowCompactRequest& other781) {
-  (void) other781;
+ShowCompactRequest::ShowCompactRequest(const ShowCompactRequest& other787) {
+  (void) other787;
 }
-ShowCompactRequest& ShowCompactRequest::operator=(const ShowCompactRequest& other782) {
-  (void) other782;
+ShowCompactRequest& ShowCompactRequest::operator=(const ShowCompactRequest& other788) {
+  (void) other788;
   return *this;
 }
 void ShowCompactRequest::printTo(std::ostream& out) const {
@@ -19718,9 +19784,9 @@ uint32_t ShowCompactResponseElement::read(::apache::thrift::protocol::TProtocol*
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast783;
-          xfer += iprot->readI32(ecast783);
-          this->type = (CompactionType::type)ecast783;
+          int32_t ecast789;
+          xfer += iprot->readI32(ecast789);
+          this->type = (CompactionType::type)ecast789;
           isset_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -19907,37 +19973,37 @@ void swap(ShowCompactResponseElement &a, ShowCompactResponseElement &b) {
   swap(a.__isset, b.__isset);
 }
 
-ShowCompactResponseElement::ShowCompactResponseElement(const ShowCompactResponseElement& other784) {
-  dbname = other784.dbname;
-  tablename = other784.tablename;
-  partitionname = other784.partitionname;
-  type = other784.type;
-  state = other784.state;
-  workerid = other784.workerid;
-  start = other784.start;
-  runAs = other784.runAs;
-  hightestTxnId = other784.hightestTxnId;
-  metaInfo = other784.metaInfo;
-  endTime = other784.endTime;
-  hadoopJobId = other784.hadoopJobId;
-  id = other784.id;
-  __isset = other784.__isset;
-}
-ShowCompactResponseElement& ShowCompactResponseElement::operator=(const ShowCompactResponseElement& other785) {
-  dbname = other785.dbname;
-  tablename = other785.tablename;
-  partitionname = other785.partitionname;
-  type = other785.type;
-  state = other785.state;
-  workerid = other785.workerid;
-  start = other785.start;
-  runAs = other785.runAs;
-  hightestTxnId = other785.hightestTxnId;
-  metaInfo = other785.metaInfo;
-  endTime = other785.endTime;
-  hadoopJobId = other785.hadoopJobId;
-  id = other785.id;
-  __isset = other785.__isset;
+ShowCompactResponseElement::ShowCompactResponseElement(const ShowCompactResponseElement& other790) {
+  dbname = other790.dbname;
+  tablename = other790.tablename;
+  partitionname = other790.partitionname;
+  type = other790.type;
+  state = other790.state;
+  workerid = other790.workerid;
+  start = other790.start;
+  runAs = other790.runAs;
+  hightestTxnId = other790.hightestTxnId;
+  metaInfo = other790.metaInfo;
+  endTime = other790.endTime;
+  hadoopJobId = other790.hadoopJobId;
+  id = other790.id;
+  __isset = other790.__isset;
+}
+ShowCompactResponseElement& ShowCompactResponseElement::operator=(const ShowCompactResponseElement& other791) {
+  dbname = other791.dbname;
+  tablename = other791.tablename;
+  partitionname = other791.partitionname;
+  type = other791.type;
+  state = other791.state;
+  workerid = other791.workerid;
+  start = other791.start;
+  runAs = other791.runAs;
+  hightestTxnId = other791.hightestTxnId;
+  metaInfo = other791.metaInfo;
+  endTime = other791.endTime;
+  hadoopJobId = other791.hadoopJobId;
+  id = other791.id;
+  __isset = other791.__isset;
   return *this;
 }
 void ShowCompactResponseElement::printTo(std::ostream& out) const {
@@ -19994,14 +20060,14 @@ uint32_t ShowCompactResponse::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->compacts.clear();
-            uint32_t _size786;
-            ::apache::thrift::protocol::TType _etype789;
-            xfer += iprot->readListBegin(_etype789, _size786);
-            this->compacts.resize(_size786);
-            uint32_t _i790;
-            for (_i790 = 0; _i790 < _size786; ++_i790)
+            uint32_t _size792;
+            ::apache::thrift::protocol::TType _etype795;
+            xfer += iprot->readListBegin(_etype795, _size792);
+            this->compacts.resize(_size792);
+            uint32_t _i796;
+            for (_i796 = 0; _i796 < _size792; ++_i796)
             {
-              xfer += this->compacts[_i790].read(iprot);
+              xfer += this->compacts[_i796].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -20032,10 +20098,10 @@ uint32_t ShowCompactResponse::write(::apache::thrift::protocol::TProtocol* oprot
   xfer += oprot->writeFieldBegin("compacts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->compacts.size()));
-    std::vector<ShowCompactResponseElement> ::const_iterator _iter791;
-    for (_iter791 = this->compacts.begin(); _iter791 != this->compacts.end(); ++_iter791)
+    std::vector<ShowCompactResponseElement> ::const_iterator _iter797;
+    for (_iter797 = this->compacts.begin(); _iter797 != this->compacts.end(); ++_iter797)
     {
-      xfer += (*_iter791).write(oprot);
+      xfer += (*_iter797).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -20051,11 +20117,11 @@ void swap(ShowCompactResponse &a, ShowCompactResponse &b) {
   swap(a.compacts, b.compacts);
 }
 
-ShowCompactResponse::ShowCompactResponse(const ShowCompactResponse& other792) {
-  compacts = other792.compacts;
+ShowCompactResponse::ShowCompactResponse(const ShowCompactResponse& other798) {
+  compacts = other798.compacts;
 }
-ShowCompactResponse& ShowCompactResponse::operator=(const ShowCompactResponse& other793) {
-  compacts = other793.compacts;
+ShowCompactResponse& ShowCompactResponse::operator=(const ShowCompactResponse& other799) {
+  compacts = other799.compacts;
   return *this;
 }
 void ShowCompactResponse::printTo(std::ostream& out) const {
@@ -20157,14 +20223,14 @@ uint32_t AddDynamicPartitions::read(::apache::thrift::protocol::TProtocol* iprot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitionnames.clear();
-            uint32_t _size794;
-            ::apache::thrift::protocol::TType _etype797;
-            xfer += iprot->readListBegin(_etype797, _size794);
-            this->partitionnames.resize(_size794);
-            uint32_t _i798;
-            for (_i798 = 0; _i798 < _size794; ++_i798)
+            uint32_t _size800;
+            ::apache::thrift::protocol::TType _etype803;
+            xfer += iprot->readListBegin(_etype803, _size800);
+            this->partitionnames.resize(_size800);
+            uint32_t _i804;
+            for (_i804 = 0; _i804 < _size800; ++_i804)
             {
-              xfer += iprot->readString(this->partitionnames[_i798]);
+              xfer += iprot->readString(this->partitionnames[_i804]);
             }
             xfer += iprot->readListEnd();
           }
@@ -20175,9 +20241,9 @@ uint32_t AddDynamicPartitions::read(::apache::thrift::protocol::TProtocol* iprot
         break;
       case 6:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast799;
-          xfer += iprot->readI32(ecast799);
-          this->operationType = (DataOperationType::type)ecast799;
+          int32_t ecast805;
+          xfer += iprot->readI32(ecast805);
+          this->operationType = (DataOperationType::type)ecast805;
           this->__isset.operationType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -20229,10 +20295,10 @@ uint32_t AddDynamicPartitions::write(::apache::thrift::protocol::TProtocol* opro
   xfer += oprot->writeFieldBegin("partitionnames", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionnames.size()));
-    std::vector<std::string> ::const_iterator _iter800;
-    for (_iter800 = this->partitionnames.begin(); _iter800 != this->partitionnames.end(); ++_iter800)
+    std::vector<std::string> ::const_iterator _iter806;
+    for (_iter806 = this->partitionnames.begin(); _iter806 != this->partitionnames.end(); ++_iter806)
     {
-      xfer += oprot->writeString((*_iter800));
+      xfer += oprot->writeString((*_iter806));
     }
     xfer += oprot->writeListEnd();
   }
@@ -20259,23 +20325,23 @@ void swap(AddDynamicPartitions &a, AddDynamicPartitions &b) {
   swap(a.__isset, b.__isset);
 }
 
-AddDynamicPartitions::AddDynamicPartitions(const AddDynamicPartitions& other801) {
-  txnid = other801.txnid;
-  writeid = other801.writeid;
-  dbname = other801.dbname;
-  tablename = other801.tablename;
-  partitionnames = other801.partitionnames;
-  operationType = other801.operationType;
-  __isset = other801.__isset;
-}
-AddDynamicPartitions& AddDynamicPartitions::operator=(const AddDynamicPartitions& other802) {
-  txnid = other802.txnid;
-  writeid = other802.writeid;
-  dbname = other802.dbname;
-  tablename = other802.tablename;
-  partitionnames = other802.partitionnames;
-  operationType = other802.operationType;
-  __isset = other802.__isset;
+AddDynamicPartitions::AddDynamicPartitions(const AddDynamicPartitions& other807) {
+  txnid = other807.txnid;
+  writeid = other807.writeid;
+  dbname = other807.dbname;
+  tablename = other807.tablename;
+  partitionnames = other807.partitionnames;
+  operationType = other807.operationType;
+  __isset = other807.__isset;
+}
+AddDynamicPartitions& AddDynamicPartitions::operator=(const AddDynamicPartitions& other808) {
+  txnid = other808.txnid;
+  writeid = other808.writeid;
+  dbname = other808.dbname;
+  tablename = other808.tablename;
+  partitionnames = other808.partitionnames;
+  operationType = other808.operationType;
+  __isset = other808.__isset;
   return *this;
 }
 void AddDynamicPartitions::printTo(std::ostream& out) const {
@@ -20458,23 +20524,23 @@ void swap(BasicTxnInfo &a, BasicTxnInfo &b) {
   swap(a.__isset, b.__isset);
 }
 
-BasicTxnInfo::BasicTxnInfo(const BasicTxnInfo& other803) {
-  isnull = other803.isnull;
-  time = other803.time;
-  txnid = other803.txnid;
-  dbname = other803.dbname;
-  tablename = other803.tablename;
-  partitionname = other803.partitionname;
-  __isset = other803.__isset;
-}
-BasicTxnInfo& BasicTxnInfo::operator=(const BasicTxnInfo& other804) {
-  isnull = other804.isnull;
-  time = other804.time;
-  txnid = other804.txnid;
-  dbname = other804.dbname;
-  tablename = other804.tablename;
-  partitionname = other804.partitionname;
-  __isset = other804.__isset;
+BasicTxnInfo::BasicTxnInfo(const BasicTxnInfo& other809) {
+  isnull = other809.isnull;
+  time = other809.time;
+  txnid = other809.txnid;
+  dbname = other809.dbname;
+  tablename = other809.tablename;
+  partitionname = other809.partitionname;
+  __isset = other809.__isset;
+}
+BasicTxnInfo& BasicTxnInfo::operator=(const BasicTxnInfo& other810) {
+  isnull = other810.isnull;
+  time = other810.time;
+  txnid = other810.txnid;
+  dbname = other810.dbname;
+  tablename = other810.tablename;
+  partitionname = other810.partitionname;
+  __isset = other810.__isset;
   return *this;
 }
 void BasicTxnInfo::printTo(std::ostream& out) const {
@@ -20568,15 +20634,15 @@ uint32_t CreationMetadata::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             this->tablesUsed.clear();
-            uint32_t _size805;
-            ::apache::thrift::protocol::TType _etype808;
-            xfer += iprot->readSetBegin(_etype808, _size805);
-            uint32_t _i809;
-            for (_i809 = 0; _i809 < _size805; ++_i809)
+            uint32_t _size811;
+            ::apache::thrift::protocol::TType _etype814;
+            xfer += iprot->readSetBegin(_etype814, _size811);
+            uint32_t _i815;
+            for (_i815 = 0; _i815 < _size811; ++_i815)
             {
-              std::string _elem810;
-              xfer += iprot->readString(_elem810);
-              this->tablesUsed.insert(_elem810);
+              std::string _elem816;
+              xfer += iprot->readString(_elem816);
+              this->tablesUsed.insert(_elem816);
             }
             xfer += iprot->readSetEnd();
           }
@@ -20633,10 +20699,10 @@ uint32_t CreationMetadata::write(::apache::thrift::protocol::TProtocol* oprot) c
   xfer += oprot->writeFieldBegin("tablesUsed", ::apache::thrift::protocol::T_SET, 4);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tablesUsed.size()));
-    std::set<std::string> ::const_iterator _iter811;
-    for (_iter811 = this->tablesUsed.begin(); _iter811 != this->tablesUsed.end(); ++_iter811)
+    std::set<std::string> ::const_iterator _iter817;
+    for (_iter817 = this->tablesUsed.begin(); _iter817 != this->tablesUsed.end(); ++_iter817)
     {
-      xfer += oprot->writeString((*_iter811));
+      xfer += oprot->writeString((*_iter817));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -20662,21 +20728,21 @@ void swap(CreationMetadata &a, CreationMetadata &b) {
   swap(a.__isset, b.__isset);
 }
 
-CreationMetadata::CreationMetadata(const CreationMetadata& other812) {
-  catName = other812.catName;
-  dbName = other812.dbName;
-  tblName = other812.tblName;
-  tablesUsed = other812.tablesUsed;
-  validTxnList = other812.validTxnList;
-  __isset = other812.__isset;
-}
-CreationMetadata& CreationMetadata::operator=(const CreationMetadata& other813) {
-  catName = other813.catName;
-  dbName = other813.dbName;
-  tblName = other813.tblName;
-  tablesUsed = other813.tablesUsed;
-  validTxnList = other813.validTxnList;
-  __isset = other813.__isset;
+CreationMetadata::CreationMetadata(const CreationMetadata& other818) {
+  catName = other818.catName;
+  dbName = other818.dbName;
+  tblName = other818.tblName;
+  tablesUsed = other818.tablesUsed;
+  validTxnList = other818.validTxnList;
+  __isset = other818.__isset;
+}
+CreationMetadata& CreationMetadata::operator=(const CreationMetadata& other819) {
+  catName = other819.catName;
+  dbName = other819.dbName;
+  tblName = other819.tblName;
+  tablesUsed = other819.tablesUsed;
+  validTxnList = other819.validTxnList;
+  __isset = other819.__isset;
   return *this;
 }
 void CreationMetadata::printTo(std::ostream& out) const {
@@ -20782,15 +20848,15 @@ void swap(NotificationEventRequest &a, NotificationEventRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-NotificationEventRequest::NotificationEventRequest(const NotificationEventRequest& other814) {
-  lastEvent = other814.lastEvent;
-  maxEvents = other814.maxEvents;
-  __isset = other814.__isset;
+NotificationEventRequest::NotificationEventRequest(const NotificationEventRequest& other820) {
+  lastEvent = other820.lastEvent;
+  maxEvents = other820.maxEvents;
+  __isset = other820.__isset;
 }
-NotificationEventRequest& NotificationEventRequest::operator=(const NotificationEventRequest& other815) {
-  lastEvent = other815.lastEvent;
-  maxEvents = other815.maxEvents;
-  __isset = other815.__isset;
+NotificationEventRequest& NotificationEventRequest::operator=(const NotificationEventRequest& other821) {
+  lastEvent = other821.lastEvent;
+  maxEvents = other821.maxEvents;
+  __isset = other821.__isset;
   return *this;
 }
 void NotificationEventRequest::printTo(std::ostream& out) const {
@@ -21010,27 +21076,27 @@ void swap(NotificationEvent &a, NotificationEvent &b) {
   swap(a.__isset, b.__isset);
 }
 
-NotificationEvent::NotificationEvent(const NotificationEvent& other816) {
-  eventId = other816.eventId;
-  eventTime = other816.eventTime;
-  eventType = other816.eventType;
-  dbName = other816.dbName;
-  tableName = other816.tableName;
-  message = other816.message;
-  messageFormat = other816.messageFormat;
-  catName = other816.catName;
-  __isset = other816.__isset;
-}
-NotificationEvent& NotificationEvent::operator=(const NotificationEvent& other817) {
-  eventId = other817.eventId;
-  eventTime = other817.eventTime;
-  eventType = other817.eventType;
-  dbName = other817.dbName;
-  tableName = other817.tableName;
-  message = other817.message;
-  messageFormat = other817.messageFormat;
-  catName = other817.catName;
-  __isset = other817.__isset;
+NotificationEvent::NotificationEvent(const NotificationEvent& other822) {
+  eventId = other822.eventId;
+  eventTime = other822.eventTime;
+  eventType = other822.eventType;
+  dbName = other822.dbName;
+  tableName = other822.tableName;
+  message = other822.message;
+  messageFormat = other822.messageFormat;
+  catName = other822.catName;
+  __isset = other822.__isset;
+}
+NotificationEvent& NotificationEvent::operator=(const NotificationEvent& other823) {
+  eventId = other823.eventId;
+  eventTime = other823.eventTime;
+  eventType = other823.eventType;
+  dbName = other823.dbName;
+  tableName = other823.tableName;
+  message = other823.message;
+  messageFormat = other823.messageFormat;
+  catName = other823.catName;
+  __isset = other823.__isset;
   return *this;
 }
 void NotificationEvent::printTo(std::ostream& out) const {
@@ -21082,14 +21148,14 @@ uint32_t NotificationEventResponse::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->events.clear();
-            uint32_t _size818;
-            ::apache::thrift::protocol::TType _etype821;
-            xfer += iprot->readListBegin(_etype821, _size818);
-            this->events.resize(_size818);
-            uint32_t _i822;
-            for (_i822 = 0; _i822 < _size818; ++_i822)
+            uint32_t _size824;
+            ::apache::thrift::protocol::TType _etype827;
+            xfer += iprot->readListBegin(_etype827, _size824);
+            this->events.resize(_size824);
+            uint32_t _i828;
+            for (_i828 = 0; _i828 < _size824; ++_i828)
             {
-              xfer += this->events[_i822].read(iprot);
+              xfer += this->events[_i828].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -21120,10 +21186,10 @@ uint32_t NotificationEventResponse::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("events", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->events.size()));
-    std::vector<NotificationEvent> ::const_iterator _iter823;
-    for (_iter823 = this->events.begin(); _iter823 != this->events.end(); ++_iter823)
+    std::vector<NotificationEvent> ::const_iterator _iter829;
+    for (_iter829 = this->events.begin(); _iter829 != this->events.end(); ++_iter829)
     {
-      xfer += (*_iter823).write(oprot);
+      xfer += (*_iter829).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -21139,11 +21205,11 @@ void swap(NotificationEventResponse &a, NotificationEventResponse &b) {
   swap(a.events, b.events);
 }
 
-NotificationEventResponse::NotificationEventResponse(const NotificationEventResponse& other824) {
-  events = other824.events;
+NotificationEventResponse::NotificationEventResponse(const NotificationEventResponse& other830) {
+  events = other830.events;
 }
-NotificationEventResponse& NotificationEventResponse::operator=(const NotificationEventResponse& other825) {
-  events = other825.events;
+NotificationEventResponse& NotificationEventResponse::operator=(const NotificationEventResponse& other831) {
+  events = other831.events;
   return *this;
 }
 void NotificationEventResponse::printTo(std::ostream& out) const {
@@ -21225,11 +21291,11 @@ void swap(CurrentNotificationEventId &a, CurrentNotificationEventId &b) {
   swap(a.eventId, b.eventId);
 }
 
-CurrentNotificationEventId::CurrentNotificationEventId(const CurrentNotificationEventId& other826) {
-  eventId = other826.eventId;
+CurrentNotificationEventId::CurrentNotificationEventId(const CurrentNotificationEventId& other832) {
+  eventId = other832.eventId;
 }
-CurrentNotificationEventId& CurrentNotificationEventId::operator=(const CurrentNotificationEventId& other827) {
-  eventId = other827.eventId;
+CurrentNotificationEventId& CurrentNotificationEventId::operator=(const CurrentNotificationEventId& other833) {
+  eventId = other833.eventId;
   return *this;
 }
 void CurrentNotificationEventId::printTo(std::ostream& out) const {
@@ -21351,17 +21417,17 @@ void swap(NotificationEventsCountRequest &a, NotificationEventsCountRequest &b)
   swap(a.__isset, b.__isset);
 }
 
-NotificationEventsCountRequest::NotificationEventsCountRequest(const NotificationEventsCountRequest& other828) {
-  fromEventId = other828.fromEventId;
-  dbName = other828.dbName;
-  catName = other828.catName;
-  __isset = other828.__isset;
+NotificationEventsCountRequest::NotificationEventsCountRequest(const NotificationEventsCountRequest& other834) {
+  fromEventId = other834.fromEventId;
+  dbName = other834.dbName;
+  catName = other834.catName;
+  __isset = other834.__isset;
 }
-NotificationEventsCountRequest& NotificationEventsCountRequest::operator=(const NotificationEventsCountRequest& other829) {
-  fromEventId = other829.fromEventId;
-  dbName = other829.dbName;
-  catName = other829.catName;
-  __isset = other829.__isset;
+NotificationEventsCountRequest& NotificationEventsCountRequest::operator=(const NotificationEventsCountRequest& other835) {
+  fromEventId = other835.fromEventId;
+  dbName = other835.dbName;
+  catName = other835.catName;
+  __isset = other835.__isset;
   return *this;
 }
 void NotificationEventsCountRequest::printTo(std::ostream& out) const {
@@ -21445,11 +21511,11 @@ void swap(NotificationEventsCountResponse &a, NotificationEventsCountResponse &b
   swap(a.eventsCount, b.eventsCount);
 }
 
-NotificationEventsCountResponse::NotificationEventsCountResponse(const NotificationEventsCountResponse& other830) {
-  eventsCount = other830.eventsCount;
+NotificationEventsCountResponse::NotificationEventsCountResponse(const NotificationEventsCountResponse& other836) {
+  eventsCount = other836.eventsCount;
 }
-NotificationEventsCountResponse& NotificationEventsCountResponse::operator=(const NotificationEventsCountResponse& other831) {
-  eventsCount = other831.eventsCount;
+NotificationEventsCountResponse& NotificationEventsCountResponse::operator=(const NotificationEventsCountResponse& other837) {
+  eventsCount = other837.eventsCount;
   return *this;
 }
 void NotificationEventsCountResponse::printTo(std::ostream& out) const {
@@ -21512,14 +21578,14 @@ uint32_t InsertEventRequestData::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->filesAdded.clear();
-            uint32_t _size832;
-            ::apache::thrift::protocol::TType _etype835;
-            xfer += iprot->readListBegin(_etype835, _size832);
-            this->filesAdded.resize(_size832);
-            uint32_t _i836;
-            for (_i836 = 0; _i836 < _size832; ++_i836)
+            uint32_t _size838;
+            ::apache::thrift::protocol::TType _etype841;
+            xfer += iprot->readListBegin(_etype841, _size838);
+            this->filesAdded.resize(_size838);
+            uint32_t _i842;
+            for (_i842 = 0; _i842 < _size838; ++_i842)
             {
-              xfer += iprot->readString(this->filesAdded[_i836]);
+              xfer += iprot->readString(this->filesAdded[_i842]);
             }
             xfer += iprot->readListEnd();
           }
@@ -21532,14 +21598,14 @@ uint32_t InsertEventRequestData::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->filesAddedChecksum.clear();
-            uint32_t _size837;
-            ::apache::thrift::protocol::TType _etype840;
-            xfer += iprot->readListBegin(_etype840, _size837);
-            this->filesAddedChecksum.resize(_size837);
-            uint32_t _i841;
-            for (_i841 = 0; _i841 < _size837; ++_i841)
+            uint32_t _size843;
+            ::apache::thrift::protocol::TType _etype846;
+            xfer += iprot->readListBegin(_etype846, _size843);
+            this->filesAddedChecksum.resize(_size843);
+            uint32_t _i847;
+            for (_i847 = 0; _i847 < _size843; ++_i847)
             {
-              xfer += iprot->readString(this->filesAddedChecksum[_i841]);
+              xfer += iprot->readString(this->filesAddedChecksum[_i847]);
             }
             xfer += iprot->readListEnd();
           }
@@ -21575,10 +21641,10 @@ uint32_t InsertEventRequestData::write(::apache::thrift::protocol::TProtocol* op
   xfer += oprot->writeFieldBegin("filesAdded", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->filesAdded.size()));
-    std::vector<std::string> ::const_iterator _iter842;
-    for (_iter842 = this->filesAdded.begin(); _iter842 != this->filesAdded.end(); ++_iter842)
+    std::vector<std::string> ::const_iterator _iter848;
+    for (_iter848 = this->filesAdded.begin(); _iter848 != this->filesAdded.end(); ++_iter848)
     {
-      xfer += oprot->writeString((*_iter842));
+      xfer += oprot->writeString((*_iter848));
     }
     xfer += oprot->writeListEnd();
   }
@@ -21588,10 +21654,10 @@ uint32_t InsertEventRequestData::write(::apache::thrift::protocol::TProtocol* op
     xfer += oprot->writeFieldBegin("filesAddedChecksum", ::apache::thrift::protocol::T_LIST, 3);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->filesAddedChecksum.size()));
-      std::vector<std::string> ::const_iterator _iter843;
-      for (_iter843 = this->filesAddedChecksum.begin(); _iter843 != this->filesAddedChecksum.end(); ++_iter843)
+      std::vector<std::string> ::const_iterator _iter849;
+      for (_iter849 = this->filesAddedChecksum.begin(); _iter849 != this->filesAddedChecksum.end(); ++_iter849)
       {
-        xfer += oprot->writeString((*_iter843));
+        xfer += oprot->writeString((*_iter849));
       }
       xfer += oprot->writeListEnd();
     }
@@ -21610,17 +21676,17 @@ void swap(InsertEventRequestData &a, InsertEventRequestData &b) {
   swap(a.__isset, b.__isset);
 }
 
-InsertEventRequestData::InsertEventRequestData(const InsertEventRequestData& other844) {
-  replace = other844.replace;
-  filesAdded = other844.filesAdded;
-  filesAddedChecksum = other844.filesAddedChecksum;
-  __isset = other844.__isset;
+InsertEventRequestData::InsertEventRequestData(const InsertEventRequestData& other850) {
+  replace = other850.replace;
+  filesAdded = other850.filesAdded;
+  filesAddedChecksum = other850.filesAddedChecksum;
+  __isset = other850.__isset;
 }
-InsertEventRequestData& InsertEventRequestData::operator=(const InsertEventRequestData& other845) {
-  replace = other845.replace;
-  filesAdded = other845.filesAdded;
-  filesAddedChecksum = other845.filesAddedChecksum;
-  __isset = other845.__isset;
+InsertEventRequestData& InsertEventRequestData::operator=(const InsertEventRequestData& other851) {
+  replace = other851.replace;
+  filesAdded = other851.filesAdded;
+  filesAddedChecksum = other851.filesAddedChecksum;
+  __isset = other851.__isset;
   return *this;
 }
 void InsertEventRequestData::printTo(std::ostream& out) const {
@@ -21702,13 +21768,13 @@ void swap(FireEventRequestData &a, FireEventRequestData &b) {
   swap(a.__isset, b.__isset);
 }
 
-FireEventRequestData::FireEventRequestData(const FireEventRequestData& other846) {
-  insertData = other846.insertData;
-  __isset = other846.__isset;
+FireEventRequestData::FireEventRequestData(const FireEventRequestData& other852) {
+  insertData = other852.insertData;
+  __isset = other852.__isset;
 }
-FireEventRequestData& FireEventRequestData::operator=(const FireEventRequestData& other847) {
-  insertData = other847.insertData;
-  __isset = other847.__isset;
+FireEventRequestData& FireEventRequestData::operator=(const FireEventRequestData& other853) {
+  insertData = other853.insertData;
+  __isset = other853.__isset;
   return *this;
 }
 void FireEventRequestData::printTo(std::ostream& out) const {
@@ -21810,14 +21876,14 @@ uint32_t FireEventRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitionVals.clear();
-            uint32_t _size848;
-            ::apache::thrift::protocol::TType _etype851;
-            xfer += iprot->readListBegin(_etype851, _size848);
-            this->partitionVals.resize(_size848);
-            uint32_t _i852;
-            for (_i852 = 0; _i852 < _size848; ++_i852)
+            uint32_t _size854;
+            ::apache::thrift::protocol::TType _etype857;
+            xfer += iprot->readListBegin(_etype857, _size854);
+            this->partitionVals.resize(_size854);
+            uint32_t _i858;
+            for (_i858 = 0; _i858 < _size854; ++_i858)
             {
-              xfer += iprot->readString(this->partitionVals[_i852]);
+              xfer += iprot->readString(this->partitionVals[_i858]);
             }
             xfer += iprot->readListEnd();
           }
@@ -21877,10 +21943,10 @@ uint32_t FireEventRequest::write(::apache::thrift::protocol::TProtocol* oprot) c
     xfer += oprot->writeFieldBegin("partitionVals", ::apache::thrift::protocol::T_LIST, 5);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionVals.size()));
-      std::vector<std::string> ::const_iterator _iter853;
-      for (_iter853 = this->partitionVals.begin(); _iter853 != this->partitionVals.end(); ++_iter853)
+      std::vector<std::string> ::const_iterator _iter859;
+      for (_iter859 = this->partitionVals.begin(); _iter859 != this->partitionVals.end(); ++_iter859)
       {
-        xfer += oprot->writeString((*_iter853));
+        xfer += oprot->writeString((*_iter859));
       }
       xfer += oprot->writeListEnd();
     }
@@ -21907,23 +21973,23 @@ void swap(FireEventRequest &a, FireEventRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-FireEventRequest::FireEventRequest(const FireEventRequest& other854) {
-  successful = other854.successful;
-  data = other854.data;
-  dbName = other854.dbName;
-  tableName = other854.tableName;
-  partitionVals = other854.partitionVals;
-  catName = other854.catName;
-  __isset = other854.__isset;
-}
-FireEventRequest& FireEventRequest::operator=(const FireEventRequest& other855) {
-  successful = other855.successful;
-  data = other855.data;
-  dbName = other855.dbName;
-  tableName = other855.tableName;
-  partitionVals = other855.partitionVals;
-  catName = other855.catName;
-  __isset = other855.__isset;
+FireEventRequest::FireEventRequest(const FireEventRequest& other860) {
+  successful = other860.successful;
+  data = other860.data;
+  dbName = other860.dbName;
+  tableName = other860.tableName;
+  partitionVals = other860.partitionVals;
+  catName = other860.catName;
+  __isset = other860.__isset;
+}
+FireEventRequest& FireEventRequest::operator=(const FireEventRequest& other861) {
+  successful = other861.successful;
+  data = other861.data;
+  dbName = other861.dbName;
+  tableName = other861.tableName;
+  partitionVals = other861.partitionVals;
+  catName = other861.catName;
+  __isset = other861.__isset;
   return *this;
 }
 void FireEventRequest::printTo(std::ostream& out) const {
@@ -21987,11 +22053,11 @@ void swap(FireEventResponse &a, FireEventResponse &b) {
   (void) b;
 }
 
-FireEventResponse::FireEventResponse(const FireEventResponse& other856) {
-  (void) other856;
+FireEventResponse::FireEventResponse(const FireEventResponse& other862) {
+  (void) other862;
 }
-FireEventResponse& FireEventResponse::operator=(const FireEventResponse& other857) {
-  (void) other857;
+FireEventResponse& FireEventResponse::operator=(const FireEventResponse& other863) {
+  (void) other863;
   return *this;
 }
 void FireEventResponse::printTo(std::ostream& out) const {
@@ -22091,15 +22157,15 @@ void swap(MetadataPpdResult &a, MetadataPpdResult &b) {
   swap(a.__isset, b.__isset);
 }
 
-MetadataPpdResult::MetadataPpdResult(const MetadataPpdResult& other858) {
-  metadata = other858.metadata;
-  includeBitset = other858.includeBitset;
-  __isset = other858.__isset;
+MetadataPpdResult::MetadataPpdResult(const MetadataPpdResult& other864) {
+  metadata = other864.metadata;
+  includeBitset = other864.includeBitset;
+  __isset = other864.__isset;
 }
-MetadataPpdResult& MetadataPpdResult::operator=(const MetadataPpdResult& other859) {
-  metadata = other859.metadata;
-  includeBitset = other859.includeBitset;
-  __isset = other859.__isset;
+MetadataPpdResult& MetadataPpdResult::operator=(const MetadataPpdResult& other865) {
+  metadata = other865.metadata;
+  includeBitset = other865.includeBitset;
+  __isset = other865.__isset;
   return *this;
 }
 void MetadataPpdResult::printTo(std::ostream& out) const {
@@ -22150,17 +22216,17 @@ uint32_t GetFileMetadataByExprResult::read(::apache::thrift::protocol::TProtocol
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->metadata.clear();
-            uint32_t _size860;
-            ::apache::thrift::protocol::TType _ktype861;
-            ::apache::thrift::protocol::TType _vtype862;
-            xfer += iprot->readMapBegin(_ktype861, _vtype862, _size860);
-            uint32_t _i864;
-            for (_i864 = 0; _i864 < _size860; ++_i864)
+            uint32_t _size866;
+            ::apache::thrift::protocol::TType _ktype867;
+            ::apache::thrift::protocol::TType _vtype868;
+            xfer += iprot->readMapBegin(_ktype867, _vtype868, _size866);
+            uint32_t _i870;
+            for (_i870 = 0; _i870 < _size866; ++_i870)
             {
-              int64_t _key865;
-              xfer += iprot->readI64(_key865);
-              MetadataPpdResult& _val866 = this->metadata[_key865];
-              xfer += _val866.read(iprot);
+              int64_t _key871;
+              xfer += iprot->readI64(_key871);
+              MetadataPpdResult& _val872 = this->metadata[_key871];
+              xfer += _val872.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -22201,11 +22267,11 @@ uint32_t GetFileMetadataByExprResult::write(::apache::thrift::protocol::TProtoco
   xfer += oprot->writeFieldBegin("metadata", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_I64, ::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->metadata.size()));
-    std::map<int64_t, MetadataPpdResult> ::const_iterator _iter867;
-    for (_iter867 = this->metadata.begin(); _iter867 != this->metadata.end(); ++_iter867)
+    std::map<int64_t, MetadataPpdResult> ::const_iterator _iter873;
+    for (_iter873 = this->metadata.begin(); _iter873 != this->metadata.end(); ++_iter873)
     {
-      xfer += oprot->writeI64(_iter867->first);
-      xfer += _iter867->second.write(oprot);
+      xfer += oprot->writeI64(_iter873->first);
+      xfer += _iter873->second.write(oprot);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -22226,13 +22292,13 @@ void swap(GetFileMetadataByExprResult &a, GetFileMetadataByExprResult &b) {
   swap(a.isSupported, b.isSupported);
 }
 
-GetFileMetadataByExprResult::GetFileMetadataByExprResult(const GetFileMetadataByExprResult& other868) {
-  metadata = other868.metadata;
-  isSupported = other868.isSupported;
+GetFileMetadataByExprResult::GetFileMetadataByExprResult(const GetFileMetadataByExprResult& other874) {
+  metadata = other874.metadata;
+  isSupported = other874.isSupported;
 }
-GetFileMetadataByExprResult& GetFileMetadataByExprResult::operator=(const GetFileMetadataByExprResult& other869) {
-  metadata = other869.metadata;
-  isSupported = other869.isSupported;
+GetFileMetadataByExprResult& GetFileMetadataByExprResult::operator=(const GetFileMetadataByExprResult& other875) {
+  metadata = other875.metadata;
+  isSupported = other875.isSupported;
   return *this;
 }
 void GetFileMetadataByExprResult::printTo(std::ostream& out) const {
@@ -22293,14 +22359,14 @@ uint32_t GetFileMetadataByExprRequest::read(::apache::thrift::protocol::TProtoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->fileIds.clear();
-            uint32_t _size870;
-            ::apache::thrift::protocol::TType _etype873;
-            xfer += iprot->readListBegin(_etype873, _size870);
-            this->fileIds.resize(_size870);
-            uint32_t _i874;
-            for (_i874 = 0; _i874 < _size870; ++_i874)
+            uint32_t _size876;
+            ::apache::thrift::protocol::TType _etype879;
+            xfer += iprot->readListBegin(_etype879, _size876);
+            this->fileIds.resize(_size876);
+            uint32_t _i880;
+            for (_i880 = 0; _i880 < _size876; ++_i880)
             {
-              xfer += iprot->readI64(this->fileIds[_i874]);
+              xfer += iprot->readI64(this->fileIds[_i880]);
             }
             xfer += iprot->readListEnd();
           }
@@ -22327,9 +22393,9 @@ uint32_t GetFileMetadataByExprRequest::read(::apache::thrift::protocol::TProtoco
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast875;
-          xfer += iprot->readI32(ecast875);
-          this->type = (FileMetadataExprType::type)ecast875;
+          int32_t ecast881;
+          xfer += iprot->readI32(ecast881);
+          this->type = (FileMetadataExprType::type)ecast881;
           this->__isset.type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -22359,10 +22425,10 @@ uint32_t GetFileMetadataByExprRequest::write(::apache::thrift::protocol::TProtoc
   xfer += oprot->writeFieldBegin("fileIds", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->fileIds.size()));
-    std::vector<int64_t> ::const_iterator _iter876;
-    for (_iter876 = this->fileIds.begin(); _iter876 != this->fileIds.end(); ++_iter876)
+    std::vector<int64_t> ::const_iterator _iter882;
+    for (_iter882 = this->fileIds.begin(); _iter882 != this->fileIds.end(); ++_iter882)
     {
-      xfer += oprot->writeI64((*_iter876));
+      xfer += oprot->writeI64((*_iter882));
     }
     xfer += oprot->writeListEnd();
   }
@@ -22396,19 +22462,19 @@ void swap(GetFileMetadataByExprRequest &a, GetFileMetadataByExprRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-GetFileMetadataByExprRequest::GetFileMetadataByExprRequest(const GetFileMetadataByExprRequest& other877) {
-  fileIds = other877.fileIds;
-  expr = other877.expr;
-  doGetFooters = other877.doGetFooters;
-  type = other877.type;
-  __isset = other877.__isset;
+GetFileMetadataByExprRequest::GetFileMetadataByExprRequest(const GetFileMetadataByExprRequest& other883) {
+  fileIds = other883.fileIds;
+  expr = other883.expr;
+  doGetFooters = other883.doGetFooters;
+  type = other883.type;
+  __isset = other883.__isset;
 }
-GetFileMetadataByExprRequest& GetFileMetadataByExprRequest::operator=(const GetFileMetadataByExprRequest& other878) {
-  fileIds = other878.fileIds;
-  expr = other878.expr;
-  doGetFooters = other878.doGetFooters;
-  type = other878.type;
-  __isset = other878.__isset;
+GetFileMetadataByExprRequest& GetFileMetadataByExprRequest::operator=(const GetFileMetadataByExprRequest& other884) {
+  fileIds = other884.fileIds;
+  expr = other884.expr;
+  doGetFooters = other884.doGetFooters;
+  type = other884.type;
+  __isset = other884.__isset;
   return *this;
 }
 void GetFileMetadataByExprRequest::printTo(std::ostream& out) const {
@@ -22461,17 +22527,17 @@ uint32_t GetFileMetadataResult::read(::apache::thrift::protocol::TProtocol* ipro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->metadata.clear();
-            uint32_t _size879;
-            ::apache::thrift::protocol::TType _ktype880;
-            ::apache::thrift::protocol::TType _vtype881;
-            xfer += iprot->readMapBegin(_ktype880, _vtype881, _size879);
-            uint32_t _i883;
-            for (_i883 = 0; _i883 < _size879; ++_i883)
+            uint32_t _size885;
+            ::apache::thrift::protocol::TType _ktype886;
+            ::apache::thrift::protocol::TType _vtype887;
+            xfer += iprot->readMapBegin(_ktype886, _vtype887, _size885);
+            uint32_t _i889;
+            for (_i889 = 0; _i889 < _size885; ++_i889)
             {
-              int64_t _key884;
-              xfer += iprot->readI64(_key884);
-              std::string& _val885 = this->metadata[_key884];
-              xfer += iprot->readBinary(_val885);
+              int64_t _key890;
+              xfer += iprot->readI64(_key890);
+              std::string& _val891 = this->metadata[_key890];
+              xfer += iprot->readBinary(_val891);
             }
             xfer += iprot->readMapEnd();
           }
@@ -22512,11 +22578,11 @@ uint32_t GetFileMetadataResult::write(::apache::thrift::protocol::TProtocol* opr
   xfer += oprot->writeFieldBegin("metadata", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_I64, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->metadata.size()));
-    std::map<int64_t, std::string> ::const_iterator _iter886;
-    for (_iter886 = this->metadata.begin(); _iter886 != this->metadata.end(); ++_iter886)
+    std::map<int64_t, std::string> ::const_iterator _iter892;
+    for (_iter892 = this->metadata.begin(); _iter892 != this->metadata.end(); ++_iter892)
     {
-      xfer += oprot->writeI64(_iter886->first);
-      xfer += oprot->writeBinary(_iter886->second);
+      xfer += oprot->writeI64(_iter892->first);
+      xfer += oprot->writeBinary(_iter892->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -22537,13 +22603,13 @@ void swap(GetFileMetadataResult &a, GetFileMetadataResult &b) {
   swap(a.isSupported, b.isSupported);
 }
 
-GetFileMetadataResult::GetFileMetadataResult(const GetFileMetadataResult& other887) {
-  metadata = other887.metadata;
-  isSupported = other887.isSupported;
+GetFileMetadataResult::GetFileMetadataResult(const GetFileMetadataResult& other893) {
+  metadata = other893.metadata;
+  isSupported = other893.isSupported;
 }
-GetFileMetadataResult& GetFileMetadataResult::operator=(const GetFileMetadataResult& other888) {
-  metadata = other888.metadata;
-  isSupported = other888.isSupported;
+GetFileMetadataResult& GetFileMetadataResult::operator=(const GetFileMetadataResult& other894) {
+  metadata = other894.metadata;
+  isSupported = other894.isSupported;
   return *this;
 }
 void GetFileMetadataResult::printTo(std::ostream& out) const {
@@ -22589,14 +22655,14 @@ uint32_t GetFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->fileIds.clear();
-            uint32_t _size889;
-            ::apache::thrift::protocol::TType _etype892;
-            xfer += iprot->readListBegin(_etype892, _size889);
-            this->fileIds.resize(_size889);
-            uint32_t _i893;
-            for (_i893 = 0; _i893 < _size889; ++_i893)
+            uint32_t _size895;
+            ::apache::thrift::protocol::TType _etype898;
+            xfer += iprot->readListBegin(_etype898, _size895);
+            this->fileIds.resize(_size895);
+            uint32_t _i899;
+            for (_i899 = 0; _i899 < _size895; ++_i899)
             {
-              xfer += iprot->readI64(this->fileIds[_i893]);
+              xfer += iprot->readI64(this->fileIds[_i899]);
             }
             xfer += iprot->readListEnd();
           }
@@ -22627,10 +22693,10 @@ uint32_t GetFileMetadataRequest::write(::apache::thrift::protocol::TProtocol* op
   xfer += oprot->writeFieldBegin("fileIds", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->fileIds.size()));
-    std::vector<int64_t> ::const_iterator _iter894;
-    for (_iter894 = this->fileIds.begin(); _iter894 != this->fileIds.end(); ++_iter894)
+    std::vector<int64_t> ::const_iterator _iter900;
+    for (_iter900 = this->fileIds.begin(); _iter900 != this->fileIds.end(); ++_iter900)
     {
-      xfer += oprot->writeI64((*_iter894));
+      xfer += oprot->writeI64((*_iter900));
     }
     xfer += oprot->writeListEnd();
   }
@@ -22646,11 +22712,11 @@ void swap(GetFileMetadataRequest &a, GetFileMetadataRequest &b) {
   swap(a.fileIds, b.fileIds);
 }
 
-GetFileMetadataRequest::GetFileMetadataRequest(const GetFileMetadataRequest& other895) {
-  fileIds = other895.fileIds;
+GetFileMetadataRequest::GetFileMetadataRequest(const GetFileMetadataRequest& other901) {
+  fileIds = other901.fileIds;
 }
-GetFileMetadataRequest& GetFileMetadataRequest::operator=(const GetFileMetadataRequest& other896) {
-  fileIds = other896.fileIds;
+GetFileMetadataRequest& GetFileMetadataRequest::operator=(const GetFileMetadataRequest& other902) {
+  fileIds = other902.fileIds;
   return *this;
 }
 void GetFileMetadataRequest::printTo(std::ostream& out) const {
@@ -22709,11 +22775,11 @@ void swap(PutFileMetadataResult &a, PutFileMetadataResult &b) {
   (void) b;
 }
 
-PutFileMetadataResult::PutFileMetadataResult(const PutFileMetadataResult& other897) {
-  (void) other897;
+PutFileMetadataResult::PutFileMetadataResult(const PutFileMetadataResult& other903) {
+  (void) other903;
 }
-PutFileMetadataResult& PutFileMetadataResult::operator=(const PutFileMetadataResult& other898) {
-  (void) other898;
+PutFileMetadataResult& PutFileMetadataResult::operator=(const PutFileMetadataResult& other904) {
+  (void) other904;
   return *this;
 }
 void PutFileMetadataResult::printTo(std::ostream& out) const {
@@ -22767,14 +22833,14 @@ uint32_t PutFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->fileIds.clear();
-            uint32_t _size899;
-            ::apache::thrift::protocol::TType _etype902;
-            xfer += iprot->readListBegin(_etype902, _size899);
-            this->fileIds.resize(_size899);
-            uint32_t _i903;
-            for (_i903 = 0; _i903 < _size899; ++_i903)
+            uint32_t _size905;
+            ::apache::thrift::protocol::TType _etype908;
+            xfer += iprot->readListBegin(_etype908, _size905);
+            this->fileIds.resize(_size905);
+            uint32_t _i909;
+            for (_i909 = 0; _i909 < _size905; ++_i909)
             {
-              xfer += iprot->readI64(this->fileIds[_i903]);
+              xfer += iprot->readI64(this->fileIds[_i909]);
             }
             xfer += iprot->readListEnd();
           }
@@ -22787,14 +22853,14 @@ uint32_t PutFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->metadata.clear();
-            uint32_t _size904;
-            ::apache::thrift::protocol::TType _etype907;
-            xfer += iprot->readListBegin(_etype907, _size904);
-            this->metadata.resize(_size904);
-            uint32_t _i908;
-            for (_i908 = 0; _i908 < _size904; ++_i908)
+            uint32_t _size910;
+            ::apache::thrift::protocol::TType _etype913;
+            xfer += iprot->readListBegin(_etype913, _size910);
+            this->metadata.resize(_size910);
+            uint32_t _i914;
+            for (_i914 = 0; _i914 < _size910; ++_i914)
             {
-              xfer += iprot->readBinary(this->metadata[_i908]);
+              xfer += iprot->readBinary(this->metadata[_i914]);
             }
             xfer += iprot->readListEnd();
           }
@@ -22805,9 +22871,9 @@ uint32_t PutFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* ipr
         break;
       case 3:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast909;
-          xfer += iprot->readI32(ecast909);
-          this->type = (FileMetadataExprType::type)ecast909;
+          int32_t ecast915;
+          xfer += iprot->readI32(ecast915);
+          t

<TRUNCATED>

[09/12] hive git commit: HIVE-19089: Create/Replicate Allocate write-id event (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 55a97de..2c95007 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
@@ -7114,34 +7114,58 @@ inline std::ostream& operator<<(std::ostream& out, const GetValidWriteIdsRespons
   return out;
 }
 
+typedef struct _AllocateTableWriteIdsRequest__isset {
+  _AllocateTableWriteIdsRequest__isset() : txnIds(false), replPolicy(false), srcTxnToWriteIdList(false) {}
+  bool txnIds :1;
+  bool replPolicy :1;
+  bool srcTxnToWriteIdList :1;
+} _AllocateTableWriteIdsRequest__isset;
 
 class AllocateTableWriteIdsRequest {
  public:
 
   AllocateTableWriteIdsRequest(const AllocateTableWriteIdsRequest&);
   AllocateTableWriteIdsRequest& operator=(const AllocateTableWriteIdsRequest&);
-  AllocateTableWriteIdsRequest() : dbName(), tableName() {
+  AllocateTableWriteIdsRequest() : dbName(), tableName(), replPolicy() {
   }
 
   virtual ~AllocateTableWriteIdsRequest() throw();
-  std::vector<int64_t>  txnIds;
   std::string dbName;
   std::string tableName;
+  std::vector<int64_t>  txnIds;
+  std::string replPolicy;
+  std::vector<TxnToWriteId>  srcTxnToWriteIdList;
 
-  void __set_txnIds(const std::vector<int64_t> & val);
+  _AllocateTableWriteIdsRequest__isset __isset;
 
   void __set_dbName(const std::string& val);
 
   void __set_tableName(const std::string& val);
 
+  void __set_txnIds(const std::vector<int64_t> & val);
+
+  void __set_replPolicy(const std::string& val);
+
+  void __set_srcTxnToWriteIdList(const std::vector<TxnToWriteId> & val);
+
   bool operator == (const AllocateTableWriteIdsRequest & rhs) const
   {
-    if (!(txnIds == rhs.txnIds))
-      return false;
     if (!(dbName == rhs.dbName))
       return false;
     if (!(tableName == rhs.tableName))
       return false;
+    if (__isset.txnIds != rhs.__isset.txnIds)
+      return false;
+    else if (__isset.txnIds && !(txnIds == rhs.txnIds))
+      return false;
+    if (__isset.replPolicy != rhs.__isset.replPolicy)
+      return false;
+    else if (__isset.replPolicy && !(replPolicy == rhs.replPolicy))
+      return false;
+    if (__isset.srcTxnToWriteIdList != rhs.__isset.srcTxnToWriteIdList)
+      return false;
+    else if (__isset.srcTxnToWriteIdList && !(srcTxnToWriteIdList == rhs.srcTxnToWriteIdList))
+      return false;
     return true;
   }
   bool operator != (const AllocateTableWriteIdsRequest &rhs) const {

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 3acdec5..6411129 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
@@ -816,13 +816,13 @@ import org.slf4j.LoggerFactory;
           case 5: // PARTITIONNAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list684 = iprot.readListBegin();
-                struct.partitionnames = new ArrayList<String>(_list684.size);
-                String _elem685;
-                for (int _i686 = 0; _i686 < _list684.size; ++_i686)
+                org.apache.thrift.protocol.TList _list692 = iprot.readListBegin();
+                struct.partitionnames = new ArrayList<String>(_list692.size);
+                String _elem693;
+                for (int _i694 = 0; _i694 < _list692.size; ++_i694)
                 {
-                  _elem685 = iprot.readString();
-                  struct.partitionnames.add(_elem685);
+                  _elem693 = iprot.readString();
+                  struct.partitionnames.add(_elem693);
                 }
                 iprot.readListEnd();
               }
@@ -872,9 +872,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 _iter687 : struct.partitionnames)
+          for (String _iter695 : struct.partitionnames)
           {
-            oprot.writeString(_iter687);
+            oprot.writeString(_iter695);
           }
           oprot.writeListEnd();
         }
@@ -910,9 +910,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeString(struct.tablename);
       {
         oprot.writeI32(struct.partitionnames.size());
-        for (String _iter688 : struct.partitionnames)
+        for (String _iter696 : struct.partitionnames)
         {
-          oprot.writeString(_iter688);
+          oprot.writeString(_iter696);
         }
       }
       BitSet optionals = new BitSet();
@@ -937,13 +937,13 @@ import org.slf4j.LoggerFactory;
       struct.tablename = iprot.readString();
       struct.setTablenameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list689 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.partitionnames = new ArrayList<String>(_list689.size);
-        String _elem690;
-        for (int _i691 = 0; _i691 < _list689.size; ++_i691)
+        org.apache.thrift.protocol.TList _list697 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.partitionnames = new ArrayList<String>(_list697.size);
+        String _elem698;
+        for (int _i699 = 0; _i699 < _list697.size; ++_i699)
         {
-          _elem690 = iprot.readString();
-          struct.partitionnames.add(_elem690);
+          _elem698 = iprot.readString();
+          struct.partitionnames.add(_elem698);
         }
       }
       struct.setPartitionnamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java
index 35ccef7..5a60e95 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java
@@ -38,9 +38,11 @@ import org.slf4j.LoggerFactory;
 @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class AllocateTableWriteIdsRequest implements org.apache.thrift.TBase<AllocateTableWriteIdsRequest, AllocateTableWriteIdsRequest._Fields>, java.io.Serializable, Cloneable, Comparable<AllocateTableWriteIdsRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AllocateTableWriteIdsRequest");
 
-  private static final org.apache.thrift.protocol.TField TXN_IDS_FIELD_DESC = new org.apache.thrift.protocol.TField("txnIds", org.apache.thrift.protocol.TType.LIST, (short)1);
-  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)2);
-  private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)3);
+  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 TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField TXN_IDS_FIELD_DESC = new org.apache.thrift.protocol.TField("txnIds", org.apache.thrift.protocol.TType.LIST, (short)3);
+  private static final org.apache.thrift.protocol.TField REPL_POLICY_FIELD_DESC = new org.apache.thrift.protocol.TField("replPolicy", org.apache.thrift.protocol.TType.STRING, (short)4);
+  private static final org.apache.thrift.protocol.TField SRC_TXN_TO_WRITE_ID_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("srcTxnToWriteIdList", org.apache.thrift.protocol.TType.LIST, (short)5);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -48,15 +50,19 @@ import org.slf4j.LoggerFactory;
     schemes.put(TupleScheme.class, new AllocateTableWriteIdsRequestTupleSchemeFactory());
   }
 
-  private List<Long> txnIds; // required
   private String dbName; // required
   private String tableName; // required
+  private List<Long> txnIds; // optional
+  private String replPolicy; // optional
+  private List<TxnToWriteId> srcTxnToWriteIdList; // 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 {
-    TXN_IDS((short)1, "txnIds"),
-    DB_NAME((short)2, "dbName"),
-    TABLE_NAME((short)3, "tableName");
+    DB_NAME((short)1, "dbName"),
+    TABLE_NAME((short)2, "tableName"),
+    TXN_IDS((short)3, "txnIds"),
+    REPL_POLICY((short)4, "replPolicy"),
+    SRC_TXN_TO_WRITE_ID_LIST((short)5, "srcTxnToWriteIdList");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -71,12 +77,16 @@ import org.slf4j.LoggerFactory;
      */
     public static _Fields findByThriftId(int fieldId) {
       switch(fieldId) {
-        case 1: // TXN_IDS
-          return TXN_IDS;
-        case 2: // DB_NAME
+        case 1: // DB_NAME
           return DB_NAME;
-        case 3: // TABLE_NAME
+        case 2: // TABLE_NAME
           return TABLE_NAME;
+        case 3: // TXN_IDS
+          return TXN_IDS;
+        case 4: // REPL_POLICY
+          return REPL_POLICY;
+        case 5: // SRC_TXN_TO_WRITE_ID_LIST
+          return SRC_TXN_TO_WRITE_ID_LIST;
         default:
           return null;
       }
@@ -117,16 +127,22 @@ import org.slf4j.LoggerFactory;
   }
 
   // isset id assignments
+  private static final _Fields optionals[] = {_Fields.TXN_IDS,_Fields.REPL_POLICY,_Fields.SRC_TXN_TO_WRITE_ID_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.TXN_IDS, new org.apache.thrift.meta_data.FieldMetaData("txnIds", 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))));
     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.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TXN_IDS, new org.apache.thrift.meta_data.FieldMetaData("txnIds", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
+    tmpMap.put(_Fields.REPL_POLICY, new org.apache.thrift.meta_data.FieldMetaData("replPolicy", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.SRC_TXN_TO_WRITE_ID_LIST, new org.apache.thrift.meta_data.FieldMetaData("srcTxnToWriteIdList", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT            , "TxnToWriteId"))));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(AllocateTableWriteIdsRequest.class, metaDataMap);
   }
@@ -135,12 +151,10 @@ import org.slf4j.LoggerFactory;
   }
 
   public AllocateTableWriteIdsRequest(
-    List<Long> txnIds,
     String dbName,
     String tableName)
   {
     this();
-    this.txnIds = txnIds;
     this.dbName = dbName;
     this.tableName = tableName;
   }
@@ -149,16 +163,26 @@ import org.slf4j.LoggerFactory;
    * Performs a deep copy on <i>other</i>.
    */
   public AllocateTableWriteIdsRequest(AllocateTableWriteIdsRequest other) {
-    if (other.isSetTxnIds()) {
-      List<Long> __this__txnIds = new ArrayList<Long>(other.txnIds);
-      this.txnIds = __this__txnIds;
-    }
     if (other.isSetDbName()) {
       this.dbName = other.dbName;
     }
     if (other.isSetTableName()) {
       this.tableName = other.tableName;
     }
+    if (other.isSetTxnIds()) {
+      List<Long> __this__txnIds = new ArrayList<Long>(other.txnIds);
+      this.txnIds = __this__txnIds;
+    }
+    if (other.isSetReplPolicy()) {
+      this.replPolicy = other.replPolicy;
+    }
+    if (other.isSetSrcTxnToWriteIdList()) {
+      List<TxnToWriteId> __this__srcTxnToWriteIdList = new ArrayList<TxnToWriteId>(other.srcTxnToWriteIdList.size());
+      for (TxnToWriteId other_element : other.srcTxnToWriteIdList) {
+        __this__srcTxnToWriteIdList.add(other_element);
+      }
+      this.srcTxnToWriteIdList = __this__srcTxnToWriteIdList;
+    }
   }
 
   public AllocateTableWriteIdsRequest deepCopy() {
@@ -167,11 +191,59 @@ import org.slf4j.LoggerFactory;
 
   @Override
   public void clear() {
+    this.dbName = null;
+    this.tableName = null;
     this.txnIds = null;
+    this.replPolicy = null;
+    this.srcTxnToWriteIdList = 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 getTableName() {
+    return this.tableName;
+  }
+
+  public void setTableName(String tableName) {
+    this.tableName = tableName;
+  }
+
+  public void unsetTableName() {
     this.tableName = null;
   }
 
+  /** Returns true if field tableName is set (has been assigned a value) and false otherwise */
+  public boolean isSetTableName() {
+    return this.tableName != null;
+  }
+
+  public void setTableNameIsSet(boolean value) {
+    if (!value) {
+      this.tableName = null;
+    }
+  }
+
   public int getTxnIdsSize() {
     return (this.txnIds == null) ? 0 : this.txnIds.size();
   }
@@ -210,54 +282,85 @@ import org.slf4j.LoggerFactory;
     }
   }
 
-  public String getDbName() {
-    return this.dbName;
+  public String getReplPolicy() {
+    return this.replPolicy;
   }
 
-  public void setDbName(String dbName) {
-    this.dbName = dbName;
+  public void setReplPolicy(String replPolicy) {
+    this.replPolicy = replPolicy;
   }
 
-  public void unsetDbName() {
-    this.dbName = null;
+  public void unsetReplPolicy() {
+    this.replPolicy = null;
   }
 
-  /** Returns true if field dbName is set (has been assigned a value) and false otherwise */
-  public boolean isSetDbName() {
-    return this.dbName != null;
+  /** Returns true if field replPolicy is set (has been assigned a value) and false otherwise */
+  public boolean isSetReplPolicy() {
+    return this.replPolicy != null;
   }
 
-  public void setDbNameIsSet(boolean value) {
+  public void setReplPolicyIsSet(boolean value) {
     if (!value) {
-      this.dbName = null;
+      this.replPolicy = null;
     }
   }
 
-  public String getTableName() {
-    return this.tableName;
+  public int getSrcTxnToWriteIdListSize() {
+    return (this.srcTxnToWriteIdList == null) ? 0 : this.srcTxnToWriteIdList.size();
   }
 
-  public void setTableName(String tableName) {
-    this.tableName = tableName;
+  public java.util.Iterator<TxnToWriteId> getSrcTxnToWriteIdListIterator() {
+    return (this.srcTxnToWriteIdList == null) ? null : this.srcTxnToWriteIdList.iterator();
   }
 
-  public void unsetTableName() {
-    this.tableName = null;
+  public void addToSrcTxnToWriteIdList(TxnToWriteId elem) {
+    if (this.srcTxnToWriteIdList == null) {
+      this.srcTxnToWriteIdList = new ArrayList<TxnToWriteId>();
+    }
+    this.srcTxnToWriteIdList.add(elem);
   }
 
-  /** Returns true if field tableName is set (has been assigned a value) and false otherwise */
-  public boolean isSetTableName() {
-    return this.tableName != null;
+  public List<TxnToWriteId> getSrcTxnToWriteIdList() {
+    return this.srcTxnToWriteIdList;
   }
 
-  public void setTableNameIsSet(boolean value) {
+  public void setSrcTxnToWriteIdList(List<TxnToWriteId> srcTxnToWriteIdList) {
+    this.srcTxnToWriteIdList = srcTxnToWriteIdList;
+  }
+
+  public void unsetSrcTxnToWriteIdList() {
+    this.srcTxnToWriteIdList = null;
+  }
+
+  /** Returns true if field srcTxnToWriteIdList is set (has been assigned a value) and false otherwise */
+  public boolean isSetSrcTxnToWriteIdList() {
+    return this.srcTxnToWriteIdList != null;
+  }
+
+  public void setSrcTxnToWriteIdListIsSet(boolean value) {
     if (!value) {
-      this.tableName = null;
+      this.srcTxnToWriteIdList = null;
     }
   }
 
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
+    case DB_NAME:
+      if (value == null) {
+        unsetDbName();
+      } else {
+        setDbName((String)value);
+      }
+      break;
+
+    case TABLE_NAME:
+      if (value == null) {
+        unsetTableName();
+      } else {
+        setTableName((String)value);
+      }
+      break;
+
     case TXN_IDS:
       if (value == null) {
         unsetTxnIds();
@@ -266,19 +369,19 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
-    case DB_NAME:
+    case REPL_POLICY:
       if (value == null) {
-        unsetDbName();
+        unsetReplPolicy();
       } else {
-        setDbName((String)value);
+        setReplPolicy((String)value);
       }
       break;
 
-    case TABLE_NAME:
+    case SRC_TXN_TO_WRITE_ID_LIST:
       if (value == null) {
-        unsetTableName();
+        unsetSrcTxnToWriteIdList();
       } else {
-        setTableName((String)value);
+        setSrcTxnToWriteIdList((List<TxnToWriteId>)value);
       }
       break;
 
@@ -287,15 +390,21 @@ import org.slf4j.LoggerFactory;
 
   public Object getFieldValue(_Fields field) {
     switch (field) {
-    case TXN_IDS:
-      return getTxnIds();
-
     case DB_NAME:
       return getDbName();
 
     case TABLE_NAME:
       return getTableName();
 
+    case TXN_IDS:
+      return getTxnIds();
+
+    case REPL_POLICY:
+      return getReplPolicy();
+
+    case SRC_TXN_TO_WRITE_ID_LIST:
+      return getSrcTxnToWriteIdList();
+
     }
     throw new IllegalStateException();
   }
@@ -307,12 +416,16 @@ import org.slf4j.LoggerFactory;
     }
 
     switch (field) {
-    case TXN_IDS:
-      return isSetTxnIds();
     case DB_NAME:
       return isSetDbName();
     case TABLE_NAME:
       return isSetTableName();
+    case TXN_IDS:
+      return isSetTxnIds();
+    case REPL_POLICY:
+      return isSetReplPolicy();
+    case SRC_TXN_TO_WRITE_ID_LIST:
+      return isSetSrcTxnToWriteIdList();
     }
     throw new IllegalStateException();
   }
@@ -330,15 +443,6 @@ import org.slf4j.LoggerFactory;
     if (that == null)
       return false;
 
-    boolean this_present_txnIds = true && this.isSetTxnIds();
-    boolean that_present_txnIds = true && that.isSetTxnIds();
-    if (this_present_txnIds || that_present_txnIds) {
-      if (!(this_present_txnIds && that_present_txnIds))
-        return false;
-      if (!this.txnIds.equals(that.txnIds))
-        return false;
-    }
-
     boolean this_present_dbName = true && this.isSetDbName();
     boolean that_present_dbName = true && that.isSetDbName();
     if (this_present_dbName || that_present_dbName) {
@@ -357,6 +461,33 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_txnIds = true && this.isSetTxnIds();
+    boolean that_present_txnIds = true && that.isSetTxnIds();
+    if (this_present_txnIds || that_present_txnIds) {
+      if (!(this_present_txnIds && that_present_txnIds))
+        return false;
+      if (!this.txnIds.equals(that.txnIds))
+        return false;
+    }
+
+    boolean this_present_replPolicy = true && this.isSetReplPolicy();
+    boolean that_present_replPolicy = true && that.isSetReplPolicy();
+    if (this_present_replPolicy || that_present_replPolicy) {
+      if (!(this_present_replPolicy && that_present_replPolicy))
+        return false;
+      if (!this.replPolicy.equals(that.replPolicy))
+        return false;
+    }
+
+    boolean this_present_srcTxnToWriteIdList = true && this.isSetSrcTxnToWriteIdList();
+    boolean that_present_srcTxnToWriteIdList = true && that.isSetSrcTxnToWriteIdList();
+    if (this_present_srcTxnToWriteIdList || that_present_srcTxnToWriteIdList) {
+      if (!(this_present_srcTxnToWriteIdList && that_present_srcTxnToWriteIdList))
+        return false;
+      if (!this.srcTxnToWriteIdList.equals(that.srcTxnToWriteIdList))
+        return false;
+    }
+
     return true;
   }
 
@@ -364,11 +495,6 @@ import org.slf4j.LoggerFactory;
   public int hashCode() {
     List<Object> list = new ArrayList<Object>();
 
-    boolean present_txnIds = true && (isSetTxnIds());
-    list.add(present_txnIds);
-    if (present_txnIds)
-      list.add(txnIds);
-
     boolean present_dbName = true && (isSetDbName());
     list.add(present_dbName);
     if (present_dbName)
@@ -379,6 +505,21 @@ import org.slf4j.LoggerFactory;
     if (present_tableName)
       list.add(tableName);
 
+    boolean present_txnIds = true && (isSetTxnIds());
+    list.add(present_txnIds);
+    if (present_txnIds)
+      list.add(txnIds);
+
+    boolean present_replPolicy = true && (isSetReplPolicy());
+    list.add(present_replPolicy);
+    if (present_replPolicy)
+      list.add(replPolicy);
+
+    boolean present_srcTxnToWriteIdList = true && (isSetSrcTxnToWriteIdList());
+    list.add(present_srcTxnToWriteIdList);
+    if (present_srcTxnToWriteIdList)
+      list.add(srcTxnToWriteIdList);
+
     return list.hashCode();
   }
 
@@ -390,6 +531,26 @@ import org.slf4j.LoggerFactory;
 
     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(isSetTableName()).compareTo(other.isSetTableName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTableName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     lastComparison = Boolean.valueOf(isSetTxnIds()).compareTo(other.isSetTxnIds());
     if (lastComparison != 0) {
       return lastComparison;
@@ -400,22 +561,22 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(isSetDbName()).compareTo(other.isSetDbName());
+    lastComparison = Boolean.valueOf(isSetReplPolicy()).compareTo(other.isSetReplPolicy());
     if (lastComparison != 0) {
       return lastComparison;
     }
-    if (isSetDbName()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.dbName, other.dbName);
+    if (isSetReplPolicy()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.replPolicy, other.replPolicy);
       if (lastComparison != 0) {
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName());
+    lastComparison = Boolean.valueOf(isSetSrcTxnToWriteIdList()).compareTo(other.isSetSrcTxnToWriteIdList());
     if (lastComparison != 0) {
       return lastComparison;
     }
-    if (isSetTableName()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName);
+    if (isSetSrcTxnToWriteIdList()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.srcTxnToWriteIdList, other.srcTxnToWriteIdList);
       if (lastComparison != 0) {
         return lastComparison;
       }
@@ -440,14 +601,6 @@ import org.slf4j.LoggerFactory;
     StringBuilder sb = new StringBuilder("AllocateTableWriteIdsRequest(");
     boolean first = true;
 
-    sb.append("txnIds:");
-    if (this.txnIds == null) {
-      sb.append("null");
-    } else {
-      sb.append(this.txnIds);
-    }
-    first = false;
-    if (!first) sb.append(", ");
     sb.append("dbName:");
     if (this.dbName == null) {
       sb.append("null");
@@ -463,16 +616,42 @@ import org.slf4j.LoggerFactory;
       sb.append(this.tableName);
     }
     first = false;
+    if (isSetTxnIds()) {
+      if (!first) sb.append(", ");
+      sb.append("txnIds:");
+      if (this.txnIds == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.txnIds);
+      }
+      first = false;
+    }
+    if (isSetReplPolicy()) {
+      if (!first) sb.append(", ");
+      sb.append("replPolicy:");
+      if (this.replPolicy == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.replPolicy);
+      }
+      first = false;
+    }
+    if (isSetSrcTxnToWriteIdList()) {
+      if (!first) sb.append(", ");
+      sb.append("srcTxnToWriteIdList:");
+      if (this.srcTxnToWriteIdList == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.srcTxnToWriteIdList);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
 
   public void validate() throws org.apache.thrift.TException {
     // check for required fields
-    if (!isSetTxnIds()) {
-      throw new org.apache.thrift.protocol.TProtocolException("Required field 'txnIds' is unset! Struct:" + toString());
-    }
-
     if (!isSetDbName()) {
       throw new org.apache.thrift.protocol.TProtocolException("Required field 'dbName' is unset! Struct:" + toString());
     }
@@ -518,7 +697,23 @@ import org.slf4j.LoggerFactory;
           break;
         }
         switch (schemeField.id) {
-          case 1: // TXN_IDS
+          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: // TABLE_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.tableName = iprot.readString();
+              struct.setTableNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // TXN_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
                 org.apache.thrift.protocol.TList _list618 = iprot.readListBegin();
@@ -536,18 +731,29 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 2: // DB_NAME
+          case 4: // REPL_POLICY
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-              struct.dbName = iprot.readString();
-              struct.setDbNameIsSet(true);
+              struct.replPolicy = iprot.readString();
+              struct.setReplPolicyIsSet(true);
             } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 3: // TABLE_NAME
-            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-              struct.tableName = iprot.readString();
-              struct.setTableNameIsSet(true);
+          case 5: // SRC_TXN_TO_WRITE_ID_LIST
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list621 = iprot.readListBegin();
+                struct.srcTxnToWriteIdList = new ArrayList<TxnToWriteId>(_list621.size);
+                TxnToWriteId _elem622;
+                for (int _i623 = 0; _i623 < _list621.size; ++_i623)
+                {
+                  _elem622 = new TxnToWriteId();
+                  _elem622.read(iprot);
+                  struct.srcTxnToWriteIdList.add(_elem622);
+                }
+                iprot.readListEnd();
+              }
+              struct.setSrcTxnToWriteIdListIsSet(true);
             } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
@@ -565,18 +771,6 @@ import org.slf4j.LoggerFactory;
       struct.validate();
 
       oprot.writeStructBegin(STRUCT_DESC);
-      if (struct.txnIds != null) {
-        oprot.writeFieldBegin(TXN_IDS_FIELD_DESC);
-        {
-          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.txnIds.size()));
-          for (long _iter621 : struct.txnIds)
-          {
-            oprot.writeI64(_iter621);
-          }
-          oprot.writeListEnd();
-        }
-        oprot.writeFieldEnd();
-      }
       if (struct.dbName != null) {
         oprot.writeFieldBegin(DB_NAME_FIELD_DESC);
         oprot.writeString(struct.dbName);
@@ -587,6 +781,41 @@ import org.slf4j.LoggerFactory;
         oprot.writeString(struct.tableName);
         oprot.writeFieldEnd();
       }
+      if (struct.txnIds != null) {
+        if (struct.isSetTxnIds()) {
+          oprot.writeFieldBegin(TXN_IDS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.txnIds.size()));
+            for (long _iter624 : struct.txnIds)
+            {
+              oprot.writeI64(_iter624);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.replPolicy != null) {
+        if (struct.isSetReplPolicy()) {
+          oprot.writeFieldBegin(REPL_POLICY_FIELD_DESC);
+          oprot.writeString(struct.replPolicy);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.srcTxnToWriteIdList != null) {
+        if (struct.isSetSrcTxnToWriteIdList()) {
+          oprot.writeFieldBegin(SRC_TXN_TO_WRITE_ID_LIST_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.srcTxnToWriteIdList.size()));
+            for (TxnToWriteId _iter625 : struct.srcTxnToWriteIdList)
+            {
+              _iter625.write(oprot);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -604,35 +833,81 @@ import org.slf4j.LoggerFactory;
     @Override
     public void write(org.apache.thrift.protocol.TProtocol prot, AllocateTableWriteIdsRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol oprot = (TTupleProtocol) prot;
-      {
-        oprot.writeI32(struct.txnIds.size());
-        for (long _iter622 : struct.txnIds)
+      oprot.writeString(struct.dbName);
+      oprot.writeString(struct.tableName);
+      BitSet optionals = new BitSet();
+      if (struct.isSetTxnIds()) {
+        optionals.set(0);
+      }
+      if (struct.isSetReplPolicy()) {
+        optionals.set(1);
+      }
+      if (struct.isSetSrcTxnToWriteIdList()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
+      if (struct.isSetTxnIds()) {
         {
-          oprot.writeI64(_iter622);
+          oprot.writeI32(struct.txnIds.size());
+          for (long _iter626 : struct.txnIds)
+          {
+            oprot.writeI64(_iter626);
+          }
+        }
+      }
+      if (struct.isSetReplPolicy()) {
+        oprot.writeString(struct.replPolicy);
+      }
+      if (struct.isSetSrcTxnToWriteIdList()) {
+        {
+          oprot.writeI32(struct.srcTxnToWriteIdList.size());
+          for (TxnToWriteId _iter627 : struct.srcTxnToWriteIdList)
+          {
+            _iter627.write(oprot);
+          }
         }
       }
-      oprot.writeString(struct.dbName);
-      oprot.writeString(struct.tableName);
     }
 
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, AllocateTableWriteIdsRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      {
-        org.apache.thrift.protocol.TList _list623 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.txnIds = new ArrayList<Long>(_list623.size);
-        long _elem624;
-        for (int _i625 = 0; _i625 < _list623.size; ++_i625)
-        {
-          _elem624 = iprot.readI64();
-          struct.txnIds.add(_elem624);
-        }
-      }
-      struct.setTxnIdsIsSet(true);
       struct.dbName = iprot.readString();
       struct.setDbNameIsSet(true);
       struct.tableName = iprot.readString();
       struct.setTableNameIsSet(true);
+      BitSet incoming = iprot.readBitSet(3);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TList _list628 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.txnIds = new ArrayList<Long>(_list628.size);
+          long _elem629;
+          for (int _i630 = 0; _i630 < _list628.size; ++_i630)
+          {
+            _elem629 = iprot.readI64();
+            struct.txnIds.add(_elem629);
+          }
+        }
+        struct.setTxnIdsIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.replPolicy = iprot.readString();
+        struct.setReplPolicyIsSet(true);
+      }
+      if (incoming.get(2)) {
+        {
+          org.apache.thrift.protocol.TList _list631 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.srcTxnToWriteIdList = new ArrayList<TxnToWriteId>(_list631.size);
+          TxnToWriteId _elem632;
+          for (int _i633 = 0; _i633 < _list631.size; ++_i633)
+          {
+            _elem632 = new TxnToWriteId();
+            _elem632.read(iprot);
+            struct.srcTxnToWriteIdList.add(_elem632);
+          }
+        }
+        struct.setSrcTxnToWriteIdListIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java
index 35cbca3..7bba38c 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // TXN_TO_WRITE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list626 = iprot.readListBegin();
-                struct.txnToWriteIds = new ArrayList<TxnToWriteId>(_list626.size);
-                TxnToWriteId _elem627;
-                for (int _i628 = 0; _i628 < _list626.size; ++_i628)
+                org.apache.thrift.protocol.TList _list634 = iprot.readListBegin();
+                struct.txnToWriteIds = new ArrayList<TxnToWriteId>(_list634.size);
+                TxnToWriteId _elem635;
+                for (int _i636 = 0; _i636 < _list634.size; ++_i636)
                 {
-                  _elem627 = new TxnToWriteId();
-                  _elem627.read(iprot);
-                  struct.txnToWriteIds.add(_elem627);
+                  _elem635 = new TxnToWriteId();
+                  _elem635.read(iprot);
+                  struct.txnToWriteIds.add(_elem635);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TXN_TO_WRITE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.txnToWriteIds.size()));
-          for (TxnToWriteId _iter629 : struct.txnToWriteIds)
+          for (TxnToWriteId _iter637 : struct.txnToWriteIds)
           {
-            _iter629.write(oprot);
+            _iter637.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.txnToWriteIds.size());
-        for (TxnToWriteId _iter630 : struct.txnToWriteIds)
+        for (TxnToWriteId _iter638 : struct.txnToWriteIds)
         {
-          _iter630.write(oprot);
+          _iter638.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, AllocateTableWriteIdsResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list631 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.txnToWriteIds = new ArrayList<TxnToWriteId>(_list631.size);
-        TxnToWriteId _elem632;
-        for (int _i633 = 0; _i633 < _list631.size; ++_i633)
+        org.apache.thrift.protocol.TList _list639 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.txnToWriteIds = new ArrayList<TxnToWriteId>(_list639.size);
+        TxnToWriteId _elem640;
+        for (int _i641 = 0; _i641 < _list639.size; ++_i641)
         {
-          _elem632 = new TxnToWriteId();
-          _elem632.read(iprot);
-          struct.txnToWriteIds.add(_elem632);
+          _elem640 = new TxnToWriteId();
+          _elem640.read(iprot);
+          struct.txnToWriteIds.add(_elem640);
         }
       }
       struct.setTxnToWriteIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 2162163..7db0801 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 _list784 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list784.size);
-                long _elem785;
-                for (int _i786 = 0; _i786 < _list784.size; ++_i786)
+                org.apache.thrift.protocol.TList _list792 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list792.size);
+                long _elem793;
+                for (int _i794 = 0; _i794 < _list792.size; ++_i794)
                 {
-                  _elem785 = iprot.readI64();
-                  struct.fileIds.add(_elem785);
+                  _elem793 = iprot.readI64();
+                  struct.fileIds.add(_elem793);
                 }
                 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 _iter787 : struct.fileIds)
+          for (long _iter795 : struct.fileIds)
           {
-            oprot.writeI64(_iter787);
+            oprot.writeI64(_iter795);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter788 : struct.fileIds)
+        for (long _iter796 : struct.fileIds)
         {
-          oprot.writeI64(_iter788);
+          oprot.writeI64(_iter796);
         }
       }
     }
@@ -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 _list789 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list789.size);
-        long _elem790;
-        for (int _i791 = 0; _i791 < _list789.size; ++_i791)
+        org.apache.thrift.protocol.TList _list797 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list797.size);
+        long _elem798;
+        for (int _i799 = 0; _i799 < _list797.size; ++_i799)
         {
-          _elem790 = iprot.readI64();
-          struct.fileIds.add(_elem790);
+          _elem798 = iprot.readI64();
+          struct.fileIds.add(_elem798);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 65e9e4c..a83c0bb 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 _list800 = iprot.readListBegin();
-                struct.values = new ArrayList<ClientCapability>(_list800.size);
-                ClientCapability _elem801;
-                for (int _i802 = 0; _i802 < _list800.size; ++_i802)
+                org.apache.thrift.protocol.TList _list808 = iprot.readListBegin();
+                struct.values = new ArrayList<ClientCapability>(_list808.size);
+                ClientCapability _elem809;
+                for (int _i810 = 0; _i810 < _list808.size; ++_i810)
                 {
-                  _elem801 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
-                  struct.values.add(_elem801);
+                  _elem809 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
+                  struct.values.add(_elem809);
                 }
                 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 _iter803 : struct.values)
+          for (ClientCapability _iter811 : struct.values)
           {
-            oprot.writeI32(_iter803.getValue());
+            oprot.writeI32(_iter811.getValue());
           }
           oprot.writeListEnd();
         }
@@ -413,9 +413,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.values.size());
-        for (ClientCapability _iter804 : struct.values)
+        for (ClientCapability _iter812 : struct.values)
         {
-          oprot.writeI32(_iter804.getValue());
+          oprot.writeI32(_iter812.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 _list805 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32());
-        struct.values = new ArrayList<ClientCapability>(_list805.size);
-        ClientCapability _elem806;
-        for (int _i807 = 0; _i807 < _list805.size; ++_i807)
+        org.apache.thrift.protocol.TList _list813 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32());
+        struct.values = new ArrayList<ClientCapability>(_list813.size);
+        ClientCapability _elem814;
+        for (int _i815 = 0; _i815 < _list813.size; ++_i815)
         {
-          _elem806 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
-          struct.values.add(_elem806);
+          _elem814 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
+          struct.values.add(_elem814);
         }
       }
       struct.setValuesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 e499e80..524a48e 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 _map666 = iprot.readMapBegin();
-                struct.properties = new HashMap<String,String>(2*_map666.size);
-                String _key667;
-                String _val668;
-                for (int _i669 = 0; _i669 < _map666.size; ++_i669)
+                org.apache.thrift.protocol.TMap _map674 = iprot.readMapBegin();
+                struct.properties = new HashMap<String,String>(2*_map674.size);
+                String _key675;
+                String _val676;
+                for (int _i677 = 0; _i677 < _map674.size; ++_i677)
                 {
-                  _key667 = iprot.readString();
-                  _val668 = iprot.readString();
-                  struct.properties.put(_key667, _val668);
+                  _key675 = iprot.readString();
+                  _val676 = iprot.readString();
+                  struct.properties.put(_key675, _val676);
                 }
                 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> _iter670 : struct.properties.entrySet())
+            for (Map.Entry<String, String> _iter678 : struct.properties.entrySet())
             {
-              oprot.writeString(_iter670.getKey());
-              oprot.writeString(_iter670.getValue());
+              oprot.writeString(_iter678.getKey());
+              oprot.writeString(_iter678.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -928,10 +928,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetProperties()) {
         {
           oprot.writeI32(struct.properties.size());
-          for (Map.Entry<String, String> _iter671 : struct.properties.entrySet())
+          for (Map.Entry<String, String> _iter679 : struct.properties.entrySet())
           {
-            oprot.writeString(_iter671.getKey());
-            oprot.writeString(_iter671.getValue());
+            oprot.writeString(_iter679.getKey());
+            oprot.writeString(_iter679.getValue());
           }
         }
       }
@@ -957,15 +957,15 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TMap _map672 = 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*_map672.size);
-          String _key673;
-          String _val674;
-          for (int _i675 = 0; _i675 < _map672.size; ++_i675)
+          org.apache.thrift.protocol.TMap _map680 = 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*_map680.size);
+          String _key681;
+          String _val682;
+          for (int _i683 = 0; _i683 < _map680.size; ++_i683)
           {
-            _key673 = iprot.readString();
-            _val674 = iprot.readString();
-            struct.properties.put(_key673, _val674);
+            _key681 = iprot.readString();
+            _val682 = iprot.readString();
+            struct.properties.put(_key681, _val682);
           }
         }
         struct.setPropertiesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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
index 1a16fac..8e4144e 100644
--- 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
@@ -712,13 +712,13 @@ import org.slf4j.LoggerFactory;
           case 4: // TABLES_USED
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set692 = iprot.readSetBegin();
-                struct.tablesUsed = new HashSet<String>(2*_set692.size);
-                String _elem693;
-                for (int _i694 = 0; _i694 < _set692.size; ++_i694)
+                org.apache.thrift.protocol.TSet _set700 = iprot.readSetBegin();
+                struct.tablesUsed = new HashSet<String>(2*_set700.size);
+                String _elem701;
+                for (int _i702 = 0; _i702 < _set700.size; ++_i702)
                 {
-                  _elem693 = iprot.readString();
-                  struct.tablesUsed.add(_elem693);
+                  _elem701 = iprot.readString();
+                  struct.tablesUsed.add(_elem701);
                 }
                 iprot.readSetEnd();
               }
@@ -767,9 +767,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 _iter695 : struct.tablesUsed)
+          for (String _iter703 : struct.tablesUsed)
           {
-            oprot.writeString(_iter695);
+            oprot.writeString(_iter703);
           }
           oprot.writeSetEnd();
         }
@@ -804,9 +804,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeString(struct.tblName);
       {
         oprot.writeI32(struct.tablesUsed.size());
-        for (String _iter696 : struct.tablesUsed)
+        for (String _iter704 : struct.tablesUsed)
         {
-          oprot.writeString(_iter696);
+          oprot.writeString(_iter704);
         }
       }
       BitSet optionals = new BitSet();
@@ -829,13 +829,13 @@ import org.slf4j.LoggerFactory;
       struct.tblName = iprot.readString();
       struct.setTblNameIsSet(true);
       {
-        org.apache.thrift.protocol.TSet _set697 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.tablesUsed = new HashSet<String>(2*_set697.size);
-        String _elem698;
-        for (int _i699 = 0; _i699 < _set697.size; ++_i699)
+        org.apache.thrift.protocol.TSet _set705 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.tablesUsed = new HashSet<String>(2*_set705.size);
+        String _elem706;
+        for (int _i707 = 0; _i707 < _set705.size; ++_i707)
         {
-          _elem698 = iprot.readString();
-          struct.tablesUsed.add(_elem698);
+          _elem706 = iprot.readString();
+          struct.tablesUsed.add(_elem706);
         }
       }
       struct.setTablesUsedIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java
index c06b95c..bb64086 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java
@@ -350,14 +350,14 @@ import org.slf4j.LoggerFactory;
           case 1: // SCHEMA_VERSIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list904 = iprot.readListBegin();
-                struct.schemaVersions = new ArrayList<SchemaVersionDescriptor>(_list904.size);
-                SchemaVersionDescriptor _elem905;
-                for (int _i906 = 0; _i906 < _list904.size; ++_i906)
+                org.apache.thrift.protocol.TList _list912 = iprot.readListBegin();
+                struct.schemaVersions = new ArrayList<SchemaVersionDescriptor>(_list912.size);
+                SchemaVersionDescriptor _elem913;
+                for (int _i914 = 0; _i914 < _list912.size; ++_i914)
                 {
-                  _elem905 = new SchemaVersionDescriptor();
-                  _elem905.read(iprot);
-                  struct.schemaVersions.add(_elem905);
+                  _elem913 = new SchemaVersionDescriptor();
+                  _elem913.read(iprot);
+                  struct.schemaVersions.add(_elem913);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(SCHEMA_VERSIONS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.schemaVersions.size()));
-          for (SchemaVersionDescriptor _iter907 : struct.schemaVersions)
+          for (SchemaVersionDescriptor _iter915 : struct.schemaVersions)
           {
-            _iter907.write(oprot);
+            _iter915.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -416,9 +416,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetSchemaVersions()) {
         {
           oprot.writeI32(struct.schemaVersions.size());
-          for (SchemaVersionDescriptor _iter908 : struct.schemaVersions)
+          for (SchemaVersionDescriptor _iter916 : struct.schemaVersions)
           {
-            _iter908.write(oprot);
+            _iter916.write(oprot);
           }
         }
       }
@@ -430,14 +430,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list909 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.schemaVersions = new ArrayList<SchemaVersionDescriptor>(_list909.size);
-          SchemaVersionDescriptor _elem910;
-          for (int _i911 = 0; _i911 < _list909.size; ++_i911)
+          org.apache.thrift.protocol.TList _list917 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.schemaVersions = new ArrayList<SchemaVersionDescriptor>(_list917.size);
+          SchemaVersionDescriptor _elem918;
+          for (int _i919 = 0; _i919 < _list917.size; ++_i919)
           {
-            _elem910 = new SchemaVersionDescriptor();
-            _elem910.read(iprot);
-            struct.schemaVersions.add(_elem910);
+            _elem918 = new SchemaVersionDescriptor();
+            _elem918.read(iprot);
+            struct.schemaVersions.add(_elem918);
           }
         }
         struct.setSchemaVersionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 14e6abe..42c9b53 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
@@ -794,13 +794,13 @@ import org.slf4j.LoggerFactory;
           case 5: // PARTITION_VALS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list724 = iprot.readListBegin();
-                struct.partitionVals = new ArrayList<String>(_list724.size);
-                String _elem725;
-                for (int _i726 = 0; _i726 < _list724.size; ++_i726)
+                org.apache.thrift.protocol.TList _list732 = iprot.readListBegin();
+                struct.partitionVals = new ArrayList<String>(_list732.size);
+                String _elem733;
+                for (int _i734 = 0; _i734 < _list732.size; ++_i734)
                 {
-                  _elem725 = iprot.readString();
-                  struct.partitionVals.add(_elem725);
+                  _elem733 = iprot.readString();
+                  struct.partitionVals.add(_elem733);
                 }
                 iprot.readListEnd();
               }
@@ -857,9 +857,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 _iter727 : struct.partitionVals)
+            for (String _iter735 : struct.partitionVals)
             {
-              oprot.writeString(_iter727);
+              oprot.writeString(_iter735);
             }
             oprot.writeListEnd();
           }
@@ -915,9 +915,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitionVals()) {
         {
           oprot.writeI32(struct.partitionVals.size());
-          for (String _iter728 : struct.partitionVals)
+          for (String _iter736 : struct.partitionVals)
           {
-            oprot.writeString(_iter728);
+            oprot.writeString(_iter736);
           }
         }
       }
@@ -945,13 +945,13 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list729 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.partitionVals = new ArrayList<String>(_list729.size);
-          String _elem730;
-          for (int _i731 = 0; _i731 < _list729.size; ++_i731)
+          org.apache.thrift.protocol.TList _list737 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.partitionVals = new ArrayList<String>(_list737.size);
+          String _elem738;
+          for (int _i739 = 0; _i739 < _list737.size; ++_i739)
           {
-            _elem730 = iprot.readString();
-            struct.partitionVals.add(_elem730);
+            _elem738 = iprot.readString();
+            struct.partitionVals.add(_elem738);
           }
         }
         struct.setPartitionValsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 dfdc357..32e6543 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 _list792 = iprot.readListBegin();
-                struct.functions = new ArrayList<Function>(_list792.size);
-                Function _elem793;
-                for (int _i794 = 0; _i794 < _list792.size; ++_i794)
+                org.apache.thrift.protocol.TList _list800 = iprot.readListBegin();
+                struct.functions = new ArrayList<Function>(_list800.size);
+                Function _elem801;
+                for (int _i802 = 0; _i802 < _list800.size; ++_i802)
                 {
-                  _elem793 = new Function();
-                  _elem793.read(iprot);
-                  struct.functions.add(_elem793);
+                  _elem801 = new Function();
+                  _elem801.read(iprot);
+                  struct.functions.add(_elem801);
                 }
                 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 _iter795 : struct.functions)
+            for (Function _iter803 : struct.functions)
             {
-              _iter795.write(oprot);
+              _iter803.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetFunctions()) {
         {
           oprot.writeI32(struct.functions.size());
-          for (Function _iter796 : struct.functions)
+          for (Function _iter804 : struct.functions)
           {
-            _iter796.write(oprot);
+            _iter804.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list797 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.functions = new ArrayList<Function>(_list797.size);
-          Function _elem798;
-          for (int _i799 = 0; _i799 < _list797.size; ++_i799)
+          org.apache.thrift.protocol.TList _list805 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.functions = new ArrayList<Function>(_list805.size);
+          Function _elem806;
+          for (int _i807 = 0; _i807 < _list805.size; ++_i807)
           {
-            _elem798 = new Function();
-            _elem798.read(iprot);
-            struct.functions.add(_elem798);
+            _elem806 = new Function();
+            _elem806.read(iprot);
+            struct.functions.add(_elem806);
           }
         }
         struct.setFunctionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 0adb11d..8dab985 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 _list742 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list742.size);
-                long _elem743;
-                for (int _i744 = 0; _i744 < _list742.size; ++_i744)
+                org.apache.thrift.protocol.TList _list750 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list750.size);
+                long _elem751;
+                for (int _i752 = 0; _i752 < _list750.size; ++_i752)
                 {
-                  _elem743 = iprot.readI64();
-                  struct.fileIds.add(_elem743);
+                  _elem751 = iprot.readI64();
+                  struct.fileIds.add(_elem751);
                 }
                 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 _iter745 : struct.fileIds)
+          for (long _iter753 : struct.fileIds)
           {
-            oprot.writeI64(_iter745);
+            oprot.writeI64(_iter753);
           }
           oprot.writeListEnd();
         }
@@ -719,9 +719,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter746 : struct.fileIds)
+        for (long _iter754 : struct.fileIds)
         {
-          oprot.writeI64(_iter746);
+          oprot.writeI64(_iter754);
         }
       }
       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 _list747 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list747.size);
-        long _elem748;
-        for (int _i749 = 0; _i749 < _list747.size; ++_i749)
+        org.apache.thrift.protocol.TList _list755 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list755.size);
+        long _elem756;
+        for (int _i757 = 0; _i757 < _list755.size; ++_i757)
         {
-          _elem748 = iprot.readI64();
-          struct.fileIds.add(_elem748);
+          _elem756 = iprot.readI64();
+          struct.fileIds.add(_elem756);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 f86d9ea..d94ea73 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 _map732 = iprot.readMapBegin();
-                struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map732.size);
-                long _key733;
-                MetadataPpdResult _val734;
-                for (int _i735 = 0; _i735 < _map732.size; ++_i735)
+                org.apache.thrift.protocol.TMap _map740 = iprot.readMapBegin();
+                struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map740.size);
+                long _key741;
+                MetadataPpdResult _val742;
+                for (int _i743 = 0; _i743 < _map740.size; ++_i743)
                 {
-                  _key733 = iprot.readI64();
-                  _val734 = new MetadataPpdResult();
-                  _val734.read(iprot);
-                  struct.metadata.put(_key733, _val734);
+                  _key741 = iprot.readI64();
+                  _val742 = new MetadataPpdResult();
+                  _val742.read(iprot);
+                  struct.metadata.put(_key741, _val742);
                 }
                 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> _iter736 : struct.metadata.entrySet())
+          for (Map.Entry<Long, MetadataPpdResult> _iter744 : struct.metadata.entrySet())
           {
-            oprot.writeI64(_iter736.getKey());
-            _iter736.getValue().write(oprot);
+            oprot.writeI64(_iter744.getKey());
+            _iter744.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> _iter737 : struct.metadata.entrySet())
+        for (Map.Entry<Long, MetadataPpdResult> _iter745 : struct.metadata.entrySet())
         {
-          oprot.writeI64(_iter737.getKey());
-          _iter737.getValue().write(oprot);
+          oprot.writeI64(_iter745.getKey());
+          _iter745.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 _map738 = 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*_map738.size);
-        long _key739;
-        MetadataPpdResult _val740;
-        for (int _i741 = 0; _i741 < _map738.size; ++_i741)
+        org.apache.thrift.protocol.TMap _map746 = 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*_map746.size);
+        long _key747;
+        MetadataPpdResult _val748;
+        for (int _i749 = 0; _i749 < _map746.size; ++_i749)
         {
-          _key739 = iprot.readI64();
-          _val740 = new MetadataPpdResult();
-          _val740.read(iprot);
-          struct.metadata.put(_key739, _val740);
+          _key747 = iprot.readI64();
+          _val748 = new MetadataPpdResult();
+          _val748.read(iprot);
+          struct.metadata.put(_key747, _val748);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 b98375c..9dfc484 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 _list760 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list760.size);
-                long _elem761;
-                for (int _i762 = 0; _i762 < _list760.size; ++_i762)
+                org.apache.thrift.protocol.TList _list768 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list768.size);
+                long _elem769;
+                for (int _i770 = 0; _i770 < _list768.size; ++_i770)
                 {
-                  _elem761 = iprot.readI64();
-                  struct.fileIds.add(_elem761);
+                  _elem769 = iprot.readI64();
+                  struct.fileIds.add(_elem769);
                 }
                 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 _iter763 : struct.fileIds)
+          for (long _iter771 : struct.fileIds)
           {
-            oprot.writeI64(_iter763);
+            oprot.writeI64(_iter771);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter764 : struct.fileIds)
+        for (long _iter772 : struct.fileIds)
         {
-          oprot.writeI64(_iter764);
+          oprot.writeI64(_iter772);
         }
       }
     }
@@ -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 _list765 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list765.size);
-        long _elem766;
-        for (int _i767 = 0; _i767 < _list765.size; ++_i767)
+        org.apache.thrift.protocol.TList _list773 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list773.size);
+        long _elem774;
+        for (int _i775 = 0; _i775 < _list773.size; ++_i775)
         {
-          _elem766 = iprot.readI64();
-          struct.fileIds.add(_elem766);
+          _elem774 = iprot.readI64();
+          struct.fileIds.add(_elem774);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 d8d903e..d454340 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 _map750 = iprot.readMapBegin();
-                struct.metadata = new HashMap<Long,ByteBuffer>(2*_map750.size);
-                long _key751;
-                ByteBuffer _val752;
-                for (int _i753 = 0; _i753 < _map750.size; ++_i753)
+                org.apache.thrift.protocol.TMap _map758 = iprot.readMapBegin();
+                struct.metadata = new HashMap<Long,ByteBuffer>(2*_map758.size);
+                long _key759;
+                ByteBuffer _val760;
+                for (int _i761 = 0; _i761 < _map758.size; ++_i761)
                 {
-                  _key751 = iprot.readI64();
-                  _val752 = iprot.readBinary();
-                  struct.metadata.put(_key751, _val752);
+                  _key759 = iprot.readI64();
+                  _val760 = iprot.readBinary();
+                  struct.metadata.put(_key759, _val760);
                 }
                 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> _iter754 : struct.metadata.entrySet())
+          for (Map.Entry<Long, ByteBuffer> _iter762 : struct.metadata.entrySet())
           {
-            oprot.writeI64(_iter754.getKey());
-            oprot.writeBinary(_iter754.getValue());
+            oprot.writeI64(_iter762.getKey());
+            oprot.writeBinary(_iter762.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> _iter755 : struct.metadata.entrySet())
+        for (Map.Entry<Long, ByteBuffer> _iter763 : struct.metadata.entrySet())
         {
-          oprot.writeI64(_iter755.getKey());
-          oprot.writeBinary(_iter755.getValue());
+          oprot.writeI64(_iter763.getKey());
+          oprot.writeBinary(_iter763.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 _map756 = 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*_map756.size);
-        long _key757;
-        ByteBuffer _val758;
-        for (int _i759 = 0; _i759 < _map756.size; ++_i759)
+        org.apache.thrift.protocol.TMap _map764 = 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*_map764.size);
+        long _key765;
+        ByteBuffer _val766;
+        for (int _i767 = 0; _i767 < _map764.size; ++_i767)
         {
-          _key757 = iprot.readI64();
-          _val758 = iprot.readBinary();
-          struct.metadata.put(_key757, _val758);
+          _key765 = iprot.readI64();
+          _val766 = iprot.readBinary();
+          struct.metadata.put(_key765, _val766);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 a264cdd..f2be7ec 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
@@ -606,13 +606,13 @@ import org.slf4j.LoggerFactory;
           case 2: // TBL_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list808 = iprot.readListBegin();
-                struct.tblNames = new ArrayList<String>(_list808.size);
-                String _elem809;
-                for (int _i810 = 0; _i810 < _list808.size; ++_i810)
+                org.apache.thrift.protocol.TList _list816 = iprot.readListBegin();
+                struct.tblNames = new ArrayList<String>(_list816.size);
+                String _elem817;
+                for (int _i818 = 0; _i818 < _list816.size; ++_i818)
                 {
-                  _elem809 = iprot.readString();
-                  struct.tblNames.add(_elem809);
+                  _elem817 = iprot.readString();
+                  struct.tblNames.add(_elem817);
                 }
                 iprot.readListEnd();
               }
@@ -661,9 +661,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 _iter811 : struct.tblNames)
+            for (String _iter819 : struct.tblNames)
             {
-              oprot.writeString(_iter811);
+              oprot.writeString(_iter819);
             }
             oprot.writeListEnd();
           }
@@ -716,9 +716,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetTblNames()) {
         {
           oprot.writeI32(struct.tblNames.size());
-          for (String _iter812 : struct.tblNames)
+          for (String _iter820 : struct.tblNames)
           {
-            oprot.writeString(_iter812);
+            oprot.writeString(_iter820);
           }
         }
       }
@@ -738,13 +738,13 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list813 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.tblNames = new ArrayList<String>(_list813.size);
-          String _elem814;
-          for (int _i815 = 0; _i815 < _list813.size; ++_i815)
+          org.apache.thrift.protocol.TList _list821 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.tblNames = new ArrayList<String>(_list821.size);
+          String _elem822;
+          for (int _i823 = 0; _i823 < _list821.size; ++_i823)
           {
-            _elem814 = iprot.readString();
-            struct.tblNames.add(_elem814);
+            _elem822 = iprot.readString();
+            struct.tblNames.add(_elem822);
           }
         }
         struct.setTblNamesIsSet(true);


[06/12] hive git commit: HIVE-19089: Create/Replicate Allocate write-id event (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 ad2c0b6..f4e30f0 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 _list832 = iprot.readListBegin();
-                struct.pools = new ArrayList<WMPool>(_list832.size);
-                WMPool _elem833;
-                for (int _i834 = 0; _i834 < _list832.size; ++_i834)
+                org.apache.thrift.protocol.TList _list840 = iprot.readListBegin();
+                struct.pools = new ArrayList<WMPool>(_list840.size);
+                WMPool _elem841;
+                for (int _i842 = 0; _i842 < _list840.size; ++_i842)
                 {
-                  _elem833 = new WMPool();
-                  _elem833.read(iprot);
-                  struct.pools.add(_elem833);
+                  _elem841 = new WMPool();
+                  _elem841.read(iprot);
+                  struct.pools.add(_elem841);
                 }
                 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 _list835 = iprot.readListBegin();
-                struct.mappings = new ArrayList<WMMapping>(_list835.size);
-                WMMapping _elem836;
-                for (int _i837 = 0; _i837 < _list835.size; ++_i837)
+                org.apache.thrift.protocol.TList _list843 = iprot.readListBegin();
+                struct.mappings = new ArrayList<WMMapping>(_list843.size);
+                WMMapping _elem844;
+                for (int _i845 = 0; _i845 < _list843.size; ++_i845)
                 {
-                  _elem836 = new WMMapping();
-                  _elem836.read(iprot);
-                  struct.mappings.add(_elem836);
+                  _elem844 = new WMMapping();
+                  _elem844.read(iprot);
+                  struct.mappings.add(_elem844);
                 }
                 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 _list838 = iprot.readListBegin();
-                struct.triggers = new ArrayList<WMTrigger>(_list838.size);
-                WMTrigger _elem839;
-                for (int _i840 = 0; _i840 < _list838.size; ++_i840)
+                org.apache.thrift.protocol.TList _list846 = iprot.readListBegin();
+                struct.triggers = new ArrayList<WMTrigger>(_list846.size);
+                WMTrigger _elem847;
+                for (int _i848 = 0; _i848 < _list846.size; ++_i848)
                 {
-                  _elem839 = new WMTrigger();
-                  _elem839.read(iprot);
-                  struct.triggers.add(_elem839);
+                  _elem847 = new WMTrigger();
+                  _elem847.read(iprot);
+                  struct.triggers.add(_elem847);
                 }
                 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 _list841 = iprot.readListBegin();
-                struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list841.size);
-                WMPoolTrigger _elem842;
-                for (int _i843 = 0; _i843 < _list841.size; ++_i843)
+                org.apache.thrift.protocol.TList _list849 = iprot.readListBegin();
+                struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list849.size);
+                WMPoolTrigger _elem850;
+                for (int _i851 = 0; _i851 < _list849.size; ++_i851)
                 {
-                  _elem842 = new WMPoolTrigger();
-                  _elem842.read(iprot);
-                  struct.poolTriggers.add(_elem842);
+                  _elem850 = new WMPoolTrigger();
+                  _elem850.read(iprot);
+                  struct.poolTriggers.add(_elem850);
                 }
                 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 _iter844 : struct.pools)
+          for (WMPool _iter852 : struct.pools)
           {
-            _iter844.write(oprot);
+            _iter852.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 _iter845 : struct.mappings)
+            for (WMMapping _iter853 : struct.mappings)
             {
-              _iter845.write(oprot);
+              _iter853.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 _iter846 : struct.triggers)
+            for (WMTrigger _iter854 : struct.triggers)
             {
-              _iter846.write(oprot);
+              _iter854.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 _iter847 : struct.poolTriggers)
+            for (WMPoolTrigger _iter855 : struct.poolTriggers)
             {
-              _iter847.write(oprot);
+              _iter855.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -920,9 +920,9 @@ import org.slf4j.LoggerFactory;
       struct.plan.write(oprot);
       {
         oprot.writeI32(struct.pools.size());
-        for (WMPool _iter848 : struct.pools)
+        for (WMPool _iter856 : struct.pools)
         {
-          _iter848.write(oprot);
+          _iter856.write(oprot);
         }
       }
       BitSet optionals = new BitSet();
@@ -939,27 +939,27 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetMappings()) {
         {
           oprot.writeI32(struct.mappings.size());
-          for (WMMapping _iter849 : struct.mappings)
+          for (WMMapping _iter857 : struct.mappings)
           {
-            _iter849.write(oprot);
+            _iter857.write(oprot);
           }
         }
       }
       if (struct.isSetTriggers()) {
         {
           oprot.writeI32(struct.triggers.size());
-          for (WMTrigger _iter850 : struct.triggers)
+          for (WMTrigger _iter858 : struct.triggers)
           {
-            _iter850.write(oprot);
+            _iter858.write(oprot);
           }
         }
       }
       if (struct.isSetPoolTriggers()) {
         {
           oprot.writeI32(struct.poolTriggers.size());
-          for (WMPoolTrigger _iter851 : struct.poolTriggers)
+          for (WMPoolTrigger _iter859 : struct.poolTriggers)
           {
-            _iter851.write(oprot);
+            _iter859.write(oprot);
           }
         }
       }
@@ -972,56 +972,56 @@ import org.slf4j.LoggerFactory;
       struct.plan.read(iprot);
       struct.setPlanIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list852 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.pools = new ArrayList<WMPool>(_list852.size);
-        WMPool _elem853;
-        for (int _i854 = 0; _i854 < _list852.size; ++_i854)
+        org.apache.thrift.protocol.TList _list860 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.pools = new ArrayList<WMPool>(_list860.size);
+        WMPool _elem861;
+        for (int _i862 = 0; _i862 < _list860.size; ++_i862)
         {
-          _elem853 = new WMPool();
-          _elem853.read(iprot);
-          struct.pools.add(_elem853);
+          _elem861 = new WMPool();
+          _elem861.read(iprot);
+          struct.pools.add(_elem861);
         }
       }
       struct.setPoolsIsSet(true);
       BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list855 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.mappings = new ArrayList<WMMapping>(_list855.size);
-          WMMapping _elem856;
-          for (int _i857 = 0; _i857 < _list855.size; ++_i857)
+          org.apache.thrift.protocol.TList _list863 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.mappings = new ArrayList<WMMapping>(_list863.size);
+          WMMapping _elem864;
+          for (int _i865 = 0; _i865 < _list863.size; ++_i865)
           {
-            _elem856 = new WMMapping();
-            _elem856.read(iprot);
-            struct.mappings.add(_elem856);
+            _elem864 = new WMMapping();
+            _elem864.read(iprot);
+            struct.mappings.add(_elem864);
           }
         }
         struct.setMappingsIsSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list858 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.triggers = new ArrayList<WMTrigger>(_list858.size);
-          WMTrigger _elem859;
-          for (int _i860 = 0; _i860 < _list858.size; ++_i860)
+          org.apache.thrift.protocol.TList _list866 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.triggers = new ArrayList<WMTrigger>(_list866.size);
+          WMTrigger _elem867;
+          for (int _i868 = 0; _i868 < _list866.size; ++_i868)
           {
-            _elem859 = new WMTrigger();
-            _elem859.read(iprot);
-            struct.triggers.add(_elem859);
+            _elem867 = new WMTrigger();
+            _elem867.read(iprot);
+            struct.triggers.add(_elem867);
           }
         }
         struct.setTriggersIsSet(true);
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list861 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list861.size);
-          WMPoolTrigger _elem862;
-          for (int _i863 = 0; _i863 < _list861.size; ++_i863)
+          org.apache.thrift.protocol.TList _list869 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list869.size);
+          WMPoolTrigger _elem870;
+          for (int _i871 = 0; _i871 < _list869.size; ++_i871)
           {
-            _elem862 = new WMPoolTrigger();
-            _elem862.read(iprot);
-            struct.poolTriggers.add(_elem862);
+            _elem870 = new WMPoolTrigger();
+            _elem870.read(iprot);
+            struct.poolTriggers.add(_elem870);
           }
         }
         struct.setPoolTriggersIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 cf50206..ba81ce9 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 _list864 = iprot.readListBegin();
-                struct.resourcePlans = new ArrayList<WMResourcePlan>(_list864.size);
-                WMResourcePlan _elem865;
-                for (int _i866 = 0; _i866 < _list864.size; ++_i866)
+                org.apache.thrift.protocol.TList _list872 = iprot.readListBegin();
+                struct.resourcePlans = new ArrayList<WMResourcePlan>(_list872.size);
+                WMResourcePlan _elem873;
+                for (int _i874 = 0; _i874 < _list872.size; ++_i874)
                 {
-                  _elem865 = new WMResourcePlan();
-                  _elem865.read(iprot);
-                  struct.resourcePlans.add(_elem865);
+                  _elem873 = new WMResourcePlan();
+                  _elem873.read(iprot);
+                  struct.resourcePlans.add(_elem873);
                 }
                 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 _iter867 : struct.resourcePlans)
+            for (WMResourcePlan _iter875 : struct.resourcePlans)
             {
-              _iter867.write(oprot);
+              _iter875.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetResourcePlans()) {
         {
           oprot.writeI32(struct.resourcePlans.size());
-          for (WMResourcePlan _iter868 : struct.resourcePlans)
+          for (WMResourcePlan _iter876 : struct.resourcePlans)
           {
-            _iter868.write(oprot);
+            _iter876.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list869 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.resourcePlans = new ArrayList<WMResourcePlan>(_list869.size);
-          WMResourcePlan _elem870;
-          for (int _i871 = 0; _i871 < _list869.size; ++_i871)
+          org.apache.thrift.protocol.TList _list877 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.resourcePlans = new ArrayList<WMResourcePlan>(_list877.size);
+          WMResourcePlan _elem878;
+          for (int _i879 = 0; _i879 < _list877.size; ++_i879)
           {
-            _elem870 = new WMResourcePlan();
-            _elem870.read(iprot);
-            struct.resourcePlans.add(_elem870);
+            _elem878 = new WMResourcePlan();
+            _elem878.read(iprot);
+            struct.resourcePlans.add(_elem878);
           }
         }
         struct.setResourcePlansIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 b23a7f8..10ed67c 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 _list888 = iprot.readListBegin();
-                struct.triggers = new ArrayList<WMTrigger>(_list888.size);
-                WMTrigger _elem889;
-                for (int _i890 = 0; _i890 < _list888.size; ++_i890)
+                org.apache.thrift.protocol.TList _list896 = iprot.readListBegin();
+                struct.triggers = new ArrayList<WMTrigger>(_list896.size);
+                WMTrigger _elem897;
+                for (int _i898 = 0; _i898 < _list896.size; ++_i898)
                 {
-                  _elem889 = new WMTrigger();
-                  _elem889.read(iprot);
-                  struct.triggers.add(_elem889);
+                  _elem897 = new WMTrigger();
+                  _elem897.read(iprot);
+                  struct.triggers.add(_elem897);
                 }
                 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 _iter891 : struct.triggers)
+            for (WMTrigger _iter899 : struct.triggers)
             {
-              _iter891.write(oprot);
+              _iter899.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetTriggers()) {
         {
           oprot.writeI32(struct.triggers.size());
-          for (WMTrigger _iter892 : struct.triggers)
+          for (WMTrigger _iter900 : struct.triggers)
           {
-            _iter892.write(oprot);
+            _iter900.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list893 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.triggers = new ArrayList<WMTrigger>(_list893.size);
-          WMTrigger _elem894;
-          for (int _i895 = 0; _i895 < _list893.size; ++_i895)
+          org.apache.thrift.protocol.TList _list901 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.triggers = new ArrayList<WMTrigger>(_list901.size);
+          WMTrigger _elem902;
+          for (int _i903 = 0; _i903 < _list901.size; ++_i903)
           {
-            _elem894 = new WMTrigger();
-            _elem894.read(iprot);
-            struct.triggers.add(_elem894);
+            _elem902 = new WMTrigger();
+            _elem902.read(iprot);
+            struct.triggers.add(_elem902);
           }
         }
         struct.setTriggersIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 53a0443..86d7d5c 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 _list872 = iprot.readListBegin();
-                struct.errors = new ArrayList<String>(_list872.size);
-                String _elem873;
-                for (int _i874 = 0; _i874 < _list872.size; ++_i874)
+                org.apache.thrift.protocol.TList _list880 = iprot.readListBegin();
+                struct.errors = new ArrayList<String>(_list880.size);
+                String _elem881;
+                for (int _i882 = 0; _i882 < _list880.size; ++_i882)
                 {
-                  _elem873 = iprot.readString();
-                  struct.errors.add(_elem873);
+                  _elem881 = iprot.readString();
+                  struct.errors.add(_elem881);
                 }
                 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 _list875 = iprot.readListBegin();
-                struct.warnings = new ArrayList<String>(_list875.size);
-                String _elem876;
-                for (int _i877 = 0; _i877 < _list875.size; ++_i877)
+                org.apache.thrift.protocol.TList _list883 = iprot.readListBegin();
+                struct.warnings = new ArrayList<String>(_list883.size);
+                String _elem884;
+                for (int _i885 = 0; _i885 < _list883.size; ++_i885)
                 {
-                  _elem876 = iprot.readString();
-                  struct.warnings.add(_elem876);
+                  _elem884 = iprot.readString();
+                  struct.warnings.add(_elem884);
                 }
                 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 _iter878 : struct.errors)
+            for (String _iter886 : struct.errors)
             {
-              oprot.writeString(_iter878);
+              oprot.writeString(_iter886);
             }
             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 _iter879 : struct.warnings)
+            for (String _iter887 : struct.warnings)
             {
-              oprot.writeString(_iter879);
+              oprot.writeString(_iter887);
             }
             oprot.writeListEnd();
           }
@@ -543,18 +543,18 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetErrors()) {
         {
           oprot.writeI32(struct.errors.size());
-          for (String _iter880 : struct.errors)
+          for (String _iter888 : struct.errors)
           {
-            oprot.writeString(_iter880);
+            oprot.writeString(_iter888);
           }
         }
       }
       if (struct.isSetWarnings()) {
         {
           oprot.writeI32(struct.warnings.size());
-          for (String _iter881 : struct.warnings)
+          for (String _iter889 : struct.warnings)
           {
-            oprot.writeString(_iter881);
+            oprot.writeString(_iter889);
           }
         }
       }
@@ -566,26 +566,26 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list882 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.errors = new ArrayList<String>(_list882.size);
-          String _elem883;
-          for (int _i884 = 0; _i884 < _list882.size; ++_i884)
+          org.apache.thrift.protocol.TList _list890 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.errors = new ArrayList<String>(_list890.size);
+          String _elem891;
+          for (int _i892 = 0; _i892 < _list890.size; ++_i892)
           {
-            _elem883 = iprot.readString();
-            struct.errors.add(_elem883);
+            _elem891 = iprot.readString();
+            struct.errors.add(_elem891);
           }
         }
         struct.setErrorsIsSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list885 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.warnings = new ArrayList<String>(_list885.size);
-          String _elem886;
-          for (int _i887 = 0; _i887 < _list885.size; ++_i887)
+          org.apache.thrift.protocol.TList _list893 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.warnings = new ArrayList<String>(_list893.size);
+          String _elem894;
+          for (int _i895 = 0; _i895 < _list893.size; ++_i895)
           {
-            _elem886 = iprot.readString();
-            struct.warnings.add(_elem886);
+            _elem894 = iprot.readString();
+            struct.warnings.add(_elem894);
           }
         }
         struct.setWarningsIsSet(true);


[04/12] hive git commit: HIVE-19089: Create/Replicate Allocate write-id event (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 14416b4..c4969d5 100644
--- a/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -16920,22 +16920,38 @@ class AllocateTableWriteIdsRequest {
   static $_TSPEC;
 
   /**
+   * @var string
+   */
+  public $dbName = null;
+  /**
+   * @var string
+   */
+  public $tableName = null;
+  /**
    * @var int[]
    */
   public $txnIds = null;
   /**
    * @var string
    */
-  public $dbName = null;
+  public $replPolicy = null;
   /**
-   * @var string
+   * @var \metastore\TxnToWriteId[]
    */
-  public $tableName = null;
+  public $srcTxnToWriteIdList = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
+          'var' => 'dbName',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'tableName',
+          'type' => TType::STRING,
+          ),
+        3 => array(
           'var' => 'txnIds',
           'type' => TType::LST,
           'etype' => TType::I64,
@@ -16943,26 +16959,37 @@ class AllocateTableWriteIdsRequest {
             'type' => TType::I64,
             ),
           ),
-        2 => array(
-          'var' => 'dbName',
+        4 => array(
+          'var' => 'replPolicy',
           'type' => TType::STRING,
           ),
-        3 => array(
-          'var' => 'tableName',
-          'type' => TType::STRING,
+        5 => array(
+          'var' => 'srcTxnToWriteIdList',
+          'type' => TType::LST,
+          'etype' => TType::STRUCT,
+          'elem' => array(
+            'type' => TType::STRUCT,
+            'class' => '\metastore\TxnToWriteId',
+            ),
           ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['txnIds'])) {
-        $this->txnIds = $vals['txnIds'];
-      }
       if (isset($vals['dbName'])) {
         $this->dbName = $vals['dbName'];
       }
       if (isset($vals['tableName'])) {
         $this->tableName = $vals['tableName'];
       }
+      if (isset($vals['txnIds'])) {
+        $this->txnIds = $vals['txnIds'];
+      }
+      if (isset($vals['replPolicy'])) {
+        $this->replPolicy = $vals['replPolicy'];
+      }
+      if (isset($vals['srcTxnToWriteIdList'])) {
+        $this->srcTxnToWriteIdList = $vals['srcTxnToWriteIdList'];
+      }
     }
   }
 
@@ -16986,6 +17013,20 @@ class AllocateTableWriteIdsRequest {
       switch ($fid)
       {
         case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->dbName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->tableName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
           if ($ftype == TType::LST) {
             $this->txnIds = array();
             $_size544 = 0;
@@ -17002,16 +17043,27 @@ class AllocateTableWriteIdsRequest {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 2:
+        case 4:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->dbName);
+            $xfer += $input->readString($this->replPolicy);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 3:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->tableName);
+        case 5:
+          if ($ftype == TType::LST) {
+            $this->srcTxnToWriteIdList = array();
+            $_size550 = 0;
+            $_etype553 = 0;
+            $xfer += $input->readListBegin($_etype553, $_size550);
+            for ($_i554 = 0; $_i554 < $_size550; ++$_i554)
+            {
+              $elem555 = null;
+              $elem555 = new \metastore\TxnToWriteId();
+              $xfer += $elem555->read($input);
+              $this->srcTxnToWriteIdList []= $elem555;
+            }
+            $xfer += $input->readListEnd();
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -17029,31 +17081,53 @@ class AllocateTableWriteIdsRequest {
   public function write($output) {
     $xfer = 0;
     $xfer += $output->writeStructBegin('AllocateTableWriteIdsRequest');
+    if ($this->dbName !== null) {
+      $xfer += $output->writeFieldBegin('dbName', TType::STRING, 1);
+      $xfer += $output->writeString($this->dbName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->tableName !== null) {
+      $xfer += $output->writeFieldBegin('tableName', TType::STRING, 2);
+      $xfer += $output->writeString($this->tableName);
+      $xfer += $output->writeFieldEnd();
+    }
     if ($this->txnIds !== null) {
       if (!is_array($this->txnIds)) {
         throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
       }
-      $xfer += $output->writeFieldBegin('txnIds', TType::LST, 1);
+      $xfer += $output->writeFieldBegin('txnIds', TType::LST, 3);
       {
         $output->writeListBegin(TType::I64, count($this->txnIds));
         {
-          foreach ($this->txnIds as $iter550)
+          foreach ($this->txnIds as $iter556)
           {
-            $xfer += $output->writeI64($iter550);
+            $xfer += $output->writeI64($iter556);
           }
         }
         $output->writeListEnd();
       }
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->dbName !== null) {
-      $xfer += $output->writeFieldBegin('dbName', TType::STRING, 2);
-      $xfer += $output->writeString($this->dbName);
+    if ($this->replPolicy !== null) {
+      $xfer += $output->writeFieldBegin('replPolicy', TType::STRING, 4);
+      $xfer += $output->writeString($this->replPolicy);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->tableName !== null) {
-      $xfer += $output->writeFieldBegin('tableName', TType::STRING, 3);
-      $xfer += $output->writeString($this->tableName);
+    if ($this->srcTxnToWriteIdList !== null) {
+      if (!is_array($this->srcTxnToWriteIdList)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('srcTxnToWriteIdList', TType::LST, 5);
+      {
+        $output->writeListBegin(TType::STRUCT, count($this->srcTxnToWriteIdList));
+        {
+          foreach ($this->srcTxnToWriteIdList as $iter557)
+          {
+            $xfer += $iter557->write($output);
+          }
+        }
+        $output->writeListEnd();
+      }
       $xfer += $output->writeFieldEnd();
     }
     $xfer += $output->writeFieldStop();
@@ -17212,15 +17286,15 @@ class AllocateTableWriteIdsResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->txnToWriteIds = array();
-            $_size551 = 0;
-            $_etype554 = 0;
-            $xfer += $input->readListBegin($_etype554, $_size551);
-            for ($_i555 = 0; $_i555 < $_size551; ++$_i555)
+            $_size558 = 0;
+            $_etype561 = 0;
+            $xfer += $input->readListBegin($_etype561, $_size558);
+            for ($_i562 = 0; $_i562 < $_size558; ++$_i562)
             {
-              $elem556 = null;
-              $elem556 = new \metastore\TxnToWriteId();
-              $xfer += $elem556->read($input);
-              $this->txnToWriteIds []= $elem556;
+              $elem563 = null;
+              $elem563 = new \metastore\TxnToWriteId();
+              $xfer += $elem563->read($input);
+              $this->txnToWriteIds []= $elem563;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17248,9 +17322,9 @@ class AllocateTableWriteIdsResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->txnToWriteIds));
         {
-          foreach ($this->txnToWriteIds as $iter557)
+          foreach ($this->txnToWriteIds as $iter564)
           {
-            $xfer += $iter557->write($output);
+            $xfer += $iter564->write($output);
           }
         }
         $output->writeListEnd();
@@ -17595,15 +17669,15 @@ class LockRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->component = array();
-            $_size558 = 0;
-            $_etype561 = 0;
-            $xfer += $input->readListBegin($_etype561, $_size558);
-            for ($_i562 = 0; $_i562 < $_size558; ++$_i562)
+            $_size565 = 0;
+            $_etype568 = 0;
+            $xfer += $input->readListBegin($_etype568, $_size565);
+            for ($_i569 = 0; $_i569 < $_size565; ++$_i569)
             {
-              $elem563 = null;
-              $elem563 = new \metastore\LockComponent();
-              $xfer += $elem563->read($input);
-              $this->component []= $elem563;
+              $elem570 = null;
+              $elem570 = new \metastore\LockComponent();
+              $xfer += $elem570->read($input);
+              $this->component []= $elem570;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17659,9 +17733,9 @@ class LockRequest {
       {
         $output->writeListBegin(TType::STRUCT, count($this->component));
         {
-          foreach ($this->component as $iter564)
+          foreach ($this->component as $iter571)
           {
-            $xfer += $iter564->write($output);
+            $xfer += $iter571->write($output);
           }
         }
         $output->writeListEnd();
@@ -18604,15 +18678,15 @@ class ShowLocksResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->locks = array();
-            $_size565 = 0;
-            $_etype568 = 0;
-            $xfer += $input->readListBegin($_etype568, $_size565);
-            for ($_i569 = 0; $_i569 < $_size565; ++$_i569)
+            $_size572 = 0;
+            $_etype575 = 0;
+            $xfer += $input->readListBegin($_etype575, $_size572);
+            for ($_i576 = 0; $_i576 < $_size572; ++$_i576)
             {
-              $elem570 = null;
-              $elem570 = new \metastore\ShowLocksResponseElement();
-              $xfer += $elem570->read($input);
-              $this->locks []= $elem570;
+              $elem577 = null;
+              $elem577 = new \metastore\ShowLocksResponseElement();
+              $xfer += $elem577->read($input);
+              $this->locks []= $elem577;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18640,9 +18714,9 @@ class ShowLocksResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->locks));
         {
-          foreach ($this->locks as $iter571)
+          foreach ($this->locks as $iter578)
           {
-            $xfer += $iter571->write($output);
+            $xfer += $iter578->write($output);
           }
         }
         $output->writeListEnd();
@@ -18917,17 +18991,17 @@ class HeartbeatTxnRangeResponse {
         case 1:
           if ($ftype == TType::SET) {
             $this->aborted = array();
-            $_size572 = 0;
-            $_etype575 = 0;
-            $xfer += $input->readSetBegin($_etype575, $_size572);
-            for ($_i576 = 0; $_i576 < $_size572; ++$_i576)
+            $_size579 = 0;
+            $_etype582 = 0;
+            $xfer += $input->readSetBegin($_etype582, $_size579);
+            for ($_i583 = 0; $_i583 < $_size579; ++$_i583)
             {
-              $elem577 = null;
-              $xfer += $input->readI64($elem577);
-              if (is_scalar($elem577)) {
-                $this->aborted[$elem577] = true;
+              $elem584 = null;
+              $xfer += $input->readI64($elem584);
+              if (is_scalar($elem584)) {
+                $this->aborted[$elem584] = true;
               } else {
-                $this->aborted []= $elem577;
+                $this->aborted []= $elem584;
               }
             }
             $xfer += $input->readSetEnd();
@@ -18938,17 +19012,17 @@ class HeartbeatTxnRangeResponse {
         case 2:
           if ($ftype == TType::SET) {
             $this->nosuch = array();
-            $_size578 = 0;
-            $_etype581 = 0;
-            $xfer += $input->readSetBegin($_etype581, $_size578);
-            for ($_i582 = 0; $_i582 < $_size578; ++$_i582)
+            $_size585 = 0;
+            $_etype588 = 0;
+            $xfer += $input->readSetBegin($_etype588, $_size585);
+            for ($_i589 = 0; $_i589 < $_size585; ++$_i589)
             {
-              $elem583 = null;
-              $xfer += $input->readI64($elem583);
-              if (is_scalar($elem583)) {
-                $this->nosuch[$elem583] = true;
+              $elem590 = null;
+              $xfer += $input->readI64($elem590);
+              if (is_scalar($elem590)) {
+                $this->nosuch[$elem590] = true;
               } else {
-                $this->nosuch []= $elem583;
+                $this->nosuch []= $elem590;
               }
             }
             $xfer += $input->readSetEnd();
@@ -18977,12 +19051,12 @@ class HeartbeatTxnRangeResponse {
       {
         $output->writeSetBegin(TType::I64, count($this->aborted));
         {
-          foreach ($this->aborted as $iter584 => $iter585)
+          foreach ($this->aborted as $iter591 => $iter592)
           {
-            if (is_scalar($iter585)) {
-            $xfer += $output->writeI64($iter584);
+            if (is_scalar($iter592)) {
+            $xfer += $output->writeI64($iter591);
             } else {
-            $xfer += $output->writeI64($iter585);
+            $xfer += $output->writeI64($iter592);
             }
           }
         }
@@ -18998,12 +19072,12 @@ class HeartbeatTxnRangeResponse {
       {
         $output->writeSetBegin(TType::I64, count($this->nosuch));
         {
-          foreach ($this->nosuch as $iter586 => $iter587)
+          foreach ($this->nosuch as $iter593 => $iter594)
           {
-            if (is_scalar($iter587)) {
-            $xfer += $output->writeI64($iter586);
+            if (is_scalar($iter594)) {
+            $xfer += $output->writeI64($iter593);
             } else {
-            $xfer += $output->writeI64($iter587);
+            $xfer += $output->writeI64($iter594);
             }
           }
         }
@@ -19162,17 +19236,17 @@ class CompactionRequest {
         case 6:
           if ($ftype == TType::MAP) {
             $this->properties = array();
-            $_size588 = 0;
-            $_ktype589 = 0;
-            $_vtype590 = 0;
-            $xfer += $input->readMapBegin($_ktype589, $_vtype590, $_size588);
-            for ($_i592 = 0; $_i592 < $_size588; ++$_i592)
+            $_size595 = 0;
+            $_ktype596 = 0;
+            $_vtype597 = 0;
+            $xfer += $input->readMapBegin($_ktype596, $_vtype597, $_size595);
+            for ($_i599 = 0; $_i599 < $_size595; ++$_i599)
             {
-              $key593 = '';
-              $val594 = '';
-              $xfer += $input->readString($key593);
-              $xfer += $input->readString($val594);
-              $this->properties[$key593] = $val594;
+              $key600 = '';
+              $val601 = '';
+              $xfer += $input->readString($key600);
+              $xfer += $input->readString($val601);
+              $this->properties[$key600] = $val601;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -19225,10 +19299,10 @@ class CompactionRequest {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->properties));
         {
-          foreach ($this->properties as $kiter595 => $viter596)
+          foreach ($this->properties as $kiter602 => $viter603)
           {
-            $xfer += $output->writeString($kiter595);
-            $xfer += $output->writeString($viter596);
+            $xfer += $output->writeString($kiter602);
+            $xfer += $output->writeString($viter603);
           }
         }
         $output->writeMapEnd();
@@ -19815,15 +19889,15 @@ class ShowCompactResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->compacts = array();
-            $_size597 = 0;
-            $_etype600 = 0;
-            $xfer += $input->readListBegin($_etype600, $_size597);
-            for ($_i601 = 0; $_i601 < $_size597; ++$_i601)
+            $_size604 = 0;
+            $_etype607 = 0;
+            $xfer += $input->readListBegin($_etype607, $_size604);
+            for ($_i608 = 0; $_i608 < $_size604; ++$_i608)
             {
-              $elem602 = null;
-              $elem602 = new \metastore\ShowCompactResponseElement();
-              $xfer += $elem602->read($input);
-              $this->compacts []= $elem602;
+              $elem609 = null;
+              $elem609 = new \metastore\ShowCompactResponseElement();
+              $xfer += $elem609->read($input);
+              $this->compacts []= $elem609;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19851,9 +19925,9 @@ class ShowCompactResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->compacts));
         {
-          foreach ($this->compacts as $iter603)
+          foreach ($this->compacts as $iter610)
           {
-            $xfer += $iter603->write($output);
+            $xfer += $iter610->write($output);
           }
         }
         $output->writeListEnd();
@@ -20000,14 +20074,14 @@ class AddDynamicPartitions {
         case 5:
           if ($ftype == TType::LST) {
             $this->partitionnames = array();
-            $_size604 = 0;
-            $_etype607 = 0;
-            $xfer += $input->readListBegin($_etype607, $_size604);
-            for ($_i608 = 0; $_i608 < $_size604; ++$_i608)
+            $_size611 = 0;
+            $_etype614 = 0;
+            $xfer += $input->readListBegin($_etype614, $_size611);
+            for ($_i615 = 0; $_i615 < $_size611; ++$_i615)
             {
-              $elem609 = null;
-              $xfer += $input->readString($elem609);
-              $this->partitionnames []= $elem609;
+              $elem616 = null;
+              $xfer += $input->readString($elem616);
+              $this->partitionnames []= $elem616;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20062,9 +20136,9 @@ class AddDynamicPartitions {
       {
         $output->writeListBegin(TType::STRING, count($this->partitionnames));
         {
-          foreach ($this->partitionnames as $iter610)
+          foreach ($this->partitionnames as $iter617)
           {
-            $xfer += $output->writeString($iter610);
+            $xfer += $output->writeString($iter617);
           }
         }
         $output->writeListEnd();
@@ -20388,17 +20462,17 @@ class CreationMetadata {
         case 4:
           if ($ftype == TType::SET) {
             $this->tablesUsed = array();
-            $_size611 = 0;
-            $_etype614 = 0;
-            $xfer += $input->readSetBegin($_etype614, $_size611);
-            for ($_i615 = 0; $_i615 < $_size611; ++$_i615)
+            $_size618 = 0;
+            $_etype621 = 0;
+            $xfer += $input->readSetBegin($_etype621, $_size618);
+            for ($_i622 = 0; $_i622 < $_size618; ++$_i622)
             {
-              $elem616 = null;
-              $xfer += $input->readString($elem616);
-              if (is_scalar($elem616)) {
-                $this->tablesUsed[$elem616] = true;
+              $elem623 = null;
+              $xfer += $input->readString($elem623);
+              if (is_scalar($elem623)) {
+                $this->tablesUsed[$elem623] = true;
               } else {
-                $this->tablesUsed []= $elem616;
+                $this->tablesUsed []= $elem623;
               }
             }
             $xfer += $input->readSetEnd();
@@ -20449,12 +20523,12 @@ class CreationMetadata {
       {
         $output->writeSetBegin(TType::STRING, count($this->tablesUsed));
         {
-          foreach ($this->tablesUsed as $iter617 => $iter618)
+          foreach ($this->tablesUsed as $iter624 => $iter625)
           {
-            if (is_scalar($iter618)) {
-            $xfer += $output->writeString($iter617);
+            if (is_scalar($iter625)) {
+            $xfer += $output->writeString($iter624);
             } else {
-            $xfer += $output->writeString($iter618);
+            $xfer += $output->writeString($iter625);
             }
           }
         }
@@ -20859,15 +20933,15 @@ class NotificationEventResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->events = array();
-            $_size619 = 0;
-            $_etype622 = 0;
-            $xfer += $input->readListBegin($_etype622, $_size619);
-            for ($_i623 = 0; $_i623 < $_size619; ++$_i623)
+            $_size626 = 0;
+            $_etype629 = 0;
+            $xfer += $input->readListBegin($_etype629, $_size626);
+            for ($_i630 = 0; $_i630 < $_size626; ++$_i630)
             {
-              $elem624 = null;
-              $elem624 = new \metastore\NotificationEvent();
-              $xfer += $elem624->read($input);
-              $this->events []= $elem624;
+              $elem631 = null;
+              $elem631 = new \metastore\NotificationEvent();
+              $xfer += $elem631->read($input);
+              $this->events []= $elem631;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20895,9 +20969,9 @@ class NotificationEventResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->events));
         {
-          foreach ($this->events as $iter625)
+          foreach ($this->events as $iter632)
           {
-            $xfer += $iter625->write($output);
+            $xfer += $iter632->write($output);
           }
         }
         $output->writeListEnd();
@@ -21265,14 +21339,14 @@ class InsertEventRequestData {
         case 2:
           if ($ftype == TType::LST) {
             $this->filesAdded = array();
-            $_size626 = 0;
-            $_etype629 = 0;
-            $xfer += $input->readListBegin($_etype629, $_size626);
-            for ($_i630 = 0; $_i630 < $_size626; ++$_i630)
+            $_size633 = 0;
+            $_etype636 = 0;
+            $xfer += $input->readListBegin($_etype636, $_size633);
+            for ($_i637 = 0; $_i637 < $_size633; ++$_i637)
             {
-              $elem631 = null;
-              $xfer += $input->readString($elem631);
-              $this->filesAdded []= $elem631;
+              $elem638 = null;
+              $xfer += $input->readString($elem638);
+              $this->filesAdded []= $elem638;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21282,14 +21356,14 @@ class InsertEventRequestData {
         case 3:
           if ($ftype == TType::LST) {
             $this->filesAddedChecksum = array();
-            $_size632 = 0;
-            $_etype635 = 0;
-            $xfer += $input->readListBegin($_etype635, $_size632);
-            for ($_i636 = 0; $_i636 < $_size632; ++$_i636)
+            $_size639 = 0;
+            $_etype642 = 0;
+            $xfer += $input->readListBegin($_etype642, $_size639);
+            for ($_i643 = 0; $_i643 < $_size639; ++$_i643)
             {
-              $elem637 = null;
-              $xfer += $input->readString($elem637);
-              $this->filesAddedChecksum []= $elem637;
+              $elem644 = null;
+              $xfer += $input->readString($elem644);
+              $this->filesAddedChecksum []= $elem644;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21322,9 +21396,9 @@ class InsertEventRequestData {
       {
         $output->writeListBegin(TType::STRING, count($this->filesAdded));
         {
-          foreach ($this->filesAdded as $iter638)
+          foreach ($this->filesAdded as $iter645)
           {
-            $xfer += $output->writeString($iter638);
+            $xfer += $output->writeString($iter645);
           }
         }
         $output->writeListEnd();
@@ -21339,9 +21413,9 @@ class InsertEventRequestData {
       {
         $output->writeListBegin(TType::STRING, count($this->filesAddedChecksum));
         {
-          foreach ($this->filesAddedChecksum as $iter639)
+          foreach ($this->filesAddedChecksum as $iter646)
           {
-            $xfer += $output->writeString($iter639);
+            $xfer += $output->writeString($iter646);
           }
         }
         $output->writeListEnd();
@@ -21570,14 +21644,14 @@ class FireEventRequest {
         case 5:
           if ($ftype == TType::LST) {
             $this->partitionVals = array();
-            $_size640 = 0;
-            $_etype643 = 0;
-            $xfer += $input->readListBegin($_etype643, $_size640);
-            for ($_i644 = 0; $_i644 < $_size640; ++$_i644)
+            $_size647 = 0;
+            $_etype650 = 0;
+            $xfer += $input->readListBegin($_etype650, $_size647);
+            for ($_i651 = 0; $_i651 < $_size647; ++$_i651)
             {
-              $elem645 = null;
-              $xfer += $input->readString($elem645);
-              $this->partitionVals []= $elem645;
+              $elem652 = null;
+              $xfer += $input->readString($elem652);
+              $this->partitionVals []= $elem652;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21635,9 +21709,9 @@ class FireEventRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->partitionVals));
         {
-          foreach ($this->partitionVals as $iter646)
+          foreach ($this->partitionVals as $iter653)
           {
-            $xfer += $output->writeString($iter646);
+            $xfer += $output->writeString($iter653);
           }
         }
         $output->writeListEnd();
@@ -21870,18 +21944,18 @@ class GetFileMetadataByExprResult {
         case 1:
           if ($ftype == TType::MAP) {
             $this->metadata = array();
-            $_size647 = 0;
-            $_ktype648 = 0;
-            $_vtype649 = 0;
-            $xfer += $input->readMapBegin($_ktype648, $_vtype649, $_size647);
-            for ($_i651 = 0; $_i651 < $_size647; ++$_i651)
+            $_size654 = 0;
+            $_ktype655 = 0;
+            $_vtype656 = 0;
+            $xfer += $input->readMapBegin($_ktype655, $_vtype656, $_size654);
+            for ($_i658 = 0; $_i658 < $_size654; ++$_i658)
             {
-              $key652 = 0;
-              $val653 = new \metastore\MetadataPpdResult();
-              $xfer += $input->readI64($key652);
-              $val653 = new \metastore\MetadataPpdResult();
-              $xfer += $val653->read($input);
-              $this->metadata[$key652] = $val653;
+              $key659 = 0;
+              $val660 = new \metastore\MetadataPpdResult();
+              $xfer += $input->readI64($key659);
+              $val660 = new \metastore\MetadataPpdResult();
+              $xfer += $val660->read($input);
+              $this->metadata[$key659] = $val660;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -21916,10 +21990,10 @@ class GetFileMetadataByExprResult {
       {
         $output->writeMapBegin(TType::I64, TType::STRUCT, count($this->metadata));
         {
-          foreach ($this->metadata as $kiter654 => $viter655)
+          foreach ($this->metadata as $kiter661 => $viter662)
           {
-            $xfer += $output->writeI64($kiter654);
-            $xfer += $viter655->write($output);
+            $xfer += $output->writeI64($kiter661);
+            $xfer += $viter662->write($output);
           }
         }
         $output->writeMapEnd();
@@ -22021,14 +22095,14 @@ class GetFileMetadataByExprRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fileIds = array();
-            $_size656 = 0;
-            $_etype659 = 0;
-            $xfer += $input->readListBegin($_etype659, $_size656);
-            for ($_i660 = 0; $_i660 < $_size656; ++$_i660)
+            $_size663 = 0;
+            $_etype666 = 0;
+            $xfer += $input->readListBegin($_etype666, $_size663);
+            for ($_i667 = 0; $_i667 < $_size663; ++$_i667)
             {
-              $elem661 = null;
-              $xfer += $input->readI64($elem661);
-              $this->fileIds []= $elem661;
+              $elem668 = null;
+              $xfer += $input->readI64($elem668);
+              $this->fileIds []= $elem668;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22077,9 +22151,9 @@ class GetFileMetadataByExprRequest {
       {
         $output->writeListBegin(TType::I64, count($this->fileIds));
         {
-          foreach ($this->fileIds as $iter662)
+          foreach ($this->fileIds as $iter669)
           {
-            $xfer += $output->writeI64($iter662);
+            $xfer += $output->writeI64($iter669);
           }
         }
         $output->writeListEnd();
@@ -22173,17 +22247,17 @@ class GetFileMetadataResult {
         case 1:
           if ($ftype == TType::MAP) {
             $this->metadata = array();
-            $_size663 = 0;
-            $_ktype664 = 0;
-            $_vtype665 = 0;
-            $xfer += $input->readMapBegin($_ktype664, $_vtype665, $_size663);
-            for ($_i667 = 0; $_i667 < $_size663; ++$_i667)
+            $_size670 = 0;
+            $_ktype671 = 0;
+            $_vtype672 = 0;
+            $xfer += $input->readMapBegin($_ktype671, $_vtype672, $_size670);
+            for ($_i674 = 0; $_i674 < $_size670; ++$_i674)
             {
-              $key668 = 0;
-              $val669 = '';
-              $xfer += $input->readI64($key668);
-              $xfer += $input->readString($val669);
-              $this->metadata[$key668] = $val669;
+              $key675 = 0;
+              $val676 = '';
+              $xfer += $input->readI64($key675);
+              $xfer += $input->readString($val676);
+              $this->metadata[$key675] = $val676;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -22218,10 +22292,10 @@ class GetFileMetadataResult {
       {
         $output->writeMapBegin(TType::I64, TType::STRING, count($this->metadata));
         {
-          foreach ($this->metadata as $kiter670 => $viter671)
+          foreach ($this->metadata as $kiter677 => $viter678)
           {
-            $xfer += $output->writeI64($kiter670);
-            $xfer += $output->writeString($viter671);
+            $xfer += $output->writeI64($kiter677);
+            $xfer += $output->writeString($viter678);
           }
         }
         $output->writeMapEnd();
@@ -22290,14 +22364,14 @@ class GetFileMetadataRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fileIds = array();
-            $_size672 = 0;
-            $_etype675 = 0;
-            $xfer += $input->readListBegin($_etype675, $_size672);
-            for ($_i676 = 0; $_i676 < $_size672; ++$_i676)
+            $_size679 = 0;
+            $_etype682 = 0;
+            $xfer += $input->readListBegin($_etype682, $_size679);
+            for ($_i683 = 0; $_i683 < $_size679; ++$_i683)
             {
-              $elem677 = null;
-              $xfer += $input->readI64($elem677);
-              $this->fileIds []= $elem677;
+              $elem684 = null;
+              $xfer += $input->readI64($elem684);
+              $this->fileIds []= $elem684;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22325,9 +22399,9 @@ class GetFileMetadataRequest {
       {
         $output->writeListBegin(TType::I64, count($this->fileIds));
         {
-          foreach ($this->fileIds as $iter678)
+          foreach ($this->fileIds as $iter685)
           {
-            $xfer += $output->writeI64($iter678);
+            $xfer += $output->writeI64($iter685);
           }
         }
         $output->writeListEnd();
@@ -22467,14 +22541,14 @@ class PutFileMetadataRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fileIds = array();
-            $_size679 = 0;
-            $_etype682 = 0;
-            $xfer += $input->readListBegin($_etype682, $_size679);
-            for ($_i683 = 0; $_i683 < $_size679; ++$_i683)
+            $_size686 = 0;
+            $_etype689 = 0;
+            $xfer += $input->readListBegin($_etype689, $_size686);
+            for ($_i690 = 0; $_i690 < $_size686; ++$_i690)
             {
-              $elem684 = null;
-              $xfer += $input->readI64($elem684);
-              $this->fileIds []= $elem684;
+              $elem691 = null;
+              $xfer += $input->readI64($elem691);
+              $this->fileIds []= $elem691;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22484,14 +22558,14 @@ class PutFileMetadataRequest {
         case 2:
           if ($ftype == TType::LST) {
             $this->metadata = array();
-            $_size685 = 0;
-            $_etype688 = 0;
-            $xfer += $input->readListBegin($_etype688, $_size685);
-            for ($_i689 = 0; $_i689 < $_size685; ++$_i689)
+            $_size692 = 0;
+            $_etype695 = 0;
+            $xfer += $input->readListBegin($_etype695, $_size692);
+            for ($_i696 = 0; $_i696 < $_size692; ++$_i696)
             {
-              $elem690 = null;
-              $xfer += $input->readString($elem690);
-              $this->metadata []= $elem690;
+              $elem697 = null;
+              $xfer += $input->readString($elem697);
+              $this->metadata []= $elem697;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22526,9 +22600,9 @@ class PutFileMetadataRequest {
       {
         $output->writeListBegin(TType::I64, count($this->fileIds));
         {
-          foreach ($this->fileIds as $iter691)
+          foreach ($this->fileIds as $iter698)
           {
-            $xfer += $output->writeI64($iter691);
+            $xfer += $output->writeI64($iter698);
           }
         }
         $output->writeListEnd();
@@ -22543,9 +22617,9 @@ class PutFileMetadataRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->metadata));
         {
-          foreach ($this->metadata as $iter692)
+          foreach ($this->metadata as $iter699)
           {
-            $xfer += $output->writeString($iter692);
+            $xfer += $output->writeString($iter699);
           }
         }
         $output->writeListEnd();
@@ -22664,14 +22738,14 @@ class ClearFileMetadataRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fileIds = array();
-            $_size693 = 0;
-            $_etype696 = 0;
-            $xfer += $input->readListBegin($_etype696, $_size693);
-            for ($_i697 = 0; $_i697 < $_size693; ++$_i697)
+            $_size700 = 0;
+            $_etype703 = 0;
+            $xfer += $input->readListBegin($_etype703, $_size700);
+            for ($_i704 = 0; $_i704 < $_size700; ++$_i704)
             {
-              $elem698 = null;
-              $xfer += $input->readI64($elem698);
-              $this->fileIds []= $elem698;
+              $elem705 = null;
+              $xfer += $input->readI64($elem705);
+              $this->fileIds []= $elem705;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22699,9 +22773,9 @@ class ClearFileMetadataRequest {
       {
         $output->writeListBegin(TType::I64, count($this->fileIds));
         {
-          foreach ($this->fileIds as $iter699)
+          foreach ($this->fileIds as $iter706)
           {
-            $xfer += $output->writeI64($iter699);
+            $xfer += $output->writeI64($iter706);
           }
         }
         $output->writeListEnd();
@@ -22985,15 +23059,15 @@ class GetAllFunctionsResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->functions = array();
-            $_size700 = 0;
-            $_etype703 = 0;
-            $xfer += $input->readListBegin($_etype703, $_size700);
-            for ($_i704 = 0; $_i704 < $_size700; ++$_i704)
+            $_size707 = 0;
+            $_etype710 = 0;
+            $xfer += $input->readListBegin($_etype710, $_size707);
+            for ($_i711 = 0; $_i711 < $_size707; ++$_i711)
             {
-              $elem705 = null;
-              $elem705 = new \metastore\Function();
-              $xfer += $elem705->read($input);
-              $this->functions []= $elem705;
+              $elem712 = null;
+              $elem712 = new \metastore\Function();
+              $xfer += $elem712->read($input);
+              $this->functions []= $elem712;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23021,9 +23095,9 @@ class GetAllFunctionsResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->functions));
         {
-          foreach ($this->functions as $iter706)
+          foreach ($this->functions as $iter713)
           {
-            $xfer += $iter706->write($output);
+            $xfer += $iter713->write($output);
           }
         }
         $output->writeListEnd();
@@ -23087,14 +23161,14 @@ class ClientCapabilities {
         case 1:
           if ($ftype == TType::LST) {
             $this->values = array();
-            $_size707 = 0;
-            $_etype710 = 0;
-            $xfer += $input->readListBegin($_etype710, $_size707);
-            for ($_i711 = 0; $_i711 < $_size707; ++$_i711)
+            $_size714 = 0;
+            $_etype717 = 0;
+            $xfer += $input->readListBegin($_etype717, $_size714);
+            for ($_i718 = 0; $_i718 < $_size714; ++$_i718)
             {
-              $elem712 = null;
-              $xfer += $input->readI32($elem712);
-              $this->values []= $elem712;
+              $elem719 = null;
+              $xfer += $input->readI32($elem719);
+              $this->values []= $elem719;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23122,9 +23196,9 @@ class ClientCapabilities {
       {
         $output->writeListBegin(TType::I32, count($this->values));
         {
-          foreach ($this->values as $iter713)
+          foreach ($this->values as $iter720)
           {
-            $xfer += $output->writeI32($iter713);
+            $xfer += $output->writeI32($iter720);
           }
         }
         $output->writeListEnd();
@@ -23458,14 +23532,14 @@ class GetTablesRequest {
         case 2:
           if ($ftype == TType::LST) {
             $this->tblNames = array();
-            $_size714 = 0;
-            $_etype717 = 0;
-            $xfer += $input->readListBegin($_etype717, $_size714);
-            for ($_i718 = 0; $_i718 < $_size714; ++$_i718)
+            $_size721 = 0;
+            $_etype724 = 0;
+            $xfer += $input->readListBegin($_etype724, $_size721);
+            for ($_i725 = 0; $_i725 < $_size721; ++$_i725)
             {
-              $elem719 = null;
-              $xfer += $input->readString($elem719);
-              $this->tblNames []= $elem719;
+              $elem726 = null;
+              $xfer += $input->readString($elem726);
+              $this->tblNames []= $elem726;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23513,9 +23587,9 @@ class GetTablesRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->tblNames));
         {
-          foreach ($this->tblNames as $iter720)
+          foreach ($this->tblNames as $iter727)
           {
-            $xfer += $output->writeString($iter720);
+            $xfer += $output->writeString($iter727);
           }
         }
         $output->writeListEnd();
@@ -23593,15 +23667,15 @@ class GetTablesResult {
         case 1:
           if ($ftype == TType::LST) {
             $this->tables = array();
-            $_size721 = 0;
-            $_etype724 = 0;
-            $xfer += $input->readListBegin($_etype724, $_size721);
-            for ($_i725 = 0; $_i725 < $_size721; ++$_i725)
+            $_size728 = 0;
+            $_etype731 = 0;
+            $xfer += $input->readListBegin($_etype731, $_size728);
+            for ($_i732 = 0; $_i732 < $_size728; ++$_i732)
             {
-              $elem726 = null;
-              $elem726 = new \metastore\Table();
-              $xfer += $elem726->read($input);
-              $this->tables []= $elem726;
+              $elem733 = null;
+              $elem733 = new \metastore\Table();
+              $xfer += $elem733->read($input);
+              $this->tables []= $elem733;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23629,9 +23703,9 @@ class GetTablesResult {
       {
         $output->writeListBegin(TType::STRUCT, count($this->tables));
         {
-          foreach ($this->tables as $iter727)
+          foreach ($this->tables as $iter734)
           {
-            $xfer += $iter727->write($output);
+            $xfer += $iter734->write($output);
           }
         }
         $output->writeListEnd();
@@ -24043,17 +24117,17 @@ class Materialization {
         case 1:
           if ($ftype == TType::SET) {
             $this->tablesUsed = array();
-            $_size728 = 0;
-            $_etype731 = 0;
-            $xfer += $input->readSetBegin($_etype731, $_size728);
-            for ($_i732 = 0; $_i732 < $_size728; ++$_i732)
+            $_size735 = 0;
+            $_etype738 = 0;
+            $xfer += $input->readSetBegin($_etype738, $_size735);
+            for ($_i739 = 0; $_i739 < $_size735; ++$_i739)
             {
-              $elem733 = null;
-              $xfer += $input->readString($elem733);
-              if (is_scalar($elem733)) {
-                $this->tablesUsed[$elem733] = true;
+              $elem740 = null;
+              $xfer += $input->readString($elem740);
+              if (is_scalar($elem740)) {
+                $this->tablesUsed[$elem740] = true;
               } else {
-                $this->tablesUsed []= $elem733;
+                $this->tablesUsed []= $elem740;
               }
             }
             $xfer += $input->readSetEnd();
@@ -24103,12 +24177,12 @@ class Materialization {
       {
         $output->writeSetBegin(TType::STRING, count($this->tablesUsed));
         {
-          foreach ($this->tablesUsed as $iter734 => $iter735)
+          foreach ($this->tablesUsed as $iter741 => $iter742)
           {
-            if (is_scalar($iter735)) {
-            $xfer += $output->writeString($iter734);
+            if (is_scalar($iter742)) {
+            $xfer += $output->writeString($iter741);
             } else {
-            $xfer += $output->writeString($iter735);
+            $xfer += $output->writeString($iter742);
             }
           }
         }
@@ -25380,15 +25454,15 @@ class WMFullResourcePlan {
         case 2:
           if ($ftype == TType::LST) {
             $this->pools = array();
-            $_size736 = 0;
-            $_etype739 = 0;
-            $xfer += $input->readListBegin($_etype739, $_size736);
-            for ($_i740 = 0; $_i740 < $_size736; ++$_i740)
+            $_size743 = 0;
+            $_etype746 = 0;
+            $xfer += $input->readListBegin($_etype746, $_size743);
+            for ($_i747 = 0; $_i747 < $_size743; ++$_i747)
             {
-              $elem741 = null;
-              $elem741 = new \metastore\WMPool();
-              $xfer += $elem741->read($input);
-              $this->pools []= $elem741;
+              $elem748 = null;
+              $elem748 = new \metastore\WMPool();
+              $xfer += $elem748->read($input);
+              $this->pools []= $elem748;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25398,15 +25472,15 @@ class WMFullResourcePlan {
         case 3:
           if ($ftype == TType::LST) {
             $this->mappings = array();
-            $_size742 = 0;
-            $_etype745 = 0;
-            $xfer += $input->readListBegin($_etype745, $_size742);
-            for ($_i746 = 0; $_i746 < $_size742; ++$_i746)
+            $_size749 = 0;
+            $_etype752 = 0;
+            $xfer += $input->readListBegin($_etype752, $_size749);
+            for ($_i753 = 0; $_i753 < $_size749; ++$_i753)
             {
-              $elem747 = null;
-              $elem747 = new \metastore\WMMapping();
-              $xfer += $elem747->read($input);
-              $this->mappings []= $elem747;
+              $elem754 = null;
+              $elem754 = new \metastore\WMMapping();
+              $xfer += $elem754->read($input);
+              $this->mappings []= $elem754;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25416,15 +25490,15 @@ class WMFullResourcePlan {
         case 4:
           if ($ftype == TType::LST) {
             $this->triggers = array();
-            $_size748 = 0;
-            $_etype751 = 0;
-            $xfer += $input->readListBegin($_etype751, $_size748);
-            for ($_i752 = 0; $_i752 < $_size748; ++$_i752)
+            $_size755 = 0;
+            $_etype758 = 0;
+            $xfer += $input->readListBegin($_etype758, $_size755);
+            for ($_i759 = 0; $_i759 < $_size755; ++$_i759)
             {
-              $elem753 = null;
-              $elem753 = new \metastore\WMTrigger();
-              $xfer += $elem753->read($input);
-              $this->triggers []= $elem753;
+              $elem760 = null;
+              $elem760 = new \metastore\WMTrigger();
+              $xfer += $elem760->read($input);
+              $this->triggers []= $elem760;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25434,15 +25508,15 @@ class WMFullResourcePlan {
         case 5:
           if ($ftype == TType::LST) {
             $this->poolTriggers = array();
-            $_size754 = 0;
-            $_etype757 = 0;
-            $xfer += $input->readListBegin($_etype757, $_size754);
-            for ($_i758 = 0; $_i758 < $_size754; ++$_i758)
+            $_size761 = 0;
+            $_etype764 = 0;
+            $xfer += $input->readListBegin($_etype764, $_size761);
+            for ($_i765 = 0; $_i765 < $_size761; ++$_i765)
             {
-              $elem759 = null;
-              $elem759 = new \metastore\WMPoolTrigger();
-              $xfer += $elem759->read($input);
-              $this->poolTriggers []= $elem759;
+              $elem766 = null;
+              $elem766 = new \metastore\WMPoolTrigger();
+              $xfer += $elem766->read($input);
+              $this->poolTriggers []= $elem766;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25478,9 +25552,9 @@ class WMFullResourcePlan {
       {
         $output->writeListBegin(TType::STRUCT, count($this->pools));
         {
-          foreach ($this->pools as $iter760)
+          foreach ($this->pools as $iter767)
           {
-            $xfer += $iter760->write($output);
+            $xfer += $iter767->write($output);
           }
         }
         $output->writeListEnd();
@@ -25495,9 +25569,9 @@ class WMFullResourcePlan {
       {
         $output->writeListBegin(TType::STRUCT, count($this->mappings));
         {
-          foreach ($this->mappings as $iter761)
+          foreach ($this->mappings as $iter768)
           {
-            $xfer += $iter761->write($output);
+            $xfer += $iter768->write($output);
           }
         }
         $output->writeListEnd();
@@ -25512,9 +25586,9 @@ class WMFullResourcePlan {
       {
         $output->writeListBegin(TType::STRUCT, count($this->triggers));
         {
-          foreach ($this->triggers as $iter762)
+          foreach ($this->triggers as $iter769)
           {
-            $xfer += $iter762->write($output);
+            $xfer += $iter769->write($output);
           }
         }
         $output->writeListEnd();
@@ -25529,9 +25603,9 @@ class WMFullResourcePlan {
       {
         $output->writeListBegin(TType::STRUCT, count($this->poolTriggers));
         {
-          foreach ($this->poolTriggers as $iter763)
+          foreach ($this->poolTriggers as $iter770)
           {
-            $xfer += $iter763->write($output);
+            $xfer += $iter770->write($output);
           }
         }
         $output->writeListEnd();
@@ -26084,15 +26158,15 @@ class WMGetAllResourcePlanResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->resourcePlans = array();
-            $_size764 = 0;
-            $_etype767 = 0;
-            $xfer += $input->readListBegin($_etype767, $_size764);
-            for ($_i768 = 0; $_i768 < $_size764; ++$_i768)
+            $_size771 = 0;
+            $_etype774 = 0;
+            $xfer += $input->readListBegin($_etype774, $_size771);
+            for ($_i775 = 0; $_i775 < $_size771; ++$_i775)
             {
-              $elem769 = null;
-              $elem769 = new \metastore\WMResourcePlan();
-              $xfer += $elem769->read($input);
-              $this->resourcePlans []= $elem769;
+              $elem776 = null;
+              $elem776 = new \metastore\WMResourcePlan();
+              $xfer += $elem776->read($input);
+              $this->resourcePlans []= $elem776;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26120,9 +26194,9 @@ class WMGetAllResourcePlanResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->resourcePlans));
         {
-          foreach ($this->resourcePlans as $iter770)
+          foreach ($this->resourcePlans as $iter777)
           {
-            $xfer += $iter770->write($output);
+            $xfer += $iter777->write($output);
           }
         }
         $output->writeListEnd();
@@ -26528,14 +26602,14 @@ class WMValidateResourcePlanResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->errors = array();
-            $_size771 = 0;
-            $_etype774 = 0;
-            $xfer += $input->readListBegin($_etype774, $_size771);
-            for ($_i775 = 0; $_i775 < $_size771; ++$_i775)
+            $_size778 = 0;
+            $_etype781 = 0;
+            $xfer += $input->readListBegin($_etype781, $_size778);
+            for ($_i782 = 0; $_i782 < $_size778; ++$_i782)
             {
-              $elem776 = null;
-              $xfer += $input->readString($elem776);
-              $this->errors []= $elem776;
+              $elem783 = null;
+              $xfer += $input->readString($elem783);
+              $this->errors []= $elem783;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26545,14 +26619,14 @@ class WMValidateResourcePlanResponse {
         case 2:
           if ($ftype == TType::LST) {
             $this->warnings = array();
-            $_size777 = 0;
-            $_etype780 = 0;
-            $xfer += $input->readListBegin($_etype780, $_size777);
-            for ($_i781 = 0; $_i781 < $_size777; ++$_i781)
+            $_size784 = 0;
+            $_etype787 = 0;
+            $xfer += $input->readListBegin($_etype787, $_size784);
+            for ($_i788 = 0; $_i788 < $_size784; ++$_i788)
             {
-              $elem782 = null;
-              $xfer += $input->readString($elem782);
-              $this->warnings []= $elem782;
+              $elem789 = null;
+              $xfer += $input->readString($elem789);
+              $this->warnings []= $elem789;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26580,9 +26654,9 @@ class WMValidateResourcePlanResponse {
       {
         $output->writeListBegin(TType::STRING, count($this->errors));
         {
-          foreach ($this->errors as $iter783)
+          foreach ($this->errors as $iter790)
           {
-            $xfer += $output->writeString($iter783);
+            $xfer += $output->writeString($iter790);
           }
         }
         $output->writeListEnd();
@@ -26597,9 +26671,9 @@ class WMValidateResourcePlanResponse {
       {
         $output->writeListBegin(TType::STRING, count($this->warnings));
         {
-          foreach ($this->warnings as $iter784)
+          foreach ($this->warnings as $iter791)
           {
-            $xfer += $output->writeString($iter784);
+            $xfer += $output->writeString($iter791);
           }
         }
         $output->writeListEnd();
@@ -27272,15 +27346,15 @@ class WMGetTriggersForResourePlanResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->triggers = array();
-            $_size785 = 0;
-            $_etype788 = 0;
-            $xfer += $input->readListBegin($_etype788, $_size785);
-            for ($_i789 = 0; $_i789 < $_size785; ++$_i789)
+            $_size792 = 0;
+            $_etype795 = 0;
+            $xfer += $input->readListBegin($_etype795, $_size792);
+            for ($_i796 = 0; $_i796 < $_size792; ++$_i796)
             {
-              $elem790 = null;
-              $elem790 = new \metastore\WMTrigger();
-              $xfer += $elem790->read($input);
-              $this->triggers []= $elem790;
+              $elem797 = null;
+              $elem797 = new \metastore\WMTrigger();
+              $xfer += $elem797->read($input);
+              $this->triggers []= $elem797;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27308,9 +27382,9 @@ class WMGetTriggersForResourePlanResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->triggers));
         {
-          foreach ($this->triggers as $iter791)
+          foreach ($this->triggers as $iter798)
           {
-            $xfer += $iter791->write($output);
+            $xfer += $iter798->write($output);
           }
         }
         $output->writeListEnd();
@@ -28894,15 +28968,15 @@ class SchemaVersion {
         case 4:
           if ($ftype == TType::LST) {
             $this->cols = array();
-            $_size792 = 0;
-            $_etype795 = 0;
-            $xfer += $input->readListBegin($_etype795, $_size792);
-            for ($_i796 = 0; $_i796 < $_size792; ++$_i796)
+            $_size799 = 0;
+            $_etype802 = 0;
+            $xfer += $input->readListBegin($_etype802, $_size799);
+            for ($_i803 = 0; $_i803 < $_size799; ++$_i803)
             {
-              $elem797 = null;
-              $elem797 = new \metastore\FieldSchema();
-              $xfer += $elem797->read($input);
-              $this->cols []= $elem797;
+              $elem804 = null;
+              $elem804 = new \metastore\FieldSchema();
+              $xfer += $elem804->read($input);
+              $this->cols []= $elem804;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28991,9 +29065,9 @@ class SchemaVersion {
       {
         $output->writeListBegin(TType::STRUCT, count($this->cols));
         {
-          foreach ($this->cols as $iter798)
+          foreach ($this->cols as $iter805)
           {
-            $xfer += $iter798->write($output);
+            $xfer += $iter805->write($output);
           }
         }
         $output->writeListEnd();
@@ -29315,15 +29389,15 @@ class FindSchemasByColsResp {
         case 1:
           if ($ftype == TType::LST) {
             $this->schemaVersions = array();
-            $_size799 = 0;
-            $_etype802 = 0;
-            $xfer += $input->readListBegin($_etype802, $_size799);
-            for ($_i803 = 0; $_i803 < $_size799; ++$_i803)
+            $_size806 = 0;
+            $_etype809 = 0;
+            $xfer += $input->readListBegin($_etype809, $_size806);
+            for ($_i810 = 0; $_i810 < $_size806; ++$_i810)
             {
-              $elem804 = null;
-              $elem804 = new \metastore\SchemaVersionDescriptor();
-              $xfer += $elem804->read($input);
-              $this->schemaVersions []= $elem804;
+              $elem811 = null;
+              $elem811 = new \metastore\SchemaVersionDescriptor();
+              $xfer += $elem811->read($input);
+              $this->schemaVersions []= $elem811;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29351,9 +29425,9 @@ class FindSchemasByColsResp {
       {
         $output->writeListBegin(TType::STRUCT, count($this->schemaVersions));
         {
-          foreach ($this->schemaVersions as $iter805)
+          foreach ($this->schemaVersions as $iter812)
           {
-            $xfer += $iter805->write($output);
+            $xfer += $iter812->write($output);
           }
         }
         $output->writeListEnd();


[05/12] hive git commit: HIVE-19089: Create/Replicate Allocate write-id event (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 7a8a42a..9c94942 100644
--- a/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -14937,14 +14937,14 @@ class ThriftHiveMetastore_get_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size806 = 0;
-            $_etype809 = 0;
-            $xfer += $input->readListBegin($_etype809, $_size806);
-            for ($_i810 = 0; $_i810 < $_size806; ++$_i810)
+            $_size813 = 0;
+            $_etype816 = 0;
+            $xfer += $input->readListBegin($_etype816, $_size813);
+            for ($_i817 = 0; $_i817 < $_size813; ++$_i817)
             {
-              $elem811 = null;
-              $xfer += $input->readString($elem811);
-              $this->success []= $elem811;
+              $elem818 = null;
+              $xfer += $input->readString($elem818);
+              $this->success []= $elem818;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14980,9 +14980,9 @@ class ThriftHiveMetastore_get_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter812)
+          foreach ($this->success as $iter819)
           {
-            $xfer += $output->writeString($iter812);
+            $xfer += $output->writeString($iter819);
           }
         }
         $output->writeListEnd();
@@ -15113,14 +15113,14 @@ class ThriftHiveMetastore_get_all_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size813 = 0;
-            $_etype816 = 0;
-            $xfer += $input->readListBegin($_etype816, $_size813);
-            for ($_i817 = 0; $_i817 < $_size813; ++$_i817)
+            $_size820 = 0;
+            $_etype823 = 0;
+            $xfer += $input->readListBegin($_etype823, $_size820);
+            for ($_i824 = 0; $_i824 < $_size820; ++$_i824)
             {
-              $elem818 = null;
-              $xfer += $input->readString($elem818);
-              $this->success []= $elem818;
+              $elem825 = null;
+              $xfer += $input->readString($elem825);
+              $this->success []= $elem825;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15156,9 +15156,9 @@ class ThriftHiveMetastore_get_all_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter819)
+          foreach ($this->success as $iter826)
           {
-            $xfer += $output->writeString($iter819);
+            $xfer += $output->writeString($iter826);
           }
         }
         $output->writeListEnd();
@@ -16159,18 +16159,18 @@ class ThriftHiveMetastore_get_type_all_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size820 = 0;
-            $_ktype821 = 0;
-            $_vtype822 = 0;
-            $xfer += $input->readMapBegin($_ktype821, $_vtype822, $_size820);
-            for ($_i824 = 0; $_i824 < $_size820; ++$_i824)
+            $_size827 = 0;
+            $_ktype828 = 0;
+            $_vtype829 = 0;
+            $xfer += $input->readMapBegin($_ktype828, $_vtype829, $_size827);
+            for ($_i831 = 0; $_i831 < $_size827; ++$_i831)
             {
-              $key825 = '';
-              $val826 = new \metastore\Type();
-              $xfer += $input->readString($key825);
-              $val826 = new \metastore\Type();
-              $xfer += $val826->read($input);
-              $this->success[$key825] = $val826;
+              $key832 = '';
+              $val833 = new \metastore\Type();
+              $xfer += $input->readString($key832);
+              $val833 = new \metastore\Type();
+              $xfer += $val833->read($input);
+              $this->success[$key832] = $val833;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -16206,10 +16206,10 @@ class ThriftHiveMetastore_get_type_all_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $kiter827 => $viter828)
+          foreach ($this->success as $kiter834 => $viter835)
           {
-            $xfer += $output->writeString($kiter827);
-            $xfer += $viter828->write($output);
+            $xfer += $output->writeString($kiter834);
+            $xfer += $viter835->write($output);
           }
         }
         $output->writeMapEnd();
@@ -16413,15 +16413,15 @@ class ThriftHiveMetastore_get_fields_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size829 = 0;
-            $_etype832 = 0;
-            $xfer += $input->readListBegin($_etype832, $_size829);
-            for ($_i833 = 0; $_i833 < $_size829; ++$_i833)
+            $_size836 = 0;
+            $_etype839 = 0;
+            $xfer += $input->readListBegin($_etype839, $_size836);
+            for ($_i840 = 0; $_i840 < $_size836; ++$_i840)
             {
-              $elem834 = null;
-              $elem834 = new \metastore\FieldSchema();
-              $xfer += $elem834->read($input);
-              $this->success []= $elem834;
+              $elem841 = null;
+              $elem841 = new \metastore\FieldSchema();
+              $xfer += $elem841->read($input);
+              $this->success []= $elem841;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16473,9 +16473,9 @@ class ThriftHiveMetastore_get_fields_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter835)
+          foreach ($this->success as $iter842)
           {
-            $xfer += $iter835->write($output);
+            $xfer += $iter842->write($output);
           }
         }
         $output->writeListEnd();
@@ -16717,15 +16717,15 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size836 = 0;
-            $_etype839 = 0;
-            $xfer += $input->readListBegin($_etype839, $_size836);
-            for ($_i840 = 0; $_i840 < $_size836; ++$_i840)
+            $_size843 = 0;
+            $_etype846 = 0;
+            $xfer += $input->readListBegin($_etype846, $_size843);
+            for ($_i847 = 0; $_i847 < $_size843; ++$_i847)
             {
-              $elem841 = null;
-              $elem841 = new \metastore\FieldSchema();
-              $xfer += $elem841->read($input);
-              $this->success []= $elem841;
+              $elem848 = null;
+              $elem848 = new \metastore\FieldSchema();
+              $xfer += $elem848->read($input);
+              $this->success []= $elem848;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16777,9 +16777,9 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter842)
+          foreach ($this->success as $iter849)
           {
-            $xfer += $iter842->write($output);
+            $xfer += $iter849->write($output);
           }
         }
         $output->writeListEnd();
@@ -16993,15 +16993,15 @@ class ThriftHiveMetastore_get_schema_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size843 = 0;
-            $_etype846 = 0;
-            $xfer += $input->readListBegin($_etype846, $_size843);
-            for ($_i847 = 0; $_i847 < $_size843; ++$_i847)
+            $_size850 = 0;
+            $_etype853 = 0;
+            $xfer += $input->readListBegin($_etype853, $_size850);
+            for ($_i854 = 0; $_i854 < $_size850; ++$_i854)
             {
-              $elem848 = null;
-              $elem848 = new \metastore\FieldSchema();
-              $xfer += $elem848->read($input);
-              $this->success []= $elem848;
+              $elem855 = null;
+              $elem855 = new \metastore\FieldSchema();
+              $xfer += $elem855->read($input);
+              $this->success []= $elem855;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17053,9 +17053,9 @@ class ThriftHiveMetastore_get_schema_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter849)
+          foreach ($this->success as $iter856)
           {
-            $xfer += $iter849->write($output);
+            $xfer += $iter856->write($output);
           }
         }
         $output->writeListEnd();
@@ -17297,15 +17297,15 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size850 = 0;
-            $_etype853 = 0;
-            $xfer += $input->readListBegin($_etype853, $_size850);
-            for ($_i854 = 0; $_i854 < $_size850; ++$_i854)
+            $_size857 = 0;
+            $_etype860 = 0;
+            $xfer += $input->readListBegin($_etype860, $_size857);
+            for ($_i861 = 0; $_i861 < $_size857; ++$_i861)
             {
-              $elem855 = null;
-              $elem855 = new \metastore\FieldSchema();
-              $xfer += $elem855->read($input);
-              $this->success []= $elem855;
+              $elem862 = null;
+              $elem862 = new \metastore\FieldSchema();
+              $xfer += $elem862->read($input);
+              $this->success []= $elem862;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17357,9 +17357,9 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter856)
+          foreach ($this->success as $iter863)
           {
-            $xfer += $iter856->write($output);
+            $xfer += $iter863->write($output);
           }
         }
         $output->writeListEnd();
@@ -18031,15 +18031,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->primaryKeys = array();
-            $_size857 = 0;
-            $_etype860 = 0;
-            $xfer += $input->readListBegin($_etype860, $_size857);
-            for ($_i861 = 0; $_i861 < $_size857; ++$_i861)
+            $_size864 = 0;
+            $_etype867 = 0;
+            $xfer += $input->readListBegin($_etype867, $_size864);
+            for ($_i868 = 0; $_i868 < $_size864; ++$_i868)
             {
-              $elem862 = null;
-              $elem862 = new \metastore\SQLPrimaryKey();
-              $xfer += $elem862->read($input);
-              $this->primaryKeys []= $elem862;
+              $elem869 = null;
+              $elem869 = new \metastore\SQLPrimaryKey();
+              $xfer += $elem869->read($input);
+              $this->primaryKeys []= $elem869;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18049,15 +18049,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->foreignKeys = array();
-            $_size863 = 0;
-            $_etype866 = 0;
-            $xfer += $input->readListBegin($_etype866, $_size863);
-            for ($_i867 = 0; $_i867 < $_size863; ++$_i867)
+            $_size870 = 0;
+            $_etype873 = 0;
+            $xfer += $input->readListBegin($_etype873, $_size870);
+            for ($_i874 = 0; $_i874 < $_size870; ++$_i874)
             {
-              $elem868 = null;
-              $elem868 = new \metastore\SQLForeignKey();
-              $xfer += $elem868->read($input);
-              $this->foreignKeys []= $elem868;
+              $elem875 = null;
+              $elem875 = new \metastore\SQLForeignKey();
+              $xfer += $elem875->read($input);
+              $this->foreignKeys []= $elem875;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18067,15 +18067,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 4:
           if ($ftype == TType::LST) {
             $this->uniqueConstraints = array();
-            $_size869 = 0;
-            $_etype872 = 0;
-            $xfer += $input->readListBegin($_etype872, $_size869);
-            for ($_i873 = 0; $_i873 < $_size869; ++$_i873)
+            $_size876 = 0;
+            $_etype879 = 0;
+            $xfer += $input->readListBegin($_etype879, $_size876);
+            for ($_i880 = 0; $_i880 < $_size876; ++$_i880)
             {
-              $elem874 = null;
-              $elem874 = new \metastore\SQLUniqueConstraint();
-              $xfer += $elem874->read($input);
-              $this->uniqueConstraints []= $elem874;
+              $elem881 = null;
+              $elem881 = new \metastore\SQLUniqueConstraint();
+              $xfer += $elem881->read($input);
+              $this->uniqueConstraints []= $elem881;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18085,15 +18085,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->notNullConstraints = array();
-            $_size875 = 0;
-            $_etype878 = 0;
-            $xfer += $input->readListBegin($_etype878, $_size875);
-            for ($_i879 = 0; $_i879 < $_size875; ++$_i879)
+            $_size882 = 0;
+            $_etype885 = 0;
+            $xfer += $input->readListBegin($_etype885, $_size882);
+            for ($_i886 = 0; $_i886 < $_size882; ++$_i886)
             {
-              $elem880 = null;
-              $elem880 = new \metastore\SQLNotNullConstraint();
-              $xfer += $elem880->read($input);
-              $this->notNullConstraints []= $elem880;
+              $elem887 = null;
+              $elem887 = new \metastore\SQLNotNullConstraint();
+              $xfer += $elem887->read($input);
+              $this->notNullConstraints []= $elem887;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18103,15 +18103,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->defaultConstraints = array();
-            $_size881 = 0;
-            $_etype884 = 0;
-            $xfer += $input->readListBegin($_etype884, $_size881);
-            for ($_i885 = 0; $_i885 < $_size881; ++$_i885)
+            $_size888 = 0;
+            $_etype891 = 0;
+            $xfer += $input->readListBegin($_etype891, $_size888);
+            for ($_i892 = 0; $_i892 < $_size888; ++$_i892)
             {
-              $elem886 = null;
-              $elem886 = new \metastore\SQLDefaultConstraint();
-              $xfer += $elem886->read($input);
-              $this->defaultConstraints []= $elem886;
+              $elem893 = null;
+              $elem893 = new \metastore\SQLDefaultConstraint();
+              $xfer += $elem893->read($input);
+              $this->defaultConstraints []= $elem893;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18121,15 +18121,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 7:
           if ($ftype == TType::LST) {
             $this->checkConstraints = array();
-            $_size887 = 0;
-            $_etype890 = 0;
-            $xfer += $input->readListBegin($_etype890, $_size887);
-            for ($_i891 = 0; $_i891 < $_size887; ++$_i891)
+            $_size894 = 0;
+            $_etype897 = 0;
+            $xfer += $input->readListBegin($_etype897, $_size894);
+            for ($_i898 = 0; $_i898 < $_size894; ++$_i898)
             {
-              $elem892 = null;
-              $elem892 = new \metastore\SQLCheckConstraint();
-              $xfer += $elem892->read($input);
-              $this->checkConstraints []= $elem892;
+              $elem899 = null;
+              $elem899 = new \metastore\SQLCheckConstraint();
+              $xfer += $elem899->read($input);
+              $this->checkConstraints []= $elem899;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18165,9 +18165,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->primaryKeys));
         {
-          foreach ($this->primaryKeys as $iter893)
+          foreach ($this->primaryKeys as $iter900)
           {
-            $xfer += $iter893->write($output);
+            $xfer += $iter900->write($output);
           }
         }
         $output->writeListEnd();
@@ -18182,9 +18182,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->foreignKeys));
         {
-          foreach ($this->foreignKeys as $iter894)
+          foreach ($this->foreignKeys as $iter901)
           {
-            $xfer += $iter894->write($output);
+            $xfer += $iter901->write($output);
           }
         }
         $output->writeListEnd();
@@ -18199,9 +18199,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->uniqueConstraints));
         {
-          foreach ($this->uniqueConstraints as $iter895)
+          foreach ($this->uniqueConstraints as $iter902)
           {
-            $xfer += $iter895->write($output);
+            $xfer += $iter902->write($output);
           }
         }
         $output->writeListEnd();
@@ -18216,9 +18216,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->notNullConstraints));
         {
-          foreach ($this->notNullConstraints as $iter896)
+          foreach ($this->notNullConstraints as $iter903)
           {
-            $xfer += $iter896->write($output);
+            $xfer += $iter903->write($output);
           }
         }
         $output->writeListEnd();
@@ -18233,9 +18233,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->defaultConstraints));
         {
-          foreach ($this->defaultConstraints as $iter897)
+          foreach ($this->defaultConstraints as $iter904)
           {
-            $xfer += $iter897->write($output);
+            $xfer += $iter904->write($output);
           }
         }
         $output->writeListEnd();
@@ -18250,9 +18250,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->checkConstraints));
         {
-          foreach ($this->checkConstraints as $iter898)
+          foreach ($this->checkConstraints as $iter905)
           {
-            $xfer += $iter898->write($output);
+            $xfer += $iter905->write($output);
           }
         }
         $output->writeListEnd();
@@ -20252,14 +20252,14 @@ class ThriftHiveMetastore_truncate_table_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->partNames = array();
-            $_size899 = 0;
-            $_etype902 = 0;
-            $xfer += $input->readListBegin($_etype902, $_size899);
-            for ($_i903 = 0; $_i903 < $_size899; ++$_i903)
+            $_size906 = 0;
+            $_etype909 = 0;
+            $xfer += $input->readListBegin($_etype909, $_size906);
+            for ($_i910 = 0; $_i910 < $_size906; ++$_i910)
             {
-              $elem904 = null;
-              $xfer += $input->readString($elem904);
-              $this->partNames []= $elem904;
+              $elem911 = null;
+              $xfer += $input->readString($elem911);
+              $this->partNames []= $elem911;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20297,9 +20297,9 @@ class ThriftHiveMetastore_truncate_table_args {
       {
         $output->writeListBegin(TType::STRING, count($this->partNames));
         {
-          foreach ($this->partNames as $iter905)
+          foreach ($this->partNames as $iter912)
           {
-            $xfer += $output->writeString($iter905);
+            $xfer += $output->writeString($iter912);
           }
         }
         $output->writeListEnd();
@@ -20550,14 +20550,14 @@ class ThriftHiveMetastore_get_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size906 = 0;
-            $_etype909 = 0;
-            $xfer += $input->readListBegin($_etype909, $_size906);
-            for ($_i910 = 0; $_i910 < $_size906; ++$_i910)
+            $_size913 = 0;
+            $_etype916 = 0;
+            $xfer += $input->readListBegin($_etype916, $_size913);
+            for ($_i917 = 0; $_i917 < $_size913; ++$_i917)
             {
-              $elem911 = null;
-              $xfer += $input->readString($elem911);
-              $this->success []= $elem911;
+              $elem918 = null;
+              $xfer += $input->readString($elem918);
+              $this->success []= $elem918;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20593,9 +20593,9 @@ class ThriftHiveMetastore_get_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter912)
+          foreach ($this->success as $iter919)
           {
-            $xfer += $output->writeString($iter912);
+            $xfer += $output->writeString($iter919);
           }
         }
         $output->writeListEnd();
@@ -20797,14 +20797,14 @@ class ThriftHiveMetastore_get_tables_by_type_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size913 = 0;
-            $_etype916 = 0;
-            $xfer += $input->readListBegin($_etype916, $_size913);
-            for ($_i917 = 0; $_i917 < $_size913; ++$_i917)
+            $_size920 = 0;
+            $_etype923 = 0;
+            $xfer += $input->readListBegin($_etype923, $_size920);
+            for ($_i924 = 0; $_i924 < $_size920; ++$_i924)
             {
-              $elem918 = null;
-              $xfer += $input->readString($elem918);
-              $this->success []= $elem918;
+              $elem925 = null;
+              $xfer += $input->readString($elem925);
+              $this->success []= $elem925;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20840,9 +20840,9 @@ class ThriftHiveMetastore_get_tables_by_type_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter919)
+          foreach ($this->success as $iter926)
           {
-            $xfer += $output->writeString($iter919);
+            $xfer += $output->writeString($iter926);
           }
         }
         $output->writeListEnd();
@@ -20998,14 +20998,14 @@ class ThriftHiveMetastore_get_materialized_views_for_rewriting_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size920 = 0;
-            $_etype923 = 0;
-            $xfer += $input->readListBegin($_etype923, $_size920);
-            for ($_i924 = 0; $_i924 < $_size920; ++$_i924)
+            $_size927 = 0;
+            $_etype930 = 0;
+            $xfer += $input->readListBegin($_etype930, $_size927);
+            for ($_i931 = 0; $_i931 < $_size927; ++$_i931)
             {
-              $elem925 = null;
-              $xfer += $input->readString($elem925);
-              $this->success []= $elem925;
+              $elem932 = null;
+              $xfer += $input->readString($elem932);
+              $this->success []= $elem932;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21041,9 +21041,9 @@ class ThriftHiveMetastore_get_materialized_views_for_rewriting_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter926)
+          foreach ($this->success as $iter933)
           {
-            $xfer += $output->writeString($iter926);
+            $xfer += $output->writeString($iter933);
           }
         }
         $output->writeListEnd();
@@ -21148,14 +21148,14 @@ class ThriftHiveMetastore_get_table_meta_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->tbl_types = array();
-            $_size927 = 0;
-            $_etype930 = 0;
-            $xfer += $input->readListBegin($_etype930, $_size927);
-            for ($_i931 = 0; $_i931 < $_size927; ++$_i931)
+            $_size934 = 0;
+            $_etype937 = 0;
+            $xfer += $input->readListBegin($_etype937, $_size934);
+            for ($_i938 = 0; $_i938 < $_size934; ++$_i938)
             {
-              $elem932 = null;
-              $xfer += $input->readString($elem932);
-              $this->tbl_types []= $elem932;
+              $elem939 = null;
+              $xfer += $input->readString($elem939);
+              $this->tbl_types []= $elem939;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21193,9 +21193,9 @@ class ThriftHiveMetastore_get_table_meta_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_types));
         {
-          foreach ($this->tbl_types as $iter933)
+          foreach ($this->tbl_types as $iter940)
           {
-            $xfer += $output->writeString($iter933);
+            $xfer += $output->writeString($iter940);
           }
         }
         $output->writeListEnd();
@@ -21272,15 +21272,15 @@ class ThriftHiveMetastore_get_table_meta_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size934 = 0;
-            $_etype937 = 0;
-            $xfer += $input->readListBegin($_etype937, $_size934);
-            for ($_i938 = 0; $_i938 < $_size934; ++$_i938)
+            $_size941 = 0;
+            $_etype944 = 0;
+            $xfer += $input->readListBegin($_etype944, $_size941);
+            for ($_i945 = 0; $_i945 < $_size941; ++$_i945)
             {
-              $elem939 = null;
-              $elem939 = new \metastore\TableMeta();
-              $xfer += $elem939->read($input);
-              $this->success []= $elem939;
+              $elem946 = null;
+              $elem946 = new \metastore\TableMeta();
+              $xfer += $elem946->read($input);
+              $this->success []= $elem946;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21316,9 +21316,9 @@ class ThriftHiveMetastore_get_table_meta_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter940)
+          foreach ($this->success as $iter947)
           {
-            $xfer += $iter940->write($output);
+            $xfer += $iter947->write($output);
           }
         }
         $output->writeListEnd();
@@ -21474,14 +21474,14 @@ class ThriftHiveMetastore_get_all_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size941 = 0;
-            $_etype944 = 0;
-            $xfer += $input->readListBegin($_etype944, $_size941);
-            for ($_i945 = 0; $_i945 < $_size941; ++$_i945)
+            $_size948 = 0;
+            $_etype951 = 0;
+            $xfer += $input->readListBegin($_etype951, $_size948);
+            for ($_i952 = 0; $_i952 < $_size948; ++$_i952)
             {
-              $elem946 = null;
-              $xfer += $input->readString($elem946);
-              $this->success []= $elem946;
+              $elem953 = null;
+              $xfer += $input->readString($elem953);
+              $this->success []= $elem953;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21517,9 +21517,9 @@ class ThriftHiveMetastore_get_all_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter947)
+          foreach ($this->success as $iter954)
           {
-            $xfer += $output->writeString($iter947);
+            $xfer += $output->writeString($iter954);
           }
         }
         $output->writeListEnd();
@@ -21834,14 +21834,14 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size948 = 0;
-            $_etype951 = 0;
-            $xfer += $input->readListBegin($_etype951, $_size948);
-            for ($_i952 = 0; $_i952 < $_size948; ++$_i952)
+            $_size955 = 0;
+            $_etype958 = 0;
+            $xfer += $input->readListBegin($_etype958, $_size955);
+            for ($_i959 = 0; $_i959 < $_size955; ++$_i959)
             {
-              $elem953 = null;
-              $xfer += $input->readString($elem953);
-              $this->tbl_names []= $elem953;
+              $elem960 = null;
+              $xfer += $input->readString($elem960);
+              $this->tbl_names []= $elem960;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21874,9 +21874,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter954)
+          foreach ($this->tbl_names as $iter961)
           {
-            $xfer += $output->writeString($iter954);
+            $xfer += $output->writeString($iter961);
           }
         }
         $output->writeListEnd();
@@ -21941,15 +21941,15 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size955 = 0;
-            $_etype958 = 0;
-            $xfer += $input->readListBegin($_etype958, $_size955);
-            for ($_i959 = 0; $_i959 < $_size955; ++$_i959)
+            $_size962 = 0;
+            $_etype965 = 0;
+            $xfer += $input->readListBegin($_etype965, $_size962);
+            for ($_i966 = 0; $_i966 < $_size962; ++$_i966)
             {
-              $elem960 = null;
-              $elem960 = new \metastore\Table();
-              $xfer += $elem960->read($input);
-              $this->success []= $elem960;
+              $elem967 = null;
+              $elem967 = new \metastore\Table();
+              $xfer += $elem967->read($input);
+              $this->success []= $elem967;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21977,9 +21977,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter961)
+          foreach ($this->success as $iter968)
           {
-            $xfer += $iter961->write($output);
+            $xfer += $iter968->write($output);
           }
         }
         $output->writeListEnd();
@@ -22506,14 +22506,14 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size962 = 0;
-            $_etype965 = 0;
-            $xfer += $input->readListBegin($_etype965, $_size962);
-            for ($_i966 = 0; $_i966 < $_size962; ++$_i966)
+            $_size969 = 0;
+            $_etype972 = 0;
+            $xfer += $input->readListBegin($_etype972, $_size969);
+            for ($_i973 = 0; $_i973 < $_size969; ++$_i973)
             {
-              $elem967 = null;
-              $xfer += $input->readString($elem967);
-              $this->tbl_names []= $elem967;
+              $elem974 = null;
+              $xfer += $input->readString($elem974);
+              $this->tbl_names []= $elem974;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22546,9 +22546,9 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter968)
+          foreach ($this->tbl_names as $iter975)
           {
-            $xfer += $output->writeString($iter968);
+            $xfer += $output->writeString($iter975);
           }
         }
         $output->writeListEnd();
@@ -22653,18 +22653,18 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size969 = 0;
-            $_ktype970 = 0;
-            $_vtype971 = 0;
-            $xfer += $input->readMapBegin($_ktype970, $_vtype971, $_size969);
-            for ($_i973 = 0; $_i973 < $_size969; ++$_i973)
+            $_size976 = 0;
+            $_ktype977 = 0;
+            $_vtype978 = 0;
+            $xfer += $input->readMapBegin($_ktype977, $_vtype978, $_size976);
+            for ($_i980 = 0; $_i980 < $_size976; ++$_i980)
             {
-              $key974 = '';
-              $val975 = new \metastore\Materialization();
-              $xfer += $input->readString($key974);
-              $val975 = new \metastore\Materialization();
-              $xfer += $val975->read($input);
-              $this->success[$key974] = $val975;
+              $key981 = '';
+              $val982 = new \metastore\Materialization();
+              $xfer += $input->readString($key981);
+              $val982 = new \metastore\Materialization();
+              $xfer += $val982->read($input);
+              $this->success[$key981] = $val982;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -22716,10 +22716,10 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $kiter976 => $viter977)
+          foreach ($this->success as $kiter983 => $viter984)
           {
-            $xfer += $output->writeString($kiter976);
-            $xfer += $viter977->write($output);
+            $xfer += $output->writeString($kiter983);
+            $xfer += $viter984->write($output);
           }
         }
         $output->writeMapEnd();
@@ -23231,14 +23231,14 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size978 = 0;
-            $_etype981 = 0;
-            $xfer += $input->readListBegin($_etype981, $_size978);
-            for ($_i982 = 0; $_i982 < $_size978; ++$_i982)
+            $_size985 = 0;
+            $_etype988 = 0;
+            $xfer += $input->readListBegin($_etype988, $_size985);
+            for ($_i989 = 0; $_i989 < $_size985; ++$_i989)
             {
-              $elem983 = null;
-              $xfer += $input->readString($elem983);
-              $this->success []= $elem983;
+              $elem990 = null;
+              $xfer += $input->readString($elem990);
+              $this->success []= $elem990;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23290,9 +23290,9 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter984)
+          foreach ($this->success as $iter991)
           {
-            $xfer += $output->writeString($iter984);
+            $xfer += $output->writeString($iter991);
           }
         }
         $output->writeListEnd();
@@ -24605,15 +24605,15 @@ class ThriftHiveMetastore_add_partitions_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size985 = 0;
-            $_etype988 = 0;
-            $xfer += $input->readListBegin($_etype988, $_size985);
-            for ($_i989 = 0; $_i989 < $_size985; ++$_i989)
+            $_size992 = 0;
+            $_etype995 = 0;
+            $xfer += $input->readListBegin($_etype995, $_size992);
+            for ($_i996 = 0; $_i996 < $_size992; ++$_i996)
             {
-              $elem990 = null;
-              $elem990 = new \metastore\Partition();
-              $xfer += $elem990->read($input);
-              $this->new_parts []= $elem990;
+              $elem997 = null;
+              $elem997 = new \metastore\Partition();
+              $xfer += $elem997->read($input);
+              $this->new_parts []= $elem997;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24641,9 +24641,9 @@ class ThriftHiveMetastore_add_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter991)
+          foreach ($this->new_parts as $iter998)
           {
-            $xfer += $iter991->write($output);
+            $xfer += $iter998->write($output);
           }
         }
         $output->writeListEnd();
@@ -24858,15 +24858,15 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size992 = 0;
-            $_etype995 = 0;
-            $xfer += $input->readListBegin($_etype995, $_size992);
-            for ($_i996 = 0; $_i996 < $_size992; ++$_i996)
+            $_size999 = 0;
+            $_etype1002 = 0;
+            $xfer += $input->readListBegin($_etype1002, $_size999);
+            for ($_i1003 = 0; $_i1003 < $_size999; ++$_i1003)
             {
-              $elem997 = null;
-              $elem997 = new \metastore\PartitionSpec();
-              $xfer += $elem997->read($input);
-              $this->new_parts []= $elem997;
+              $elem1004 = null;
+              $elem1004 = new \metastore\PartitionSpec();
+              $xfer += $elem1004->read($input);
+              $this->new_parts []= $elem1004;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24894,9 +24894,9 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter998)
+          foreach ($this->new_parts as $iter1005)
           {
-            $xfer += $iter998->write($output);
+            $xfer += $iter1005->write($output);
           }
         }
         $output->writeListEnd();
@@ -25146,14 +25146,14 @@ class ThriftHiveMetastore_append_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size999 = 0;
-            $_etype1002 = 0;
-            $xfer += $input->readListBegin($_etype1002, $_size999);
-            for ($_i1003 = 0; $_i1003 < $_size999; ++$_i1003)
+            $_size1006 = 0;
+            $_etype1009 = 0;
+            $xfer += $input->readListBegin($_etype1009, $_size1006);
+            for ($_i1010 = 0; $_i1010 < $_size1006; ++$_i1010)
             {
-              $elem1004 = null;
-              $xfer += $input->readString($elem1004);
-              $this->part_vals []= $elem1004;
+              $elem1011 = null;
+              $xfer += $input->readString($elem1011);
+              $this->part_vals []= $elem1011;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25191,9 +25191,9 @@ class ThriftHiveMetastore_append_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1005)
+          foreach ($this->part_vals as $iter1012)
           {
-            $xfer += $output->writeString($iter1005);
+            $xfer += $output->writeString($iter1012);
           }
         }
         $output->writeListEnd();
@@ -25695,14 +25695,14 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1006 = 0;
-            $_etype1009 = 0;
-            $xfer += $input->readListBegin($_etype1009, $_size1006);
-            for ($_i1010 = 0; $_i1010 < $_size1006; ++$_i1010)
+            $_size1013 = 0;
+            $_etype1016 = 0;
+            $xfer += $input->readListBegin($_etype1016, $_size1013);
+            for ($_i1017 = 0; $_i1017 < $_size1013; ++$_i1017)
             {
-              $elem1011 = null;
-              $xfer += $input->readString($elem1011);
-              $this->part_vals []= $elem1011;
+              $elem1018 = null;
+              $xfer += $input->readString($elem1018);
+              $this->part_vals []= $elem1018;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25748,9 +25748,9 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1012)
+          foreach ($this->part_vals as $iter1019)
           {
-            $xfer += $output->writeString($iter1012);
+            $xfer += $output->writeString($iter1019);
           }
         }
         $output->writeListEnd();
@@ -26604,14 +26604,14 @@ class ThriftHiveMetastore_drop_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1013 = 0;
-            $_etype1016 = 0;
-            $xfer += $input->readListBegin($_etype1016, $_size1013);
-            for ($_i1017 = 0; $_i1017 < $_size1013; ++$_i1017)
+            $_size1020 = 0;
+            $_etype1023 = 0;
+            $xfer += $input->readListBegin($_etype1023, $_size1020);
+            for ($_i1024 = 0; $_i1024 < $_size1020; ++$_i1024)
             {
-              $elem1018 = null;
-              $xfer += $input->readString($elem1018);
-              $this->part_vals []= $elem1018;
+              $elem1025 = null;
+              $xfer += $input->readString($elem1025);
+              $this->part_vals []= $elem1025;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26656,9 +26656,9 @@ class ThriftHiveMetastore_drop_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1019)
+          foreach ($this->part_vals as $iter1026)
           {
-            $xfer += $output->writeString($iter1019);
+            $xfer += $output->writeString($iter1026);
           }
         }
         $output->writeListEnd();
@@ -26911,14 +26911,14 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_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)
+            $_size1027 = 0;
+            $_etype1030 = 0;
+            $xfer += $input->readListBegin($_etype1030, $_size1027);
+            for ($_i1031 = 0; $_i1031 < $_size1027; ++$_i1031)
             {
-              $elem1025 = null;
-              $xfer += $input->readString($elem1025);
-              $this->part_vals []= $elem1025;
+              $elem1032 = null;
+              $xfer += $input->readString($elem1032);
+              $this->part_vals []= $elem1032;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26971,9 +26971,9 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1026)
+          foreach ($this->part_vals as $iter1033)
           {
-            $xfer += $output->writeString($iter1026);
+            $xfer += $output->writeString($iter1033);
           }
         }
         $output->writeListEnd();
@@ -27987,14 +27987,14 @@ class ThriftHiveMetastore_get_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1027 = 0;
-            $_etype1030 = 0;
-            $xfer += $input->readListBegin($_etype1030, $_size1027);
-            for ($_i1031 = 0; $_i1031 < $_size1027; ++$_i1031)
+            $_size1034 = 0;
+            $_etype1037 = 0;
+            $xfer += $input->readListBegin($_etype1037, $_size1034);
+            for ($_i1038 = 0; $_i1038 < $_size1034; ++$_i1038)
             {
-              $elem1032 = null;
-              $xfer += $input->readString($elem1032);
-              $this->part_vals []= $elem1032;
+              $elem1039 = null;
+              $xfer += $input->readString($elem1039);
+              $this->part_vals []= $elem1039;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28032,9 +28032,9 @@ class ThriftHiveMetastore_get_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1033)
+          foreach ($this->part_vals as $iter1040)
           {
-            $xfer += $output->writeString($iter1033);
+            $xfer += $output->writeString($iter1040);
           }
         }
         $output->writeListEnd();
@@ -28276,17 +28276,17 @@ class ThriftHiveMetastore_exchange_partition_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size1034 = 0;
-            $_ktype1035 = 0;
-            $_vtype1036 = 0;
-            $xfer += $input->readMapBegin($_ktype1035, $_vtype1036, $_size1034);
-            for ($_i1038 = 0; $_i1038 < $_size1034; ++$_i1038)
+            $_size1041 = 0;
+            $_ktype1042 = 0;
+            $_vtype1043 = 0;
+            $xfer += $input->readMapBegin($_ktype1042, $_vtype1043, $_size1041);
+            for ($_i1045 = 0; $_i1045 < $_size1041; ++$_i1045)
             {
-              $key1039 = '';
-              $val1040 = '';
-              $xfer += $input->readString($key1039);
-              $xfer += $input->readString($val1040);
-              $this->partitionSpecs[$key1039] = $val1040;
+              $key1046 = '';
+              $val1047 = '';
+              $xfer += $input->readString($key1046);
+              $xfer += $input->readString($val1047);
+              $this->partitionSpecs[$key1046] = $val1047;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -28342,10 +28342,10 @@ class ThriftHiveMetastore_exchange_partition_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter1041 => $viter1042)
+          foreach ($this->partitionSpecs as $kiter1048 => $viter1049)
           {
-            $xfer += $output->writeString($kiter1041);
-            $xfer += $output->writeString($viter1042);
+            $xfer += $output->writeString($kiter1048);
+            $xfer += $output->writeString($viter1049);
           }
         }
         $output->writeMapEnd();
@@ -28657,17 +28657,17 @@ class ThriftHiveMetastore_exchange_partitions_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size1043 = 0;
-            $_ktype1044 = 0;
-            $_vtype1045 = 0;
-            $xfer += $input->readMapBegin($_ktype1044, $_vtype1045, $_size1043);
-            for ($_i1047 = 0; $_i1047 < $_size1043; ++$_i1047)
+            $_size1050 = 0;
+            $_ktype1051 = 0;
+            $_vtype1052 = 0;
+            $xfer += $input->readMapBegin($_ktype1051, $_vtype1052, $_size1050);
+            for ($_i1054 = 0; $_i1054 < $_size1050; ++$_i1054)
             {
-              $key1048 = '';
-              $val1049 = '';
-              $xfer += $input->readString($key1048);
-              $xfer += $input->readString($val1049);
-              $this->partitionSpecs[$key1048] = $val1049;
+              $key1055 = '';
+              $val1056 = '';
+              $xfer += $input->readString($key1055);
+              $xfer += $input->readString($val1056);
+              $this->partitionSpecs[$key1055] = $val1056;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -28723,10 +28723,10 @@ class ThriftHiveMetastore_exchange_partitions_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter1050 => $viter1051)
+          foreach ($this->partitionSpecs as $kiter1057 => $viter1058)
           {
-            $xfer += $output->writeString($kiter1050);
-            $xfer += $output->writeString($viter1051);
+            $xfer += $output->writeString($kiter1057);
+            $xfer += $output->writeString($viter1058);
           }
         }
         $output->writeMapEnd();
@@ -28859,15 +28859,15 @@ class ThriftHiveMetastore_exchange_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1052 = 0;
-            $_etype1055 = 0;
-            $xfer += $input->readListBegin($_etype1055, $_size1052);
-            for ($_i1056 = 0; $_i1056 < $_size1052; ++$_i1056)
+            $_size1059 = 0;
+            $_etype1062 = 0;
+            $xfer += $input->readListBegin($_etype1062, $_size1059);
+            for ($_i1063 = 0; $_i1063 < $_size1059; ++$_i1063)
             {
-              $elem1057 = null;
-              $elem1057 = new \metastore\Partition();
-              $xfer += $elem1057->read($input);
-              $this->success []= $elem1057;
+              $elem1064 = null;
+              $elem1064 = new \metastore\Partition();
+              $xfer += $elem1064->read($input);
+              $this->success []= $elem1064;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28927,9 +28927,9 @@ class ThriftHiveMetastore_exchange_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1058)
+          foreach ($this->success as $iter1065)
           {
-            $xfer += $iter1058->write($output);
+            $xfer += $iter1065->write($output);
           }
         }
         $output->writeListEnd();
@@ -29075,14 +29075,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1059 = 0;
-            $_etype1062 = 0;
-            $xfer += $input->readListBegin($_etype1062, $_size1059);
-            for ($_i1063 = 0; $_i1063 < $_size1059; ++$_i1063)
+            $_size1066 = 0;
+            $_etype1069 = 0;
+            $xfer += $input->readListBegin($_etype1069, $_size1066);
+            for ($_i1070 = 0; $_i1070 < $_size1066; ++$_i1070)
             {
-              $elem1064 = null;
-              $xfer += $input->readString($elem1064);
-              $this->part_vals []= $elem1064;
+              $elem1071 = null;
+              $xfer += $input->readString($elem1071);
+              $this->part_vals []= $elem1071;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29099,14 +29099,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1065 = 0;
-            $_etype1068 = 0;
-            $xfer += $input->readListBegin($_etype1068, $_size1065);
-            for ($_i1069 = 0; $_i1069 < $_size1065; ++$_i1069)
+            $_size1072 = 0;
+            $_etype1075 = 0;
+            $xfer += $input->readListBegin($_etype1075, $_size1072);
+            for ($_i1076 = 0; $_i1076 < $_size1072; ++$_i1076)
             {
-              $elem1070 = null;
-              $xfer += $input->readString($elem1070);
-              $this->group_names []= $elem1070;
+              $elem1077 = null;
+              $xfer += $input->readString($elem1077);
+              $this->group_names []= $elem1077;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29144,9 +29144,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1071)
+          foreach ($this->part_vals as $iter1078)
           {
-            $xfer += $output->writeString($iter1071);
+            $xfer += $output->writeString($iter1078);
           }
         }
         $output->writeListEnd();
@@ -29166,9 +29166,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1072)
+          foreach ($this->group_names as $iter1079)
           {
-            $xfer += $output->writeString($iter1072);
+            $xfer += $output->writeString($iter1079);
           }
         }
         $output->writeListEnd();
@@ -29759,15 +29759,15 @@ class ThriftHiveMetastore_get_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1073 = 0;
-            $_etype1076 = 0;
-            $xfer += $input->readListBegin($_etype1076, $_size1073);
-            for ($_i1077 = 0; $_i1077 < $_size1073; ++$_i1077)
+            $_size1080 = 0;
+            $_etype1083 = 0;
+            $xfer += $input->readListBegin($_etype1083, $_size1080);
+            for ($_i1084 = 0; $_i1084 < $_size1080; ++$_i1084)
             {
-              $elem1078 = null;
-              $elem1078 = new \metastore\Partition();
-              $xfer += $elem1078->read($input);
-              $this->success []= $elem1078;
+              $elem1085 = null;
+              $elem1085 = new \metastore\Partition();
+              $xfer += $elem1085->read($input);
+              $this->success []= $elem1085;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29811,9 +29811,9 @@ class ThriftHiveMetastore_get_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1079)
+          foreach ($this->success as $iter1086)
           {
-            $xfer += $iter1079->write($output);
+            $xfer += $iter1086->write($output);
           }
         }
         $output->writeListEnd();
@@ -29959,14 +29959,14 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1080 = 0;
-            $_etype1083 = 0;
-            $xfer += $input->readListBegin($_etype1083, $_size1080);
-            for ($_i1084 = 0; $_i1084 < $_size1080; ++$_i1084)
+            $_size1087 = 0;
+            $_etype1090 = 0;
+            $xfer += $input->readListBegin($_etype1090, $_size1087);
+            for ($_i1091 = 0; $_i1091 < $_size1087; ++$_i1091)
             {
-              $elem1085 = null;
-              $xfer += $input->readString($elem1085);
-              $this->group_names []= $elem1085;
+              $elem1092 = null;
+              $xfer += $input->readString($elem1092);
+              $this->group_names []= $elem1092;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30014,9 +30014,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1086)
+          foreach ($this->group_names as $iter1093)
           {
-            $xfer += $output->writeString($iter1086);
+            $xfer += $output->writeString($iter1093);
           }
         }
         $output->writeListEnd();
@@ -30105,15 +30105,15 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1087 = 0;
-            $_etype1090 = 0;
-            $xfer += $input->readListBegin($_etype1090, $_size1087);
-            for ($_i1091 = 0; $_i1091 < $_size1087; ++$_i1091)
+            $_size1094 = 0;
+            $_etype1097 = 0;
+            $xfer += $input->readListBegin($_etype1097, $_size1094);
+            for ($_i1098 = 0; $_i1098 < $_size1094; ++$_i1098)
             {
-              $elem1092 = null;
-              $elem1092 = new \metastore\Partition();
-              $xfer += $elem1092->read($input);
-              $this->success []= $elem1092;
+              $elem1099 = null;
+              $elem1099 = new \metastore\Partition();
+              $xfer += $elem1099->read($input);
+              $this->success []= $elem1099;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30157,9 +30157,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1093)
+          foreach ($this->success as $iter1100)
           {
-            $xfer += $iter1093->write($output);
+            $xfer += $iter1100->write($output);
           }
         }
         $output->writeListEnd();
@@ -30379,15 +30379,15 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1094 = 0;
-            $_etype1097 = 0;
-            $xfer += $input->readListBegin($_etype1097, $_size1094);
-            for ($_i1098 = 0; $_i1098 < $_size1094; ++$_i1098)
+            $_size1101 = 0;
+            $_etype1104 = 0;
+            $xfer += $input->readListBegin($_etype1104, $_size1101);
+            for ($_i1105 = 0; $_i1105 < $_size1101; ++$_i1105)
             {
-              $elem1099 = null;
-              $elem1099 = new \metastore\PartitionSpec();
-              $xfer += $elem1099->read($input);
-              $this->success []= $elem1099;
+              $elem1106 = null;
+              $elem1106 = new \metastore\PartitionSpec();
+              $xfer += $elem1106->read($input);
+              $this->success []= $elem1106;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30431,9 +30431,9 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1100)
+          foreach ($this->success as $iter1107)
           {
-            $xfer += $iter1100->write($output);
+            $xfer += $iter1107->write($output);
           }
         }
         $output->writeListEnd();
@@ -30652,14 +30652,14 @@ class ThriftHiveMetastore_get_partition_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1101 = 0;
-            $_etype1104 = 0;
-            $xfer += $input->readListBegin($_etype1104, $_size1101);
-            for ($_i1105 = 0; $_i1105 < $_size1101; ++$_i1105)
+            $_size1108 = 0;
+            $_etype1111 = 0;
+            $xfer += $input->readListBegin($_etype1111, $_size1108);
+            for ($_i1112 = 0; $_i1112 < $_size1108; ++$_i1112)
             {
-              $elem1106 = null;
-              $xfer += $input->readString($elem1106);
-              $this->success []= $elem1106;
+              $elem1113 = null;
+              $xfer += $input->readString($elem1113);
+              $this->success []= $elem1113;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30703,9 +30703,9 @@ class ThriftHiveMetastore_get_partition_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1107)
+          foreach ($this->success as $iter1114)
           {
-            $xfer += $output->writeString($iter1107);
+            $xfer += $output->writeString($iter1114);
           }
         }
         $output->writeListEnd();
@@ -31036,14 +31036,14 @@ class ThriftHiveMetastore_get_partitions_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1108 = 0;
-            $_etype1111 = 0;
-            $xfer += $input->readListBegin($_etype1111, $_size1108);
-            for ($_i1112 = 0; $_i1112 < $_size1108; ++$_i1112)
+            $_size1115 = 0;
+            $_etype1118 = 0;
+            $xfer += $input->readListBegin($_etype1118, $_size1115);
+            for ($_i1119 = 0; $_i1119 < $_size1115; ++$_i1119)
             {
-              $elem1113 = null;
-              $xfer += $input->readString($elem1113);
-              $this->part_vals []= $elem1113;
+              $elem1120 = null;
+              $xfer += $input->readString($elem1120);
+              $this->part_vals []= $elem1120;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31088,9 +31088,9 @@ class ThriftHiveMetastore_get_partitions_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1114)
+          foreach ($this->part_vals as $iter1121)
           {
-            $xfer += $output->writeString($iter1114);
+            $xfer += $output->writeString($iter1121);
           }
         }
         $output->writeListEnd();
@@ -31184,15 +31184,15 @@ class ThriftHiveMetastore_get_partitions_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1115 = 0;
-            $_etype1118 = 0;
-            $xfer += $input->readListBegin($_etype1118, $_size1115);
-            for ($_i1119 = 0; $_i1119 < $_size1115; ++$_i1119)
+            $_size1122 = 0;
+            $_etype1125 = 0;
+            $xfer += $input->readListBegin($_etype1125, $_size1122);
+            for ($_i1126 = 0; $_i1126 < $_size1122; ++$_i1126)
             {
-              $elem1120 = null;
-              $elem1120 = new \metastore\Partition();
-              $xfer += $elem1120->read($input);
-              $this->success []= $elem1120;
+              $elem1127 = null;
+              $elem1127 = new \metastore\Partition();
+              $xfer += $elem1127->read($input);
+              $this->success []= $elem1127;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31236,9 +31236,9 @@ class ThriftHiveMetastore_get_partitions_ps_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1121)
+          foreach ($this->success as $iter1128)
           {
-            $xfer += $iter1121->write($output);
+            $xfer += $iter1128->write($output);
           }
         }
         $output->writeListEnd();
@@ -31385,14 +31385,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1122 = 0;
-            $_etype1125 = 0;
-            $xfer += $input->readListBegin($_etype1125, $_size1122);
-            for ($_i1126 = 0; $_i1126 < $_size1122; ++$_i1126)
+            $_size1129 = 0;
+            $_etype1132 = 0;
+            $xfer += $input->readListBegin($_etype1132, $_size1129);
+            for ($_i1133 = 0; $_i1133 < $_size1129; ++$_i1133)
             {
-              $elem1127 = null;
-              $xfer += $input->readString($elem1127);
-              $this->part_vals []= $elem1127;
+              $elem1134 = null;
+              $xfer += $input->readString($elem1134);
+              $this->part_vals []= $elem1134;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31416,14 +31416,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1128 = 0;
-            $_etype1131 = 0;
-            $xfer += $input->readListBegin($_etype1131, $_size1128);
-            for ($_i1132 = 0; $_i1132 < $_size1128; ++$_i1132)
+            $_size1135 = 0;
+            $_etype1138 = 0;
+            $xfer += $input->readListBegin($_etype1138, $_size1135);
+            for ($_i1139 = 0; $_i1139 < $_size1135; ++$_i1139)
             {
-              $elem1133 = null;
-              $xfer += $input->readString($elem1133);
-              $this->group_names []= $elem1133;
+              $elem1140 = null;
+              $xfer += $input->readString($elem1140);
+              $this->group_names []= $elem1140;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31461,9 +31461,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1134)
+          foreach ($this->part_vals as $iter1141)
           {
-            $xfer += $output->writeString($iter1134);
+            $xfer += $output->writeString($iter1141);
           }
         }
         $output->writeListEnd();
@@ -31488,9 +31488,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1135)
+          foreach ($this->group_names as $iter1142)
           {
-            $xfer += $output->writeString($iter1135);
+            $xfer += $output->writeString($iter1142);
           }
         }
         $output->writeListEnd();
@@ -31579,15 +31579,15 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1136 = 0;
-            $_etype1139 = 0;
-            $xfer += $input->readListBegin($_etype1139, $_size1136);
-            for ($_i1140 = 0; $_i1140 < $_size1136; ++$_i1140)
+            $_size1143 = 0;
+            $_etype1146 = 0;
+            $xfer += $input->readListBegin($_etype1146, $_size1143);
+            for ($_i1147 = 0; $_i1147 < $_size1143; ++$_i1147)
             {
-              $elem1141 = null;
-              $elem1141 = new \metastore\Partition();
-              $xfer += $elem1141->read($input);
-              $this->success []= $elem1141;
+              $elem1148 = null;
+              $elem1148 = new \metastore\Partition();
+              $xfer += $elem1148->read($input);
+              $this->success []= $elem1148;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31631,9 +31631,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1142)
+          foreach ($this->success as $iter1149)
           {
-            $xfer += $iter1142->write($output);
+            $xfer += $iter1149->write($output);
           }
         }
         $output->writeListEnd();
@@ -31754,14 +31754,14 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1143 = 0;
-            $_etype1146 = 0;
-            $xfer += $input->readListBegin($_etype1146, $_size1143);
-            for ($_i1147 = 0; $_i1147 < $_size1143; ++$_i1147)
+            $_size1150 = 0;
+            $_etype1153 = 0;
+            $xfer += $input->readListBegin($_etype1153, $_size1150);
+            for ($_i1154 = 0; $_i1154 < $_size1150; ++$_i1154)
             {
-              $elem1148 = null;
-              $xfer += $input->readString($elem1148);
-              $this->part_vals []= $elem1148;
+              $elem1155 = null;
+              $xfer += $input->readString($elem1155);
+              $this->part_vals []= $elem1155;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31806,9 +31806,9 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1149)
+          foreach ($this->part_vals as $iter1156)
           {
-            $xfer += $output->writeString($iter1149);
+            $xfer += $output->writeString($iter1156);
           }
         }
         $output->writeListEnd();
@@ -31901,14 +31901,14 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1150 = 0;
-            $_etype1153 = 0;
-            $xfer += $input->readListBegin($_etype1153, $_size1150);
-            for ($_i1154 = 0; $_i1154 < $_size1150; ++$_i1154)
+            $_size1157 = 0;
+            $_etype1160 = 0;
+            $xfer += $input->readListBegin($_etype1160, $_size1157);
+            for ($_i1161 = 0; $_i1161 < $_size1157; ++$_i1161)
             {
-              $elem1155 = null;
-              $xfer += $input->readString($elem1155);
-              $this->success []= $elem1155;
+              $elem1162 = null;
+              $xfer += $input->readString($elem1162);
+              $this->success []= $elem1162;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31952,9 +31952,9 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1156)
+          foreach ($this->success as $iter1163)
           {
-            $xfer += $output->writeString($iter1156);
+            $xfer += $output->writeString($iter1163);
           }
         }
         $output->writeListEnd();
@@ -32197,15 +32197,15 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1157 = 0;
-            $_etype1160 = 0;
-            $xfer += $input->readListBegin($_etype1160, $_size1157);
-            for ($_i1161 = 0; $_i1161 < $_size1157; ++$_i1161)
+            $_size1164 = 0;
+            $_etype1167 = 0;
+            $xfer += $input->readListBegin($_etype1167, $_size1164);
+            for ($_i1168 = 0; $_i1168 < $_size1164; ++$_i1168)
             {
-              $elem1162 = null;
-              $elem1162 = new \metastore\Partition();
-              $xfer += $elem1162->read($input);
-              $this->success []= $elem1162;
+              $elem1169 = null;
+              $elem1169 = new \metastore\Partition();
+              $xfer += $elem1169->read($input);
+              $this->success []= $elem1169;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32249,9 +32249,9 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1163)
+          foreach ($this->success as $iter1170)
           {
-            $xfer += $iter1163->write($output);
+            $xfer += $iter1170->write($output);
           }
         }
         $output->writeListEnd();
@@ -32494,15 +32494,15 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1164 = 0;
-            $_etype1167 = 0;
-            $xfer += $input->readListBegin($_etype1167, $_size1164);
-            for ($_i1168 = 0; $_i1168 < $_size1164; ++$_i1168)
+            $_size1171 = 0;
+            $_etype1174 = 0;
+            $xfer += $input->readListBegin($_etype1174, $_size1171);
+            for ($_i1175 = 0; $_i1175 < $_size1171; ++$_i1175)
             {
-              $elem1169 = null;
-              $elem1169 = new \metastore\PartitionSpec();
-              $xfer += $elem1169->read($input);
-              $this->success []= $elem1169;
+              $elem1176 = null;
+              $elem1176 = new \metastore\PartitionSpec();
+              $xfer += $elem1176->read($input);
+              $this->success []= $elem1176;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32546,9 +32546,9 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1170)
+          foreach ($this->success as $iter1177)
           {
-            $xfer += $iter1170->write($output);
+            $xfer += $iter1177->write($output);
           }
         }
         $output->writeListEnd();
@@ -33114,14 +33114,14 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->names = array();
-            $_size1171 = 0;
-            $_etype1174 = 0;
-            $xfer += $input->readListBegin($_etype1174, $_size1171);
-            for ($_i1175 = 0; $_i1175 < $_size1171; ++$_i1175)
+            $_size1178 = 0;
+            $_etype1181 = 0;
+            $xfer += $input->readListBegin($_etype1181, $_size1178);
+            for ($_i1182 = 0; $_i1182 < $_size1178; ++$_i1182)
             {
-              $elem1176 = null;
-              $xfer += $input->readString($elem1176);
-              $this->names []= $elem1176;
+              $elem1183 = null;
+              $xfer += $input->readString($elem1183);
+              $this->names []= $elem1183;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33159,9 +33159,9 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
       {
         $output->writeListBegin(TType::STRING, count($this->names));
         {
-          foreach ($this->names as $iter1177)
+          foreach ($this->names as $iter1184)
           {
-            $xfer += $output->writeString($iter1177);
+            $xfer += $output->writeString($iter1184);
           }
         }
         $output->writeListEnd();
@@ -33250,15 +33250,15 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1178 = 0;
-            $_etype1181 = 0;
-            $xfer += $input->readListBegin($_etype1181, $_size1178);
-            for ($_i1182 = 0; $_i1182 < $_size1178; ++$_i1182)
+            $_size1185 = 0;
+            $_etype1188 = 0;
+            $xfer += $input->readListBegin($_etype1188, $_size1185);
+            for ($_i1189 = 0; $_i1189 < $_size1185; ++$_i1189)
             {
-              $elem1183 = null;
-              $elem1183 = new \metastore\Partition();
-              $xfer += $elem1183->read($input);
-              $this->success []= $elem1183;
+              $elem1190 = null;
+              $elem1190 = new \metastore\Partition();
+              $xfer += $elem1190->read($input);
+              $this->success []= $elem1190;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33302,9 +33302,9 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1184)
+          foreach ($this->success as $iter1191)
           {
-            $xfer += $iter1184->write($output);
+            $xfer += $iter1191->write($output);
           }
         }
         $output->writeListEnd();
@@ -33643,15 +33643,15 @@ class ThriftHiveMetastore_alter_partitions_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size1185 = 0;
-            $_etype1188 = 0;
-            $xfer += $input->readListBegin($_etype1188, $_size1185);
-            for ($_i1189 = 0; $_i1189 < $_size1185; ++$_i1189)
+            $_size1192 = 0;
+            $_etype1195 = 0;
+            $xfer += $input->readListBegin($_etype1195, $_size1192);
+            for ($_i1196 = 0; $_i1196 < $_size1192; ++$_i1196)
             {
-              $elem1190 = null;
-              $elem1190 = new \metastore\Partition();
-              $xfer += $elem1190->read($input);
-              $this->new_parts []= $elem1190;
+              $elem1197 = null;
+              $elem1197 = new \metastore\Partition();
+              $xfer += $elem1197->read($input);
+              $this->new_parts []= $elem1197;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33689,9 +33689,9 @@ class ThriftHiveMetastore_alter_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1191)
+          foreach ($this->new_parts as $iter1198)
           {
-            $xfer += $iter1191->write($output);
+            $xfer += $iter1198->write($output);
           }
         }
         $output->writeListEnd();
@@ -33906,15 +33906,15 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size1192 = 0;
-            $_etype1195 = 0;
-            $xfer += $input->readListBegin($_etype1195, $_size1192);
-            for ($_i1196 = 0; $_i1196 < $_size1192; ++$_i1196)
+            $_size1199 = 0;
+            $_etype1202 = 0;
+            $xfer += $input->readListBegin($_etype1202, $_size1199);
+            for ($_i1203 = 0; $_i1203 < $_size1199; ++$_i1203)
             {
-              $elem1197 = null;
-              $elem1197 = new \metastore\Partition();
-              $xfer += $elem1197->read($input);
-              $this->new_parts []= $elem1197;
+              $elem1204 = null;
+              $elem1204 = new \metastore\Partition();
+              $xfer += $elem1204->read($input);
+              $this->new_parts []= $elem1204;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33960,9 +33960,9 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1198)
+          foreach ($this->new_parts as $iter1205)
           {
-            $xfer += $iter1198->write($output);
+            $xfer += $iter1205->write($output);
           }
         }
         $output->writeListEnd();
@@ -34440,14 +34440,14 @@ class ThriftHiveMetastore_rename_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1199 = 0;
-            $_etype1202 = 0;
-            $xfer += $input->readListBegin($_etype1202, $_size1199);
-            for ($_i1203 = 0; $_i1203 < $_size1199; ++$_i1203)
+            $_size1206 = 0;
+            $_etype1209 = 0;
+            $xfer += $input->readListBegin($_etype1209, $_size1206);
+            for ($_i1210 = 0; $_i1210 < $_size1206; ++$_i1210)
             {
-              $elem1204 = null;
-              $xfer += $input->readString($elem1204);
-              $this->part_vals []= $elem1204;
+              $elem1211 = null;
+              $xfer += $input->readString($elem1211);
+              $this->part_vals []= $elem1211;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34493,9 +34493,9 @@ class ThriftHiveMetastore_rename_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1205)
+          foreach ($this->part_vals as $iter1212)
           {
-            $xfer += $output->writeString($iter1205);
+            $xfer += $output->writeString($iter1212);
           }
         }
         $output->writeListEnd();
@@ -34680,14 +34680,14 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1206 = 0;
-            $_etype1209 = 0;
-            $xfer += $input->readListBegin($_etype1209, $_size1206);
-            for ($_i1210 = 0; $_i1210 < $_size1206; ++$_i1210)
+            $_size1213 = 0;
+            $_etype1216 = 0;
+            $xfer += $input->readListBegin($_etype1216, $_size1213);
+            for ($_i1217 = 0; $_i1217 < $_size1213; ++$_i1217)
             {
-              $elem1211 = null;
-              $xfer += $input->readString($elem1211);
-              $this->part_vals []= $elem1211;
+              $elem1218 = null;
+              $xfer += $input->readString($elem1218);
+              $this->part_vals []= $elem1218;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34722,9 +34722,9 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1212)
+          foreach ($this->part_vals as $iter1219)
           {
-            $xfer += $output->writeString($iter1212);
+            $xfer += $output->writeString($iter1219);
           }
         }
         $output->writeListEnd();
@@ -35178,14 +35178,14 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1213 = 0;
-            $_etype1216 = 0;
-            $xfer += $input->readListBegin($_etype1216, $_size1213);
-            for ($_i1217 = 0; $_i1217 < $_size1213; ++$_i1217)
+            $_size1220 = 0;
+            $_etype1223 = 0;
+            $xfer += $input->readListBegin($_etype1223, $_size1220);
+            for ($_i1224 = 0; $_i1224 < $_size1220; ++$_i1224)
             {
-              $elem1218 = null;
-              $xfer += $input->readString($elem1218);
-              $this->success []= $elem1218;
+              $elem1225 = null;
+              $xfer += $input->readString($elem1225);
+              $this->success []= $elem1225;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -35221,9 +35221,9 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1219)
+          foreach ($this->success as $iter1226)
           {
-            $xfer += $output->writeString($iter1219);
+            $xfer += $output->writeString($iter1226);
           }
         }
         $output->writeListEnd();
@@ -35383,17 +35383,17 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size1220 = 0;
-            $_ktype1221 = 0;
-            $_vtype1222 = 0;
-            $xfer += $input->readMapBegin($_ktype1221, $_vtype1222, $_size1220);
-            for ($_i1224 = 0; $_i1224 < $_size1220; ++$_i1224)
+            $_size1227 = 0;
+            $_ktype1228 = 0;
+            $_vtype1229 = 0;
+            $xfer += $input->readMapBegin($_ktype1228, $_vtype1229, $_size1227);
+            for ($_i1231 = 0; $_i1231 < $_size1227; ++$_i1231)
             {
-              $key1225 = '';
-              $val1226 = '';
-              $xfer += $input->readString($key1225);
-              $xfer += $input->readString($val1226);
-              $this->success[$key1225] = $val1226;
+              $key1232 = '';
+              $val1233 = '';
+              $xfer += $input->readString($key1232);
+              $xfer += $input->readString($val1233);
+              $this->success[$key1232] = $val1233;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -35429,10 +35429,10 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->success));
         {
-          foreach ($this->success as $kiter1227 => $viter1228)
+          foreach ($this->success as $kiter1234 => $viter1235)
           {
-            $xfer += $output->writeString($kiter1227);
-            $xfer += $output->writeString($viter1228);
+            $xfer += $output->writeString($kiter1234);
+            $xfer += $output->writeString($viter1235);
           }
         }
         $output->writeMapEnd();
@@ -35552,17 +35552,17 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size1229 = 0;
-            $_ktype1230 = 0;
-            $_vtype1231 = 0;
-            $xfer += $input->readMapBegin($_ktype1230, $_vtype1231, $_size1229);
-            for ($_i1233 = 0; $_i1233 < $_size1229; ++$_i1233)
+            $_size1236 = 0;
+            $_ktype1237 = 0;
+            $_vtype1238 = 0;
+            $xfer += $input->readMapBegin($_ktype1237, $_vtype1238, $_size1236);
+            for ($_i1240 = 0; $_i1240 < $_size1236; ++$_i1240)
             {
-              $key1234 = '';
-              $val1235 = '';
-              $xfer += $input->readString($key1234);
-              $xfer += $input->readString($val1235);
-              $this->part_vals[$key1234] = $val1235;
+              $key1241 = '';
+              $val1242 = '';
+              $xfer += $input->readString($key1241);
+              $xfer += $input->readString($val1242);
+              $this->part_vals[$key1241] = $val1242;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -35607,10 +35607,10 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $kiter1236 => $viter1237)
+          foreach ($this->part_vals as $kiter1243 => $viter1244)
           {
-            $xfer += $output->writeString($kiter1236);
-            $xfer += $output->writeString($viter1237);
+            $xfer += $output->writeString($kiter1243);
+            $xfer += $output->writeString($viter1244);
           }
         }
         $output->writeMapEnd();
@@ -35932,17 +35932,17 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size1238 = 0;
-            $_ktype1239 = 0;
-            $_vtype1240 = 0;
-            $xfer += $input->readMapBegin($_ktype1239, $_vtype1240, $_size1238);
-            for ($_i1242 = 0; $_i1242 < $_size1238; ++$_i1242)
+            $_size1245 = 0;
+            $_ktype1246 = 0;
+            $_vtype1247 = 0;
+            $xfer += $input->readMapBegin($_ktype1246, $_vtype1247, $_size1245);
+            for ($_i1249 = 0; $_i1249 < $_size1245; ++$_i1249)
             {
-              $key1243 = '';
-              $val1244 = '';
-              $xfer += $input->readString($key1243);
-              $xfer += $input->readString($val1244);
-              $this->part_vals[$key1243] = $val1244;
+              $key1250 = '';
+              $val1251 = '';
+              $xfer += $input->readString($key1250);
+              $xfer += $input->readString($val1251);
+              $this->part_vals[$key1250] = $val1251;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -35987,10 +35987,10 @@ class ThriftHiveMetastore_isPartit

<TRUNCATED>

[12/12] hive git commit: HIVE-19089: Create/Replicate Allocate write-id event (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
HIVE-19089: Create/Replicate Allocate write-id event (Mahesh Kumar Behera, reviewed by Sankar Hariappan)


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

Branch: refs/heads/master
Commit: fa9e743e7afbcd6409a51955e39e4e2bb3c109d2
Parents: 7fb088b
Author: Sankar Hariappan <sa...@apache.org>
Authored: Tue Apr 17 12:54:16 2018 +0530
Committer: Sankar Hariappan <sa...@apache.org>
Committed: Tue Apr 17 12:54:16 2018 +0530

----------------------------------------------------------------------
 .../listener/DbNotificationListener.java        |   46 +-
 .../listener/TestDbNotificationListener.java    |    5 +
 .../TestReplicationScenariosAcidTables.java     |   25 +-
 .../apache/hadoop/hive/ql/exec/ReplTxnTask.java |   40 +-
 .../apache/hadoop/hive/ql/exec/ReplTxnWork.java |   55 +-
 .../hadoop/hive/ql/lockmgr/DbTxnManager.java    |    9 +
 .../hadoop/hive/ql/lockmgr/DummyTxnManager.java |    6 +
 .../hadoop/hive/ql/lockmgr/HiveTxnManager.java  |   12 +
 .../hadoop/hive/ql/metadata/HiveUtils.java      |   10 +
 .../ql/parse/ReplicationSemanticAnalyzer.java   |   53 +-
 .../hadoop/hive/ql/parse/repl/DumpType.java     |    7 +
 .../repl/dump/events/AllocWriteIdHandler.java   |   41 +
 .../repl/dump/events/EventHandlerFactory.java   |    1 +
 .../repl/load/message/AbortTxnHandler.java      |    4 +-
 .../repl/load/message/AllocWriteIdHandler.java  |   64 +
 .../repl/load/message/CommitTxnHandler.java     |    7 +-
 .../parse/repl/load/message/OpenTxnHandler.java |    4 +-
 .../metastore/txn/TestCompactionTxnHandler.java |    5 +-
 .../hive/metastore/txn/TestTxnHandler.java      |  115 +-
 .../hive/ql/lockmgr/TestDbTxnManager2.java      |   21 +-
 .../hive/ql/txn/compactor/CompactorTest.java    |    6 +-
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 2258 ++++++-------
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp | 3016 +++++++++---------
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |   34 +-
 .../metastore/api/AddDynamicPartitions.java     |   32 +-
 .../api/AllocateTableWriteIdsRequest.java       |  519 ++-
 .../api/AllocateTableWriteIdsResponse.java      |   36 +-
 .../metastore/api/ClearFileMetadataRequest.java |   32 +-
 .../hive/metastore/api/ClientCapabilities.java  |   32 +-
 .../hive/metastore/api/CompactionRequest.java   |   44 +-
 .../hive/metastore/api/CreationMetadata.java    |   32 +-
 .../metastore/api/FindSchemasByColsResp.java    |   36 +-
 .../hive/metastore/api/FireEventRequest.java    |   32 +-
 .../metastore/api/GetAllFunctionsResponse.java  |   36 +-
 .../api/GetFileMetadataByExprRequest.java       |   32 +-
 .../api/GetFileMetadataByExprResult.java        |   48 +-
 .../metastore/api/GetFileMetadataRequest.java   |   32 +-
 .../metastore/api/GetFileMetadataResult.java    |   44 +-
 .../hive/metastore/api/GetTablesRequest.java    |   32 +-
 .../hive/metastore/api/GetTablesResult.java     |   36 +-
 .../api/HeartbeatTxnRangeResponse.java          |   64 +-
 .../metastore/api/InsertEventRequestData.java   |   64 +-
 .../hadoop/hive/metastore/api/LockRequest.java  |   36 +-
 .../hive/metastore/api/Materialization.java     |   32 +-
 .../api/NotificationEventResponse.java          |   36 +-
 .../metastore/api/PutFileMetadataRequest.java   |   64 +-
 .../hive/metastore/api/SchemaVersion.java       |   36 +-
 .../hive/metastore/api/ShowCompactResponse.java |   36 +-
 .../hive/metastore/api/ShowLocksResponse.java   |   36 +-
 .../hive/metastore/api/ThriftHiveMetastore.java | 2476 +++++++-------
 .../hive/metastore/api/WMFullResourcePlan.java  |  144 +-
 .../api/WMGetAllResourcePlanResponse.java       |   36 +-
 .../WMGetTriggersForResourePlanResponse.java    |   36 +-
 .../api/WMValidateResourcePlanResponse.java     |   64 +-
 .../gen-php/metastore/ThriftHiveMetastore.php   | 1394 ++++----
 .../src/gen/thrift/gen-php/metastore/Types.php  |  842 ++---
 .../hive_metastore/ThriftHiveMetastore.py       |  940 +++---
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |  543 ++--
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |   15 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    |   11 +-
 .../hive/metastore/HiveMetaStoreClient.java     |   20 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java |   13 +-
 .../hive/metastore/MetaStoreEventListener.java  |   34 +-
 .../metastore/MetaStoreListenerNotifier.java    |   85 +-
 .../hive/metastore/events/AbortTxnEvent.java    |   32 -
 .../metastore/events/AllocWriteIdEvent.java     |   57 +
 .../hive/metastore/events/CommitTxnEvent.java   |   32 -
 .../hive/metastore/events/ListenerEvent.java    |   16 -
 .../hive/metastore/events/OpenTxnEvent.java     |   32 -
 .../messaging/AllocWriteIdMessage.java          |   36 +
 .../hive/metastore/messaging/EventMessage.java  |    3 +-
 .../messaging/MessageDeserializer.java          |    5 +
 .../metastore/messaging/MessageFactory.java     |   13 +
 .../event/filters/DatabaseAndTableFilter.java   |    3 +-
 .../messaging/json/JSONAllocWriteIdMessage.java |  111 +
 .../messaging/json/JSONMessageDeserializer.java |    9 +
 .../messaging/json/JSONMessageFactory.java      |    7 +
 .../hadoop/hive/metastore/txn/TxnHandler.java   |  166 +-
 .../src/main/thrift/hive_metastore.thrift       |   11 +-
 .../HiveMetaStoreClientPreCatalog.java          |   19 +-
 80 files changed, 7789 insertions(+), 6719 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
index 7f21573..59d1e3a 100644
--- a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
@@ -18,6 +18,7 @@
 package org.apache.hive.hcatalog.listener;
 
 import java.io.IOException;
+import java.sql.Connection;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
@@ -72,6 +73,7 @@ import org.apache.hadoop.hive.metastore.events.LoadPartitionDoneEvent;
 import org.apache.hadoop.hive.metastore.events.OpenTxnEvent;
 import org.apache.hadoop.hive.metastore.events.CommitTxnEvent;
 import org.apache.hadoop.hive.metastore.events.AbortTxnEvent;
+import org.apache.hadoop.hive.metastore.events.AllocWriteIdEvent;
 import org.apache.hadoop.hive.metastore.events.ListenerEvent;
 import org.apache.hadoop.hive.metastore.messaging.EventMessage.EventType;
 import org.apache.hadoop.hive.metastore.messaging.MessageFactory;
@@ -449,7 +451,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
   }
 
   @Override
-  public void onOpenTxn(OpenTxnEvent openTxnEvent) throws MetaException {
+  public void onOpenTxn(OpenTxnEvent openTxnEvent, Connection dbConn, SQLGenerator sqlGenerator) throws MetaException {
     int lastTxnIdx = openTxnEvent.getTxnIds().size() - 1;
     OpenTxnMessage msg = msgFactory.buildOpenTxnMessage(openTxnEvent.getTxnIds().get(0),
             openTxnEvent.getTxnIds().get(lastTxnIdx));
@@ -457,35 +459,37 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
             new NotificationEvent(0, now(), EventType.OPEN_TXN.toString(), msg.toString());
 
     try {
-      addNotificationLog(event, openTxnEvent);
+      addNotificationLog(event, openTxnEvent, dbConn, sqlGenerator);
     } catch (SQLException e) {
       throw new MetaException("Unable to execute direct SQL " + StringUtils.stringifyException(e));
     }
   }
 
   @Override
-  public void onCommitTxn(CommitTxnEvent commitTxnEvent) throws MetaException {
+  public void onCommitTxn(CommitTxnEvent commitTxnEvent, Connection dbConn, SQLGenerator sqlGenerator)
+          throws MetaException {
     NotificationEvent event =
             new NotificationEvent(0, now(), EventType.COMMIT_TXN.toString(), msgFactory.buildCommitTxnMessage(
                     commitTxnEvent.getTxnId())
                     .toString());
 
     try {
-      addNotificationLog(event, commitTxnEvent);
+      addNotificationLog(event, commitTxnEvent, dbConn, sqlGenerator);
     } catch (SQLException e) {
       throw new MetaException("Unable to execute direct SQL " + StringUtils.stringifyException(e));
     }
   }
 
   @Override
-  public void onAbortTxn(AbortTxnEvent abortTxnEvent) throws MetaException {
+  public void onAbortTxn(AbortTxnEvent abortTxnEvent, Connection dbConn, SQLGenerator sqlGenerator)
+          throws MetaException {
     NotificationEvent event =
         new NotificationEvent(0, now(), EventType.ABORT_TXN.toString(), msgFactory.buildAbortTxnMessage(
             abortTxnEvent.getTxnId())
             .toString());
 
     try {
-      addNotificationLog(event, abortTxnEvent);
+      addNotificationLog(event, abortTxnEvent, dbConn, sqlGenerator);
     } catch (SQLException e) {
       throw new MetaException("Unable to execute direct SQL " + StringUtils.stringifyException(e));
     }
@@ -591,6 +595,27 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     process(event, dropConstraintEvent);
   }
 
+  /***
+   * @param allocWriteIdEvent Alloc write id event
+   * @throws MetaException
+   */
+  @Override
+  public void onAllocWriteId(AllocWriteIdEvent allocWriteIdEvent, Connection dbConn, SQLGenerator sqlGenerator)
+          throws MetaException {
+    String tableName = allocWriteIdEvent.getTableName();
+    String dbName = allocWriteIdEvent.getDbName();
+    NotificationEvent event =
+            new NotificationEvent(0, now(), EventType.ALLOC_WRITE_ID.toString(), msgFactory
+                    .buildAllocWriteIdMessage(allocWriteIdEvent.getTxnToWriteIdList(), dbName, tableName).toString());
+    event.setTableName(tableName);
+    event.setDbName(dbName);
+    try {
+      addNotificationLog(event, allocWriteIdEvent, dbConn, sqlGenerator);
+    } catch (SQLException e) {
+      throw new MetaException("Unable to execute direct SQL " + StringUtils.stringifyException(e));
+    }
+  }
+
   private int now() {
     long millis = System.currentTimeMillis();
     millis /= 1000;
@@ -606,9 +631,9 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     return "'" + input + "'";
   }
 
-  private void addNotificationLog(NotificationEvent event, ListenerEvent listenerEvent)
-          throws MetaException, SQLException {
-    if ((listenerEvent.getConnection() == null) || (listenerEvent.getSqlGenerator() == null)) {
+  private void addNotificationLog(NotificationEvent event, ListenerEvent listenerEvent, Connection dbConn,
+                                  SQLGenerator sqlGenerator) throws MetaException, SQLException {
+    if ((dbConn == null) || (sqlGenerator == null)) {
       LOG.info("connection or sql generator is not set so executing sql via DN");
       process(event, listenerEvent);
       return;
@@ -616,8 +641,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     Statement stmt = null;
     ResultSet rs = null;
     try {
-      stmt = listenerEvent.getConnection().createStatement();
-      SQLGenerator sqlGenerator = listenerEvent.getSqlGenerator();
+      stmt = dbConn.createStatement();
       event.setMessageFormat(msgFactory.getMessageFormat());
 
       if (sqlGenerator.getDbProduct() == MYSQL) {

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
index 70c6a94..eef917e 100644
--- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
+++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
@@ -74,6 +74,7 @@ import org.apache.hadoop.hive.metastore.events.OpenTxnEvent;
 import org.apache.hadoop.hive.metastore.events.CommitTxnEvent;
 import org.apache.hadoop.hive.metastore.events.AbortTxnEvent;
 import org.apache.hadoop.hive.metastore.events.ListenerEvent;
+import org.apache.hadoop.hive.metastore.events.AllocWriteIdEvent;
 import org.apache.hadoop.hive.metastore.messaging.AddPartitionMessage;
 import org.apache.hadoop.hive.metastore.messaging.AlterPartitionMessage;
 import org.apache.hadoop.hive.metastore.messaging.AlterTableMessage;
@@ -233,6 +234,10 @@ public class TestDbNotificationListener {
     public void onAbortTxn(AbortTxnEvent abortTxnEvent) throws MetaException {
       pushEventId(EventType.ABORT_TXN, abortTxnEvent);
     }
+
+    public void onAllocWriteId(AllocWriteIdEvent allocWriteIdEvent) throws MetaException {
+      pushEventId(EventType.ALLOC_WRITE_ID, allocWriteIdEvent);
+    }
   }
 
   @SuppressWarnings("rawtypes")

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTables.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTables.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTables.java
index cac9922..2ad83b6 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTables.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTables.java
@@ -100,7 +100,8 @@ public class TestReplicationScenariosAcidTables {
             .verifyResult(bootStrapDump.lastReplicationId);
 
     // create table will start and coomit the transaction
-    primary.run("CREATE TABLE " + tableName +
+    primary.run("use " + primaryDbName)
+           .run("CREATE TABLE " + tableName +
             " (key int, value int) PARTITIONED BY (load_date date) " +
             "CLUSTERED BY(key) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ('transactional'='true')")
             .run("SHOW TABLES LIKE '" + tableName + "'")
@@ -113,16 +114,12 @@ public class TestReplicationScenariosAcidTables {
             primary.dump(primaryDbName, bootStrapDump.lastReplicationId);
     replica.load(replicatedDbName, incrementalDump.dumpLocation)
             .run("REPL STATUS " + replicatedDbName)
-            .verifyResult(incrementalDump.lastReplicationId)
-            .run("SHOW TABLES LIKE '" + tableName + "'")
-            .verifyResult(tableName);
+            .verifyResult(incrementalDump.lastReplicationId);
 
     // Test the idempotent behavior of Open and Commit Txn
     replica.load(replicatedDbName, incrementalDump.dumpLocation)
             .run("REPL STATUS " + replicatedDbName)
-            .verifyResult(incrementalDump.lastReplicationId)
-            .run("SHOW TABLES LIKE '" + tableName + "'")
-            .verifyResult(tableName);
+            .verifyResult(incrementalDump.lastReplicationId);
   }
 
   @Test
@@ -135,7 +132,8 @@ public class TestReplicationScenariosAcidTables {
             .verifyResult(bootStrapDump.lastReplicationId);
 
     // this should fail
-    primary.runFailure("CREATE TABLE " + tableNameFail +
+    primary.run("use " + primaryDbName)
+            .runFailure("CREATE TABLE " + tableNameFail +
             " (key int, value int) PARTITIONED BY (load_date date) " +
             "CLUSTERED BY(key) ('transactional'='true')")
             .run("SHOW TABLES LIKE '" + tableNameFail + "'")
@@ -145,16 +143,12 @@ public class TestReplicationScenariosAcidTables {
             primary.dump(primaryDbName, bootStrapDump.lastReplicationId);
     replica.load(replicatedDbName, incrementalDump.dumpLocation)
             .run("REPL STATUS " + replicatedDbName)
-            .verifyResult(incrementalDump.lastReplicationId)
-            .run("SHOW TABLES LIKE '" + tableNameFail + "'")
-            .verifyFailure(new String[]{tableNameFail});
+            .verifyResult(incrementalDump.lastReplicationId);
 
     // Test the idempotent behavior of Abort Txn
     replica.load(replicatedDbName, incrementalDump.dumpLocation)
             .run("REPL STATUS " + replicatedDbName)
-            .verifyResult(incrementalDump.lastReplicationId)
-            .run("SHOW TABLES LIKE '" + tableNameFail + "'")
-            .verifyFailure(new String[]{tableNameFail});
+            .verifyResult(incrementalDump.lastReplicationId);
   }
 
   @Test
@@ -165,7 +159,8 @@ public class TestReplicationScenariosAcidTables {
             .run("REPL STATUS " + replicatedDbName)
             .verifyResult(bootStrapDump.lastReplicationId);
 
-    primary.run("CREATE TABLE " + tableName +
+    primary.run("use " + primaryDbName)
+            .run("CREATE TABLE " + tableName +
             " (key int, value int) PARTITIONED BY (load_date date) " +
             "CLUSTERED BY(key) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ('transactional'='true')")
             .run("SHOW TABLES LIKE '" + tableName + "'")

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplTxnTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplTxnTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplTxnTask.java
index 1cdeeb6..2615072 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplTxnTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplTxnTask.java
@@ -18,8 +18,14 @@
 
 package org.apache.hadoop.hive.ql.exec;
 
+import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.lockmgr.HiveTxnManager;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.InvalidTableException;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ReplicationSpec;
 import org.apache.hadoop.hive.ql.plan.api.StageType;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
@@ -44,11 +50,32 @@ public class ReplTxnTask extends Task<ReplTxnWork> {
       Utilities.FILE_OP_LOGGER.trace("Executing ReplTxnTask " + work.getOperationType().toString() +
               " for txn ids : " + work.getTxnIds().toString() + " replPolicy : " + replPolicy);
     }
+
+    String tableName = work.getTableName() == null || work.getTableName().isEmpty() ? null : work.getTableName();
+    if (tableName != null) {
+      Table tbl;
+      try {
+        tbl = Hive.get().getTable(work.getDbName(), tableName);
+        ReplicationSpec replicationSpec = work.getReplicationSpec();
+        if (replicationSpec != null && !replicationSpec.allowReplacementInto(tbl.getParameters())) {
+          // if the event is already replayed, then no need to replay it again.
+          LOG.debug("ReplTxnTask: Event is skipped as it is already replayed. Event Id: " +
+                  replicationSpec.getReplicationState() + "Event Type: " + work.getOperationType());
+          return 0;
+        }
+      } catch (InvalidTableException e) {
+        LOG.info("Table does not exist so, ignoring the operation as it might be a retry(idempotent) case.");
+        return 0;
+      } catch (HiveException e) {
+        LOG.error("Get table failed with exception " + e.getMessage());
+        return 1;
+      }
+    }
+
     try {
       HiveTxnManager txnManager = driverContext.getCtx().getHiveTxnManager();
       String user = UserGroupInformation.getCurrentUser().getUserName();
-      LOG.debug("Replaying " + work.getOperationType().toString() + " Event for policy " +
-              replPolicy + " with srcTxn " + work.getTxnIds().toString());
+      LOG.debug("Replaying " + work.getOperationType().toString() + " Event for policy " + replPolicy);
       switch(work.getOperationType()) {
       case REPL_OPEN_TXN:
         List<Long> txnIds = txnManager.replOpenTxn(replPolicy, work.getTxnIds(), user);
@@ -68,6 +95,15 @@ public class ReplTxnTask extends Task<ReplTxnWork> {
           LOG.info("Replayed CommitTxn Event for policy " + replPolicy + " with srcTxn " + txnId);
         }
         return 0;
+      case REPL_ALLOC_WRITE_ID:
+        assert work.getTxnToWriteIdList() != null;
+        String dbName = work.getDbName();
+        String tblName = work.getTableName();
+        List <TxnToWriteId> txnToWriteIdList = work.getTxnToWriteIdList();
+        txnManager.replAllocateTableWriteIdsBatch(dbName, tblName, replPolicy, txnToWriteIdList);
+        LOG.info("Replayed alloc write Id Event for repl policy: " + replPolicy + " db Name : " + dbName +
+                " txnToWriteIdList: " +txnToWriteIdList.toString() + " table name: " + tblName);
+        return 0;
       default:
         LOG.error("Operation Type " + work.getOperationType() + " is not supported ");
         return 1;

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplTxnWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplTxnWork.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplTxnWork.java
index 9467415..530e9be 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplTxnWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplTxnWork.java
@@ -19,9 +19,11 @@ package org.apache.hadoop.hive.ql.exec;
 
 import java.io.Serializable;
 
-import com.google.common.collect.Lists;
+import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
+import org.apache.hadoop.hive.ql.parse.ReplicationSpec;
 import org.apache.hadoop.hive.ql.plan.Explain;
 import org.apache.hadoop.hive.ql.plan.Explain.Level;
+import java.util.Collections;
 import java.util.List;
 
 /**
@@ -33,38 +35,49 @@ public class ReplTxnWork implements Serializable {
   private static final long serialVersionUID = 1L;
   private String dbName;
   private String tableName;
+  private String replPolicy;
   private List<Long> txnIds;
+  private List<TxnToWriteId> txnToWriteIdList;
+  private ReplicationSpec replicationSpec;
 
   /**
    * OperationType.
    * Different kind of events supported for replaying.
    */
   public enum OperationType {
-    REPL_OPEN_TXN, REPL_ABORT_TXN, REPL_COMMIT_TXN
+    REPL_OPEN_TXN, REPL_ABORT_TXN, REPL_COMMIT_TXN, REPL_ALLOC_WRITE_ID
   }
 
   OperationType operation;
 
-  public ReplTxnWork(String dbName, String tableName, List<Long> txnIds, OperationType type) {
+  public ReplTxnWork(String replPolicy, String dbName, String tableName, List<Long> txnIds, OperationType type,
+                     List<TxnToWriteId> txnToWriteIdList, ReplicationSpec replicationSpec) {
     this.txnIds = txnIds;
     this.dbName = dbName;
     this.tableName = tableName;
     this.operation = type;
+    this.replPolicy = replPolicy;
+    this.txnToWriteIdList = txnToWriteIdList;
+    this.replicationSpec = replicationSpec;
   }
 
-  public ReplTxnWork(String dbName, String tableName, Long txnId, OperationType type) {
-    this.txnIds = Lists.newArrayList(txnId);
-    this.dbName = dbName;
-    this.tableName = tableName;
-    this.operation = type;
+  public ReplTxnWork(String replPolicy, String dbName, String tableName, List<Long> txnIds, OperationType type,
+                     ReplicationSpec replicationSpec) {
+    this(replPolicy, dbName, tableName, txnIds, type, null, replicationSpec);
   }
 
-  public List<Long> getTxnIds() {
-    return txnIds;
+  public ReplTxnWork(String replPolicy, String dbName, String tableName, Long txnId,
+                     OperationType type, ReplicationSpec replicationSpec) {
+    this(replPolicy, dbName, tableName, Collections.singletonList(txnId), type, null, replicationSpec);
   }
 
-  public Long getTxnId(int idx) {
-    return txnIds.get(idx);
+  public ReplTxnWork(String replPolicy, String dbName, String tableName, OperationType type,
+                     List<TxnToWriteId> txnToWriteIdList, ReplicationSpec replicationSpec) {
+    this(replPolicy, dbName, tableName, null, type, txnToWriteIdList, replicationSpec);
+  }
+
+  public List<Long> getTxnIds() {
+    return txnIds;
   }
 
   public String getDbName() {
@@ -75,17 +88,19 @@ public class ReplTxnWork implements Serializable {
     return tableName;
   }
 
-  public String getReplPolicy() {
-    if ((dbName == null) || (dbName.isEmpty())) {
-      return null;
-    } else if ((tableName == null) || (tableName.isEmpty())) {
-      return dbName.toLowerCase() + ".*";
-    } else {
-      return dbName.toLowerCase() + "." + tableName.toLowerCase();
-    }
+  public String getReplPolicy()  {
+    return replPolicy;
   }
 
   public OperationType getOperationType() {
     return operation;
   }
+
+  public List<TxnToWriteId> getTxnToWriteIdList() {
+    return txnToWriteIdList;
+  }
+
+  public ReplicationSpec getReplicationSpec() {
+    return replicationSpec;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
index f566842..7b7fd5d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
@@ -1013,6 +1013,15 @@ public final class DbTxnManager extends HiveTxnManagerImpl {
       throw new LockException(ErrorMsg.METASTORE_COMMUNICATION_FAILED.getMsg(), e);
     }
   }
+  
+  public void replAllocateTableWriteIdsBatch(String dbName, String tableName, String replPolicy,
+                                             List<TxnToWriteId> srcTxnToWriteIdList) throws LockException {
+    try {
+      getMS().replAllocateTableWriteIdsBatch(dbName, tableName, replPolicy, srcTxnToWriteIdList);
+    } catch (TException e) {
+      throw new LockException(ErrorMsg.METASTORE_COMMUNICATION_FAILED.getMsg(), e);
+    }
+  }
 
   private static long getHeartbeatInterval(Configuration conf) throws LockException {
     // Retrieve HIVE_TXN_TIMEOUT in MILLISECONDS (it's defined as SECONDS),

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java
index 9057bb9..78eedd3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hive.ql.lockmgr;
 
 import org.apache.hadoop.hive.common.ValidTxnWriteIdList;
+import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.common.ValidTxnList;
@@ -77,6 +78,11 @@ class DummyTxnManager extends HiveTxnManagerImpl {
     return 0L;
   }
   @Override
+  public void replAllocateTableWriteIdsBatch(String dbName, String tableName, String replPolicy,
+                                             List<TxnToWriteId> srcTxnToWriteIdList) throws LockException {
+    return;
+  }
+  @Override
   public HiveLockManager getLockManager() throws LockException {
     if (lockMgr == null) {
       boolean supportConcurrency =

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManager.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManager.java
index e0e235b..ec11fec 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManager.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.lockmgr;
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.common.ValidTxnWriteIdList;
 import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.Driver.LockedDriverState;
 import org.apache.hadoop.hive.ql.QueryPlan;
@@ -265,6 +266,17 @@ public interface HiveTxnManager {
   long getTableWriteId(String dbName, String tableName) throws LockException;
 
   /**
+   * Allocates write id for each transaction in the list.
+   * @param dbName database name
+   * @param tableName the name of the table to allocate the write id
+   * @param replPolicy used by replication task to identify the source cluster
+   * @param srcTxnToWriteIdList List of txn id to write id Map
+   * @throws LockException
+   */
+  void replAllocateTableWriteIdsBatch(String dbName, String tableName, String replPolicy,
+                                      List<TxnToWriteId> srcTxnToWriteIdList) throws LockException;
+
+  /**
    * Should be though of more as a unique write operation ID in a given txn (at QueryPlan level).
    * Each statement writing data within a multi statement txn should have a unique WriteId.
    * Even a single statement, (e.g. Merge, multi-insert may generates several writes).

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java
index dae18fb..f1c4d98 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java
@@ -428,4 +428,14 @@ public final class HiveUtils {
 
     return null;
   }
+
+  public static String getReplPolicy(String dbName, String tableName) {
+    if ((dbName == null) || (dbName.isEmpty())) {
+      return null;
+    } else if ((tableName == null) || (tableName.isEmpty())) {
+      return dbName.toLowerCase() + ".*";
+    } else {
+      return dbName.toLowerCase() + "." + tableName.toLowerCase();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/ql/src/java/org/apache/hadoop/hive/ql/parse/ReplicationSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ReplicationSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ReplicationSemanticAnalyzer.java
index f7c78e2..1fda478 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ReplicationSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ReplicationSemanticAnalyzer.java
@@ -244,16 +244,19 @@ public class ReplicationSemanticAnalyzer extends BaseSemanticAnalyzer {
     }
   }
 
-  private boolean shouldReplayEvent(FileStatus dir, DumpType dumpType) throws SemanticException {
-    // This functions filters out all the events which are already replayed. This can be done only
-    // for transaction related events as for other kind of events we can not gurantee that the last
-    // repl id stored in the database/table is valid.
-    if ((dumpType != DumpType.EVENT_ABORT_TXN) &&
-            (dumpType != DumpType.EVENT_OPEN_TXN) &&
-            (dumpType != DumpType.EVENT_COMMIT_TXN)) {
-      return true;
+  private boolean isEventNotReplayed(Map<String, String> params, FileStatus dir, DumpType dumpType) {
+    if (params != null && (params.containsKey(ReplicationSpec.KEY.CURR_STATE_ID.toString()))) {
+      String replLastId = params.get(ReplicationSpec.KEY.CURR_STATE_ID.toString());
+      if (Long.parseLong(replLastId) >= Long.parseLong(dir.getPath().getName())) {
+        LOG.debug("Event " + dumpType + " with replId " + Long.parseLong(dir.getPath().getName())
+                + " is already replayed. LastReplId - " +  Long.parseLong(replLastId));
+        return false;
+      }
     }
+    return true;
+  }
 
+  private boolean shouldReplayEvent(FileStatus dir, DumpType dumpType) throws SemanticException {
     // if database itself is null then we can not filter out anything.
     if (dbNameOrPattern == null || dbNameOrPattern.isEmpty()) {
       return true;
@@ -261,41 +264,23 @@ public class ReplicationSemanticAnalyzer extends BaseSemanticAnalyzer {
       Database database;
       try {
         database = Hive.get().getDatabase(dbNameOrPattern);
+        return isEventNotReplayed(database.getParameters(), dir, dumpType);
       } catch (HiveException e) {
-        LOG.error("failed to get the database " + dbNameOrPattern);
-        throw new SemanticException(e);
-      }
-      String replLastId;
-      Map<String, String> params = database.getParameters();
-      if (params != null && (params.containsKey(ReplicationSpec.KEY.CURR_STATE_ID.toString()))) {
-        replLastId = params.get(ReplicationSpec.KEY.CURR_STATE_ID.toString());
-        if (Long.parseLong(replLastId) >= Long.parseLong(dir.getPath().getName())) {
-          LOG.debug("Event " + dumpType + " with replId " + Long.parseLong(dir.getPath().getName())
-                  + " is already replayed. LastReplId - " +  Long.parseLong(replLastId));
-          return false;
-        }
+        //may be the db is getting created in this load
+        LOG.debug("failed to get the database " + dbNameOrPattern);
+        return true;
       }
     } else {
       Table tbl;
       try {
         tbl = Hive.get().getTable(dbNameOrPattern, tblNameOrPattern);
+        return isEventNotReplayed(tbl.getParameters(), dir, dumpType);
       } catch (HiveException e) {
-        LOG.error("failed to get the table " + dbNameOrPattern + "." + tblNameOrPattern);
-        throw new SemanticException(e);
-      }
-      if (tbl != null) {
-        Map<String, String> params = tbl.getParameters();
-        if (params != null && (params.containsKey(ReplicationSpec.KEY.CURR_STATE_ID.toString()))) {
-          String replLastId = params.get(ReplicationSpec.KEY.CURR_STATE_ID.toString());
-          if (Long.parseLong(replLastId) >= Long.parseLong(dir.getPath().getName())) {
-            LOG.debug("Event " + dumpType + " with replId " + Long.parseLong(dir.getPath().getName())
-                    + " is already replayed. LastReplId - " +  Long.parseLong(replLastId));
-            return false;
-          }
-        }
+        // may be the table is getting created in this load
+        LOG.debug("failed to get the table " + dbNameOrPattern + "." + tblNameOrPattern);
+        return true;
       }
     }
-    return true;
   }
 
   /*

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/DumpType.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/DumpType.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/DumpType.java
index 5fab0d2..2e42267 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/DumpType.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/DumpType.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hive.ql.parse.repl.load.message.TruncateTableHandler;
 import org.apache.hadoop.hive.ql.parse.repl.load.message.OpenTxnHandler;
 import org.apache.hadoop.hive.ql.parse.repl.load.message.CommitTxnHandler;
 import org.apache.hadoop.hive.ql.parse.repl.load.message.AbortTxnHandler;
+import org.apache.hadoop.hive.ql.parse.repl.load.message.AllocWriteIdHandler;
 
 public enum DumpType {
 
@@ -204,6 +205,12 @@ public enum DumpType {
     public MessageHandler handler() {
       return new AbortTxnHandler();
     }
+  },
+  EVENT_ALLOC_WRITE_ID("EVENT_ALLOC_WRITE_ID") {
+    @Override
+    public MessageHandler handler() {
+      return new AllocWriteIdHandler();
+    }
   };
 
   String type = null;

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/AllocWriteIdHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/AllocWriteIdHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/AllocWriteIdHandler.java
new file mode 100644
index 0000000..38efbd7
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/AllocWriteIdHandler.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.parse.repl.dump.events;
+
+import org.apache.hadoop.hive.metastore.api.NotificationEvent;
+import org.apache.hadoop.hive.ql.parse.repl.DumpType;
+import org.apache.hadoop.hive.ql.parse.repl.load.DumpMetaData;
+
+class AllocWriteIdHandler extends AbstractEventHandler {
+  AllocWriteIdHandler(NotificationEvent event) {
+    super(event);
+  }
+
+  @Override
+  public void handle(Context withinContext) throws Exception {
+    LOG.info("Processing#{} ALLOC_WRITE_ID message : {}", fromEventId(), event.getMessage());
+    DumpMetaData dmd = withinContext.createDmd(this);
+    dmd.setPayload(event.getMessage());
+    dmd.write();
+  }
+
+  @Override
+  public DumpType dumpType() {
+    return DumpType.EVENT_ALLOC_WRITE_ID;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/EventHandlerFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/EventHandlerFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/EventHandlerFactory.java
index 10ff21c..a1d61f9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/EventHandlerFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/EventHandlerFactory.java
@@ -53,6 +53,7 @@ public class EventHandlerFactory {
     register(MessageFactory.OPEN_TXN_EVENT, OpenTxnHandler.class);
     register(MessageFactory.COMMIT_TXN_EVENT, CommitTxnHandler.class);
     register(MessageFactory.ABORT_TXN_EVENT, AbortTxnHandler.class);
+    register(MessageFactory.ALLOC_WRITE_ID_EVENT, AllocWriteIdHandler.class);
   }
 
   static void register(String event, Class<? extends EventHandler> handlerClazz) {

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AbortTxnHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AbortTxnHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AbortTxnHandler.java
index 5b2c85b..3c28f84 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AbortTxnHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AbortTxnHandler.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.hive.ql.exec.ReplTxnWork;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.apache.hadoop.hive.ql.metadata.HiveUtils;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import java.io.Serializable;
 import java.util.Collections;
@@ -43,7 +44,8 @@ public class AbortTxnHandler extends AbstractMessageHandler {
     AbortTxnMessage msg = deserializer.getAbortTxnMessage(context.dmd.getPayload());
 
     Task<ReplTxnWork> abortTxnTask = TaskFactory.get(
-        new ReplTxnWork(context.dbName, context.tableName, msg.getTxnId(), ReplTxnWork.OperationType.REPL_ABORT_TXN),
+        new ReplTxnWork(HiveUtils.getReplPolicy(context.dbName, context.tableName), context.dbName, context.tableName,
+                msg.getTxnId(), ReplTxnWork.OperationType.REPL_ABORT_TXN, context.eventOnlyReplicationSpec()),
         context.hiveConf
     );
     updatedMetadata.set(context.dmd.getEventTo().toString(), context.dbName, context.tableName, null);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AllocWriteIdHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AllocWriteIdHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AllocWriteIdHandler.java
new file mode 100644
index 0000000..ef51396
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AllocWriteIdHandler.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.parse.repl.load.message;
+
+import org.apache.hadoop.hive.metastore.messaging.AllocWriteIdMessage;
+import org.apache.hadoop.hive.ql.exec.ReplTxnWork;
+import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.apache.hadoop.hive.ql.metadata.HiveUtils;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * AllocWriteIdHandler
+ * Target(Load) side handler for alloc write id event.
+ */
+public class AllocWriteIdHandler extends AbstractMessageHandler {
+  @Override
+  public List<Task<? extends Serializable>> handle(Context context)
+      throws SemanticException {
+    if (!AcidUtils.isAcidEnabled(context.hiveConf)) {
+      context.log.error("Cannot load alloc write id event as acid is not enabled");
+      throw new SemanticException("Cannot load alloc write id event as acid is not enabled");
+    }
+
+    AllocWriteIdMessage msg =
+        deserializer.getAllocWriteIdMessage(context.dmd.getPayload());
+
+    String dbName = (context.dbName != null && !context.dbName.isEmpty() ? context.dbName : msg.getDB());
+
+    // The context table name can be null if repl load is done on a full db.
+    // But we need table name for alloc write id and that is received from source.
+    String tableName = (context.tableName != null && !context.tableName.isEmpty() ? context.tableName : msg
+            .getTableName());
+
+    // Repl policy should be created based on the table name in context.
+    ReplTxnWork work = new ReplTxnWork(HiveUtils.getReplPolicy(dbName, context.tableName), dbName, tableName,
+        ReplTxnWork.OperationType.REPL_ALLOC_WRITE_ID, msg.getTxnToWriteIdList(), context.eventOnlyReplicationSpec());
+
+    Task<? extends Serializable> allocWriteIdTask = TaskFactory.get(work, context.hiveConf);
+    context.log.info("Added alloc write id task : {}", allocWriteIdTask.getId());
+    updatedMetadata.set(context.dmd.getEventTo().toString(), dbName, tableName, null);
+    return Collections.singletonList(allocWriteIdTask);
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CommitTxnHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CommitTxnHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CommitTxnHandler.java
index 461a0f1..1274213 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CommitTxnHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CommitTxnHandler.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.hive.ql.exec.ReplTxnWork;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.apache.hadoop.hive.ql.metadata.HiveUtils;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import java.io.Serializable;
 import java.util.Collections;
@@ -41,10 +42,10 @@ public class CommitTxnHandler extends AbstractMessageHandler {
     }
 
     CommitTxnMessage msg = deserializer.getCommitTxnMessage(context.dmd.getPayload());
-
     Task<ReplTxnWork> commitTxnTask = TaskFactory.get(
-        new ReplTxnWork(context.dbName, context.tableName, msg.getTxnId(),
-                ReplTxnWork.OperationType.REPL_COMMIT_TXN), context.hiveConf
+        new ReplTxnWork(HiveUtils.getReplPolicy(context.dbName, context.tableName), context.dbName, context.tableName,
+              msg.getTxnId(), ReplTxnWork.OperationType.REPL_COMMIT_TXN, context.eventOnlyReplicationSpec()),
+        context.hiveConf
     );
     updatedMetadata.set(context.dmd.getEventTo().toString(), context.dbName, context.tableName, null);
     context.log.debug("Added Commit txn task : {}", commitTxnTask.getId());

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/OpenTxnHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/OpenTxnHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/OpenTxnHandler.java
index c6349ea..a502117 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/OpenTxnHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/OpenTxnHandler.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.hive.ql.exec.ReplTxnWork;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.apache.hadoop.hive.ql.metadata.HiveUtils;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import java.io.Serializable;
 import java.util.Collections;
@@ -42,7 +43,8 @@ public class OpenTxnHandler extends AbstractMessageHandler {
     OpenTxnMessage msg = deserializer.getOpenTxnMessage(context.dmd.getPayload());
 
     Task<ReplTxnWork> openTxnTask = TaskFactory.get(
-        new ReplTxnWork(context.dbName, context.tableName, msg.getTxnIds(), ReplTxnWork.OperationType.REPL_OPEN_TXN),
+        new ReplTxnWork(HiveUtils.getReplPolicy(context.dbName, context.tableName), context.dbName, context.tableName,
+                msg.getTxnIds(), ReplTxnWork.OperationType.REPL_OPEN_TXN, context.eventOnlyReplicationSpec()),
         context.hiveConf
     );
     updatedMetadata.set(context.dmd.getEventTo().toString(), context.dbName, context.tableName, null);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java b/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java
index 45890ed..f27a5b0 100644
--- a/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java
+++ b/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java
@@ -481,8 +481,9 @@ public class TestCompactionTxnHandler {
     OpenTxnsResponse openTxns = txnHandler.openTxns(new OpenTxnRequest(1, "me", "localhost"));
     long txnId = openTxns.getTxn_ids().get(0);
 
-    AllocateTableWriteIdsResponse writeIds
-            = txnHandler.allocateTableWriteIds(new AllocateTableWriteIdsRequest(openTxns.getTxn_ids(), dbName, tableName));
+    AllocateTableWriteIdsRequest rqst = new AllocateTableWriteIdsRequest(dbName, tableName);
+    rqst.setTxnIds(openTxns.getTxn_ids());
+    AllocateTableWriteIdsResponse writeIds = txnHandler.allocateTableWriteIds(rqst);
     long writeId = writeIds.getTxnToWriteIds().get(0).getWriteId();
     assertEquals(txnId, writeIds.getTxnToWriteIds().get(0).getTxnId());
     assertEquals(1, writeId);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java b/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
index f1da15b..372c709 100644
--- a/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
+++ b/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
@@ -57,6 +57,7 @@ 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.UnlockRequest;
+import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
@@ -156,8 +157,9 @@ public class TestTxnHandler {
     List<String> parts = new ArrayList<String>();
     parts.add("p=1");
 
-    AllocateTableWriteIdsResponse writeIds
-            = txnHandler.allocateTableWriteIds(new AllocateTableWriteIdsRequest(Collections.singletonList(3L), "default", "T"));
+    AllocateTableWriteIdsRequest rqst = new AllocateTableWriteIdsRequest("default", "T");
+    rqst.setTxnIds(Collections.singletonList(3L));
+    AllocateTableWriteIdsResponse writeIds = txnHandler.allocateTableWriteIds(rqst);
     long writeId = writeIds.getTxnToWriteIds().get(0).getWriteId();
     assertEquals(3, writeIds.getTxnToWriteIds().get(0).getTxnId());
     assertEquals(1, writeId);
@@ -1544,24 +1546,117 @@ public class TestTxnHandler {
     Assert.assertTrue("regex should be retryable", result);
   }
 
-  @Test
-  public void testReplOpenTxn() throws Exception {
-    int numTxn = 50000;
+  private List<Long> replOpenTxnForTest(long startId, int numTxn, String replPolicy)
+          throws Exception {
     conf.setIntVar(HiveConf.ConfVars.HIVE_TXN_MAX_OPEN_BATCH, numTxn);
+    long lastId = startId + numTxn - 1;
     OpenTxnRequest rqst = new OpenTxnRequest(numTxn, "me", "localhost");
-    rqst.setReplPolicy("default.*");
-    rqst.setReplSrcTxnIds(LongStream.rangeClosed(1, numTxn)
+    rqst.setReplPolicy(replPolicy);
+    rqst.setReplSrcTxnIds(LongStream.rangeClosed(startId, lastId)
             .boxed().collect(Collectors.toList()));
+
     OpenTxnsResponse openedTxns = txnHandler.openTxns(rqst);
     List<Long> txnList = openedTxns.getTxn_ids();
     assertEquals(txnList.size(), numTxn);
-    for (long i = 0; i < numTxn; i++) {
-      long txnId = txnList.get((int) i);
-      assertEquals(i+1, txnId);
+    int numTxnPresentNow = TxnDbUtil.countQueryAgent(conf, "select count(*) from TXNS where TXN_ID >= " +
+            txnList.get(0) + " and TXN_ID <= " + txnList.get(numTxn - 1));
+    assertEquals(numTxn, numTxnPresentNow);
+
+    checkReplTxnForTest(startId, lastId, replPolicy, txnList);
+    return txnList;
+  }
+
+  private void replAbortTxnForTest(List<Long> txnList, String replPolicy)
+          throws Exception {
+    for (Long txnId : txnList) {
+      AbortTxnRequest rqst = new AbortTxnRequest(txnId);
+      rqst.setReplPolicy(replPolicy);
+      txnHandler.abortTxn(rqst);
     }
+    checkReplTxnForTest(txnList.get(0), txnList.get(txnList.size() - 1), replPolicy, new ArrayList<>());
+  }
+
+  private void checkReplTxnForTest(Long startTxnId, Long endTxnId, String replPolicy, List<Long> targetTxnId)
+          throws Exception {
+    String[] output = TxnDbUtil.queryToString(conf, "select RTM_TARGET_TXN_ID from REPL_TXN_MAP where " +
+            " RTM_SRC_TXN_ID >=  " + startTxnId + "and RTM_SRC_TXN_ID <=  " + endTxnId +
+            " and RTM_REPL_POLICY = \'" + replPolicy + "\'").split("\n");
+    assertEquals(output.length - 1, targetTxnId.size());
+    for (int idx = 1; idx < output.length; idx++) {
+      long txnId = Long.parseLong(output[idx].trim());
+      assertEquals(txnId, targetTxnId.get(idx-1).longValue());
+    }
+  }
+
+  @Test
+  public void testReplOpenTxn() throws Exception {
+    int numTxn = 50000;
+    String[] output = TxnDbUtil.queryToString(conf, "select ntxn_next from NEXT_TXN_ID").split("\n");
+    long startTxnId = Long.parseLong(output[1].trim());
+    List<Long> txnList = replOpenTxnForTest(startTxnId, numTxn, "default.*");
+    assert(txnList.size() == numTxn);
     txnHandler.abortTxns(new AbortTxnsRequest(txnList));
   }
 
+  @Test
+  public void testReplAllocWriteId() throws Exception {
+    int numTxn = 2;
+    String[] output = TxnDbUtil.queryToString(conf, "select ntxn_next from NEXT_TXN_ID").split("\n");
+    long startTxnId = Long.parseLong(output[1].trim());
+    List<Long> srcTxnIdList = LongStream.rangeClosed(startTxnId, numTxn+startTxnId-1)
+            .boxed().collect(Collectors.toList());
+    List<Long> targetTxnList = replOpenTxnForTest(startTxnId, numTxn, "destdb.*");
+    assert(targetTxnList.size() == numTxn);
+
+    List<TxnToWriteId> srcTxnToWriteId;
+    List<TxnToWriteId> targetTxnToWriteId;
+    srcTxnToWriteId = new ArrayList<>();
+
+    for (int idx = 0; idx < numTxn; idx++) {
+      srcTxnToWriteId.add(new TxnToWriteId(startTxnId+idx, idx+1));
+    }
+    AllocateTableWriteIdsRequest allocMsg = new AllocateTableWriteIdsRequest("destdb", "tbl1");
+    allocMsg.setReplPolicy("destdb.*");
+    allocMsg.setSrcTxnToWriteIdList(srcTxnToWriteId);
+    targetTxnToWriteId = txnHandler.allocateTableWriteIds(allocMsg).getTxnToWriteIds();
+    for (int idx = 0; idx < targetTxnList.size(); idx++) {
+      assertEquals(targetTxnToWriteId.get(idx).getWriteId(), srcTxnToWriteId.get(idx).getWriteId());
+      assertEquals(Long.valueOf(targetTxnToWriteId.get(idx).getTxnId()), targetTxnList.get(idx));
+    }
+
+    // idempotent case for destdb db
+    targetTxnToWriteId = txnHandler.allocateTableWriteIds(allocMsg).getTxnToWriteIds();
+    for (int idx = 0; idx < targetTxnList.size(); idx++) {
+      assertEquals(targetTxnToWriteId.get(idx).getWriteId(), srcTxnToWriteId.get(idx).getWriteId());
+      assertEquals(Long.valueOf(targetTxnToWriteId.get(idx).getTxnId()), targetTxnList.get(idx));
+    }
+
+    //invalid case
+    boolean failed = false;
+    srcTxnToWriteId = new ArrayList<>();
+    srcTxnToWriteId.add(new TxnToWriteId(startTxnId, 2*numTxn+1));
+    allocMsg = new AllocateTableWriteIdsRequest("destdb", "tbl2");
+    allocMsg.setReplPolicy("destdb.*");
+    allocMsg.setSrcTxnToWriteIdList(srcTxnToWriteId);
+    try {
+      txnHandler.allocateTableWriteIds(allocMsg).getTxnToWriteIds();
+    } catch (IllegalStateException e) {
+      failed = true;
+    }
+    assertTrue(failed);
+
+    replAbortTxnForTest(srcTxnIdList, "destdb.*");
+
+    // Test for aborted transactions
+    failed = false;
+    try {
+      txnHandler.allocateTableWriteIds(allocMsg).getTxnToWriteIds();
+    } catch (RuntimeException e) {
+      failed = true;
+    }
+    assertTrue(failed);
+  }
+
   private void updateTxns(Connection conn) throws SQLException {
     Statement stmt = conn.createStatement();
     stmt.executeUpdate("update TXNS set txn_last_heartbeat = txn_last_heartbeat + 1");

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
index 8406caa..0926663 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
@@ -1031,9 +1031,9 @@ public class TestDbTxnManager2 {
     //update stmt has p=blah, thus nothing is actually update and we generate empty dyn part list
     Assert.assertEquals(0, TxnDbUtil.countQueryAgent(conf, "select count(*) from WRITE_SET"));
 
-    AllocateTableWriteIdsResponse writeIds
-            = txnHandler.allocateTableWriteIds(new AllocateTableWriteIdsRequest(Collections.singletonList(txnMgr2.getCurrentTxnId()),
-            "default", "tab2"));
+    AllocateTableWriteIdsRequest rqst = new AllocateTableWriteIdsRequest("default", "tab2");
+    rqst.setTxnIds(Collections.singletonList(txnMgr2.getCurrentTxnId()));
+    AllocateTableWriteIdsResponse writeIds = txnHandler.allocateTableWriteIds(rqst);
     Assert.assertEquals(txnMgr2.getCurrentTxnId(), writeIds.getTxnToWriteIds().get(0).getTxnId());
 
     AddDynamicPartitions adp = new AddDynamicPartitions(txnMgr2.getCurrentTxnId(), writeIds.getTxnToWriteIds().get(0).getWriteId(),
@@ -1054,8 +1054,9 @@ public class TestDbTxnManager2 {
     //update stmt has p=blah, thus nothing is actually update and we generate empty dyn part list
     Assert.assertEquals(0, TxnDbUtil.countQueryAgent(conf, "select count(*) from WRITE_SET"));
 
-    writeIds = txnHandler.allocateTableWriteIds(new AllocateTableWriteIdsRequest(Collections.singletonList(txnMgr2.getCurrentTxnId()),
-            "default", "tab2"));
+    rqst = new AllocateTableWriteIdsRequest("default", "tab2");
+    rqst.setTxnIds(Collections.singletonList(txnMgr2.getCurrentTxnId()));
+    writeIds = txnHandler.allocateTableWriteIds(rqst);
     Assert.assertEquals(txnMgr2.getCurrentTxnId(), writeIds.getTxnToWriteIds().get(0).getTxnId());
 
     adp = new AddDynamicPartitions(txnMgr2.getCurrentTxnId(), writeIds.getTxnToWriteIds().get(0).getWriteId(),
@@ -1074,8 +1075,9 @@ public class TestDbTxnManager2 {
     checkCmdOnDriver(driver.compileAndRespond("update TAB2 set b = 17 where a = 1", true));//no rows match
     txnMgr.acquireLocks(driver.getPlan(), ctx, "Long Running");
 
-    writeIds = txnHandler.allocateTableWriteIds(new AllocateTableWriteIdsRequest(Collections.singletonList(txnMgr.getCurrentTxnId()),
-            "default", "tab2"));
+    rqst = new AllocateTableWriteIdsRequest("default", "tab2");
+    rqst.setTxnIds(Collections.singletonList(txnMgr.getCurrentTxnId()));
+    writeIds = txnHandler.allocateTableWriteIds(rqst);
     Assert.assertEquals(txnMgr.getCurrentTxnId(), writeIds.getTxnToWriteIds().get(0).getTxnId());
 
     //so generate empty Dyn Part call
@@ -1119,8 +1121,9 @@ public class TestDbTxnManager2 {
     checkLock(LockType.SHARED_WRITE, LockState.WAITING, "default", "TAB_PART", "p=blah", locks);
     txnMgr.rollbackTxn();
 
-    AllocateTableWriteIdsResponse writeIds
-            = txnHandler.allocateTableWriteIds(new AllocateTableWriteIdsRequest(Collections.singletonList(txnId), "default", "TAB_PART"));
+    AllocateTableWriteIdsRequest rqst = new AllocateTableWriteIdsRequest("default", "TAB_PART");
+    rqst.setTxnIds(Collections.singletonList(txnId));
+    AllocateTableWriteIdsResponse writeIds = txnHandler.allocateTableWriteIds(rqst);
     Assert.assertEquals(txnId, writeIds.getTxnToWriteIds().get(0).getTxnId());
 
     AddDynamicPartitions adp = new AddDynamicPartitions(txnId, writeIds.getTxnToWriteIds().get(0).getWriteId(),

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java
index 083c671..124c97e 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java
@@ -202,7 +202,8 @@ public abstract class CompactorTest {
   protected long allocateWriteId(String dbName, String tblName, long txnid)
           throws MetaException, TxnAbortedException, NoSuchTxnException {
     AllocateTableWriteIdsRequest awiRqst
-            = new AllocateTableWriteIdsRequest(Collections.singletonList(txnid), dbName, tblName);
+            = new AllocateTableWriteIdsRequest(dbName, tblName);
+    awiRqst.setTxnIds(Collections.singletonList(txnid));
     AllocateTableWriteIdsResponse awiResp = txnHandler.allocateTableWriteIds(awiRqst);
     return awiResp.getTxnToWriteIds().get(0).getWriteId();
   }
@@ -254,7 +255,8 @@ public abstract class CompactorTest {
   protected void burnThroughTransactions(String dbName, String tblName, int num, Set<Long> open, Set<Long> aborted)
       throws MetaException, NoSuchTxnException, TxnAbortedException {
     OpenTxnsResponse rsp = txnHandler.openTxns(new OpenTxnRequest(num, "me", "localhost"));
-    AllocateTableWriteIdsRequest awiRqst = new AllocateTableWriteIdsRequest(rsp.getTxn_ids(), dbName, tblName);
+    AllocateTableWriteIdsRequest awiRqst = new AllocateTableWriteIdsRequest(dbName, tblName);
+    awiRqst.setTxnIds(rsp.getTxn_ids());
     AllocateTableWriteIdsResponse awiResp = txnHandler.allocateTableWriteIds(awiRqst);
     int i = 0;
     for (long tid : rsp.getTxn_ids()) {


[08/12] hive git commit: HIVE-19089: Create/Replicate Allocate write-id event (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 f4ccc8b..371757a 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 _list816 = iprot.readListBegin();
-                struct.tables = new ArrayList<Table>(_list816.size);
-                Table _elem817;
-                for (int _i818 = 0; _i818 < _list816.size; ++_i818)
+                org.apache.thrift.protocol.TList _list824 = iprot.readListBegin();
+                struct.tables = new ArrayList<Table>(_list824.size);
+                Table _elem825;
+                for (int _i826 = 0; _i826 < _list824.size; ++_i826)
                 {
-                  _elem817 = new Table();
-                  _elem817.read(iprot);
-                  struct.tables.add(_elem817);
+                  _elem825 = new Table();
+                  _elem825.read(iprot);
+                  struct.tables.add(_elem825);
                 }
                 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 _iter819 : struct.tables)
+          for (Table _iter827 : struct.tables)
           {
-            _iter819.write(oprot);
+            _iter827.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.tables.size());
-        for (Table _iter820 : struct.tables)
+        for (Table _iter828 : struct.tables)
         {
-          _iter820.write(oprot);
+          _iter828.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 _list821 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.tables = new ArrayList<Table>(_list821.size);
-        Table _elem822;
-        for (int _i823 = 0; _i823 < _list821.size; ++_i823)
+        org.apache.thrift.protocol.TList _list829 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.tables = new ArrayList<Table>(_list829.size);
+        Table _elem830;
+        for (int _i831 = 0; _i831 < _list829.size; ++_i831)
         {
-          _elem822 = new Table();
-          _elem822.read(iprot);
-          struct.tables.add(_elem822);
+          _elem830 = new Table();
+          _elem830.read(iprot);
+          struct.tables.add(_elem830);
         }
       }
       struct.setTablesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 b270439..75ddaf6 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 _set650 = iprot.readSetBegin();
-                struct.aborted = new HashSet<Long>(2*_set650.size);
-                long _elem651;
-                for (int _i652 = 0; _i652 < _set650.size; ++_i652)
+                org.apache.thrift.protocol.TSet _set658 = iprot.readSetBegin();
+                struct.aborted = new HashSet<Long>(2*_set658.size);
+                long _elem659;
+                for (int _i660 = 0; _i660 < _set658.size; ++_i660)
                 {
-                  _elem651 = iprot.readI64();
-                  struct.aborted.add(_elem651);
+                  _elem659 = iprot.readI64();
+                  struct.aborted.add(_elem659);
                 }
                 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 _set653 = iprot.readSetBegin();
-                struct.nosuch = new HashSet<Long>(2*_set653.size);
-                long _elem654;
-                for (int _i655 = 0; _i655 < _set653.size; ++_i655)
+                org.apache.thrift.protocol.TSet _set661 = iprot.readSetBegin();
+                struct.nosuch = new HashSet<Long>(2*_set661.size);
+                long _elem662;
+                for (int _i663 = 0; _i663 < _set661.size; ++_i663)
                 {
-                  _elem654 = iprot.readI64();
-                  struct.nosuch.add(_elem654);
+                  _elem662 = iprot.readI64();
+                  struct.nosuch.add(_elem662);
                 }
                 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 _iter656 : struct.aborted)
+          for (long _iter664 : struct.aborted)
           {
-            oprot.writeI64(_iter656);
+            oprot.writeI64(_iter664);
           }
           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 _iter657 : struct.nosuch)
+          for (long _iter665 : struct.nosuch)
           {
-            oprot.writeI64(_iter657);
+            oprot.writeI64(_iter665);
           }
           oprot.writeSetEnd();
         }
@@ -542,16 +542,16 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.aborted.size());
-        for (long _iter658 : struct.aborted)
+        for (long _iter666 : struct.aborted)
         {
-          oprot.writeI64(_iter658);
+          oprot.writeI64(_iter666);
         }
       }
       {
         oprot.writeI32(struct.nosuch.size());
-        for (long _iter659 : struct.nosuch)
+        for (long _iter667 : struct.nosuch)
         {
-          oprot.writeI64(_iter659);
+          oprot.writeI64(_iter667);
         }
       }
     }
@@ -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 _set660 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.aborted = new HashSet<Long>(2*_set660.size);
-        long _elem661;
-        for (int _i662 = 0; _i662 < _set660.size; ++_i662)
+        org.apache.thrift.protocol.TSet _set668 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.aborted = new HashSet<Long>(2*_set668.size);
+        long _elem669;
+        for (int _i670 = 0; _i670 < _set668.size; ++_i670)
         {
-          _elem661 = iprot.readI64();
-          struct.aborted.add(_elem661);
+          _elem669 = iprot.readI64();
+          struct.aborted.add(_elem669);
         }
       }
       struct.setAbortedIsSet(true);
       {
-        org.apache.thrift.protocol.TSet _set663 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.nosuch = new HashSet<Long>(2*_set663.size);
-        long _elem664;
-        for (int _i665 = 0; _i665 < _set663.size; ++_i665)
+        org.apache.thrift.protocol.TSet _set671 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.nosuch = new HashSet<Long>(2*_set671.size);
+        long _elem672;
+        for (int _i673 = 0; _i673 < _set671.size; ++_i673)
         {
-          _elem664 = iprot.readI64();
-          struct.nosuch.add(_elem664);
+          _elem672 = iprot.readI64();
+          struct.nosuch.add(_elem672);
         }
       }
       struct.setNosuchIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 79570a5..c7100a7 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 _list708 = iprot.readListBegin();
-                struct.filesAdded = new ArrayList<String>(_list708.size);
-                String _elem709;
-                for (int _i710 = 0; _i710 < _list708.size; ++_i710)
+                org.apache.thrift.protocol.TList _list716 = iprot.readListBegin();
+                struct.filesAdded = new ArrayList<String>(_list716.size);
+                String _elem717;
+                for (int _i718 = 0; _i718 < _list716.size; ++_i718)
                 {
-                  _elem709 = iprot.readString();
-                  struct.filesAdded.add(_elem709);
+                  _elem717 = iprot.readString();
+                  struct.filesAdded.add(_elem717);
                 }
                 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 _list711 = iprot.readListBegin();
-                struct.filesAddedChecksum = new ArrayList<String>(_list711.size);
-                String _elem712;
-                for (int _i713 = 0; _i713 < _list711.size; ++_i713)
+                org.apache.thrift.protocol.TList _list719 = iprot.readListBegin();
+                struct.filesAddedChecksum = new ArrayList<String>(_list719.size);
+                String _elem720;
+                for (int _i721 = 0; _i721 < _list719.size; ++_i721)
                 {
-                  _elem712 = iprot.readString();
-                  struct.filesAddedChecksum.add(_elem712);
+                  _elem720 = iprot.readString();
+                  struct.filesAddedChecksum.add(_elem720);
                 }
                 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 _iter714 : struct.filesAdded)
+          for (String _iter722 : struct.filesAdded)
           {
-            oprot.writeString(_iter714);
+            oprot.writeString(_iter722);
           }
           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 _iter715 : struct.filesAddedChecksum)
+            for (String _iter723 : struct.filesAddedChecksum)
             {
-              oprot.writeString(_iter715);
+              oprot.writeString(_iter723);
             }
             oprot.writeListEnd();
           }
@@ -634,9 +634,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.filesAdded.size());
-        for (String _iter716 : struct.filesAdded)
+        for (String _iter724 : struct.filesAdded)
         {
-          oprot.writeString(_iter716);
+          oprot.writeString(_iter724);
         }
       }
       BitSet optionals = new BitSet();
@@ -653,9 +653,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetFilesAddedChecksum()) {
         {
           oprot.writeI32(struct.filesAddedChecksum.size());
-          for (String _iter717 : struct.filesAddedChecksum)
+          for (String _iter725 : struct.filesAddedChecksum)
           {
-            oprot.writeString(_iter717);
+            oprot.writeString(_iter725);
           }
         }
       }
@@ -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 _list718 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.filesAdded = new ArrayList<String>(_list718.size);
-        String _elem719;
-        for (int _i720 = 0; _i720 < _list718.size; ++_i720)
+        org.apache.thrift.protocol.TList _list726 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.filesAdded = new ArrayList<String>(_list726.size);
+        String _elem727;
+        for (int _i728 = 0; _i728 < _list726.size; ++_i728)
         {
-          _elem719 = iprot.readString();
-          struct.filesAdded.add(_elem719);
+          _elem727 = iprot.readString();
+          struct.filesAdded.add(_elem727);
         }
       }
       struct.setFilesAddedIsSet(true);
@@ -682,13 +682,13 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list721 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.filesAddedChecksum = new ArrayList<String>(_list721.size);
-          String _elem722;
-          for (int _i723 = 0; _i723 < _list721.size; ++_i723)
+          org.apache.thrift.protocol.TList _list729 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.filesAddedChecksum = new ArrayList<String>(_list729.size);
+          String _elem730;
+          for (int _i731 = 0; _i731 < _list729.size; ++_i731)
           {
-            _elem722 = iprot.readString();
-            struct.filesAddedChecksum.add(_elem722);
+            _elem730 = iprot.readString();
+            struct.filesAddedChecksum.add(_elem730);
           }
         }
         struct.setFilesAddedChecksumIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 62f0dd6..d1134b5 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 _list634 = iprot.readListBegin();
-                struct.component = new ArrayList<LockComponent>(_list634.size);
-                LockComponent _elem635;
-                for (int _i636 = 0; _i636 < _list634.size; ++_i636)
+                org.apache.thrift.protocol.TList _list642 = iprot.readListBegin();
+                struct.component = new ArrayList<LockComponent>(_list642.size);
+                LockComponent _elem643;
+                for (int _i644 = 0; _i644 < _list642.size; ++_i644)
                 {
-                  _elem635 = new LockComponent();
-                  _elem635.read(iprot);
-                  struct.component.add(_elem635);
+                  _elem643 = new LockComponent();
+                  _elem643.read(iprot);
+                  struct.component.add(_elem643);
                 }
                 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 _iter637 : struct.component)
+          for (LockComponent _iter645 : struct.component)
           {
-            _iter637.write(oprot);
+            _iter645.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -803,9 +803,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.component.size());
-        for (LockComponent _iter638 : struct.component)
+        for (LockComponent _iter646 : struct.component)
         {
-          _iter638.write(oprot);
+          _iter646.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 _list639 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.component = new ArrayList<LockComponent>(_list639.size);
-        LockComponent _elem640;
-        for (int _i641 = 0; _i641 < _list639.size; ++_i641)
+        org.apache.thrift.protocol.TList _list647 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.component = new ArrayList<LockComponent>(_list647.size);
+        LockComponent _elem648;
+        for (int _i649 = 0; _i649 < _list647.size; ++_i649)
         {
-          _elem640 = new LockComponent();
-          _elem640.read(iprot);
-          struct.component.add(_elem640);
+          _elem648 = new LockComponent();
+          _elem648.read(iprot);
+          struct.component.add(_elem648);
         }
       }
       struct.setComponentIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 556207e..403c7aa 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
@@ -589,13 +589,13 @@ import org.slf4j.LoggerFactory;
           case 1: // TABLES_USED
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set824 = iprot.readSetBegin();
-                struct.tablesUsed = new HashSet<String>(2*_set824.size);
-                String _elem825;
-                for (int _i826 = 0; _i826 < _set824.size; ++_i826)
+                org.apache.thrift.protocol.TSet _set832 = iprot.readSetBegin();
+                struct.tablesUsed = new HashSet<String>(2*_set832.size);
+                String _elem833;
+                for (int _i834 = 0; _i834 < _set832.size; ++_i834)
                 {
-                  _elem825 = iprot.readString();
-                  struct.tablesUsed.add(_elem825);
+                  _elem833 = iprot.readString();
+                  struct.tablesUsed.add(_elem833);
                 }
                 iprot.readSetEnd();
               }
@@ -645,9 +645,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 _iter827 : struct.tablesUsed)
+          for (String _iter835 : struct.tablesUsed)
           {
-            oprot.writeString(_iter827);
+            oprot.writeString(_iter835);
           }
           oprot.writeSetEnd();
         }
@@ -689,9 +689,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.tablesUsed.size());
-        for (String _iter828 : struct.tablesUsed)
+        for (String _iter836 : struct.tablesUsed)
         {
-          oprot.writeString(_iter828);
+          oprot.writeString(_iter836);
         }
       }
       BitSet optionals = new BitSet();
@@ -720,13 +720,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, Materialization struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TSet _set829 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.tablesUsed = new HashSet<String>(2*_set829.size);
-        String _elem830;
-        for (int _i831 = 0; _i831 < _set829.size; ++_i831)
+        org.apache.thrift.protocol.TSet _set837 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.tablesUsed = new HashSet<String>(2*_set837.size);
+        String _elem838;
+        for (int _i839 = 0; _i839 < _set837.size; ++_i839)
         {
-          _elem830 = iprot.readString();
-          struct.tablesUsed.add(_elem830);
+          _elem838 = iprot.readString();
+          struct.tablesUsed.add(_elem838);
         }
       }
       struct.setTablesUsedIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 3c35a0e..baa4224 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 _list700 = iprot.readListBegin();
-                struct.events = new ArrayList<NotificationEvent>(_list700.size);
-                NotificationEvent _elem701;
-                for (int _i702 = 0; _i702 < _list700.size; ++_i702)
+                org.apache.thrift.protocol.TList _list708 = iprot.readListBegin();
+                struct.events = new ArrayList<NotificationEvent>(_list708.size);
+                NotificationEvent _elem709;
+                for (int _i710 = 0; _i710 < _list708.size; ++_i710)
                 {
-                  _elem701 = new NotificationEvent();
-                  _elem701.read(iprot);
-                  struct.events.add(_elem701);
+                  _elem709 = new NotificationEvent();
+                  _elem709.read(iprot);
+                  struct.events.add(_elem709);
                 }
                 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 _iter703 : struct.events)
+          for (NotificationEvent _iter711 : struct.events)
           {
-            _iter703.write(oprot);
+            _iter711.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.events.size());
-        for (NotificationEvent _iter704 : struct.events)
+        for (NotificationEvent _iter712 : struct.events)
         {
-          _iter704.write(oprot);
+          _iter712.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 _list705 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.events = new ArrayList<NotificationEvent>(_list705.size);
-        NotificationEvent _elem706;
-        for (int _i707 = 0; _i707 < _list705.size; ++_i707)
+        org.apache.thrift.protocol.TList _list713 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.events = new ArrayList<NotificationEvent>(_list713.size);
+        NotificationEvent _elem714;
+        for (int _i715 = 0; _i715 < _list713.size; ++_i715)
         {
-          _elem706 = new NotificationEvent();
-          _elem706.read(iprot);
-          struct.events.add(_elem706);
+          _elem714 = new NotificationEvent();
+          _elem714.read(iprot);
+          struct.events.add(_elem714);
         }
       }
       struct.setEventsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 474555f..ad42dbe 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 _list768 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list768.size);
-                long _elem769;
-                for (int _i770 = 0; _i770 < _list768.size; ++_i770)
+                org.apache.thrift.protocol.TList _list776 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list776.size);
+                long _elem777;
+                for (int _i778 = 0; _i778 < _list776.size; ++_i778)
                 {
-                  _elem769 = iprot.readI64();
-                  struct.fileIds.add(_elem769);
+                  _elem777 = iprot.readI64();
+                  struct.fileIds.add(_elem777);
                 }
                 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 _list771 = iprot.readListBegin();
-                struct.metadata = new ArrayList<ByteBuffer>(_list771.size);
-                ByteBuffer _elem772;
-                for (int _i773 = 0; _i773 < _list771.size; ++_i773)
+                org.apache.thrift.protocol.TList _list779 = iprot.readListBegin();
+                struct.metadata = new ArrayList<ByteBuffer>(_list779.size);
+                ByteBuffer _elem780;
+                for (int _i781 = 0; _i781 < _list779.size; ++_i781)
                 {
-                  _elem772 = iprot.readBinary();
-                  struct.metadata.add(_elem772);
+                  _elem780 = iprot.readBinary();
+                  struct.metadata.add(_elem780);
                 }
                 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 _iter774 : struct.fileIds)
+          for (long _iter782 : struct.fileIds)
           {
-            oprot.writeI64(_iter774);
+            oprot.writeI64(_iter782);
           }
           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 _iter775 : struct.metadata)
+          for (ByteBuffer _iter783 : struct.metadata)
           {
-            oprot.writeBinary(_iter775);
+            oprot.writeBinary(_iter783);
           }
           oprot.writeListEnd();
         }
@@ -651,16 +651,16 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter776 : struct.fileIds)
+        for (long _iter784 : struct.fileIds)
         {
-          oprot.writeI64(_iter776);
+          oprot.writeI64(_iter784);
         }
       }
       {
         oprot.writeI32(struct.metadata.size());
-        for (ByteBuffer _iter777 : struct.metadata)
+        for (ByteBuffer _iter785 : struct.metadata)
         {
-          oprot.writeBinary(_iter777);
+          oprot.writeBinary(_iter785);
         }
       }
       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 _list778 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list778.size);
-        long _elem779;
-        for (int _i780 = 0; _i780 < _list778.size; ++_i780)
+        org.apache.thrift.protocol.TList _list786 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list786.size);
+        long _elem787;
+        for (int _i788 = 0; _i788 < _list786.size; ++_i788)
         {
-          _elem779 = iprot.readI64();
-          struct.fileIds.add(_elem779);
+          _elem787 = iprot.readI64();
+          struct.fileIds.add(_elem787);
         }
       }
       struct.setFileIdsIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list781 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.metadata = new ArrayList<ByteBuffer>(_list781.size);
-        ByteBuffer _elem782;
-        for (int _i783 = 0; _i783 < _list781.size; ++_i783)
+        org.apache.thrift.protocol.TList _list789 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.metadata = new ArrayList<ByteBuffer>(_list789.size);
+        ByteBuffer _elem790;
+        for (int _i791 = 0; _i791 < _list789.size; ++_i791)
         {
-          _elem782 = iprot.readBinary();
-          struct.metadata.add(_elem782);
+          _elem790 = iprot.readBinary();
+          struct.metadata.add(_elem790);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
index 12a8d1b..62bc3b4 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
@@ -1119,14 +1119,14 @@ import org.slf4j.LoggerFactory;
           case 4: // COLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list896 = iprot.readListBegin();
-                struct.cols = new ArrayList<FieldSchema>(_list896.size);
-                FieldSchema _elem897;
-                for (int _i898 = 0; _i898 < _list896.size; ++_i898)
+                org.apache.thrift.protocol.TList _list904 = iprot.readListBegin();
+                struct.cols = new ArrayList<FieldSchema>(_list904.size);
+                FieldSchema _elem905;
+                for (int _i906 = 0; _i906 < _list904.size; ++_i906)
                 {
-                  _elem897 = new FieldSchema();
-                  _elem897.read(iprot);
-                  struct.cols.add(_elem897);
+                  _elem905 = new FieldSchema();
+                  _elem905.read(iprot);
+                  struct.cols.add(_elem905);
                 }
                 iprot.readListEnd();
               }
@@ -1212,9 +1212,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.cols.size()));
-          for (FieldSchema _iter899 : struct.cols)
+          for (FieldSchema _iter907 : struct.cols)
           {
-            _iter899.write(oprot);
+            _iter907.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1323,9 +1323,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetCols()) {
         {
           oprot.writeI32(struct.cols.size());
-          for (FieldSchema _iter900 : struct.cols)
+          for (FieldSchema _iter908 : struct.cols)
           {
-            _iter900.write(oprot);
+            _iter908.write(oprot);
           }
         }
       }
@@ -1368,14 +1368,14 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(3)) {
         {
-          org.apache.thrift.protocol.TList _list901 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.cols = new ArrayList<FieldSchema>(_list901.size);
-          FieldSchema _elem902;
-          for (int _i903 = 0; _i903 < _list901.size; ++_i903)
+          org.apache.thrift.protocol.TList _list909 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.cols = new ArrayList<FieldSchema>(_list909.size);
+          FieldSchema _elem910;
+          for (int _i911 = 0; _i911 < _list909.size; ++_i911)
           {
-            _elem902 = new FieldSchema();
-            _elem902.read(iprot);
-            struct.cols.add(_elem902);
+            _elem910 = new FieldSchema();
+            _elem910.read(iprot);
+            struct.cols.add(_elem910);
           }
         }
         struct.setColsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 6c418f5..4e465ac 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 _list676 = iprot.readListBegin();
-                struct.compacts = new ArrayList<ShowCompactResponseElement>(_list676.size);
-                ShowCompactResponseElement _elem677;
-                for (int _i678 = 0; _i678 < _list676.size; ++_i678)
+                org.apache.thrift.protocol.TList _list684 = iprot.readListBegin();
+                struct.compacts = new ArrayList<ShowCompactResponseElement>(_list684.size);
+                ShowCompactResponseElement _elem685;
+                for (int _i686 = 0; _i686 < _list684.size; ++_i686)
                 {
-                  _elem677 = new ShowCompactResponseElement();
-                  _elem677.read(iprot);
-                  struct.compacts.add(_elem677);
+                  _elem685 = new ShowCompactResponseElement();
+                  _elem685.read(iprot);
+                  struct.compacts.add(_elem685);
                 }
                 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 _iter679 : struct.compacts)
+          for (ShowCompactResponseElement _iter687 : struct.compacts)
           {
-            _iter679.write(oprot);
+            _iter687.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.compacts.size());
-        for (ShowCompactResponseElement _iter680 : struct.compacts)
+        for (ShowCompactResponseElement _iter688 : struct.compacts)
         {
-          _iter680.write(oprot);
+          _iter688.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 _list681 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.compacts = new ArrayList<ShowCompactResponseElement>(_list681.size);
-        ShowCompactResponseElement _elem682;
-        for (int _i683 = 0; _i683 < _list681.size; ++_i683)
+        org.apache.thrift.protocol.TList _list689 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.compacts = new ArrayList<ShowCompactResponseElement>(_list689.size);
+        ShowCompactResponseElement _elem690;
+        for (int _i691 = 0; _i691 < _list689.size; ++_i691)
         {
-          _elem682 = new ShowCompactResponseElement();
-          _elem682.read(iprot);
-          struct.compacts.add(_elem682);
+          _elem690 = new ShowCompactResponseElement();
+          _elem690.read(iprot);
+          struct.compacts.add(_elem690);
         }
       }
       struct.setCompactsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa9e743e/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 857dc7a..cfc7f9c 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 _list642 = iprot.readListBegin();
-                struct.locks = new ArrayList<ShowLocksResponseElement>(_list642.size);
-                ShowLocksResponseElement _elem643;
-                for (int _i644 = 0; _i644 < _list642.size; ++_i644)
+                org.apache.thrift.protocol.TList _list650 = iprot.readListBegin();
+                struct.locks = new ArrayList<ShowLocksResponseElement>(_list650.size);
+                ShowLocksResponseElement _elem651;
+                for (int _i652 = 0; _i652 < _list650.size; ++_i652)
                 {
-                  _elem643 = new ShowLocksResponseElement();
-                  _elem643.read(iprot);
-                  struct.locks.add(_elem643);
+                  _elem651 = new ShowLocksResponseElement();
+                  _elem651.read(iprot);
+                  struct.locks.add(_elem651);
                 }
                 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 _iter645 : struct.locks)
+          for (ShowLocksResponseElement _iter653 : struct.locks)
           {
-            _iter645.write(oprot);
+            _iter653.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -416,9 +416,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetLocks()) {
         {
           oprot.writeI32(struct.locks.size());
-          for (ShowLocksResponseElement _iter646 : struct.locks)
+          for (ShowLocksResponseElement _iter654 : struct.locks)
           {
-            _iter646.write(oprot);
+            _iter654.write(oprot);
           }
         }
       }
@@ -430,14 +430,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list647 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.locks = new ArrayList<ShowLocksResponseElement>(_list647.size);
-          ShowLocksResponseElement _elem648;
-          for (int _i649 = 0; _i649 < _list647.size; ++_i649)
+          org.apache.thrift.protocol.TList _list655 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.locks = new ArrayList<ShowLocksResponseElement>(_list655.size);
+          ShowLocksResponseElement _elem656;
+          for (int _i657 = 0; _i657 < _list655.size; ++_i657)
           {
-            _elem648 = new ShowLocksResponseElement();
-            _elem648.read(iprot);
-            struct.locks.add(_elem648);
+            _elem656 = new ShowLocksResponseElement();
+            _elem656.read(iprot);
+            struct.locks.add(_elem656);
           }
         }
         struct.setLocksIsSet(true);