You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by an...@apache.org on 2020/07/20 04:20:41 UTC

[hive] branch master updated: HIVE-23560: Optimize bootstrap dump to abort only write Transactions (Aasha Medhi, reviewed by Pravin Kumar Sinha)

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

anishek pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new 8c8b2a5  HIVE-23560: Optimize bootstrap dump to abort only write Transactions (Aasha Medhi, reviewed by Pravin Kumar Sinha)
8c8b2a5 is described below

commit 8c8b2a5d83bc10d7855b9a4ad0674f955735bf83
Author: Anishek Agarwal <an...@gmail.com>
AuthorDate: Mon Jul 20 09:50:25 2020 +0530

    HIVE-23560: Optimize bootstrap dump to abort only write Transactions (Aasha Medhi, reviewed by Pravin Kumar Sinha)
---
 .../java/org/apache/hadoop/hive/conf/HiveConf.java |    5 +
 .../parse/BaseReplicationScenariosAcidTables.java  |   48 +-
 .../parse/TestReplicationScenariosAcidTables.java  |  201 +-
 ...estReplicationScenariosAcidTablesBootstrap.java |    3 +-
 .../hadoop/hive/ql/exec/repl/ReplDumpTask.java     |   68 +-
 .../hadoop/hive/ql/lockmgr/DbTxnManager.java       |   11 +
 .../hadoop/hive/ql/lockmgr/DummyTxnManager.java    |    5 +
 .../hadoop/hive/ql/lockmgr/HiveTxnManager.java     |   12 +
 .../hadoop/hive/ql/lockmgr/TestDbTxnManager2.java  |   46 +
 .../hive/metastore/api/GetOpenTxnsRequest.java     |  441 +++
 .../hive/metastore/api/ThriftHiveMetastore.java    | 4106 ++++++++++++--------
 .../gen-php/metastore/ThriftHiveMetastore.php      | 1660 ++++----
 .../src/gen/thrift/gen-php/metastore/Types.php     |  101 +
 .../hive_metastore/ThriftHiveMetastore-remote      |    7 +
 .../gen-py/hive_metastore/ThriftHiveMetastore.py   | 1165 +++---
 .../src/gen/thrift/gen-py/hive_metastore/ttypes.py |   75 +
 .../src/gen/thrift/gen-rb/hive_metastore_types.rb  |   17 +
 .../src/gen/thrift/gen-rb/thrift_hive_metastore.rb |   54 +
 .../hadoop/hive/metastore/HiveMetaStoreClient.java |    6 +
 .../hadoop/hive/metastore/IMetaStoreClient.java    |   10 +
 .../src/main/thrift/hive_metastore.thrift          |    5 +
 .../hadoop/hive/metastore/HiveMetaStore.java       |    5 +
 .../hadoop/hive/metastore/txn/OpenTxnList.java     |   32 +-
 .../hadoop/hive/metastore/txn/TxnHandler.java      |    6 +
 .../apache/hadoop/hive/metastore/txn/TxnStore.java |    9 +
 .../metastore/HiveMetaStoreClientPreCatalog.java   |    6 +
 26 files changed, 5235 insertions(+), 2869 deletions(-)

diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index e6a188a..d9e3681 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -550,6 +550,11 @@ public class HiveConf extends Configuration {
         "Indicates the timeout for all transactions which are opened before triggering bootstrap REPL DUMP. "
             + "If these open transactions are not closed within the timeout value, then REPL DUMP will "
             + "forcefully abort those transactions and continue with bootstrap dump."),
+    REPL_BOOTSTRAP_DUMP_ABORT_WRITE_TXN_AFTER_TIMEOUT("hive.repl.bootstrap.dump.abort.write.txn.after.timeout",
+      true,
+      "Indicates whether to abort write transactions belonging to the db under replication while doing a" +
+        " bootstrap dump after the timeout configured by hive.repl.bootstrap.dump.open.txn.timeout. If set to false," +
+        " bootstrap dump will fail."),
     //https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/TransparentEncryption.html#Running_as_the_superuser
     REPL_ADD_RAW_RESERVED_NAMESPACE("hive.repl.add.raw.reserved.namespace", false,
         "For TDE with same encryption keys on source and target, allow Distcp super user to access \n"
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/BaseReplicationScenariosAcidTables.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/BaseReplicationScenariosAcidTables.java
index 521a2ef..58b3ab4 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/BaseReplicationScenariosAcidTables.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/BaseReplicationScenariosAcidTables.java
@@ -25,6 +25,15 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.AllocateTableWriteIdsRequest;
 import org.apache.hadoop.hive.metastore.api.OpenTxnRequest;
 import org.apache.hadoop.hive.metastore.api.OpenTxnsResponse;
+import org.apache.hadoop.hive.metastore.api.LockComponent;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.LockLevel;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.TxnOpenException;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.UnlockRequest;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.txn.TxnDbUtil;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
@@ -43,6 +52,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
@@ -336,18 +346,30 @@ public class BaseReplicationScenariosAcidTables {
     return txns;
   }
 
-  void allocateWriteIdsForTables(String primaryDbName, Map<String, Long> tables,
-                                         TxnStore txnHandler,
-                                         List<Long> txns, HiveConf primaryConf) throws Throwable {
+  List<Long> allocateWriteIdsForTablesAndAquireLocks(String primaryDbName, Map<String, Long> tables,
+                                                     TxnStore txnHandler,
+                                                     List<Long> txns, HiveConf primaryConf) throws Throwable {
     AllocateTableWriteIdsRequest rqst = new AllocateTableWriteIdsRequest();
     rqst.setDbName(primaryDbName);
-
+    List<Long> lockIds = new ArrayList<>();
     for(Map.Entry<String, Long> entry : tables.entrySet()) {
       rqst.setTableName(entry.getKey());
       rqst.setTxnIds(txns);
       txnHandler.allocateTableWriteIds(rqst);
+      for (long txnId : txns) {
+        LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.TABLE,
+          primaryDbName);
+        comp.setTablename(entry.getKey());
+        comp.setOperationType(DataOperationType.UPDATE);
+        List<LockComponent> components = new ArrayList<LockComponent>(1);
+        components.add(comp);
+        LockRequest lockRequest = new LockRequest(components, "u1", "hostname");
+        lockRequest.setTxnid(txnId);
+        lockIds.add(txnHandler.lock(lockRequest).getLockid());
+      }
     }
     verifyWriteIdsForTables(tables, primaryConf, primaryDbName);
+    return lockIds;
   }
 
   void verifyWriteIdsForTables(Map<String, Long> tables, HiveConf conf, String dbName)
@@ -373,6 +395,17 @@ public class BaseReplicationScenariosAcidTables {
                     "select count(*) from TXNS where txn_state = 'a' and " + txnIdRange));
   }
 
+  void verifyAllOpenTxnsNotAborted(List<Long> txns, HiveConf primaryConf) throws Throwable {
+    int numTxns = txns.size();
+    String txnIdRange = " txn_id >= " + txns.get(0) + " and txn_id <= " + txns.get(numTxns - 1);
+    Assert.assertEquals(TxnDbUtil.queryToString(primaryConf, "select * from TXNS"),
+      numTxns, TxnDbUtil.countQueryAgent(primaryConf,
+        "select count(*) from TXNS where txn_state = 'o' and " + txnIdRange));
+    Assert.assertEquals(TxnDbUtil.queryToString(primaryConf, "select * from TXNS"),
+      0, TxnDbUtil.countQueryAgent(primaryConf,
+        "select count(*) from TXNS where txn_state = 'a' and " + txnIdRange));
+  }
+
   void verifyNextId(Map<String, Long> tables, String dbName, HiveConf conf) throws Throwable {
     // Verify the next write id
     for(Map.Entry<String, Long> entry : tables.entrySet()) {
@@ -395,4 +428,11 @@ public class BaseReplicationScenariosAcidTables {
                             + "' and cq_table = '" + entry.getKey() + "'"));
     }
   }
+
+  void releaseLocks(TxnStore txnStore, List<Long> lockIds) throws NoSuchLockException,
+    TxnOpenException, MetaException {
+    for (Long lockId : lockIds) {
+      txnStore.unlock(new UnlockRequest(lockId));
+    }
+  }
 }
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 529b387..71326ec 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
@@ -24,6 +24,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.AbortTxnsRequest;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.messaging.json.gzip.GzipJSONMessageEncoder;
@@ -229,19 +230,21 @@ public class TestReplicationScenariosAcidTables extends BaseReplicationScenarios
     // Allocate write ids for both tables t1 and t2 for all txns
     // t1=5+1(insert) and t2=5+2(insert)
     Map<String, Long> tables = new HashMap<>();
-    tables.put("t1", numTxns+1L);
-    tables.put("t2", numTxns+2L);
-    allocateWriteIdsForTables(primaryDbName, tables, txnHandler, txns, primaryConf);
+    tables.put("t1", numTxns + 1L);
+    tables.put("t2", numTxns + 2L);
+    List<Long> lockIds = allocateWriteIdsForTablesAndAquireLocks(primaryDbName, tables, txnHandler, txns, primaryConf);
 
     // Bootstrap dump with open txn timeout as 1s.
     List<String> withConfigs = Arrays.asList(
-            "'hive.repl.bootstrap.dump.open.txn.timeout'='1s'");
+      "'"+ HiveConf.ConfVars.REPL_BOOTSTRAP_DUMP_OPEN_TXN_TIMEOUT+"'='1s'");
     WarehouseInstance.Tuple bootstrapDump = primary
             .run("use " + primaryDbName)
             .dump(primaryDbName, withConfigs);
 
     // After bootstrap dump, all the opened txns should be aborted. Verify it.
     verifyAllOpenTxnsAborted(txns, primaryConf);
+    //Release the locks
+    releaseLocks(txnHandler, lockIds);
     verifyNextId(tables, primaryDbName, primaryConf);
 
     // Bootstrap load which should also replicate the aborted write ids on both tables.
@@ -275,6 +278,196 @@ public class TestReplicationScenariosAcidTables extends BaseReplicationScenarios
   }
 
   @Test
+  public void testAcidTablesBootstrapWithOpenTxnsDiffDb() throws Throwable {
+    int numTxns = 5;
+    HiveConf primaryConf = primary.getConf();
+    TxnStore txnHandler = TxnUtils.getTxnStore(primary.getConf());
+    // Open 5 txns
+    List<Long> txns = openTxns(numTxns, txnHandler, primaryConf);
+
+    // Create 2 tables, one partitioned and other not. Also, have both types of full ACID and MM tables.
+    primary.run("use " + primaryDbName)
+      .run("create table t1 (id int) clustered by(id) into 3 buckets stored as orc " +
+        "tblproperties (\"transactional\"=\"true\")")
+      .run("insert into t1 values(1)")
+      .run("create table t2 (rank int) partitioned by (name string) tblproperties(\"transactional\"=\"true\", " +
+        "\"transactional_properties\"=\"insert_only\")")
+      .run("insert into t2 partition(name='Bob') values(11)")
+      .run("insert into t2 partition(name='Carl') values(10)");
+
+    // Allocate write ids for both tables of secondary db for all txns
+    // t1=5 and t2=5
+    Map<String, Long> tablesInSecDb = new HashMap<>();
+    tablesInSecDb.put("t1", (long) numTxns);
+    tablesInSecDb.put("t2", (long) numTxns);
+    List<Long> lockIds = allocateWriteIdsForTablesAndAquireLocks(primaryDbName + "_extra",
+      tablesInSecDb, txnHandler, txns, primaryConf);
+
+    // Bootstrap dump with open txn timeout as 1s.
+    List<String> withConfigs = Arrays.asList(
+      "'" + HiveConf.ConfVars.REPL_BOOTSTRAP_DUMP_OPEN_TXN_TIMEOUT + "'='1s'");
+    WarehouseInstance.Tuple bootstrapDump = primary
+      .run("use " + primaryDbName)
+      .dump(primaryDbName, withConfigs);
+
+    // After bootstrap dump, all the opened txns should not be aborted as itr belongs to a diff db. Verify it.
+    verifyAllOpenTxnsNotAborted(txns, primaryConf);
+    Map<String, Long> tablesInPrimary = new HashMap<>();
+    tablesInPrimary.put("t1", 1L);
+    tablesInPrimary.put("t2", 2L);
+    verifyNextId(tablesInPrimary, primaryDbName, primaryConf);
+
+    // Bootstrap load which should not replicate the write ids on both tables as they are on different db.
+    HiveConf replicaConf = replica.getConf();
+    replica.load(replicatedDbName, primaryDbName)
+      .run("use " + replicatedDbName)
+      .run("show tables")
+      .verifyResults(new String[] {"t1", "t2"})
+      .run("repl status " + replicatedDbName)
+      .verifyResult(bootstrapDump.lastReplicationId)
+      .run("select id from t1")
+      .verifyResults(new String[]{"1"})
+      .run("select rank from t2 order by rank")
+      .verifyResults(new String[] {"10", "11"});
+
+    // Verify if HWM is properly set after REPL LOAD
+    verifyNextId(tablesInPrimary, replicatedDbName, replicaConf);
+
+    // Verify if none of the write ids are not replicated to the replicated DB as they belong to diff db
+    for(Map.Entry<String, Long> entry : tablesInPrimary.entrySet()) {
+      entry.setValue((long) 0);
+    }
+    verifyWriteIdsForTables(tablesInPrimary, replicaConf, replicatedDbName);
+    //Abort the txns
+    txnHandler.abortTxns(new AbortTxnsRequest(txns));
+    //Release the locks
+    releaseLocks(txnHandler, lockIds);
+  }
+
+  @Test
+  public void testAcidTablesBootstrapWithOpenTxnsPrimaryAndSecondaryDb() throws Throwable {
+    int numTxns = 5;
+    HiveConf primaryConf = primary.getConf();
+    TxnStore txnHandler = TxnUtils.getTxnStore(primary.getConf());
+    // Open 5 txns for secondary db
+    List<Long> txns = openTxns(numTxns, txnHandler, primaryConf);
+    // Open 5 txns for primary db
+    List<Long> txnsSameDb = openTxns(numTxns, txnHandler, primaryConf);
+
+    // Create 2 tables, one partitioned and other not. Also, have both types of full ACID and MM tables.
+    primary.run("use " + primaryDbName)
+      .run("create table t1 (id int) clustered by(id) into 3 buckets stored as orc " +
+        "tblproperties (\"transactional\"=\"true\")")
+      .run("insert into t1 values(1)")
+      .run("create table t2 (rank int) partitioned by (name string) tblproperties(\"transactional\"=\"true\", " +
+        "\"transactional_properties\"=\"insert_only\")")
+      .run("insert into t2 partition(name='Bob') values(11)")
+      .run("insert into t2 partition(name='Carl') values(10)");
+
+    // Allocate write ids for both tables of secondary db for all txns
+    // t1=5 and t2=5
+    Map<String, Long> tablesInSecDb = new HashMap<>();
+    tablesInSecDb.put("t1", (long) numTxns);
+    tablesInSecDb.put("t2", (long) numTxns);
+    List<Long> lockIds = allocateWriteIdsForTablesAndAquireLocks(primaryDbName + "_extra",
+      tablesInSecDb, txnHandler, txns, primaryConf);
+    // Allocate write ids for both tables of primary db for all txns
+    // t1=5+1L and t2=5+2L inserts
+    Map<String, Long> tablesInPrimDb = new HashMap<>();
+    tablesInPrimDb.put("t1", (long) numTxns + 1L);
+    tablesInPrimDb.put("t2", (long) numTxns + 2L);
+    lockIds.addAll(allocateWriteIdsForTablesAndAquireLocks(primaryDbName,
+      tablesInPrimDb, txnHandler, txnsSameDb, primaryConf));
+
+    // Bootstrap dump with open txn timeout as 1s.
+    List<String> withConfigs = Arrays.asList(
+      "'" + HiveConf.ConfVars.REPL_BOOTSTRAP_DUMP_OPEN_TXN_TIMEOUT + "'='1s'");
+    WarehouseInstance.Tuple bootstrapDump = primary
+      .run("use " + primaryDbName)
+      .dump(primaryDbName, withConfigs);
+
+    // After bootstrap dump, all the opened txns should not be aborted as it belongs to a diff db. Verify it.
+    verifyAllOpenTxnsNotAborted(txns, primaryConf);
+    // After bootstrap dump, all the opened txns should be aborted as it belongs to db under replication. Verify it.
+    verifyAllOpenTxnsAborted(txnsSameDb, primaryConf);
+    verifyNextId(tablesInPrimDb, primaryDbName, primaryConf);
+
+    // Bootstrap load which should replicate the write ids on both tables as they are on same db and
+    // not on different db.
+    HiveConf replicaConf = replica.getConf();
+    replica.load(replicatedDbName, primaryDbName)
+      .run("use " + replicatedDbName)
+      .run("show tables")
+      .verifyResults(new String[] {"t1", "t2"})
+      .run("repl status " + replicatedDbName)
+      .verifyResult(bootstrapDump.lastReplicationId)
+      .run("select id from t1")
+      .verifyResults(new String[]{"1"})
+      .run("select rank from t2 order by rank")
+      .verifyResults(new String[] {"10", "11"});
+
+    // Verify if HWM is properly set after REPL LOAD
+    verifyNextId(tablesInPrimDb, replicatedDbName, replicaConf);
+
+    // Verify if only the write ids belonging to primary db are replicated to the replicated DB.
+    for(Map.Entry<String, Long> entry : tablesInPrimDb.entrySet()) {
+      entry.setValue((long) numTxns);
+    }
+    verifyWriteIdsForTables(tablesInPrimDb, replicaConf, replicatedDbName);
+    //Abort the txns for secondary db
+    txnHandler.abortTxns(new AbortTxnsRequest(txns));
+    //Release the locks
+    releaseLocks(txnHandler, lockIds);
+  }
+
+  @Test
+  public void testAcidTablesBootstrapWithOpenTxnsAbortDisabled() throws Throwable {
+    int numTxns = 5;
+    HiveConf primaryConf = primary.getConf();
+    TxnStore txnHandler = TxnUtils.getTxnStore(primary.getConf());
+    // Open 5 txns
+    List<Long> txns = openTxns(numTxns, txnHandler, primaryConf);
+
+    // Create 2 tables, one partitioned and other not. Also, have both types of full ACID and MM tables.
+    primary.run("use " + primaryDbName)
+      .run("create table t1 (id int) clustered by(id) into 3 buckets stored as orc " +
+        "tblproperties (\"transactional\"=\"true\")")
+      .run("insert into t1 values(1)")
+      .run("create table t2 (rank int) partitioned by (name string) tblproperties(\"transactional\"=\"true\", " +
+        "\"transactional_properties\"=\"insert_only\")")
+      .run("insert into t2 partition(name='Bob') values(11)")
+      .run("insert into t2 partition(name='Carl') values(10)");
+
+    // Allocate write ids for both tables t1 and t2 for all txns
+    // t1=5+1(insert) and t2=5+2(insert)
+    Map<String, Long> tables = new HashMap<>();
+    tables.put("t1", numTxns + 1L);
+    tables.put("t2", numTxns + 2L);
+    List<Long> lockIds = allocateWriteIdsForTablesAndAquireLocks(primaryDbName, tables, txnHandler, txns, primaryConf);
+
+    // Bootstrap dump with open txn timeout as 1s.
+    List<String> withConfigs = Arrays.asList(
+      "'" + HiveConf.ConfVars.REPL_BOOTSTRAP_DUMP_OPEN_TXN_TIMEOUT + "'='1s'",
+      "'" + HiveConf.ConfVars.REPL_BOOTSTRAP_DUMP_ABORT_WRITE_TXN_AFTER_TIMEOUT + "'='false'");
+    try {
+      WarehouseInstance.Tuple bootstrapDump = primary
+        .run("use " + primaryDbName)
+        .dump(primaryDbName, withConfigs);
+    } catch (Exception e) {
+      Assert.assertEquals("REPL DUMP cannot proceed. Force abort all the open txns is disabled. Enable " +
+        "hive.repl.bootstrap.dump.abort.write.txn.after.timeout to proceed.", e.getMessage());
+    }
+
+    // After bootstrap dump, all the opened txns should not be aborted as it belongs to diff db. Verify it.
+    verifyAllOpenTxnsNotAborted(txns, primaryConf);
+    //Abort the txns
+    txnHandler.abortTxns(new AbortTxnsRequest(txns));
+    //Release the locks
+    releaseLocks(txnHandler, lockIds);
+  }
+
+
+  @Test
   public void testAcidTablesBootstrapWithConcurrentWrites() throws Throwable {
     HiveConf primaryConf = primary.getConf();
     primary.run("use " + primaryDbName)
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTablesBootstrap.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTablesBootstrap.java
index 5d94db7..f52975a 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTablesBootstrap.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTablesBootstrap.java
@@ -194,7 +194,7 @@ public class TestReplicationScenariosAcidTablesBootstrap
     Map<String, Long> tables = new HashMap<>();
     tables.put("t1", numTxns+2L);
     tables.put("t2", numTxns+6L);
-    allocateWriteIdsForTables(primaryDbName, tables, txnHandler, txns, primaryConf);
+    List<Long> lockIds = allocateWriteIdsForTablesAndAquireLocks(primaryDbName, tables, txnHandler, txns, primaryConf);
 
     // Bootstrap dump with open txn timeout as 1s.
     List<String> withConfigs = new LinkedList<>(dumpWithAcidBootstrapClause);
@@ -205,6 +205,7 @@ public class TestReplicationScenariosAcidTablesBootstrap
 
     // After bootstrap dump, all the opened txns should be aborted. Verify it.
     verifyAllOpenTxnsAborted(txns, primaryConf);
+    releaseLocks(txnHandler, lockIds);
     verifyNextId(tables, primaryDbName, primaryConf);
 
     // Incremental load with ACID bootstrap should also replicate the aborted write ids on
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
index b15b326..68cc69a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
@@ -37,11 +37,14 @@ import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 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.ShowLocksResponse;
+import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement;
+import org.apache.hadoop.hive.metastore.api.TxnType;
 import org.apache.hadoop.hive.metastore.messaging.event.filters.AndFilter;
 import org.apache.hadoop.hive.metastore.messaging.event.filters.EventBoundaryFilter;
 import org.apache.hadoop.hive.metastore.messaging.event.filters.ReplEventFilter;
 import org.apache.hadoop.hive.metastore.utils.Retry;
-import org.apache.hadoop.hive.metastore.utils.StringUtils;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
@@ -50,6 +53,8 @@ import org.apache.hadoop.hive.ql.exec.repl.util.ReplUtils;
 import org.apache.hadoop.hive.ql.exec.repl.util.TaskTracker;
 import org.apache.hadoop.hive.ql.exec.util.DAGTraversal;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.apache.hadoop.hive.ql.lockmgr.DbLockManager;
+import org.apache.hadoop.hive.ql.lockmgr.HiveLockManager;
 import org.apache.hadoop.hive.ql.lockmgr.LockException;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -92,7 +97,6 @@ import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.Serializable;
 import java.io.UnsupportedEncodingException;
-import java.net.URI;
 import java.nio.charset.Charset;
 import java.nio.charset.StandardCharsets;
 import java.util.Set;
@@ -107,6 +111,8 @@ import java.util.ArrayList;
 import java.util.Map;
 import java.util.HashMap;
 import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.REPL_BOOTSTRAP_DUMP_ABORT_WRITE_TXN_AFTER_TIMEOUT;
 import static org.apache.hadoop.hive.ql.exec.repl.ReplExternalTables.Writer;
 import static org.apache.hadoop.hive.ql.exec.repl.ReplAck.LOAD_ACKNOWLEDGEMENT;
 import static org.apache.hadoop.hive.ql.exec.repl.util.ReplUtils.RANGER_AUTHORIZER;
@@ -975,6 +981,33 @@ public class ReplDumpTask extends Task<ReplDumpWork> implements Serializable {
     return openTxns;
   }
 
+  List<Long> getOpenTxns(ValidTxnList validTxnList, String dbName) throws LockException {
+    HiveLockManager lockManager = getTxnMgr().getLockManager();
+    long[] invalidTxns = validTxnList.getInvalidTransactions();
+    List<Long> openTxns = new ArrayList<>();
+    Set<Long> dbTxns = new HashSet<>();
+    if (lockManager instanceof DbLockManager) {
+      ShowLocksRequest request = new ShowLocksRequest();
+      request.setDbname(dbName.toLowerCase());
+      ShowLocksResponse showLocksResponse = ((DbLockManager)lockManager).getLocks(request);
+      for (ShowLocksResponseElement showLocksResponseElement : showLocksResponse.getLocks()) {
+        dbTxns.add(showLocksResponseElement.getTxnid());
+      }
+      for (long invalidTxn : invalidTxns) {
+        if (dbTxns.contains(invalidTxn) && !validTxnList.isTxnAborted(invalidTxn)) {
+          openTxns.add(invalidTxn);
+        }
+      }
+    } else {
+      for (long invalidTxn : invalidTxns) {
+        if (!validTxnList.isTxnAborted(invalidTxn)) {
+          openTxns.add(invalidTxn);
+        }
+      }
+    }
+    return openTxns;
+  }
+
   // Get list of valid transactions for Repl Dump. Also wait for a given amount of time for the
   // open transactions to finish. Abort any open transactions after the wait is over.
   String getValidTxnListForReplDump(Hive hiveDb, long waitUntilTime) throws HiveException {
@@ -986,7 +1019,9 @@ public class ReplDumpTask extends Task<ReplDumpWork> implements Serializable {
     // phase won't be able to replicate those txns. So, the logic is to wait for the given amount
     // of time to see if all open txns < current txn is getting aborted/committed. If not, then
     // we forcefully abort those txns just like AcidHouseKeeperService.
-    ValidTxnList validTxnList = getTxnMgr().getValidTxns();
+    //Exclude readonly and repl created tranasactions
+    List<TxnType> excludedTxns = Arrays.asList(TxnType.READ_ONLY, TxnType.REPL_CREATED);
+    ValidTxnList validTxnList = getTxnMgr().getValidTxns(excludedTxns);
     while (System.currentTimeMillis() < waitUntilTime) {
       // If there are no txns which are open for the given ValidTxnList snapshot, then just return it.
       if (getOpenTxns(validTxnList).isEmpty()) {
@@ -999,20 +1034,27 @@ public class ReplDumpTask extends Task<ReplDumpWork> implements Serializable {
       } catch (InterruptedException e) {
         LOG.info("REPL DUMP thread sleep interrupted", e);
       }
-      validTxnList = getTxnMgr().getValidTxns();
+      validTxnList = getTxnMgr().getValidTxns(excludedTxns);
     }
 
     // After the timeout just force abort the open txns
-    List<Long> openTxns = getOpenTxns(validTxnList);
-    if (!openTxns.isEmpty()) {
-      hiveDb.abortTransactions(openTxns);
-      validTxnList = getTxnMgr().getValidTxns();
-      if (validTxnList.getMinOpenTxn() != null) {
-        openTxns = getOpenTxns(validTxnList);
-        LOG.warn("REPL DUMP unable to force abort all the open txns: {} after timeout due to unknown reasons. " +
-                "However, this is rare case that shouldn't happen.", openTxns);
-        throw new IllegalStateException("REPL DUMP triggered abort txns failed for unknown reasons.");
+    if (conf.getBoolVar(REPL_BOOTSTRAP_DUMP_ABORT_WRITE_TXN_AFTER_TIMEOUT)) {
+      List<Long> openTxns = getOpenTxns(validTxnList, work.dbNameOrPattern);
+      if (!openTxns.isEmpty()) {
+        //abort only write transactions for the db under replication if abort transactions is enabled.
+        hiveDb.abortTransactions(openTxns);
+        validTxnList = getTxnMgr().getValidTxns(excludedTxns);
+        openTxns = getOpenTxns(validTxnList, work.dbNameOrPattern);
+        if (!openTxns.isEmpty()) {
+          LOG.warn("REPL DUMP unable to force abort all the open txns: {} after timeout due to unknown reasons. " +
+            "However, this is rare case that shouldn't happen.", openTxns);
+          throw new IllegalStateException("REPL DUMP triggered abort txns failed for unknown reasons.");
+        }
       }
+    } else {
+      LOG.warn("Force abort all the open txns is disabled after timeout");
+      throw new IllegalStateException("REPL DUMP cannot proceed. Force abort all the open txns is disabled. Enable " +
+        "hive.repl.bootstrap.dump.abort.write.txn.after.timeout to proceed.");
     }
     return validTxnList.toString();
   }
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 6b163d6..acee1c3 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
@@ -773,6 +773,17 @@ public final class DbTxnManager extends HiveTxnManagerImpl {
   }
 
   @Override
+  public ValidTxnList getValidTxns(List<TxnType> excludeTxnTypes) throws LockException {
+    assert isTxnOpen();
+    init();
+    try {
+      return getMS().getValidTxns(txnId, excludeTxnTypes);
+    } catch (TException e) {
+      throw new LockException(ErrorMsg.METASTORE_COMMUNICATION_FAILED.getMsg(), e);
+    }
+  }
+
+  @Override
   public ValidTxnWriteIdList getValidWriteIds(List<String> tableList,
                                               String validTxnList) throws LockException {
     assert isTxnOpen();
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 29266db..977d8ef 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
@@ -276,6 +276,11 @@ class DummyTxnManager extends HiveTxnManagerImpl {
   }
 
   @Override
+  public ValidTxnList getValidTxns(List<TxnType> excludeTxnTypes) throws LockException {
+    return new ValidReadTxnList();
+  }
+
+  @Override
   public ValidTxnWriteIdList getValidWriteIds(List<String> tableList,
                                               String validTxnList) throws LockException {
     return new ValidTxnWriteIdList(getCurrentTxnId());
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 5c75e63..a2effe1 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
@@ -186,6 +186,18 @@ public interface HiveTxnManager {
    */
   ValidTxnList getValidTxns() throws LockException;
 
+ /**
+  * Get the transactions that are currently valid.  The resulting
+  * {@link ValidTxnList} object can be passed as string to the processing
+  * tasks for use in the reading the data.  This call should be made once up
+  * front by the planner and should never be called on the backend,
+  * as this will violate the isolation level semantics.
+  * @return list of valid transactions.
+  * @param  excludeTxnTypes list of transaction types that should be excluded.
+  * @throws LockException
+  */
+  ValidTxnList getValidTxns(List<TxnType> excludeTxnTypes) throws LockException;
+
   /**
    * Get the table write Ids that are valid for the current transaction.  The resulting
    * {@link ValidTxnWriteIdList} object can be passed as string to the processing
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 32f9932..b18f98b 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
@@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.lockmgr;
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hive.common.JavaUtils;
+import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.common.ValidWriteIdList;
 import org.apache.hadoop.hive.metastore.MetastoreTaskThread;
 import org.apache.hadoop.hive.metastore.api.AddDynamicPartitions;
@@ -30,6 +31,7 @@ import org.apache.hadoop.hive.metastore.api.LockType;
 import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
 import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
 import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement;
+import org.apache.hadoop.hive.metastore.api.TxnType;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.txn.AcidHouseKeeperService;
 import org.apache.hadoop.hive.ql.DriverFactory;
@@ -2984,6 +2986,50 @@ public class TestDbTxnManager2 extends DbTxnManagerEndToEndTestBase{
     driver.run("drop database if exists temp cascade");
   }
 
+  @Test
+  public void testValidTxnList() throws Exception {
+    long readTxnId = txnMgr.openTxn(ctx, "u0", TxnType.READ_ONLY);
+    HiveTxnManager txnManager1 = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
+    txnManager1.openTxn(ctx, "u0");
+    //Excludes open read only txns by default
+    ValidTxnList validTxns = txnManager1.getValidTxns();
+    Assert.assertEquals(0, validTxns.getInvalidTransactions().length);
+
+    //Exclude open repl created only txns
+    validTxns = txnManager1.getValidTxns(Arrays.asList(TxnType.REPL_CREATED));
+    Assert.assertEquals(1, validTxns.getInvalidTransactions().length);
+    Assert.assertEquals(readTxnId, validTxns.getInvalidTransactions()[0]);
+    txnManager1.commitTxn();
+    txnMgr.commitTxn();
+
+    long replTxnId = txnMgr.openTxn(ctx, "u0", TxnType.REPL_CREATED);
+    txnManager1 = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
+    txnManager1.openTxn(ctx, "u0");
+    //Excludes open read only txns by default
+    validTxns = txnManager1.getValidTxns();
+    Assert.assertEquals(1, validTxns.getInvalidTransactions().length);
+    Assert.assertEquals(replTxnId, validTxns.getInvalidTransactions()[0]);
+
+    //Exclude open repl created only txns
+    validTxns = txnManager1.getValidTxns(Arrays.asList(TxnType.REPL_CREATED));
+    Assert.assertEquals(0, validTxns.getInvalidTransactions().length);
+
+    //Exclude open read only txns
+    validTxns = txnManager1.getValidTxns(Arrays.asList(TxnType.READ_ONLY));
+    Assert.assertEquals(1, validTxns.getInvalidTransactions().length);
+    Assert.assertEquals(replTxnId, validTxns.getInvalidTransactions()[0]);
+    txnMgr.commitTxn();
+
+    //Transaction is committed. So no open txn
+    validTxns = txnManager1.getValidTxns();
+    Assert.assertEquals(0, validTxns.getInvalidTransactions().length);
+
+    //Exclude open read only txns
+    validTxns = txnManager1.getValidTxns(Arrays.asList(TxnType.READ_ONLY));
+    Assert.assertEquals(0, validTxns.getInvalidTransactions().length);
+    txnManager1.commitTxn();
+  }
+
   @Rule
   public TemporaryFolder exportFolder = new TemporaryFolder();
 
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsRequest.java
new file mode 100644
index 0000000..2bb1710
--- /dev/null
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsRequest.java
@@ -0,0 +1,441 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class GetOpenTxnsRequest implements org.apache.thrift.TBase<GetOpenTxnsRequest, GetOpenTxnsRequest._Fields>, java.io.Serializable, Cloneable, Comparable<GetOpenTxnsRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetOpenTxnsRequest");
+
+  private static final org.apache.thrift.protocol.TField EXCLUDE_TXN_TYPES_FIELD_DESC = new org.apache.thrift.protocol.TField("excludeTxnTypes", org.apache.thrift.protocol.TType.LIST, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new GetOpenTxnsRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new GetOpenTxnsRequestTupleSchemeFactory());
+  }
+
+  private List<TxnType> excludeTxnTypes; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    EXCLUDE_TXN_TYPES((short)1, "excludeTxnTypes");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // EXCLUDE_TXN_TYPES
+          return EXCLUDE_TXN_TYPES;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  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.EXCLUDE_TXN_TYPES, new org.apache.thrift.meta_data.FieldMetaData("excludeTxnTypes", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TxnType.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetOpenTxnsRequest.class, metaDataMap);
+  }
+
+  public GetOpenTxnsRequest() {
+  }
+
+  public GetOpenTxnsRequest(
+    List<TxnType> excludeTxnTypes)
+  {
+    this();
+    this.excludeTxnTypes = excludeTxnTypes;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public GetOpenTxnsRequest(GetOpenTxnsRequest other) {
+    if (other.isSetExcludeTxnTypes()) {
+      List<TxnType> __this__excludeTxnTypes = new ArrayList<TxnType>(other.excludeTxnTypes.size());
+      for (TxnType other_element : other.excludeTxnTypes) {
+        __this__excludeTxnTypes.add(other_element);
+      }
+      this.excludeTxnTypes = __this__excludeTxnTypes;
+    }
+  }
+
+  public GetOpenTxnsRequest deepCopy() {
+    return new GetOpenTxnsRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.excludeTxnTypes = null;
+  }
+
+  public int getExcludeTxnTypesSize() {
+    return (this.excludeTxnTypes == null) ? 0 : this.excludeTxnTypes.size();
+  }
+
+  public java.util.Iterator<TxnType> getExcludeTxnTypesIterator() {
+    return (this.excludeTxnTypes == null) ? null : this.excludeTxnTypes.iterator();
+  }
+
+  public void addToExcludeTxnTypes(TxnType elem) {
+    if (this.excludeTxnTypes == null) {
+      this.excludeTxnTypes = new ArrayList<TxnType>();
+    }
+    this.excludeTxnTypes.add(elem);
+  }
+
+  public List<TxnType> getExcludeTxnTypes() {
+    return this.excludeTxnTypes;
+  }
+
+  public void setExcludeTxnTypes(List<TxnType> excludeTxnTypes) {
+    this.excludeTxnTypes = excludeTxnTypes;
+  }
+
+  public void unsetExcludeTxnTypes() {
+    this.excludeTxnTypes = null;
+  }
+
+  /** Returns true if field excludeTxnTypes is set (has been assigned a value) and false otherwise */
+  public boolean isSetExcludeTxnTypes() {
+    return this.excludeTxnTypes != null;
+  }
+
+  public void setExcludeTxnTypesIsSet(boolean value) {
+    if (!value) {
+      this.excludeTxnTypes = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case EXCLUDE_TXN_TYPES:
+      if (value == null) {
+        unsetExcludeTxnTypes();
+      } else {
+        setExcludeTxnTypes((List<TxnType>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case EXCLUDE_TXN_TYPES:
+      return getExcludeTxnTypes();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case EXCLUDE_TXN_TYPES:
+      return isSetExcludeTxnTypes();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof GetOpenTxnsRequest)
+      return this.equals((GetOpenTxnsRequest)that);
+    return false;
+  }
+
+  public boolean equals(GetOpenTxnsRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_excludeTxnTypes = true && this.isSetExcludeTxnTypes();
+    boolean that_present_excludeTxnTypes = true && that.isSetExcludeTxnTypes();
+    if (this_present_excludeTxnTypes || that_present_excludeTxnTypes) {
+      if (!(this_present_excludeTxnTypes && that_present_excludeTxnTypes))
+        return false;
+      if (!this.excludeTxnTypes.equals(that.excludeTxnTypes))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_excludeTxnTypes = true && (isSetExcludeTxnTypes());
+    list.add(present_excludeTxnTypes);
+    if (present_excludeTxnTypes)
+      list.add(excludeTxnTypes);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(GetOpenTxnsRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetExcludeTxnTypes()).compareTo(other.isSetExcludeTxnTypes());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetExcludeTxnTypes()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.excludeTxnTypes, other.excludeTxnTypes);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("GetOpenTxnsRequest(");
+    boolean first = true;
+
+    sb.append("excludeTxnTypes:");
+    if (this.excludeTxnTypes == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.excludeTxnTypes);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetExcludeTxnTypes()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'excludeTxnTypes' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class GetOpenTxnsRequestStandardSchemeFactory implements SchemeFactory {
+    public GetOpenTxnsRequestStandardScheme getScheme() {
+      return new GetOpenTxnsRequestStandardScheme();
+    }
+  }
+
+  private static class GetOpenTxnsRequestStandardScheme extends StandardScheme<GetOpenTxnsRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, GetOpenTxnsRequest struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // EXCLUDE_TXN_TYPES
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list1288 = iprot.readListBegin();
+                struct.excludeTxnTypes = new ArrayList<TxnType>(_list1288.size);
+                TxnType _elem1289;
+                for (int _i1290 = 0; _i1290 < _list1288.size; ++_i1290)
+                {
+                  _elem1289 = org.apache.hadoop.hive.metastore.api.TxnType.findByValue(iprot.readI32());
+                  struct.excludeTxnTypes.add(_elem1289);
+                }
+                iprot.readListEnd();
+              }
+              struct.setExcludeTxnTypesIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, GetOpenTxnsRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.excludeTxnTypes != null) {
+        oprot.writeFieldBegin(EXCLUDE_TXN_TYPES_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, struct.excludeTxnTypes.size()));
+          for (TxnType _iter1291 : struct.excludeTxnTypes)
+          {
+            oprot.writeI32(_iter1291.getValue());
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class GetOpenTxnsRequestTupleSchemeFactory implements SchemeFactory {
+    public GetOpenTxnsRequestTupleScheme getScheme() {
+      return new GetOpenTxnsRequestTupleScheme();
+    }
+  }
+
+  private static class GetOpenTxnsRequestTupleScheme extends TupleScheme<GetOpenTxnsRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, GetOpenTxnsRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.excludeTxnTypes.size());
+        for (TxnType _iter1292 : struct.excludeTxnTypes)
+        {
+          oprot.writeI32(_iter1292.getValue());
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, GetOpenTxnsRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TList _list1293 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32());
+        struct.excludeTxnTypes = new ArrayList<TxnType>(_list1293.size);
+        TxnType _elem1294;
+        for (int _i1295 = 0; _i1295 < _list1293.size; ++_i1295)
+        {
+          _elem1294 = org.apache.hadoop.hive.metastore.api.TxnType.findByValue(iprot.readI32());
+          struct.excludeTxnTypes.add(_elem1294);
+        }
+      }
+      struct.setExcludeTxnTypesIsSet(true);
+    }
+  }
+
+}
+
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index 4863551..809d2cd 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@ -530,6 +530,8 @@ import org.slf4j.LoggerFactory;
 
     public ReplicationMetricList get_replication_metrics(GetReplicationMetricsRequest rqst) throws MetaException, org.apache.thrift.TException;
 
+    public GetOpenTxnsResponse get_open_txns_req(GetOpenTxnsRequest getOpenTxnsRequest) throws org.apache.thrift.TException;
+
   }
 
   @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public interface AsyncIface extends com.facebook.fb303.FacebookService .AsyncIface {
@@ -1022,6 +1024,8 @@ import org.slf4j.LoggerFactory;
 
     public void get_replication_metrics(GetReplicationMetricsRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void get_open_txns_req(GetOpenTxnsRequest getOpenTxnsRequest, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
   }
 
   @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class Client extends com.facebook.fb303.FacebookService.Client implements Iface {
@@ -7993,6 +7997,29 @@ import org.slf4j.LoggerFactory;
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_replication_metrics failed: unknown result");
     }
 
+    public GetOpenTxnsResponse get_open_txns_req(GetOpenTxnsRequest getOpenTxnsRequest) throws org.apache.thrift.TException
+    {
+      send_get_open_txns_req(getOpenTxnsRequest);
+      return recv_get_open_txns_req();
+    }
+
+    public void send_get_open_txns_req(GetOpenTxnsRequest getOpenTxnsRequest) throws org.apache.thrift.TException
+    {
+      get_open_txns_req_args args = new get_open_txns_req_args();
+      args.setGetOpenTxnsRequest(getOpenTxnsRequest);
+      sendBase("get_open_txns_req", args);
+    }
+
+    public GetOpenTxnsResponse recv_get_open_txns_req() throws org.apache.thrift.TException
+    {
+      get_open_txns_req_result result = new get_open_txns_req_result();
+      receiveBase(result, "get_open_txns_req");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_open_txns_req failed: unknown result");
+    }
+
   }
   @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class AsyncClient extends com.facebook.fb303.FacebookService.AsyncClient implements AsyncIface {
     @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
@@ -16320,6 +16347,38 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    public void get_open_txns_req(GetOpenTxnsRequest getOpenTxnsRequest, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      get_open_txns_req_call method_call = new get_open_txns_req_call(getOpenTxnsRequest, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_open_txns_req_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private GetOpenTxnsRequest getOpenTxnsRequest;
+      public get_open_txns_req_call(GetOpenTxnsRequest getOpenTxnsRequest, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.getOpenTxnsRequest = getOpenTxnsRequest;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_open_txns_req", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        get_open_txns_req_args args = new get_open_txns_req_args();
+        args.setGetOpenTxnsRequest(getOpenTxnsRequest);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public GetOpenTxnsResponse getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_get_open_txns_req();
+      }
+    }
+
   }
 
   @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class Processor<I extends Iface> extends com.facebook.fb303.FacebookService.Processor<I> implements org.apache.thrift.TProcessor {
@@ -16577,6 +16636,7 @@ import org.slf4j.LoggerFactory;
       processMap.put("get_scheduled_query", new get_scheduled_query());
       processMap.put("add_replication_metrics", new add_replication_metrics());
       processMap.put("get_replication_metrics", new get_replication_metrics());
+      processMap.put("get_open_txns_req", new get_open_txns_req());
       return processMap;
     }
 
@@ -22820,6 +22880,26 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_open_txns_req<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_open_txns_req_args> {
+      public get_open_txns_req() {
+        super("get_open_txns_req");
+      }
+
+      public get_open_txns_req_args getEmptyArgsInstance() {
+        return new get_open_txns_req_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public get_open_txns_req_result getResult(I iface, get_open_txns_req_args args) throws org.apache.thrift.TException {
+        get_open_txns_req_result result = new get_open_txns_req_result();
+        result.success = iface.get_open_txns_req(args.getOpenTxnsRequest);
+        return result;
+      }
+    }
+
   }
 
   @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class AsyncProcessor<I extends AsyncIface> extends com.facebook.fb303.FacebookService.AsyncProcessor<I> {
@@ -23077,6 +23157,7 @@ import org.slf4j.LoggerFactory;
       processMap.put("get_scheduled_query", new get_scheduled_query());
       processMap.put("add_replication_metrics", new add_replication_metrics());
       processMap.put("get_replication_metrics", new get_replication_metrics());
+      processMap.put("get_open_txns_req", new get_open_txns_req());
       return processMap;
     }
 
@@ -37970,6 +38051,57 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_open_txns_req<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_open_txns_req_args, GetOpenTxnsResponse> {
+      public get_open_txns_req() {
+        super("get_open_txns_req");
+      }
+
+      public get_open_txns_req_args getEmptyArgsInstance() {
+        return new get_open_txns_req_args();
+      }
+
+      public AsyncMethodCallback<GetOpenTxnsResponse> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<GetOpenTxnsResponse>() { 
+          public void onComplete(GetOpenTxnsResponse o) {
+            get_open_txns_req_result result = new get_open_txns_req_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            get_open_txns_req_result result = new get_open_txns_req_result();
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, get_open_txns_req_args args, org.apache.thrift.async.AsyncMethodCallback<GetOpenTxnsResponse> resultHandler) throws TException {
+        iface.get_open_txns_req(args.getOpenTxnsRequest,resultHandler);
+      }
+    }
+
   }
 
   @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class getMetaConf_args implements org.apache.thrift.TBase<getMetaConf_args, getMetaConf_args._Fields>, java.io.Serializable, Cloneable, Comparable<getMetaConf_args>   {
@@ -48761,13 +48893,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1288 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1288.size);
-                  String _elem1289;
-                  for (int _i1290 = 0; _i1290 < _list1288.size; ++_i1290)
+                  org.apache.thrift.protocol.TList _list1296 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1296.size);
+                  String _elem1297;
+                  for (int _i1298 = 0; _i1298 < _list1296.size; ++_i1298)
                   {
-                    _elem1289 = iprot.readString();
-                    struct.success.add(_elem1289);
+                    _elem1297 = iprot.readString();
+                    struct.success.add(_elem1297);
                   }
                   iprot.readListEnd();
                 }
@@ -48802,9 +48934,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 _iter1291 : struct.success)
+            for (String _iter1299 : struct.success)
             {
-              oprot.writeString(_iter1291);
+              oprot.writeString(_iter1299);
             }
             oprot.writeListEnd();
           }
@@ -48843,9 +48975,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1292 : struct.success)
+            for (String _iter1300 : struct.success)
             {
-              oprot.writeString(_iter1292);
+              oprot.writeString(_iter1300);
             }
           }
         }
@@ -48860,13 +48992,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1293 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1293.size);
-            String _elem1294;
-            for (int _i1295 = 0; _i1295 < _list1293.size; ++_i1295)
+            org.apache.thrift.protocol.TList _list1301 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1301.size);
+            String _elem1302;
+            for (int _i1303 = 0; _i1303 < _list1301.size; ++_i1303)
             {
-              _elem1294 = iprot.readString();
-              struct.success.add(_elem1294);
+              _elem1302 = iprot.readString();
+              struct.success.add(_elem1302);
             }
           }
           struct.setSuccessIsSet(true);
@@ -49520,13 +49652,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1296 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1296.size);
-                  String _elem1297;
-                  for (int _i1298 = 0; _i1298 < _list1296.size; ++_i1298)
+                  org.apache.thrift.protocol.TList _list1304 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1304.size);
+                  String _elem1305;
+                  for (int _i1306 = 0; _i1306 < _list1304.size; ++_i1306)
                   {
-                    _elem1297 = iprot.readString();
-                    struct.success.add(_elem1297);
+                    _elem1305 = iprot.readString();
+                    struct.success.add(_elem1305);
                   }
                   iprot.readListEnd();
                 }
@@ -49561,9 +49693,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 _iter1299 : struct.success)
+            for (String _iter1307 : struct.success)
             {
-              oprot.writeString(_iter1299);
+              oprot.writeString(_iter1307);
             }
             oprot.writeListEnd();
           }
@@ -49602,9 +49734,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1300 : struct.success)
+            for (String _iter1308 : struct.success)
             {
-              oprot.writeString(_iter1300);
+              oprot.writeString(_iter1308);
             }
           }
         }
@@ -49619,13 +49751,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1301 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1301.size);
-            String _elem1302;
-            for (int _i1303 = 0; _i1303 < _list1301.size; ++_i1303)
+            org.apache.thrift.protocol.TList _list1309 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1309.size);
+            String _elem1310;
+            for (int _i1311 = 0; _i1311 < _list1309.size; ++_i1311)
             {
-              _elem1302 = iprot.readString();
-              struct.success.add(_elem1302);
+              _elem1310 = iprot.readString();
+              struct.success.add(_elem1310);
             }
           }
           struct.setSuccessIsSet(true);
@@ -54232,16 +54364,16 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map1304 = iprot.readMapBegin();
-                  struct.success = new HashMap<String,Type>(2*_map1304.size);
-                  String _key1305;
-                  Type _val1306;
-                  for (int _i1307 = 0; _i1307 < _map1304.size; ++_i1307)
+                  org.apache.thrift.protocol.TMap _map1312 = iprot.readMapBegin();
+                  struct.success = new HashMap<String,Type>(2*_map1312.size);
+                  String _key1313;
+                  Type _val1314;
+                  for (int _i1315 = 0; _i1315 < _map1312.size; ++_i1315)
                   {
-                    _key1305 = iprot.readString();
-                    _val1306 = new Type();
-                    _val1306.read(iprot);
-                    struct.success.put(_key1305, _val1306);
+                    _key1313 = iprot.readString();
+                    _val1314 = new Type();
+                    _val1314.read(iprot);
+                    struct.success.put(_key1313, _val1314);
                   }
                   iprot.readMapEnd();
                 }
@@ -54276,10 +54408,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> _iter1308 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter1316 : struct.success.entrySet())
             {
-              oprot.writeString(_iter1308.getKey());
-              _iter1308.getValue().write(oprot);
+              oprot.writeString(_iter1316.getKey());
+              _iter1316.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -54318,10 +54450,10 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Map.Entry<String, Type> _iter1309 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter1317 : struct.success.entrySet())
             {
-              oprot.writeString(_iter1309.getKey());
-              _iter1309.getValue().write(oprot);
+              oprot.writeString(_iter1317.getKey());
+              _iter1317.getValue().write(oprot);
             }
           }
         }
@@ -54336,16 +54468,16 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map1310 = 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*_map1310.size);
-            String _key1311;
-            Type _val1312;
-            for (int _i1313 = 0; _i1313 < _map1310.size; ++_i1313)
+            org.apache.thrift.protocol.TMap _map1318 = 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*_map1318.size);
+            String _key1319;
+            Type _val1320;
+            for (int _i1321 = 0; _i1321 < _map1318.size; ++_i1321)
             {
-              _key1311 = iprot.readString();
-              _val1312 = new Type();
-              _val1312.read(iprot);
-              struct.success.put(_key1311, _val1312);
+              _key1319 = iprot.readString();
+              _val1320 = new Type();
+              _val1320.read(iprot);
+              struct.success.put(_key1319, _val1320);
             }
           }
           struct.setSuccessIsSet(true);
@@ -55380,14 +55512,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1314 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list1314.size);
-                  FieldSchema _elem1315;
-                  for (int _i1316 = 0; _i1316 < _list1314.size; ++_i1316)
+                  org.apache.thrift.protocol.TList _list1322 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list1322.size);
+                  FieldSchema _elem1323;
+                  for (int _i1324 = 0; _i1324 < _list1322.size; ++_i1324)
                   {
-                    _elem1315 = new FieldSchema();
-                    _elem1315.read(iprot);
-                    struct.success.add(_elem1315);
+                    _elem1323 = new FieldSchema();
+                    _elem1323.read(iprot);
+                    struct.success.add(_elem1323);
                   }
                   iprot.readListEnd();
                 }
@@ -55440,9 +55572,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 _iter1317 : struct.success)
+            for (FieldSchema _iter1325 : struct.success)
             {
-              _iter1317.write(oprot);
+              _iter1325.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -55497,9 +55629,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter1318 : struct.success)
+            for (FieldSchema _iter1326 : struct.success)
             {
-              _iter1318.write(oprot);
+              _iter1326.write(oprot);
             }
           }
         }
@@ -55520,14 +55652,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1319 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list1319.size);
-            FieldSchema _elem1320;
-            for (int _i1321 = 0; _i1321 < _list1319.size; ++_i1321)
+            org.apache.thrift.protocol.TList _list1327 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list1327.size);
+            FieldSchema _elem1328;
+            for (int _i1329 = 0; _i1329 < _list1327.size; ++_i1329)
             {
-              _elem1320 = new FieldSchema();
-              _elem1320.read(iprot);
-              struct.success.add(_elem1320);
+              _elem1328 = new FieldSchema();
+              _elem1328.read(iprot);
+              struct.success.add(_elem1328);
             }
           }
           struct.setSuccessIsSet(true);
@@ -56681,14 +56813,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1322 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list1322.size);
-                  FieldSchema _elem1323;
-                  for (int _i1324 = 0; _i1324 < _list1322.size; ++_i1324)
+                  org.apache.thrift.protocol.TList _list1330 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list1330.size);
+                  FieldSchema _elem1331;
+                  for (int _i1332 = 0; _i1332 < _list1330.size; ++_i1332)
                   {
-                    _elem1323 = new FieldSchema();
-                    _elem1323.read(iprot);
-                    struct.success.add(_elem1323);
+                    _elem1331 = new FieldSchema();
+                    _elem1331.read(iprot);
+                    struct.success.add(_elem1331);
                   }
                   iprot.readListEnd();
                 }
@@ -56741,9 +56873,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 _iter1325 : struct.success)
+            for (FieldSchema _iter1333 : struct.success)
             {
-              _iter1325.write(oprot);
+              _iter1333.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -56798,9 +56930,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter1326 : struct.success)
+            for (FieldSchema _iter1334 : struct.success)
             {
-              _iter1326.write(oprot);
+              _iter1334.write(oprot);
             }
           }
         }
@@ -56821,14 +56953,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1327 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list1327.size);
-            FieldSchema _elem1328;
-            for (int _i1329 = 0; _i1329 < _list1327.size; ++_i1329)
+            org.apache.thrift.protocol.TList _list1335 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list1335.size);
+            FieldSchema _elem1336;
+            for (int _i1337 = 0; _i1337 < _list1335.size; ++_i1337)
             {
-              _elem1328 = new FieldSchema();
-              _elem1328.read(iprot);
-              struct.success.add(_elem1328);
+              _elem1336 = new FieldSchema();
+              _elem1336.read(iprot);
+              struct.success.add(_elem1336);
             }
           }
           struct.setSuccessIsSet(true);
@@ -58917,14 +59049,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1330 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list1330.size);
-                  FieldSchema _elem1331;
-                  for (int _i1332 = 0; _i1332 < _list1330.size; ++_i1332)
+                  org.apache.thrift.protocol.TList _list1338 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list1338.size);
+                  FieldSchema _elem1339;
+                  for (int _i1340 = 0; _i1340 < _list1338.size; ++_i1340)
                   {
-                    _elem1331 = new FieldSchema();
-                    _elem1331.read(iprot);
-                    struct.success.add(_elem1331);
+                    _elem1339 = new FieldSchema();
+                    _elem1339.read(iprot);
+                    struct.success.add(_elem1339);
                   }
                   iprot.readListEnd();
                 }
@@ -58977,9 +59109,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 _iter1333 : struct.success)
+            for (FieldSchema _iter1341 : struct.success)
             {
-              _iter1333.write(oprot);
+              _iter1341.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -59034,9 +59166,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter1334 : struct.success)
+            for (FieldSchema _iter1342 : struct.success)
             {
-              _iter1334.write(oprot);
+              _iter1342.write(oprot);
             }
           }
         }
@@ -59057,14 +59189,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1335 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list1335.size);
-            FieldSchema _elem1336;
-            for (int _i1337 = 0; _i1337 < _list1335.size; ++_i1337)
+            org.apache.thrift.protocol.TList _list1343 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list1343.size);
+            FieldSchema _elem1344;
+            for (int _i1345 = 0; _i1345 < _list1343.size; ++_i1345)
             {
-              _elem1336 = new FieldSchema();
-              _elem1336.read(iprot);
-              struct.success.add(_elem1336);
+              _elem1344 = new FieldSchema();
+              _elem1344.read(iprot);
+              struct.success.add(_elem1344);
             }
           }
           struct.setSuccessIsSet(true);
@@ -60218,14 +60350,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1338 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list1338.size);
-                  FieldSchema _elem1339;
-                  for (int _i1340 = 0; _i1340 < _list1338.size; ++_i1340)
+                  org.apache.thrift.protocol.TList _list1346 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list1346.size);
+                  FieldSchema _elem1347;
+                  for (int _i1348 = 0; _i1348 < _list1346.size; ++_i1348)
                   {
-                    _elem1339 = new FieldSchema();
-                    _elem1339.read(iprot);
-                    struct.success.add(_elem1339);
+                    _elem1347 = new FieldSchema();
+                    _elem1347.read(iprot);
+                    struct.success.add(_elem1347);
                   }
                   iprot.readListEnd();
                 }
@@ -60278,9 +60410,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 _iter1341 : struct.success)
+            for (FieldSchema _iter1349 : struct.success)
             {
-              _iter1341.write(oprot);
+              _iter1349.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -60335,9 +60467,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter1342 : struct.success)
+            for (FieldSchema _iter1350 : struct.success)
             {
-              _iter1342.write(oprot);
+              _iter1350.write(oprot);
             }
           }
         }
@@ -60358,14 +60490,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1343 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list1343.size);
-            FieldSchema _elem1344;
-            for (int _i1345 = 0; _i1345 < _list1343.size; ++_i1345)
+            org.apache.thrift.protocol.TList _list1351 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list1351.size);
+            FieldSchema _elem1352;
+            for (int _i1353 = 0; _i1353 < _list1351.size; ++_i1353)
             {
-              _elem1344 = new FieldSchema();
-              _elem1344.read(iprot);
-              struct.success.add(_elem1344);
+              _elem1352 = new FieldSchema();
+              _elem1352.read(iprot);
+              struct.success.add(_elem1352);
             }
           }
           struct.setSuccessIsSet(true);
@@ -64538,14 +64670,14 @@ import org.slf4j.LoggerFactory;
             case 2: // PRIMARY_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1346 = iprot.readListBegin();
-                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1346.size);
-                  SQLPrimaryKey _elem1347;
-                  for (int _i1348 = 0; _i1348 < _list1346.size; ++_i1348)
+                  org.apache.thrift.protocol.TList _list1354 = iprot.readListBegin();
+                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1354.size);
+                  SQLPrimaryKey _elem1355;
+                  for (int _i1356 = 0; _i1356 < _list1354.size; ++_i1356)
                   {
-                    _elem1347 = new SQLPrimaryKey();
-                    _elem1347.read(iprot);
-                    struct.primaryKeys.add(_elem1347);
+                    _elem1355 = new SQLPrimaryKey();
+                    _elem1355.read(iprot);
+                    struct.primaryKeys.add(_elem1355);
                   }
                   iprot.readListEnd();
                 }
@@ -64557,14 +64689,14 @@ import org.slf4j.LoggerFactory;
             case 3: // FOREIGN_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1349 = iprot.readListBegin();
-                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1349.size);
-                  SQLForeignKey _elem1350;
-                  for (int _i1351 = 0; _i1351 < _list1349.size; ++_i1351)
+                  org.apache.thrift.protocol.TList _list1357 = iprot.readListBegin();
+                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1357.size);
+                  SQLForeignKey _elem1358;
+                  for (int _i1359 = 0; _i1359 < _list1357.size; ++_i1359)
                   {
-                    _elem1350 = new SQLForeignKey();
-                    _elem1350.read(iprot);
-                    struct.foreignKeys.add(_elem1350);
+                    _elem1358 = new SQLForeignKey();
+                    _elem1358.read(iprot);
+                    struct.foreignKeys.add(_elem1358);
                   }
                   iprot.readListEnd();
                 }
@@ -64576,14 +64708,14 @@ import org.slf4j.LoggerFactory;
             case 4: // UNIQUE_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1352 = iprot.readListBegin();
-                  struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1352.size);
-                  SQLUniqueConstraint _elem1353;
-                  for (int _i1354 = 0; _i1354 < _list1352.size; ++_i1354)
+                  org.apache.thrift.protocol.TList _list1360 = iprot.readListBegin();
+                  struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1360.size);
+                  SQLUniqueConstraint _elem1361;
+                  for (int _i1362 = 0; _i1362 < _list1360.size; ++_i1362)
                   {
-                    _elem1353 = new SQLUniqueConstraint();
-                    _elem1353.read(iprot);
-                    struct.uniqueConstraints.add(_elem1353);
+                    _elem1361 = new SQLUniqueConstraint();
+                    _elem1361.read(iprot);
+                    struct.uniqueConstraints.add(_elem1361);
                   }
                   iprot.readListEnd();
                 }
@@ -64595,14 +64727,14 @@ import org.slf4j.LoggerFactory;
             case 5: // NOT_NULL_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1355 = iprot.readListBegin();
-                  struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1355.size);
-                  SQLNotNullConstraint _elem1356;
-                  for (int _i1357 = 0; _i1357 < _list1355.size; ++_i1357)
+                  org.apache.thrift.protocol.TList _list1363 = iprot.readListBegin();
+                  struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1363.size);
+                  SQLNotNullConstraint _elem1364;
+                  for (int _i1365 = 0; _i1365 < _list1363.size; ++_i1365)
                   {
-                    _elem1356 = new SQLNotNullConstraint();
-                    _elem1356.read(iprot);
-                    struct.notNullConstraints.add(_elem1356);
+                    _elem1364 = new SQLNotNullConstraint();
+                    _elem1364.read(iprot);
+                    struct.notNullConstraints.add(_elem1364);
                   }
                   iprot.readListEnd();
                 }
@@ -64614,14 +64746,14 @@ import org.slf4j.LoggerFactory;
             case 6: // DEFAULT_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1358 = iprot.readListBegin();
-                  struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1358.size);
-                  SQLDefaultConstraint _elem1359;
-                  for (int _i1360 = 0; _i1360 < _list1358.size; ++_i1360)
+                  org.apache.thrift.protocol.TList _list1366 = iprot.readListBegin();
+                  struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1366.size);
+                  SQLDefaultConstraint _elem1367;
+                  for (int _i1368 = 0; _i1368 < _list1366.size; ++_i1368)
                   {
-                    _elem1359 = new SQLDefaultConstraint();
-                    _elem1359.read(iprot);
-                    struct.defaultConstraints.add(_elem1359);
+                    _elem1367 = new SQLDefaultConstraint();
+                    _elem1367.read(iprot);
+                    struct.defaultConstraints.add(_elem1367);
                   }
                   iprot.readListEnd();
                 }
@@ -64633,14 +64765,14 @@ import org.slf4j.LoggerFactory;
             case 7: // CHECK_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1361 = iprot.readListBegin();
-                  struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1361.size);
-                  SQLCheckConstraint _elem1362;
-                  for (int _i1363 = 0; _i1363 < _list1361.size; ++_i1363)
+                  org.apache.thrift.protocol.TList _list1369 = iprot.readListBegin();
+                  struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1369.size);
+                  SQLCheckConstraint _elem1370;
+                  for (int _i1371 = 0; _i1371 < _list1369.size; ++_i1371)
                   {
-                    _elem1362 = new SQLCheckConstraint();
-                    _elem1362.read(iprot);
-                    struct.checkConstraints.add(_elem1362);
+                    _elem1370 = new SQLCheckConstraint();
+                    _elem1370.read(iprot);
+                    struct.checkConstraints.add(_elem1370);
                   }
                   iprot.readListEnd();
                 }
@@ -64671,9 +64803,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 _iter1364 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter1372 : struct.primaryKeys)
             {
-              _iter1364.write(oprot);
+              _iter1372.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -64683,9 +64815,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 _iter1365 : struct.foreignKeys)
+            for (SQLForeignKey _iter1373 : struct.foreignKeys)
             {
-              _iter1365.write(oprot);
+              _iter1373.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -64695,9 +64827,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 _iter1366 : struct.uniqueConstraints)
+            for (SQLUniqueConstraint _iter1374 : struct.uniqueConstraints)
             {
-              _iter1366.write(oprot);
+              _iter1374.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -64707,9 +64839,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 _iter1367 : struct.notNullConstraints)
+            for (SQLNotNullConstraint _iter1375 : struct.notNullConstraints)
             {
-              _iter1367.write(oprot);
+              _iter1375.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -64719,9 +64851,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 _iter1368 : struct.defaultConstraints)
+            for (SQLDefaultConstraint _iter1376 : struct.defaultConstraints)
             {
-              _iter1368.write(oprot);
+              _iter1376.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -64731,9 +64863,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 _iter1369 : struct.checkConstraints)
+            for (SQLCheckConstraint _iter1377 : struct.checkConstraints)
             {
-              _iter1369.write(oprot);
+              _iter1377.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -64785,54 +64917,54 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPrimaryKeys()) {
           {
             oprot.writeI32(struct.primaryKeys.size());
-            for (SQLPrimaryKey _iter1370 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter1378 : struct.primaryKeys)
             {
-              _iter1370.write(oprot);
+              _iter1378.write(oprot);
             }
           }
         }
         if (struct.isSetForeignKeys()) {
           {
             oprot.writeI32(struct.foreignKeys.size());
-            for (SQLForeignKey _iter1371 : struct.foreignKeys)
+            for (SQLForeignKey _iter1379 : struct.foreignKeys)
             {
-              _iter1371.write(oprot);
+              _iter1379.write(oprot);
             }
           }
         }
         if (struct.isSetUniqueConstraints()) {
           {
             oprot.writeI32(struct.uniqueConstraints.size());
-            for (SQLUniqueConstraint _iter1372 : struct.uniqueConstraints)
+            for (SQLUniqueConstraint _iter1380 : struct.uniqueConstraints)
             {
-              _iter1372.write(oprot);
+              _iter1380.write(oprot);
             }
           }
         }
         if (struct.isSetNotNullConstraints()) {
           {
             oprot.writeI32(struct.notNullConstraints.size());
-            for (SQLNotNullConstraint _iter1373 : struct.notNullConstraints)
+            for (SQLNotNullConstraint _iter1381 : struct.notNullConstraints)
             {
-              _iter1373.write(oprot);
+              _iter1381.write(oprot);
             }
           }
         }
         if (struct.isSetDefaultConstraints()) {
           {
             oprot.writeI32(struct.defaultConstraints.size());
-            for (SQLDefaultConstraint _iter1374 : struct.defaultConstraints)
+            for (SQLDefaultConstraint _iter1382 : struct.defaultConstraints)
             {
-              _iter1374.write(oprot);
+              _iter1382.write(oprot);
             }
           }
         }
         if (struct.isSetCheckConstraints()) {
           {
             oprot.writeI32(struct.checkConstraints.size());
-            for (SQLCheckConstraint _iter1375 : struct.checkConstraints)
+            for (SQLCheckConstraint _iter1383 : struct.checkConstraints)
             {
-              _iter1375.write(oprot);
+              _iter1383.write(oprot);
             }
           }
         }
@@ -64849,84 +64981,84 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list1376 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1376.size);
-            SQLPrimaryKey _elem1377;
-            for (int _i1378 = 0; _i1378 < _list1376.size; ++_i1378)
+            org.apache.thrift.protocol.TList _list1384 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1384.size);
+            SQLPrimaryKey _elem1385;
+            for (int _i1386 = 0; _i1386 < _list1384.size; ++_i1386)
             {
-              _elem1377 = new SQLPrimaryKey();
-              _elem1377.read(iprot);
-              struct.primaryKeys.add(_elem1377);
+              _elem1385 = new SQLPrimaryKey();
+              _elem1385.read(iprot);
+              struct.primaryKeys.add(_elem1385);
             }
           }
           struct.setPrimaryKeysIsSet(true);
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1379 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1379.size);
-            SQLForeignKey _elem1380;
-            for (int _i1381 = 0; _i1381 < _list1379.size; ++_i1381)
+            org.apache.thrift.protocol.TList _list1387 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1387.size);
+            SQLForeignKey _elem1388;
+            for (int _i1389 = 0; _i1389 < _list1387.size; ++_i1389)
             {
-              _elem1380 = new SQLForeignKey();
-              _elem1380.read(iprot);
-              struct.foreignKeys.add(_elem1380);
+              _elem1388 = new SQLForeignKey();
+              _elem1388.read(iprot);
+              struct.foreignKeys.add(_elem1388);
             }
           }
           struct.setForeignKeysIsSet(true);
         }
         if (incoming.get(3)) {
           {
-            org.apache.thrift.protocol.TList _list1382 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1382.size);
-            SQLUniqueConstraint _elem1383;
-            for (int _i1384 = 0; _i1384 < _list1382.size; ++_i1384)
+            org.apache.thrift.protocol.TList _list1390 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1390.size);
+            SQLUniqueConstraint _elem1391;
+            for (int _i1392 = 0; _i1392 < _list1390.size; ++_i1392)
             {
-              _elem1383 = new SQLUniqueConstraint();
-              _elem1383.read(iprot);
-              struct.uniqueConstraints.add(_elem1383);
+              _elem1391 = new SQLUniqueConstraint();
+              _elem1391.read(iprot);
+              struct.uniqueConstraints.add(_elem1391);
             }
           }
           struct.setUniqueConstraintsIsSet(true);
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol.TList _list1385 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1385.size);
-            SQLNotNullConstraint _elem1386;
-            for (int _i1387 = 0; _i1387 < _list1385.size; ++_i1387)
+            org.apache.thrift.protocol.TList _list1393 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1393.size);
+            SQLNotNullConstraint _elem1394;
+            for (int _i1395 = 0; _i1395 < _list1393.size; ++_i1395)
             {
-              _elem1386 = new SQLNotNullConstraint();
-              _elem1386.read(iprot);
-              struct.notNullConstraints.add(_elem1386);
+              _elem1394 = new SQLNotNullConstraint();
+              _elem1394.read(iprot);
+              struct.notNullConstraints.add(_elem1394);
             }
           }
           struct.setNotNullConstraintsIsSet(true);
         }
         if (incoming.get(5)) {
           {
-            org.apache.thrift.protocol.TList _list1388 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1388.size);
-            SQLDefaultConstraint _elem1389;
-            for (int _i1390 = 0; _i1390 < _list1388.size; ++_i1390)
+            org.apache.thrift.protocol.TList _list1396 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1396.size);
+            SQLDefaultConstraint _elem1397;
+            for (int _i1398 = 0; _i1398 < _list1396.size; ++_i1398)
             {
-              _elem1389 = new SQLDefaultConstraint();
-              _elem1389.read(iprot);
-              struct.defaultConstraints.add(_elem1389);
+              _elem1397 = new SQLDefaultConstraint();
+              _elem1397.read(iprot);
+              struct.defaultConstraints.add(_elem1397);
             }
           }
           struct.setDefaultConstraintsIsSet(true);
         }
         if (incoming.get(6)) {
           {
-            org.apache.thrift.protocol.TList _list1391 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1391.size);
-            SQLCheckConstraint _elem1392;
-            for (int _i1393 = 0; _i1393 < _list1391.size; ++_i1393)
+            org.apache.thrift.protocol.TList _list1399 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1399.size);
+            SQLCheckConstraint _elem1400;
+            for (int _i1401 = 0; _i1401 < _list1399.size; ++_i1401)
             {
-              _elem1392 = new SQLCheckConstraint();
-              _elem1392.read(iprot);
-              struct.checkConstraints.add(_elem1392);
+              _elem1400 = new SQLCheckConstraint();
+              _elem1400.read(iprot);
+              struct.checkConstraints.add(_elem1400);
             }
           }
           struct.setCheckConstraintsIsSet(true);
@@ -75117,13 +75249,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1394 = iprot.readListBegin();
-                  struct.partNames = new ArrayList<String>(_list1394.size);
-                  String _elem1395;
-                  for (int _i1396 = 0; _i1396 < _list1394.size; ++_i1396)
+                  org.apache.thrift.protocol.TList _list1402 = iprot.readListBegin();
+                  struct.partNames = new ArrayList<String>(_list1402.size);
+                  String _elem1403;
+                  for (int _i1404 = 0; _i1404 < _list1402.size; ++_i1404)
                   {
-                    _elem1395 = iprot.readString();
-                    struct.partNames.add(_elem1395);
+                    _elem1403 = iprot.readString();
+                    struct.partNames.add(_elem1403);
                   }
                   iprot.readListEnd();
                 }
@@ -75159,9 +75291,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 _iter1397 : struct.partNames)
+            for (String _iter1405 : struct.partNames)
             {
-              oprot.writeString(_iter1397);
+              oprot.writeString(_iter1405);
             }
             oprot.writeListEnd();
           }
@@ -75204,9 +75336,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPartNames()) {
           {
             oprot.writeI32(struct.partNames.size());
-            for (String _iter1398 : struct.partNames)
+            for (String _iter1406 : struct.partNames)
             {
-              oprot.writeString(_iter1398);
+              oprot.writeString(_iter1406);
             }
           }
         }
@@ -75226,13 +75358,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1399 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.partNames = new ArrayList<String>(_list1399.size);
-            String _elem1400;
-            for (int _i1401 = 0; _i1401 < _list1399.size; ++_i1401)
+            org.apache.thrift.protocol.TList _list1407 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.partNames = new ArrayList<String>(_list1407.size);
+            String _elem1408;
+            for (int _i1409 = 0; _i1409 < _list1407.size; ++_i1409)
             {
-              _elem1400 = iprot.readString();
-              struct.partNames.add(_elem1400);
+              _elem1408 = iprot.readString();
+              struct.partNames.add(_elem1408);
             }
           }
           struct.setPartNamesIsSet(true);
@@ -77289,13 +77421,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1402 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1402.size);
-                  String _elem1403;
-                  for (int _i1404 = 0; _i1404 < _list1402.size; ++_i1404)
+                  org.apache.thrift.protocol.TList _list1410 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1410.size);
+                  String _elem1411;
+                  for (int _i1412 = 0; _i1412 < _list1410.size; ++_i1412)
                   {
-                    _elem1403 = iprot.readString();
-                    struct.success.add(_elem1403);
+                    _elem1411 = iprot.readString();
+                    struct.success.add(_elem1411);
                   }
                   iprot.readListEnd();
                 }
@@ -77330,9 +77462,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 _iter1405 : struct.success)
+            for (String _iter1413 : struct.success)
             {
-              oprot.writeString(_iter1405);
+              oprot.writeString(_iter1413);
             }
             oprot.writeListEnd();
           }
@@ -77371,9 +77503,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1406 : struct.success)
+            for (String _iter1414 : struct.success)
             {
-              oprot.writeString(_iter1406);
+              oprot.writeString(_iter1414);
             }
           }
         }
@@ -77388,13 +77520,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1407 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1407.size);
-            String _elem1408;
-            for (int _i1409 = 0; _i1409 < _list1407.size; ++_i1409)
+            org.apache.thrift.protocol.TList _list1415 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1415.size);
+            String _elem1416;
+            for (int _i1417 = 0; _i1417 < _list1415.size; ++_i1417)
             {
-              _elem1408 = iprot.readString();
-              struct.success.add(_elem1408);
+              _elem1416 = iprot.readString();
+              struct.success.add(_elem1416);
             }
           }
           struct.setSuccessIsSet(true);
@@ -78368,13 +78500,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1410 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1410.size);
-                  String _elem1411;
-                  for (int _i1412 = 0; _i1412 < _list1410.size; ++_i1412)
+                  org.apache.thrift.protocol.TList _list1418 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1418.size);
+                  String _elem1419;
+                  for (int _i1420 = 0; _i1420 < _list1418.size; ++_i1420)
                   {
-                    _elem1411 = iprot.readString();
-                    struct.success.add(_elem1411);
+                    _elem1419 = iprot.readString();
+                    struct.success.add(_elem1419);
                   }
                   iprot.readListEnd();
                 }
@@ -78409,9 +78541,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 _iter1413 : struct.success)
+            for (String _iter1421 : struct.success)
             {
-              oprot.writeString(_iter1413);
+              oprot.writeString(_iter1421);
             }
             oprot.writeListEnd();
           }
@@ -78450,9 +78582,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1414 : struct.success)
+            for (String _iter1422 : struct.success)
             {
-              oprot.writeString(_iter1414);
+              oprot.writeString(_iter1422);
             }
           }
         }
@@ -78467,13 +78599,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1415 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1415.size);
-            String _elem1416;
-            for (int _i1417 = 0; _i1417 < _list1415.size; ++_i1417)
+            org.apache.thrift.protocol.TList _list1423 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1423.size);
+            String _elem1424;
+            for (int _i1425 = 0; _i1425 < _list1423.size; ++_i1425)
             {
-              _elem1416 = iprot.readString();
-              struct.success.add(_elem1416);
+              _elem1424 = iprot.readString();
+              struct.success.add(_elem1424);
             }
           }
           struct.setSuccessIsSet(true);
@@ -79130,14 +79262,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1418 = iprot.readListBegin();
-                  struct.success = new ArrayList<Table>(_list1418.size);
-                  Table _elem1419;
-                  for (int _i1420 = 0; _i1420 < _list1418.size; ++_i1420)
+                  org.apache.thrift.protocol.TList _list1426 = iprot.readListBegin();
+                  struct.success = new ArrayList<Table>(_list1426.size);
+                  Table _elem1427;
+                  for (int _i1428 = 0; _i1428 < _list1426.size; ++_i1428)
                   {
-                    _elem1419 = new Table();
-                    _elem1419.read(iprot);
-                    struct.success.add(_elem1419);
+                    _elem1427 = new Table();
+                    _elem1427.read(iprot);
+                    struct.success.add(_elem1427);
                   }
                   iprot.readListEnd();
                 }
@@ -79172,9 +79304,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 _iter1421 : struct.success)
+            for (Table _iter1429 : struct.success)
             {
-              _iter1421.write(oprot);
+              _iter1429.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -79213,9 +79345,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Table _iter1422 : struct.success)
+            for (Table _iter1430 : struct.success)
             {
-              _iter1422.write(oprot);
+              _iter1430.write(oprot);
             }
           }
         }
@@ -79230,14 +79362,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1423 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Table>(_list1423.size);
-            Table _elem1424;
-            for (int _i1425 = 0; _i1425 < _list1423.size; ++_i1425)
+            org.apache.thrift.protocol.TList _list1431 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Table>(_list1431.size);
+            Table _elem1432;
+            for (int _i1433 = 0; _i1433 < _list1431.size; ++_i1433)
             {
-              _elem1424 = new Table();
-              _elem1424.read(iprot);
-              struct.success.add(_elem1424);
+              _elem1432 = new Table();
+              _elem1432.read(iprot);
+              struct.success.add(_elem1432);
             }
           }
           struct.setSuccessIsSet(true);
@@ -80003,13 +80135,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1426 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1426.size);
-                  String _elem1427;
-                  for (int _i1428 = 0; _i1428 < _list1426.size; ++_i1428)
+                  org.apache.thrift.protocol.TList _list1434 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1434.size);
+                  String _elem1435;
+                  for (int _i1436 = 0; _i1436 < _list1434.size; ++_i1436)
                   {
-                    _elem1427 = iprot.readString();
-                    struct.success.add(_elem1427);
+                    _elem1435 = iprot.readString();
+                    struct.success.add(_elem1435);
                   }
                   iprot.readListEnd();
                 }
@@ -80044,9 +80176,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 _iter1429 : struct.success)
+            for (String _iter1437 : struct.success)
             {
-              oprot.writeString(_iter1429);
+              oprot.writeString(_iter1437);
             }
             oprot.writeListEnd();
           }
@@ -80085,9 +80217,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1430 : struct.success)
+            for (String _iter1438 : struct.success)
             {
-              oprot.writeString(_iter1430);
+              oprot.writeString(_iter1438);
             }
           }
         }
@@ -80102,13 +80234,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1431 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1431.size);
-            String _elem1432;
-            for (int _i1433 = 0; _i1433 < _list1431.size; ++_i1433)
+            org.apache.thrift.protocol.TList _list1439 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1439.size);
+            String _elem1440;
+            for (int _i1441 = 0; _i1441 < _list1439.size; ++_i1441)
             {
-              _elem1432 = iprot.readString();
-              struct.success.add(_elem1432);
+              _elem1440 = iprot.readString();
+              struct.success.add(_elem1440);
             }
           }
           struct.setSuccessIsSet(true);
@@ -80613,13 +80745,13 @@ import org.slf4j.LoggerFactory;
             case 3: // TBL_TYPES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1434 = iprot.readListBegin();
-                  struct.tbl_types = new ArrayList<String>(_list1434.size);
-                  String _elem1435;
-                  for (int _i1436 = 0; _i1436 < _list1434.size; ++_i1436)
+                  org.apache.thrift.protocol.TList _list1442 = iprot.readListBegin();
+                  struct.tbl_types = new ArrayList<String>(_list1442.size);
+                  String _elem1443;
+                  for (int _i1444 = 0; _i1444 < _list1442.size; ++_i1444)
                   {
-                    _elem1435 = iprot.readString();
-                    struct.tbl_types.add(_elem1435);
+                    _elem1443 = iprot.readString();
+                    struct.tbl_types.add(_elem1443);
                   }
                   iprot.readListEnd();
                 }
@@ -80655,9 +80787,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 _iter1437 : struct.tbl_types)
+            for (String _iter1445 : struct.tbl_types)
             {
-              oprot.writeString(_iter1437);
+              oprot.writeString(_iter1445);
             }
             oprot.writeListEnd();
           }
@@ -80700,9 +80832,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetTbl_types()) {
           {
             oprot.writeI32(struct.tbl_types.size());
-            for (String _iter1438 : struct.tbl_types)
+            for (String _iter1446 : struct.tbl_types)
             {
-              oprot.writeString(_iter1438);
+              oprot.writeString(_iter1446);
             }
           }
         }
@@ -80722,13 +80854,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1439 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_types = new ArrayList<String>(_list1439.size);
-            String _elem1440;
-            for (int _i1441 = 0; _i1441 < _list1439.size; ++_i1441)
+            org.apache.thrift.protocol.TList _list1447 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_types = new ArrayList<String>(_list1447.size);
+            String _elem1448;
+            for (int _i1449 = 0; _i1449 < _list1447.size; ++_i1449)
             {
-              _elem1440 = iprot.readString();
-              struct.tbl_types.add(_elem1440);
+              _elem1448 = iprot.readString();
+              struct.tbl_types.add(_elem1448);
             }
           }
           struct.setTbl_typesIsSet(true);
@@ -81134,14 +81266,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1442 = iprot.readListBegin();
-                  struct.success = new ArrayList<TableMeta>(_list1442.size);
-                  TableMeta _elem1443;
-                  for (int _i1444 = 0; _i1444 < _list1442.size; ++_i1444)
+                  org.apache.thrift.protocol.TList _list1450 = iprot.readListBegin();
+                  struct.success = new ArrayList<TableMeta>(_list1450.size);
+                  TableMeta _elem1451;
+                  for (int _i1452 = 0; _i1452 < _list1450.size; ++_i1452)
                   {
-                    _elem1443 = new TableMeta();
-                    _elem1443.read(iprot);
-                    struct.success.add(_elem1443);
+                    _elem1451 = new TableMeta();
+                    _elem1451.read(iprot);
+                    struct.success.add(_elem1451);
                   }
                   iprot.readListEnd();
                 }
@@ -81176,9 +81308,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 _iter1445 : struct.success)
+            for (TableMeta _iter1453 : struct.success)
             {
-              _iter1445.write(oprot);
+              _iter1453.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -81217,9 +81349,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (TableMeta _iter1446 : struct.success)
+            for (TableMeta _iter1454 : struct.success)
             {
-              _iter1446.write(oprot);
+              _iter1454.write(oprot);
             }
           }
         }
@@ -81234,14 +81366,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1447 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<TableMeta>(_list1447.size);
-            TableMeta _elem1448;
-            for (int _i1449 = 0; _i1449 < _list1447.size; ++_i1449)
+            org.apache.thrift.protocol.TList _list1455 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<TableMeta>(_list1455.size);
+            TableMeta _elem1456;
+            for (int _i1457 = 0; _i1457 < _list1455.size; ++_i1457)
             {
-              _elem1448 = new TableMeta();
-              _elem1448.read(iprot);
-              struct.success.add(_elem1448);
+              _elem1456 = new TableMeta();
+              _elem1456.read(iprot);
+              struct.success.add(_elem1456);
             }
           }
           struct.setSuccessIsSet(true);
@@ -82007,13 +82139,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1450 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1450.size);
-                  String _elem1451;
-                  for (int _i1452 = 0; _i1452 < _list1450.size; ++_i1452)
+                  org.apache.thrift.protocol.TList _list1458 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1458.size);
+                  String _elem1459;
+                  for (int _i1460 = 0; _i1460 < _list1458.size; ++_i1460)
                   {
-                    _elem1451 = iprot.readString();
-                    struct.success.add(_elem1451);
+                    _elem1459 = iprot.readString();
+                    struct.success.add(_elem1459);
                   }
                   iprot.readListEnd();
                 }
@@ -82048,9 +82180,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 _iter1453 : struct.success)
+            for (String _iter1461 : struct.success)
             {
-              oprot.writeString(_iter1453);
+              oprot.writeString(_iter1461);
             }
             oprot.writeListEnd();
           }
@@ -82089,9 +82221,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1454 : struct.success)
+            for (String _iter1462 : struct.success)
             {
-              oprot.writeString(_iter1454);
+              oprot.writeString(_iter1462);
             }
           }
         }
@@ -82106,13 +82238,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1455 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1455.size);
-            String _elem1456;
-            for (int _i1457 = 0; _i1457 < _list1455.size; ++_i1457)
+            org.apache.thrift.protocol.TList _list1463 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1463.size);
+            String _elem1464;
+            for (int _i1465 = 0; _i1465 < _list1463.size; ++_i1465)
             {
-              _elem1456 = iprot.readString();
-              struct.success.add(_elem1456);
+              _elem1464 = iprot.readString();
+              struct.success.add(_elem1464);
             }
           }
           struct.setSuccessIsSet(true);
@@ -83565,13 +83697,13 @@ import org.slf4j.LoggerFactory;
             case 2: // TBL_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1458 = iprot.readListBegin();
-                  struct.tbl_names = new ArrayList<String>(_list1458.size);
-                  String _elem1459;
-                  for (int _i1460 = 0; _i1460 < _list1458.size; ++_i1460)
+                  org.apache.thrift.protocol.TList _list1466 = iprot.readListBegin();
+                  struct.tbl_names = new ArrayList<String>(_list1466.size);
+                  String _elem1467;
+                  for (int _i1468 = 0; _i1468 < _list1466.size; ++_i1468)
                   {
-                    _elem1459 = iprot.readString();
-                    struct.tbl_names.add(_elem1459);
+                    _elem1467 = iprot.readString();
+                    struct.tbl_names.add(_elem1467);
                   }
                   iprot.readListEnd();
                 }
@@ -83602,9 +83734,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 _iter1461 : struct.tbl_names)
+            for (String _iter1469 : struct.tbl_names)
             {
-              oprot.writeString(_iter1461);
+              oprot.writeString(_iter1469);
             }
             oprot.writeListEnd();
           }
@@ -83641,9 +83773,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetTbl_names()) {
           {
             oprot.writeI32(struct.tbl_names.size());
-            for (String _iter1462 : struct.tbl_names)
+            for (String _iter1470 : struct.tbl_names)
             {
-              oprot.writeString(_iter1462);
+              oprot.writeString(_iter1470);
             }
           }
         }
@@ -83659,13 +83791,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list1463 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_names = new ArrayList<String>(_list1463.size);
-            String _elem1464;
-            for (int _i1465 = 0; _i1465 < _list1463.size; ++_i1465)
+            org.apache.thrift.protocol.TList _list1471 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_names = new ArrayList<String>(_list1471.size);
+            String _elem1472;
+            for (int _i1473 = 0; _i1473 < _list1471.size; ++_i1473)
             {
-              _elem1464 = iprot.readString();
-              struct.tbl_names.add(_elem1464);
+              _elem1472 = iprot.readString();
+              struct.tbl_names.add(_elem1472);
             }
           }
           struct.setTbl_namesIsSet(true);
@@ -83990,14 +84122,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1466 = iprot.readListBegin();
-                  struct.success = new ArrayList<Table>(_list1466.size);
-                  Table _elem1467;
-                  for (int _i1468 = 0; _i1468 < _list1466.size; ++_i1468)
+                  org.apache.thrift.protocol.TList _list1474 = iprot.readListBegin();
+                  struct.success = new ArrayList<Table>(_list1474.size);
+                  Table _elem1475;
+                  for (int _i1476 = 0; _i1476 < _list1474.size; ++_i1476)
                   {
-                    _elem1467 = new Table();
-                    _elem1467.read(iprot);
-                    struct.success.add(_elem1467);
+                    _elem1475 = new Table();
+                    _elem1475.read(iprot);
+                    struct.success.add(_elem1475);
                   }
                   iprot.readListEnd();
                 }
@@ -84023,9 +84155,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 _iter1469 : struct.success)
+            for (Table _iter1477 : struct.success)
             {
-              _iter1469.write(oprot);
+              _iter1477.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -84056,9 +84188,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Table _iter1470 : struct.success)
+            for (Table _iter1478 : struct.success)
             {
-              _iter1470.write(oprot);
+              _iter1478.write(oprot);
             }
           }
         }
@@ -84070,14 +84202,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1471 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Table>(_list1471.size);
-            Table _elem1472;
-            for (int _i1473 = 0; _i1473 < _list1471.size; ++_i1473)
+            org.apache.thrift.protocol.TList _list1479 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Table>(_list1479.size);
+            Table _elem1480;
+            for (int _i1481 = 0; _i1481 < _list1479.size; ++_i1481)
             {
-              _elem1472 = new Table();
-              _elem1472.read(iprot);
-              struct.success.add(_elem1472);
+              _elem1480 = new Table();
+              _elem1480.read(iprot);
+              struct.success.add(_elem1480);
             }
           }
           struct.setSuccessIsSet(true);
@@ -84846,14 +84978,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1474 = iprot.readListBegin();
-                  struct.success = new ArrayList<ExtendedTableInfo>(_list1474.size);
-                  ExtendedTableInfo _elem1475;
-                  for (int _i1476 = 0; _i1476 < _list1474.size; ++_i1476)
+                  org.apache.thrift.protocol.TList _list1482 = iprot.readListBegin();
+                  struct.success = new ArrayList<ExtendedTableInfo>(_list1482.size);
+                  ExtendedTableInfo _elem1483;
+                  for (int _i1484 = 0; _i1484 < _list1482.size; ++_i1484)
                   {
-                    _elem1475 = new ExtendedTableInfo();
-                    _elem1475.read(iprot);
-                    struct.success.add(_elem1475);
+                    _elem1483 = new ExtendedTableInfo();
+                    _elem1483.read(iprot);
+                    struct.success.add(_elem1483);
                   }
                   iprot.readListEnd();
                 }
@@ -84888,9 +85020,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 (ExtendedTableInfo _iter1477 : struct.success)
+            for (ExtendedTableInfo _iter1485 : struct.success)
             {
-              _iter1477.write(oprot);
+              _iter1485.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -84929,9 +85061,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (ExtendedTableInfo _iter1478 : struct.success)
+            for (ExtendedTableInfo _iter1486 : struct.success)
             {
-              _iter1478.write(oprot);
+              _iter1486.write(oprot);
             }
           }
         }
@@ -84946,14 +85078,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1479 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<ExtendedTableInfo>(_list1479.size);
-            ExtendedTableInfo _elem1480;
-            for (int _i1481 = 0; _i1481 < _list1479.size; ++_i1481)
+            org.apache.thrift.protocol.TList _list1487 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<ExtendedTableInfo>(_list1487.size);
+            ExtendedTableInfo _elem1488;
+            for (int _i1489 = 0; _i1489 < _list1487.size; ++_i1489)
             {
-              _elem1480 = new ExtendedTableInfo();
-              _elem1480.read(iprot);
-              struct.success.add(_elem1480);
+              _elem1488 = new ExtendedTableInfo();
+              _elem1488.read(iprot);
+              struct.success.add(_elem1488);
             }
           }
           struct.setSuccessIsSet(true);
@@ -90466,13 +90598,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1482 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1482.size);
-                  String _elem1483;
-                  for (int _i1484 = 0; _i1484 < _list1482.size; ++_i1484)
+                  org.apache.thrift.protocol.TList _list1490 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1490.size);
+                  String _elem1491;
+                  for (int _i1492 = 0; _i1492 < _list1490.size; ++_i1492)
                   {
-                    _elem1483 = iprot.readString();
-                    struct.success.add(_elem1483);
+                    _elem1491 = iprot.readString();
+                    struct.success.add(_elem1491);
                   }
                   iprot.readListEnd();
                 }
@@ -90525,9 +90657,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 _iter1485 : struct.success)
+            for (String _iter1493 : struct.success)
             {
-              oprot.writeString(_iter1485);
+              oprot.writeString(_iter1493);
             }
             oprot.writeListEnd();
           }
@@ -90582,9 +90714,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1486 : struct.success)
+            for (String _iter1494 : struct.success)
             {
-              oprot.writeString(_iter1486);
+              oprot.writeString(_iter1494);
             }
           }
         }
@@ -90605,13 +90737,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1487 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1487.size);
-            String _elem1488;
-            for (int _i1489 = 0; _i1489 < _list1487.size; ++_i1489)
+            org.apache.thrift.protocol.TList _list1495 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1495.size);
+            String _elem1496;
+            for (int _i1497 = 0; _i1497 < _list1495.size; ++_i1497)
             {
-              _elem1488 = iprot.readString();
-              struct.success.add(_elem1488);
+              _elem1496 = iprot.readString();
+              struct.success.add(_elem1496);
             }
           }
           struct.setSuccessIsSet(true);
@@ -97408,14 +97540,14 @@ import org.slf4j.LoggerFactory;
             case 1: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1490 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<Partition>(_list1490.size);
-                  Partition _elem1491;
-                  for (int _i1492 = 0; _i1492 < _list1490.size; ++_i1492)
+                  org.apache.thrift.protocol.TList _list1498 = iprot.readListBegin();
+                  struct.new_parts = new ArrayList<Partition>(_list1498.size);
+                  Partition _elem1499;
+                  for (int _i1500 = 0; _i1500 < _list1498.size; ++_i1500)
                   {
-                    _elem1491 = new Partition();
-                    _elem1491.read(iprot);
-                    struct.new_parts.add(_elem1491);
+                    _elem1499 = new Partition();
+                    _elem1499.read(iprot);
+                    struct.new_parts.add(_elem1499);
                   }
                   iprot.readListEnd();
                 }
@@ -97441,9 +97573,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 _iter1493 : struct.new_parts)
+            for (Partition _iter1501 : struct.new_parts)
             {
-              _iter1493.write(oprot);
+              _iter1501.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -97474,9 +97606,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (Partition _iter1494 : struct.new_parts)
+            for (Partition _iter1502 : struct.new_parts)
             {
-              _iter1494.write(oprot);
+              _iter1502.write(oprot);
             }
           }
         }
@@ -97488,14 +97620,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1495 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.new_parts = new ArrayList<Partition>(_list1495.size);
-            Partition _elem1496;
-            for (int _i1497 = 0; _i1497 < _list1495.size; ++_i1497)
+            org.apache.thrift.protocol.TList _list1503 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.new_parts = new ArrayList<Partition>(_list1503.size);
+            Partition _elem1504;
+            for (int _i1505 = 0; _i1505 < _list1503.size; ++_i1505)
             {
-              _elem1496 = new Partition();
-              _elem1496.read(iprot);
-              struct.new_parts.add(_elem1496);
+              _elem1504 = new Partition();
+              _elem1504.read(iprot);
+              struct.new_parts.add(_elem1504);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -98496,14 +98628,14 @@ import org.slf4j.LoggerFactory;
             case 1: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1498 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<PartitionSpec>(_list1498.size);
-                  PartitionSpec _elem1499;
-                  for (int _i1500 = 0; _i1500 < _list1498.size; ++_i1500)
+                  org.apache.thrift.protocol.TList _list1506 = iprot.readListBegin();
+                  struct.new_parts = new ArrayList<PartitionSpec>(_list1506.size);
+                  PartitionSpec _elem1507;
+                  for (int _i1508 = 0; _i1508 < _list1506.size; ++_i1508)
                   {
-                    _elem1499 = new PartitionSpec();
-                    _elem1499.read(iprot);
-                    struct.new_parts.add(_elem1499);
+                    _elem1507 = new PartitionSpec();
+                    _elem1507.read(iprot);
+                    struct.new_parts.add(_elem1507);
                   }
                   iprot.readListEnd();
                 }
@@ -98529,9 +98661,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 _iter1501 : struct.new_parts)
+            for (PartitionSpec _iter1509 : struct.new_parts)
             {
-              _iter1501.write(oprot);
+              _iter1509.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -98562,9 +98694,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (PartitionSpec _iter1502 : struct.new_parts)
+            for (PartitionSpec _iter1510 : struct.new_parts)
             {
-              _iter1502.write(oprot);
+              _iter1510.write(oprot);
             }
           }
         }
@@ -98576,14 +98708,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1503 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.new_parts = new ArrayList<PartitionSpec>(_list1503.size);
-            PartitionSpec _elem1504;
-            for (int _i1505 = 0; _i1505 < _list1503.size; ++_i1505)
+            org.apache.thrift.protocol.TList _list1511 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.new_parts = new ArrayList<PartitionSpec>(_list1511.size);
+            PartitionSpec _elem1512;
+            for (int _i1513 = 0; _i1513 < _list1511.size; ++_i1513)
             {
-              _elem1504 = new PartitionSpec();
-              _elem1504.read(iprot);
-              struct.new_parts.add(_elem1504);
+              _elem1512 = new PartitionSpec();
+              _elem1512.read(iprot);
+              struct.new_parts.add(_elem1512);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -99759,13 +99891,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1506 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list1506.size);
-                  String _elem1507;
-                  for (int _i1508 = 0; _i1508 < _list1506.size; ++_i1508)
+                  org.apache.thrift.protocol.TList _list1514 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list1514.size);
+                  String _elem1515;
+                  for (int _i1516 = 0; _i1516 < _list1514.size; ++_i1516)
                   {
-                    _elem1507 = iprot.readString();
-                    struct.part_vals.add(_elem1507);
+                    _elem1515 = iprot.readString();
+                    struct.part_vals.add(_elem1515);
                   }
                   iprot.readListEnd();
                 }
@@ -99801,9 +99933,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter1509 : struct.part_vals)
+            for (String _iter1517 : struct.part_vals)
             {
-              oprot.writeString(_iter1509);
+              oprot.writeString(_iter1517);
             }
             oprot.writeListEnd();
           }
@@ -99846,9 +99978,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter1510 : struct.part_vals)
+            for (String _iter1518 : struct.part_vals)
             {
-              oprot.writeString(_iter1510);
+              oprot.writeString(_iter1518);
             }
           }
         }
@@ -99868,13 +100000,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1511 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list1511.size);
-            String _elem1512;
-            for (int _i1513 = 0; _i1513 < _list1511.size; ++_i1513)
+            org.apache.thrift.protocol.TList _list1519 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list1519.size);
+            String _elem1520;
+            for (int _i1521 = 0; _i1521 < _list1519.size; ++_i1521)
             {
-              _elem1512 = iprot.readString();
-              struct.part_vals.add(_elem1512);
+              _elem1520 = iprot.readString();
+              struct.part_vals.add(_elem1520);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -102183,13 +102315,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1514 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list1514.size);
-                  String _elem1515;
-                  for (int _i1516 = 0; _i1516 < _list1514.size; ++_i1516)
+                  org.apache.thrift.protocol.TList _list1522 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list1522.size);
+                  String _elem1523;
+                  for (int _i1524 = 0; _i1524 < _list1522.size; ++_i1524)
                   {
-                    _elem1515 = iprot.readString();
-                    struct.part_vals.add(_elem1515);
+                    _elem1523 = iprot.readString();
+                    struct.part_vals.add(_elem1523);
                   }
                   iprot.readListEnd();
                 }
@@ -102234,9 +102366,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter1517 : struct.part_vals)
+            for (String _iter1525 : struct.part_vals)
             {
-              oprot.writeString(_iter1517);
+              oprot.writeString(_iter1525);
             }
             oprot.writeListEnd();
           }
@@ -102287,9 +102419,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter1518 : struct.part_vals)
+            for (String _iter1526 : struct.part_vals)
             {
-              oprot.writeString(_iter1518);
+              oprot.writeString(_iter1526);
             }
           }
         }
@@ -102312,13 +102444,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1519 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list1519.size);
-            String _elem1520;
-            for (int _i1521 = 0; _i1521 < _list1519.size; ++_i1521)
+            org.apache.thrift.protocol.TList _list1527 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list1527.size);
+            String _elem1528;
+            for (int _i1529 = 0; _i1529 < _list1527.size; ++_i1529)
             {
-              _elem1520 = iprot.readString();
-              struct.part_vals.add(_elem1520);
+              _elem1528 = iprot.readString();
+              struct.part_vals.add(_elem1528);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -106188,13 +106320,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1522 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list1522.size);
-                  String _elem1523;
-                  for (int _i1524 = 0; _i1524 < _list1522.size; ++_i1524)
+                  org.apache.thrift.protocol.TList _list1530 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list1530.size);
+                  String _elem1531;
+                  for (int _i1532 = 0; _i1532 < _list1530.size; ++_i1532)
                   {
-                    _elem1523 = iprot.readString();
-                    struct.part_vals.add(_elem1523);
+                    _elem1531 = iprot.readString();
+                    struct.part_vals.add(_elem1531);
                   }
                   iprot.readListEnd();
                 }
@@ -106238,9 +106370,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter1525 : struct.part_vals)
+            for (String _iter1533 : struct.part_vals)
             {
-              oprot.writeString(_iter1525);
+              oprot.writeString(_iter1533);
             }
             oprot.writeListEnd();
           }
@@ -106289,9 +106421,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter1526 : struct.part_vals)
+            for (String _iter1534 : struct.part_vals)
             {
-              oprot.writeString(_iter1526);
+              oprot.writeString(_iter1534);
             }
           }
         }
@@ -106314,13 +106446,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1527 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list1527.size);
-            String _elem1528;
-            for (int _i1529 = 0; _i1529 < _list1527.size; ++_i1529)
+            org.apache.thrift.protocol.TList _list1535 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list1535.size);
+            String _elem1536;
+            for (int _i1537 = 0; _i1537 < _list1535.size; ++_i1537)
             {
-              _elem1528 = iprot.readString();
-              struct.part_vals.add(_elem1528);
+              _elem1536 = iprot.readString();
+              struct.part_vals.add(_elem1536);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -107559,13 +107691,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1530 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list1530.size);
-                  String _elem1531;
-                  for (int _i1532 = 0; _i1532 < _list1530.size; ++_i1532)
+                  org.apache.thrift.protocol.TList _list1538 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list1538.size);
+                  String _elem1539;
+                  for (int _i1540 = 0; _i1540 < _list1538.size; ++_i1540)
                   {
-                    _elem1531 = iprot.readString();
-                    struct.part_vals.add(_elem1531);
+                    _elem1539 = iprot.readString();
+                    struct.part_vals.add(_elem1539);
                   }
                   iprot.readListEnd();
                 }
@@ -107618,9 +107750,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter1533 : struct.part_vals)
+            for (String _iter1541 : struct.part_vals)
             {
-              oprot.writeString(_iter1533);
+              oprot.writeString(_iter1541);
             }
             oprot.writeListEnd();
           }
@@ -107677,9 +107809,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter1534 : struct.part_vals)
+            for (String _iter1542 : struct.part_vals)
             {
-              oprot.writeString(_iter1534);
+              oprot.writeString(_iter1542);
             }
           }
         }
@@ -107705,13 +107837,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1535 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list1535.size);
-            String _elem1536;
-            for (int _i1537 = 0; _i1537 < _list1535.size; ++_i1537)
+            org.apache.thrift.protocol.TList _list1543 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list1543.size);
+            String _elem1544;
+            for (int _i1545 = 0; _i1545 < _list1543.size; ++_i1545)
             {
-              _elem1536 = iprot.readString();
-              struct.part_vals.add(_elem1536);
+              _elem1544 = iprot.readString();
+              struct.part_vals.add(_elem1544);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -112313,13 +112445,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1538 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list1538.size);
-                  String _elem1539;
-                  for (int _i1540 = 0; _i1540 < _list1538.size; ++_i1540)
+                  org.apache.thrift.protocol.TList _list1546 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list1546.size);
+                  String _elem1547;
+                  for (int _i1548 = 0; _i1548 < _list1546.size; ++_i1548)
                   {
-                    _elem1539 = iprot.readString();
-                    struct.part_vals.add(_elem1539);
+                    _elem1547 = iprot.readString();
+                    struct.part_vals.add(_elem1547);
                   }
                   iprot.readListEnd();
                 }
@@ -112355,9 +112487,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter1541 : struct.part_vals)
+            for (String _iter1549 : struct.part_vals)
             {
-              oprot.writeString(_iter1541);
+              oprot.writeString(_iter1549);
             }
             oprot.writeListEnd();
           }
@@ -112400,9 +112532,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter1542 : struct.part_vals)
+            for (String _iter1550 : struct.part_vals)
             {
-              oprot.writeString(_iter1542);
+              oprot.writeString(_iter1550);
             }
           }
         }
@@ -112422,13 +112554,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1543 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list1543.size);
-            String _elem1544;
-            for (int _i1545 = 0; _i1545 < _list1543.size; ++_i1545)
+            org.apache.thrift.protocol.TList _list1551 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list1551.size);
+            String _elem1552;
+            for (int _i1553 = 0; _i1553 < _list1551.size; ++_i1553)
             {
-              _elem1544 = iprot.readString();
-              struct.part_vals.add(_elem1544);
+              _elem1552 = iprot.readString();
+              struct.part_vals.add(_elem1552);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -114584,15 +114716,15 @@ import org.slf4j.LoggerFactory;
             case 1: // PARTITION_SPECS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map1546 = iprot.readMapBegin();
-                  struct.partitionSpecs = new HashMap<String,String>(2*_map1546.size);
-                  String _key1547;
-                  String _val1548;
-                  for (int _i1549 = 0; _i1549 < _map1546.size; ++_i1549)
+                  org.apache.thrift.protocol.TMap _map1554 = iprot.readMapBegin();
+                  struct.partitionSpecs = new HashMap<String,String>(2*_map1554.size);
+                  String _key1555;
+                  String _val1556;
+                  for (int _i1557 = 0; _i1557 < _map1554.size; ++_i1557)
                   {
-                    _key1547 = iprot.readString();
-                    _val1548 = iprot.readString();
-                    struct.partitionSpecs.put(_key1547, _val1548);
+                    _key1555 = iprot.readString();
+                    _val1556 = iprot.readString();
+                    struct.partitionSpecs.put(_key1555, _val1556);
                   }
                   iprot.readMapEnd();
                 }
@@ -114650,10 +114782,10 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PARTITION_SPECS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.partitionSpecs.size()));
-            for (Map.Entry<String, String> _iter1550 : struct.partitionSpecs.entrySet())
+            for (Map.Entry<String, String> _iter1558 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter1550.getKey());
-              oprot.writeString(_iter1550.getValue());
+              oprot.writeString(_iter1558.getKey());
+              oprot.writeString(_iter1558.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -114716,10 +114848,10 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPartitionSpecs()) {
           {
             oprot.writeI32(struct.partitionSpecs.size());
-            for (Map.Entry<String, String> _iter1551 : struct.partitionSpecs.entrySet())
+            for (Map.Entry<String, String> _iter1559 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter1551.getKey());
-              oprot.writeString(_iter1551.getValue());
+              oprot.writeString(_iter1559.getKey());
+              oprot.writeString(_iter1559.getValue());
             }
           }
         }
@@ -114743,15 +114875,15 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map1552 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.partitionSpecs = new HashMap<String,String>(2*_map1552.size);
-            String _key1553;
-            String _val1554;
-            for (int _i1555 = 0; _i1555 < _map1552.size; ++_i1555)
+            org.apache.thrift.protocol.TMap _map1560 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.partitionSpecs = new HashMap<String,String>(2*_map1560.size);
+            String _key1561;
+            String _val1562;
+            for (int _i1563 = 0; _i1563 < _map1560.size; ++_i1563)
             {
-              _key1553 = iprot.readString();
-              _val1554 = iprot.readString();
-              struct.partitionSpecs.put(_key1553, _val1554);
+              _key1561 = iprot.readString();
+              _val1562 = iprot.readString();
+              struct.partitionSpecs.put(_key1561, _val1562);
             }
           }
           struct.setPartitionSpecsIsSet(true);
@@ -116197,15 +116329,15 @@ import org.slf4j.LoggerFactory;
             case 1: // PARTITION_SPECS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map1556 = iprot.readMapBegin();
-                  struct.partitionSpecs = new HashMap<String,String>(2*_map1556.size);
-                  String _key1557;
-                  String _val1558;
-                  for (int _i1559 = 0; _i1559 < _map1556.size; ++_i1559)
+                  org.apache.thrift.protocol.TMap _map1564 = iprot.readMapBegin();
+                  struct.partitionSpecs = new HashMap<String,String>(2*_map1564.size);
+                  String _key1565;
+                  String _val1566;
+                  for (int _i1567 = 0; _i1567 < _map1564.size; ++_i1567)
                   {
-                    _key1557 = iprot.readString();
-                    _val1558 = iprot.readString();
-                    struct.partitionSpecs.put(_key1557, _val1558);
+                    _key1565 = iprot.readString();
+                    _val1566 = iprot.readString();
+                    struct.partitionSpecs.put(_key1565, _val1566);
                   }
                   iprot.readMapEnd();
                 }
@@ -116263,10 +116395,10 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PARTITION_SPECS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.partitionSpecs.size()));
-            for (Map.Entry<String, String> _iter1560 : struct.partitionSpecs.entrySet())
+            for (Map.Entry<String, String> _iter1568 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter1560.getKey());
-              oprot.writeString(_iter1560.getValue());
+              oprot.writeString(_iter1568.getKey());
+              oprot.writeString(_iter1568.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -116329,10 +116461,10 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPartitionSpecs()) {
           {
             oprot.writeI32(struct.partitionSpecs.size());
-            for (Map.Entry<String, String> _iter1561 : struct.partitionSpecs.entrySet())
+            for (Map.Entry<String, String> _iter1569 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter1561.getKey());
-              oprot.writeString(_iter1561.getValue());
+              oprot.writeString(_iter1569.getKey());
+              oprot.writeString(_iter1569.getValue());
             }
           }
         }
@@ -116356,15 +116488,15 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map1562 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.partitionSpecs = new HashMap<String,String>(2*_map1562.size);
-            String _key1563;
-            String _val1564;
-            for (int _i1565 = 0; _i1565 < _map1562.size; ++_i1565)
+            org.apache.thrift.protocol.TMap _map1570 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.partitionSpecs = new HashMap<String,String>(2*_map1570.size);
+            String _key1571;
+            String _val1572;
+            for (int _i1573 = 0; _i1573 < _map1570.size; ++_i1573)
             {
-              _key1563 = iprot.readString();
-              _val1564 = iprot.readString();
-              struct.partitionSpecs.put(_key1563, _val1564);
+              _key1571 = iprot.readString();
+              _val1572 = iprot.readString();
+              struct.partitionSpecs.put(_key1571, _val1572);
             }
           }
           struct.setPartitionSpecsIsSet(true);
@@ -117029,14 +117161,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1566 = iprot.readListBegin();
-                  struct.success = new ArrayList<Partition>(_list1566.size);
-                  Partition _elem1567;
-                  for (int _i1568 = 0; _i1568 < _list1566.size; ++_i1568)
+                  org.apache.thrift.protocol.TList _list1574 = iprot.readListBegin();
+                  struct.success = new ArrayList<Partition>(_list1574.size);
+                  Partition _elem1575;
+                  for (int _i1576 = 0; _i1576 < _list1574.size; ++_i1576)
                   {
-                    _elem1567 = new Partition();
-                    _elem1567.read(iprot);
-                    struct.success.add(_elem1567);
+                    _elem1575 = new Partition();
+                    _elem1575.read(iprot);
+                    struct.success.add(_elem1575);
                   }
                   iprot.readListEnd();
                 }
@@ -117098,9 +117230,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 (Partition _iter1569 : struct.success)
+            for (Partition _iter1577 : struct.success)
             {
-              _iter1569.write(oprot);
+              _iter1577.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -117163,9 +117295,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Partition _iter1570 : struct.success)
+            for (Partition _iter1578 : struct.success)
             {
-              _iter1570.write(oprot);
+              _iter1578.write(oprot);
             }
           }
         }
@@ -117189,14 +117321,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1571 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Partition>(_list1571.size);
-            Partition _elem1572;
-            for (int _i1573 = 0; _i1573 < _list1571.size; ++_i1573)
+            org.apache.thrift.protocol.TList _list1579 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Partition>(_list1579.size);
+            Partition _elem1580;
+            for (int _i1581 = 0; _i1581 < _list1579.size; ++_i1581)
             {
-              _elem1572 = new Partition();
-              _elem1572.read(iprot);
-              struct.success.add(_elem1572);
+              _elem1580 = new Partition();
+              _elem1580.read(iprot);
+              struct.success.add(_elem1580);
             }
           }
           struct.setSuccessIsSet(true);
@@ -117895,13 +118027,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1574 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list1574.size);
-                  String _elem1575;
-                  for (int _i1576 = 0; _i1576 < _list1574.size; ++_i1576)
+                  org.apache.thrift.protocol.TList _list1582 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list1582.size);
+                  String _elem1583;
+                  for (int _i1584 = 0; _i1584 < _list1582.size; ++_i1584)
                   {
-                    _elem1575 = iprot.readString();
-                    struct.part_vals.add(_elem1575);
+                    _elem1583 = iprot.readString();
+                    struct.part_vals.add(_elem1583);
                   }
                   iprot.readListEnd();
                 }
@@ -117921,13 +118053,13 @@ import org.slf4j.LoggerFactory;
             case 5: // GROUP_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1577 = iprot.readListBegin();
-                  struct.group_names = new ArrayList<String>(_list1577.size);
-                  String _elem1578;
-                  for (int _i1579 = 0; _i1579 < _list1577.size; ++_i1579)
+                  org.apache.thrift.protocol.TList _list1585 = iprot.readListBegin();
+                  struct.group_names = new ArrayList<String>(_list1585.size);
+                  String _elem1586;
+                  for (int _i1587 = 0; _i1587 < _list1585.size; ++_i1587)
                   {
-                    _elem1578 = iprot.readString();
-                    struct.group_names.add(_elem1578);
+                    _elem1586 = iprot.readString();
+                    struct.group_names.add(_elem1586);
                   }
                   iprot.readListEnd();
                 }
@@ -117963,9 +118095,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter1580 : struct.part_vals)
+            for (String _iter1588 : struct.part_vals)
             {
-              oprot.writeString(_iter1580);
+              oprot.writeString(_iter1588);
             }
             oprot.writeListEnd();
           }
@@ -117980,9 +118112,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.group_names.size()));
-            for (String _iter1581 : struct.group_names)
+            for (String _iter1589 : struct.group_names)
             {
-              oprot.writeString(_iter1581);
+              oprot.writeString(_iter1589);
             }
             oprot.writeListEnd();
           }
@@ -118031,9 +118163,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter1582 : struct.part_vals)
+            for (String _iter1590 : struct.part_vals)
             {
-              oprot.writeString(_iter1582);
+              oprot.writeString(_iter1590);
             }
           }
         }
@@ -118043,9 +118175,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetGroup_names()) {
           {
             oprot.writeI32(struct.group_names.size());
-            for (String _iter1583 : struct.group_names)
+            for (String _iter1591 : struct.group_names)
             {
-              oprot.writeString(_iter1583);
+              oprot.writeString(_iter1591);
             }
           }
         }
@@ -118065,13 +118197,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1584 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list1584.size);
-            String _elem1585;
-            for (int _i1586 = 0; _i1586 < _list1584.size; ++_i1586)
+            org.apache.thrift.protocol.TList _list1592 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list1592.size);
+            String _elem1593;
+            for (int _i1594 = 0; _i1594 < _list1592.size; ++_i1594)
             {
-              _elem1585 = iprot.readString();
-              struct.part_vals.add(_elem1585);
+              _elem1593 = iprot.readString();
+              struct.part_vals.add(_elem1593);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -118082,13 +118214,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol.TList _list1587 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.group_names = new ArrayList<String>(_list1587.size);
-            String _elem1588;
-            for (int _i1589 = 0; _i1589 < _list1587.size; ++_i1589)
+            org.apache.thrift.protocol.TList _list1595 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.group_names = new ArrayList<String>(_list1595.size);
+            String _elem1596;
+            for (int _i1597 = 0; _i1597 < _list1595.size; ++_i1597)
             {
-              _elem1588 = iprot.readString();
-              struct.group_names.add(_elem1588);
+              _elem1596 = iprot.readString();
+              struct.group_names.add(_elem1596);
             }
           }
           struct.setGroup_namesIsSet(true);
@@ -120857,14 +120989,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1590 = iprot.readListBegin();
-                  struct.success = new ArrayList<Partition>(_list1590.size);
-                  Partition _elem1591;
-                  for (int _i1592 = 0; _i1592 < _list1590.size; ++_i1592)
+                  org.apache.thrift.protocol.TList _list1598 = iprot.readListBegin();
+                  struct.success = new ArrayList<Partition>(_list1598.size);
+                  Partition _elem1599;
+                  for (int _i1600 = 0; _i1600 < _list1598.size; ++_i1600)
                   {
-                    _elem1591 = new Partition();
-                    _elem1591.read(iprot);
-                    struct.success.add(_elem1591);
+                    _elem1599 = new Partition();
+                    _elem1599.read(iprot);
+                    struct.success.add(_elem1599);
                   }
                   iprot.readListEnd();
                 }
@@ -120908,9 +121040,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 (Partition _iter1593 : struct.success)
+            for (Partition _iter1601 : struct.success)
             {
-              _iter1593.write(oprot);
+              _iter1601.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -120957,9 +121089,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Partition _iter1594 : struct.success)
+            for (Partition _iter1602 : struct.success)
             {
-              _iter1594.write(oprot);
+              _iter1602.write(oprot);
             }
           }
         }
@@ -120977,14 +121109,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1595 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Partition>(_list1595.size);
-            Partition _elem1596;
-            for (int _i1597 = 0; _i1597 < _list1595.size; ++_i1597)
+            org.apache.thrift.protocol.TList _list1603 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Partition>(_list1603.size);
+            Partition _elem1604;
+            for (int _i1605 = 0; _i1605 < _list1603.size; ++_i1605)
             {
-              _elem1596 = new Partition();
-              _elem1596.read(iprot);
-              struct.success.add(_elem1596);
+              _elem1604 = new Partition();
+              _elem1604.read(iprot);
+              struct.success.add(_elem1604);
             }
           }
           struct.setSuccessIsSet(true);
@@ -122612,13 +122744,13 @@ import org.slf4j.LoggerFactory;
             case 5: // GROUP_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1598 = iprot.readListBegin();
-                  struct.group_names = new ArrayList<String>(_list1598.size);
-                  String _elem1599;
-                  for (int _i1600 = 0; _i1600 < _list1598.size; ++_i1600)
+                  org.apache.thrift.protocol.TList _list1606 = iprot.readListBegin();
+                  struct.group_names = new ArrayList<String>(_list1606.size);
+                  String _elem1607;
+                  for (int _i1608 = 0; _i1608 < _list1606.size; ++_i1608)
                   {
-                    _elem1599 = iprot.readString();
-                    struct.group_names.add(_elem1599);
+                    _elem1607 = iprot.readString();
+                    struct.group_names.add(_elem1607);
                   }
                   iprot.readListEnd();
                 }
@@ -122662,9 +122794,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.group_names.size()));
-            for (String _iter1601 : struct.group_names)
+            for (String _iter1609 : struct.group_names)
             {
-              oprot.writeString(_iter1601);
+              oprot.writeString(_iter1609);
             }
             oprot.writeListEnd();
           }
@@ -122719,9 +122851,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetGroup_names()) {
           {
             oprot.writeI32(struct.group_names.size());
-            for (String _iter1602 : struct.group_names)
+            for (String _iter1610 : struct.group_names)
             {
-              oprot.writeString(_iter1602);
+              oprot.writeString(_iter1610);
             }
           }
         }
@@ -122749,13 +122881,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol.TList _list1603 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.group_names = new ArrayList<String>(_list1603.size);
-            String _elem1604;
-            for (int _i1605 = 0; _i1605 < _list1603.size; ++_i1605)
+            org.apache.thrift.protocol.TList _list1611 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.group_names = new ArrayList<String>(_list1611.size);
+            String _elem1612;
+            for (int _i1613 = 0; _i1613 < _list1611.size; ++_i1613)
             {
-              _elem1604 = iprot.readString();
-              struct.group_names.add(_elem1604);
+              _elem1612 = iprot.readString();
+              struct.group_names.add(_elem1612);
             }
           }
           struct.setGroup_namesIsSet(true);
@@ -123242,14 +123374,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1606 = iprot.readListBegin();
-                  struct.success = new ArrayList<Partition>(_list1606.size);
-                  Partition _elem1607;
-                  for (int _i1608 = 0; _i1608 < _list1606.size; ++_i1608)
+                  org.apache.thrift.protocol.TList _list1614 = iprot.readListBegin();
+                  struct.success = new ArrayList<Partition>(_list1614.size);
+                  Partition _elem1615;
+                  for (int _i1616 = 0; _i1616 < _list1614.size; ++_i1616)
                   {
-                    _elem1607 = new Partition();
-                    _elem1607.read(iprot);
-                    struct.success.add(_elem1607);
+                    _elem1615 = new Partition();
+                    _elem1615.read(iprot);
+                    struct.success.add(_elem1615);
                   }
                   iprot.readListEnd();
                 }
@@ -123293,9 +123425,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 (Partition _iter1609 : struct.success)
+            for (Partition _iter1617 : struct.success)
             {
-              _iter1609.write(oprot);
+              _iter1617.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -123342,9 +123474,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Partition _iter1610 : struct.success)
+            for (Partition _iter1618 : struct.success)
             {
-              _iter1610.write(oprot);
+              _iter1618.write(oprot);
             }
           }
         }
@@ -123362,14 +123494,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1611 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Partition>(_list1611.size);
-            Partition _elem1612;
-            for (int _i1613 = 0; _i1613 < _list1611.size; ++_i1613)
+            org.apache.thrift.protocol.TList _list1619 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Partition>(_list1619.size);
+            Partition _elem1620;
+            for (int _i1621 = 0; _i1621 < _list1619.size; ++_i1621)
             {
-              _elem1612 = new Partition();
-              _elem1612.read(iprot);
-              struct.success.add(_elem1612);
+              _elem1620 = new Partition();
+              _elem1620.read(iprot);
+              struct.success.add(_elem1620);
             }
           }
           struct.setSuccessIsSet(true);
@@ -124432,14 +124564,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1614 = iprot.readListBegin();
-                  struct.success = new ArrayList<PartitionSpec>(_list1614.size);
-                  PartitionSpec _elem1615;
-                  for (int _i1616 = 0; _i1616 < _list1614.size; ++_i1616)
+                  org.apache.thrift.protocol.TList _list1622 = iprot.readListBegin();
+                  struct.success = new ArrayList<PartitionSpec>(_list1622.size);
+                  PartitionSpec _elem1623;
+                  for (int _i1624 = 0; _i1624 < _list1622.size; ++_i1624)
                   {
-                    _elem1615 = new PartitionSpec();
-                    _elem1615.read(iprot);
-                    struct.success.add(_elem1615);
+                    _elem1623 = new PartitionSpec();
+                    _elem1623.read(iprot);
+                    struct.success.add(_elem1623);
                   }
                   iprot.readListEnd();
                 }
@@ -124483,9 +124615,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 (PartitionSpec _iter1617 : struct.success)
+            for (PartitionSpec _iter1625 : struct.success)
             {
-              _iter1617.write(oprot);
+              _iter1625.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -124532,9 +124664,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (PartitionSpec _iter1618 : struct.success)
+            for (PartitionSpec _iter1626 : struct.success)
             {
-              _iter1618.write(oprot);
+              _iter1626.write(oprot);
             }
           }
         }
@@ -124552,14 +124684,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1619 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<PartitionSpec>(_list1619.size);
-            PartitionSpec _elem1620;
-            for (int _i1621 = 0; _i1621 < _list1619.size; ++_i1621)
+            org.apache.thrift.protocol.TList _list1627 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<PartitionSpec>(_list1627.size);
+            PartitionSpec _elem1628;
+            for (int _i1629 = 0; _i1629 < _list1627.size; ++_i1629)
             {
-              _elem1620 = new PartitionSpec();
-              _elem1620.read(iprot);
-              struct.success.add(_elem1620);
+              _elem1628 = new PartitionSpec();
+              _elem1628.read(iprot);
+              struct.success.add(_elem1628);
             }
           }
           struct.setSuccessIsSet(true);
@@ -125619,13 +125751,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1622 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1622.size);
-                  String _elem1623;
-                  for (int _i1624 = 0; _i1624 < _list1622.size; ++_i1624)
+                  org.apache.thrift.protocol.TList _list1630 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1630.size);
+                  String _elem1631;
+                  for (int _i1632 = 0; _i1632 < _list1630.size; ++_i1632)
                   {
-                    _elem1623 = iprot.readString();
-                    struct.success.add(_elem1623);
+                    _elem1631 = iprot.readString();
+                    struct.success.add(_elem1631);
                   }
                   iprot.readListEnd();
                 }
@@ -125669,9 +125801,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 _iter1625 : struct.success)
+            for (String _iter1633 : struct.success)
             {
-              oprot.writeString(_iter1625);
+              oprot.writeString(_iter1633);
             }
             oprot.writeListEnd();
           }
@@ -125718,9 +125850,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1626 : struct.success)
+            for (String _iter1634 : struct.success)
             {
-              oprot.writeString(_iter1626);
+              oprot.writeString(_iter1634);
             }
           }
         }
@@ -125738,13 +125870,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1627 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1627.size);
-            String _elem1628;
-            for (int _i1629 = 0; _i1629 < _list1627.size; ++_i1629)
+            org.apache.thrift.protocol.TList _list1635 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1635.size);
+            String _elem1636;
+            for (int _i1637 = 0; _i1637 < _list1635.size; ++_i1637)
             {
-              _elem1628 = iprot.readString();
-              struct.success.add(_elem1628);
+              _elem1636 = iprot.readString();
+              struct.success.add(_elem1636);
             }
           }
           struct.setSuccessIsSet(true);
@@ -127275,13 +127407,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1630 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list1630.size);
-                  String _elem1631;
-                  for (int _i1632 = 0; _i1632 < _list1630.size; ++_i1632)
+                  org.apache.thrift.protocol.TList _list1638 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list1638.size);
+                  String _elem1639;
+                  for (int _i1640 = 0; _i1640 < _list1638.size; ++_i1640)
                   {
-                    _elem1631 = iprot.readString();
-                    struct.part_vals.add(_elem1631);
+                    _elem1639 = iprot.readString();
+                    struct.part_vals.add(_elem1639);
                   }
                   iprot.readListEnd();
                 }
@@ -127325,9 +127457,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter1633 : struct.part_vals)
+            for (String _iter1641 : struct.part_vals)
             {
-              oprot.writeString(_iter1633);
+              oprot.writeString(_iter1641);
             }
             oprot.writeListEnd();
           }
@@ -127376,9 +127508,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter1634 : struct.part_vals)
+            for (String _iter1642 : struct.part_vals)
             {
-              oprot.writeString(_iter1634);
+              oprot.writeString(_iter1642);
             }
           }
         }
@@ -127401,13 +127533,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1635 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list1635.size);
-            String _elem1636;
-            for (int _i1637 = 0; _i1637 < _list1635.size; ++_i1637)
+            org.apache.thrift.protocol.TList _list1643 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list1643.size);
+            String _elem1644;
+            for (int _i1645 = 0; _i1645 < _list1643.size; ++_i1645)
             {
-              _elem1636 = iprot.readString();
-              struct.part_vals.add(_elem1636);
+              _elem1644 = iprot.readString();
+              struct.part_vals.add(_elem1644);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -127898,14 +128030,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1638 = iprot.readListBegin();
-                  struct.success = new ArrayList<Partition>(_list1638.size);
-                  Partition _elem1639;
-                  for (int _i1640 = 0; _i1640 < _list1638.size; ++_i1640)
+                  org.apache.thrift.protocol.TList _list1646 = iprot.readListBegin();
+                  struct.success = new ArrayList<Partition>(_list1646.size);
+                  Partition _elem1647;
+                  for (int _i1648 = 0; _i1648 < _list1646.size; ++_i1648)
                   {
-                    _elem1639 = new Partition();
-                    _elem1639.read(iprot);
-                    struct.success.add(_elem1639);
+                    _elem1647 = new Partition();
+                    _elem1647.read(iprot);
+                    struct.success.add(_elem1647);
                   }
                   iprot.readListEnd();
                 }
@@ -127949,9 +128081,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 (Partition _iter1641 : struct.success)
+            for (Partition _iter1649 : struct.success)
             {
-              _iter1641.write(oprot);
+              _iter1649.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -127998,9 +128130,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Partition _iter1642 : struct.success)
+            for (Partition _iter1650 : struct.success)
             {
-              _iter1642.write(oprot);
+              _iter1650.write(oprot);
             }
           }
         }
@@ -128018,14 +128150,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1643 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Partition>(_list1643.size);
-            Partition _elem1644;
-            for (int _i1645 = 0; _i1645 < _list1643.size; ++_i1645)
+            org.apache.thrift.protocol.TList _list1651 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Partition>(_list1651.size);
+            Partition _elem1652;
+            for (int _i1653 = 0; _i1653 < _list1651.size; ++_i1653)
             {
-              _elem1644 = new Partition();
-              _elem1644.read(iprot);
-              struct.success.add(_elem1644);
+              _elem1652 = new Partition();
+              _elem1652.read(iprot);
+              struct.success.add(_elem1652);
             }
           }
           struct.setSuccessIsSet(true);
@@ -128797,13 +128929,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1646 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list1646.size);
-                  String _elem1647;
-                  for (int _i1648 = 0; _i1648 < _list1646.size; ++_i1648)
+                  org.apache.thrift.protocol.TList _list1654 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list1654.size);
+                  String _elem1655;
+                  for (int _i1656 = 0; _i1656 < _list1654.size; ++_i1656)
                   {
-                    _elem1647 = iprot.readString();
-                    struct.part_vals.add(_elem1647);
+                    _elem1655 = iprot.readString();
+                    struct.part_vals.add(_elem1655);
                   }
                   iprot.readListEnd();
                 }
@@ -128831,13 +128963,13 @@ import org.slf4j.LoggerFactory;
             case 6: // GROUP_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1649 = iprot.readListBegin();
-                  struct.group_names = new ArrayList<String>(_list1649.size);
-                  String _elem1650;
-                  for (int _i1651 = 0; _i1651 < _list1649.size; ++_i1651)
+                  org.apache.thrift.protocol.TList _list1657 = iprot.readListBegin();
+                  struct.group_names = new ArrayList<String>(_list1657.size);
+                  String _elem1658;
+                  for (int _i1659 = 0; _i1659 < _list1657.size; ++_i1659)
                   {
-                    _elem1650 = iprot.readString();
-                    struct.group_names.add(_elem1650);
+                    _elem1658 = iprot.readString();
+                    struct.group_names.add(_elem1658);
                   }
                   iprot.readListEnd();
                 }
@@ -128873,9 +129005,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter1652 : struct.part_vals)
+            for (String _iter1660 : struct.part_vals)
             {
-              oprot.writeString(_iter1652);
+              oprot.writeString(_iter1660);
             }
             oprot.writeListEnd();
           }
@@ -128893,9 +129025,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.group_names.size()));
-            for (String _iter1653 : struct.group_names)
+            for (String _iter1661 : struct.group_names)
             {
-              oprot.writeString(_iter1653);
+              oprot.writeString(_iter1661);
             }
             oprot.writeListEnd();
           }
@@ -128947,9 +129079,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter1654 : struct.part_vals)
+            for (String _iter1662 : struct.part_vals)
             {
-              oprot.writeString(_iter1654);
+              oprot.writeString(_iter1662);
             }
           }
         }
@@ -128962,9 +129094,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetGroup_names()) {
           {
             oprot.writeI32(struct.group_names.size());
-            for (String _iter1655 : struct.group_names)
+            for (String _iter1663 : struct.group_names)
             {
-              oprot.writeString(_iter1655);
+              oprot.writeString(_iter1663);
             }
           }
         }
@@ -128984,13 +129116,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1656 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list1656.size);
-            String _elem1657;
-            for (int _i1658 = 0; _i1658 < _list1656.size; ++_i1658)
+            org.apache.thrift.protocol.TList _list1664 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list1664.size);
+            String _elem1665;
+            for (int _i1666 = 0; _i1666 < _list1664.size; ++_i1666)
             {
-              _elem1657 = iprot.readString();
-              struct.part_vals.add(_elem1657);
+              _elem1665 = iprot.readString();
+              struct.part_vals.add(_elem1665);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -129005,13 +129137,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(5)) {
           {
-            org.apache.thrift.protocol.TList _list1659 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.group_names = new ArrayList<String>(_list1659.size);
-            String _elem1660;
-            for (int _i1661 = 0; _i1661 < _list1659.size; ++_i1661)
+            org.apache.thrift.protocol.TList _list1667 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.group_names = new ArrayList<String>(_list1667.size);
+            String _elem1668;
+            for (int _i1669 = 0; _i1669 < _list1667.size; ++_i1669)
             {
-              _elem1660 = iprot.readString();
-              struct.group_names.add(_elem1660);
+              _elem1668 = iprot.readString();
+              struct.group_names.add(_elem1668);
             }
           }
           struct.setGroup_namesIsSet(true);
@@ -129498,14 +129630,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1662 = iprot.readListBegin();
-                  struct.success = new ArrayList<Partition>(_list1662.size);
-                  Partition _elem1663;
-                  for (int _i1664 = 0; _i1664 < _list1662.size; ++_i1664)
+                  org.apache.thrift.protocol.TList _list1670 = iprot.readListBegin();
+                  struct.success = new ArrayList<Partition>(_list1670.size);
+                  Partition _elem1671;
+                  for (int _i1672 = 0; _i1672 < _list1670.size; ++_i1672)
                   {
-                    _elem1663 = new Partition();
-                    _elem1663.read(iprot);
-                    struct.success.add(_elem1663);
+                    _elem1671 = new Partition();
+                    _elem1671.read(iprot);
+                    struct.success.add(_elem1671);
                   }
                   iprot.readListEnd();
                 }
@@ -129549,9 +129681,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 (Partition _iter1665 : struct.success)
+            for (Partition _iter1673 : struct.success)
             {
-              _iter1665.write(oprot);
+              _iter1673.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -129598,9 +129730,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Partition _iter1666 : struct.success)
+            for (Partition _iter1674 : struct.success)
             {
-              _iter1666.write(oprot);
+              _iter1674.write(oprot);
             }
           }
         }
@@ -129618,14 +129750,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1667 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Partition>(_list1667.size);
-            Partition _elem1668;
-            for (int _i1669 = 0; _i1669 < _list1667.size; ++_i1669)
+            org.apache.thrift.protocol.TList _list1675 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Partition>(_list1675.size);
+            Partition _elem1676;
+            for (int _i1677 = 0; _i1677 < _list1675.size; ++_i1677)
             {
-              _elem1668 = new Partition();
-              _elem1668.read(iprot);
-              struct.success.add(_elem1668);
+              _elem1676 = new Partition();
+              _elem1676.read(iprot);
+              struct.success.add(_elem1676);
             }
           }
           struct.setSuccessIsSet(true);
@@ -131156,13 +131288,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1670 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list1670.size);
-                  String _elem1671;
-                  for (int _i1672 = 0; _i1672 < _list1670.size; ++_i1672)
+                  org.apache.thrift.protocol.TList _list1678 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list1678.size);
+                  String _elem1679;
+                  for (int _i1680 = 0; _i1680 < _list1678.size; ++_i1680)
                   {
-                    _elem1671 = iprot.readString();
-                    struct.part_vals.add(_elem1671);
+                    _elem1679 = iprot.readString();
+                    struct.part_vals.add(_elem1679);
                   }
                   iprot.readListEnd();
                 }
@@ -131206,9 +131338,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter1673 : struct.part_vals)
+            for (String _iter1681 : struct.part_vals)
             {
-              oprot.writeString(_iter1673);
+              oprot.writeString(_iter1681);
             }
             oprot.writeListEnd();
           }
@@ -131257,9 +131389,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter1674 : struct.part_vals)
+            for (String _iter1682 : struct.part_vals)
             {
-              oprot.writeString(_iter1674);
+              oprot.writeString(_iter1682);
             }
           }
         }
@@ -131282,13 +131414,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1675 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list1675.size);
-            String _elem1676;
-            for (int _i1677 = 0; _i1677 < _list1675.size; ++_i1677)
+            org.apache.thrift.protocol.TList _list1683 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list1683.size);
+            String _elem1684;
+            for (int _i1685 = 0; _i1685 < _list1683.size; ++_i1685)
             {
-              _elem1676 = iprot.readString();
-              struct.part_vals.add(_elem1676);
+              _elem1684 = iprot.readString();
+              struct.part_vals.add(_elem1684);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -131776,13 +131908,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1678 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1678.size);
-                  String _elem1679;
-                  for (int _i1680 = 0; _i1680 < _list1678.size; ++_i1680)
+                  org.apache.thrift.protocol.TList _list1686 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1686.size);
+                  String _elem1687;
+                  for (int _i1688 = 0; _i1688 < _list1686.size; ++_i1688)
                   {
-                    _elem1679 = iprot.readString();
-                    struct.success.add(_elem1679);
+                    _elem1687 = iprot.readString();
+                    struct.success.add(_elem1687);
                   }
                   iprot.readListEnd();
                 }
@@ -131826,9 +131958,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 _iter1681 : struct.success)
+            for (String _iter1689 : struct.success)
             {
-              oprot.writeString(_iter1681);
+              oprot.writeString(_iter1689);
             }
             oprot.writeListEnd();
           }
@@ -131875,9 +132007,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1682 : struct.success)
+            for (String _iter1690 : struct.success)
             {
-              oprot.writeString(_iter1682);
+              oprot.writeString(_iter1690);
             }
           }
         }
@@ -131895,13 +132027,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1683 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1683.size);
-            String _elem1684;
-            for (int _i1685 = 0; _i1685 < _list1683.size; ++_i1685)
+            org.apache.thrift.protocol.TList _list1691 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1691.size);
+            String _elem1692;
+            for (int _i1693 = 0; _i1693 < _list1691.size; ++_i1693)
             {
-              _elem1684 = iprot.readString();
-              struct.success.add(_elem1684);
+              _elem1692 = iprot.readString();
+              struct.success.add(_elem1692);
             }
           }
           struct.setSuccessIsSet(true);
@@ -133696,13 +133828,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1686 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1686.size);
-                  String _elem1687;
-                  for (int _i1688 = 0; _i1688 < _list1686.size; ++_i1688)
+                  org.apache.thrift.protocol.TList _list1694 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1694.size);
+                  String _elem1695;
+                  for (int _i1696 = 0; _i1696 < _list1694.size; ++_i1696)
                   {
-                    _elem1687 = iprot.readString();
-                    struct.success.add(_elem1687);
+                    _elem1695 = iprot.readString();
+                    struct.success.add(_elem1695);
                   }
                   iprot.readListEnd();
                 }
@@ -133746,9 +133878,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 _iter1689 : struct.success)
+            for (String _iter1697 : struct.success)
             {
-              oprot.writeString(_iter1689);
+              oprot.writeString(_iter1697);
             }
             oprot.writeListEnd();
           }
@@ -133795,9 +133927,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1690 : struct.success)
+            for (String _iter1698 : struct.success)
             {
-              oprot.writeString(_iter1690);
+              oprot.writeString(_iter1698);
             }
           }
         }
@@ -133815,13 +133947,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1691 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1691.size);
-            String _elem1692;
-            for (int _i1693 = 0; _i1693 < _list1691.size; ++_i1693)
+            org.apache.thrift.protocol.TList _list1699 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1699.size);
+            String _elem1700;
+            for (int _i1701 = 0; _i1701 < _list1699.size; ++_i1701)
             {
-              _elem1692 = iprot.readString();
-              struct.success.add(_elem1692);
+              _elem1700 = iprot.readString();
+              struct.success.add(_elem1700);
             }
           }
           struct.setSuccessIsSet(true);
@@ -134988,14 +135120,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1694 = iprot.readListBegin();
-                  struct.success = new ArrayList<Partition>(_list1694.size);
-                  Partition _elem1695;
-                  for (int _i1696 = 0; _i1696 < _list1694.size; ++_i1696)
+                  org.apache.thrift.protocol.TList _list1702 = iprot.readListBegin();
+                  struct.success = new ArrayList<Partition>(_list1702.size);
+                  Partition _elem1703;
+                  for (int _i1704 = 0; _i1704 < _list1702.size; ++_i1704)
                   {
-                    _elem1695 = new Partition();
-                    _elem1695.read(iprot);
-                    struct.success.add(_elem1695);
+                    _elem1703 = new Partition();
+                    _elem1703.read(iprot);
+                    struct.success.add(_elem1703);
                   }
                   iprot.readListEnd();
                 }
@@ -135039,9 +135171,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 (Partition _iter1697 : struct.success)
+            for (Partition _iter1705 : struct.success)
             {
-              _iter1697.write(oprot);
+              _iter1705.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -135088,9 +135220,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Partition _iter1698 : struct.success)
+            for (Partition _iter1706 : struct.success)
             {
-              _iter1698.write(oprot);
+              _iter1706.write(oprot);
             }
           }
         }
@@ -135108,14 +135240,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1699 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Partition>(_list1699.size);
-            Partition _elem1700;
-            for (int _i1701 = 0; _i1701 < _list1699.size; ++_i1701)
+            org.apache.thrift.protocol.TList _list1707 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Partition>(_list1707.size);
+            Partition _elem1708;
+            for (int _i1709 = 0; _i1709 < _list1707.size; ++_i1709)
             {
-              _elem1700 = new Partition();
-              _elem1700.read(iprot);
-              struct.success.add(_elem1700);
+              _elem1708 = new Partition();
+              _elem1708.read(iprot);
+              struct.success.add(_elem1708);
             }
           }
           struct.setSuccessIsSet(true);
@@ -136282,14 +136414,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1702 = iprot.readListBegin();
-                  struct.success = new ArrayList<PartitionSpec>(_list1702.size);
-                  PartitionSpec _elem1703;
-                  for (int _i1704 = 0; _i1704 < _list1702.size; ++_i1704)
+                  org.apache.thrift.protocol.TList _list1710 = iprot.readListBegin();
+                  struct.success = new ArrayList<PartitionSpec>(_list1710.size);
+                  PartitionSpec _elem1711;
+                  for (int _i1712 = 0; _i1712 < _list1710.size; ++_i1712)
                   {
-                    _elem1703 = new PartitionSpec();
-                    _elem1703.read(iprot);
-                    struct.success.add(_elem1703);
+                    _elem1711 = new PartitionSpec();
+                    _elem1711.read(iprot);
+                    struct.success.add(_elem1711);
                   }
                   iprot.readListEnd();
                 }
@@ -136333,9 +136465,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 (PartitionSpec _iter1705 : struct.success)
+            for (PartitionSpec _iter1713 : struct.success)
             {
-              _iter1705.write(oprot);
+              _iter1713.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -136382,9 +136514,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (PartitionSpec _iter1706 : struct.success)
+            for (PartitionSpec _iter1714 : struct.success)
             {
-              _iter1706.write(oprot);
+              _iter1714.write(oprot);
             }
           }
         }
@@ -136402,14 +136534,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1707 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<PartitionSpec>(_list1707.size);
-            PartitionSpec _elem1708;
-            for (int _i1709 = 0; _i1709 < _list1707.size; ++_i1709)
+            org.apache.thrift.protocol.TList _list1715 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<PartitionSpec>(_list1715.size);
+            PartitionSpec _elem1716;
+            for (int _i1717 = 0; _i1717 < _list1715.size; ++_i1717)
             {
-              _elem1708 = new PartitionSpec();
-              _elem1708.read(iprot);
-              struct.success.add(_elem1708);
+              _elem1716 = new PartitionSpec();
+              _elem1716.read(iprot);
+              struct.success.add(_elem1716);
             }
           }
           struct.setSuccessIsSet(true);
@@ -139931,13 +140063,13 @@ import org.slf4j.LoggerFactory;
             case 3: // NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1710 = iprot.readListBegin();
-                  struct.names = new ArrayList<String>(_list1710.size);
-                  String _elem1711;
-                  for (int _i1712 = 0; _i1712 < _list1710.size; ++_i1712)
+                  org.apache.thrift.protocol.TList _list1718 = iprot.readListBegin();
+                  struct.names = new ArrayList<String>(_list1718.size);
+                  String _elem1719;
+                  for (int _i1720 = 0; _i1720 < _list1718.size; ++_i1720)
                   {
-                    _elem1711 = iprot.readString();
-                    struct.names.add(_elem1711);
+                    _elem1719 = iprot.readString();
+                    struct.names.add(_elem1719);
                   }
                   iprot.readListEnd();
                 }
@@ -139973,9 +140105,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.names.size()));
-            for (String _iter1713 : struct.names)
+            for (String _iter1721 : struct.names)
             {
-              oprot.writeString(_iter1713);
+              oprot.writeString(_iter1721);
             }
             oprot.writeListEnd();
           }
@@ -140018,9 +140150,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetNames()) {
           {
             oprot.writeI32(struct.names.size());
-            for (String _iter1714 : struct.names)
+            for (String _iter1722 : struct.names)
             {
-              oprot.writeString(_iter1714);
+              oprot.writeString(_iter1722);
             }
           }
         }
@@ -140040,13 +140172,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1715 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.names = new ArrayList<String>(_list1715.size);
-            String _elem1716;
-            for (int _i1717 = 0; _i1717 < _list1715.size; ++_i1717)
+            org.apache.thrift.protocol.TList _list1723 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.names = new ArrayList<String>(_list1723.size);
+            String _elem1724;
+            for (int _i1725 = 0; _i1725 < _list1723.size; ++_i1725)
             {
-              _elem1716 = iprot.readString();
-              struct.names.add(_elem1716);
+              _elem1724 = iprot.readString();
+              struct.names.add(_elem1724);
             }
           }
           struct.setNamesIsSet(true);
@@ -140533,14 +140665,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1718 = iprot.readListBegin();
-                  struct.success = new ArrayList<Partition>(_list1718.size);
-                  Partition _elem1719;
-                  for (int _i1720 = 0; _i1720 < _list1718.size; ++_i1720)
+                  org.apache.thrift.protocol.TList _list1726 = iprot.readListBegin();
+                  struct.success = new ArrayList<Partition>(_list1726.size);
+                  Partition _elem1727;
+                  for (int _i1728 = 0; _i1728 < _list1726.size; ++_i1728)
                   {
-                    _elem1719 = new Partition();
-                    _elem1719.read(iprot);
-                    struct.success.add(_elem1719);
+                    _elem1727 = new Partition();
+                    _elem1727.read(iprot);
+                    struct.success.add(_elem1727);
                   }
                   iprot.readListEnd();
                 }
@@ -140584,9 +140716,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 (Partition _iter1721 : struct.success)
+            for (Partition _iter1729 : struct.success)
             {
-              _iter1721.write(oprot);
+              _iter1729.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -140633,9 +140765,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Partition _iter1722 : struct.success)
+            for (Partition _iter1730 : struct.success)
             {
-              _iter1722.write(oprot);
+              _iter1730.write(oprot);
             }
           }
         }
@@ -140653,14 +140785,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1723 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Partition>(_list1723.size);
-            Partition _elem1724;
-            for (int _i1725 = 0; _i1725 < _list1723.size; ++_i1725)
+            org.apache.thrift.protocol.TList _list1731 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Partition>(_list1731.size);
+            Partition _elem1732;
+            for (int _i1733 = 0; _i1733 < _list1731.size; ++_i1733)
             {
-              _elem1724 = new Partition();
-              _elem1724.read(iprot);
-              struct.success.add(_elem1724);
+              _elem1732 = new Partition();
+              _elem1732.read(iprot);
+              struct.success.add(_elem1732);
             }
           }
           struct.setSuccessIsSet(true);
@@ -143148,14 +143280,14 @@ import org.slf4j.LoggerFactory;
             case 3: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1726 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<Partition>(_list1726.size);
-                  Partition _elem1727;
-                  for (int _i1728 = 0; _i1728 < _list1726.size; ++_i1728)
+                  org.apache.thrift.protocol.TList _list1734 = iprot.readListBegin();
+                  struct.new_parts = new ArrayList<Partition>(_list1734.size);
+                  Partition _elem1735;
+                  for (int _i1736 = 0; _i1736 < _list1734.size; ++_i1736)
                   {
-                    _elem1727 = new Partition();
-                    _elem1727.read(iprot);
-                    struct.new_parts.add(_elem1727);
+                    _elem1735 = new Partition();
+                    _elem1735.read(iprot);
+                    struct.new_parts.add(_elem1735);
                   }
                   iprot.readListEnd();
                 }
@@ -143191,9 +143323,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 _iter1729 : struct.new_parts)
+            for (Partition _iter1737 : struct.new_parts)
             {
-              _iter1729.write(oprot);
+              _iter1737.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -143236,9 +143368,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (Partition _iter1730 : struct.new_parts)
+            for (Partition _iter1738 : struct.new_parts)
             {
-              _iter1730.write(oprot);
+              _iter1738.write(oprot);
             }
           }
         }
@@ -143258,14 +143390,14 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1731 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.new_parts = new ArrayList<Partition>(_list1731.size);
-            Partition _elem1732;
-            for (int _i1733 = 0; _i1733 < _list1731.size; ++_i1733)
+            org.apache.thrift.protocol.TList _list1739 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.new_parts = new ArrayList<Partition>(_list1739.size);
+            Partition _elem1740;
+            for (int _i1741 = 0; _i1741 < _list1739.size; ++_i1741)
             {
-              _elem1732 = new Partition();
-              _elem1732.read(iprot);
-              struct.new_parts.add(_elem1732);
+              _elem1740 = new Partition();
+              _elem1740.read(iprot);
+              struct.new_parts.add(_elem1740);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -144318,14 +144450,14 @@ import org.slf4j.LoggerFactory;
             case 3: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1734 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<Partition>(_list1734.size);
-                  Partition _elem1735;
-                  for (int _i1736 = 0; _i1736 < _list1734.size; ++_i1736)
+                  org.apache.thrift.protocol.TList _list1742 = iprot.readListBegin();
+                  struct.new_parts = new ArrayList<Partition>(_list1742.size);
+                  Partition _elem1743;
+                  for (int _i1744 = 0; _i1744 < _list1742.size; ++_i1744)
                   {
-                    _elem1735 = new Partition();
-                    _elem1735.read(iprot);
-                    struct.new_parts.add(_elem1735);
+                    _elem1743 = new Partition();
+                    _elem1743.read(iprot);
+                    struct.new_parts.add(_elem1743);
                   }
                   iprot.readListEnd();
                 }
@@ -144370,9 +144502,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 _iter1737 : struct.new_parts)
+            for (Partition _iter1745 : struct.new_parts)
             {
-              _iter1737.write(oprot);
+              _iter1745.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -144423,9 +144555,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (Partition _iter1738 : struct.new_parts)
+            for (Partition _iter1746 : struct.new_parts)
             {
-              _iter1738.write(oprot);
+              _iter1746.write(oprot);
             }
           }
         }
@@ -144448,14 +144580,14 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1739 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.new_parts = new ArrayList<Partition>(_list1739.size);
-            Partition _elem1740;
-            for (int _i1741 = 0; _i1741 < _list1739.size; ++_i1741)
+            org.apache.thrift.protocol.TList _list1747 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.new_parts = new ArrayList<Partition>(_list1747.size);
+            Partition _elem1748;
+            for (int _i1749 = 0; _i1749 < _list1747.size; ++_i1749)
             {
-              _elem1740 = new Partition();
-              _elem1740.read(iprot);
-              struct.new_parts.add(_elem1740);
+              _elem1748 = new Partition();
+              _elem1748.read(iprot);
+              struct.new_parts.add(_elem1748);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -147594,13 +147726,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1742 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list1742.size);
-                  String _elem1743;
-                  for (int _i1744 = 0; _i1744 < _list1742.size; ++_i1744)
+                  org.apache.thrift.protocol.TList _list1750 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list1750.size);
+                  String _elem1751;
+                  for (int _i1752 = 0; _i1752 < _list1750.size; ++_i1752)
                   {
-                    _elem1743 = iprot.readString();
-                    struct.part_vals.add(_elem1743);
+                    _elem1751 = iprot.readString();
+                    struct.part_vals.add(_elem1751);
                   }
                   iprot.readListEnd();
                 }
@@ -147645,9 +147777,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter1745 : struct.part_vals)
+            for (String _iter1753 : struct.part_vals)
             {
-              oprot.writeString(_iter1745);
+              oprot.writeString(_iter1753);
             }
             oprot.writeListEnd();
           }
@@ -147698,9 +147830,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter1746 : struct.part_vals)
+            for (String _iter1754 : struct.part_vals)
             {
-              oprot.writeString(_iter1746);
+              oprot.writeString(_iter1754);
             }
           }
         }
@@ -147723,13 +147855,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1747 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list1747.size);
-            String _elem1748;
-            for (int _i1749 = 0; _i1749 < _list1747.size; ++_i1749)
+            org.apache.thrift.protocol.TList _list1755 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list1755.size);
+            String _elem1756;
+            for (int _i1757 = 0; _i1757 < _list1755.size; ++_i1757)
             {
-              _elem1748 = iprot.readString();
-              struct.part_vals.add(_elem1748);
+              _elem1756 = iprot.readString();
+              struct.part_vals.add(_elem1756);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -149541,13 +149673,13 @@ import org.slf4j.LoggerFactory;
             case 1: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1750 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list1750.size);
-                  String _elem1751;
-                  for (int _i1752 = 0; _i1752 < _list1750.size; ++_i1752)
+                  org.apache.thrift.protocol.TList _list1758 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list1758.size);
+                  String _elem1759;
+                  for (int _i1760 = 0; _i1760 < _list1758.size; ++_i1760)
                   {
-                    _elem1751 = iprot.readString();
-                    struct.part_vals.add(_elem1751);
+                    _elem1759 = iprot.readString();
+                    struct.part_vals.add(_elem1759);
                   }
                   iprot.readListEnd();
                 }
@@ -149581,9 +149713,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter1753 : struct.part_vals)
+            for (String _iter1761 : struct.part_vals)
             {
-              oprot.writeString(_iter1753);
+              oprot.writeString(_iter1761);
             }
             oprot.writeListEnd();
           }
@@ -149620,9 +149752,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter1754 : struct.part_vals)
+            for (String _iter1762 : struct.part_vals)
             {
-              oprot.writeString(_iter1754);
+              oprot.writeString(_iter1762);
             }
           }
         }
@@ -149637,13 +149769,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1755 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list1755.size);
-            String _elem1756;
-            for (int _i1757 = 0; _i1757 < _list1755.size; ++_i1757)
+            org.apache.thrift.protocol.TList _list1763 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list1763.size);
+            String _elem1764;
+            for (int _i1765 = 0; _i1765 < _list1763.size; ++_i1765)
             {
-              _elem1756 = iprot.readString();
-              struct.part_vals.add(_elem1756);
+              _elem1764 = iprot.readString();
+              struct.part_vals.add(_elem1764);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -151798,13 +151930,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1758 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1758.size);
-                  String _elem1759;
-                  for (int _i1760 = 0; _i1760 < _list1758.size; ++_i1760)
+                  org.apache.thrift.protocol.TList _list1766 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1766.size);
+                  String _elem1767;
+                  for (int _i1768 = 0; _i1768 < _list1766.size; ++_i1768)
                   {
-                    _elem1759 = iprot.readString();
-                    struct.success.add(_elem1759);
+                    _elem1767 = iprot.readString();
+                    struct.success.add(_elem1767);
                   }
                   iprot.readListEnd();
                 }
@@ -151839,9 +151971,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 _iter1761 : struct.success)
+            for (String _iter1769 : struct.success)
             {
-              oprot.writeString(_iter1761);
+              oprot.writeString(_iter1769);
             }
             oprot.writeListEnd();
           }
@@ -151880,9 +152012,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1762 : struct.success)
+            for (String _iter1770 : struct.success)
             {
-              oprot.writeString(_iter1762);
+              oprot.writeString(_iter1770);
             }
           }
         }
@@ -151897,13 +152029,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1763 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1763.size);
-            String _elem1764;
-            for (int _i1765 = 0; _i1765 < _list1763.size; ++_i1765)
+            org.apache.thrift.protocol.TList _list1771 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1771.size);
+            String _elem1772;
+            for (int _i1773 = 0; _i1773 < _list1771.size; ++_i1773)
             {
-              _elem1764 = iprot.readString();
-              struct.success.add(_elem1764);
+              _elem1772 = iprot.readString();
+              struct.success.add(_elem1772);
             }
           }
           struct.setSuccessIsSet(true);
@@ -152666,15 +152798,15 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map1766 = iprot.readMapBegin();
-                  struct.success = new HashMap<String,String>(2*_map1766.size);
-                  String _key1767;
-                  String _val1768;
-                  for (int _i1769 = 0; _i1769 < _map1766.size; ++_i1769)
+                  org.apache.thrift.protocol.TMap _map1774 = iprot.readMapBegin();
+                  struct.success = new HashMap<String,String>(2*_map1774.size);
+                  String _key1775;
+                  String _val1776;
+                  for (int _i1777 = 0; _i1777 < _map1774.size; ++_i1777)
                   {
-                    _key1767 = iprot.readString();
-                    _val1768 = iprot.readString();
-                    struct.success.put(_key1767, _val1768);
+                    _key1775 = iprot.readString();
+                    _val1776 = iprot.readString();
+                    struct.success.put(_key1775, _val1776);
                   }
                   iprot.readMapEnd();
                 }
@@ -152709,10 +152841,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.STRING, struct.success.size()));
-            for (Map.Entry<String, String> _iter1770 : struct.success.entrySet())
+            for (Map.Entry<String, String> _iter1778 : struct.success.entrySet())
             {
-              oprot.writeString(_iter1770.getKey());
-              oprot.writeString(_iter1770.getValue());
+              oprot.writeString(_iter1778.getKey());
+              oprot.writeString(_iter1778.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -152751,10 +152883,10 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Map.Entry<String, String> _iter1771 : struct.success.entrySet())
+            for (Map.Entry<String, String> _iter1779 : struct.success.entrySet())
             {
-              oprot.writeString(_iter1771.getKey());
-              oprot.writeString(_iter1771.getValue());
+              oprot.writeString(_iter1779.getKey());
+              oprot.writeString(_iter1779.getValue());
             }
           }
         }
@@ -152769,15 +152901,15 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map1772 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new HashMap<String,String>(2*_map1772.size);
-            String _key1773;
-            String _val1774;
-            for (int _i1775 = 0; _i1775 < _map1772.size; ++_i1775)
+            org.apache.thrift.protocol.TMap _map1780 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new HashMap<String,String>(2*_map1780.size);
+            String _key1781;
+            String _val1782;
+            for (int _i1783 = 0; _i1783 < _map1780.size; ++_i1783)
             {
-              _key1773 = iprot.readString();
-              _val1774 = iprot.readString();
-              struct.success.put(_key1773, _val1774);
+              _key1781 = iprot.readString();
+              _val1782 = iprot.readString();
+              struct.success.put(_key1781, _val1782);
             }
           }
           struct.setSuccessIsSet(true);
@@ -153372,15 +153504,15 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map1776 = iprot.readMapBegin();
-                  struct.part_vals = new HashMap<String,String>(2*_map1776.size);
-                  String _key1777;
-                  String _val1778;
-                  for (int _i1779 = 0; _i1779 < _map1776.size; ++_i1779)
+                  org.apache.thrift.protocol.TMap _map1784 = iprot.readMapBegin();
+                  struct.part_vals = new HashMap<String,String>(2*_map1784.size);
+                  String _key1785;
+                  String _val1786;
+                  for (int _i1787 = 0; _i1787 < _map1784.size; ++_i1787)
                   {
-                    _key1777 = iprot.readString();
-                    _val1778 = iprot.readString();
-                    struct.part_vals.put(_key1777, _val1778);
+                    _key1785 = iprot.readString();
+                    _val1786 = iprot.readString();
+                    struct.part_vals.put(_key1785, _val1786);
                   }
                   iprot.readMapEnd();
                 }
@@ -153424,10 +153556,10 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (Map.Entry<String, String> _iter1780 : struct.part_vals.entrySet())
+            for (Map.Entry<String, String> _iter1788 : struct.part_vals.entrySet())
             {
-              oprot.writeString(_iter1780.getKey());
-              oprot.writeString(_iter1780.getValue());
+              oprot.writeString(_iter1788.getKey());
+              oprot.writeString(_iter1788.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -153478,10 +153610,10 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (Map.Entry<String, String> _iter1781 : struct.part_vals.entrySet())
+            for (Map.Entry<String, String> _iter1789 : struct.part_vals.entrySet())
             {
-              oprot.writeString(_iter1781.getKey());
-              oprot.writeString(_iter1781.getValue());
+              oprot.writeString(_iter1789.getKey());
+              oprot.writeString(_iter1789.getValue());
             }
           }
         }
@@ -153504,15 +153636,15 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TMap _map1782 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new HashMap<String,String>(2*_map1782.size);
-            String _key1783;
-            String _val1784;
-            for (int _i1785 = 0; _i1785 < _map1782.size; ++_i1785)
+            org.apache.thrift.protocol.TMap _map1790 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new HashMap<String,String>(2*_map1790.size);
+            String _key1791;
+            String _val1792;
+            for (int _i1793 = 0; _i1793 < _map1790.size; ++_i1793)
             {
-              _key1783 = iprot.readString();
-              _val1784 = iprot.readString();
-              struct.part_vals.put(_key1783, _val1784);
+              _key1791 = iprot.readString();
+              _val1792 = iprot.readString();
+              struct.part_vals.put(_key1791, _val1792);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -154996,15 +155128,15 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map1786 = iprot.readMapBegin();
-                  struct.part_vals = new HashMap<String,String>(2*_map1786.size);
-                  String _key1787;
-                  String _val1788;
-                  for (int _i1789 = 0; _i1789 < _map1786.size; ++_i1789)
+                  org.apache.thrift.protocol.TMap _map1794 = iprot.readMapBegin();
+                  struct.part_vals = new HashMap<String,String>(2*_map1794.size);
+                  String _key1795;
+                  String _val1796;
+                  for (int _i1797 = 0; _i1797 < _map1794.size; ++_i1797)
                   {
-                    _key1787 = iprot.readString();
-                    _val1788 = iprot.readString();
-                    struct.part_vals.put(_key1787, _val1788);
+                    _key1795 = iprot.readString();
+                    _val1796 = iprot.readString();
+                    struct.part_vals.put(_key1795, _val1796);
                   }
                   iprot.readMapEnd();
                 }
@@ -155048,10 +155180,10 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (Map.Entry<String, String> _iter1790 : struct.part_vals.entrySet())
+            for (Map.Entry<String, String> _iter1798 : struct.part_vals.entrySet())
             {
-              oprot.writeString(_iter1790.getKey());
-              oprot.writeString(_iter1790.getValue());
+              oprot.writeString(_iter1798.getKey());
+              oprot.writeString(_iter1798.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -155102,10 +155234,10 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (Map.Entry<String, String> _iter1791 : struct.part_vals.entrySet())
+            for (Map.Entry<String, String> _iter1799 : struct.part_vals.entrySet())
             {
-              oprot.writeString(_iter1791.getKey());
-              oprot.writeString(_iter1791.getValue());
+              oprot.writeString(_iter1799.getKey());
+              oprot.writeString(_iter1799.getValue());
             }
           }
         }
@@ -155128,15 +155260,15 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TMap _map1792 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new HashMap<String,String>(2*_map1792.size);
-            String _key1793;
-            String _val1794;
-            for (int _i1795 = 0; _i1795 < _map1792.size; ++_i1795)
+            org.apache.thrift.protocol.TMap _map1800 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new HashMap<String,String>(2*_map1800.size);
+            String _key1801;
+            String _val1802;
+            for (int _i1803 = 0; _i1803 < _map1800.size; ++_i1803)
             {
-              _key1793 = iprot.readString();
-              _val1794 = iprot.readString();
-              struct.part_vals.put(_key1793, _val1794);
+              _key1801 = iprot.readString();
+              _val1802 = iprot.readString();
+              struct.part_vals.put(_key1801, _val1802);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -180000,13 +180132,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1796 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1796.size);
-                  String _elem1797;
-                  for (int _i1798 = 0; _i1798 < _list1796.size; ++_i1798)
+                  org.apache.thrift.protocol.TList _list1804 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1804.size);
+                  String _elem1805;
+                  for (int _i1806 = 0; _i1806 < _list1804.size; ++_i1806)
                   {
-                    _elem1797 = iprot.readString();
-                    struct.success.add(_elem1797);
+                    _elem1805 = iprot.readString();
+                    struct.success.add(_elem1805);
                   }
                   iprot.readListEnd();
                 }
@@ -180041,9 +180173,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 _iter1799 : struct.success)
+            for (String _iter1807 : struct.success)
             {
-              oprot.writeString(_iter1799);
+              oprot.writeString(_iter1807);
             }
             oprot.writeListEnd();
           }
@@ -180082,9 +180214,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1800 : struct.success)
+            for (String _iter1808 : struct.success)
             {
-              oprot.writeString(_iter1800);
+              oprot.writeString(_iter1808);
             }
           }
         }
@@ -180099,13 +180231,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1801 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1801.size);
-            String _elem1802;
-            for (int _i1803 = 0; _i1803 < _list1801.size; ++_i1803)
+            org.apache.thrift.protocol.TList _list1809 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1809.size);
+            String _elem1810;
+            for (int _i1811 = 0; _i1811 < _list1809.size; ++_i1811)
             {
-              _elem1802 = iprot.readString();
-              struct.success.add(_elem1802);
+              _elem1810 = iprot.readString();
+              struct.success.add(_elem1810);
             }
           }
           struct.setSuccessIsSet(true);
@@ -184160,13 +184292,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1804 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1804.size);
-                  String _elem1805;
-                  for (int _i1806 = 0; _i1806 < _list1804.size; ++_i1806)
+                  org.apache.thrift.protocol.TList _list1812 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1812.size);
+                  String _elem1813;
+                  for (int _i1814 = 0; _i1814 < _list1812.size; ++_i1814)
                   {
-                    _elem1805 = iprot.readString();
-                    struct.success.add(_elem1805);
+                    _elem1813 = iprot.readString();
+                    struct.success.add(_elem1813);
                   }
                   iprot.readListEnd();
                 }
@@ -184201,9 +184333,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 _iter1807 : struct.success)
+            for (String _iter1815 : struct.success)
             {
-              oprot.writeString(_iter1807);
+              oprot.writeString(_iter1815);
             }
             oprot.writeListEnd();
           }
@@ -184242,9 +184374,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1808 : struct.success)
+            for (String _iter1816 : struct.success)
             {
-              oprot.writeString(_iter1808);
+              oprot.writeString(_iter1816);
             }
           }
         }
@@ -184259,13 +184391,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1809 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1809.size);
-            String _elem1810;
-            for (int _i1811 = 0; _i1811 < _list1809.size; ++_i1811)
+            org.apache.thrift.protocol.TList _list1817 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1817.size);
+            String _elem1818;
+            for (int _i1819 = 0; _i1819 < _list1817.size; ++_i1819)
             {
-              _elem1810 = iprot.readString();
-              struct.success.add(_elem1810);
+              _elem1818 = iprot.readString();
+              struct.success.add(_elem1818);
             }
           }
           struct.setSuccessIsSet(true);
@@ -187556,14 +187688,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1812 = iprot.readListBegin();
-                  struct.success = new ArrayList<Role>(_list1812.size);
-                  Role _elem1813;
-                  for (int _i1814 = 0; _i1814 < _list1812.size; ++_i1814)
+                  org.apache.thrift.protocol.TList _list1820 = iprot.readListBegin();
+                  struct.success = new ArrayList<Role>(_list1820.size);
+                  Role _elem1821;
+                  for (int _i1822 = 0; _i1822 < _list1820.size; ++_i1822)
                   {
-                    _elem1813 = new Role();
-                    _elem1813.read(iprot);
-                    struct.success.add(_elem1813);
+                    _elem1821 = new Role();
+                    _elem1821.read(iprot);
+                    struct.success.add(_elem1821);
                   }
                   iprot.readListEnd();
                 }
@@ -187598,9 +187730,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 (Role _iter1815 : struct.success)
+            for (Role _iter1823 : struct.success)
             {
-              _iter1815.write(oprot);
+              _iter1823.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -187639,9 +187771,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Role _iter1816 : struct.success)
+            for (Role _iter1824 : struct.success)
             {
-              _iter1816.write(oprot);
+              _iter1824.write(oprot);
             }
           }
         }
@@ -187656,14 +187788,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1817 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Role>(_list1817.size);
-            Role _elem1818;
-            for (int _i1819 = 0; _i1819 < _list1817.size; ++_i1819)
+            org.apache.thrift.protocol.TList _list1825 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Role>(_list1825.size);
+            Role _elem1826;
+            for (int _i1827 = 0; _i1827 < _list1825.size; ++_i1827)
             {
-              _elem1818 = new Role();
-              _elem1818.read(iprot);
-              struct.success.add(_elem1818);
+              _elem1826 = new Role();
+              _elem1826.read(iprot);
+              struct.success.add(_elem1826);
             }
           }
           struct.setSuccessIsSet(true);
@@ -190668,13 +190800,13 @@ import org.slf4j.LoggerFactory;
             case 3: // GROUP_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1820 = iprot.readListBegin();
-                  struct.group_names = new ArrayList<String>(_list1820.size);
-                  String _elem1821;
-                  for (int _i1822 = 0; _i1822 < _list1820.size; ++_i1822)
+                  org.apache.thrift.protocol.TList _list1828 = iprot.readListBegin();
+                  struct.group_names = new ArrayList<String>(_list1828.size);
+                  String _elem1829;
+                  for (int _i1830 = 0; _i1830 < _list1828.size; ++_i1830)
                   {
-                    _elem1821 = iprot.readString();
-                    struct.group_names.add(_elem1821);
+                    _elem1829 = iprot.readString();
+                    struct.group_names.add(_elem1829);
                   }
                   iprot.readListEnd();
                 }
@@ -190710,9 +190842,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.group_names.size()));
-            for (String _iter1823 : struct.group_names)
+            for (String _iter1831 : struct.group_names)
             {
-              oprot.writeString(_iter1823);
+              oprot.writeString(_iter1831);
             }
             oprot.writeListEnd();
           }
@@ -190755,9 +190887,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetGroup_names()) {
           {
             oprot.writeI32(struct.group_names.size());
-            for (String _iter1824 : struct.group_names)
+            for (String _iter1832 : struct.group_names)
             {
-              oprot.writeString(_iter1824);
+              oprot.writeString(_iter1832);
             }
           }
         }
@@ -190778,13 +190910,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1825 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.group_names = new ArrayList<String>(_list1825.size);
-            String _elem1826;
-            for (int _i1827 = 0; _i1827 < _list1825.size; ++_i1827)
+            org.apache.thrift.protocol.TList _list1833 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.group_names = new ArrayList<String>(_list1833.size);
+            String _elem1834;
+            for (int _i1835 = 0; _i1835 < _list1833.size; ++_i1835)
             {
-              _elem1826 = iprot.readString();
-              struct.group_names.add(_elem1826);
+              _elem1834 = iprot.readString();
+              struct.group_names.add(_elem1834);
             }
           }
           struct.setGroup_namesIsSet(true);
@@ -192242,14 +192374,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1828 = iprot.readListBegin();
-                  struct.success = new ArrayList<HiveObjectPrivilege>(_list1828.size);
-                  HiveObjectPrivilege _elem1829;
-                  for (int _i1830 = 0; _i1830 < _list1828.size; ++_i1830)
+                  org.apache.thrift.protocol.TList _list1836 = iprot.readListBegin();
+                  struct.success = new ArrayList<HiveObjectPrivilege>(_list1836.size);
+                  HiveObjectPrivilege _elem1837;
+                  for (int _i1838 = 0; _i1838 < _list1836.size; ++_i1838)
                   {
-                    _elem1829 = new HiveObjectPrivilege();
-                    _elem1829.read(iprot);
-                    struct.success.add(_elem1829);
+                    _elem1837 = new HiveObjectPrivilege();
+                    _elem1837.read(iprot);
+                    struct.success.add(_elem1837);
                   }
                   iprot.readListEnd();
                 }
@@ -192284,9 +192416,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 (HiveObjectPrivilege _iter1831 : struct.success)
+            for (HiveObjectPrivilege _iter1839 : struct.success)
             {
-              _iter1831.write(oprot);
+              _iter1839.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -192325,9 +192457,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (HiveObjectPrivilege _iter1832 : struct.success)
+            for (HiveObjectPrivilege _iter1840 : struct.success)
             {
-              _iter1832.write(oprot);
+              _iter1840.write(oprot);
             }
           }
         }
@@ -192342,14 +192474,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1833 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<HiveObjectPrivilege>(_list1833.size);
-            HiveObjectPrivilege _elem1834;
-            for (int _i1835 = 0; _i1835 < _list1833.size; ++_i1835)
+            org.apache.thrift.protocol.TList _list1841 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<HiveObjectPrivilege>(_list1841.size);
+            HiveObjectPrivilege _elem1842;
+            for (int _i1843 = 0; _i1843 < _list1841.size; ++_i1843)
             {
-              _elem1834 = new HiveObjectPrivilege();
-              _elem1834.read(iprot);
-              struct.success.add(_elem1834);
+              _elem1842 = new HiveObjectPrivilege();
+              _elem1842.read(iprot);
+              struct.success.add(_elem1842);
             }
           }
           struct.setSuccessIsSet(true);
@@ -196296,13 +196428,13 @@ import org.slf4j.LoggerFactory;
             case 2: // GROUP_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1836 = iprot.readListBegin();
-                  struct.group_names = new ArrayList<String>(_list1836.size);
-                  String _elem1837;
-                  for (int _i1838 = 0; _i1838 < _list1836.size; ++_i1838)
+                  org.apache.thrift.protocol.TList _list1844 = iprot.readListBegin();
+                  struct.group_names = new ArrayList<String>(_list1844.size);
+                  String _elem1845;
+                  for (int _i1846 = 0; _i1846 < _list1844.size; ++_i1846)
                   {
-                    _elem1837 = iprot.readString();
-                    struct.group_names.add(_elem1837);
+                    _elem1845 = iprot.readString();
+                    struct.group_names.add(_elem1845);
                   }
                   iprot.readListEnd();
                 }
@@ -196333,9 +196465,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.group_names.size()));
-            for (String _iter1839 : struct.group_names)
+            for (String _iter1847 : struct.group_names)
             {
-              oprot.writeString(_iter1839);
+              oprot.writeString(_iter1847);
             }
             oprot.writeListEnd();
           }
@@ -196372,9 +196504,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetGroup_names()) {
           {
             oprot.writeI32(struct.group_names.size());
-            for (String _iter1840 : struct.group_names)
+            for (String _iter1848 : struct.group_names)
             {
-              oprot.writeString(_iter1840);
+              oprot.writeString(_iter1848);
             }
           }
         }
@@ -196390,13 +196522,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list1841 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.group_names = new ArrayList<String>(_list1841.size);
-            String _elem1842;
-            for (int _i1843 = 0; _i1843 < _list1841.size; ++_i1843)
+            org.apache.thrift.protocol.TList _list1849 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.group_names = new ArrayList<String>(_list1849.size);
+            String _elem1850;
+            for (int _i1851 = 0; _i1851 < _list1849.size; ++_i1851)
             {
-              _elem1842 = iprot.readString();
-              struct.group_names.add(_elem1842);
+              _elem1850 = iprot.readString();
+              struct.group_names.add(_elem1850);
             }
           }
           struct.setGroup_namesIsSet(true);
@@ -196799,13 +196931,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1844 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1844.size);
-                  String _elem1845;
-                  for (int _i1846 = 0; _i1846 < _list1844.size; ++_i1846)
+                  org.apache.thrift.protocol.TList _list1852 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1852.size);
+                  String _elem1853;
+                  for (int _i1854 = 0; _i1854 < _list1852.size; ++_i1854)
                   {
-                    _elem1845 = iprot.readString();
-                    struct.success.add(_elem1845);
+                    _elem1853 = iprot.readString();
+                    struct.success.add(_elem1853);
                   }
                   iprot.readListEnd();
                 }
@@ -196840,9 +196972,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 _iter1847 : struct.success)
+            for (String _iter1855 : struct.success)
             {
-              oprot.writeString(_iter1847);
+              oprot.writeString(_iter1855);
             }
             oprot.writeListEnd();
           }
@@ -196881,9 +197013,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1848 : struct.success)
+            for (String _iter1856 : struct.success)
             {
-              oprot.writeString(_iter1848);
+              oprot.writeString(_iter1856);
             }
           }
         }
@@ -196898,13 +197030,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1849 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1849.size);
-            String _elem1850;
-            for (int _i1851 = 0; _i1851 < _list1849.size; ++_i1851)
+            org.apache.thrift.protocol.TList _list1857 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1857.size);
+            String _elem1858;
+            for (int _i1859 = 0; _i1859 < _list1857.size; ++_i1859)
             {
-              _elem1850 = iprot.readString();
-              struct.success.add(_elem1850);
+              _elem1858 = iprot.readString();
+              struct.success.add(_elem1858);
             }
           }
           struct.setSuccessIsSet(true);
@@ -202195,13 +202327,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1852 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1852.size);
-                  String _elem1853;
-                  for (int _i1854 = 0; _i1854 < _list1852.size; ++_i1854)
+                  org.apache.thrift.protocol.TList _list1860 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1860.size);
+                  String _elem1861;
+                  for (int _i1862 = 0; _i1862 < _list1860.size; ++_i1862)
                   {
-                    _elem1853 = iprot.readString();
-                    struct.success.add(_elem1853);
+                    _elem1861 = iprot.readString();
+                    struct.success.add(_elem1861);
                   }
                   iprot.readListEnd();
                 }
@@ -202227,9 +202359,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 _iter1855 : struct.success)
+            for (String _iter1863 : struct.success)
             {
-              oprot.writeString(_iter1855);
+              oprot.writeString(_iter1863);
             }
             oprot.writeListEnd();
           }
@@ -202260,9 +202392,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1856 : struct.success)
+            for (String _iter1864 : struct.success)
             {
-              oprot.writeString(_iter1856);
+              oprot.writeString(_iter1864);
             }
           }
         }
@@ -202274,13 +202406,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1857 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1857.size);
-            String _elem1858;
-            for (int _i1859 = 0; _i1859 < _list1857.size; ++_i1859)
+            org.apache.thrift.protocol.TList _list1865 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1865.size);
+            String _elem1866;
+            for (int _i1867 = 0; _i1867 < _list1865.size; ++_i1867)
             {
-              _elem1858 = iprot.readString();
-              struct.success.add(_elem1858);
+              _elem1866 = iprot.readString();
+              struct.success.add(_elem1866);
             }
           }
           struct.setSuccessIsSet(true);
@@ -205310,13 +205442,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1860 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1860.size);
-                  String _elem1861;
-                  for (int _i1862 = 0; _i1862 < _list1860.size; ++_i1862)
+                  org.apache.thrift.protocol.TList _list1868 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1868.size);
+                  String _elem1869;
+                  for (int _i1870 = 0; _i1870 < _list1868.size; ++_i1870)
                   {
-                    _elem1861 = iprot.readString();
-                    struct.success.add(_elem1861);
+                    _elem1869 = iprot.readString();
+                    struct.success.add(_elem1869);
                   }
                   iprot.readListEnd();
                 }
@@ -205342,9 +205474,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 _iter1863 : struct.success)
+            for (String _iter1871 : struct.success)
             {
-              oprot.writeString(_iter1863);
+              oprot.writeString(_iter1871);
             }
             oprot.writeListEnd();
           }
@@ -205375,9 +205507,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1864 : struct.success)
+            for (String _iter1872 : struct.success)
             {
-              oprot.writeString(_iter1864);
+              oprot.writeString(_iter1872);
             }
           }
         }
@@ -205389,13 +205521,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1865 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1865.size);
-            String _elem1866;
-            for (int _i1867 = 0; _i1867 < _list1865.size; ++_i1867)
+            org.apache.thrift.protocol.TList _list1873 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1873.size);
+            String _elem1874;
+            for (int _i1875 = 0; _i1875 < _list1873.size; ++_i1875)
             {
-              _elem1866 = iprot.readString();
-              struct.success.add(_elem1866);
+              _elem1874 = iprot.readString();
+              struct.success.add(_elem1874);
             }
           }
           struct.setSuccessIsSet(true);
@@ -224794,13 +224926,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1868 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1868.size);
-                  String _elem1869;
-                  for (int _i1870 = 0; _i1870 < _list1868.size; ++_i1870)
+                  org.apache.thrift.protocol.TList _list1876 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1876.size);
+                  String _elem1877;
+                  for (int _i1878 = 0; _i1878 < _list1876.size; ++_i1878)
                   {
-                    _elem1869 = iprot.readString();
-                    struct.success.add(_elem1869);
+                    _elem1877 = iprot.readString();
+                    struct.success.add(_elem1877);
                   }
                   iprot.readListEnd();
                 }
@@ -224826,9 +224958,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 _iter1871 : struct.success)
+            for (String _iter1879 : struct.success)
             {
-              oprot.writeString(_iter1871);
+              oprot.writeString(_iter1879);
             }
             oprot.writeListEnd();
           }
@@ -224859,9 +224991,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1872 : struct.success)
+            for (String _iter1880 : struct.success)
             {
-              oprot.writeString(_iter1872);
+              oprot.writeString(_iter1880);
             }
           }
         }
@@ -224873,13 +225005,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1873 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1873.size);
-            String _elem1874;
-            for (int _i1875 = 0; _i1875 < _list1873.size; ++_i1875)
+            org.apache.thrift.protocol.TList _list1881 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1881.size);
+            String _elem1882;
+            for (int _i1883 = 0; _i1883 < _list1881.size; ++_i1883)
             {
-              _elem1874 = iprot.readString();
-              struct.success.add(_elem1874);
+              _elem1882 = iprot.readString();
+              struct.success.add(_elem1882);
             }
           }
           struct.setSuccessIsSet(true);
@@ -261765,14 +261897,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1876 = iprot.readListBegin();
-                  struct.success = new ArrayList<SchemaVersion>(_list1876.size);
-                  SchemaVersion _elem1877;
-                  for (int _i1878 = 0; _i1878 < _list1876.size; ++_i1878)
+                  org.apache.thrift.protocol.TList _list1884 = iprot.readListBegin();
+                  struct.success = new ArrayList<SchemaVersion>(_list1884.size);
+                  SchemaVersion _elem1885;
+                  for (int _i1886 = 0; _i1886 < _list1884.size; ++_i1886)
                   {
-                    _elem1877 = new SchemaVersion();
-                    _elem1877.read(iprot);
-                    struct.success.add(_elem1877);
+                    _elem1885 = new SchemaVersion();
+                    _elem1885.read(iprot);
+                    struct.success.add(_elem1885);
                   }
                   iprot.readListEnd();
                 }
@@ -261816,9 +261948,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 (SchemaVersion _iter1879 : struct.success)
+            for (SchemaVersion _iter1887 : struct.success)
             {
-              _iter1879.write(oprot);
+              _iter1887.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -261865,9 +261997,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (SchemaVersion _iter1880 : struct.success)
+            for (SchemaVersion _iter1888 : struct.success)
             {
-              _iter1880.write(oprot);
+              _iter1888.write(oprot);
             }
           }
         }
@@ -261885,14 +262017,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1881 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<SchemaVersion>(_list1881.size);
-            SchemaVersion _elem1882;
-            for (int _i1883 = 0; _i1883 < _list1881.size; ++_i1883)
+            org.apache.thrift.protocol.TList _list1889 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<SchemaVersion>(_list1889.size);
+            SchemaVersion _elem1890;
+            for (int _i1891 = 0; _i1891 < _list1889.size; ++_i1891)
             {
-              _elem1882 = new SchemaVersion();
-              _elem1882.read(iprot);
-              struct.success.add(_elem1882);
+              _elem1890 = new SchemaVersion();
+              _elem1890.read(iprot);
+              struct.success.add(_elem1890);
             }
           }
           struct.setSuccessIsSet(true);
@@ -270435,14 +270567,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1884 = iprot.readListBegin();
-                  struct.success = new ArrayList<RuntimeStat>(_list1884.size);
-                  RuntimeStat _elem1885;
-                  for (int _i1886 = 0; _i1886 < _list1884.size; ++_i1886)
+                  org.apache.thrift.protocol.TList _list1892 = iprot.readListBegin();
+                  struct.success = new ArrayList<RuntimeStat>(_list1892.size);
+                  RuntimeStat _elem1893;
+                  for (int _i1894 = 0; _i1894 < _list1892.size; ++_i1894)
                   {
-                    _elem1885 = new RuntimeStat();
-                    _elem1885.read(iprot);
-                    struct.success.add(_elem1885);
+                    _elem1893 = new RuntimeStat();
+                    _elem1893.read(iprot);
+                    struct.success.add(_elem1893);
                   }
                   iprot.readListEnd();
                 }
@@ -270477,9 +270609,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 (RuntimeStat _iter1887 : struct.success)
+            for (RuntimeStat _iter1895 : struct.success)
             {
-              _iter1887.write(oprot);
+              _iter1895.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -270518,9 +270650,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (RuntimeStat _iter1888 : struct.success)
+            for (RuntimeStat _iter1896 : struct.success)
             {
-              _iter1888.write(oprot);
+              _iter1896.write(oprot);
             }
           }
         }
@@ -270535,14 +270667,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1889 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<RuntimeStat>(_list1889.size);
-            RuntimeStat _elem1890;
-            for (int _i1891 = 0; _i1891 < _list1889.size; ++_i1891)
+            org.apache.thrift.protocol.TList _list1897 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<RuntimeStat>(_list1897.size);
+            RuntimeStat _elem1898;
+            for (int _i1899 = 0; _i1899 < _list1897.size; ++_i1899)
             {
-              _elem1890 = new RuntimeStat();
-              _elem1890.read(iprot);
-              struct.success.add(_elem1890);
+              _elem1898 = new RuntimeStat();
+              _elem1898.read(iprot);
+              struct.success.add(_elem1898);
             }
           }
           struct.setSuccessIsSet(true);
@@ -275104,70 +275236,793 @@ import org.slf4j.LoggerFactory;
     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.REPLICATION_METRIC_LIST, new org.apache.thrift.meta_data.FieldMetaData("replicationMetricList", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ReplicationMetricList.class)));
+      tmpMap.put(_Fields.REPLICATION_METRIC_LIST, new org.apache.thrift.meta_data.FieldMetaData("replicationMetricList", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ReplicationMetricList.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(add_replication_metrics_args.class, metaDataMap);
+    }
+
+    public add_replication_metrics_args() {
+    }
+
+    public add_replication_metrics_args(
+      ReplicationMetricList replicationMetricList)
+    {
+      this();
+      this.replicationMetricList = replicationMetricList;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public add_replication_metrics_args(add_replication_metrics_args other) {
+      if (other.isSetReplicationMetricList()) {
+        this.replicationMetricList = new ReplicationMetricList(other.replicationMetricList);
+      }
+    }
+
+    public add_replication_metrics_args deepCopy() {
+      return new add_replication_metrics_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.replicationMetricList = null;
+    }
+
+    public ReplicationMetricList getReplicationMetricList() {
+      return this.replicationMetricList;
+    }
+
+    public void setReplicationMetricList(ReplicationMetricList replicationMetricList) {
+      this.replicationMetricList = replicationMetricList;
+    }
+
+    public void unsetReplicationMetricList() {
+      this.replicationMetricList = null;
+    }
+
+    /** Returns true if field replicationMetricList is set (has been assigned a value) and false otherwise */
+    public boolean isSetReplicationMetricList() {
+      return this.replicationMetricList != null;
+    }
+
+    public void setReplicationMetricListIsSet(boolean value) {
+      if (!value) {
+        this.replicationMetricList = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case REPLICATION_METRIC_LIST:
+        if (value == null) {
+          unsetReplicationMetricList();
+        } else {
+          setReplicationMetricList((ReplicationMetricList)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case REPLICATION_METRIC_LIST:
+        return getReplicationMetricList();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case REPLICATION_METRIC_LIST:
+        return isSetReplicationMetricList();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof add_replication_metrics_args)
+        return this.equals((add_replication_metrics_args)that);
+      return false;
+    }
+
+    public boolean equals(add_replication_metrics_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_replicationMetricList = true && this.isSetReplicationMetricList();
+      boolean that_present_replicationMetricList = true && that.isSetReplicationMetricList();
+      if (this_present_replicationMetricList || that_present_replicationMetricList) {
+        if (!(this_present_replicationMetricList && that_present_replicationMetricList))
+          return false;
+        if (!this.replicationMetricList.equals(that.replicationMetricList))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_replicationMetricList = true && (isSetReplicationMetricList());
+      list.add(present_replicationMetricList);
+      if (present_replicationMetricList)
+        list.add(replicationMetricList);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(add_replication_metrics_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(isSetReplicationMetricList()).compareTo(other.isSetReplicationMetricList());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetReplicationMetricList()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.replicationMetricList, other.replicationMetricList);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("add_replication_metrics_args(");
+      boolean first = true;
+
+      sb.append("replicationMetricList:");
+      if (this.replicationMetricList == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.replicationMetricList);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (replicationMetricList != null) {
+        replicationMetricList.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class add_replication_metrics_argsStandardSchemeFactory implements SchemeFactory {
+      public add_replication_metrics_argsStandardScheme getScheme() {
+        return new add_replication_metrics_argsStandardScheme();
+      }
+    }
+
+    private static class add_replication_metrics_argsStandardScheme extends StandardScheme<add_replication_metrics_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, add_replication_metrics_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // REPLICATION_METRIC_LIST
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.replicationMetricList = new ReplicationMetricList();
+                struct.replicationMetricList.read(iprot);
+                struct.setReplicationMetricListIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, add_replication_metrics_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.replicationMetricList != null) {
+          oprot.writeFieldBegin(REPLICATION_METRIC_LIST_FIELD_DESC);
+          struct.replicationMetricList.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class add_replication_metrics_argsTupleSchemeFactory implements SchemeFactory {
+      public add_replication_metrics_argsTupleScheme getScheme() {
+        return new add_replication_metrics_argsTupleScheme();
+      }
+    }
+
+    private static class add_replication_metrics_argsTupleScheme extends TupleScheme<add_replication_metrics_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, add_replication_metrics_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetReplicationMetricList()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetReplicationMetricList()) {
+          struct.replicationMetricList.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, add_replication_metrics_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.replicationMetricList = new ReplicationMetricList();
+          struct.replicationMetricList.read(iprot);
+          struct.setReplicationMetricListIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_replication_metrics_result implements org.apache.thrift.TBase<add_replication_metrics_result, add_replication_metrics_result._Fields>, java.io.Serializable, Cloneable, Comparable<add_replication_metrics_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("add_replication_metrics_result");
+
+    private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new add_replication_metrics_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new add_replication_metrics_resultTupleSchemeFactory());
+    }
+
+    private MetaException o1; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      O1((short)1, "o1");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // O1
+            return O1;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    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.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(add_replication_metrics_result.class, metaDataMap);
+    }
+
+    public add_replication_metrics_result() {
+    }
+
+    public add_replication_metrics_result(
+      MetaException o1)
+    {
+      this();
+      this.o1 = o1;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public add_replication_metrics_result(add_replication_metrics_result other) {
+      if (other.isSetO1()) {
+        this.o1 = new MetaException(other.o1);
+      }
+    }
+
+    public add_replication_metrics_result deepCopy() {
+      return new add_replication_metrics_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.o1 = null;
+    }
+
+    public MetaException getO1() {
+      return this.o1;
+    }
+
+    public void setO1(MetaException o1) {
+      this.o1 = o1;
+    }
+
+    public void unsetO1() {
+      this.o1 = null;
+    }
+
+    /** Returns true if field o1 is set (has been assigned a value) and false otherwise */
+    public boolean isSetO1() {
+      return this.o1 != null;
+    }
+
+    public void setO1IsSet(boolean value) {
+      if (!value) {
+        this.o1 = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case O1:
+        if (value == null) {
+          unsetO1();
+        } else {
+          setO1((MetaException)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case O1:
+        return getO1();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case O1:
+        return isSetO1();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof add_replication_metrics_result)
+        return this.equals((add_replication_metrics_result)that);
+      return false;
+    }
+
+    public boolean equals(add_replication_metrics_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_o1 = true && this.isSetO1();
+      boolean that_present_o1 = true && that.isSetO1();
+      if (this_present_o1 || that_present_o1) {
+        if (!(this_present_o1 && that_present_o1))
+          return false;
+        if (!this.o1.equals(that.o1))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_o1 = true && (isSetO1());
+      list.add(present_o1);
+      if (present_o1)
+        list.add(o1);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(add_replication_metrics_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(isSetO1()).compareTo(other.isSetO1());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetO1()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, other.o1);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+      }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("add_replication_metrics_result(");
+      boolean first = true;
+
+      sb.append("o1:");
+      if (this.o1 == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.o1);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class add_replication_metrics_resultStandardSchemeFactory implements SchemeFactory {
+      public add_replication_metrics_resultStandardScheme getScheme() {
+        return new add_replication_metrics_resultStandardScheme();
+      }
+    }
+
+    private static class add_replication_metrics_resultStandardScheme extends StandardScheme<add_replication_metrics_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, add_replication_metrics_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // O1
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.o1 = new MetaException();
+                struct.o1.read(iprot);
+                struct.setO1IsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, add_replication_metrics_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.o1 != null) {
+          oprot.writeFieldBegin(O1_FIELD_DESC);
+          struct.o1.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class add_replication_metrics_resultTupleSchemeFactory implements SchemeFactory {
+      public add_replication_metrics_resultTupleScheme getScheme() {
+        return new add_replication_metrics_resultTupleScheme();
+      }
+    }
+
+    private static class add_replication_metrics_resultTupleScheme extends TupleScheme<add_replication_metrics_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, add_replication_metrics_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetO1()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetO1()) {
+          struct.o1.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, add_replication_metrics_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.o1 = new MetaException();
+          struct.o1.read(iprot);
+          struct.setO1IsSet(true);
+        }
+      }
+    }
+
+  }
+
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_replication_metrics_args implements org.apache.thrift.TBase<get_replication_metrics_args, get_replication_metrics_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_replication_metrics_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_replication_metrics_args");
+
+    private static final org.apache.thrift.protocol.TField RQST_FIELD_DESC = new org.apache.thrift.protocol.TField("rqst", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new get_replication_metrics_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_replication_metrics_argsTupleSchemeFactory());
+    }
+
+    private GetReplicationMetricsRequest rqst; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      RQST((short)1, "rqst");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // RQST
+            return RQST;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    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.RQST, new org.apache.thrift.meta_data.FieldMetaData("rqst", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetReplicationMetricsRequest.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(add_replication_metrics_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_replication_metrics_args.class, metaDataMap);
     }
 
-    public add_replication_metrics_args() {
+    public get_replication_metrics_args() {
     }
 
-    public add_replication_metrics_args(
-      ReplicationMetricList replicationMetricList)
+    public get_replication_metrics_args(
+      GetReplicationMetricsRequest rqst)
     {
       this();
-      this.replicationMetricList = replicationMetricList;
+      this.rqst = rqst;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public add_replication_metrics_args(add_replication_metrics_args other) {
-      if (other.isSetReplicationMetricList()) {
-        this.replicationMetricList = new ReplicationMetricList(other.replicationMetricList);
+    public get_replication_metrics_args(get_replication_metrics_args other) {
+      if (other.isSetRqst()) {
+        this.rqst = new GetReplicationMetricsRequest(other.rqst);
       }
     }
 
-    public add_replication_metrics_args deepCopy() {
-      return new add_replication_metrics_args(this);
+    public get_replication_metrics_args deepCopy() {
+      return new get_replication_metrics_args(this);
     }
 
     @Override
     public void clear() {
-      this.replicationMetricList = null;
+      this.rqst = null;
     }
 
-    public ReplicationMetricList getReplicationMetricList() {
-      return this.replicationMetricList;
+    public GetReplicationMetricsRequest getRqst() {
+      return this.rqst;
     }
 
-    public void setReplicationMetricList(ReplicationMetricList replicationMetricList) {
-      this.replicationMetricList = replicationMetricList;
+    public void setRqst(GetReplicationMetricsRequest rqst) {
+      this.rqst = rqst;
     }
 
-    public void unsetReplicationMetricList() {
-      this.replicationMetricList = null;
+    public void unsetRqst() {
+      this.rqst = null;
     }
 
-    /** Returns true if field replicationMetricList is set (has been assigned a value) and false otherwise */
-    public boolean isSetReplicationMetricList() {
-      return this.replicationMetricList != null;
+    /** Returns true if field rqst is set (has been assigned a value) and false otherwise */
+    public boolean isSetRqst() {
+      return this.rqst != null;
     }
 
-    public void setReplicationMetricListIsSet(boolean value) {
+    public void setRqstIsSet(boolean value) {
       if (!value) {
-        this.replicationMetricList = null;
+        this.rqst = null;
       }
     }
 
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
-      case REPLICATION_METRIC_LIST:
+      case RQST:
         if (value == null) {
-          unsetReplicationMetricList();
+          unsetRqst();
         } else {
-          setReplicationMetricList((ReplicationMetricList)value);
+          setRqst((GetReplicationMetricsRequest)value);
         }
         break;
 
@@ -275176,8 +276031,8 @@ import org.slf4j.LoggerFactory;
 
     public Object getFieldValue(_Fields field) {
       switch (field) {
-      case REPLICATION_METRIC_LIST:
-        return getReplicationMetricList();
+      case RQST:
+        return getRqst();
 
       }
       throw new IllegalStateException();
@@ -275190,8 +276045,8 @@ import org.slf4j.LoggerFactory;
       }
 
       switch (field) {
-      case REPLICATION_METRIC_LIST:
-        return isSetReplicationMetricList();
+      case RQST:
+        return isSetRqst();
       }
       throw new IllegalStateException();
     }
@@ -275200,21 +276055,21 @@ import org.slf4j.LoggerFactory;
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof add_replication_metrics_args)
-        return this.equals((add_replication_metrics_args)that);
+      if (that instanceof get_replication_metrics_args)
+        return this.equals((get_replication_metrics_args)that);
       return false;
     }
 
-    public boolean equals(add_replication_metrics_args that) {
+    public boolean equals(get_replication_metrics_args that) {
       if (that == null)
         return false;
 
-      boolean this_present_replicationMetricList = true && this.isSetReplicationMetricList();
-      boolean that_present_replicationMetricList = true && that.isSetReplicationMetricList();
-      if (this_present_replicationMetricList || that_present_replicationMetricList) {
-        if (!(this_present_replicationMetricList && that_present_replicationMetricList))
+      boolean this_present_rqst = true && this.isSetRqst();
+      boolean that_present_rqst = true && that.isSetRqst();
+      if (this_present_rqst || that_present_rqst) {
+        if (!(this_present_rqst && that_present_rqst))
           return false;
-        if (!this.replicationMetricList.equals(that.replicationMetricList))
+        if (!this.rqst.equals(that.rqst))
           return false;
       }
 
@@ -275225,28 +276080,28 @@ import org.slf4j.LoggerFactory;
     public int hashCode() {
       List<Object> list = new ArrayList<Object>();
 
-      boolean present_replicationMetricList = true && (isSetReplicationMetricList());
-      list.add(present_replicationMetricList);
-      if (present_replicationMetricList)
-        list.add(replicationMetricList);
+      boolean present_rqst = true && (isSetRqst());
+      list.add(present_rqst);
+      if (present_rqst)
+        list.add(rqst);
 
       return list.hashCode();
     }
 
     @Override
-    public int compareTo(add_replication_metrics_args other) {
+    public int compareTo(get_replication_metrics_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
 
-      lastComparison = Boolean.valueOf(isSetReplicationMetricList()).compareTo(other.isSetReplicationMetricList());
+      lastComparison = Boolean.valueOf(isSetRqst()).compareTo(other.isSetRqst());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetReplicationMetricList()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.replicationMetricList, other.replicationMetricList);
+      if (isSetRqst()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rqst, other.rqst);
         if (lastComparison != 0) {
           return lastComparison;
         }
@@ -275268,14 +276123,14 @@ import org.slf4j.LoggerFactory;
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("add_replication_metrics_args(");
+      StringBuilder sb = new StringBuilder("get_replication_metrics_args(");
       boolean first = true;
 
-      sb.append("replicationMetricList:");
-      if (this.replicationMetricList == null) {
+      sb.append("rqst:");
+      if (this.rqst == null) {
         sb.append("null");
       } else {
-        sb.append(this.replicationMetricList);
+        sb.append(this.rqst);
       }
       first = false;
       sb.append(")");
@@ -275285,8 +276140,8 @@ import org.slf4j.LoggerFactory;
     public void validate() throws org.apache.thrift.TException {
       // check for required fields
       // check for sub-struct validity
-      if (replicationMetricList != null) {
-        replicationMetricList.validate();
+      if (rqst != null) {
+        rqst.validate();
       }
     }
 
@@ -275306,15 +276161,15 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    private static class add_replication_metrics_argsStandardSchemeFactory implements SchemeFactory {
-      public add_replication_metrics_argsStandardScheme getScheme() {
-        return new add_replication_metrics_argsStandardScheme();
+    private static class get_replication_metrics_argsStandardSchemeFactory implements SchemeFactory {
+      public get_replication_metrics_argsStandardScheme getScheme() {
+        return new get_replication_metrics_argsStandardScheme();
       }
     }
 
-    private static class add_replication_metrics_argsStandardScheme extends StandardScheme<add_replication_metrics_args> {
+    private static class get_replication_metrics_argsStandardScheme extends StandardScheme<get_replication_metrics_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, add_replication_metrics_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_replication_metrics_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -275324,11 +276179,11 @@ import org.slf4j.LoggerFactory;
             break;
           }
           switch (schemeField.id) {
-            case 1: // REPLICATION_METRIC_LIST
+            case 1: // RQST
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.replicationMetricList = new ReplicationMetricList();
-                struct.replicationMetricList.read(iprot);
-                struct.setReplicationMetricListIsSet(true);
+                struct.rqst = new GetReplicationMetricsRequest();
+                struct.rqst.read(iprot);
+                struct.setRqstIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
@@ -275342,13 +276197,13 @@ import org.slf4j.LoggerFactory;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, add_replication_metrics_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, get_replication_metrics_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.replicationMetricList != null) {
-          oprot.writeFieldBegin(REPLICATION_METRIC_LIST_FIELD_DESC);
-          struct.replicationMetricList.write(oprot);
+        if (struct.rqst != null) {
+          oprot.writeFieldBegin(RQST_FIELD_DESC);
+          struct.rqst.write(oprot);
           oprot.writeFieldEnd();
         }
         oprot.writeFieldStop();
@@ -275357,56 +276212,59 @@ import org.slf4j.LoggerFactory;
 
     }
 
-    private static class add_replication_metrics_argsTupleSchemeFactory implements SchemeFactory {
-      public add_replication_metrics_argsTupleScheme getScheme() {
-        return new add_replication_metrics_argsTupleScheme();
+    private static class get_replication_metrics_argsTupleSchemeFactory implements SchemeFactory {
+      public get_replication_metrics_argsTupleScheme getScheme() {
+        return new get_replication_metrics_argsTupleScheme();
       }
     }
 
-    private static class add_replication_metrics_argsTupleScheme extends TupleScheme<add_replication_metrics_args> {
+    private static class get_replication_metrics_argsTupleScheme extends TupleScheme<get_replication_metrics_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, add_replication_metrics_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_replication_metrics_args struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
-        if (struct.isSetReplicationMetricList()) {
+        if (struct.isSetRqst()) {
           optionals.set(0);
         }
         oprot.writeBitSet(optionals, 1);
-        if (struct.isSetReplicationMetricList()) {
-          struct.replicationMetricList.write(oprot);
+        if (struct.isSetRqst()) {
+          struct.rqst.write(oprot);
         }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, add_replication_metrics_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_replication_metrics_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
-          struct.replicationMetricList = new ReplicationMetricList();
-          struct.replicationMetricList.read(iprot);
-          struct.setReplicationMetricListIsSet(true);
+          struct.rqst = new GetReplicationMetricsRequest();
+          struct.rqst.read(iprot);
+          struct.setRqstIsSet(true);
         }
       }
     }
 
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_replication_metrics_result implements org.apache.thrift.TBase<add_replication_metrics_result, add_replication_metrics_result._Fields>, java.io.Serializable, Cloneable, Comparable<add_replication_metrics_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("add_replication_metrics_result");
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_replication_metrics_result implements org.apache.thrift.TBase<get_replication_metrics_result, get_replication_metrics_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_replication_metrics_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_replication_metrics_result");
 
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
     private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new add_replication_metrics_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new add_replication_metrics_resultTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new get_replication_metrics_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_replication_metrics_resultTupleSchemeFactory());
     }
 
+    private ReplicationMetricList success; // required
     private MetaException o1; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success"),
       O1((short)1, "o1");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -275422,6 +276280,8 @@ import org.slf4j.LoggerFactory;
        */
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
           case 1: // O1
             return O1;
           default:
@@ -275467,40 +276327,71 @@ import org.slf4j.LoggerFactory;
     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.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ReplicationMetricList.class)));
       tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(add_replication_metrics_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_replication_metrics_result.class, metaDataMap);
     }
 
-    public add_replication_metrics_result() {
+    public get_replication_metrics_result() {
     }
 
-    public add_replication_metrics_result(
+    public get_replication_metrics_result(
+      ReplicationMetricList success,
       MetaException o1)
     {
       this();
+      this.success = success;
       this.o1 = o1;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public add_replication_metrics_result(add_replication_metrics_result other) {
+    public get_replication_metrics_result(get_replication_metrics_result other) {
+      if (other.isSetSuccess()) {
+        this.success = new ReplicationMetricList(other.success);
+      }
       if (other.isSetO1()) {
         this.o1 = new MetaException(other.o1);
       }
     }
 
-    public add_replication_metrics_result deepCopy() {
-      return new add_replication_metrics_result(this);
+    public get_replication_metrics_result deepCopy() {
+      return new get_replication_metrics_result(this);
     }
 
     @Override
     public void clear() {
+      this.success = null;
       this.o1 = null;
     }
 
+    public ReplicationMetricList getSuccess() {
+      return this.success;
+    }
+
+    public void setSuccess(ReplicationMetricList success) {
+      this.success = success;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
     public MetaException getO1() {
       return this.o1;
     }
@@ -275526,6 +276417,14 @@ import org.slf4j.LoggerFactory;
 
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((ReplicationMetricList)value);
+        }
+        break;
+
       case O1:
         if (value == null) {
           unsetO1();
@@ -275539,6 +276438,9 @@ import org.slf4j.LoggerFactory;
 
     public Object getFieldValue(_Fields field) {
       switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
       case O1:
         return getO1();
 
@@ -275553,6 +276455,8 @@ import org.slf4j.LoggerFactory;
       }
 
       switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
       case O1:
         return isSetO1();
       }
@@ -275563,15 +276467,24 @@ import org.slf4j.LoggerFactory;
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof add_replication_metrics_result)
-        return this.equals((add_replication_metrics_result)that);
+      if (that instanceof get_replication_metrics_result)
+        return this.equals((get_replication_metrics_result)that);
       return false;
     }
 
-    public boolean equals(add_replication_metrics_result that) {
+    public boolean equals(get_replication_metrics_result that) {
       if (that == null)
         return false;
 
+      boolean this_present_success = true && this.isSetSuccess();
+      boolean that_present_success = true && that.isSetSuccess();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
       boolean this_present_o1 = true && this.isSetO1();
       boolean that_present_o1 = true && that.isSetO1();
       if (this_present_o1 || that_present_o1) {
@@ -275588,6 +276501,11 @@ import org.slf4j.LoggerFactory;
     public int hashCode() {
       List<Object> list = new ArrayList<Object>();
 
+      boolean present_success = true && (isSetSuccess());
+      list.add(present_success);
+      if (present_success)
+        list.add(success);
+
       boolean present_o1 = true && (isSetO1());
       list.add(present_o1);
       if (present_o1)
@@ -275597,13 +276515,23 @@ import org.slf4j.LoggerFactory;
     }
 
     @Override
-    public int compareTo(add_replication_metrics_result other) {
+    public int compareTo(get_replication_metrics_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
 
+      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       lastComparison = Boolean.valueOf(isSetO1()).compareTo(other.isSetO1());
       if (lastComparison != 0) {
         return lastComparison;
@@ -275631,9 +276559,17 @@ import org.slf4j.LoggerFactory;
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("add_replication_metrics_result(");
+      StringBuilder sb = new StringBuilder("get_replication_metrics_result(");
       boolean first = true;
 
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      if (!first) sb.append(", ");
       sb.append("o1:");
       if (this.o1 == null) {
         sb.append("null");
@@ -275648,6 +276584,9 @@ import org.slf4j.LoggerFactory;
     public void validate() throws org.apache.thrift.TException {
       // check for required fields
       // check for sub-struct validity
+      if (success != null) {
+        success.validate();
+      }
     }
 
     private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -275666,15 +276605,15 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    private static class add_replication_metrics_resultStandardSchemeFactory implements SchemeFactory {
-      public add_replication_metrics_resultStandardScheme getScheme() {
-        return new add_replication_metrics_resultStandardScheme();
+    private static class get_replication_metrics_resultStandardSchemeFactory implements SchemeFactory {
+      public get_replication_metrics_resultStandardScheme getScheme() {
+        return new get_replication_metrics_resultStandardScheme();
       }
     }
 
-    private static class add_replication_metrics_resultStandardScheme extends StandardScheme<add_replication_metrics_result> {
+    private static class get_replication_metrics_resultStandardScheme extends StandardScheme<get_replication_metrics_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, add_replication_metrics_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_replication_metrics_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -275684,6 +276623,15 @@ import org.slf4j.LoggerFactory;
             break;
           }
           switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.success = new ReplicationMetricList();
+                struct.success.read(iprot);
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
             case 1: // O1
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
                 struct.o1 = new MetaException();
@@ -275702,10 +276650,15 @@ import org.slf4j.LoggerFactory;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, add_replication_metrics_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, get_replication_metrics_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          struct.success.write(oprot);
+          oprot.writeFieldEnd();
+        }
         if (struct.o1 != null) {
           oprot.writeFieldBegin(O1_FIELD_DESC);
           struct.o1.write(oprot);
@@ -275717,32 +276670,43 @@ import org.slf4j.LoggerFactory;
 
     }
 
-    private static class add_replication_metrics_resultTupleSchemeFactory implements SchemeFactory {
-      public add_replication_metrics_resultTupleScheme getScheme() {
-        return new add_replication_metrics_resultTupleScheme();
+    private static class get_replication_metrics_resultTupleSchemeFactory implements SchemeFactory {
+      public get_replication_metrics_resultTupleScheme getScheme() {
+        return new get_replication_metrics_resultTupleScheme();
       }
     }
 
-    private static class add_replication_metrics_resultTupleScheme extends TupleScheme<add_replication_metrics_result> {
+    private static class get_replication_metrics_resultTupleScheme extends TupleScheme<get_replication_metrics_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, add_replication_metrics_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_replication_metrics_result struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
-        if (struct.isSetO1()) {
+        if (struct.isSetSuccess()) {
           optionals.set(0);
         }
-        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetO1()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetSuccess()) {
+          struct.success.write(oprot);
+        }
         if (struct.isSetO1()) {
           struct.o1.write(oprot);
         }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, add_replication_metrics_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_replication_metrics_result struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
-        BitSet incoming = iprot.readBitSet(1);
+        BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
+          struct.success = new ReplicationMetricList();
+          struct.success.read(iprot);
+          struct.setSuccessIsSet(true);
+        }
+        if (incoming.get(1)) {
           struct.o1 = new MetaException();
           struct.o1.read(iprot);
           struct.setO1IsSet(true);
@@ -275752,22 +276716,22 @@ import org.slf4j.LoggerFactory;
 
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_replication_metrics_args implements org.apache.thrift.TBase<get_replication_metrics_args, get_replication_metrics_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_replication_metrics_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_replication_metrics_args");
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_open_txns_req_args implements org.apache.thrift.TBase<get_open_txns_req_args, get_open_txns_req_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_open_txns_req_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_open_txns_req_args");
 
-    private static final org.apache.thrift.protocol.TField RQST_FIELD_DESC = new org.apache.thrift.protocol.TField("rqst", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField GET_OPEN_TXNS_REQUEST_FIELD_DESC = new org.apache.thrift.protocol.TField("getOpenTxnsRequest", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new get_replication_metrics_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new get_replication_metrics_argsTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new get_open_txns_req_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_open_txns_req_argsTupleSchemeFactory());
     }
 
-    private GetReplicationMetricsRequest rqst; // required
+    private GetOpenTxnsRequest getOpenTxnsRequest; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      RQST((short)1, "rqst");
+      GET_OPEN_TXNS_REQUEST((short)1, "getOpenTxnsRequest");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -275782,8 +276746,8 @@ import org.slf4j.LoggerFactory;
        */
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
-          case 1: // RQST
-            return RQST;
+          case 1: // GET_OPEN_TXNS_REQUEST
+            return GET_OPEN_TXNS_REQUEST;
           default:
             return null;
         }
@@ -275827,70 +276791,70 @@ import org.slf4j.LoggerFactory;
     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.RQST, new org.apache.thrift.meta_data.FieldMetaData("rqst", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetReplicationMetricsRequest.class)));
+      tmpMap.put(_Fields.GET_OPEN_TXNS_REQUEST, new org.apache.thrift.meta_data.FieldMetaData("getOpenTxnsRequest", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetOpenTxnsRequest.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_replication_metrics_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_open_txns_req_args.class, metaDataMap);
     }
 
-    public get_replication_metrics_args() {
+    public get_open_txns_req_args() {
     }
 
-    public get_replication_metrics_args(
-      GetReplicationMetricsRequest rqst)
+    public get_open_txns_req_args(
+      GetOpenTxnsRequest getOpenTxnsRequest)
     {
       this();
-      this.rqst = rqst;
+      this.getOpenTxnsRequest = getOpenTxnsRequest;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public get_replication_metrics_args(get_replication_metrics_args other) {
-      if (other.isSetRqst()) {
-        this.rqst = new GetReplicationMetricsRequest(other.rqst);
+    public get_open_txns_req_args(get_open_txns_req_args other) {
+      if (other.isSetGetOpenTxnsRequest()) {
+        this.getOpenTxnsRequest = new GetOpenTxnsRequest(other.getOpenTxnsRequest);
       }
     }
 
-    public get_replication_metrics_args deepCopy() {
-      return new get_replication_metrics_args(this);
+    public get_open_txns_req_args deepCopy() {
+      return new get_open_txns_req_args(this);
     }
 
     @Override
     public void clear() {
-      this.rqst = null;
+      this.getOpenTxnsRequest = null;
     }
 
-    public GetReplicationMetricsRequest getRqst() {
-      return this.rqst;
+    public GetOpenTxnsRequest getGetOpenTxnsRequest() {
+      return this.getOpenTxnsRequest;
     }
 
-    public void setRqst(GetReplicationMetricsRequest rqst) {
-      this.rqst = rqst;
+    public void setGetOpenTxnsRequest(GetOpenTxnsRequest getOpenTxnsRequest) {
+      this.getOpenTxnsRequest = getOpenTxnsRequest;
     }
 
-    public void unsetRqst() {
-      this.rqst = null;
+    public void unsetGetOpenTxnsRequest() {
+      this.getOpenTxnsRequest = null;
     }
 
-    /** Returns true if field rqst is set (has been assigned a value) and false otherwise */
-    public boolean isSetRqst() {
-      return this.rqst != null;
+    /** Returns true if field getOpenTxnsRequest is set (has been assigned a value) and false otherwise */
+    public boolean isSetGetOpenTxnsRequest() {
+      return this.getOpenTxnsRequest != null;
     }
 
-    public void setRqstIsSet(boolean value) {
+    public void setGetOpenTxnsRequestIsSet(boolean value) {
       if (!value) {
-        this.rqst = null;
+        this.getOpenTxnsRequest = null;
       }
     }
 
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
-      case RQST:
+      case GET_OPEN_TXNS_REQUEST:
         if (value == null) {
-          unsetRqst();
+          unsetGetOpenTxnsRequest();
         } else {
-          setRqst((GetReplicationMetricsRequest)value);
+          setGetOpenTxnsRequest((GetOpenTxnsRequest)value);
         }
         break;
 
@@ -275899,8 +276863,8 @@ import org.slf4j.LoggerFactory;
 
     public Object getFieldValue(_Fields field) {
       switch (field) {
-      case RQST:
-        return getRqst();
+      case GET_OPEN_TXNS_REQUEST:
+        return getGetOpenTxnsRequest();
 
       }
       throw new IllegalStateException();
@@ -275913,8 +276877,8 @@ import org.slf4j.LoggerFactory;
       }
 
       switch (field) {
-      case RQST:
-        return isSetRqst();
+      case GET_OPEN_TXNS_REQUEST:
+        return isSetGetOpenTxnsRequest();
       }
       throw new IllegalStateException();
     }
@@ -275923,21 +276887,21 @@ import org.slf4j.LoggerFactory;
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof get_replication_metrics_args)
-        return this.equals((get_replication_metrics_args)that);
+      if (that instanceof get_open_txns_req_args)
+        return this.equals((get_open_txns_req_args)that);
       return false;
     }
 
-    public boolean equals(get_replication_metrics_args that) {
+    public boolean equals(get_open_txns_req_args that) {
       if (that == null)
         return false;
 
-      boolean this_present_rqst = true && this.isSetRqst();
-      boolean that_present_rqst = true && that.isSetRqst();
-      if (this_present_rqst || that_present_rqst) {
-        if (!(this_present_rqst && that_present_rqst))
+      boolean this_present_getOpenTxnsRequest = true && this.isSetGetOpenTxnsRequest();
+      boolean that_present_getOpenTxnsRequest = true && that.isSetGetOpenTxnsRequest();
+      if (this_present_getOpenTxnsRequest || that_present_getOpenTxnsRequest) {
+        if (!(this_present_getOpenTxnsRequest && that_present_getOpenTxnsRequest))
           return false;
-        if (!this.rqst.equals(that.rqst))
+        if (!this.getOpenTxnsRequest.equals(that.getOpenTxnsRequest))
           return false;
       }
 
@@ -275948,28 +276912,28 @@ import org.slf4j.LoggerFactory;
     public int hashCode() {
       List<Object> list = new ArrayList<Object>();
 
-      boolean present_rqst = true && (isSetRqst());
-      list.add(present_rqst);
-      if (present_rqst)
-        list.add(rqst);
+      boolean present_getOpenTxnsRequest = true && (isSetGetOpenTxnsRequest());
+      list.add(present_getOpenTxnsRequest);
+      if (present_getOpenTxnsRequest)
+        list.add(getOpenTxnsRequest);
 
       return list.hashCode();
     }
 
     @Override
-    public int compareTo(get_replication_metrics_args other) {
+    public int compareTo(get_open_txns_req_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
 
-      lastComparison = Boolean.valueOf(isSetRqst()).compareTo(other.isSetRqst());
+      lastComparison = Boolean.valueOf(isSetGetOpenTxnsRequest()).compareTo(other.isSetGetOpenTxnsRequest());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetRqst()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rqst, other.rqst);
+      if (isSetGetOpenTxnsRequest()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.getOpenTxnsRequest, other.getOpenTxnsRequest);
         if (lastComparison != 0) {
           return lastComparison;
         }
@@ -275991,14 +276955,14 @@ import org.slf4j.LoggerFactory;
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("get_replication_metrics_args(");
+      StringBuilder sb = new StringBuilder("get_open_txns_req_args(");
       boolean first = true;
 
-      sb.append("rqst:");
-      if (this.rqst == null) {
+      sb.append("getOpenTxnsRequest:");
+      if (this.getOpenTxnsRequest == null) {
         sb.append("null");
       } else {
-        sb.append(this.rqst);
+        sb.append(this.getOpenTxnsRequest);
       }
       first = false;
       sb.append(")");
@@ -276008,8 +276972,8 @@ import org.slf4j.LoggerFactory;
     public void validate() throws org.apache.thrift.TException {
       // check for required fields
       // check for sub-struct validity
-      if (rqst != null) {
-        rqst.validate();
+      if (getOpenTxnsRequest != null) {
+        getOpenTxnsRequest.validate();
       }
     }
 
@@ -276029,15 +276993,15 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    private static class get_replication_metrics_argsStandardSchemeFactory implements SchemeFactory {
-      public get_replication_metrics_argsStandardScheme getScheme() {
-        return new get_replication_metrics_argsStandardScheme();
+    private static class get_open_txns_req_argsStandardSchemeFactory implements SchemeFactory {
+      public get_open_txns_req_argsStandardScheme getScheme() {
+        return new get_open_txns_req_argsStandardScheme();
       }
     }
 
-    private static class get_replication_metrics_argsStandardScheme extends StandardScheme<get_replication_metrics_args> {
+    private static class get_open_txns_req_argsStandardScheme extends StandardScheme<get_open_txns_req_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, get_replication_metrics_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_open_txns_req_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -276047,11 +277011,11 @@ import org.slf4j.LoggerFactory;
             break;
           }
           switch (schemeField.id) {
-            case 1: // RQST
+            case 1: // GET_OPEN_TXNS_REQUEST
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.rqst = new GetReplicationMetricsRequest();
-                struct.rqst.read(iprot);
-                struct.setRqstIsSet(true);
+                struct.getOpenTxnsRequest = new GetOpenTxnsRequest();
+                struct.getOpenTxnsRequest.read(iprot);
+                struct.setGetOpenTxnsRequestIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
@@ -276065,13 +277029,13 @@ import org.slf4j.LoggerFactory;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, get_replication_metrics_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, get_open_txns_req_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.rqst != null) {
-          oprot.writeFieldBegin(RQST_FIELD_DESC);
-          struct.rqst.write(oprot);
+        if (struct.getOpenTxnsRequest != null) {
+          oprot.writeFieldBegin(GET_OPEN_TXNS_REQUEST_FIELD_DESC);
+          struct.getOpenTxnsRequest.write(oprot);
           oprot.writeFieldEnd();
         }
         oprot.writeFieldStop();
@@ -276080,60 +277044,57 @@ import org.slf4j.LoggerFactory;
 
     }
 
-    private static class get_replication_metrics_argsTupleSchemeFactory implements SchemeFactory {
-      public get_replication_metrics_argsTupleScheme getScheme() {
-        return new get_replication_metrics_argsTupleScheme();
+    private static class get_open_txns_req_argsTupleSchemeFactory implements SchemeFactory {
+      public get_open_txns_req_argsTupleScheme getScheme() {
+        return new get_open_txns_req_argsTupleScheme();
       }
     }
 
-    private static class get_replication_metrics_argsTupleScheme extends TupleScheme<get_replication_metrics_args> {
+    private static class get_open_txns_req_argsTupleScheme extends TupleScheme<get_open_txns_req_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, get_replication_metrics_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_open_txns_req_args struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
-        if (struct.isSetRqst()) {
+        if (struct.isSetGetOpenTxnsRequest()) {
           optionals.set(0);
         }
         oprot.writeBitSet(optionals, 1);
-        if (struct.isSetRqst()) {
-          struct.rqst.write(oprot);
+        if (struct.isSetGetOpenTxnsRequest()) {
+          struct.getOpenTxnsRequest.write(oprot);
         }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, get_replication_metrics_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_open_txns_req_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
-          struct.rqst = new GetReplicationMetricsRequest();
-          struct.rqst.read(iprot);
-          struct.setRqstIsSet(true);
+          struct.getOpenTxnsRequest = new GetOpenTxnsRequest();
+          struct.getOpenTxnsRequest.read(iprot);
+          struct.setGetOpenTxnsRequestIsSet(true);
         }
       }
     }
 
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_replication_metrics_result implements org.apache.thrift.TBase<get_replication_metrics_result, get_replication_metrics_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_replication_metrics_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_replication_metrics_result");
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_open_txns_req_result implements org.apache.thrift.TBase<get_open_txns_req_result, get_open_txns_req_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_open_txns_req_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_open_txns_req_result");
 
     private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
-    private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new get_replication_metrics_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new get_replication_metrics_resultTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new get_open_txns_req_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_open_txns_req_resultTupleSchemeFactory());
     }
 
-    private ReplicationMetricList success; // required
-    private MetaException o1; // required
+    private GetOpenTxnsResponse success; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      SUCCESS((short)0, "success"),
-      O1((short)1, "o1");
+      SUCCESS((short)0, "success");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -276150,8 +277111,6 @@ import org.slf4j.LoggerFactory;
         switch(fieldId) {
           case 0: // SUCCESS
             return SUCCESS;
-          case 1: // O1
-            return O1;
           default:
             return null;
         }
@@ -276196,52 +277155,44 @@ import org.slf4j.LoggerFactory;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ReplicationMetricList.class)));
-      tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetOpenTxnsResponse.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_replication_metrics_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_open_txns_req_result.class, metaDataMap);
     }
 
-    public get_replication_metrics_result() {
+    public get_open_txns_req_result() {
     }
 
-    public get_replication_metrics_result(
-      ReplicationMetricList success,
-      MetaException o1)
+    public get_open_txns_req_result(
+      GetOpenTxnsResponse success)
     {
       this();
       this.success = success;
-      this.o1 = o1;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public get_replication_metrics_result(get_replication_metrics_result other) {
+    public get_open_txns_req_result(get_open_txns_req_result other) {
       if (other.isSetSuccess()) {
-        this.success = new ReplicationMetricList(other.success);
-      }
-      if (other.isSetO1()) {
-        this.o1 = new MetaException(other.o1);
+        this.success = new GetOpenTxnsResponse(other.success);
       }
     }
 
-    public get_replication_metrics_result deepCopy() {
-      return new get_replication_metrics_result(this);
+    public get_open_txns_req_result deepCopy() {
+      return new get_open_txns_req_result(this);
     }
 
     @Override
     public void clear() {
       this.success = null;
-      this.o1 = null;
     }
 
-    public ReplicationMetricList getSuccess() {
+    public GetOpenTxnsResponse getSuccess() {
       return this.success;
     }
 
-    public void setSuccess(ReplicationMetricList success) {
+    public void setSuccess(GetOpenTxnsResponse success) {
       this.success = success;
     }
 
@@ -276260,44 +277211,13 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public MetaException getO1() {
-      return this.o1;
-    }
-
-    public void setO1(MetaException o1) {
-      this.o1 = o1;
-    }
-
-    public void unsetO1() {
-      this.o1 = null;
-    }
-
-    /** Returns true if field o1 is set (has been assigned a value) and false otherwise */
-    public boolean isSetO1() {
-      return this.o1 != null;
-    }
-
-    public void setO1IsSet(boolean value) {
-      if (!value) {
-        this.o1 = null;
-      }
-    }
-
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
       case SUCCESS:
         if (value == null) {
           unsetSuccess();
         } else {
-          setSuccess((ReplicationMetricList)value);
-        }
-        break;
-
-      case O1:
-        if (value == null) {
-          unsetO1();
-        } else {
-          setO1((MetaException)value);
+          setSuccess((GetOpenTxnsResponse)value);
         }
         break;
 
@@ -276309,9 +277229,6 @@ import org.slf4j.LoggerFactory;
       case SUCCESS:
         return getSuccess();
 
-      case O1:
-        return getO1();
-
       }
       throw new IllegalStateException();
     }
@@ -276325,8 +277242,6 @@ import org.slf4j.LoggerFactory;
       switch (field) {
       case SUCCESS:
         return isSetSuccess();
-      case O1:
-        return isSetO1();
       }
       throw new IllegalStateException();
     }
@@ -276335,12 +277250,12 @@ import org.slf4j.LoggerFactory;
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof get_replication_metrics_result)
-        return this.equals((get_replication_metrics_result)that);
+      if (that instanceof get_open_txns_req_result)
+        return this.equals((get_open_txns_req_result)that);
       return false;
     }
 
-    public boolean equals(get_replication_metrics_result that) {
+    public boolean equals(get_open_txns_req_result that) {
       if (that == null)
         return false;
 
@@ -276353,15 +277268,6 @@ import org.slf4j.LoggerFactory;
           return false;
       }
 
-      boolean this_present_o1 = true && this.isSetO1();
-      boolean that_present_o1 = true && that.isSetO1();
-      if (this_present_o1 || that_present_o1) {
-        if (!(this_present_o1 && that_present_o1))
-          return false;
-        if (!this.o1.equals(that.o1))
-          return false;
-      }
-
       return true;
     }
 
@@ -276374,16 +277280,11 @@ import org.slf4j.LoggerFactory;
       if (present_success)
         list.add(success);
 
-      boolean present_o1 = true && (isSetO1());
-      list.add(present_o1);
-      if (present_o1)
-        list.add(o1);
-
       return list.hashCode();
     }
 
     @Override
-    public int compareTo(get_replication_metrics_result other) {
+    public int compareTo(get_open_txns_req_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -276400,16 +277301,6 @@ import org.slf4j.LoggerFactory;
           return lastComparison;
         }
       }
-      lastComparison = Boolean.valueOf(isSetO1()).compareTo(other.isSetO1());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetO1()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, other.o1);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
       return 0;
     }
 
@@ -276427,7 +277318,7 @@ import org.slf4j.LoggerFactory;
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("get_replication_metrics_result(");
+      StringBuilder sb = new StringBuilder("get_open_txns_req_result(");
       boolean first = true;
 
       sb.append("success:");
@@ -276437,14 +277328,6 @@ import org.slf4j.LoggerFactory;
         sb.append(this.success);
       }
       first = false;
-      if (!first) sb.append(", ");
-      sb.append("o1:");
-      if (this.o1 == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.o1);
-      }
-      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -276473,15 +277356,15 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    private static class get_replication_metrics_resultStandardSchemeFactory implements SchemeFactory {
-      public get_replication_metrics_resultStandardScheme getScheme() {
-        return new get_replication_metrics_resultStandardScheme();
+    private static class get_open_txns_req_resultStandardSchemeFactory implements SchemeFactory {
+      public get_open_txns_req_resultStandardScheme getScheme() {
+        return new get_open_txns_req_resultStandardScheme();
       }
     }
 
-    private static class get_replication_metrics_resultStandardScheme extends StandardScheme<get_replication_metrics_result> {
+    private static class get_open_txns_req_resultStandardScheme extends StandardScheme<get_open_txns_req_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, get_replication_metrics_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_open_txns_req_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -276493,22 +277376,13 @@ import org.slf4j.LoggerFactory;
           switch (schemeField.id) {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.success = new ReplicationMetricList();
+                struct.success = new GetOpenTxnsResponse();
                 struct.success.read(iprot);
                 struct.setSuccessIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
-            case 1: // O1
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.o1 = new MetaException();
-                struct.o1.read(iprot);
-                struct.setO1IsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
             default:
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
           }
@@ -276518,7 +277392,7 @@ import org.slf4j.LoggerFactory;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, get_replication_metrics_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, get_open_txns_req_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -276527,58 +277401,42 @@ import org.slf4j.LoggerFactory;
           struct.success.write(oprot);
           oprot.writeFieldEnd();
         }
-        if (struct.o1 != null) {
-          oprot.writeFieldBegin(O1_FIELD_DESC);
-          struct.o1.write(oprot);
-          oprot.writeFieldEnd();
-        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
 
     }
 
-    private static class get_replication_metrics_resultTupleSchemeFactory implements SchemeFactory {
-      public get_replication_metrics_resultTupleScheme getScheme() {
-        return new get_replication_metrics_resultTupleScheme();
+    private static class get_open_txns_req_resultTupleSchemeFactory implements SchemeFactory {
+      public get_open_txns_req_resultTupleScheme getScheme() {
+        return new get_open_txns_req_resultTupleScheme();
       }
     }
 
-    private static class get_replication_metrics_resultTupleScheme extends TupleScheme<get_replication_metrics_result> {
+    private static class get_open_txns_req_resultTupleScheme extends TupleScheme<get_open_txns_req_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, get_replication_metrics_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_open_txns_req_result struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetSuccess()) {
           optionals.set(0);
         }
-        if (struct.isSetO1()) {
-          optionals.set(1);
-        }
-        oprot.writeBitSet(optionals, 2);
+        oprot.writeBitSet(optionals, 1);
         if (struct.isSetSuccess()) {
           struct.success.write(oprot);
         }
-        if (struct.isSetO1()) {
-          struct.o1.write(oprot);
-        }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, get_replication_metrics_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_open_txns_req_result struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
-        BitSet incoming = iprot.readBitSet(2);
+        BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
-          struct.success = new ReplicationMetricList();
+          struct.success = new GetOpenTxnsResponse();
           struct.success.read(iprot);
           struct.setSuccessIsSet(true);
         }
-        if (incoming.get(1)) {
-          struct.o1 = new MetaException();
-          struct.o1.read(iprot);
-          struct.setO1IsSet(true);
-        }
       }
     }
 
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
index 30ce207..35716e7 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -1797,6 +1797,11 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    * @throws \metastore\MetaException
    */
   public function get_replication_metrics(\metastore\GetReplicationMetricsRequest $rqst);
+  /**
+   * @param \metastore\GetOpenTxnsRequest $getOpenTxnsRequest
+   * @return \metastore\GetOpenTxnsResponse
+   */
+  public function get_open_txns_req(\metastore\GetOpenTxnsRequest $getOpenTxnsRequest);
 }
 
 class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metastore\ThriftHiveMetastoreIf {
@@ -15585,6 +15590,57 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     throw new \Exception("get_replication_metrics failed: unknown result");
   }
 
+  public function get_open_txns_req(\metastore\GetOpenTxnsRequest $getOpenTxnsRequest)
+  {
+    $this->send_get_open_txns_req($getOpenTxnsRequest);
+    return $this->recv_get_open_txns_req();
+  }
+
+  public function send_get_open_txns_req(\metastore\GetOpenTxnsRequest $getOpenTxnsRequest)
+  {
+    $args = new \metastore\ThriftHiveMetastore_get_open_txns_req_args();
+    $args->getOpenTxnsRequest = $getOpenTxnsRequest;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'get_open_txns_req', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('get_open_txns_req', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_get_open_txns_req()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_get_open_txns_req_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
+
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
+      }
+      $result = new \metastore\ThriftHiveMetastore_get_open_txns_req_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->success !== null) {
+      return $result->success;
+    }
+    throw new \Exception("get_open_txns_req failed: unknown result");
+  }
+
 }
 
 // HELPER FUNCTIONS AND STRUCTURES
@@ -17932,14 +17988,14 @@ class ThriftHiveMetastore_get_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1134 = 0;
-            $_etype1137 = 0;
-            $xfer += $input->readListBegin($_etype1137, $_size1134);
-            for ($_i1138 = 0; $_i1138 < $_size1134; ++$_i1138)
+            $_size1141 = 0;
+            $_etype1144 = 0;
+            $xfer += $input->readListBegin($_etype1144, $_size1141);
+            for ($_i1145 = 0; $_i1145 < $_size1141; ++$_i1145)
             {
-              $elem1139 = null;
-              $xfer += $input->readString($elem1139);
-              $this->success []= $elem1139;
+              $elem1146 = null;
+              $xfer += $input->readString($elem1146);
+              $this->success []= $elem1146;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17975,9 +18031,9 @@ class ThriftHiveMetastore_get_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1140)
+          foreach ($this->success as $iter1147)
           {
-            $xfer += $output->writeString($iter1140);
+            $xfer += $output->writeString($iter1147);
           }
         }
         $output->writeListEnd();
@@ -18108,14 +18164,14 @@ class ThriftHiveMetastore_get_all_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1141 = 0;
-            $_etype1144 = 0;
-            $xfer += $input->readListBegin($_etype1144, $_size1141);
-            for ($_i1145 = 0; $_i1145 < $_size1141; ++$_i1145)
+            $_size1148 = 0;
+            $_etype1151 = 0;
+            $xfer += $input->readListBegin($_etype1151, $_size1148);
+            for ($_i1152 = 0; $_i1152 < $_size1148; ++$_i1152)
             {
-              $elem1146 = null;
-              $xfer += $input->readString($elem1146);
-              $this->success []= $elem1146;
+              $elem1153 = null;
+              $xfer += $input->readString($elem1153);
+              $this->success []= $elem1153;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18151,9 +18207,9 @@ class ThriftHiveMetastore_get_all_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1147)
+          foreach ($this->success as $iter1154)
           {
-            $xfer += $output->writeString($iter1147);
+            $xfer += $output->writeString($iter1154);
           }
         }
         $output->writeListEnd();
@@ -19154,18 +19210,18 @@ class ThriftHiveMetastore_get_type_all_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size1148 = 0;
-            $_ktype1149 = 0;
-            $_vtype1150 = 0;
-            $xfer += $input->readMapBegin($_ktype1149, $_vtype1150, $_size1148);
-            for ($_i1152 = 0; $_i1152 < $_size1148; ++$_i1152)
+            $_size1155 = 0;
+            $_ktype1156 = 0;
+            $_vtype1157 = 0;
+            $xfer += $input->readMapBegin($_ktype1156, $_vtype1157, $_size1155);
+            for ($_i1159 = 0; $_i1159 < $_size1155; ++$_i1159)
             {
-              $key1153 = '';
-              $val1154 = new \metastore\Type();
-              $xfer += $input->readString($key1153);
-              $val1154 = new \metastore\Type();
-              $xfer += $val1154->read($input);
-              $this->success[$key1153] = $val1154;
+              $key1160 = '';
+              $val1161 = new \metastore\Type();
+              $xfer += $input->readString($key1160);
+              $val1161 = new \metastore\Type();
+              $xfer += $val1161->read($input);
+              $this->success[$key1160] = $val1161;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -19201,10 +19257,10 @@ class ThriftHiveMetastore_get_type_all_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $kiter1155 => $viter1156)
+          foreach ($this->success as $kiter1162 => $viter1163)
           {
-            $xfer += $output->writeString($kiter1155);
-            $xfer += $viter1156->write($output);
+            $xfer += $output->writeString($kiter1162);
+            $xfer += $viter1163->write($output);
           }
         }
         $output->writeMapEnd();
@@ -19408,15 +19464,15 @@ class ThriftHiveMetastore_get_fields_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\FieldSchema();
-              $xfer += $elem1162->read($input);
-              $this->success []= $elem1162;
+              $elem1169 = null;
+              $elem1169 = new \metastore\FieldSchema();
+              $xfer += $elem1169->read($input);
+              $this->success []= $elem1169;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19468,9 +19524,9 @@ class ThriftHiveMetastore_get_fields_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();
@@ -19712,15 +19768,15 @@ class ThriftHiveMetastore_get_fields_with_environment_context_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\FieldSchema();
-              $xfer += $elem1169->read($input);
-              $this->success []= $elem1169;
+              $elem1176 = null;
+              $elem1176 = new \metastore\FieldSchema();
+              $xfer += $elem1176->read($input);
+              $this->success []= $elem1176;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19772,9 +19828,9 @@ class ThriftHiveMetastore_get_fields_with_environment_context_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();
@@ -20223,15 +20279,15 @@ class ThriftHiveMetastore_get_schema_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = 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;
-              $elem1176 = new \metastore\FieldSchema();
-              $xfer += $elem1176->read($input);
-              $this->success []= $elem1176;
+              $elem1183 = null;
+              $elem1183 = new \metastore\FieldSchema();
+              $xfer += $elem1183->read($input);
+              $this->success []= $elem1183;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20283,9 +20339,9 @@ class ThriftHiveMetastore_get_schema_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1177)
+          foreach ($this->success as $iter1184)
           {
-            $xfer += $iter1177->write($output);
+            $xfer += $iter1184->write($output);
           }
         }
         $output->writeListEnd();
@@ -20527,15 +20583,15 @@ class ThriftHiveMetastore_get_schema_with_environment_context_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\FieldSchema();
-              $xfer += $elem1183->read($input);
-              $this->success []= $elem1183;
+              $elem1190 = null;
+              $elem1190 = new \metastore\FieldSchema();
+              $xfer += $elem1190->read($input);
+              $this->success []= $elem1190;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20587,9 +20643,9 @@ class ThriftHiveMetastore_get_schema_with_environment_context_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();
@@ -21496,15 +21552,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->primaryKeys = 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\SQLPrimaryKey();
-              $xfer += $elem1190->read($input);
-              $this->primaryKeys []= $elem1190;
+              $elem1197 = null;
+              $elem1197 = new \metastore\SQLPrimaryKey();
+              $xfer += $elem1197->read($input);
+              $this->primaryKeys []= $elem1197;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21514,15 +21570,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->foreignKeys = array();
-            $_size1191 = 0;
-            $_etype1194 = 0;
-            $xfer += $input->readListBegin($_etype1194, $_size1191);
-            for ($_i1195 = 0; $_i1195 < $_size1191; ++$_i1195)
+            $_size1198 = 0;
+            $_etype1201 = 0;
+            $xfer += $input->readListBegin($_etype1201, $_size1198);
+            for ($_i1202 = 0; $_i1202 < $_size1198; ++$_i1202)
             {
-              $elem1196 = null;
-              $elem1196 = new \metastore\SQLForeignKey();
-              $xfer += $elem1196->read($input);
-              $this->foreignKeys []= $elem1196;
+              $elem1203 = null;
+              $elem1203 = new \metastore\SQLForeignKey();
+              $xfer += $elem1203->read($input);
+              $this->foreignKeys []= $elem1203;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21532,15 +21588,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 4:
           if ($ftype == TType::LST) {
             $this->uniqueConstraints = array();
-            $_size1197 = 0;
-            $_etype1200 = 0;
-            $xfer += $input->readListBegin($_etype1200, $_size1197);
-            for ($_i1201 = 0; $_i1201 < $_size1197; ++$_i1201)
+            $_size1204 = 0;
+            $_etype1207 = 0;
+            $xfer += $input->readListBegin($_etype1207, $_size1204);
+            for ($_i1208 = 0; $_i1208 < $_size1204; ++$_i1208)
             {
-              $elem1202 = null;
-              $elem1202 = new \metastore\SQLUniqueConstraint();
-              $xfer += $elem1202->read($input);
-              $this->uniqueConstraints []= $elem1202;
+              $elem1209 = null;
+              $elem1209 = new \metastore\SQLUniqueConstraint();
+              $xfer += $elem1209->read($input);
+              $this->uniqueConstraints []= $elem1209;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21550,15 +21606,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->notNullConstraints = array();
-            $_size1203 = 0;
-            $_etype1206 = 0;
-            $xfer += $input->readListBegin($_etype1206, $_size1203);
-            for ($_i1207 = 0; $_i1207 < $_size1203; ++$_i1207)
+            $_size1210 = 0;
+            $_etype1213 = 0;
+            $xfer += $input->readListBegin($_etype1213, $_size1210);
+            for ($_i1214 = 0; $_i1214 < $_size1210; ++$_i1214)
             {
-              $elem1208 = null;
-              $elem1208 = new \metastore\SQLNotNullConstraint();
-              $xfer += $elem1208->read($input);
-              $this->notNullConstraints []= $elem1208;
+              $elem1215 = null;
+              $elem1215 = new \metastore\SQLNotNullConstraint();
+              $xfer += $elem1215->read($input);
+              $this->notNullConstraints []= $elem1215;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21568,15 +21624,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->defaultConstraints = array();
-            $_size1209 = 0;
-            $_etype1212 = 0;
-            $xfer += $input->readListBegin($_etype1212, $_size1209);
-            for ($_i1213 = 0; $_i1213 < $_size1209; ++$_i1213)
+            $_size1216 = 0;
+            $_etype1219 = 0;
+            $xfer += $input->readListBegin($_etype1219, $_size1216);
+            for ($_i1220 = 0; $_i1220 < $_size1216; ++$_i1220)
             {
-              $elem1214 = null;
-              $elem1214 = new \metastore\SQLDefaultConstraint();
-              $xfer += $elem1214->read($input);
-              $this->defaultConstraints []= $elem1214;
+              $elem1221 = null;
+              $elem1221 = new \metastore\SQLDefaultConstraint();
+              $xfer += $elem1221->read($input);
+              $this->defaultConstraints []= $elem1221;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21586,15 +21642,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 7:
           if ($ftype == TType::LST) {
             $this->checkConstraints = array();
-            $_size1215 = 0;
-            $_etype1218 = 0;
-            $xfer += $input->readListBegin($_etype1218, $_size1215);
-            for ($_i1219 = 0; $_i1219 < $_size1215; ++$_i1219)
+            $_size1222 = 0;
+            $_etype1225 = 0;
+            $xfer += $input->readListBegin($_etype1225, $_size1222);
+            for ($_i1226 = 0; $_i1226 < $_size1222; ++$_i1226)
             {
-              $elem1220 = null;
-              $elem1220 = new \metastore\SQLCheckConstraint();
-              $xfer += $elem1220->read($input);
-              $this->checkConstraints []= $elem1220;
+              $elem1227 = null;
+              $elem1227 = new \metastore\SQLCheckConstraint();
+              $xfer += $elem1227->read($input);
+              $this->checkConstraints []= $elem1227;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21630,9 +21686,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->primaryKeys));
         {
-          foreach ($this->primaryKeys as $iter1221)
+          foreach ($this->primaryKeys as $iter1228)
           {
-            $xfer += $iter1221->write($output);
+            $xfer += $iter1228->write($output);
           }
         }
         $output->writeListEnd();
@@ -21647,9 +21703,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->foreignKeys));
         {
-          foreach ($this->foreignKeys as $iter1222)
+          foreach ($this->foreignKeys as $iter1229)
           {
-            $xfer += $iter1222->write($output);
+            $xfer += $iter1229->write($output);
           }
         }
         $output->writeListEnd();
@@ -21664,9 +21720,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->uniqueConstraints));
         {
-          foreach ($this->uniqueConstraints as $iter1223)
+          foreach ($this->uniqueConstraints as $iter1230)
           {
-            $xfer += $iter1223->write($output);
+            $xfer += $iter1230->write($output);
           }
         }
         $output->writeListEnd();
@@ -21681,9 +21737,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->notNullConstraints));
         {
-          foreach ($this->notNullConstraints as $iter1224)
+          foreach ($this->notNullConstraints as $iter1231)
           {
-            $xfer += $iter1224->write($output);
+            $xfer += $iter1231->write($output);
           }
         }
         $output->writeListEnd();
@@ -21698,9 +21754,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->defaultConstraints));
         {
-          foreach ($this->defaultConstraints as $iter1225)
+          foreach ($this->defaultConstraints as $iter1232)
           {
-            $xfer += $iter1225->write($output);
+            $xfer += $iter1232->write($output);
           }
         }
         $output->writeListEnd();
@@ -21715,9 +21771,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->checkConstraints));
         {
-          foreach ($this->checkConstraints as $iter1226)
+          foreach ($this->checkConstraints as $iter1233)
           {
-            $xfer += $iter1226->write($output);
+            $xfer += $iter1233->write($output);
           }
         }
         $output->writeListEnd();
@@ -23949,14 +24005,14 @@ class ThriftHiveMetastore_truncate_table_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->partNames = array();
-            $_size1227 = 0;
-            $_etype1230 = 0;
-            $xfer += $input->readListBegin($_etype1230, $_size1227);
-            for ($_i1231 = 0; $_i1231 < $_size1227; ++$_i1231)
+            $_size1234 = 0;
+            $_etype1237 = 0;
+            $xfer += $input->readListBegin($_etype1237, $_size1234);
+            for ($_i1238 = 0; $_i1238 < $_size1234; ++$_i1238)
             {
-              $elem1232 = null;
-              $xfer += $input->readString($elem1232);
-              $this->partNames []= $elem1232;
+              $elem1239 = null;
+              $xfer += $input->readString($elem1239);
+              $this->partNames []= $elem1239;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23994,9 +24050,9 @@ class ThriftHiveMetastore_truncate_table_args {
       {
         $output->writeListBegin(TType::STRING, count($this->partNames));
         {
-          foreach ($this->partNames as $iter1233)
+          foreach ($this->partNames as $iter1240)
           {
-            $xfer += $output->writeString($iter1233);
+            $xfer += $output->writeString($iter1240);
           }
         }
         $output->writeListEnd();
@@ -24432,14 +24488,14 @@ class ThriftHiveMetastore_get_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1234 = 0;
-            $_etype1237 = 0;
-            $xfer += $input->readListBegin($_etype1237, $_size1234);
-            for ($_i1238 = 0; $_i1238 < $_size1234; ++$_i1238)
+            $_size1241 = 0;
+            $_etype1244 = 0;
+            $xfer += $input->readListBegin($_etype1244, $_size1241);
+            for ($_i1245 = 0; $_i1245 < $_size1241; ++$_i1245)
             {
-              $elem1239 = null;
-              $xfer += $input->readString($elem1239);
-              $this->success []= $elem1239;
+              $elem1246 = null;
+              $xfer += $input->readString($elem1246);
+              $this->success []= $elem1246;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24475,9 +24531,9 @@ class ThriftHiveMetastore_get_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1240)
+          foreach ($this->success as $iter1247)
           {
-            $xfer += $output->writeString($iter1240);
+            $xfer += $output->writeString($iter1247);
           }
         }
         $output->writeListEnd();
@@ -24679,14 +24735,14 @@ class ThriftHiveMetastore_get_tables_by_type_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1241 = 0;
-            $_etype1244 = 0;
-            $xfer += $input->readListBegin($_etype1244, $_size1241);
-            for ($_i1245 = 0; $_i1245 < $_size1241; ++$_i1245)
+            $_size1248 = 0;
+            $_etype1251 = 0;
+            $xfer += $input->readListBegin($_etype1251, $_size1248);
+            for ($_i1252 = 0; $_i1252 < $_size1248; ++$_i1252)
             {
-              $elem1246 = null;
-              $xfer += $input->readString($elem1246);
-              $this->success []= $elem1246;
+              $elem1253 = null;
+              $xfer += $input->readString($elem1253);
+              $this->success []= $elem1253;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24722,9 +24778,9 @@ class ThriftHiveMetastore_get_tables_by_type_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1247)
+          foreach ($this->success as $iter1254)
           {
-            $xfer += $output->writeString($iter1247);
+            $xfer += $output->writeString($iter1254);
           }
         }
         $output->writeListEnd();
@@ -24856,15 +24912,15 @@ class ThriftHiveMetastore_get_all_materialized_view_objects_for_rewriting_result
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1248 = 0;
-            $_etype1251 = 0;
-            $xfer += $input->readListBegin($_etype1251, $_size1248);
-            for ($_i1252 = 0; $_i1252 < $_size1248; ++$_i1252)
+            $_size1255 = 0;
+            $_etype1258 = 0;
+            $xfer += $input->readListBegin($_etype1258, $_size1255);
+            for ($_i1259 = 0; $_i1259 < $_size1255; ++$_i1259)
             {
-              $elem1253 = null;
-              $elem1253 = new \metastore\Table();
-              $xfer += $elem1253->read($input);
-              $this->success []= $elem1253;
+              $elem1260 = null;
+              $elem1260 = new \metastore\Table();
+              $xfer += $elem1260->read($input);
+              $this->success []= $elem1260;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24900,9 +24956,9 @@ class ThriftHiveMetastore_get_all_materialized_view_objects_for_rewriting_result
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1254)
+          foreach ($this->success as $iter1261)
           {
-            $xfer += $iter1254->write($output);
+            $xfer += $iter1261->write($output);
           }
         }
         $output->writeListEnd();
@@ -25058,14 +25114,14 @@ class ThriftHiveMetastore_get_materialized_views_for_rewriting_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1255 = 0;
-            $_etype1258 = 0;
-            $xfer += $input->readListBegin($_etype1258, $_size1255);
-            for ($_i1259 = 0; $_i1259 < $_size1255; ++$_i1259)
+            $_size1262 = 0;
+            $_etype1265 = 0;
+            $xfer += $input->readListBegin($_etype1265, $_size1262);
+            for ($_i1266 = 0; $_i1266 < $_size1262; ++$_i1266)
             {
-              $elem1260 = null;
-              $xfer += $input->readString($elem1260);
-              $this->success []= $elem1260;
+              $elem1267 = null;
+              $xfer += $input->readString($elem1267);
+              $this->success []= $elem1267;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25101,9 +25157,9 @@ class ThriftHiveMetastore_get_materialized_views_for_rewriting_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1261)
+          foreach ($this->success as $iter1268)
           {
-            $xfer += $output->writeString($iter1261);
+            $xfer += $output->writeString($iter1268);
           }
         }
         $output->writeListEnd();
@@ -25208,14 +25264,14 @@ class ThriftHiveMetastore_get_table_meta_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->tbl_types = array();
-            $_size1262 = 0;
-            $_etype1265 = 0;
-            $xfer += $input->readListBegin($_etype1265, $_size1262);
-            for ($_i1266 = 0; $_i1266 < $_size1262; ++$_i1266)
+            $_size1269 = 0;
+            $_etype1272 = 0;
+            $xfer += $input->readListBegin($_etype1272, $_size1269);
+            for ($_i1273 = 0; $_i1273 < $_size1269; ++$_i1273)
             {
-              $elem1267 = null;
-              $xfer += $input->readString($elem1267);
-              $this->tbl_types []= $elem1267;
+              $elem1274 = null;
+              $xfer += $input->readString($elem1274);
+              $this->tbl_types []= $elem1274;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25253,9 +25309,9 @@ class ThriftHiveMetastore_get_table_meta_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_types));
         {
-          foreach ($this->tbl_types as $iter1268)
+          foreach ($this->tbl_types as $iter1275)
           {
-            $xfer += $output->writeString($iter1268);
+            $xfer += $output->writeString($iter1275);
           }
         }
         $output->writeListEnd();
@@ -25332,15 +25388,15 @@ class ThriftHiveMetastore_get_table_meta_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1269 = 0;
-            $_etype1272 = 0;
-            $xfer += $input->readListBegin($_etype1272, $_size1269);
-            for ($_i1273 = 0; $_i1273 < $_size1269; ++$_i1273)
+            $_size1276 = 0;
+            $_etype1279 = 0;
+            $xfer += $input->readListBegin($_etype1279, $_size1276);
+            for ($_i1280 = 0; $_i1280 < $_size1276; ++$_i1280)
             {
-              $elem1274 = null;
-              $elem1274 = new \metastore\TableMeta();
-              $xfer += $elem1274->read($input);
-              $this->success []= $elem1274;
+              $elem1281 = null;
+              $elem1281 = new \metastore\TableMeta();
+              $xfer += $elem1281->read($input);
+              $this->success []= $elem1281;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25376,9 +25432,9 @@ class ThriftHiveMetastore_get_table_meta_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1275)
+          foreach ($this->success as $iter1282)
           {
-            $xfer += $iter1275->write($output);
+            $xfer += $iter1282->write($output);
           }
         }
         $output->writeListEnd();
@@ -25534,14 +25590,14 @@ class ThriftHiveMetastore_get_all_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1276 = 0;
-            $_etype1279 = 0;
-            $xfer += $input->readListBegin($_etype1279, $_size1276);
-            for ($_i1280 = 0; $_i1280 < $_size1276; ++$_i1280)
+            $_size1283 = 0;
+            $_etype1286 = 0;
+            $xfer += $input->readListBegin($_etype1286, $_size1283);
+            for ($_i1287 = 0; $_i1287 < $_size1283; ++$_i1287)
             {
-              $elem1281 = null;
-              $xfer += $input->readString($elem1281);
-              $this->success []= $elem1281;
+              $elem1288 = null;
+              $xfer += $input->readString($elem1288);
+              $this->success []= $elem1288;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25577,9 +25633,9 @@ class ThriftHiveMetastore_get_all_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1282)
+          foreach ($this->success as $iter1289)
           {
-            $xfer += $output->writeString($iter1282);
+            $xfer += $output->writeString($iter1289);
           }
         }
         $output->writeListEnd();
@@ -25894,14 +25950,14 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size1283 = 0;
-            $_etype1286 = 0;
-            $xfer += $input->readListBegin($_etype1286, $_size1283);
-            for ($_i1287 = 0; $_i1287 < $_size1283; ++$_i1287)
+            $_size1290 = 0;
+            $_etype1293 = 0;
+            $xfer += $input->readListBegin($_etype1293, $_size1290);
+            for ($_i1294 = 0; $_i1294 < $_size1290; ++$_i1294)
             {
-              $elem1288 = null;
-              $xfer += $input->readString($elem1288);
-              $this->tbl_names []= $elem1288;
+              $elem1295 = null;
+              $xfer += $input->readString($elem1295);
+              $this->tbl_names []= $elem1295;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25934,9 +25990,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter1289)
+          foreach ($this->tbl_names as $iter1296)
           {
-            $xfer += $output->writeString($iter1289);
+            $xfer += $output->writeString($iter1296);
           }
         }
         $output->writeListEnd();
@@ -26001,15 +26057,15 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1290 = 0;
-            $_etype1293 = 0;
-            $xfer += $input->readListBegin($_etype1293, $_size1290);
-            for ($_i1294 = 0; $_i1294 < $_size1290; ++$_i1294)
+            $_size1297 = 0;
+            $_etype1300 = 0;
+            $xfer += $input->readListBegin($_etype1300, $_size1297);
+            for ($_i1301 = 0; $_i1301 < $_size1297; ++$_i1301)
             {
-              $elem1295 = null;
-              $elem1295 = new \metastore\Table();
-              $xfer += $elem1295->read($input);
-              $this->success []= $elem1295;
+              $elem1302 = null;
+              $elem1302 = new \metastore\Table();
+              $xfer += $elem1302->read($input);
+              $this->success []= $elem1302;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26037,9 +26093,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1296)
+          foreach ($this->success as $iter1303)
           {
-            $xfer += $iter1296->write($output);
+            $xfer += $iter1303->write($output);
           }
         }
         $output->writeListEnd();
@@ -26196,15 +26252,15 @@ class ThriftHiveMetastore_get_tables_ext_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1297 = 0;
-            $_etype1300 = 0;
-            $xfer += $input->readListBegin($_etype1300, $_size1297);
-            for ($_i1301 = 0; $_i1301 < $_size1297; ++$_i1301)
+            $_size1304 = 0;
+            $_etype1307 = 0;
+            $xfer += $input->readListBegin($_etype1307, $_size1304);
+            for ($_i1308 = 0; $_i1308 < $_size1304; ++$_i1308)
             {
-              $elem1302 = null;
-              $elem1302 = new \metastore\ExtendedTableInfo();
-              $xfer += $elem1302->read($input);
-              $this->success []= $elem1302;
+              $elem1309 = null;
+              $elem1309 = new \metastore\ExtendedTableInfo();
+              $xfer += $elem1309->read($input);
+              $this->success []= $elem1309;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26240,9 +26296,9 @@ class ThriftHiveMetastore_get_tables_ext_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1303)
+          foreach ($this->success as $iter1310)
           {
-            $xfer += $iter1303->write($output);
+            $xfer += $iter1310->write($output);
           }
         }
         $output->writeListEnd();
@@ -27447,14 +27503,14 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1304 = 0;
-            $_etype1307 = 0;
-            $xfer += $input->readListBegin($_etype1307, $_size1304);
-            for ($_i1308 = 0; $_i1308 < $_size1304; ++$_i1308)
+            $_size1311 = 0;
+            $_etype1314 = 0;
+            $xfer += $input->readListBegin($_etype1314, $_size1311);
+            for ($_i1315 = 0; $_i1315 < $_size1311; ++$_i1315)
             {
-              $elem1309 = null;
-              $xfer += $input->readString($elem1309);
-              $this->success []= $elem1309;
+              $elem1316 = null;
+              $xfer += $input->readString($elem1316);
+              $this->success []= $elem1316;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27506,9 +27562,9 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1310)
+          foreach ($this->success as $iter1317)
           {
-            $xfer += $output->writeString($iter1310);
+            $xfer += $output->writeString($iter1317);
           }
         }
         $output->writeListEnd();
@@ -29031,15 +29087,15 @@ class ThriftHiveMetastore_add_partitions_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size1311 = 0;
-            $_etype1314 = 0;
-            $xfer += $input->readListBegin($_etype1314, $_size1311);
-            for ($_i1315 = 0; $_i1315 < $_size1311; ++$_i1315)
+            $_size1318 = 0;
+            $_etype1321 = 0;
+            $xfer += $input->readListBegin($_etype1321, $_size1318);
+            for ($_i1322 = 0; $_i1322 < $_size1318; ++$_i1322)
             {
-              $elem1316 = null;
-              $elem1316 = new \metastore\Partition();
-              $xfer += $elem1316->read($input);
-              $this->new_parts []= $elem1316;
+              $elem1323 = null;
+              $elem1323 = new \metastore\Partition();
+              $xfer += $elem1323->read($input);
+              $this->new_parts []= $elem1323;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29067,9 +29123,9 @@ class ThriftHiveMetastore_add_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1317)
+          foreach ($this->new_parts as $iter1324)
           {
-            $xfer += $iter1317->write($output);
+            $xfer += $iter1324->write($output);
           }
         }
         $output->writeListEnd();
@@ -29284,15 +29340,15 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size1318 = 0;
-            $_etype1321 = 0;
-            $xfer += $input->readListBegin($_etype1321, $_size1318);
-            for ($_i1322 = 0; $_i1322 < $_size1318; ++$_i1322)
+            $_size1325 = 0;
+            $_etype1328 = 0;
+            $xfer += $input->readListBegin($_etype1328, $_size1325);
+            for ($_i1329 = 0; $_i1329 < $_size1325; ++$_i1329)
             {
-              $elem1323 = null;
-              $elem1323 = new \metastore\PartitionSpec();
-              $xfer += $elem1323->read($input);
-              $this->new_parts []= $elem1323;
+              $elem1330 = null;
+              $elem1330 = new \metastore\PartitionSpec();
+              $xfer += $elem1330->read($input);
+              $this->new_parts []= $elem1330;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29320,9 +29376,9 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1324)
+          foreach ($this->new_parts as $iter1331)
           {
-            $xfer += $iter1324->write($output);
+            $xfer += $iter1331->write($output);
           }
         }
         $output->writeListEnd();
@@ -29572,14 +29628,14 @@ class ThriftHiveMetastore_append_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1325 = 0;
-            $_etype1328 = 0;
-            $xfer += $input->readListBegin($_etype1328, $_size1325);
-            for ($_i1329 = 0; $_i1329 < $_size1325; ++$_i1329)
+            $_size1332 = 0;
+            $_etype1335 = 0;
+            $xfer += $input->readListBegin($_etype1335, $_size1332);
+            for ($_i1336 = 0; $_i1336 < $_size1332; ++$_i1336)
             {
-              $elem1330 = null;
-              $xfer += $input->readString($elem1330);
-              $this->part_vals []= $elem1330;
+              $elem1337 = null;
+              $xfer += $input->readString($elem1337);
+              $this->part_vals []= $elem1337;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29617,9 +29673,9 @@ class ThriftHiveMetastore_append_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1331)
+          foreach ($this->part_vals as $iter1338)
           {
-            $xfer += $output->writeString($iter1331);
+            $xfer += $output->writeString($iter1338);
           }
         }
         $output->writeListEnd();
@@ -30121,14 +30177,14 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1332 = 0;
-            $_etype1335 = 0;
-            $xfer += $input->readListBegin($_etype1335, $_size1332);
-            for ($_i1336 = 0; $_i1336 < $_size1332; ++$_i1336)
+            $_size1339 = 0;
+            $_etype1342 = 0;
+            $xfer += $input->readListBegin($_etype1342, $_size1339);
+            for ($_i1343 = 0; $_i1343 < $_size1339; ++$_i1343)
             {
-              $elem1337 = null;
-              $xfer += $input->readString($elem1337);
-              $this->part_vals []= $elem1337;
+              $elem1344 = null;
+              $xfer += $input->readString($elem1344);
+              $this->part_vals []= $elem1344;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30174,9 +30230,9 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1338)
+          foreach ($this->part_vals as $iter1345)
           {
-            $xfer += $output->writeString($iter1338);
+            $xfer += $output->writeString($iter1345);
           }
         }
         $output->writeListEnd();
@@ -31030,14 +31086,14 @@ class ThriftHiveMetastore_drop_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1339 = 0;
-            $_etype1342 = 0;
-            $xfer += $input->readListBegin($_etype1342, $_size1339);
-            for ($_i1343 = 0; $_i1343 < $_size1339; ++$_i1343)
+            $_size1346 = 0;
+            $_etype1349 = 0;
+            $xfer += $input->readListBegin($_etype1349, $_size1346);
+            for ($_i1350 = 0; $_i1350 < $_size1346; ++$_i1350)
             {
-              $elem1344 = null;
-              $xfer += $input->readString($elem1344);
-              $this->part_vals []= $elem1344;
+              $elem1351 = null;
+              $xfer += $input->readString($elem1351);
+              $this->part_vals []= $elem1351;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31082,9 +31138,9 @@ class ThriftHiveMetastore_drop_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1345)
+          foreach ($this->part_vals as $iter1352)
           {
-            $xfer += $output->writeString($iter1345);
+            $xfer += $output->writeString($iter1352);
           }
         }
         $output->writeListEnd();
@@ -31337,14 +31393,14 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1346 = 0;
-            $_etype1349 = 0;
-            $xfer += $input->readListBegin($_etype1349, $_size1346);
-            for ($_i1350 = 0; $_i1350 < $_size1346; ++$_i1350)
+            $_size1353 = 0;
+            $_etype1356 = 0;
+            $xfer += $input->readListBegin($_etype1356, $_size1353);
+            for ($_i1357 = 0; $_i1357 < $_size1353; ++$_i1357)
             {
-              $elem1351 = null;
-              $xfer += $input->readString($elem1351);
-              $this->part_vals []= $elem1351;
+              $elem1358 = null;
+              $xfer += $input->readString($elem1358);
+              $this->part_vals []= $elem1358;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31397,9 +31453,9 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1352)
+          foreach ($this->part_vals as $iter1359)
           {
-            $xfer += $output->writeString($iter1352);
+            $xfer += $output->writeString($iter1359);
           }
         }
         $output->writeListEnd();
@@ -32413,14 +32469,14 @@ class ThriftHiveMetastore_get_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1353 = 0;
-            $_etype1356 = 0;
-            $xfer += $input->readListBegin($_etype1356, $_size1353);
-            for ($_i1357 = 0; $_i1357 < $_size1353; ++$_i1357)
+            $_size1360 = 0;
+            $_etype1363 = 0;
+            $xfer += $input->readListBegin($_etype1363, $_size1360);
+            for ($_i1364 = 0; $_i1364 < $_size1360; ++$_i1364)
             {
-              $elem1358 = null;
-              $xfer += $input->readString($elem1358);
-              $this->part_vals []= $elem1358;
+              $elem1365 = null;
+              $xfer += $input->readString($elem1365);
+              $this->part_vals []= $elem1365;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32458,9 +32514,9 @@ class ThriftHiveMetastore_get_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1359)
+          foreach ($this->part_vals as $iter1366)
           {
-            $xfer += $output->writeString($iter1359);
+            $xfer += $output->writeString($iter1366);
           }
         }
         $output->writeListEnd();
@@ -32912,17 +32968,17 @@ class ThriftHiveMetastore_exchange_partition_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size1360 = 0;
-            $_ktype1361 = 0;
-            $_vtype1362 = 0;
-            $xfer += $input->readMapBegin($_ktype1361, $_vtype1362, $_size1360);
-            for ($_i1364 = 0; $_i1364 < $_size1360; ++$_i1364)
+            $_size1367 = 0;
+            $_ktype1368 = 0;
+            $_vtype1369 = 0;
+            $xfer += $input->readMapBegin($_ktype1368, $_vtype1369, $_size1367);
+            for ($_i1371 = 0; $_i1371 < $_size1367; ++$_i1371)
             {
-              $key1365 = '';
-              $val1366 = '';
-              $xfer += $input->readString($key1365);
-              $xfer += $input->readString($val1366);
-              $this->partitionSpecs[$key1365] = $val1366;
+              $key1372 = '';
+              $val1373 = '';
+              $xfer += $input->readString($key1372);
+              $xfer += $input->readString($val1373);
+              $this->partitionSpecs[$key1372] = $val1373;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -32978,10 +33034,10 @@ class ThriftHiveMetastore_exchange_partition_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter1367 => $viter1368)
+          foreach ($this->partitionSpecs as $kiter1374 => $viter1375)
           {
-            $xfer += $output->writeString($kiter1367);
-            $xfer += $output->writeString($viter1368);
+            $xfer += $output->writeString($kiter1374);
+            $xfer += $output->writeString($viter1375);
           }
         }
         $output->writeMapEnd();
@@ -33293,17 +33349,17 @@ class ThriftHiveMetastore_exchange_partitions_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size1369 = 0;
-            $_ktype1370 = 0;
-            $_vtype1371 = 0;
-            $xfer += $input->readMapBegin($_ktype1370, $_vtype1371, $_size1369);
-            for ($_i1373 = 0; $_i1373 < $_size1369; ++$_i1373)
+            $_size1376 = 0;
+            $_ktype1377 = 0;
+            $_vtype1378 = 0;
+            $xfer += $input->readMapBegin($_ktype1377, $_vtype1378, $_size1376);
+            for ($_i1380 = 0; $_i1380 < $_size1376; ++$_i1380)
             {
-              $key1374 = '';
-              $val1375 = '';
-              $xfer += $input->readString($key1374);
-              $xfer += $input->readString($val1375);
-              $this->partitionSpecs[$key1374] = $val1375;
+              $key1381 = '';
+              $val1382 = '';
+              $xfer += $input->readString($key1381);
+              $xfer += $input->readString($val1382);
+              $this->partitionSpecs[$key1381] = $val1382;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -33359,10 +33415,10 @@ class ThriftHiveMetastore_exchange_partitions_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter1376 => $viter1377)
+          foreach ($this->partitionSpecs as $kiter1383 => $viter1384)
           {
-            $xfer += $output->writeString($kiter1376);
-            $xfer += $output->writeString($viter1377);
+            $xfer += $output->writeString($kiter1383);
+            $xfer += $output->writeString($viter1384);
           }
         }
         $output->writeMapEnd();
@@ -33495,15 +33551,15 @@ class ThriftHiveMetastore_exchange_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1378 = 0;
-            $_etype1381 = 0;
-            $xfer += $input->readListBegin($_etype1381, $_size1378);
-            for ($_i1382 = 0; $_i1382 < $_size1378; ++$_i1382)
+            $_size1385 = 0;
+            $_etype1388 = 0;
+            $xfer += $input->readListBegin($_etype1388, $_size1385);
+            for ($_i1389 = 0; $_i1389 < $_size1385; ++$_i1389)
             {
-              $elem1383 = null;
-              $elem1383 = new \metastore\Partition();
-              $xfer += $elem1383->read($input);
-              $this->success []= $elem1383;
+              $elem1390 = null;
+              $elem1390 = new \metastore\Partition();
+              $xfer += $elem1390->read($input);
+              $this->success []= $elem1390;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33563,9 +33619,9 @@ class ThriftHiveMetastore_exchange_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1384)
+          foreach ($this->success as $iter1391)
           {
-            $xfer += $iter1384->write($output);
+            $xfer += $iter1391->write($output);
           }
         }
         $output->writeListEnd();
@@ -33711,14 +33767,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1385 = 0;
-            $_etype1388 = 0;
-            $xfer += $input->readListBegin($_etype1388, $_size1385);
-            for ($_i1389 = 0; $_i1389 < $_size1385; ++$_i1389)
+            $_size1392 = 0;
+            $_etype1395 = 0;
+            $xfer += $input->readListBegin($_etype1395, $_size1392);
+            for ($_i1396 = 0; $_i1396 < $_size1392; ++$_i1396)
             {
-              $elem1390 = null;
-              $xfer += $input->readString($elem1390);
-              $this->part_vals []= $elem1390;
+              $elem1397 = null;
+              $xfer += $input->readString($elem1397);
+              $this->part_vals []= $elem1397;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33735,14 +33791,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1391 = 0;
-            $_etype1394 = 0;
-            $xfer += $input->readListBegin($_etype1394, $_size1391);
-            for ($_i1395 = 0; $_i1395 < $_size1391; ++$_i1395)
+            $_size1398 = 0;
+            $_etype1401 = 0;
+            $xfer += $input->readListBegin($_etype1401, $_size1398);
+            for ($_i1402 = 0; $_i1402 < $_size1398; ++$_i1402)
             {
-              $elem1396 = null;
-              $xfer += $input->readString($elem1396);
-              $this->group_names []= $elem1396;
+              $elem1403 = null;
+              $xfer += $input->readString($elem1403);
+              $this->group_names []= $elem1403;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33780,9 +33836,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1397)
+          foreach ($this->part_vals as $iter1404)
           {
-            $xfer += $output->writeString($iter1397);
+            $xfer += $output->writeString($iter1404);
           }
         }
         $output->writeListEnd();
@@ -33802,9 +33858,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1398)
+          foreach ($this->group_names as $iter1405)
           {
-            $xfer += $output->writeString($iter1398);
+            $xfer += $output->writeString($iter1405);
           }
         }
         $output->writeListEnd();
@@ -34395,15 +34451,15 @@ class ThriftHiveMetastore_get_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1399 = 0;
-            $_etype1402 = 0;
-            $xfer += $input->readListBegin($_etype1402, $_size1399);
-            for ($_i1403 = 0; $_i1403 < $_size1399; ++$_i1403)
+            $_size1406 = 0;
+            $_etype1409 = 0;
+            $xfer += $input->readListBegin($_etype1409, $_size1406);
+            for ($_i1410 = 0; $_i1410 < $_size1406; ++$_i1410)
             {
-              $elem1404 = null;
-              $elem1404 = new \metastore\Partition();
-              $xfer += $elem1404->read($input);
-              $this->success []= $elem1404;
+              $elem1411 = null;
+              $elem1411 = new \metastore\Partition();
+              $xfer += $elem1411->read($input);
+              $this->success []= $elem1411;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34447,9 +34503,9 @@ class ThriftHiveMetastore_get_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1405)
+          foreach ($this->success as $iter1412)
           {
-            $xfer += $iter1405->write($output);
+            $xfer += $iter1412->write($output);
           }
         }
         $output->writeListEnd();
@@ -34805,14 +34861,14 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1406 = 0;
-            $_etype1409 = 0;
-            $xfer += $input->readListBegin($_etype1409, $_size1406);
-            for ($_i1410 = 0; $_i1410 < $_size1406; ++$_i1410)
+            $_size1413 = 0;
+            $_etype1416 = 0;
+            $xfer += $input->readListBegin($_etype1416, $_size1413);
+            for ($_i1417 = 0; $_i1417 < $_size1413; ++$_i1417)
             {
-              $elem1411 = null;
-              $xfer += $input->readString($elem1411);
-              $this->group_names []= $elem1411;
+              $elem1418 = null;
+              $xfer += $input->readString($elem1418);
+              $this->group_names []= $elem1418;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34860,9 +34916,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1412)
+          foreach ($this->group_names as $iter1419)
           {
-            $xfer += $output->writeString($iter1412);
+            $xfer += $output->writeString($iter1419);
           }
         }
         $output->writeListEnd();
@@ -34951,15 +35007,15 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1413 = 0;
-            $_etype1416 = 0;
-            $xfer += $input->readListBegin($_etype1416, $_size1413);
-            for ($_i1417 = 0; $_i1417 < $_size1413; ++$_i1417)
+            $_size1420 = 0;
+            $_etype1423 = 0;
+            $xfer += $input->readListBegin($_etype1423, $_size1420);
+            for ($_i1424 = 0; $_i1424 < $_size1420; ++$_i1424)
             {
-              $elem1418 = null;
-              $elem1418 = new \metastore\Partition();
-              $xfer += $elem1418->read($input);
-              $this->success []= $elem1418;
+              $elem1425 = null;
+              $elem1425 = new \metastore\Partition();
+              $xfer += $elem1425->read($input);
+              $this->success []= $elem1425;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -35003,9 +35059,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1419)
+          foreach ($this->success as $iter1426)
           {
-            $xfer += $iter1419->write($output);
+            $xfer += $iter1426->write($output);
           }
         }
         $output->writeListEnd();
@@ -35225,15 +35281,15 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1420 = 0;
-            $_etype1423 = 0;
-            $xfer += $input->readListBegin($_etype1423, $_size1420);
-            for ($_i1424 = 0; $_i1424 < $_size1420; ++$_i1424)
+            $_size1427 = 0;
+            $_etype1430 = 0;
+            $xfer += $input->readListBegin($_etype1430, $_size1427);
+            for ($_i1431 = 0; $_i1431 < $_size1427; ++$_i1431)
             {
-              $elem1425 = null;
-              $elem1425 = new \metastore\PartitionSpec();
-              $xfer += $elem1425->read($input);
-              $this->success []= $elem1425;
+              $elem1432 = null;
+              $elem1432 = new \metastore\PartitionSpec();
+              $xfer += $elem1432->read($input);
+              $this->success []= $elem1432;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -35277,9 +35333,9 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1426)
+          foreach ($this->success as $iter1433)
           {
-            $xfer += $iter1426->write($output);
+            $xfer += $iter1433->write($output);
           }
         }
         $output->writeListEnd();
@@ -35498,14 +35554,14 @@ class ThriftHiveMetastore_get_partition_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1427 = 0;
-            $_etype1430 = 0;
-            $xfer += $input->readListBegin($_etype1430, $_size1427);
-            for ($_i1431 = 0; $_i1431 < $_size1427; ++$_i1431)
+            $_size1434 = 0;
+            $_etype1437 = 0;
+            $xfer += $input->readListBegin($_etype1437, $_size1434);
+            for ($_i1438 = 0; $_i1438 < $_size1434; ++$_i1438)
             {
-              $elem1432 = null;
-              $xfer += $input->readString($elem1432);
-              $this->success []= $elem1432;
+              $elem1439 = null;
+              $xfer += $input->readString($elem1439);
+              $this->success []= $elem1439;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -35549,9 +35605,9 @@ class ThriftHiveMetastore_get_partition_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1433)
+          foreach ($this->success as $iter1440)
           {
-            $xfer += $output->writeString($iter1433);
+            $xfer += $output->writeString($iter1440);
           }
         }
         $output->writeListEnd();
@@ -35882,14 +35938,14 @@ class ThriftHiveMetastore_get_partitions_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1434 = 0;
-            $_etype1437 = 0;
-            $xfer += $input->readListBegin($_etype1437, $_size1434);
-            for ($_i1438 = 0; $_i1438 < $_size1434; ++$_i1438)
+            $_size1441 = 0;
+            $_etype1444 = 0;
+            $xfer += $input->readListBegin($_etype1444, $_size1441);
+            for ($_i1445 = 0; $_i1445 < $_size1441; ++$_i1445)
             {
-              $elem1439 = null;
-              $xfer += $input->readString($elem1439);
-              $this->part_vals []= $elem1439;
+              $elem1446 = null;
+              $xfer += $input->readString($elem1446);
+              $this->part_vals []= $elem1446;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -35934,9 +35990,9 @@ class ThriftHiveMetastore_get_partitions_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1440)
+          foreach ($this->part_vals as $iter1447)
           {
-            $xfer += $output->writeString($iter1440);
+            $xfer += $output->writeString($iter1447);
           }
         }
         $output->writeListEnd();
@@ -36030,15 +36086,15 @@ class ThriftHiveMetastore_get_partitions_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1441 = 0;
-            $_etype1444 = 0;
-            $xfer += $input->readListBegin($_etype1444, $_size1441);
-            for ($_i1445 = 0; $_i1445 < $_size1441; ++$_i1445)
+            $_size1448 = 0;
+            $_etype1451 = 0;
+            $xfer += $input->readListBegin($_etype1451, $_size1448);
+            for ($_i1452 = 0; $_i1452 < $_size1448; ++$_i1452)
             {
-              $elem1446 = null;
-              $elem1446 = new \metastore\Partition();
-              $xfer += $elem1446->read($input);
-              $this->success []= $elem1446;
+              $elem1453 = null;
+              $elem1453 = new \metastore\Partition();
+              $xfer += $elem1453->read($input);
+              $this->success []= $elem1453;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -36082,9 +36138,9 @@ class ThriftHiveMetastore_get_partitions_ps_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1447)
+          foreach ($this->success as $iter1454)
           {
-            $xfer += $iter1447->write($output);
+            $xfer += $iter1454->write($output);
... 3976 lines suppressed ...