You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ng...@apache.org on 2019/09/16 03:37:12 UTC

[hive] branch master updated: HIVE-22189: HMS Translation: Enforce strict locations for managed vs external tables.(Naveen Gangam, reviewed by Thejas Nair)

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

ngangam 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 1880bc6  HIVE-22189: HMS Translation: Enforce strict locations for managed vs external tables.(Naveen Gangam, reviewed by Thejas Nair)
1880bc6 is described below

commit 1880bc64324e693bd8db425385c9a5031c61298d
Author: Naveen Gangam <ng...@apache.org>
AuthorDate: Sun Sep 15 23:32:25 2019 -0400

    HIVE-22189: HMS Translation: Enforce strict locations for managed vs external tables.(Naveen Gangam, reviewed by Thejas Nair)
---
 .../metastore/TestHiveMetastoreTransformer.java    |  181 +-
 .../hive/metastore/api/AlterTableRequest.java      |  269 ++-
 .../metastore/api/GetPartitionsFilterSpec.java     |   32 +-
 .../metastore/api/GetPartitionsProjectionSpec.java |   32 +-
 .../hive/metastore/api/GetPartitionsRequest.java   |   64 +-
 .../hive/metastore/api/GetPartitionsResponse.java  |   36 +-
 .../hive/metastore/api/ThriftHiveMetastore.java    | 2536 ++++++++++----------
 .../gen-php/metastore/ThriftHiveMetastore.php      | 1426 +++++------
 .../src/gen/thrift/gen-php/metastore/Types.php     |  164 +-
 .../gen-py/hive_metastore/ThriftHiveMetastore.py   |  964 ++++----
 .../src/gen/thrift/gen-py/hive_metastore/ttypes.py |   98 +-
 .../src/gen/thrift/gen-rb/hive_metastore_types.rb  |    6 +-
 .../hadoop/hive/metastore/HiveMetaStoreClient.java |   12 +
 .../src/main/thrift/hive_metastore.thrift          |    2 +
 .../hadoop/hive/metastore/HiveMetaStore.java       |   14 +-
 .../metastore/IMetaStoreMetadataTransformer.java   |   10 +
 .../metastore/MetastoreDefaultTransformer.java     |   43 +-
 17 files changed, 3252 insertions(+), 2637 deletions(-)

diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetastoreTransformer.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetastoreTransformer.java
index 466cce6..6a5915d 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetastoreTransformer.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetastoreTransformer.java
@@ -89,7 +89,7 @@ public class TestHiveMetastoreTransformer {
     MetastoreConf.setVar(conf, ConfVars.METASTORE_METADATA_TRANSFORMER_CLASS,
         "org.apache.hadoop.hive.metastore.MetastoreDefaultTransformer");
     MetastoreConf.setBoolVar(conf, ConfVars.HIVE_IN_TEST, false);
-    MetastoreConf.setVar(conf, ConfVars.WAREHOUSE_EXTERNAL, ext_wh.getAbsolutePath());
+    MetastoreConf.setVar(conf, ConfVars.WAREHOUSE_EXTERNAL, ext_wh.getCanonicalPath());
     client = new HiveMetaStoreClient(conf);
   }
 
@@ -135,7 +135,6 @@ public class TestHiveMetastoreTransformer {
       Table tbl = createTableWithCapabilities(tProps);
 
       setHMSClient("testTranformerWithOldTables", (new String[] { "HIVEBUCKET2", "EXTREAD", "EXTWRITE"}));
-
       Table tbl2 = client.getTable(dbName, tblName);
       assertEquals("Created and retrieved tables do not match:" + tbl2.getTableName() + ":" + tblName,
           tbl2.getTableName(), tblName);
@@ -1152,6 +1151,179 @@ public class TestHiveMetastoreTransformer {
   }
 
   @Test
+  public void testTransformerAlterTable() throws Exception {
+    try {
+      resetHMSClient();
+
+      final String dbName = "dbalter";
+      String tblName = "test_alter_mgd_table";
+      TableType type = TableType.MANAGED_TABLE;
+      StringBuilder table_params = new StringBuilder();
+      table_params.append("key1=val1");
+      Map<String, Object> tProps = new HashMap<>();
+      tProps.put("DBNAME", dbName);
+      tProps.put("TBLNAME", tblName);
+      tProps.put("TBLTYPE", type);
+      tProps.put("PROPERTIES", table_params.toString());
+
+      Table table = createTableWithCapabilities(tProps); // should be converted to external table
+
+      // retrieve the table
+      Table tbl2 = client.getTable(dbName, tblName);
+      assertEquals("Table type expected to be EXTERNAL", "EXTERNAL_TABLE", tbl2.getTableType());
+      String tableLocation = tbl2.getSd().getLocation();
+      int idx = (tableLocation.indexOf(":") > 0) ? tableLocation.indexOf(":") : 0;
+      tableLocation = tableLocation.substring(idx+1);
+
+      String newLocation = wh.getAbsolutePath().concat(File.separator).concat(dbName).concat(File.separator)
+          .concat(tblName);
+      table.getSd().setLocation(newLocation);
+      try {
+        client.alter_table(dbName, tblName, table);
+        fail("alter_table expected to fail due to location:" + newLocation);
+      } catch (Exception e) {
+        e.printStackTrace();
+        LOG.info("alter_table failed with exception, as expected");
+      }
+
+      // retrieve the table and check that the location was not altered
+      tbl2 = client.getTable(dbName, tblName);
+      idx = (tbl2.getSd().getLocation().indexOf(":") > 0) ? tbl2.getSd().getLocation().indexOf(":") : 0;
+      assertEquals("Table location expected to be in external warehouse", tableLocation, tbl2.getSd().getLocation().substring(idx+1));
+
+      newLocation = tableLocation.concat("_new");
+      table.getSd().setLocation((new File(newLocation)).getCanonicalPath());
+      try {
+        client.alter_table(dbName, tblName, table);
+        LOG.info("alter_table with new location succeeded as expected");
+      } catch (Exception e) {
+        fail("alter_table expected to succeed with new location:" + newLocation);
+      }
+
+      // retrieve the table and check that the location was altered
+      tbl2 = client.getTable(dbName, tblName);
+      idx = (tbl2.getSd().getLocation().indexOf(":") > 0) ? tbl2.getSd().getLocation().indexOf(":") : 0;
+      assertEquals("Table location expected to be in external warehouse", newLocation, tbl2.getSd().getLocation().substring(idx+1));
+
+      tblName = "test_create_insert_table";
+      type = TableType.MANAGED_TABLE;
+      tProps.put("TBLNAME", tblName);
+      tProps.put("TBLTYPE", TableType.MANAGED_TABLE);
+      table_params = new StringBuilder();
+      table_params.append("key1=val1");
+      table_params.append(";");
+      table_params.append("transactional_properties=insert_only");
+      tProps.put("PROPERTIES", table_params.toString());
+
+      List<String> capabilities = new ArrayList<>();
+      capabilities.add("HIVEMANAGEDINSERTWRITE");
+      setHMSClient("TestAlterTableMGD#1", (String[])(capabilities.toArray(new String[0])));
+
+      table = createTableWithCapabilities(tProps);
+
+      // retrieve the table
+      tbl2 = client.getTable(dbName, tblName);
+      tableLocation = tbl2.getSd().getLocation();
+      idx = (tableLocation.indexOf(":") > 0) ? tableLocation.indexOf(":") : 0;
+      tableLocation = tableLocation.substring(idx+1);
+
+      assertEquals("Table type expected to be MANAGED", "MANAGED_TABLE", tbl2.getTableType());
+
+      newLocation = ext_wh.getAbsolutePath().concat(File.separator).concat(dbName).concat(File.separator)
+          .concat(tblName);
+      table.getSd().setLocation(newLocation);
+      try {
+        client.alter_table(dbName, tblName, table);
+        fail("alter_table expected to fail but succeeded with new location:" + newLocation);
+      } catch (Exception e) {
+        LOG.info("alter_table failed with exception as expected");
+      }
+
+      // retrieve the table and ensure location has not been changed.
+      tbl2 = client.getTable(dbName, tblName);
+      assertEquals("Table type expected to be MANAGED", "MANAGED_TABLE", tbl2.getTableType());
+      idx = (tbl2.getSd().getLocation().indexOf(":") > 0) ? tbl2.getSd().getLocation().indexOf(":") : 0;
+      assertEquals("Table location expected to remain unaltered", tableLocation, tbl2.getSd().getLocation().substring(idx+1));
+
+      newLocation = tableLocation + "_new";
+      table.getSd().setLocation(newLocation);
+      try {
+        client.alter_table(dbName, tblName, table);
+        LOG.info("alter_table succeeded with new location as expected");
+      } catch (Exception e) {
+        fail("alter_table expected to succeed but failed with new location:" + newLocation);
+      }
+      // retrieve the table and ensure location has not been changed.
+      tbl2 = client.getTable(dbName, tblName);
+      assertEquals("Table type expected to be MANAGED", "MANAGED_TABLE", tbl2.getTableType());
+      idx = (tbl2.getSd().getLocation().indexOf(":") > 0) ? tbl2.getSd().getLocation().indexOf(":") : 0;
+      assertEquals("Table location expected to be new location", newLocation, tbl2.getSd().getLocation().substring(idx+1));
+      resetHMSClient();
+
+      tblName = "test_create_acid_table";
+      type = TableType.MANAGED_TABLE;
+      tProps.put("TBLNAME", tblName);
+      tProps.put("TBLTYPE", TableType.MANAGED_TABLE);
+      table_params = new StringBuilder();
+      table_params.append("key1=val1");
+      table_params.append(";");
+      table_params.append("transactional=true");
+      tProps.put("PROPERTIES", table_params.toString());
+
+      capabilities = new ArrayList<>();
+      capabilities.add("HIVEFULLACIDWRITE");
+      setHMSClient("TestAlterTableMGD#1", (String[])(capabilities.toArray(new String[0])));
+
+      table = createTableWithCapabilities(tProps);
+
+      // retrieve the table
+      tbl2 = client.getTable(dbName, tblName);
+      tableLocation = tbl2.getSd().getLocation();
+      idx = (tableLocation.indexOf(":") > 0) ? tableLocation.indexOf(":") : 0;
+      tableLocation = tableLocation.substring(idx+1);
+
+      assertEquals("Table type expected to be MANAGED", "MANAGED_TABLE", tbl2.getTableType());
+
+      newLocation = ext_wh.getAbsolutePath().concat(File.separator).concat(dbName).concat(File.separator)
+          .concat(tblName);
+      table.getSd().setLocation(newLocation);
+      try {
+        client.alter_table(dbName, tblName, table);
+        fail("alter_table expected to fail but succeeded with new location:" + newLocation);
+      } catch (Exception e) {
+        LOG.info("alter_table failed with exception as expected");
+      }
+
+      // retrieve the table and ensure location has not been changed.
+      tbl2 = client.getTable(dbName, tblName);
+      assertEquals("Table type expected to be MANAGED", "MANAGED_TABLE", tbl2.getTableType());
+      idx = (tbl2.getSd().getLocation().indexOf(":") > 0) ? tbl2.getSd().getLocation().indexOf(":") : 0;
+      assertEquals("Table location expected to remain unaltered", tableLocation, tbl2.getSd().getLocation().substring(idx+1));
+
+      newLocation = tableLocation + "_new";
+      table.getSd().setLocation(newLocation);
+      try {
+        client.alter_table(dbName, tblName, table);
+        LOG.info("alter_table succeeded with new location as expected");
+      } catch (Exception e) {
+        e.printStackTrace();
+        fail("alter_table expected to succeed but failed with new location:" + newLocation);
+      }
+      // retrieve the table and ensure location has not been changed.
+      tbl2 = client.getTable(dbName, tblName);
+      assertEquals("Table type expected to be MANAGED", "MANAGED_TABLE", tbl2.getTableType());
+      idx = (tbl2.getSd().getLocation().indexOf(":") > 0) ? tbl2.getSd().getLocation().indexOf(":") : 0;
+      assertEquals("Table location expected to be new location", newLocation, tbl2.getSd().getLocation().substring(idx+1));
+    } catch (Exception e) {
+      System.err.println(org.apache.hadoop.util.StringUtils.stringifyException(e));
+      System.err.println("testAlterTable() failed.");
+      fail("testAlterTable failed:" + e.getMessage());
+    } finally {
+      resetHMSClient();
+    }
+  }
+
+  @Test
   public void testTransformerDatabase() throws Exception {
     try {
       resetHMSClient();
@@ -1329,6 +1501,7 @@ public class TestHiveMetastoreTransformer {
       TableType type = (TableType)props.getOrDefault("TBLTYPE", TableType.MANAGED_TABLE);
       int buckets = ((Integer)props.getOrDefault("BUCKETS", -1)).intValue();
       String properties = (String)props.getOrDefault("PROPERTIES", "");
+      String location = (String)(props.get("LOCATION"));
       boolean dropDb = ((Boolean)props.getOrDefault("DROPDB", Boolean.TRUE)).booleanValue();
       int partitionCount = ((Integer)props.getOrDefault("PARTITIONS", 0)).intValue();
 
@@ -1387,11 +1560,15 @@ public class TestHiveMetastoreTransformer {
           .setTableName(tblName)
           .setCols(typ1.getFields())
           .setType(type.name())
+          .setLocation(location)
           .setNumBuckets(buckets)
           .setTableParams(table_params)
           .addBucketCol("name")
           .addStorageDescriptorParam("test_param_1", "Use this for comments etc");
 
+      if (location != null)
+        builder.setLocation(location);
+
       if (buckets > 0)
         builder.setNumBuckets(buckets).addBucketCol("name");
 
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterTableRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterTableRequest.java
index df295c7..9618ee2 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterTableRequest.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterTableRequest.java
@@ -45,6 +45,8 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField ENVIRONMENT_CONTEXT_FIELD_DESC = new org.apache.thrift.protocol.TField("environmentContext", org.apache.thrift.protocol.TType.STRUCT, (short)5);
   private static final org.apache.thrift.protocol.TField WRITE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("writeId", org.apache.thrift.protocol.TType.I64, (short)6);
   private static final org.apache.thrift.protocol.TField VALID_WRITE_ID_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("validWriteIdList", org.apache.thrift.protocol.TType.STRING, (short)7);
+  private static final org.apache.thrift.protocol.TField PROCESSOR_CAPABILITIES_FIELD_DESC = new org.apache.thrift.protocol.TField("processorCapabilities", org.apache.thrift.protocol.TType.LIST, (short)8);
+  private static final org.apache.thrift.protocol.TField PROCESSOR_IDENTIFIER_FIELD_DESC = new org.apache.thrift.protocol.TField("processorIdentifier", org.apache.thrift.protocol.TType.STRING, (short)9);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -59,6 +61,8 @@ import org.slf4j.LoggerFactory;
   private EnvironmentContext environmentContext; // optional
   private long writeId; // optional
   private String validWriteIdList; // optional
+  private List<String> processorCapabilities; // optional
+  private String processorIdentifier; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -68,7 +72,9 @@ import org.slf4j.LoggerFactory;
     TABLE((short)4, "table"),
     ENVIRONMENT_CONTEXT((short)5, "environmentContext"),
     WRITE_ID((short)6, "writeId"),
-    VALID_WRITE_ID_LIST((short)7, "validWriteIdList");
+    VALID_WRITE_ID_LIST((short)7, "validWriteIdList"),
+    PROCESSOR_CAPABILITIES((short)8, "processorCapabilities"),
+    PROCESSOR_IDENTIFIER((short)9, "processorIdentifier");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -97,6 +103,10 @@ import org.slf4j.LoggerFactory;
           return WRITE_ID;
         case 7: // VALID_WRITE_ID_LIST
           return VALID_WRITE_ID_LIST;
+        case 8: // PROCESSOR_CAPABILITIES
+          return PROCESSOR_CAPABILITIES;
+        case 9: // PROCESSOR_IDENTIFIER
+          return PROCESSOR_IDENTIFIER;
         default:
           return null;
       }
@@ -139,7 +149,7 @@ import org.slf4j.LoggerFactory;
   // isset id assignments
   private static final int __WRITEID_ISSET_ID = 0;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.CAT_NAME,_Fields.ENVIRONMENT_CONTEXT,_Fields.WRITE_ID,_Fields.VALID_WRITE_ID_LIST};
+  private static final _Fields optionals[] = {_Fields.CAT_NAME,_Fields.ENVIRONMENT_CONTEXT,_Fields.WRITE_ID,_Fields.VALID_WRITE_ID_LIST,_Fields.PROCESSOR_CAPABILITIES,_Fields.PROCESSOR_IDENTIFIER};
   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);
@@ -157,6 +167,11 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
     tmpMap.put(_Fields.VALID_WRITE_ID_LIST, new org.apache.thrift.meta_data.FieldMetaData("validWriteIdList", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.PROCESSOR_CAPABILITIES, new org.apache.thrift.meta_data.FieldMetaData("processorCapabilities", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.PROCESSOR_IDENTIFIER, new org.apache.thrift.meta_data.FieldMetaData("processorIdentifier", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(AlterTableRequest.class, metaDataMap);
   }
@@ -201,6 +216,13 @@ import org.slf4j.LoggerFactory;
     if (other.isSetValidWriteIdList()) {
       this.validWriteIdList = other.validWriteIdList;
     }
+    if (other.isSetProcessorCapabilities()) {
+      List<String> __this__processorCapabilities = new ArrayList<String>(other.processorCapabilities);
+      this.processorCapabilities = __this__processorCapabilities;
+    }
+    if (other.isSetProcessorIdentifier()) {
+      this.processorIdentifier = other.processorIdentifier;
+    }
   }
 
   public AlterTableRequest deepCopy() {
@@ -217,6 +239,8 @@ import org.slf4j.LoggerFactory;
     this.writeId = -1L;
 
     this.validWriteIdList = null;
+    this.processorCapabilities = null;
+    this.processorIdentifier = null;
   }
 
   public String getCatName() {
@@ -379,6 +403,67 @@ import org.slf4j.LoggerFactory;
     }
   }
 
+  public int getProcessorCapabilitiesSize() {
+    return (this.processorCapabilities == null) ? 0 : this.processorCapabilities.size();
+  }
+
+  public java.util.Iterator<String> getProcessorCapabilitiesIterator() {
+    return (this.processorCapabilities == null) ? null : this.processorCapabilities.iterator();
+  }
+
+  public void addToProcessorCapabilities(String elem) {
+    if (this.processorCapabilities == null) {
+      this.processorCapabilities = new ArrayList<String>();
+    }
+    this.processorCapabilities.add(elem);
+  }
+
+  public List<String> getProcessorCapabilities() {
+    return this.processorCapabilities;
+  }
+
+  public void setProcessorCapabilities(List<String> processorCapabilities) {
+    this.processorCapabilities = processorCapabilities;
+  }
+
+  public void unsetProcessorCapabilities() {
+    this.processorCapabilities = null;
+  }
+
+  /** Returns true if field processorCapabilities is set (has been assigned a value) and false otherwise */
+  public boolean isSetProcessorCapabilities() {
+    return this.processorCapabilities != null;
+  }
+
+  public void setProcessorCapabilitiesIsSet(boolean value) {
+    if (!value) {
+      this.processorCapabilities = null;
+    }
+  }
+
+  public String getProcessorIdentifier() {
+    return this.processorIdentifier;
+  }
+
+  public void setProcessorIdentifier(String processorIdentifier) {
+    this.processorIdentifier = processorIdentifier;
+  }
+
+  public void unsetProcessorIdentifier() {
+    this.processorIdentifier = null;
+  }
+
+  /** Returns true if field processorIdentifier is set (has been assigned a value) and false otherwise */
+  public boolean isSetProcessorIdentifier() {
+    return this.processorIdentifier != null;
+  }
+
+  public void setProcessorIdentifierIsSet(boolean value) {
+    if (!value) {
+      this.processorIdentifier = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case CAT_NAME:
@@ -437,6 +522,22 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case PROCESSOR_CAPABILITIES:
+      if (value == null) {
+        unsetProcessorCapabilities();
+      } else {
+        setProcessorCapabilities((List<String>)value);
+      }
+      break;
+
+    case PROCESSOR_IDENTIFIER:
+      if (value == null) {
+        unsetProcessorIdentifier();
+      } else {
+        setProcessorIdentifier((String)value);
+      }
+      break;
+
     }
   }
 
@@ -463,6 +564,12 @@ import org.slf4j.LoggerFactory;
     case VALID_WRITE_ID_LIST:
       return getValidWriteIdList();
 
+    case PROCESSOR_CAPABILITIES:
+      return getProcessorCapabilities();
+
+    case PROCESSOR_IDENTIFIER:
+      return getProcessorIdentifier();
+
     }
     throw new IllegalStateException();
   }
@@ -488,6 +595,10 @@ import org.slf4j.LoggerFactory;
       return isSetWriteId();
     case VALID_WRITE_ID_LIST:
       return isSetValidWriteIdList();
+    case PROCESSOR_CAPABILITIES:
+      return isSetProcessorCapabilities();
+    case PROCESSOR_IDENTIFIER:
+      return isSetProcessorIdentifier();
     }
     throw new IllegalStateException();
   }
@@ -568,6 +679,24 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_processorCapabilities = true && this.isSetProcessorCapabilities();
+    boolean that_present_processorCapabilities = true && that.isSetProcessorCapabilities();
+    if (this_present_processorCapabilities || that_present_processorCapabilities) {
+      if (!(this_present_processorCapabilities && that_present_processorCapabilities))
+        return false;
+      if (!this.processorCapabilities.equals(that.processorCapabilities))
+        return false;
+    }
+
+    boolean this_present_processorIdentifier = true && this.isSetProcessorIdentifier();
+    boolean that_present_processorIdentifier = true && that.isSetProcessorIdentifier();
+    if (this_present_processorIdentifier || that_present_processorIdentifier) {
+      if (!(this_present_processorIdentifier && that_present_processorIdentifier))
+        return false;
+      if (!this.processorIdentifier.equals(that.processorIdentifier))
+        return false;
+    }
+
     return true;
   }
 
@@ -610,6 +739,16 @@ import org.slf4j.LoggerFactory;
     if (present_validWriteIdList)
       list.add(validWriteIdList);
 
+    boolean present_processorCapabilities = true && (isSetProcessorCapabilities());
+    list.add(present_processorCapabilities);
+    if (present_processorCapabilities)
+      list.add(processorCapabilities);
+
+    boolean present_processorIdentifier = true && (isSetProcessorIdentifier());
+    list.add(present_processorIdentifier);
+    if (present_processorIdentifier)
+      list.add(processorIdentifier);
+
     return list.hashCode();
   }
 
@@ -691,6 +830,26 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetProcessorCapabilities()).compareTo(other.isSetProcessorCapabilities());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetProcessorCapabilities()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.processorCapabilities, other.processorCapabilities);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetProcessorIdentifier()).compareTo(other.isSetProcessorIdentifier());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetProcessorIdentifier()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.processorIdentifier, other.processorIdentifier);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -770,6 +929,26 @@ import org.slf4j.LoggerFactory;
       }
       first = false;
     }
+    if (isSetProcessorCapabilities()) {
+      if (!first) sb.append(", ");
+      sb.append("processorCapabilities:");
+      if (this.processorCapabilities == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.processorCapabilities);
+      }
+      first = false;
+    }
+    if (isSetProcessorIdentifier()) {
+      if (!first) sb.append(", ");
+      sb.append("processorIdentifier:");
+      if (this.processorIdentifier == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.processorIdentifier);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -891,6 +1070,32 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 8: // PROCESSOR_CAPABILITIES
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list1128 = iprot.readListBegin();
+                struct.processorCapabilities = new ArrayList<String>(_list1128.size);
+                String _elem1129;
+                for (int _i1130 = 0; _i1130 < _list1128.size; ++_i1130)
+                {
+                  _elem1129 = iprot.readString();
+                  struct.processorCapabilities.add(_elem1129);
+                }
+                iprot.readListEnd();
+              }
+              struct.setProcessorCapabilitiesIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 9: // PROCESSOR_IDENTIFIER
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.processorIdentifier = iprot.readString();
+              struct.setProcessorIdentifierIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -945,6 +1150,27 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldEnd();
         }
       }
+      if (struct.processorCapabilities != null) {
+        if (struct.isSetProcessorCapabilities()) {
+          oprot.writeFieldBegin(PROCESSOR_CAPABILITIES_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.processorCapabilities.size()));
+            for (String _iter1131 : struct.processorCapabilities)
+            {
+              oprot.writeString(_iter1131);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.processorIdentifier != null) {
+        if (struct.isSetProcessorIdentifier()) {
+          oprot.writeFieldBegin(PROCESSOR_IDENTIFIER_FIELD_DESC);
+          oprot.writeString(struct.processorIdentifier);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -978,7 +1204,13 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetValidWriteIdList()) {
         optionals.set(3);
       }
-      oprot.writeBitSet(optionals, 4);
+      if (struct.isSetProcessorCapabilities()) {
+        optionals.set(4);
+      }
+      if (struct.isSetProcessorIdentifier()) {
+        optionals.set(5);
+      }
+      oprot.writeBitSet(optionals, 6);
       if (struct.isSetCatName()) {
         oprot.writeString(struct.catName);
       }
@@ -991,6 +1223,18 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetValidWriteIdList()) {
         oprot.writeString(struct.validWriteIdList);
       }
+      if (struct.isSetProcessorCapabilities()) {
+        {
+          oprot.writeI32(struct.processorCapabilities.size());
+          for (String _iter1132 : struct.processorCapabilities)
+          {
+            oprot.writeString(_iter1132);
+          }
+        }
+      }
+      if (struct.isSetProcessorIdentifier()) {
+        oprot.writeString(struct.processorIdentifier);
+      }
     }
 
     @Override
@@ -1003,7 +1247,7 @@ import org.slf4j.LoggerFactory;
       struct.table = new Table();
       struct.table.read(iprot);
       struct.setTableIsSet(true);
-      BitSet incoming = iprot.readBitSet(4);
+      BitSet incoming = iprot.readBitSet(6);
       if (incoming.get(0)) {
         struct.catName = iprot.readString();
         struct.setCatNameIsSet(true);
@@ -1021,6 +1265,23 @@ import org.slf4j.LoggerFactory;
         struct.validWriteIdList = iprot.readString();
         struct.setValidWriteIdListIsSet(true);
       }
+      if (incoming.get(4)) {
+        {
+          org.apache.thrift.protocol.TList _list1133 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.processorCapabilities = new ArrayList<String>(_list1133.size);
+          String _elem1134;
+          for (int _i1135 = 0; _i1135 < _list1133.size; ++_i1135)
+          {
+            _elem1134 = iprot.readString();
+            struct.processorCapabilities.add(_elem1134);
+          }
+        }
+        struct.setProcessorCapabilitiesIsSet(true);
+      }
+      if (incoming.get(5)) {
+        struct.processorIdentifier = iprot.readString();
+        struct.setProcessorIdentifierIsSet(true);
+      }
     }
   }
 
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsFilterSpec.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsFilterSpec.java
index 0441c85..e0520a5 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsFilterSpec.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsFilterSpec.java
@@ -444,13 +444,13 @@ import org.slf4j.LoggerFactory;
           case 8: // FILTERS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list1136 = iprot.readListBegin();
-                struct.filters = new ArrayList<String>(_list1136.size);
-                String _elem1137;
-                for (int _i1138 = 0; _i1138 < _list1136.size; ++_i1138)
+                org.apache.thrift.protocol.TList _list1144 = iprot.readListBegin();
+                struct.filters = new ArrayList<String>(_list1144.size);
+                String _elem1145;
+                for (int _i1146 = 0; _i1146 < _list1144.size; ++_i1146)
                 {
-                  _elem1137 = iprot.readString();
-                  struct.filters.add(_elem1137);
+                  _elem1145 = iprot.readString();
+                  struct.filters.add(_elem1145);
                 }
                 iprot.readListEnd();
               }
@@ -484,9 +484,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(FILTERS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filters.size()));
-            for (String _iter1139 : struct.filters)
+            for (String _iter1147 : struct.filters)
             {
-              oprot.writeString(_iter1139);
+              oprot.writeString(_iter1147);
             }
             oprot.writeListEnd();
           }
@@ -524,9 +524,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetFilters()) {
         {
           oprot.writeI32(struct.filters.size());
-          for (String _iter1140 : struct.filters)
+          for (String _iter1148 : struct.filters)
           {
-            oprot.writeString(_iter1140);
+            oprot.writeString(_iter1148);
           }
         }
       }
@@ -542,13 +542,13 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list1141 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.filters = new ArrayList<String>(_list1141.size);
-          String _elem1142;
-          for (int _i1143 = 0; _i1143 < _list1141.size; ++_i1143)
+          org.apache.thrift.protocol.TList _list1149 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.filters = new ArrayList<String>(_list1149.size);
+          String _elem1150;
+          for (int _i1151 = 0; _i1151 < _list1149.size; ++_i1151)
           {
-            _elem1142 = iprot.readString();
-            struct.filters.add(_elem1142);
+            _elem1150 = iprot.readString();
+            struct.filters.add(_elem1150);
           }
         }
         struct.setFiltersIsSet(true);
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsProjectionSpec.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsProjectionSpec.java
index 733a285..cc5076c 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsProjectionSpec.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsProjectionSpec.java
@@ -509,13 +509,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FIELD_LIST
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list1128 = iprot.readListBegin();
-                struct.fieldList = new ArrayList<String>(_list1128.size);
-                String _elem1129;
-                for (int _i1130 = 0; _i1130 < _list1128.size; ++_i1130)
+                org.apache.thrift.protocol.TList _list1136 = iprot.readListBegin();
+                struct.fieldList = new ArrayList<String>(_list1136.size);
+                String _elem1137;
+                for (int _i1138 = 0; _i1138 < _list1136.size; ++_i1138)
                 {
-                  _elem1129 = iprot.readString();
-                  struct.fieldList.add(_elem1129);
+                  _elem1137 = iprot.readString();
+                  struct.fieldList.add(_elem1137);
                 }
                 iprot.readListEnd();
               }
@@ -557,9 +557,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FIELD_LIST_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.fieldList.size()));
-          for (String _iter1131 : struct.fieldList)
+          for (String _iter1139 : struct.fieldList)
           {
-            oprot.writeString(_iter1131);
+            oprot.writeString(_iter1139);
           }
           oprot.writeListEnd();
         }
@@ -606,9 +606,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetFieldList()) {
         {
           oprot.writeI32(struct.fieldList.size());
-          for (String _iter1132 : struct.fieldList)
+          for (String _iter1140 : struct.fieldList)
           {
-            oprot.writeString(_iter1132);
+            oprot.writeString(_iter1140);
           }
         }
       }
@@ -626,13 +626,13 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list1133 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.fieldList = new ArrayList<String>(_list1133.size);
-          String _elem1134;
-          for (int _i1135 = 0; _i1135 < _list1133.size; ++_i1135)
+          org.apache.thrift.protocol.TList _list1141 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.fieldList = new ArrayList<String>(_list1141.size);
+          String _elem1142;
+          for (int _i1143 = 0; _i1143 < _list1141.size; ++_i1143)
           {
-            _elem1134 = iprot.readString();
-            struct.fieldList.add(_elem1134);
+            _elem1142 = iprot.readString();
+            struct.fieldList.add(_elem1142);
           }
         }
         struct.setFieldListIsSet(true);
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsRequest.java
index e40469e..0afb4a1 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsRequest.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsRequest.java
@@ -1139,13 +1139,13 @@ import org.slf4j.LoggerFactory;
           case 6: // GROUP_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list1152 = iprot.readListBegin();
-                struct.groupNames = new ArrayList<String>(_list1152.size);
-                String _elem1153;
-                for (int _i1154 = 0; _i1154 < _list1152.size; ++_i1154)
+                org.apache.thrift.protocol.TList _list1160 = iprot.readListBegin();
+                struct.groupNames = new ArrayList<String>(_list1160.size);
+                String _elem1161;
+                for (int _i1162 = 0; _i1162 < _list1160.size; ++_i1162)
                 {
-                  _elem1153 = iprot.readString();
-                  struct.groupNames.add(_elem1153);
+                  _elem1161 = iprot.readString();
+                  struct.groupNames.add(_elem1161);
                 }
                 iprot.readListEnd();
               }
@@ -1175,13 +1175,13 @@ import org.slf4j.LoggerFactory;
           case 9: // PROCESSOR_CAPABILITIES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list1155 = iprot.readListBegin();
-                struct.processorCapabilities = new ArrayList<String>(_list1155.size);
-                String _elem1156;
-                for (int _i1157 = 0; _i1157 < _list1155.size; ++_i1157)
+                org.apache.thrift.protocol.TList _list1163 = iprot.readListBegin();
+                struct.processorCapabilities = new ArrayList<String>(_list1163.size);
+                String _elem1164;
+                for (int _i1165 = 0; _i1165 < _list1163.size; ++_i1165)
                 {
-                  _elem1156 = iprot.readString();
-                  struct.processorCapabilities.add(_elem1156);
+                  _elem1164 = iprot.readString();
+                  struct.processorCapabilities.add(_elem1164);
                 }
                 iprot.readListEnd();
               }
@@ -1245,9 +1245,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.groupNames.size()));
-            for (String _iter1158 : struct.groupNames)
+            for (String _iter1166 : struct.groupNames)
             {
-              oprot.writeString(_iter1158);
+              oprot.writeString(_iter1166);
             }
             oprot.writeListEnd();
           }
@@ -1269,9 +1269,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PROCESSOR_CAPABILITIES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.processorCapabilities.size()));
-            for (String _iter1159 : struct.processorCapabilities)
+            for (String _iter1167 : struct.processorCapabilities)
             {
-              oprot.writeString(_iter1159);
+              oprot.writeString(_iter1167);
             }
             oprot.writeListEnd();
           }
@@ -1352,9 +1352,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetGroupNames()) {
         {
           oprot.writeI32(struct.groupNames.size());
-          for (String _iter1160 : struct.groupNames)
+          for (String _iter1168 : struct.groupNames)
           {
-            oprot.writeString(_iter1160);
+            oprot.writeString(_iter1168);
           }
         }
       }
@@ -1367,9 +1367,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetProcessorCapabilities()) {
         {
           oprot.writeI32(struct.processorCapabilities.size());
-          for (String _iter1161 : struct.processorCapabilities)
+          for (String _iter1169 : struct.processorCapabilities)
           {
-            oprot.writeString(_iter1161);
+            oprot.writeString(_iter1169);
           }
         }
       }
@@ -1404,13 +1404,13 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(5)) {
         {
-          org.apache.thrift.protocol.TList _list1162 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.groupNames = new ArrayList<String>(_list1162.size);
-          String _elem1163;
-          for (int _i1164 = 0; _i1164 < _list1162.size; ++_i1164)
+          org.apache.thrift.protocol.TList _list1170 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.groupNames = new ArrayList<String>(_list1170.size);
+          String _elem1171;
+          for (int _i1172 = 0; _i1172 < _list1170.size; ++_i1172)
           {
-            _elem1163 = iprot.readString();
-            struct.groupNames.add(_elem1163);
+            _elem1171 = iprot.readString();
+            struct.groupNames.add(_elem1171);
           }
         }
         struct.setGroupNamesIsSet(true);
@@ -1427,13 +1427,13 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(8)) {
         {
-          org.apache.thrift.protocol.TList _list1165 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.processorCapabilities = new ArrayList<String>(_list1165.size);
-          String _elem1166;
-          for (int _i1167 = 0; _i1167 < _list1165.size; ++_i1167)
+          org.apache.thrift.protocol.TList _list1173 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.processorCapabilities = new ArrayList<String>(_list1173.size);
+          String _elem1174;
+          for (int _i1175 = 0; _i1175 < _list1173.size; ++_i1175)
           {
-            _elem1166 = iprot.readString();
-            struct.processorCapabilities.add(_elem1166);
+            _elem1174 = iprot.readString();
+            struct.processorCapabilities.add(_elem1174);
           }
         }
         struct.setProcessorCapabilitiesIsSet(true);
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsResponse.java
index 423b827..839e7af 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsResponse.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsResponse.java
@@ -350,14 +350,14 @@ import org.slf4j.LoggerFactory;
           case 1: // PARTITION_SPEC
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list1144 = iprot.readListBegin();
-                struct.partitionSpec = new ArrayList<PartitionSpec>(_list1144.size);
-                PartitionSpec _elem1145;
-                for (int _i1146 = 0; _i1146 < _list1144.size; ++_i1146)
+                org.apache.thrift.protocol.TList _list1152 = iprot.readListBegin();
+                struct.partitionSpec = new ArrayList<PartitionSpec>(_list1152.size);
+                PartitionSpec _elem1153;
+                for (int _i1154 = 0; _i1154 < _list1152.size; ++_i1154)
                 {
-                  _elem1145 = new PartitionSpec();
-                  _elem1145.read(iprot);
-                  struct.partitionSpec.add(_elem1145);
+                  _elem1153 = new PartitionSpec();
+                  _elem1153.read(iprot);
+                  struct.partitionSpec.add(_elem1153);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARTITION_SPEC_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitionSpec.size()));
-          for (PartitionSpec _iter1147 : struct.partitionSpec)
+          for (PartitionSpec _iter1155 : struct.partitionSpec)
           {
-            _iter1147.write(oprot);
+            _iter1155.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -416,9 +416,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitionSpec()) {
         {
           oprot.writeI32(struct.partitionSpec.size());
-          for (PartitionSpec _iter1148 : struct.partitionSpec)
+          for (PartitionSpec _iter1156 : struct.partitionSpec)
           {
-            _iter1148.write(oprot);
+            _iter1156.write(oprot);
           }
         }
       }
@@ -430,14 +430,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list1149 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.partitionSpec = new ArrayList<PartitionSpec>(_list1149.size);
-          PartitionSpec _elem1150;
-          for (int _i1151 = 0; _i1151 < _list1149.size; ++_i1151)
+          org.apache.thrift.protocol.TList _list1157 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.partitionSpec = new ArrayList<PartitionSpec>(_list1157.size);
+          PartitionSpec _elem1158;
+          for (int _i1159 = 0; _i1159 < _list1157.size; ++_i1159)
           {
-            _elem1150 = new PartitionSpec();
-            _elem1150.read(iprot);
-            struct.partitionSpec.add(_elem1150);
+            _elem1158 = new PartitionSpec();
+            _elem1158.read(iprot);
+            struct.partitionSpec.add(_elem1158);
           }
         }
         struct.setPartitionSpecIsSet(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 10fe5d4..02c9f01 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
@@ -46166,13 +46166,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1168 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1168.size);
-                  String _elem1169;
-                  for (int _i1170 = 0; _i1170 < _list1168.size; ++_i1170)
+                  org.apache.thrift.protocol.TList _list1176 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1176.size);
+                  String _elem1177;
+                  for (int _i1178 = 0; _i1178 < _list1176.size; ++_i1178)
                   {
-                    _elem1169 = iprot.readString();
-                    struct.success.add(_elem1169);
+                    _elem1177 = iprot.readString();
+                    struct.success.add(_elem1177);
                   }
                   iprot.readListEnd();
                 }
@@ -46207,9 +46207,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 _iter1171 : struct.success)
+            for (String _iter1179 : struct.success)
             {
-              oprot.writeString(_iter1171);
+              oprot.writeString(_iter1179);
             }
             oprot.writeListEnd();
           }
@@ -46248,9 +46248,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1172 : struct.success)
+            for (String _iter1180 : struct.success)
             {
-              oprot.writeString(_iter1172);
+              oprot.writeString(_iter1180);
             }
           }
         }
@@ -46265,13 +46265,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1173 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1173.size);
-            String _elem1174;
-            for (int _i1175 = 0; _i1175 < _list1173.size; ++_i1175)
+            org.apache.thrift.protocol.TList _list1181 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1181.size);
+            String _elem1182;
+            for (int _i1183 = 0; _i1183 < _list1181.size; ++_i1183)
             {
-              _elem1174 = iprot.readString();
-              struct.success.add(_elem1174);
+              _elem1182 = iprot.readString();
+              struct.success.add(_elem1182);
             }
           }
           struct.setSuccessIsSet(true);
@@ -46925,13 +46925,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1176 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1176.size);
-                  String _elem1177;
-                  for (int _i1178 = 0; _i1178 < _list1176.size; ++_i1178)
+                  org.apache.thrift.protocol.TList _list1184 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1184.size);
+                  String _elem1185;
+                  for (int _i1186 = 0; _i1186 < _list1184.size; ++_i1186)
                   {
-                    _elem1177 = iprot.readString();
-                    struct.success.add(_elem1177);
+                    _elem1185 = iprot.readString();
+                    struct.success.add(_elem1185);
                   }
                   iprot.readListEnd();
                 }
@@ -46966,9 +46966,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 _iter1179 : struct.success)
+            for (String _iter1187 : struct.success)
             {
-              oprot.writeString(_iter1179);
+              oprot.writeString(_iter1187);
             }
             oprot.writeListEnd();
           }
@@ -47007,9 +47007,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1180 : struct.success)
+            for (String _iter1188 : struct.success)
             {
-              oprot.writeString(_iter1180);
+              oprot.writeString(_iter1188);
             }
           }
         }
@@ -47024,13 +47024,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1181 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1181.size);
-            String _elem1182;
-            for (int _i1183 = 0; _i1183 < _list1181.size; ++_i1183)
+            org.apache.thrift.protocol.TList _list1189 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1189.size);
+            String _elem1190;
+            for (int _i1191 = 0; _i1191 < _list1189.size; ++_i1191)
             {
-              _elem1182 = iprot.readString();
-              struct.success.add(_elem1182);
+              _elem1190 = iprot.readString();
+              struct.success.add(_elem1190);
             }
           }
           struct.setSuccessIsSet(true);
@@ -51637,16 +51637,16 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map1184 = iprot.readMapBegin();
-                  struct.success = new HashMap<String,Type>(2*_map1184.size);
-                  String _key1185;
-                  Type _val1186;
-                  for (int _i1187 = 0; _i1187 < _map1184.size; ++_i1187)
+                  org.apache.thrift.protocol.TMap _map1192 = iprot.readMapBegin();
+                  struct.success = new HashMap<String,Type>(2*_map1192.size);
+                  String _key1193;
+                  Type _val1194;
+                  for (int _i1195 = 0; _i1195 < _map1192.size; ++_i1195)
                   {
-                    _key1185 = iprot.readString();
-                    _val1186 = new Type();
-                    _val1186.read(iprot);
-                    struct.success.put(_key1185, _val1186);
+                    _key1193 = iprot.readString();
+                    _val1194 = new Type();
+                    _val1194.read(iprot);
+                    struct.success.put(_key1193, _val1194);
                   }
                   iprot.readMapEnd();
                 }
@@ -51681,10 +51681,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> _iter1188 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter1196 : struct.success.entrySet())
             {
-              oprot.writeString(_iter1188.getKey());
-              _iter1188.getValue().write(oprot);
+              oprot.writeString(_iter1196.getKey());
+              _iter1196.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -51723,10 +51723,10 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Map.Entry<String, Type> _iter1189 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter1197 : struct.success.entrySet())
             {
-              oprot.writeString(_iter1189.getKey());
-              _iter1189.getValue().write(oprot);
+              oprot.writeString(_iter1197.getKey());
+              _iter1197.getValue().write(oprot);
             }
           }
         }
@@ -51741,16 +51741,16 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map1190 = 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*_map1190.size);
-            String _key1191;
-            Type _val1192;
-            for (int _i1193 = 0; _i1193 < _map1190.size; ++_i1193)
+            org.apache.thrift.protocol.TMap _map1198 = 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*_map1198.size);
+            String _key1199;
+            Type _val1200;
+            for (int _i1201 = 0; _i1201 < _map1198.size; ++_i1201)
             {
-              _key1191 = iprot.readString();
-              _val1192 = new Type();
-              _val1192.read(iprot);
-              struct.success.put(_key1191, _val1192);
+              _key1199 = iprot.readString();
+              _val1200 = new Type();
+              _val1200.read(iprot);
+              struct.success.put(_key1199, _val1200);
             }
           }
           struct.setSuccessIsSet(true);
@@ -52785,14 +52785,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1194 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list1194.size);
-                  FieldSchema _elem1195;
-                  for (int _i1196 = 0; _i1196 < _list1194.size; ++_i1196)
+                  org.apache.thrift.protocol.TList _list1202 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list1202.size);
+                  FieldSchema _elem1203;
+                  for (int _i1204 = 0; _i1204 < _list1202.size; ++_i1204)
                   {
-                    _elem1195 = new FieldSchema();
-                    _elem1195.read(iprot);
-                    struct.success.add(_elem1195);
+                    _elem1203 = new FieldSchema();
+                    _elem1203.read(iprot);
+                    struct.success.add(_elem1203);
                   }
                   iprot.readListEnd();
                 }
@@ -52845,9 +52845,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 _iter1197 : struct.success)
+            for (FieldSchema _iter1205 : struct.success)
             {
-              _iter1197.write(oprot);
+              _iter1205.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -52902,9 +52902,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter1198 : struct.success)
+            for (FieldSchema _iter1206 : struct.success)
             {
-              _iter1198.write(oprot);
+              _iter1206.write(oprot);
             }
           }
         }
@@ -52925,14 +52925,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1199 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list1199.size);
-            FieldSchema _elem1200;
-            for (int _i1201 = 0; _i1201 < _list1199.size; ++_i1201)
+            org.apache.thrift.protocol.TList _list1207 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list1207.size);
+            FieldSchema _elem1208;
+            for (int _i1209 = 0; _i1209 < _list1207.size; ++_i1209)
             {
-              _elem1200 = new FieldSchema();
-              _elem1200.read(iprot);
-              struct.success.add(_elem1200);
+              _elem1208 = new FieldSchema();
+              _elem1208.read(iprot);
+              struct.success.add(_elem1208);
             }
           }
           struct.setSuccessIsSet(true);
@@ -54086,14 +54086,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1202 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list1202.size);
-                  FieldSchema _elem1203;
-                  for (int _i1204 = 0; _i1204 < _list1202.size; ++_i1204)
+                  org.apache.thrift.protocol.TList _list1210 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list1210.size);
+                  FieldSchema _elem1211;
+                  for (int _i1212 = 0; _i1212 < _list1210.size; ++_i1212)
                   {
-                    _elem1203 = new FieldSchema();
-                    _elem1203.read(iprot);
-                    struct.success.add(_elem1203);
+                    _elem1211 = new FieldSchema();
+                    _elem1211.read(iprot);
+                    struct.success.add(_elem1211);
                   }
                   iprot.readListEnd();
                 }
@@ -54146,9 +54146,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 _iter1205 : struct.success)
+            for (FieldSchema _iter1213 : struct.success)
             {
-              _iter1205.write(oprot);
+              _iter1213.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -54203,9 +54203,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter1206 : struct.success)
+            for (FieldSchema _iter1214 : struct.success)
             {
-              _iter1206.write(oprot);
+              _iter1214.write(oprot);
             }
           }
         }
@@ -54226,14 +54226,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1207 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list1207.size);
-            FieldSchema _elem1208;
-            for (int _i1209 = 0; _i1209 < _list1207.size; ++_i1209)
+            org.apache.thrift.protocol.TList _list1215 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list1215.size);
+            FieldSchema _elem1216;
+            for (int _i1217 = 0; _i1217 < _list1215.size; ++_i1217)
             {
-              _elem1208 = new FieldSchema();
-              _elem1208.read(iprot);
-              struct.success.add(_elem1208);
+              _elem1216 = new FieldSchema();
+              _elem1216.read(iprot);
+              struct.success.add(_elem1216);
             }
           }
           struct.setSuccessIsSet(true);
@@ -55278,14 +55278,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1210 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list1210.size);
-                  FieldSchema _elem1211;
-                  for (int _i1212 = 0; _i1212 < _list1210.size; ++_i1212)
+                  org.apache.thrift.protocol.TList _list1218 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list1218.size);
+                  FieldSchema _elem1219;
+                  for (int _i1220 = 0; _i1220 < _list1218.size; ++_i1220)
                   {
-                    _elem1211 = new FieldSchema();
-                    _elem1211.read(iprot);
-                    struct.success.add(_elem1211);
+                    _elem1219 = new FieldSchema();
+                    _elem1219.read(iprot);
+                    struct.success.add(_elem1219);
                   }
                   iprot.readListEnd();
                 }
@@ -55338,9 +55338,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 _iter1213 : struct.success)
+            for (FieldSchema _iter1221 : struct.success)
             {
-              _iter1213.write(oprot);
+              _iter1221.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -55395,9 +55395,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter1214 : struct.success)
+            for (FieldSchema _iter1222 : struct.success)
             {
-              _iter1214.write(oprot);
+              _iter1222.write(oprot);
             }
           }
         }
@@ -55418,14 +55418,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1215 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list1215.size);
-            FieldSchema _elem1216;
-            for (int _i1217 = 0; _i1217 < _list1215.size; ++_i1217)
+            org.apache.thrift.protocol.TList _list1223 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list1223.size);
+            FieldSchema _elem1224;
+            for (int _i1225 = 0; _i1225 < _list1223.size; ++_i1225)
             {
-              _elem1216 = new FieldSchema();
-              _elem1216.read(iprot);
-              struct.success.add(_elem1216);
+              _elem1224 = new FieldSchema();
+              _elem1224.read(iprot);
+              struct.success.add(_elem1224);
             }
           }
           struct.setSuccessIsSet(true);
@@ -56579,14 +56579,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1218 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list1218.size);
-                  FieldSchema _elem1219;
-                  for (int _i1220 = 0; _i1220 < _list1218.size; ++_i1220)
+                  org.apache.thrift.protocol.TList _list1226 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list1226.size);
+                  FieldSchema _elem1227;
+                  for (int _i1228 = 0; _i1228 < _list1226.size; ++_i1228)
                   {
-                    _elem1219 = new FieldSchema();
-                    _elem1219.read(iprot);
-                    struct.success.add(_elem1219);
+                    _elem1227 = new FieldSchema();
+                    _elem1227.read(iprot);
+                    struct.success.add(_elem1227);
                   }
                   iprot.readListEnd();
                 }
@@ -56639,9 +56639,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 _iter1221 : struct.success)
+            for (FieldSchema _iter1229 : struct.success)
             {
-              _iter1221.write(oprot);
+              _iter1229.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -56696,9 +56696,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter1222 : struct.success)
+            for (FieldSchema _iter1230 : struct.success)
             {
-              _iter1222.write(oprot);
+              _iter1230.write(oprot);
             }
           }
         }
@@ -56719,14 +56719,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1223 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list1223.size);
-            FieldSchema _elem1224;
-            for (int _i1225 = 0; _i1225 < _list1223.size; ++_i1225)
+            org.apache.thrift.protocol.TList _list1231 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list1231.size);
+            FieldSchema _elem1232;
+            for (int _i1233 = 0; _i1233 < _list1231.size; ++_i1233)
             {
-              _elem1224 = new FieldSchema();
-              _elem1224.read(iprot);
-              struct.success.add(_elem1224);
+              _elem1232 = new FieldSchema();
+              _elem1232.read(iprot);
+              struct.success.add(_elem1232);
             }
           }
           struct.setSuccessIsSet(true);
@@ -59855,14 +59855,14 @@ import org.slf4j.LoggerFactory;
             case 2: // PRIMARY_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1226 = iprot.readListBegin();
-                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1226.size);
-                  SQLPrimaryKey _elem1227;
-                  for (int _i1228 = 0; _i1228 < _list1226.size; ++_i1228)
+                  org.apache.thrift.protocol.TList _list1234 = iprot.readListBegin();
+                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1234.size);
+                  SQLPrimaryKey _elem1235;
+                  for (int _i1236 = 0; _i1236 < _list1234.size; ++_i1236)
                   {
-                    _elem1227 = new SQLPrimaryKey();
-                    _elem1227.read(iprot);
-                    struct.primaryKeys.add(_elem1227);
+                    _elem1235 = new SQLPrimaryKey();
+                    _elem1235.read(iprot);
+                    struct.primaryKeys.add(_elem1235);
                   }
                   iprot.readListEnd();
                 }
@@ -59874,14 +59874,14 @@ import org.slf4j.LoggerFactory;
             case 3: // FOREIGN_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1229 = iprot.readListBegin();
-                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1229.size);
-                  SQLForeignKey _elem1230;
-                  for (int _i1231 = 0; _i1231 < _list1229.size; ++_i1231)
+                  org.apache.thrift.protocol.TList _list1237 = iprot.readListBegin();
+                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1237.size);
+                  SQLForeignKey _elem1238;
+                  for (int _i1239 = 0; _i1239 < _list1237.size; ++_i1239)
                   {
-                    _elem1230 = new SQLForeignKey();
-                    _elem1230.read(iprot);
-                    struct.foreignKeys.add(_elem1230);
+                    _elem1238 = new SQLForeignKey();
+                    _elem1238.read(iprot);
+                    struct.foreignKeys.add(_elem1238);
                   }
                   iprot.readListEnd();
                 }
@@ -59893,14 +59893,14 @@ import org.slf4j.LoggerFactory;
             case 4: // UNIQUE_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1232 = iprot.readListBegin();
-                  struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1232.size);
-                  SQLUniqueConstraint _elem1233;
-                  for (int _i1234 = 0; _i1234 < _list1232.size; ++_i1234)
+                  org.apache.thrift.protocol.TList _list1240 = iprot.readListBegin();
+                  struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1240.size);
+                  SQLUniqueConstraint _elem1241;
+                  for (int _i1242 = 0; _i1242 < _list1240.size; ++_i1242)
                   {
-                    _elem1233 = new SQLUniqueConstraint();
-                    _elem1233.read(iprot);
-                    struct.uniqueConstraints.add(_elem1233);
+                    _elem1241 = new SQLUniqueConstraint();
+                    _elem1241.read(iprot);
+                    struct.uniqueConstraints.add(_elem1241);
                   }
                   iprot.readListEnd();
                 }
@@ -59912,14 +59912,14 @@ import org.slf4j.LoggerFactory;
             case 5: // NOT_NULL_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1235 = iprot.readListBegin();
-                  struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1235.size);
-                  SQLNotNullConstraint _elem1236;
-                  for (int _i1237 = 0; _i1237 < _list1235.size; ++_i1237)
+                  org.apache.thrift.protocol.TList _list1243 = iprot.readListBegin();
+                  struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1243.size);
+                  SQLNotNullConstraint _elem1244;
+                  for (int _i1245 = 0; _i1245 < _list1243.size; ++_i1245)
                   {
-                    _elem1236 = new SQLNotNullConstraint();
-                    _elem1236.read(iprot);
-                    struct.notNullConstraints.add(_elem1236);
+                    _elem1244 = new SQLNotNullConstraint();
+                    _elem1244.read(iprot);
+                    struct.notNullConstraints.add(_elem1244);
                   }
                   iprot.readListEnd();
                 }
@@ -59931,14 +59931,14 @@ import org.slf4j.LoggerFactory;
             case 6: // DEFAULT_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1238 = iprot.readListBegin();
-                  struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1238.size);
-                  SQLDefaultConstraint _elem1239;
-                  for (int _i1240 = 0; _i1240 < _list1238.size; ++_i1240)
+                  org.apache.thrift.protocol.TList _list1246 = iprot.readListBegin();
+                  struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1246.size);
+                  SQLDefaultConstraint _elem1247;
+                  for (int _i1248 = 0; _i1248 < _list1246.size; ++_i1248)
                   {
-                    _elem1239 = new SQLDefaultConstraint();
-                    _elem1239.read(iprot);
-                    struct.defaultConstraints.add(_elem1239);
+                    _elem1247 = new SQLDefaultConstraint();
+                    _elem1247.read(iprot);
+                    struct.defaultConstraints.add(_elem1247);
                   }
                   iprot.readListEnd();
                 }
@@ -59950,14 +59950,14 @@ import org.slf4j.LoggerFactory;
             case 7: // CHECK_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1241 = iprot.readListBegin();
-                  struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1241.size);
-                  SQLCheckConstraint _elem1242;
-                  for (int _i1243 = 0; _i1243 < _list1241.size; ++_i1243)
+                  org.apache.thrift.protocol.TList _list1249 = iprot.readListBegin();
+                  struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1249.size);
+                  SQLCheckConstraint _elem1250;
+                  for (int _i1251 = 0; _i1251 < _list1249.size; ++_i1251)
                   {
-                    _elem1242 = new SQLCheckConstraint();
-                    _elem1242.read(iprot);
-                    struct.checkConstraints.add(_elem1242);
+                    _elem1250 = new SQLCheckConstraint();
+                    _elem1250.read(iprot);
+                    struct.checkConstraints.add(_elem1250);
                   }
                   iprot.readListEnd();
                 }
@@ -59988,9 +59988,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 _iter1244 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter1252 : struct.primaryKeys)
             {
-              _iter1244.write(oprot);
+              _iter1252.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -60000,9 +60000,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 _iter1245 : struct.foreignKeys)
+            for (SQLForeignKey _iter1253 : struct.foreignKeys)
             {
-              _iter1245.write(oprot);
+              _iter1253.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -60012,9 +60012,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 _iter1246 : struct.uniqueConstraints)
+            for (SQLUniqueConstraint _iter1254 : struct.uniqueConstraints)
             {
-              _iter1246.write(oprot);
+              _iter1254.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -60024,9 +60024,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 _iter1247 : struct.notNullConstraints)
+            for (SQLNotNullConstraint _iter1255 : struct.notNullConstraints)
             {
-              _iter1247.write(oprot);
+              _iter1255.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -60036,9 +60036,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 _iter1248 : struct.defaultConstraints)
+            for (SQLDefaultConstraint _iter1256 : struct.defaultConstraints)
             {
-              _iter1248.write(oprot);
+              _iter1256.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -60048,9 +60048,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 _iter1249 : struct.checkConstraints)
+            for (SQLCheckConstraint _iter1257 : struct.checkConstraints)
             {
-              _iter1249.write(oprot);
+              _iter1257.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -60102,54 +60102,54 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPrimaryKeys()) {
           {
             oprot.writeI32(struct.primaryKeys.size());
-            for (SQLPrimaryKey _iter1250 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter1258 : struct.primaryKeys)
             {
-              _iter1250.write(oprot);
+              _iter1258.write(oprot);
             }
           }
         }
         if (struct.isSetForeignKeys()) {
           {
             oprot.writeI32(struct.foreignKeys.size());
-            for (SQLForeignKey _iter1251 : struct.foreignKeys)
+            for (SQLForeignKey _iter1259 : struct.foreignKeys)
             {
-              _iter1251.write(oprot);
+              _iter1259.write(oprot);
             }
           }
         }
         if (struct.isSetUniqueConstraints()) {
           {
             oprot.writeI32(struct.uniqueConstraints.size());
-            for (SQLUniqueConstraint _iter1252 : struct.uniqueConstraints)
+            for (SQLUniqueConstraint _iter1260 : struct.uniqueConstraints)
             {
-              _iter1252.write(oprot);
+              _iter1260.write(oprot);
             }
           }
         }
         if (struct.isSetNotNullConstraints()) {
           {
             oprot.writeI32(struct.notNullConstraints.size());
-            for (SQLNotNullConstraint _iter1253 : struct.notNullConstraints)
+            for (SQLNotNullConstraint _iter1261 : struct.notNullConstraints)
             {
-              _iter1253.write(oprot);
+              _iter1261.write(oprot);
             }
           }
         }
         if (struct.isSetDefaultConstraints()) {
           {
             oprot.writeI32(struct.defaultConstraints.size());
-            for (SQLDefaultConstraint _iter1254 : struct.defaultConstraints)
+            for (SQLDefaultConstraint _iter1262 : struct.defaultConstraints)
             {
-              _iter1254.write(oprot);
+              _iter1262.write(oprot);
             }
           }
         }
         if (struct.isSetCheckConstraints()) {
           {
             oprot.writeI32(struct.checkConstraints.size());
-            for (SQLCheckConstraint _iter1255 : struct.checkConstraints)
+            for (SQLCheckConstraint _iter1263 : struct.checkConstraints)
             {
-              _iter1255.write(oprot);
+              _iter1263.write(oprot);
             }
           }
         }
@@ -60166,84 +60166,84 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list1256 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1256.size);
-            SQLPrimaryKey _elem1257;
-            for (int _i1258 = 0; _i1258 < _list1256.size; ++_i1258)
+            org.apache.thrift.protocol.TList _list1264 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1264.size);
+            SQLPrimaryKey _elem1265;
+            for (int _i1266 = 0; _i1266 < _list1264.size; ++_i1266)
             {
-              _elem1257 = new SQLPrimaryKey();
-              _elem1257.read(iprot);
-              struct.primaryKeys.add(_elem1257);
+              _elem1265 = new SQLPrimaryKey();
+              _elem1265.read(iprot);
+              struct.primaryKeys.add(_elem1265);
             }
           }
           struct.setPrimaryKeysIsSet(true);
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1259 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1259.size);
-            SQLForeignKey _elem1260;
-            for (int _i1261 = 0; _i1261 < _list1259.size; ++_i1261)
+            org.apache.thrift.protocol.TList _list1267 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1267.size);
+            SQLForeignKey _elem1268;
+            for (int _i1269 = 0; _i1269 < _list1267.size; ++_i1269)
             {
-              _elem1260 = new SQLForeignKey();
-              _elem1260.read(iprot);
-              struct.foreignKeys.add(_elem1260);
+              _elem1268 = new SQLForeignKey();
+              _elem1268.read(iprot);
+              struct.foreignKeys.add(_elem1268);
             }
           }
           struct.setForeignKeysIsSet(true);
         }
         if (incoming.get(3)) {
           {
-            org.apache.thrift.protocol.TList _list1262 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1262.size);
-            SQLUniqueConstraint _elem1263;
-            for (int _i1264 = 0; _i1264 < _list1262.size; ++_i1264)
+            org.apache.thrift.protocol.TList _list1270 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1270.size);
+            SQLUniqueConstraint _elem1271;
+            for (int _i1272 = 0; _i1272 < _list1270.size; ++_i1272)
             {
-              _elem1263 = new SQLUniqueConstraint();
-              _elem1263.read(iprot);
-              struct.uniqueConstraints.add(_elem1263);
+              _elem1271 = new SQLUniqueConstraint();
+              _elem1271.read(iprot);
+              struct.uniqueConstraints.add(_elem1271);
             }
           }
           struct.setUniqueConstraintsIsSet(true);
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol.TList _list1265 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1265.size);
-            SQLNotNullConstraint _elem1266;
-            for (int _i1267 = 0; _i1267 < _list1265.size; ++_i1267)
+            org.apache.thrift.protocol.TList _list1273 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1273.size);
+            SQLNotNullConstraint _elem1274;
+            for (int _i1275 = 0; _i1275 < _list1273.size; ++_i1275)
             {
-              _elem1266 = new SQLNotNullConstraint();
-              _elem1266.read(iprot);
-              struct.notNullConstraints.add(_elem1266);
+              _elem1274 = new SQLNotNullConstraint();
+              _elem1274.read(iprot);
+              struct.notNullConstraints.add(_elem1274);
             }
           }
           struct.setNotNullConstraintsIsSet(true);
         }
         if (incoming.get(5)) {
           {
-            org.apache.thrift.protocol.TList _list1268 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1268.size);
-            SQLDefaultConstraint _elem1269;
-            for (int _i1270 = 0; _i1270 < _list1268.size; ++_i1270)
+            org.apache.thrift.protocol.TList _list1276 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1276.size);
+            SQLDefaultConstraint _elem1277;
+            for (int _i1278 = 0; _i1278 < _list1276.size; ++_i1278)
             {
-              _elem1269 = new SQLDefaultConstraint();
-              _elem1269.read(iprot);
-              struct.defaultConstraints.add(_elem1269);
+              _elem1277 = new SQLDefaultConstraint();
+              _elem1277.read(iprot);
+              struct.defaultConstraints.add(_elem1277);
             }
           }
           struct.setDefaultConstraintsIsSet(true);
         }
         if (incoming.get(6)) {
           {
-            org.apache.thrift.protocol.TList _list1271 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1271.size);
-            SQLCheckConstraint _elem1272;
-            for (int _i1273 = 0; _i1273 < _list1271.size; ++_i1273)
+            org.apache.thrift.protocol.TList _list1279 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1279.size);
+            SQLCheckConstraint _elem1280;
+            for (int _i1281 = 0; _i1281 < _list1279.size; ++_i1281)
             {
-              _elem1272 = new SQLCheckConstraint();
-              _elem1272.read(iprot);
-              struct.checkConstraints.add(_elem1272);
+              _elem1280 = new SQLCheckConstraint();
+              _elem1280.read(iprot);
+              struct.checkConstraints.add(_elem1280);
             }
           }
           struct.setCheckConstraintsIsSet(true);
@@ -70434,13 +70434,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1274 = iprot.readListBegin();
-                  struct.partNames = new ArrayList<String>(_list1274.size);
-                  String _elem1275;
-                  for (int _i1276 = 0; _i1276 < _list1274.size; ++_i1276)
+                  org.apache.thrift.protocol.TList _list1282 = iprot.readListBegin();
+                  struct.partNames = new ArrayList<String>(_list1282.size);
+                  String _elem1283;
+                  for (int _i1284 = 0; _i1284 < _list1282.size; ++_i1284)
                   {
-                    _elem1275 = iprot.readString();
-                    struct.partNames.add(_elem1275);
+                    _elem1283 = iprot.readString();
+                    struct.partNames.add(_elem1283);
                   }
                   iprot.readListEnd();
                 }
@@ -70476,9 +70476,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 _iter1277 : struct.partNames)
+            for (String _iter1285 : struct.partNames)
             {
-              oprot.writeString(_iter1277);
+              oprot.writeString(_iter1285);
             }
             oprot.writeListEnd();
           }
@@ -70521,9 +70521,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPartNames()) {
           {
             oprot.writeI32(struct.partNames.size());
-            for (String _iter1278 : struct.partNames)
+            for (String _iter1286 : struct.partNames)
             {
-              oprot.writeString(_iter1278);
+              oprot.writeString(_iter1286);
             }
           }
         }
@@ -70543,13 +70543,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1279 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.partNames = new ArrayList<String>(_list1279.size);
-            String _elem1280;
-            for (int _i1281 = 0; _i1281 < _list1279.size; ++_i1281)
+            org.apache.thrift.protocol.TList _list1287 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.partNames = new ArrayList<String>(_list1287.size);
+            String _elem1288;
+            for (int _i1289 = 0; _i1289 < _list1287.size; ++_i1289)
             {
-              _elem1280 = iprot.readString();
-              struct.partNames.add(_elem1280);
+              _elem1288 = iprot.readString();
+              struct.partNames.add(_elem1288);
             }
           }
           struct.setPartNamesIsSet(true);
@@ -72606,13 +72606,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1282 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1282.size);
-                  String _elem1283;
-                  for (int _i1284 = 0; _i1284 < _list1282.size; ++_i1284)
+                  org.apache.thrift.protocol.TList _list1290 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1290.size);
+                  String _elem1291;
+                  for (int _i1292 = 0; _i1292 < _list1290.size; ++_i1292)
                   {
-                    _elem1283 = iprot.readString();
-                    struct.success.add(_elem1283);
+                    _elem1291 = iprot.readString();
+                    struct.success.add(_elem1291);
                   }
                   iprot.readListEnd();
                 }
@@ -72647,9 +72647,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 _iter1285 : struct.success)
+            for (String _iter1293 : struct.success)
             {
-              oprot.writeString(_iter1285);
+              oprot.writeString(_iter1293);
             }
             oprot.writeListEnd();
           }
@@ -72688,9 +72688,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1286 : struct.success)
+            for (String _iter1294 : struct.success)
             {
-              oprot.writeString(_iter1286);
+              oprot.writeString(_iter1294);
             }
           }
         }
@@ -72705,13 +72705,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1287 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1287.size);
-            String _elem1288;
-            for (int _i1289 = 0; _i1289 < _list1287.size; ++_i1289)
+            org.apache.thrift.protocol.TList _list1295 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1295.size);
+            String _elem1296;
+            for (int _i1297 = 0; _i1297 < _list1295.size; ++_i1297)
             {
-              _elem1288 = iprot.readString();
-              struct.success.add(_elem1288);
+              _elem1296 = iprot.readString();
+              struct.success.add(_elem1296);
             }
           }
           struct.setSuccessIsSet(true);
@@ -73685,13 +73685,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1290 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1290.size);
-                  String _elem1291;
-                  for (int _i1292 = 0; _i1292 < _list1290.size; ++_i1292)
+                  org.apache.thrift.protocol.TList _list1298 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1298.size);
+                  String _elem1299;
+                  for (int _i1300 = 0; _i1300 < _list1298.size; ++_i1300)
                   {
-                    _elem1291 = iprot.readString();
-                    struct.success.add(_elem1291);
+                    _elem1299 = iprot.readString();
+                    struct.success.add(_elem1299);
                   }
                   iprot.readListEnd();
                 }
@@ -73726,9 +73726,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 _iter1293 : struct.success)
+            for (String _iter1301 : struct.success)
             {
-              oprot.writeString(_iter1293);
+              oprot.writeString(_iter1301);
             }
             oprot.writeListEnd();
           }
@@ -73767,9 +73767,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1294 : struct.success)
+            for (String _iter1302 : struct.success)
             {
-              oprot.writeString(_iter1294);
+              oprot.writeString(_iter1302);
             }
           }
         }
@@ -73784,13 +73784,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1295 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1295.size);
-            String _elem1296;
-            for (int _i1297 = 0; _i1297 < _list1295.size; ++_i1297)
+            org.apache.thrift.protocol.TList _list1303 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1303.size);
+            String _elem1304;
+            for (int _i1305 = 0; _i1305 < _list1303.size; ++_i1305)
             {
-              _elem1296 = iprot.readString();
-              struct.success.add(_elem1296);
+              _elem1304 = iprot.readString();
+              struct.success.add(_elem1304);
             }
           }
           struct.setSuccessIsSet(true);
@@ -74447,14 +74447,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1298 = iprot.readListBegin();
-                  struct.success = new ArrayList<Table>(_list1298.size);
-                  Table _elem1299;
-                  for (int _i1300 = 0; _i1300 < _list1298.size; ++_i1300)
+                  org.apache.thrift.protocol.TList _list1306 = iprot.readListBegin();
+                  struct.success = new ArrayList<Table>(_list1306.size);
+                  Table _elem1307;
+                  for (int _i1308 = 0; _i1308 < _list1306.size; ++_i1308)
                   {
-                    _elem1299 = new Table();
-                    _elem1299.read(iprot);
-                    struct.success.add(_elem1299);
+                    _elem1307 = new Table();
+                    _elem1307.read(iprot);
+                    struct.success.add(_elem1307);
                   }
                   iprot.readListEnd();
                 }
@@ -74489,9 +74489,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 _iter1301 : struct.success)
+            for (Table _iter1309 : struct.success)
             {
-              _iter1301.write(oprot);
+              _iter1309.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -74530,9 +74530,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Table _iter1302 : struct.success)
+            for (Table _iter1310 : struct.success)
             {
-              _iter1302.write(oprot);
+              _iter1310.write(oprot);
             }
           }
         }
@@ -74547,14 +74547,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1303 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Table>(_list1303.size);
-            Table _elem1304;
-            for (int _i1305 = 0; _i1305 < _list1303.size; ++_i1305)
+            org.apache.thrift.protocol.TList _list1311 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Table>(_list1311.size);
+            Table _elem1312;
+            for (int _i1313 = 0; _i1313 < _list1311.size; ++_i1313)
             {
-              _elem1304 = new Table();
-              _elem1304.read(iprot);
-              struct.success.add(_elem1304);
+              _elem1312 = new Table();
+              _elem1312.read(iprot);
+              struct.success.add(_elem1312);
             }
           }
           struct.setSuccessIsSet(true);
@@ -75320,13 +75320,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1306 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1306.size);
-                  String _elem1307;
-                  for (int _i1308 = 0; _i1308 < _list1306.size; ++_i1308)
+                  org.apache.thrift.protocol.TList _list1314 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1314.size);
+                  String _elem1315;
+                  for (int _i1316 = 0; _i1316 < _list1314.size; ++_i1316)
                   {
-                    _elem1307 = iprot.readString();
-                    struct.success.add(_elem1307);
+                    _elem1315 = iprot.readString();
+                    struct.success.add(_elem1315);
                   }
                   iprot.readListEnd();
                 }
@@ -75361,9 +75361,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 _iter1309 : struct.success)
+            for (String _iter1317 : struct.success)
             {
-              oprot.writeString(_iter1309);
+              oprot.writeString(_iter1317);
             }
             oprot.writeListEnd();
           }
@@ -75402,9 +75402,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1310 : struct.success)
+            for (String _iter1318 : struct.success)
             {
-              oprot.writeString(_iter1310);
+              oprot.writeString(_iter1318);
             }
           }
         }
@@ -75419,13 +75419,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1311 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1311.size);
-            String _elem1312;
-            for (int _i1313 = 0; _i1313 < _list1311.size; ++_i1313)
+            org.apache.thrift.protocol.TList _list1319 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1319.size);
+            String _elem1320;
+            for (int _i1321 = 0; _i1321 < _list1319.size; ++_i1321)
             {
-              _elem1312 = iprot.readString();
-              struct.success.add(_elem1312);
+              _elem1320 = iprot.readString();
+              struct.success.add(_elem1320);
             }
           }
           struct.setSuccessIsSet(true);
@@ -75930,13 +75930,13 @@ import org.slf4j.LoggerFactory;
             case 3: // TBL_TYPES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1314 = iprot.readListBegin();
-                  struct.tbl_types = new ArrayList<String>(_list1314.size);
-                  String _elem1315;
-                  for (int _i1316 = 0; _i1316 < _list1314.size; ++_i1316)
+                  org.apache.thrift.protocol.TList _list1322 = iprot.readListBegin();
+                  struct.tbl_types = new ArrayList<String>(_list1322.size);
+                  String _elem1323;
+                  for (int _i1324 = 0; _i1324 < _list1322.size; ++_i1324)
                   {
-                    _elem1315 = iprot.readString();
-                    struct.tbl_types.add(_elem1315);
+                    _elem1323 = iprot.readString();
+                    struct.tbl_types.add(_elem1323);
                   }
                   iprot.readListEnd();
                 }
@@ -75972,9 +75972,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 _iter1317 : struct.tbl_types)
+            for (String _iter1325 : struct.tbl_types)
             {
-              oprot.writeString(_iter1317);
+              oprot.writeString(_iter1325);
             }
             oprot.writeListEnd();
           }
@@ -76017,9 +76017,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetTbl_types()) {
           {
             oprot.writeI32(struct.tbl_types.size());
-            for (String _iter1318 : struct.tbl_types)
+            for (String _iter1326 : struct.tbl_types)
             {
-              oprot.writeString(_iter1318);
+              oprot.writeString(_iter1326);
             }
           }
         }
@@ -76039,13 +76039,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1319 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_types = new ArrayList<String>(_list1319.size);
-            String _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.STRING, iprot.readI32());
+            struct.tbl_types = new ArrayList<String>(_list1327.size);
+            String _elem1328;
+            for (int _i1329 = 0; _i1329 < _list1327.size; ++_i1329)
             {
-              _elem1320 = iprot.readString();
-              struct.tbl_types.add(_elem1320);
+              _elem1328 = iprot.readString();
+              struct.tbl_types.add(_elem1328);
             }
           }
           struct.setTbl_typesIsSet(true);
@@ -76451,14 +76451,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<TableMeta>(_list1322.size);
-                  TableMeta _elem1323;
-                  for (int _i1324 = 0; _i1324 < _list1322.size; ++_i1324)
+                  org.apache.thrift.protocol.TList _list1330 = iprot.readListBegin();
+                  struct.success = new ArrayList<TableMeta>(_list1330.size);
+                  TableMeta _elem1331;
+                  for (int _i1332 = 0; _i1332 < _list1330.size; ++_i1332)
                   {
-                    _elem1323 = new TableMeta();
-                    _elem1323.read(iprot);
-                    struct.success.add(_elem1323);
+                    _elem1331 = new TableMeta();
+                    _elem1331.read(iprot);
+                    struct.success.add(_elem1331);
                   }
                   iprot.readListEnd();
                 }
@@ -76493,9 +76493,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 _iter1325 : struct.success)
+            for (TableMeta _iter1333 : struct.success)
             {
-              _iter1325.write(oprot);
+              _iter1333.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -76534,9 +76534,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (TableMeta _iter1326 : struct.success)
+            for (TableMeta _iter1334 : struct.success)
             {
-              _iter1326.write(oprot);
+              _iter1334.write(oprot);
             }
           }
         }
@@ -76551,14 +76551,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         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<TableMeta>(_list1327.size);
-            TableMeta _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<TableMeta>(_list1335.size);
+            TableMeta _elem1336;
+            for (int _i1337 = 0; _i1337 < _list1335.size; ++_i1337)
             {
-              _elem1328 = new TableMeta();
-              _elem1328.read(iprot);
-              struct.success.add(_elem1328);
+              _elem1336 = new TableMeta();
+              _elem1336.read(iprot);
+              struct.success.add(_elem1336);
             }
           }
           struct.setSuccessIsSet(true);
@@ -77324,13 +77324,13 @@ 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<String>(_list1330.size);
-                  String _elem1331;
-                  for (int _i1332 = 0; _i1332 < _list1330.size; ++_i1332)
+                  org.apache.thrift.protocol.TList _list1338 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1338.size);
+                  String _elem1339;
+                  for (int _i1340 = 0; _i1340 < _list1338.size; ++_i1340)
                   {
-                    _elem1331 = iprot.readString();
-                    struct.success.add(_elem1331);
+                    _elem1339 = iprot.readString();
+                    struct.success.add(_elem1339);
                   }
                   iprot.readListEnd();
                 }
@@ -77365,9 +77365,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 _iter1333 : struct.success)
+            for (String _iter1341 : struct.success)
             {
-              oprot.writeString(_iter1333);
+              oprot.writeString(_iter1341);
             }
             oprot.writeListEnd();
           }
@@ -77406,9 +77406,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1334 : struct.success)
+            for (String _iter1342 : struct.success)
             {
-              oprot.writeString(_iter1334);
+              oprot.writeString(_iter1342);
             }
           }
         }
@@ -77423,13 +77423,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1335 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1335.size);
-            String _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.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1343.size);
+            String _elem1344;
+            for (int _i1345 = 0; _i1345 < _list1343.size; ++_i1345)
             {
-              _elem1336 = iprot.readString();
-              struct.success.add(_elem1336);
+              _elem1344 = iprot.readString();
+              struct.success.add(_elem1344);
             }
           }
           struct.setSuccessIsSet(true);
@@ -78882,13 +78882,13 @@ import org.slf4j.LoggerFactory;
             case 2: // TBL_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1338 = iprot.readListBegin();
-                  struct.tbl_names = new ArrayList<String>(_list1338.size);
-                  String _elem1339;
-                  for (int _i1340 = 0; _i1340 < _list1338.size; ++_i1340)
+                  org.apache.thrift.protocol.TList _list1346 = iprot.readListBegin();
+                  struct.tbl_names = new ArrayList<String>(_list1346.size);
+                  String _elem1347;
+                  for (int _i1348 = 0; _i1348 < _list1346.size; ++_i1348)
                   {
-                    _elem1339 = iprot.readString();
-                    struct.tbl_names.add(_elem1339);
+                    _elem1347 = iprot.readString();
+                    struct.tbl_names.add(_elem1347);
                   }
                   iprot.readListEnd();
                 }
@@ -78919,9 +78919,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 _iter1341 : struct.tbl_names)
+            for (String _iter1349 : struct.tbl_names)
             {
-              oprot.writeString(_iter1341);
+              oprot.writeString(_iter1349);
             }
             oprot.writeListEnd();
           }
@@ -78958,9 +78958,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetTbl_names()) {
           {
             oprot.writeI32(struct.tbl_names.size());
-            for (String _iter1342 : struct.tbl_names)
+            for (String _iter1350 : struct.tbl_names)
             {
-              oprot.writeString(_iter1342);
+              oprot.writeString(_iter1350);
             }
           }
         }
@@ -78976,13 +78976,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list1343 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_names = new ArrayList<String>(_list1343.size);
-            String _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.STRING, iprot.readI32());
+            struct.tbl_names = new ArrayList<String>(_list1351.size);
+            String _elem1352;
+            for (int _i1353 = 0; _i1353 < _list1351.size; ++_i1353)
             {
-              _elem1344 = iprot.readString();
-              struct.tbl_names.add(_elem1344);
+              _elem1352 = iprot.readString();
+              struct.tbl_names.add(_elem1352);
             }
           }
           struct.setTbl_namesIsSet(true);
@@ -79307,14 +79307,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1346 = iprot.readListBegin();
-                  struct.success = new ArrayList<Table>(_list1346.size);
-                  Table _elem1347;
-                  for (int _i1348 = 0; _i1348 < _list1346.size; ++_i1348)
+                  org.apache.thrift.protocol.TList _list1354 = iprot.readListBegin();
+                  struct.success = new ArrayList<Table>(_list1354.size);
+                  Table _elem1355;
+                  for (int _i1356 = 0; _i1356 < _list1354.size; ++_i1356)
                   {
-                    _elem1347 = new Table();
-                    _elem1347.read(iprot);
-                    struct.success.add(_elem1347);
+                    _elem1355 = new Table();
+                    _elem1355.read(iprot);
+                    struct.success.add(_elem1355);
                   }
                   iprot.readListEnd();
                 }
@@ -79340,9 +79340,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 _iter1349 : struct.success)
+            for (Table _iter1357 : struct.success)
             {
-              _iter1349.write(oprot);
+              _iter1357.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -79373,9 +79373,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Table _iter1350 : struct.success)
+            for (Table _iter1358 : struct.success)
             {
-              _iter1350.write(oprot);
+              _iter1358.write(oprot);
             }
           }
         }
@@ -79387,14 +79387,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1351 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Table>(_list1351.size);
-            Table _elem1352;
-            for (int _i1353 = 0; _i1353 < _list1351.size; ++_i1353)
+            org.apache.thrift.protocol.TList _list1359 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Table>(_list1359.size);
+            Table _elem1360;
+            for (int _i1361 = 0; _i1361 < _list1359.size; ++_i1361)
             {
-              _elem1352 = new Table();
-              _elem1352.read(iprot);
-              struct.success.add(_elem1352);
+              _elem1360 = new Table();
+              _elem1360.read(iprot);
+              struct.success.add(_elem1360);
             }
           }
           struct.setSuccessIsSet(true);
@@ -80163,14 +80163,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1354 = iprot.readListBegin();
-                  struct.success = new ArrayList<ExtendedTableInfo>(_list1354.size);
-                  ExtendedTableInfo _elem1355;
-                  for (int _i1356 = 0; _i1356 < _list1354.size; ++_i1356)
+                  org.apache.thrift.protocol.TList _list1362 = iprot.readListBegin();
+                  struct.success = new ArrayList<ExtendedTableInfo>(_list1362.size);
+                  ExtendedTableInfo _elem1363;
+                  for (int _i1364 = 0; _i1364 < _list1362.size; ++_i1364)
                   {
-                    _elem1355 = new ExtendedTableInfo();
-                    _elem1355.read(iprot);
-                    struct.success.add(_elem1355);
+                    _elem1363 = new ExtendedTableInfo();
+                    _elem1363.read(iprot);
+                    struct.success.add(_elem1363);
                   }
                   iprot.readListEnd();
                 }
@@ -80205,9 +80205,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 _iter1357 : struct.success)
+            for (ExtendedTableInfo _iter1365 : struct.success)
             {
-              _iter1357.write(oprot);
+              _iter1365.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -80246,9 +80246,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (ExtendedTableInfo _iter1358 : struct.success)
+            for (ExtendedTableInfo _iter1366 : struct.success)
             {
-              _iter1358.write(oprot);
+              _iter1366.write(oprot);
             }
           }
         }
@@ -80263,14 +80263,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1359 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<ExtendedTableInfo>(_list1359.size);
-            ExtendedTableInfo _elem1360;
-            for (int _i1361 = 0; _i1361 < _list1359.size; ++_i1361)
+            org.apache.thrift.protocol.TList _list1367 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<ExtendedTableInfo>(_list1367.size);
+            ExtendedTableInfo _elem1368;
+            for (int _i1369 = 0; _i1369 < _list1367.size; ++_i1369)
             {
-              _elem1360 = new ExtendedTableInfo();
-              _elem1360.read(iprot);
-              struct.success.add(_elem1360);
+              _elem1368 = new ExtendedTableInfo();
+              _elem1368.read(iprot);
+              struct.success.add(_elem1368);
             }
           }
           struct.setSuccessIsSet(true);
@@ -85783,13 +85783,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1362 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1362.size);
-                  String _elem1363;
-                  for (int _i1364 = 0; _i1364 < _list1362.size; ++_i1364)
+                  org.apache.thrift.protocol.TList _list1370 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1370.size);
+                  String _elem1371;
+                  for (int _i1372 = 0; _i1372 < _list1370.size; ++_i1372)
                   {
-                    _elem1363 = iprot.readString();
-                    struct.success.add(_elem1363);
+                    _elem1371 = iprot.readString();
+                    struct.success.add(_elem1371);
                   }
                   iprot.readListEnd();
                 }
@@ -85842,9 +85842,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 _iter1365 : struct.success)
+            for (String _iter1373 : struct.success)
             {
-              oprot.writeString(_iter1365);
+              oprot.writeString(_iter1373);
             }
             oprot.writeListEnd();
           }
@@ -85899,9 +85899,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1366 : struct.success)
+            for (String _iter1374 : struct.success)
             {
-              oprot.writeString(_iter1366);
+              oprot.writeString(_iter1374);
             }
           }
         }
@@ -85922,13 +85922,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1367 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1367.size);
-            String _elem1368;
-            for (int _i1369 = 0; _i1369 < _list1367.size; ++_i1369)
+            org.apache.thrift.protocol.TList _list1375 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1375.size);
+            String _elem1376;
+            for (int _i1377 = 0; _i1377 < _list1375.size; ++_i1377)
             {
-              _elem1368 = iprot.readString();
-              struct.success.add(_elem1368);
+              _elem1376 = iprot.readString();
+              struct.success.add(_elem1376);
             }
           }
           struct.setSuccessIsSet(true);
@@ -92725,14 +92725,14 @@ import org.slf4j.LoggerFactory;
             case 1: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1370 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<Partition>(_list1370.size);
-                  Partition _elem1371;
-                  for (int _i1372 = 0; _i1372 < _list1370.size; ++_i1372)
+                  org.apache.thrift.protocol.TList _list1378 = iprot.readListBegin();
+                  struct.new_parts = new ArrayList<Partition>(_list1378.size);
+                  Partition _elem1379;
+                  for (int _i1380 = 0; _i1380 < _list1378.size; ++_i1380)
                   {
-                    _elem1371 = new Partition();
-                    _elem1371.read(iprot);
-                    struct.new_parts.add(_elem1371);
+                    _elem1379 = new Partition();
+                    _elem1379.read(iprot);
+                    struct.new_parts.add(_elem1379);
                   }
                   iprot.readListEnd();
                 }
@@ -92758,9 +92758,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 _iter1373 : struct.new_parts)
+            for (Partition _iter1381 : struct.new_parts)
             {
-              _iter1373.write(oprot);
+              _iter1381.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -92791,9 +92791,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (Partition _iter1374 : struct.new_parts)
+            for (Partition _iter1382 : struct.new_parts)
             {
-              _iter1374.write(oprot);
+              _iter1382.write(oprot);
             }
           }
         }
@@ -92805,14 +92805,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1375 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.new_parts = new ArrayList<Partition>(_list1375.size);
-            Partition _elem1376;
-            for (int _i1377 = 0; _i1377 < _list1375.size; ++_i1377)
+            org.apache.thrift.protocol.TList _list1383 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.new_parts = new ArrayList<Partition>(_list1383.size);
+            Partition _elem1384;
+            for (int _i1385 = 0; _i1385 < _list1383.size; ++_i1385)
             {
-              _elem1376 = new Partition();
-              _elem1376.read(iprot);
-              struct.new_parts.add(_elem1376);
+              _elem1384 = new Partition();
+              _elem1384.read(iprot);
+              struct.new_parts.add(_elem1384);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -93813,14 +93813,14 @@ import org.slf4j.LoggerFactory;
             case 1: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1378 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<PartitionSpec>(_list1378.size);
-                  PartitionSpec _elem1379;
-                  for (int _i1380 = 0; _i1380 < _list1378.size; ++_i1380)
+                  org.apache.thrift.protocol.TList _list1386 = iprot.readListBegin();
+                  struct.new_parts = new ArrayList<PartitionSpec>(_list1386.size);
+                  PartitionSpec _elem1387;
+                  for (int _i1388 = 0; _i1388 < _list1386.size; ++_i1388)
                   {
-                    _elem1379 = new PartitionSpec();
-                    _elem1379.read(iprot);
-                    struct.new_parts.add(_elem1379);
+                    _elem1387 = new PartitionSpec();
+                    _elem1387.read(iprot);
+                    struct.new_parts.add(_elem1387);
                   }
                   iprot.readListEnd();
                 }
@@ -93846,9 +93846,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 _iter1381 : struct.new_parts)
+            for (PartitionSpec _iter1389 : struct.new_parts)
             {
-              _iter1381.write(oprot);
+              _iter1389.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -93879,9 +93879,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (PartitionSpec _iter1382 : struct.new_parts)
+            for (PartitionSpec _iter1390 : struct.new_parts)
             {
-              _iter1382.write(oprot);
+              _iter1390.write(oprot);
             }
           }
         }
@@ -93893,14 +93893,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1383 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.new_parts = new ArrayList<PartitionSpec>(_list1383.size);
-            PartitionSpec _elem1384;
-            for (int _i1385 = 0; _i1385 < _list1383.size; ++_i1385)
+            org.apache.thrift.protocol.TList _list1391 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.new_parts = new ArrayList<PartitionSpec>(_list1391.size);
+            PartitionSpec _elem1392;
+            for (int _i1393 = 0; _i1393 < _list1391.size; ++_i1393)
             {
-              _elem1384 = new PartitionSpec();
-              _elem1384.read(iprot);
-              struct.new_parts.add(_elem1384);
+              _elem1392 = new PartitionSpec();
+              _elem1392.read(iprot);
+              struct.new_parts.add(_elem1392);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -95076,13 +95076,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1386 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list1386.size);
-                  String _elem1387;
-                  for (int _i1388 = 0; _i1388 < _list1386.size; ++_i1388)
+                  org.apache.thrift.protocol.TList _list1394 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list1394.size);
+                  String _elem1395;
+                  for (int _i1396 = 0; _i1396 < _list1394.size; ++_i1396)
                   {
-                    _elem1387 = iprot.readString();
-                    struct.part_vals.add(_elem1387);
+                    _elem1395 = iprot.readString();
+                    struct.part_vals.add(_elem1395);
                   }
                   iprot.readListEnd();
                 }
@@ -95118,9 +95118,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 _iter1389 : struct.part_vals)
+            for (String _iter1397 : struct.part_vals)
             {
-              oprot.writeString(_iter1389);
+              oprot.writeString(_iter1397);
             }
             oprot.writeListEnd();
           }
@@ -95163,9 +95163,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter1390 : struct.part_vals)
+            for (String _iter1398 : struct.part_vals)
             {
-              oprot.writeString(_iter1390);
+              oprot.writeString(_iter1398);
             }
           }
         }
@@ -95185,13 +95185,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1391 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list1391.size);
-            String _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.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list1399.size);
+            String _elem1400;
+            for (int _i1401 = 0; _i1401 < _list1399.size; ++_i1401)
             {
-              _elem1392 = iprot.readString();
-              struct.part_vals.add(_elem1392);
+              _elem1400 = iprot.readString();
+              struct.part_vals.add(_elem1400);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -97500,13 +97500,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1394 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list1394.size);
-                  String _elem1395;
-                  for (int _i1396 = 0; _i1396 < _list1394.size; ++_i1396)
+                  org.apache.thrift.protocol.TList _list1402 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list1402.size);
+                  String _elem1403;
+                  for (int _i1404 = 0; _i1404 < _list1402.size; ++_i1404)
                   {
-                    _elem1395 = iprot.readString();
-                    struct.part_vals.add(_elem1395);
+                    _elem1403 = iprot.readString();
+                    struct.part_vals.add(_elem1403);
                   }
                   iprot.readListEnd();
                 }
@@ -97551,9 +97551,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 _iter1397 : struct.part_vals)
+            for (String _iter1405 : struct.part_vals)
             {
-              oprot.writeString(_iter1397);
+              oprot.writeString(_iter1405);
             }
             oprot.writeListEnd();
           }
@@ -97604,9 +97604,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter1398 : struct.part_vals)
+            for (String _iter1406 : struct.part_vals)
             {
-              oprot.writeString(_iter1398);
+              oprot.writeString(_iter1406);
             }
           }
         }
@@ -97629,13 +97629,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.part_vals = 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.part_vals = new ArrayList<String>(_list1407.size);
+            String _elem1408;
+            for (int _i1409 = 0; _i1409 < _list1407.size; ++_i1409)
             {
-              _elem1400 = iprot.readString();
-              struct.part_vals.add(_elem1400);
+              _elem1408 = iprot.readString();
+              struct.part_vals.add(_elem1408);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -101505,13 +101505,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1402 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list1402.size);
-                  String _elem1403;
-                  for (int _i1404 = 0; _i1404 < _list1402.size; ++_i1404)
+                  org.apache.thrift.protocol.TList _list1410 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list1410.size);
+                  String _elem1411;
+                  for (int _i1412 = 0; _i1412 < _list1410.size; ++_i1412)
                   {
-                    _elem1403 = iprot.readString();
-                    struct.part_vals.add(_elem1403);
+                    _elem1411 = iprot.readString();
+                    struct.part_vals.add(_elem1411);
                   }
                   iprot.readListEnd();
                 }
@@ -101555,9 +101555,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 _iter1405 : struct.part_vals)
+            for (String _iter1413 : struct.part_vals)
             {
-              oprot.writeString(_iter1405);
+              oprot.writeString(_iter1413);
             }
             oprot.writeListEnd();
           }
@@ -101606,9 +101606,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter1406 : struct.part_vals)
+            for (String _iter1414 : struct.part_vals)
             {
-              oprot.writeString(_iter1406);
+              oprot.writeString(_iter1414);
             }
           }
         }
@@ -101631,13 +101631,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1407 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = 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.part_vals = new ArrayList<String>(_list1415.size);
+            String _elem1416;
+            for (int _i1417 = 0; _i1417 < _list1415.size; ++_i1417)
             {
-              _elem1408 = iprot.readString();
-              struct.part_vals.add(_elem1408);
+              _elem1416 = iprot.readString();
+              struct.part_vals.add(_elem1416);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -102876,13 +102876,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1410 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list1410.size);
-                  String _elem1411;
-                  for (int _i1412 = 0; _i1412 < _list1410.size; ++_i1412)
+                  org.apache.thrift.protocol.TList _list1418 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list1418.size);
+                  String _elem1419;
+                  for (int _i1420 = 0; _i1420 < _list1418.size; ++_i1420)
                   {
-                    _elem1411 = iprot.readString();
-                    struct.part_vals.add(_elem1411);
+                    _elem1419 = iprot.readString();
+                    struct.part_vals.add(_elem1419);
                   }
                   iprot.readListEnd();
                 }
@@ -102935,9 +102935,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 _iter1413 : struct.part_vals)
+            for (String _iter1421 : struct.part_vals)
             {
-              oprot.writeString(_iter1413);
+              oprot.writeString(_iter1421);
             }
             oprot.writeListEnd();
           }
@@ -102994,9 +102994,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter1414 : struct.part_vals)
+            for (String _iter1422 : struct.part_vals)
             {
-              oprot.writeString(_iter1414);
+              oprot.writeString(_iter1422);
             }
           }
         }
@@ -103022,13 +103022,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1415 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = 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.part_vals = new ArrayList<String>(_list1423.size);
+            String _elem1424;
+            for (int _i1425 = 0; _i1425 < _list1423.size; ++_i1425)
             {
-              _elem1416 = iprot.readString();
-              struct.part_vals.add(_elem1416);
+              _elem1424 = iprot.readString();
+              struct.part_vals.add(_elem1424);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -107630,13 +107630,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1418 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list1418.size);
-                  String _elem1419;
-                  for (int _i1420 = 0; _i1420 < _list1418.size; ++_i1420)
+                  org.apache.thrift.protocol.TList _list1426 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list1426.size);
+                  String _elem1427;
+                  for (int _i1428 = 0; _i1428 < _list1426.size; ++_i1428)
                   {
-                    _elem1419 = iprot.readString();
-                    struct.part_vals.add(_elem1419);
+                    _elem1427 = iprot.readString();
+                    struct.part_vals.add(_elem1427);
                   }
                   iprot.readListEnd();
                 }
@@ -107672,9 +107672,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 _iter1421 : struct.part_vals)
+            for (String _iter1429 : struct.part_vals)
             {
-              oprot.writeString(_iter1421);
+              oprot.writeString(_iter1429);
             }
             oprot.writeListEnd();
           }
@@ -107717,9 +107717,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter1422 : struct.part_vals)
+            for (String _iter1430 : struct.part_vals)
             {
-              oprot.writeString(_iter1422);
+              oprot.writeString(_iter1430);
             }
           }
         }
@@ -107739,13 +107739,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1423 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list1423.size);
-            String _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.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list1431.size);
+            String _elem1432;
+            for (int _i1433 = 0; _i1433 < _list1431.size; ++_i1433)
             {
-              _elem1424 = iprot.readString();
-              struct.part_vals.add(_elem1424);
+              _elem1432 = iprot.readString();
+              struct.part_vals.add(_elem1432);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -108963,15 +108963,15 @@ import org.slf4j.LoggerFactory;
             case 1: // PARTITION_SPECS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map1426 = iprot.readMapBegin();
-                  struct.partitionSpecs = new HashMap<String,String>(2*_map1426.size);
-                  String _key1427;
-                  String _val1428;
-                  for (int _i1429 = 0; _i1429 < _map1426.size; ++_i1429)
+                  org.apache.thrift.protocol.TMap _map1434 = iprot.readMapBegin();
+                  struct.partitionSpecs = new HashMap<String,String>(2*_map1434.size);
+                  String _key1435;
+                  String _val1436;
+                  for (int _i1437 = 0; _i1437 < _map1434.size; ++_i1437)
                   {
-                    _key1427 = iprot.readString();
-                    _val1428 = iprot.readString();
-                    struct.partitionSpecs.put(_key1427, _val1428);
+                    _key1435 = iprot.readString();
+                    _val1436 = iprot.readString();
+                    struct.partitionSpecs.put(_key1435, _val1436);
                   }
                   iprot.readMapEnd();
                 }
@@ -109029,10 +109029,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> _iter1430 : struct.partitionSpecs.entrySet())
+            for (Map.Entry<String, String> _iter1438 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter1430.getKey());
-              oprot.writeString(_iter1430.getValue());
+              oprot.writeString(_iter1438.getKey());
+              oprot.writeString(_iter1438.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -109095,10 +109095,10 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPartitionSpecs()) {
           {
             oprot.writeI32(struct.partitionSpecs.size());
-            for (Map.Entry<String, String> _iter1431 : struct.partitionSpecs.entrySet())
+            for (Map.Entry<String, String> _iter1439 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter1431.getKey());
-              oprot.writeString(_iter1431.getValue());
+              oprot.writeString(_iter1439.getKey());
+              oprot.writeString(_iter1439.getValue());
             }
           }
         }
@@ -109122,15 +109122,15 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map1432 = 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*_map1432.size);
-            String _key1433;
-            String _val1434;
-            for (int _i1435 = 0; _i1435 < _map1432.size; ++_i1435)
+            org.apache.thrift.protocol.TMap _map1440 = 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*_map1440.size);
+            String _key1441;
+            String _val1442;
+            for (int _i1443 = 0; _i1443 < _map1440.size; ++_i1443)
             {
-              _key1433 = iprot.readString();
-              _val1434 = iprot.readString();
-              struct.partitionSpecs.put(_key1433, _val1434);
+              _key1441 = iprot.readString();
+              _val1442 = iprot.readString();
+              struct.partitionSpecs.put(_key1441, _val1442);
             }
           }
           struct.setPartitionSpecsIsSet(true);
@@ -110576,15 +110576,15 @@ import org.slf4j.LoggerFactory;
             case 1: // PARTITION_SPECS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map1436 = iprot.readMapBegin();
-                  struct.partitionSpecs = new HashMap<String,String>(2*_map1436.size);
-                  String _key1437;
-                  String _val1438;
-                  for (int _i1439 = 0; _i1439 < _map1436.size; ++_i1439)
+                  org.apache.thrift.protocol.TMap _map1444 = iprot.readMapBegin();
+                  struct.partitionSpecs = new HashMap<String,String>(2*_map1444.size);
+                  String _key1445;
+                  String _val1446;
+                  for (int _i1447 = 0; _i1447 < _map1444.size; ++_i1447)
                   {
-                    _key1437 = iprot.readString();
-                    _val1438 = iprot.readString();
-                    struct.partitionSpecs.put(_key1437, _val1438);
+                    _key1445 = iprot.readString();
+                    _val1446 = iprot.readString();
+                    struct.partitionSpecs.put(_key1445, _val1446);
                   }
                   iprot.readMapEnd();
                 }
@@ -110642,10 +110642,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> _iter1440 : struct.partitionSpecs.entrySet())
+            for (Map.Entry<String, String> _iter1448 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter1440.getKey());
-              oprot.writeString(_iter1440.getValue());
+              oprot.writeString(_iter1448.getKey());
+              oprot.writeString(_iter1448.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -110708,10 +110708,10 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPartitionSpecs()) {
           {
             oprot.writeI32(struct.partitionSpecs.size());
-            for (Map.Entry<String, String> _iter1441 : struct.partitionSpecs.entrySet())
+            for (Map.Entry<String, String> _iter1449 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter1441.getKey());
-              oprot.writeString(_iter1441.getValue());
+              oprot.writeString(_iter1449.getKey());
+              oprot.writeString(_iter1449.getValue());
             }
           }
         }
@@ -110735,15 +110735,15 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map1442 = 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*_map1442.size);
-            String _key1443;
-            String _val1444;
-            for (int _i1445 = 0; _i1445 < _map1442.size; ++_i1445)
+            org.apache.thrift.protocol.TMap _map1450 = 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*_map1450.size);
+            String _key1451;
+            String _val1452;
+            for (int _i1453 = 0; _i1453 < _map1450.size; ++_i1453)
             {
-              _key1443 = iprot.readString();
-              _val1444 = iprot.readString();
-              struct.partitionSpecs.put(_key1443, _val1444);
+              _key1451 = iprot.readString();
+              _val1452 = iprot.readString();
+              struct.partitionSpecs.put(_key1451, _val1452);
             }
           }
           struct.setPartitionSpecsIsSet(true);
@@ -111408,14 +111408,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1446 = iprot.readListBegin();
-                  struct.success = new ArrayList<Partition>(_list1446.size);
-                  Partition _elem1447;
-                  for (int _i1448 = 0; _i1448 < _list1446.size; ++_i1448)
+                  org.apache.thrift.protocol.TList _list1454 = iprot.readListBegin();
+                  struct.success = new ArrayList<Partition>(_list1454.size);
+                  Partition _elem1455;
+                  for (int _i1456 = 0; _i1456 < _list1454.size; ++_i1456)
                   {
-                    _elem1447 = new Partition();
-                    _elem1447.read(iprot);
-                    struct.success.add(_elem1447);
+                    _elem1455 = new Partition();
+                    _elem1455.read(iprot);
+                    struct.success.add(_elem1455);
                   }
                   iprot.readListEnd();
                 }
@@ -111477,9 +111477,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 _iter1449 : struct.success)
+            for (Partition _iter1457 : struct.success)
             {
-              _iter1449.write(oprot);
+              _iter1457.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -111542,9 +111542,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Partition _iter1450 : struct.success)
+            for (Partition _iter1458 : struct.success)
             {
-              _iter1450.write(oprot);
+              _iter1458.write(oprot);
             }
           }
         }
@@ -111568,14 +111568,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1451 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Partition>(_list1451.size);
-            Partition _elem1452;
-            for (int _i1453 = 0; _i1453 < _list1451.size; ++_i1453)
+            org.apache.thrift.protocol.TList _list1459 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Partition>(_list1459.size);
+            Partition _elem1460;
+            for (int _i1461 = 0; _i1461 < _list1459.size; ++_i1461)
             {
-              _elem1452 = new Partition();
-              _elem1452.read(iprot);
-              struct.success.add(_elem1452);
+              _elem1460 = new Partition();
+              _elem1460.read(iprot);
+              struct.success.add(_elem1460);
             }
           }
           struct.setSuccessIsSet(true);
@@ -112274,13 +112274,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1454 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list1454.size);
-                  String _elem1455;
-                  for (int _i1456 = 0; _i1456 < _list1454.size; ++_i1456)
+                  org.apache.thrift.protocol.TList _list1462 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list1462.size);
+                  String _elem1463;
+                  for (int _i1464 = 0; _i1464 < _list1462.size; ++_i1464)
                   {
-                    _elem1455 = iprot.readString();
-                    struct.part_vals.add(_elem1455);
+                    _elem1463 = iprot.readString();
+                    struct.part_vals.add(_elem1463);
                   }
                   iprot.readListEnd();
                 }
@@ -112300,13 +112300,13 @@ import org.slf4j.LoggerFactory;
             case 5: // GROUP_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1457 = iprot.readListBegin();
-                  struct.group_names = new ArrayList<String>(_list1457.size);
-                  String _elem1458;
-                  for (int _i1459 = 0; _i1459 < _list1457.size; ++_i1459)
+                  org.apache.thrift.protocol.TList _list1465 = iprot.readListBegin();
+                  struct.group_names = new ArrayList<String>(_list1465.size);
+                  String _elem1466;
+                  for (int _i1467 = 0; _i1467 < _list1465.size; ++_i1467)
                   {
-                    _elem1458 = iprot.readString();
-                    struct.group_names.add(_elem1458);
+                    _elem1466 = iprot.readString();
+                    struct.group_names.add(_elem1466);
                   }
                   iprot.readListEnd();
                 }
@@ -112342,9 +112342,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 _iter1460 : struct.part_vals)
+            for (String _iter1468 : struct.part_vals)
             {
-              oprot.writeString(_iter1460);
+              oprot.writeString(_iter1468);
             }
             oprot.writeListEnd();
           }
@@ -112359,9 +112359,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 _iter1461 : struct.group_names)
+            for (String _iter1469 : struct.group_names)
             {
-              oprot.writeString(_iter1461);
+              oprot.writeString(_iter1469);
             }
             oprot.writeListEnd();
           }
@@ -112410,9 +112410,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter1462 : struct.part_vals)
+            for (String _iter1470 : struct.part_vals)
             {
-              oprot.writeString(_iter1462);
+              oprot.writeString(_iter1470);
             }
           }
         }
@@ -112422,9 +112422,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetGroup_names()) {
           {
             oprot.writeI32(struct.group_names.size());
-            for (String _iter1463 : struct.group_names)
+            for (String _iter1471 : struct.group_names)
             {
-              oprot.writeString(_iter1463);
+              oprot.writeString(_iter1471);
             }
           }
         }
@@ -112444,13 +112444,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1464 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list1464.size);
-            String _elem1465;
-            for (int _i1466 = 0; _i1466 < _list1464.size; ++_i1466)
+            org.apache.thrift.protocol.TList _list1472 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list1472.size);
+            String _elem1473;
+            for (int _i1474 = 0; _i1474 < _list1472.size; ++_i1474)
             {
-              _elem1465 = iprot.readString();
-              struct.part_vals.add(_elem1465);
+              _elem1473 = iprot.readString();
+              struct.part_vals.add(_elem1473);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -112461,13 +112461,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol.TList _list1467 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.group_names = new ArrayList<String>(_list1467.size);
-            String _elem1468;
-            for (int _i1469 = 0; _i1469 < _list1467.size; ++_i1469)
+            org.apache.thrift.protocol.TList _list1475 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.group_names = new ArrayList<String>(_list1475.size);
+            String _elem1476;
+            for (int _i1477 = 0; _i1477 < _list1475.size; ++_i1477)
             {
-              _elem1468 = iprot.readString();
-              struct.group_names.add(_elem1468);
+              _elem1476 = iprot.readString();
+              struct.group_names.add(_elem1476);
             }
           }
           struct.setGroup_namesIsSet(true);
@@ -115236,14 +115236,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1470 = iprot.readListBegin();
-                  struct.success = new ArrayList<Partition>(_list1470.size);
-                  Partition _elem1471;
-                  for (int _i1472 = 0; _i1472 < _list1470.size; ++_i1472)
+                  org.apache.thrift.protocol.TList _list1478 = iprot.readListBegin();
+                  struct.success = new ArrayList<Partition>(_list1478.size);
+                  Partition _elem1479;
+                  for (int _i1480 = 0; _i1480 < _list1478.size; ++_i1480)
                   {
-                    _elem1471 = new Partition();
-                    _elem1471.read(iprot);
-                    struct.success.add(_elem1471);
+                    _elem1479 = new Partition();
+                    _elem1479.read(iprot);
+                    struct.success.add(_elem1479);
                   }
                   iprot.readListEnd();
                 }
@@ -115287,9 +115287,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 _iter1473 : struct.success)
+            for (Partition _iter1481 : struct.success)
             {
-              _iter1473.write(oprot);
+              _iter1481.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -115336,9 +115336,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Partition _iter1474 : struct.success)
+            for (Partition _iter1482 : struct.success)
             {
-              _iter1474.write(oprot);
+              _iter1482.write(oprot);
             }
           }
         }
@@ -115356,14 +115356,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1475 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Partition>(_list1475.size);
-            Partition _elem1476;
-            for (int _i1477 = 0; _i1477 < _list1475.size; ++_i1477)
+            org.apache.thrift.protocol.TList _list1483 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Partition>(_list1483.size);
+            Partition _elem1484;
+            for (int _i1485 = 0; _i1485 < _list1483.size; ++_i1485)
             {
-              _elem1476 = new Partition();
-              _elem1476.read(iprot);
-              struct.success.add(_elem1476);
+              _elem1484 = new Partition();
+              _elem1484.read(iprot);
+              struct.success.add(_elem1484);
             }
           }
           struct.setSuccessIsSet(true);
@@ -116053,13 +116053,13 @@ import org.slf4j.LoggerFactory;
             case 5: // GROUP_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1478 = iprot.readListBegin();
-                  struct.group_names = new ArrayList<String>(_list1478.size);
-                  String _elem1479;
-                  for (int _i1480 = 0; _i1480 < _list1478.size; ++_i1480)
+                  org.apache.thrift.protocol.TList _list1486 = iprot.readListBegin();
+                  struct.group_names = new ArrayList<String>(_list1486.size);
+                  String _elem1487;
+                  for (int _i1488 = 0; _i1488 < _list1486.size; ++_i1488)
                   {
-                    _elem1479 = iprot.readString();
-                    struct.group_names.add(_elem1479);
+                    _elem1487 = iprot.readString();
+                    struct.group_names.add(_elem1487);
                   }
                   iprot.readListEnd();
                 }
@@ -116103,9 +116103,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 _iter1481 : struct.group_names)
+            for (String _iter1489 : struct.group_names)
             {
-              oprot.writeString(_iter1481);
+              oprot.writeString(_iter1489);
             }
             oprot.writeListEnd();
           }
@@ -116160,9 +116160,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetGroup_names()) {
           {
             oprot.writeI32(struct.group_names.size());
-            for (String _iter1482 : struct.group_names)
+            for (String _iter1490 : struct.group_names)
             {
-              oprot.writeString(_iter1482);
+              oprot.writeString(_iter1490);
             }
           }
         }
@@ -116190,13 +116190,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol.TList _list1483 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.group_names = new ArrayList<String>(_list1483.size);
-            String _elem1484;
-            for (int _i1485 = 0; _i1485 < _list1483.size; ++_i1485)
+            org.apache.thrift.protocol.TList _list1491 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.group_names = new ArrayList<String>(_list1491.size);
+            String _elem1492;
+            for (int _i1493 = 0; _i1493 < _list1491.size; ++_i1493)
             {
-              _elem1484 = iprot.readString();
-              struct.group_names.add(_elem1484);
+              _elem1492 = iprot.readString();
+              struct.group_names.add(_elem1492);
             }
           }
           struct.setGroup_namesIsSet(true);
@@ -116683,14 +116683,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1486 = iprot.readListBegin();
-                  struct.success = new ArrayList<Partition>(_list1486.size);
-                  Partition _elem1487;
-                  for (int _i1488 = 0; _i1488 < _list1486.size; ++_i1488)
+                  org.apache.thrift.protocol.TList _list1494 = iprot.readListBegin();
+                  struct.success = new ArrayList<Partition>(_list1494.size);
+                  Partition _elem1495;
+                  for (int _i1496 = 0; _i1496 < _list1494.size; ++_i1496)
                   {
-                    _elem1487 = new Partition();
-                    _elem1487.read(iprot);
-                    struct.success.add(_elem1487);
+                    _elem1495 = new Partition();
+                    _elem1495.read(iprot);
+                    struct.success.add(_elem1495);
                   }
                   iprot.readListEnd();
                 }
@@ -116734,9 +116734,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 _iter1489 : struct.success)
+            for (Partition _iter1497 : struct.success)
             {
-              _iter1489.write(oprot);
+              _iter1497.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -116783,9 +116783,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Partition _iter1490 : struct.success)
+            for (Partition _iter1498 : struct.success)
             {
-              _iter1490.write(oprot);
+              _iter1498.write(oprot);
             }
           }
         }
@@ -116803,14 +116803,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1491 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Partition>(_list1491.size);
-            Partition _elem1492;
-            for (int _i1493 = 0; _i1493 < _list1491.size; ++_i1493)
+            org.apache.thrift.protocol.TList _list1499 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Partition>(_list1499.size);
+            Partition _elem1500;
+            for (int _i1501 = 0; _i1501 < _list1499.size; ++_i1501)
             {
-              _elem1492 = new Partition();
-              _elem1492.read(iprot);
-              struct.success.add(_elem1492);
+              _elem1500 = new Partition();
+              _elem1500.read(iprot);
+              struct.success.add(_elem1500);
             }
           }
           struct.setSuccessIsSet(true);
@@ -117873,14 +117873,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1494 = iprot.readListBegin();
-                  struct.success = new ArrayList<PartitionSpec>(_list1494.size);
-                  PartitionSpec _elem1495;
-                  for (int _i1496 = 0; _i1496 < _list1494.size; ++_i1496)
+                  org.apache.thrift.protocol.TList _list1502 = iprot.readListBegin();
+                  struct.success = new ArrayList<PartitionSpec>(_list1502.size);
+                  PartitionSpec _elem1503;
+                  for (int _i1504 = 0; _i1504 < _list1502.size; ++_i1504)
                   {
-                    _elem1495 = new PartitionSpec();
-                    _elem1495.read(iprot);
-                    struct.success.add(_elem1495);
+                    _elem1503 = new PartitionSpec();
+                    _elem1503.read(iprot);
+                    struct.success.add(_elem1503);
                   }
                   iprot.readListEnd();
                 }
@@ -117924,9 +117924,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 _iter1497 : struct.success)
+            for (PartitionSpec _iter1505 : struct.success)
             {
-              _iter1497.write(oprot);
+              _iter1505.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -117973,9 +117973,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (PartitionSpec _iter1498 : struct.success)
+            for (PartitionSpec _iter1506 : struct.success)
             {
-              _iter1498.write(oprot);
+              _iter1506.write(oprot);
             }
           }
         }
@@ -117993,14 +117993,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1499 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<PartitionSpec>(_list1499.size);
-            PartitionSpec _elem1500;
-            for (int _i1501 = 0; _i1501 < _list1499.size; ++_i1501)
+            org.apache.thrift.protocol.TList _list1507 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<PartitionSpec>(_list1507.size);
+            PartitionSpec _elem1508;
+            for (int _i1509 = 0; _i1509 < _list1507.size; ++_i1509)
             {
-              _elem1500 = new PartitionSpec();
-              _elem1500.read(iprot);
-              struct.success.add(_elem1500);
+              _elem1508 = new PartitionSpec();
+              _elem1508.read(iprot);
+              struct.success.add(_elem1508);
             }
           }
           struct.setSuccessIsSet(true);
@@ -119060,13 +119060,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1502 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1502.size);
-                  String _elem1503;
-                  for (int _i1504 = 0; _i1504 < _list1502.size; ++_i1504)
+                  org.apache.thrift.protocol.TList _list1510 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1510.size);
+                  String _elem1511;
+                  for (int _i1512 = 0; _i1512 < _list1510.size; ++_i1512)
                   {
-                    _elem1503 = iprot.readString();
-                    struct.success.add(_elem1503);
+                    _elem1511 = iprot.readString();
+                    struct.success.add(_elem1511);
                   }
                   iprot.readListEnd();
                 }
@@ -119110,9 +119110,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 _iter1505 : struct.success)
+            for (String _iter1513 : struct.success)
             {
-              oprot.writeString(_iter1505);
+              oprot.writeString(_iter1513);
             }
             oprot.writeListEnd();
           }
@@ -119159,9 +119159,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1506 : struct.success)
+            for (String _iter1514 : struct.success)
             {
-              oprot.writeString(_iter1506);
+              oprot.writeString(_iter1514);
             }
           }
         }
@@ -119179,13 +119179,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1507 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1507.size);
-            String _elem1508;
-            for (int _i1509 = 0; _i1509 < _list1507.size; ++_i1509)
+            org.apache.thrift.protocol.TList _list1515 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1515.size);
+            String _elem1516;
+            for (int _i1517 = 0; _i1517 < _list1515.size; ++_i1517)
             {
-              _elem1508 = iprot.readString();
-              struct.success.add(_elem1508);
+              _elem1516 = iprot.readString();
+              struct.success.add(_elem1516);
             }
           }
           struct.setSuccessIsSet(true);
@@ -120716,13 +120716,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1510 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list1510.size);
-                  String _elem1511;
-                  for (int _i1512 = 0; _i1512 < _list1510.size; ++_i1512)
+                  org.apache.thrift.protocol.TList _list1518 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list1518.size);
+                  String _elem1519;
+                  for (int _i1520 = 0; _i1520 < _list1518.size; ++_i1520)
                   {
-                    _elem1511 = iprot.readString();
-                    struct.part_vals.add(_elem1511);
+                    _elem1519 = iprot.readString();
+                    struct.part_vals.add(_elem1519);
                   }
                   iprot.readListEnd();
                 }
@@ -120766,9 +120766,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 _iter1513 : struct.part_vals)
+            for (String _iter1521 : struct.part_vals)
             {
-              oprot.writeString(_iter1513);
+              oprot.writeString(_iter1521);
             }
             oprot.writeListEnd();
           }
@@ -120817,9 +120817,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter1514 : struct.part_vals)
+            for (String _iter1522 : struct.part_vals)
             {
-              oprot.writeString(_iter1514);
+              oprot.writeString(_iter1522);
             }
           }
         }
@@ -120842,13 +120842,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1515 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list1515.size);
-            String _elem1516;
-            for (int _i1517 = 0; _i1517 < _list1515.size; ++_i1517)
+            org.apache.thrift.protocol.TList _list1523 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list1523.size);
+            String _elem1524;
+            for (int _i1525 = 0; _i1525 < _list1523.size; ++_i1525)
             {
-              _elem1516 = iprot.readString();
-              struct.part_vals.add(_elem1516);
+              _elem1524 = iprot.readString();
+              struct.part_vals.add(_elem1524);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -121339,14 +121339,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1518 = iprot.readListBegin();
-                  struct.success = new ArrayList<Partition>(_list1518.size);
-                  Partition _elem1519;
-                  for (int _i1520 = 0; _i1520 < _list1518.size; ++_i1520)
+                  org.apache.thrift.protocol.TList _list1526 = iprot.readListBegin();
+                  struct.success = new ArrayList<Partition>(_list1526.size);
+                  Partition _elem1527;
+                  for (int _i1528 = 0; _i1528 < _list1526.size; ++_i1528)
                   {
-                    _elem1519 = new Partition();
-                    _elem1519.read(iprot);
-                    struct.success.add(_elem1519);
+                    _elem1527 = new Partition();
+                    _elem1527.read(iprot);
+                    struct.success.add(_elem1527);
                   }
                   iprot.readListEnd();
                 }
@@ -121390,9 +121390,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 _iter1521 : struct.success)
+            for (Partition _iter1529 : struct.success)
             {
-              _iter1521.write(oprot);
+              _iter1529.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -121439,9 +121439,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Partition _iter1522 : struct.success)
+            for (Partition _iter1530 : struct.success)
             {
-              _iter1522.write(oprot);
+              _iter1530.write(oprot);
             }
           }
         }
@@ -121459,14 +121459,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1523 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Partition>(_list1523.size);
-            Partition _elem1524;
-            for (int _i1525 = 0; _i1525 < _list1523.size; ++_i1525)
+            org.apache.thrift.protocol.TList _list1531 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Partition>(_list1531.size);
+            Partition _elem1532;
+            for (int _i1533 = 0; _i1533 < _list1531.size; ++_i1533)
             {
-              _elem1524 = new Partition();
-              _elem1524.read(iprot);
-              struct.success.add(_elem1524);
+              _elem1532 = new Partition();
+              _elem1532.read(iprot);
+              struct.success.add(_elem1532);
             }
           }
           struct.setSuccessIsSet(true);
@@ -122238,13 +122238,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1526 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list1526.size);
-                  String _elem1527;
-                  for (int _i1528 = 0; _i1528 < _list1526.size; ++_i1528)
+                  org.apache.thrift.protocol.TList _list1534 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list1534.size);
+                  String _elem1535;
+                  for (int _i1536 = 0; _i1536 < _list1534.size; ++_i1536)
                   {
-                    _elem1527 = iprot.readString();
-                    struct.part_vals.add(_elem1527);
+                    _elem1535 = iprot.readString();
+                    struct.part_vals.add(_elem1535);
                   }
                   iprot.readListEnd();
                 }
@@ -122272,13 +122272,13 @@ import org.slf4j.LoggerFactory;
             case 6: // GROUP_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1529 = iprot.readListBegin();
-                  struct.group_names = new ArrayList<String>(_list1529.size);
-                  String _elem1530;
-                  for (int _i1531 = 0; _i1531 < _list1529.size; ++_i1531)
+                  org.apache.thrift.protocol.TList _list1537 = iprot.readListBegin();
+                  struct.group_names = new ArrayList<String>(_list1537.size);
+                  String _elem1538;
+                  for (int _i1539 = 0; _i1539 < _list1537.size; ++_i1539)
                   {
-                    _elem1530 = iprot.readString();
-                    struct.group_names.add(_elem1530);
+                    _elem1538 = iprot.readString();
+                    struct.group_names.add(_elem1538);
                   }
                   iprot.readListEnd();
                 }
@@ -122314,9 +122314,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 _iter1532 : struct.part_vals)
+            for (String _iter1540 : struct.part_vals)
             {
-              oprot.writeString(_iter1532);
+              oprot.writeString(_iter1540);
             }
             oprot.writeListEnd();
           }
@@ -122334,9 +122334,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 _iter1533 : struct.group_names)
+            for (String _iter1541 : struct.group_names)
             {
-              oprot.writeString(_iter1533);
+              oprot.writeString(_iter1541);
             }
             oprot.writeListEnd();
           }
@@ -122388,9 +122388,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);
             }
           }
         }
@@ -122403,9 +122403,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetGroup_names()) {
           {
             oprot.writeI32(struct.group_names.size());
-            for (String _iter1535 : struct.group_names)
+            for (String _iter1543 : struct.group_names)
             {
-              oprot.writeString(_iter1535);
+              oprot.writeString(_iter1543);
             }
           }
         }
@@ -122425,13 +122425,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1536 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list1536.size);
-            String _elem1537;
-            for (int _i1538 = 0; _i1538 < _list1536.size; ++_i1538)
+            org.apache.thrift.protocol.TList _list1544 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list1544.size);
+            String _elem1545;
+            for (int _i1546 = 0; _i1546 < _list1544.size; ++_i1546)
             {
-              _elem1537 = iprot.readString();
-              struct.part_vals.add(_elem1537);
+              _elem1545 = iprot.readString();
+              struct.part_vals.add(_elem1545);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -122446,13 +122446,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(5)) {
           {
-            org.apache.thrift.protocol.TList _list1539 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.group_names = new ArrayList<String>(_list1539.size);
-            String _elem1540;
-            for (int _i1541 = 0; _i1541 < _list1539.size; ++_i1541)
+            org.apache.thrift.protocol.TList _list1547 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.group_names = new ArrayList<String>(_list1547.size);
+            String _elem1548;
+            for (int _i1549 = 0; _i1549 < _list1547.size; ++_i1549)
             {
-              _elem1540 = iprot.readString();
-              struct.group_names.add(_elem1540);
+              _elem1548 = iprot.readString();
+              struct.group_names.add(_elem1548);
             }
           }
           struct.setGroup_namesIsSet(true);
@@ -122939,14 +122939,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1542 = iprot.readListBegin();
-                  struct.success = new ArrayList<Partition>(_list1542.size);
-                  Partition _elem1543;
-                  for (int _i1544 = 0; _i1544 < _list1542.size; ++_i1544)
+                  org.apache.thrift.protocol.TList _list1550 = iprot.readListBegin();
+                  struct.success = new ArrayList<Partition>(_list1550.size);
+                  Partition _elem1551;
+                  for (int _i1552 = 0; _i1552 < _list1550.size; ++_i1552)
                   {
-                    _elem1543 = new Partition();
-                    _elem1543.read(iprot);
-                    struct.success.add(_elem1543);
+                    _elem1551 = new Partition();
+                    _elem1551.read(iprot);
+                    struct.success.add(_elem1551);
                   }
                   iprot.readListEnd();
                 }
@@ -122990,9 +122990,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 _iter1545 : struct.success)
+            for (Partition _iter1553 : struct.success)
             {
-              _iter1545.write(oprot);
+              _iter1553.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -123039,9 +123039,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Partition _iter1546 : struct.success)
+            for (Partition _iter1554 : struct.success)
             {
-              _iter1546.write(oprot);
+              _iter1554.write(oprot);
             }
           }
         }
@@ -123059,14 +123059,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1547 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Partition>(_list1547.size);
-            Partition _elem1548;
-            for (int _i1549 = 0; _i1549 < _list1547.size; ++_i1549)
+            org.apache.thrift.protocol.TList _list1555 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Partition>(_list1555.size);
+            Partition _elem1556;
+            for (int _i1557 = 0; _i1557 < _list1555.size; ++_i1557)
             {
-              _elem1548 = new Partition();
-              _elem1548.read(iprot);
-              struct.success.add(_elem1548);
+              _elem1556 = new Partition();
+              _elem1556.read(iprot);
+              struct.success.add(_elem1556);
             }
           }
           struct.setSuccessIsSet(true);
@@ -123659,13 +123659,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1550 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list1550.size);
-                  String _elem1551;
-                  for (int _i1552 = 0; _i1552 < _list1550.size; ++_i1552)
+                  org.apache.thrift.protocol.TList _list1558 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list1558.size);
+                  String _elem1559;
+                  for (int _i1560 = 0; _i1560 < _list1558.size; ++_i1560)
                   {
-                    _elem1551 = iprot.readString();
-                    struct.part_vals.add(_elem1551);
+                    _elem1559 = iprot.readString();
+                    struct.part_vals.add(_elem1559);
                   }
                   iprot.readListEnd();
                 }
@@ -123709,9 +123709,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 _iter1553 : struct.part_vals)
+            for (String _iter1561 : struct.part_vals)
             {
-              oprot.writeString(_iter1553);
+              oprot.writeString(_iter1561);
             }
             oprot.writeListEnd();
           }
@@ -123760,9 +123760,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter1554 : struct.part_vals)
+            for (String _iter1562 : struct.part_vals)
             {
-              oprot.writeString(_iter1554);
+              oprot.writeString(_iter1562);
             }
           }
         }
@@ -123785,13 +123785,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1555 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list1555.size);
-            String _elem1556;
-            for (int _i1557 = 0; _i1557 < _list1555.size; ++_i1557)
+            org.apache.thrift.protocol.TList _list1563 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list1563.size);
+            String _elem1564;
+            for (int _i1565 = 0; _i1565 < _list1563.size; ++_i1565)
             {
-              _elem1556 = iprot.readString();
-              struct.part_vals.add(_elem1556);
+              _elem1564 = iprot.readString();
+              struct.part_vals.add(_elem1564);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -124279,13 +124279,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1558 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1558.size);
-                  String _elem1559;
-                  for (int _i1560 = 0; _i1560 < _list1558.size; ++_i1560)
+                  org.apache.thrift.protocol.TList _list1566 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1566.size);
+                  String _elem1567;
+                  for (int _i1568 = 0; _i1568 < _list1566.size; ++_i1568)
                   {
-                    _elem1559 = iprot.readString();
-                    struct.success.add(_elem1559);
+                    _elem1567 = iprot.readString();
+                    struct.success.add(_elem1567);
                   }
                   iprot.readListEnd();
                 }
@@ -124329,9 +124329,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 _iter1561 : struct.success)
+            for (String _iter1569 : struct.success)
             {
-              oprot.writeString(_iter1561);
+              oprot.writeString(_iter1569);
             }
             oprot.writeListEnd();
           }
@@ -124378,9 +124378,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1562 : struct.success)
+            for (String _iter1570 : struct.success)
             {
-              oprot.writeString(_iter1562);
+              oprot.writeString(_iter1570);
             }
           }
         }
@@ -124398,13 +124398,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1563 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1563.size);
-            String _elem1564;
-            for (int _i1565 = 0; _i1565 < _list1563.size; ++_i1565)
+            org.apache.thrift.protocol.TList _list1571 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1571.size);
+            String _elem1572;
+            for (int _i1573 = 0; _i1573 < _list1571.size; ++_i1573)
             {
-              _elem1564 = iprot.readString();
-              struct.success.add(_elem1564);
+              _elem1572 = iprot.readString();
+              struct.success.add(_elem1572);
             }
           }
           struct.setSuccessIsSet(true);
@@ -125571,14 +125571,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();
                 }
@@ -125622,9 +125622,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();
           }
@@ -125671,9 +125671,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);
             }
           }
         }
@@ -125691,14 +125691,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(3);
         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);
@@ -126865,14 +126865,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1574 = iprot.readListBegin();
-                  struct.success = new ArrayList<PartitionSpec>(_list1574.size);
-                  PartitionSpec _elem1575;
-                  for (int _i1576 = 0; _i1576 < _list1574.size; ++_i1576)
+                  org.apache.thrift.protocol.TList _list1582 = iprot.readListBegin();
+                  struct.success = new ArrayList<PartitionSpec>(_list1582.size);
+                  PartitionSpec _elem1583;
+                  for (int _i1584 = 0; _i1584 < _list1582.size; ++_i1584)
                   {
-                    _elem1575 = new PartitionSpec();
-                    _elem1575.read(iprot);
-                    struct.success.add(_elem1575);
+                    _elem1583 = new PartitionSpec();
+                    _elem1583.read(iprot);
+                    struct.success.add(_elem1583);
                   }
                   iprot.readListEnd();
                 }
@@ -126916,9 +126916,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 _iter1577 : struct.success)
+            for (PartitionSpec _iter1585 : struct.success)
             {
-              _iter1577.write(oprot);
+              _iter1585.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -126965,9 +126965,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (PartitionSpec _iter1578 : struct.success)
+            for (PartitionSpec _iter1586 : struct.success)
             {
-              _iter1578.write(oprot);
+              _iter1586.write(oprot);
             }
           }
         }
@@ -126985,14 +126985,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1579 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<PartitionSpec>(_list1579.size);
-            PartitionSpec _elem1580;
-            for (int _i1581 = 0; _i1581 < _list1579.size; ++_i1581)
+            org.apache.thrift.protocol.TList _list1587 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<PartitionSpec>(_list1587.size);
+            PartitionSpec _elem1588;
+            for (int _i1589 = 0; _i1589 < _list1587.size; ++_i1589)
             {
-              _elem1580 = new PartitionSpec();
-              _elem1580.read(iprot);
-              struct.success.add(_elem1580);
+              _elem1588 = new PartitionSpec();
+              _elem1588.read(iprot);
+              struct.success.add(_elem1588);
             }
           }
           struct.setSuccessIsSet(true);
@@ -129576,13 +129576,13 @@ import org.slf4j.LoggerFactory;
             case 3: // NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1582 = iprot.readListBegin();
-                  struct.names = new ArrayList<String>(_list1582.size);
-                  String _elem1583;
-                  for (int _i1584 = 0; _i1584 < _list1582.size; ++_i1584)
+                  org.apache.thrift.protocol.TList _list1590 = iprot.readListBegin();
+                  struct.names = new ArrayList<String>(_list1590.size);
+                  String _elem1591;
+                  for (int _i1592 = 0; _i1592 < _list1590.size; ++_i1592)
                   {
-                    _elem1583 = iprot.readString();
-                    struct.names.add(_elem1583);
+                    _elem1591 = iprot.readString();
+                    struct.names.add(_elem1591);
                   }
                   iprot.readListEnd();
                 }
@@ -129618,9 +129618,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 _iter1585 : struct.names)
+            for (String _iter1593 : struct.names)
             {
-              oprot.writeString(_iter1585);
+              oprot.writeString(_iter1593);
             }
             oprot.writeListEnd();
           }
@@ -129663,9 +129663,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetNames()) {
           {
             oprot.writeI32(struct.names.size());
-            for (String _iter1586 : struct.names)
+            for (String _iter1594 : struct.names)
             {
-              oprot.writeString(_iter1586);
+              oprot.writeString(_iter1594);
             }
           }
         }
@@ -129685,13 +129685,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1587 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.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.names = new ArrayList<String>(_list1595.size);
+            String _elem1596;
+            for (int _i1597 = 0; _i1597 < _list1595.size; ++_i1597)
             {
-              _elem1588 = iprot.readString();
-              struct.names.add(_elem1588);
+              _elem1596 = iprot.readString();
+              struct.names.add(_elem1596);
             }
           }
           struct.setNamesIsSet(true);
@@ -130178,14 +130178,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();
                 }
@@ -130229,9 +130229,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();
           }
@@ -130278,9 +130278,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);
             }
           }
         }
@@ -130298,14 +130298,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);
@@ -132793,14 +132793,14 @@ import org.slf4j.LoggerFactory;
             case 3: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1598 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<Partition>(_list1598.size);
-                  Partition _elem1599;
-                  for (int _i1600 = 0; _i1600 < _list1598.size; ++_i1600)
+                  org.apache.thrift.protocol.TList _list1606 = iprot.readListBegin();
+                  struct.new_parts = new ArrayList<Partition>(_list1606.size);
+                  Partition _elem1607;
+                  for (int _i1608 = 0; _i1608 < _list1606.size; ++_i1608)
                   {
-                    _elem1599 = new Partition();
-                    _elem1599.read(iprot);
-                    struct.new_parts.add(_elem1599);
+                    _elem1607 = new Partition();
+                    _elem1607.read(iprot);
+                    struct.new_parts.add(_elem1607);
                   }
                   iprot.readListEnd();
                 }
@@ -132836,9 +132836,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 _iter1601 : struct.new_parts)
+            for (Partition _iter1609 : struct.new_parts)
             {
-              _iter1601.write(oprot);
+              _iter1609.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -132881,9 +132881,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (Partition _iter1602 : struct.new_parts)
+            for (Partition _iter1610 : struct.new_parts)
             {
-              _iter1602.write(oprot);
+              _iter1610.write(oprot);
             }
           }
         }
@@ -132903,14 +132903,14 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1603 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.new_parts = new ArrayList<Partition>(_list1603.size);
-            Partition _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.STRUCT, iprot.readI32());
+            struct.new_parts = new ArrayList<Partition>(_list1611.size);
+            Partition _elem1612;
+            for (int _i1613 = 0; _i1613 < _list1611.size; ++_i1613)
             {
-              _elem1604 = new Partition();
-              _elem1604.read(iprot);
-              struct.new_parts.add(_elem1604);
+              _elem1612 = new Partition();
+              _elem1612.read(iprot);
+              struct.new_parts.add(_elem1612);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -133963,14 +133963,14 @@ import org.slf4j.LoggerFactory;
             case 3: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1606 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<Partition>(_list1606.size);
-                  Partition _elem1607;
-                  for (int _i1608 = 0; _i1608 < _list1606.size; ++_i1608)
+                  org.apache.thrift.protocol.TList _list1614 = iprot.readListBegin();
+                  struct.new_parts = new ArrayList<Partition>(_list1614.size);
+                  Partition _elem1615;
+                  for (int _i1616 = 0; _i1616 < _list1614.size; ++_i1616)
                   {
-                    _elem1607 = new Partition();
-                    _elem1607.read(iprot);
-                    struct.new_parts.add(_elem1607);
+                    _elem1615 = new Partition();
+                    _elem1615.read(iprot);
+                    struct.new_parts.add(_elem1615);
                   }
                   iprot.readListEnd();
                 }
@@ -134015,9 +134015,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 _iter1609 : struct.new_parts)
+            for (Partition _iter1617 : struct.new_parts)
             {
-              _iter1609.write(oprot);
+              _iter1617.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -134068,9 +134068,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (Partition _iter1610 : struct.new_parts)
+            for (Partition _iter1618 : struct.new_parts)
             {
-              _iter1610.write(oprot);
+              _iter1618.write(oprot);
             }
           }
         }
@@ -134093,14 +134093,14 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1611 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.new_parts = 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.new_parts = new ArrayList<Partition>(_list1619.size);
+            Partition _elem1620;
+            for (int _i1621 = 0; _i1621 < _list1619.size; ++_i1621)
             {
-              _elem1612 = new Partition();
-              _elem1612.read(iprot);
-              struct.new_parts.add(_elem1612);
+              _elem1620 = new Partition();
+              _elem1620.read(iprot);
+              struct.new_parts.add(_elem1620);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -137239,13 +137239,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1614 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list1614.size);
-                  String _elem1615;
-                  for (int _i1616 = 0; _i1616 < _list1614.size; ++_i1616)
+                  org.apache.thrift.protocol.TList _list1622 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list1622.size);
+                  String _elem1623;
+                  for (int _i1624 = 0; _i1624 < _list1622.size; ++_i1624)
                   {
-                    _elem1615 = iprot.readString();
-                    struct.part_vals.add(_elem1615);
+                    _elem1623 = iprot.readString();
+                    struct.part_vals.add(_elem1623);
                   }
                   iprot.readListEnd();
                 }
@@ -137290,9 +137290,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 _iter1617 : struct.part_vals)
+            for (String _iter1625 : struct.part_vals)
             {
-              oprot.writeString(_iter1617);
+              oprot.writeString(_iter1625);
             }
             oprot.writeListEnd();
           }
@@ -137343,9 +137343,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter1618 : struct.part_vals)
+            for (String _iter1626 : struct.part_vals)
             {
-              oprot.writeString(_iter1618);
+              oprot.writeString(_iter1626);
             }
           }
         }
@@ -137368,13 +137368,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1619 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list1619.size);
-            String _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.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list1627.size);
+            String _elem1628;
+            for (int _i1629 = 0; _i1629 < _list1627.size; ++_i1629)
             {
-              _elem1620 = iprot.readString();
-              struct.part_vals.add(_elem1620);
+              _elem1628 = iprot.readString();
+              struct.part_vals.add(_elem1628);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -139186,13 +139186,13 @@ import org.slf4j.LoggerFactory;
             case 1: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1622 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list1622.size);
-                  String _elem1623;
-                  for (int _i1624 = 0; _i1624 < _list1622.size; ++_i1624)
+                  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)
                   {
-                    _elem1623 = iprot.readString();
-                    struct.part_vals.add(_elem1623);
+                    _elem1631 = iprot.readString();
+                    struct.part_vals.add(_elem1631);
                   }
                   iprot.readListEnd();
                 }
@@ -139226,9 +139226,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 _iter1625 : struct.part_vals)
+            for (String _iter1633 : struct.part_vals)
             {
-              oprot.writeString(_iter1625);
+              oprot.writeString(_iter1633);
             }
             oprot.writeListEnd();
           }
@@ -139265,9 +139265,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter1626 : struct.part_vals)
+            for (String _iter1634 : struct.part_vals)
             {
-              oprot.writeString(_iter1626);
+              oprot.writeString(_iter1634);
             }
           }
         }
@@ -139282,13 +139282,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         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.part_vals = 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.part_vals = new ArrayList<String>(_list1635.size);
+            String _elem1636;
+            for (int _i1637 = 0; _i1637 < _list1635.size; ++_i1637)
             {
-              _elem1628 = iprot.readString();
-              struct.part_vals.add(_elem1628);
+              _elem1636 = iprot.readString();
+              struct.part_vals.add(_elem1636);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -141443,13 +141443,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  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)
+                  org.apache.thrift.protocol.TList _list1638 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1638.size);
+                  String _elem1639;
+                  for (int _i1640 = 0; _i1640 < _list1638.size; ++_i1640)
                   {
-                    _elem1631 = iprot.readString();
-                    struct.success.add(_elem1631);
+                    _elem1639 = iprot.readString();
+                    struct.success.add(_elem1639);
                   }
                   iprot.readListEnd();
                 }
@@ -141484,9 +141484,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 _iter1633 : struct.success)
+            for (String _iter1641 : struct.success)
             {
-              oprot.writeString(_iter1633);
+              oprot.writeString(_iter1641);
             }
             oprot.writeListEnd();
           }
@@ -141525,9 +141525,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1634 : struct.success)
+            for (String _iter1642 : struct.success)
             {
-              oprot.writeString(_iter1634);
+              oprot.writeString(_iter1642);
             }
           }
         }
@@ -141542,13 +141542,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            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)
+            org.apache.thrift.protocol.TList _list1643 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1643.size);
+            String _elem1644;
+            for (int _i1645 = 0; _i1645 < _list1643.size; ++_i1645)
             {
-              _elem1636 = iprot.readString();
-              struct.success.add(_elem1636);
+              _elem1644 = iprot.readString();
+              struct.success.add(_elem1644);
             }
           }
           struct.setSuccessIsSet(true);
@@ -142311,15 +142311,15 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map1638 = iprot.readMapBegin();
-                  struct.success = new HashMap<String,String>(2*_map1638.size);
-                  String _key1639;
-                  String _val1640;
-                  for (int _i1641 = 0; _i1641 < _map1638.size; ++_i1641)
+                  org.apache.thrift.protocol.TMap _map1646 = iprot.readMapBegin();
+                  struct.success = new HashMap<String,String>(2*_map1646.size);
+                  String _key1647;
+                  String _val1648;
+                  for (int _i1649 = 0; _i1649 < _map1646.size; ++_i1649)
                   {
-                    _key1639 = iprot.readString();
-                    _val1640 = iprot.readString();
-                    struct.success.put(_key1639, _val1640);
+                    _key1647 = iprot.readString();
+                    _val1648 = iprot.readString();
+                    struct.success.put(_key1647, _val1648);
                   }
                   iprot.readMapEnd();
                 }
@@ -142354,10 +142354,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> _iter1642 : struct.success.entrySet())
+            for (Map.Entry<String, String> _iter1650 : struct.success.entrySet())
             {
-              oprot.writeString(_iter1642.getKey());
-              oprot.writeString(_iter1642.getValue());
+              oprot.writeString(_iter1650.getKey());
+              oprot.writeString(_iter1650.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -142396,10 +142396,10 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Map.Entry<String, String> _iter1643 : struct.success.entrySet())
+            for (Map.Entry<String, String> _iter1651 : struct.success.entrySet())
             {
-              oprot.writeString(_iter1643.getKey());
-              oprot.writeString(_iter1643.getValue());
+              oprot.writeString(_iter1651.getKey());
+              oprot.writeString(_iter1651.getValue());
             }
           }
         }
@@ -142414,15 +142414,15 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map1644 = 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*_map1644.size);
-            String _key1645;
-            String _val1646;
-            for (int _i1647 = 0; _i1647 < _map1644.size; ++_i1647)
+            org.apache.thrift.protocol.TMap _map1652 = 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*_map1652.size);
+            String _key1653;
+            String _val1654;
+            for (int _i1655 = 0; _i1655 < _map1652.size; ++_i1655)
             {
-              _key1645 = iprot.readString();
-              _val1646 = iprot.readString();
-              struct.success.put(_key1645, _val1646);
+              _key1653 = iprot.readString();
+              _val1654 = iprot.readString();
+              struct.success.put(_key1653, _val1654);
             }
           }
           struct.setSuccessIsSet(true);
@@ -143017,15 +143017,15 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map1648 = iprot.readMapBegin();
-                  struct.part_vals = new HashMap<String,String>(2*_map1648.size);
-                  String _key1649;
-                  String _val1650;
-                  for (int _i1651 = 0; _i1651 < _map1648.size; ++_i1651)
+                  org.apache.thrift.protocol.TMap _map1656 = iprot.readMapBegin();
+                  struct.part_vals = new HashMap<String,String>(2*_map1656.size);
+                  String _key1657;
+                  String _val1658;
+                  for (int _i1659 = 0; _i1659 < _map1656.size; ++_i1659)
                   {
-                    _key1649 = iprot.readString();
-                    _val1650 = iprot.readString();
-                    struct.part_vals.put(_key1649, _val1650);
+                    _key1657 = iprot.readString();
+                    _val1658 = iprot.readString();
+                    struct.part_vals.put(_key1657, _val1658);
                   }
                   iprot.readMapEnd();
                 }
@@ -143069,10 +143069,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> _iter1652 : struct.part_vals.entrySet())
+            for (Map.Entry<String, String> _iter1660 : struct.part_vals.entrySet())
             {
-              oprot.writeString(_iter1652.getKey());
-              oprot.writeString(_iter1652.getValue());
+              oprot.writeString(_iter1660.getKey());
+              oprot.writeString(_iter1660.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -143123,10 +143123,10 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (Map.Entry<String, String> _iter1653 : struct.part_vals.entrySet())
+            for (Map.Entry<String, String> _iter1661 : struct.part_vals.entrySet())
             {
-              oprot.writeString(_iter1653.getKey());
-              oprot.writeString(_iter1653.getValue());
+              oprot.writeString(_iter1661.getKey());
+              oprot.writeString(_iter1661.getValue());
             }
           }
         }
@@ -143149,15 +143149,15 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TMap _map1654 = 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*_map1654.size);
-            String _key1655;
-            String _val1656;
-            for (int _i1657 = 0; _i1657 < _map1654.size; ++_i1657)
+            org.apache.thrift.protocol.TMap _map1662 = 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*_map1662.size);
+            String _key1663;
+            String _val1664;
+            for (int _i1665 = 0; _i1665 < _map1662.size; ++_i1665)
             {
-              _key1655 = iprot.readString();
-              _val1656 = iprot.readString();
-              struct.part_vals.put(_key1655, _val1656);
+              _key1663 = iprot.readString();
+              _val1664 = iprot.readString();
+              struct.part_vals.put(_key1663, _val1664);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -144641,15 +144641,15 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map1658 = iprot.readMapBegin();
-                  struct.part_vals = new HashMap<String,String>(2*_map1658.size);
-                  String _key1659;
-                  String _val1660;
-                  for (int _i1661 = 0; _i1661 < _map1658.size; ++_i1661)
+                  org.apache.thrift.protocol.TMap _map1666 = iprot.readMapBegin();
+                  struct.part_vals = new HashMap<String,String>(2*_map1666.size);
+                  String _key1667;
+                  String _val1668;
+                  for (int _i1669 = 0; _i1669 < _map1666.size; ++_i1669)
                   {
-                    _key1659 = iprot.readString();
-                    _val1660 = iprot.readString();
-                    struct.part_vals.put(_key1659, _val1660);
+                    _key1667 = iprot.readString();
+                    _val1668 = iprot.readString();
+                    struct.part_vals.put(_key1667, _val1668);
                   }
                   iprot.readMapEnd();
                 }
@@ -144693,10 +144693,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> _iter1662 : struct.part_vals.entrySet())
+            for (Map.Entry<String, String> _iter1670 : struct.part_vals.entrySet())
             {
-              oprot.writeString(_iter1662.getKey());
-              oprot.writeString(_iter1662.getValue());
+              oprot.writeString(_iter1670.getKey());
+              oprot.writeString(_iter1670.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -144747,10 +144747,10 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (Map.Entry<String, String> _iter1663 : struct.part_vals.entrySet())
+            for (Map.Entry<String, String> _iter1671 : struct.part_vals.entrySet())
             {
-              oprot.writeString(_iter1663.getKey());
-              oprot.writeString(_iter1663.getValue());
+              oprot.writeString(_iter1671.getKey());
+              oprot.writeString(_iter1671.getValue());
             }
           }
         }
@@ -144773,15 +144773,15 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TMap _map1664 = 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*_map1664.size);
-            String _key1665;
-            String _val1666;
-            for (int _i1667 = 0; _i1667 < _map1664.size; ++_i1667)
+            org.apache.thrift.protocol.TMap _map1672 = 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*_map1672.size);
+            String _key1673;
+            String _val1674;
+            for (int _i1675 = 0; _i1675 < _map1672.size; ++_i1675)
             {
-              _key1665 = iprot.readString();
-              _val1666 = iprot.readString();
-              struct.part_vals.put(_key1665, _val1666);
+              _key1673 = iprot.readString();
+              _val1674 = iprot.readString();
+              struct.part_vals.put(_key1673, _val1674);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -169645,13 +169645,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1668 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1668.size);
-                  String _elem1669;
-                  for (int _i1670 = 0; _i1670 < _list1668.size; ++_i1670)
+                  org.apache.thrift.protocol.TList _list1676 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1676.size);
+                  String _elem1677;
+                  for (int _i1678 = 0; _i1678 < _list1676.size; ++_i1678)
                   {
-                    _elem1669 = iprot.readString();
-                    struct.success.add(_elem1669);
+                    _elem1677 = iprot.readString();
+                    struct.success.add(_elem1677);
                   }
                   iprot.readListEnd();
                 }
@@ -169686,9 +169686,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 _iter1671 : struct.success)
+            for (String _iter1679 : struct.success)
             {
-              oprot.writeString(_iter1671);
+              oprot.writeString(_iter1679);
             }
             oprot.writeListEnd();
           }
@@ -169727,9 +169727,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1672 : struct.success)
+            for (String _iter1680 : struct.success)
             {
-              oprot.writeString(_iter1672);
+              oprot.writeString(_iter1680);
             }
           }
         }
@@ -169744,13 +169744,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1673 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1673.size);
-            String _elem1674;
-            for (int _i1675 = 0; _i1675 < _list1673.size; ++_i1675)
+            org.apache.thrift.protocol.TList _list1681 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1681.size);
+            String _elem1682;
+            for (int _i1683 = 0; _i1683 < _list1681.size; ++_i1683)
             {
-              _elem1674 = iprot.readString();
-              struct.success.add(_elem1674);
+              _elem1682 = iprot.readString();
+              struct.success.add(_elem1682);
             }
           }
           struct.setSuccessIsSet(true);
@@ -173805,13 +173805,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1676 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1676.size);
-                  String _elem1677;
-                  for (int _i1678 = 0; _i1678 < _list1676.size; ++_i1678)
+                  org.apache.thrift.protocol.TList _list1684 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1684.size);
+                  String _elem1685;
+                  for (int _i1686 = 0; _i1686 < _list1684.size; ++_i1686)
                   {
-                    _elem1677 = iprot.readString();
-                    struct.success.add(_elem1677);
+                    _elem1685 = iprot.readString();
+                    struct.success.add(_elem1685);
                   }
                   iprot.readListEnd();
                 }
@@ -173846,9 +173846,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 _iter1679 : struct.success)
+            for (String _iter1687 : struct.success)
             {
-              oprot.writeString(_iter1679);
+              oprot.writeString(_iter1687);
             }
             oprot.writeListEnd();
           }
@@ -173887,9 +173887,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1680 : struct.success)
+            for (String _iter1688 : struct.success)
             {
-              oprot.writeString(_iter1680);
+              oprot.writeString(_iter1688);
             }
           }
         }
@@ -173904,13 +173904,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1681 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1681.size);
-            String _elem1682;
-            for (int _i1683 = 0; _i1683 < _list1681.size; ++_i1683)
+            org.apache.thrift.protocol.TList _list1689 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1689.size);
+            String _elem1690;
+            for (int _i1691 = 0; _i1691 < _list1689.size; ++_i1691)
             {
-              _elem1682 = iprot.readString();
-              struct.success.add(_elem1682);
+              _elem1690 = iprot.readString();
+              struct.success.add(_elem1690);
             }
           }
           struct.setSuccessIsSet(true);
@@ -177201,14 +177201,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1684 = iprot.readListBegin();
-                  struct.success = new ArrayList<Role>(_list1684.size);
-                  Role _elem1685;
-                  for (int _i1686 = 0; _i1686 < _list1684.size; ++_i1686)
+                  org.apache.thrift.protocol.TList _list1692 = iprot.readListBegin();
+                  struct.success = new ArrayList<Role>(_list1692.size);
+                  Role _elem1693;
+                  for (int _i1694 = 0; _i1694 < _list1692.size; ++_i1694)
                   {
-                    _elem1685 = new Role();
-                    _elem1685.read(iprot);
-                    struct.success.add(_elem1685);
+                    _elem1693 = new Role();
+                    _elem1693.read(iprot);
+                    struct.success.add(_elem1693);
                   }
                   iprot.readListEnd();
                 }
@@ -177243,9 +177243,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 _iter1687 : struct.success)
+            for (Role _iter1695 : struct.success)
             {
-              _iter1687.write(oprot);
+              _iter1695.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -177284,9 +177284,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Role _iter1688 : struct.success)
+            for (Role _iter1696 : struct.success)
             {
-              _iter1688.write(oprot);
+              _iter1696.write(oprot);
             }
           }
         }
@@ -177301,14 +177301,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1689 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Role>(_list1689.size);
-            Role _elem1690;
-            for (int _i1691 = 0; _i1691 < _list1689.size; ++_i1691)
+            org.apache.thrift.protocol.TList _list1697 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Role>(_list1697.size);
+            Role _elem1698;
+            for (int _i1699 = 0; _i1699 < _list1697.size; ++_i1699)
             {
-              _elem1690 = new Role();
-              _elem1690.read(iprot);
-              struct.success.add(_elem1690);
+              _elem1698 = new Role();
+              _elem1698.read(iprot);
+              struct.success.add(_elem1698);
             }
           }
           struct.setSuccessIsSet(true);
@@ -180313,13 +180313,13 @@ import org.slf4j.LoggerFactory;
             case 3: // GROUP_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1692 = iprot.readListBegin();
-                  struct.group_names = new ArrayList<String>(_list1692.size);
-                  String _elem1693;
-                  for (int _i1694 = 0; _i1694 < _list1692.size; ++_i1694)
+                  org.apache.thrift.protocol.TList _list1700 = iprot.readListBegin();
+                  struct.group_names = new ArrayList<String>(_list1700.size);
+                  String _elem1701;
+                  for (int _i1702 = 0; _i1702 < _list1700.size; ++_i1702)
                   {
-                    _elem1693 = iprot.readString();
-                    struct.group_names.add(_elem1693);
+                    _elem1701 = iprot.readString();
+                    struct.group_names.add(_elem1701);
                   }
                   iprot.readListEnd();
                 }
@@ -180355,9 +180355,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 _iter1695 : struct.group_names)
+            for (String _iter1703 : struct.group_names)
             {
-              oprot.writeString(_iter1695);
+              oprot.writeString(_iter1703);
             }
             oprot.writeListEnd();
           }
@@ -180400,9 +180400,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetGroup_names()) {
           {
             oprot.writeI32(struct.group_names.size());
-            for (String _iter1696 : struct.group_names)
+            for (String _iter1704 : struct.group_names)
             {
-              oprot.writeString(_iter1696);
+              oprot.writeString(_iter1704);
             }
           }
         }
@@ -180423,13 +180423,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1697 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.group_names = new ArrayList<String>(_list1697.size);
-            String _elem1698;
-            for (int _i1699 = 0; _i1699 < _list1697.size; ++_i1699)
+            org.apache.thrift.protocol.TList _list1705 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.group_names = new ArrayList<String>(_list1705.size);
+            String _elem1706;
+            for (int _i1707 = 0; _i1707 < _list1705.size; ++_i1707)
             {
-              _elem1698 = iprot.readString();
-              struct.group_names.add(_elem1698);
+              _elem1706 = iprot.readString();
+              struct.group_names.add(_elem1706);
             }
           }
           struct.setGroup_namesIsSet(true);
@@ -181887,14 +181887,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1700 = iprot.readListBegin();
-                  struct.success = new ArrayList<HiveObjectPrivilege>(_list1700.size);
-                  HiveObjectPrivilege _elem1701;
-                  for (int _i1702 = 0; _i1702 < _list1700.size; ++_i1702)
+                  org.apache.thrift.protocol.TList _list1708 = iprot.readListBegin();
+                  struct.success = new ArrayList<HiveObjectPrivilege>(_list1708.size);
+                  HiveObjectPrivilege _elem1709;
+                  for (int _i1710 = 0; _i1710 < _list1708.size; ++_i1710)
                   {
-                    _elem1701 = new HiveObjectPrivilege();
-                    _elem1701.read(iprot);
-                    struct.success.add(_elem1701);
+                    _elem1709 = new HiveObjectPrivilege();
+                    _elem1709.read(iprot);
+                    struct.success.add(_elem1709);
                   }
                   iprot.readListEnd();
                 }
@@ -181929,9 +181929,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 _iter1703 : struct.success)
+            for (HiveObjectPrivilege _iter1711 : struct.success)
             {
-              _iter1703.write(oprot);
+              _iter1711.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -181970,9 +181970,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (HiveObjectPrivilege _iter1704 : struct.success)
+            for (HiveObjectPrivilege _iter1712 : struct.success)
             {
-              _iter1704.write(oprot);
+              _iter1712.write(oprot);
             }
           }
         }
@@ -181987,14 +181987,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1705 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<HiveObjectPrivilege>(_list1705.size);
-            HiveObjectPrivilege _elem1706;
-            for (int _i1707 = 0; _i1707 < _list1705.size; ++_i1707)
+            org.apache.thrift.protocol.TList _list1713 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<HiveObjectPrivilege>(_list1713.size);
+            HiveObjectPrivilege _elem1714;
+            for (int _i1715 = 0; _i1715 < _list1713.size; ++_i1715)
             {
-              _elem1706 = new HiveObjectPrivilege();
-              _elem1706.read(iprot);
-              struct.success.add(_elem1706);
+              _elem1714 = new HiveObjectPrivilege();
+              _elem1714.read(iprot);
+              struct.success.add(_elem1714);
             }
           }
           struct.setSuccessIsSet(true);
@@ -185941,13 +185941,13 @@ import org.slf4j.LoggerFactory;
             case 2: // GROUP_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1708 = iprot.readListBegin();
-                  struct.group_names = new ArrayList<String>(_list1708.size);
-                  String _elem1709;
-                  for (int _i1710 = 0; _i1710 < _list1708.size; ++_i1710)
+                  org.apache.thrift.protocol.TList _list1716 = iprot.readListBegin();
+                  struct.group_names = new ArrayList<String>(_list1716.size);
+                  String _elem1717;
+                  for (int _i1718 = 0; _i1718 < _list1716.size; ++_i1718)
                   {
-                    _elem1709 = iprot.readString();
-                    struct.group_names.add(_elem1709);
+                    _elem1717 = iprot.readString();
+                    struct.group_names.add(_elem1717);
                   }
                   iprot.readListEnd();
                 }
@@ -185978,9 +185978,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 _iter1711 : struct.group_names)
+            for (String _iter1719 : struct.group_names)
             {
-              oprot.writeString(_iter1711);
+              oprot.writeString(_iter1719);
             }
             oprot.writeListEnd();
           }
@@ -186017,9 +186017,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetGroup_names()) {
           {
             oprot.writeI32(struct.group_names.size());
-            for (String _iter1712 : struct.group_names)
+            for (String _iter1720 : struct.group_names)
             {
-              oprot.writeString(_iter1712);
+              oprot.writeString(_iter1720);
             }
           }
         }
@@ -186035,13 +186035,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list1713 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.group_names = new ArrayList<String>(_list1713.size);
-            String _elem1714;
-            for (int _i1715 = 0; _i1715 < _list1713.size; ++_i1715)
+            org.apache.thrift.protocol.TList _list1721 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.group_names = new ArrayList<String>(_list1721.size);
+            String _elem1722;
+            for (int _i1723 = 0; _i1723 < _list1721.size; ++_i1723)
             {
-              _elem1714 = iprot.readString();
-              struct.group_names.add(_elem1714);
+              _elem1722 = iprot.readString();
+              struct.group_names.add(_elem1722);
             }
           }
           struct.setGroup_namesIsSet(true);
@@ -186444,13 +186444,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1716 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1716.size);
-                  String _elem1717;
-                  for (int _i1718 = 0; _i1718 < _list1716.size; ++_i1718)
+                  org.apache.thrift.protocol.TList _list1724 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1724.size);
+                  String _elem1725;
+                  for (int _i1726 = 0; _i1726 < _list1724.size; ++_i1726)
                   {
-                    _elem1717 = iprot.readString();
-                    struct.success.add(_elem1717);
+                    _elem1725 = iprot.readString();
+                    struct.success.add(_elem1725);
                   }
                   iprot.readListEnd();
                 }
@@ -186485,9 +186485,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 _iter1719 : struct.success)
+            for (String _iter1727 : struct.success)
             {
-              oprot.writeString(_iter1719);
+              oprot.writeString(_iter1727);
             }
             oprot.writeListEnd();
           }
@@ -186526,9 +186526,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1720 : struct.success)
+            for (String _iter1728 : struct.success)
             {
-              oprot.writeString(_iter1720);
+              oprot.writeString(_iter1728);
             }
           }
         }
@@ -186543,13 +186543,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1721 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1721.size);
-            String _elem1722;
-            for (int _i1723 = 0; _i1723 < _list1721.size; ++_i1723)
+            org.apache.thrift.protocol.TList _list1729 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1729.size);
+            String _elem1730;
+            for (int _i1731 = 0; _i1731 < _list1729.size; ++_i1731)
             {
-              _elem1722 = iprot.readString();
-              struct.success.add(_elem1722);
+              _elem1730 = iprot.readString();
+              struct.success.add(_elem1730);
             }
           }
           struct.setSuccessIsSet(true);
@@ -191840,13 +191840,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1724 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1724.size);
-                  String _elem1725;
-                  for (int _i1726 = 0; _i1726 < _list1724.size; ++_i1726)
+                  org.apache.thrift.protocol.TList _list1732 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1732.size);
+                  String _elem1733;
+                  for (int _i1734 = 0; _i1734 < _list1732.size; ++_i1734)
                   {
-                    _elem1725 = iprot.readString();
-                    struct.success.add(_elem1725);
+                    _elem1733 = iprot.readString();
+                    struct.success.add(_elem1733);
                   }
                   iprot.readListEnd();
                 }
@@ -191872,9 +191872,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 _iter1727 : struct.success)
+            for (String _iter1735 : struct.success)
             {
-              oprot.writeString(_iter1727);
+              oprot.writeString(_iter1735);
             }
             oprot.writeListEnd();
           }
@@ -191905,9 +191905,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1728 : struct.success)
+            for (String _iter1736 : struct.success)
             {
-              oprot.writeString(_iter1728);
+              oprot.writeString(_iter1736);
             }
           }
         }
@@ -191919,13 +191919,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1729 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1729.size);
-            String _elem1730;
-            for (int _i1731 = 0; _i1731 < _list1729.size; ++_i1731)
+            org.apache.thrift.protocol.TList _list1737 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1737.size);
+            String _elem1738;
+            for (int _i1739 = 0; _i1739 < _list1737.size; ++_i1739)
             {
-              _elem1730 = iprot.readString();
-              struct.success.add(_elem1730);
+              _elem1738 = iprot.readString();
+              struct.success.add(_elem1738);
             }
           }
           struct.setSuccessIsSet(true);
@@ -194955,13 +194955,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1732 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1732.size);
-                  String _elem1733;
-                  for (int _i1734 = 0; _i1734 < _list1732.size; ++_i1734)
+                  org.apache.thrift.protocol.TList _list1740 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1740.size);
+                  String _elem1741;
+                  for (int _i1742 = 0; _i1742 < _list1740.size; ++_i1742)
                   {
-                    _elem1733 = iprot.readString();
-                    struct.success.add(_elem1733);
+                    _elem1741 = iprot.readString();
+                    struct.success.add(_elem1741);
                   }
                   iprot.readListEnd();
                 }
@@ -194987,9 +194987,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 _iter1735 : struct.success)
+            for (String _iter1743 : struct.success)
             {
-              oprot.writeString(_iter1735);
+              oprot.writeString(_iter1743);
             }
             oprot.writeListEnd();
           }
@@ -195020,9 +195020,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1736 : struct.success)
+            for (String _iter1744 : struct.success)
             {
-              oprot.writeString(_iter1736);
+              oprot.writeString(_iter1744);
             }
           }
         }
@@ -195034,13 +195034,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1737 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1737.size);
-            String _elem1738;
-            for (int _i1739 = 0; _i1739 < _list1737.size; ++_i1739)
+            org.apache.thrift.protocol.TList _list1745 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1745.size);
+            String _elem1746;
+            for (int _i1747 = 0; _i1747 < _list1745.size; ++_i1747)
             {
-              _elem1738 = iprot.readString();
-              struct.success.add(_elem1738);
+              _elem1746 = iprot.readString();
+              struct.success.add(_elem1746);
             }
           }
           struct.setSuccessIsSet(true);
@@ -212161,13 +212161,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1740 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1740.size);
-                  String _elem1741;
-                  for (int _i1742 = 0; _i1742 < _list1740.size; ++_i1742)
+                  org.apache.thrift.protocol.TList _list1748 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1748.size);
+                  String _elem1749;
+                  for (int _i1750 = 0; _i1750 < _list1748.size; ++_i1750)
                   {
-                    _elem1741 = iprot.readString();
-                    struct.success.add(_elem1741);
+                    _elem1749 = iprot.readString();
+                    struct.success.add(_elem1749);
                   }
                   iprot.readListEnd();
                 }
@@ -212193,9 +212193,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 _iter1743 : struct.success)
+            for (String _iter1751 : struct.success)
             {
-              oprot.writeString(_iter1743);
+              oprot.writeString(_iter1751);
             }
             oprot.writeListEnd();
           }
@@ -212226,9 +212226,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1744 : struct.success)
+            for (String _iter1752 : struct.success)
             {
-              oprot.writeString(_iter1744);
+              oprot.writeString(_iter1752);
             }
           }
         }
@@ -212240,13 +212240,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1745 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1745.size);
-            String _elem1746;
-            for (int _i1747 = 0; _i1747 < _list1745.size; ++_i1747)
+            org.apache.thrift.protocol.TList _list1753 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1753.size);
+            String _elem1754;
+            for (int _i1755 = 0; _i1755 < _list1753.size; ++_i1755)
             {
-              _elem1746 = iprot.readString();
-              struct.success.add(_elem1746);
+              _elem1754 = iprot.readString();
+              struct.success.add(_elem1754);
             }
           }
           struct.setSuccessIsSet(true);
@@ -249132,14 +249132,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1748 = iprot.readListBegin();
-                  struct.success = new ArrayList<SchemaVersion>(_list1748.size);
-                  SchemaVersion _elem1749;
-                  for (int _i1750 = 0; _i1750 < _list1748.size; ++_i1750)
+                  org.apache.thrift.protocol.TList _list1756 = iprot.readListBegin();
+                  struct.success = new ArrayList<SchemaVersion>(_list1756.size);
+                  SchemaVersion _elem1757;
+                  for (int _i1758 = 0; _i1758 < _list1756.size; ++_i1758)
                   {
-                    _elem1749 = new SchemaVersion();
-                    _elem1749.read(iprot);
-                    struct.success.add(_elem1749);
+                    _elem1757 = new SchemaVersion();
+                    _elem1757.read(iprot);
+                    struct.success.add(_elem1757);
                   }
                   iprot.readListEnd();
                 }
@@ -249183,9 +249183,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 _iter1751 : struct.success)
+            for (SchemaVersion _iter1759 : struct.success)
             {
-              _iter1751.write(oprot);
+              _iter1759.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -249232,9 +249232,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (SchemaVersion _iter1752 : struct.success)
+            for (SchemaVersion _iter1760 : struct.success)
             {
-              _iter1752.write(oprot);
+              _iter1760.write(oprot);
             }
           }
         }
@@ -249252,14 +249252,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1753 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<SchemaVersion>(_list1753.size);
-            SchemaVersion _elem1754;
-            for (int _i1755 = 0; _i1755 < _list1753.size; ++_i1755)
+            org.apache.thrift.protocol.TList _list1761 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<SchemaVersion>(_list1761.size);
+            SchemaVersion _elem1762;
+            for (int _i1763 = 0; _i1763 < _list1761.size; ++_i1763)
             {
-              _elem1754 = new SchemaVersion();
-              _elem1754.read(iprot);
-              struct.success.add(_elem1754);
+              _elem1762 = new SchemaVersion();
+              _elem1762.read(iprot);
+              struct.success.add(_elem1762);
             }
           }
           struct.setSuccessIsSet(true);
@@ -257802,14 +257802,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1756 = iprot.readListBegin();
-                  struct.success = new ArrayList<RuntimeStat>(_list1756.size);
-                  RuntimeStat _elem1757;
-                  for (int _i1758 = 0; _i1758 < _list1756.size; ++_i1758)
+                  org.apache.thrift.protocol.TList _list1764 = iprot.readListBegin();
+                  struct.success = new ArrayList<RuntimeStat>(_list1764.size);
+                  RuntimeStat _elem1765;
+                  for (int _i1766 = 0; _i1766 < _list1764.size; ++_i1766)
                   {
-                    _elem1757 = new RuntimeStat();
-                    _elem1757.read(iprot);
-                    struct.success.add(_elem1757);
+                    _elem1765 = new RuntimeStat();
+                    _elem1765.read(iprot);
+                    struct.success.add(_elem1765);
                   }
                   iprot.readListEnd();
                 }
@@ -257844,9 +257844,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 _iter1759 : struct.success)
+            for (RuntimeStat _iter1767 : struct.success)
             {
-              _iter1759.write(oprot);
+              _iter1767.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -257885,9 +257885,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (RuntimeStat _iter1760 : struct.success)
+            for (RuntimeStat _iter1768 : struct.success)
             {
-              _iter1760.write(oprot);
+              _iter1768.write(oprot);
             }
           }
         }
@@ -257902,14 +257902,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1761 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<RuntimeStat>(_list1761.size);
-            RuntimeStat _elem1762;
-            for (int _i1763 = 0; _i1763 < _list1761.size; ++_i1763)
+            org.apache.thrift.protocol.TList _list1769 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<RuntimeStat>(_list1769.size);
+            RuntimeStat _elem1770;
+            for (int _i1771 = 0; _i1771 < _list1769.size; ++_i1771)
             {
-              _elem1762 = new RuntimeStat();
-              _elem1762.read(iprot);
-              struct.success.add(_elem1762);
+              _elem1770 = new RuntimeStat();
+              _elem1770.read(iprot);
+              struct.success.add(_elem1770);
             }
           }
           struct.setSuccessIsSet(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 ce534f7..d834dbc 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
@@ -16872,14 +16872,14 @@ class ThriftHiveMetastore_get_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1029 = 0;
-            $_etype1032 = 0;
-            $xfer += $input->readListBegin($_etype1032, $_size1029);
-            for ($_i1033 = 0; $_i1033 < $_size1029; ++$_i1033)
+            $_size1036 = 0;
+            $_etype1039 = 0;
+            $xfer += $input->readListBegin($_etype1039, $_size1036);
+            for ($_i1040 = 0; $_i1040 < $_size1036; ++$_i1040)
             {
-              $elem1034 = null;
-              $xfer += $input->readString($elem1034);
-              $this->success []= $elem1034;
+              $elem1041 = null;
+              $xfer += $input->readString($elem1041);
+              $this->success []= $elem1041;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16915,9 +16915,9 @@ class ThriftHiveMetastore_get_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1035)
+          foreach ($this->success as $iter1042)
           {
-            $xfer += $output->writeString($iter1035);
+            $xfer += $output->writeString($iter1042);
           }
         }
         $output->writeListEnd();
@@ -17048,14 +17048,14 @@ class ThriftHiveMetastore_get_all_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1036 = 0;
-            $_etype1039 = 0;
-            $xfer += $input->readListBegin($_etype1039, $_size1036);
-            for ($_i1040 = 0; $_i1040 < $_size1036; ++$_i1040)
+            $_size1043 = 0;
+            $_etype1046 = 0;
+            $xfer += $input->readListBegin($_etype1046, $_size1043);
+            for ($_i1047 = 0; $_i1047 < $_size1043; ++$_i1047)
             {
-              $elem1041 = null;
-              $xfer += $input->readString($elem1041);
-              $this->success []= $elem1041;
+              $elem1048 = null;
+              $xfer += $input->readString($elem1048);
+              $this->success []= $elem1048;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17091,9 +17091,9 @@ class ThriftHiveMetastore_get_all_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1042)
+          foreach ($this->success as $iter1049)
           {
-            $xfer += $output->writeString($iter1042);
+            $xfer += $output->writeString($iter1049);
           }
         }
         $output->writeListEnd();
@@ -18094,18 +18094,18 @@ class ThriftHiveMetastore_get_type_all_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size1043 = 0;
-            $_ktype1044 = 0;
-            $_vtype1045 = 0;
-            $xfer += $input->readMapBegin($_ktype1044, $_vtype1045, $_size1043);
-            for ($_i1047 = 0; $_i1047 < $_size1043; ++$_i1047)
+            $_size1050 = 0;
+            $_ktype1051 = 0;
+            $_vtype1052 = 0;
+            $xfer += $input->readMapBegin($_ktype1051, $_vtype1052, $_size1050);
+            for ($_i1054 = 0; $_i1054 < $_size1050; ++$_i1054)
             {
-              $key1048 = '';
-              $val1049 = new \metastore\Type();
-              $xfer += $input->readString($key1048);
-              $val1049 = new \metastore\Type();
-              $xfer += $val1049->read($input);
-              $this->success[$key1048] = $val1049;
+              $key1055 = '';
+              $val1056 = new \metastore\Type();
+              $xfer += $input->readString($key1055);
+              $val1056 = new \metastore\Type();
+              $xfer += $val1056->read($input);
+              $this->success[$key1055] = $val1056;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -18141,10 +18141,10 @@ class ThriftHiveMetastore_get_type_all_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $kiter1050 => $viter1051)
+          foreach ($this->success as $kiter1057 => $viter1058)
           {
-            $xfer += $output->writeString($kiter1050);
-            $xfer += $viter1051->write($output);
+            $xfer += $output->writeString($kiter1057);
+            $xfer += $viter1058->write($output);
           }
         }
         $output->writeMapEnd();
@@ -18348,15 +18348,15 @@ class ThriftHiveMetastore_get_fields_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1052 = 0;
-            $_etype1055 = 0;
-            $xfer += $input->readListBegin($_etype1055, $_size1052);
-            for ($_i1056 = 0; $_i1056 < $_size1052; ++$_i1056)
+            $_size1059 = 0;
+            $_etype1062 = 0;
+            $xfer += $input->readListBegin($_etype1062, $_size1059);
+            for ($_i1063 = 0; $_i1063 < $_size1059; ++$_i1063)
             {
-              $elem1057 = null;
-              $elem1057 = new \metastore\FieldSchema();
-              $xfer += $elem1057->read($input);
-              $this->success []= $elem1057;
+              $elem1064 = null;
+              $elem1064 = new \metastore\FieldSchema();
+              $xfer += $elem1064->read($input);
+              $this->success []= $elem1064;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18408,9 +18408,9 @@ class ThriftHiveMetastore_get_fields_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1058)
+          foreach ($this->success as $iter1065)
           {
-            $xfer += $iter1058->write($output);
+            $xfer += $iter1065->write($output);
           }
         }
         $output->writeListEnd();
@@ -18652,15 +18652,15 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1059 = 0;
-            $_etype1062 = 0;
-            $xfer += $input->readListBegin($_etype1062, $_size1059);
-            for ($_i1063 = 0; $_i1063 < $_size1059; ++$_i1063)
+            $_size1066 = 0;
+            $_etype1069 = 0;
+            $xfer += $input->readListBegin($_etype1069, $_size1066);
+            for ($_i1070 = 0; $_i1070 < $_size1066; ++$_i1070)
             {
-              $elem1064 = null;
-              $elem1064 = new \metastore\FieldSchema();
-              $xfer += $elem1064->read($input);
-              $this->success []= $elem1064;
+              $elem1071 = null;
+              $elem1071 = new \metastore\FieldSchema();
+              $xfer += $elem1071->read($input);
+              $this->success []= $elem1071;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18712,9 +18712,9 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1065)
+          foreach ($this->success as $iter1072)
           {
-            $xfer += $iter1065->write($output);
+            $xfer += $iter1072->write($output);
           }
         }
         $output->writeListEnd();
@@ -18928,15 +18928,15 @@ class ThriftHiveMetastore_get_schema_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1066 = 0;
-            $_etype1069 = 0;
-            $xfer += $input->readListBegin($_etype1069, $_size1066);
-            for ($_i1070 = 0; $_i1070 < $_size1066; ++$_i1070)
+            $_size1073 = 0;
+            $_etype1076 = 0;
+            $xfer += $input->readListBegin($_etype1076, $_size1073);
+            for ($_i1077 = 0; $_i1077 < $_size1073; ++$_i1077)
             {
-              $elem1071 = null;
-              $elem1071 = new \metastore\FieldSchema();
-              $xfer += $elem1071->read($input);
-              $this->success []= $elem1071;
+              $elem1078 = null;
+              $elem1078 = new \metastore\FieldSchema();
+              $xfer += $elem1078->read($input);
+              $this->success []= $elem1078;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18988,9 +18988,9 @@ class ThriftHiveMetastore_get_schema_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1072)
+          foreach ($this->success as $iter1079)
           {
-            $xfer += $iter1072->write($output);
+            $xfer += $iter1079->write($output);
           }
         }
         $output->writeListEnd();
@@ -19232,15 +19232,15 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1073 = 0;
-            $_etype1076 = 0;
-            $xfer += $input->readListBegin($_etype1076, $_size1073);
-            for ($_i1077 = 0; $_i1077 < $_size1073; ++$_i1077)
+            $_size1080 = 0;
+            $_etype1083 = 0;
+            $xfer += $input->readListBegin($_etype1083, $_size1080);
+            for ($_i1084 = 0; $_i1084 < $_size1080; ++$_i1084)
             {
-              $elem1078 = null;
-              $elem1078 = new \metastore\FieldSchema();
-              $xfer += $elem1078->read($input);
-              $this->success []= $elem1078;
+              $elem1085 = null;
+              $elem1085 = new \metastore\FieldSchema();
+              $xfer += $elem1085->read($input);
+              $this->success []= $elem1085;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19292,9 +19292,9 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1079)
+          foreach ($this->success as $iter1086)
           {
-            $xfer += $iter1079->write($output);
+            $xfer += $iter1086->write($output);
           }
         }
         $output->writeListEnd();
@@ -19966,15 +19966,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->primaryKeys = array();
-            $_size1080 = 0;
-            $_etype1083 = 0;
-            $xfer += $input->readListBegin($_etype1083, $_size1080);
-            for ($_i1084 = 0; $_i1084 < $_size1080; ++$_i1084)
+            $_size1087 = 0;
+            $_etype1090 = 0;
+            $xfer += $input->readListBegin($_etype1090, $_size1087);
+            for ($_i1091 = 0; $_i1091 < $_size1087; ++$_i1091)
             {
-              $elem1085 = null;
-              $elem1085 = new \metastore\SQLPrimaryKey();
-              $xfer += $elem1085->read($input);
-              $this->primaryKeys []= $elem1085;
+              $elem1092 = null;
+              $elem1092 = new \metastore\SQLPrimaryKey();
+              $xfer += $elem1092->read($input);
+              $this->primaryKeys []= $elem1092;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19984,15 +19984,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->foreignKeys = array();
-            $_size1086 = 0;
-            $_etype1089 = 0;
-            $xfer += $input->readListBegin($_etype1089, $_size1086);
-            for ($_i1090 = 0; $_i1090 < $_size1086; ++$_i1090)
+            $_size1093 = 0;
+            $_etype1096 = 0;
+            $xfer += $input->readListBegin($_etype1096, $_size1093);
+            for ($_i1097 = 0; $_i1097 < $_size1093; ++$_i1097)
             {
-              $elem1091 = null;
-              $elem1091 = new \metastore\SQLForeignKey();
-              $xfer += $elem1091->read($input);
-              $this->foreignKeys []= $elem1091;
+              $elem1098 = null;
+              $elem1098 = new \metastore\SQLForeignKey();
+              $xfer += $elem1098->read($input);
+              $this->foreignKeys []= $elem1098;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20002,15 +20002,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 4:
           if ($ftype == TType::LST) {
             $this->uniqueConstraints = array();
-            $_size1092 = 0;
-            $_etype1095 = 0;
-            $xfer += $input->readListBegin($_etype1095, $_size1092);
-            for ($_i1096 = 0; $_i1096 < $_size1092; ++$_i1096)
+            $_size1099 = 0;
+            $_etype1102 = 0;
+            $xfer += $input->readListBegin($_etype1102, $_size1099);
+            for ($_i1103 = 0; $_i1103 < $_size1099; ++$_i1103)
             {
-              $elem1097 = null;
-              $elem1097 = new \metastore\SQLUniqueConstraint();
-              $xfer += $elem1097->read($input);
-              $this->uniqueConstraints []= $elem1097;
+              $elem1104 = null;
+              $elem1104 = new \metastore\SQLUniqueConstraint();
+              $xfer += $elem1104->read($input);
+              $this->uniqueConstraints []= $elem1104;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20020,15 +20020,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->notNullConstraints = array();
-            $_size1098 = 0;
-            $_etype1101 = 0;
-            $xfer += $input->readListBegin($_etype1101, $_size1098);
-            for ($_i1102 = 0; $_i1102 < $_size1098; ++$_i1102)
+            $_size1105 = 0;
+            $_etype1108 = 0;
+            $xfer += $input->readListBegin($_etype1108, $_size1105);
+            for ($_i1109 = 0; $_i1109 < $_size1105; ++$_i1109)
             {
-              $elem1103 = null;
-              $elem1103 = new \metastore\SQLNotNullConstraint();
-              $xfer += $elem1103->read($input);
-              $this->notNullConstraints []= $elem1103;
+              $elem1110 = null;
+              $elem1110 = new \metastore\SQLNotNullConstraint();
+              $xfer += $elem1110->read($input);
+              $this->notNullConstraints []= $elem1110;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20038,15 +20038,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->defaultConstraints = array();
-            $_size1104 = 0;
-            $_etype1107 = 0;
-            $xfer += $input->readListBegin($_etype1107, $_size1104);
-            for ($_i1108 = 0; $_i1108 < $_size1104; ++$_i1108)
+            $_size1111 = 0;
+            $_etype1114 = 0;
+            $xfer += $input->readListBegin($_etype1114, $_size1111);
+            for ($_i1115 = 0; $_i1115 < $_size1111; ++$_i1115)
             {
-              $elem1109 = null;
-              $elem1109 = new \metastore\SQLDefaultConstraint();
-              $xfer += $elem1109->read($input);
-              $this->defaultConstraints []= $elem1109;
+              $elem1116 = null;
+              $elem1116 = new \metastore\SQLDefaultConstraint();
+              $xfer += $elem1116->read($input);
+              $this->defaultConstraints []= $elem1116;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20056,15 +20056,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 7:
           if ($ftype == TType::LST) {
             $this->checkConstraints = array();
-            $_size1110 = 0;
-            $_etype1113 = 0;
-            $xfer += $input->readListBegin($_etype1113, $_size1110);
-            for ($_i1114 = 0; $_i1114 < $_size1110; ++$_i1114)
+            $_size1117 = 0;
+            $_etype1120 = 0;
+            $xfer += $input->readListBegin($_etype1120, $_size1117);
+            for ($_i1121 = 0; $_i1121 < $_size1117; ++$_i1121)
             {
-              $elem1115 = null;
-              $elem1115 = new \metastore\SQLCheckConstraint();
-              $xfer += $elem1115->read($input);
-              $this->checkConstraints []= $elem1115;
+              $elem1122 = null;
+              $elem1122 = new \metastore\SQLCheckConstraint();
+              $xfer += $elem1122->read($input);
+              $this->checkConstraints []= $elem1122;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20100,9 +20100,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->primaryKeys));
         {
-          foreach ($this->primaryKeys as $iter1116)
+          foreach ($this->primaryKeys as $iter1123)
           {
-            $xfer += $iter1116->write($output);
+            $xfer += $iter1123->write($output);
           }
         }
         $output->writeListEnd();
@@ -20117,9 +20117,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->foreignKeys));
         {
-          foreach ($this->foreignKeys as $iter1117)
+          foreach ($this->foreignKeys as $iter1124)
           {
-            $xfer += $iter1117->write($output);
+            $xfer += $iter1124->write($output);
           }
         }
         $output->writeListEnd();
@@ -20134,9 +20134,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->uniqueConstraints));
         {
-          foreach ($this->uniqueConstraints as $iter1118)
+          foreach ($this->uniqueConstraints as $iter1125)
           {
-            $xfer += $iter1118->write($output);
+            $xfer += $iter1125->write($output);
           }
         }
         $output->writeListEnd();
@@ -20151,9 +20151,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->notNullConstraints));
         {
-          foreach ($this->notNullConstraints as $iter1119)
+          foreach ($this->notNullConstraints as $iter1126)
           {
-            $xfer += $iter1119->write($output);
+            $xfer += $iter1126->write($output);
           }
         }
         $output->writeListEnd();
@@ -20168,9 +20168,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->defaultConstraints));
         {
-          foreach ($this->defaultConstraints as $iter1120)
+          foreach ($this->defaultConstraints as $iter1127)
           {
-            $xfer += $iter1120->write($output);
+            $xfer += $iter1127->write($output);
           }
         }
         $output->writeListEnd();
@@ -20185,9 +20185,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->checkConstraints));
         {
-          foreach ($this->checkConstraints as $iter1121)
+          foreach ($this->checkConstraints as $iter1128)
           {
-            $xfer += $iter1121->write($output);
+            $xfer += $iter1128->write($output);
           }
         }
         $output->writeListEnd();
@@ -22419,14 +22419,14 @@ class ThriftHiveMetastore_truncate_table_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->partNames = array();
-            $_size1122 = 0;
-            $_etype1125 = 0;
-            $xfer += $input->readListBegin($_etype1125, $_size1122);
-            for ($_i1126 = 0; $_i1126 < $_size1122; ++$_i1126)
+            $_size1129 = 0;
+            $_etype1132 = 0;
+            $xfer += $input->readListBegin($_etype1132, $_size1129);
+            for ($_i1133 = 0; $_i1133 < $_size1129; ++$_i1133)
             {
-              $elem1127 = null;
-              $xfer += $input->readString($elem1127);
-              $this->partNames []= $elem1127;
+              $elem1134 = null;
+              $xfer += $input->readString($elem1134);
+              $this->partNames []= $elem1134;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22464,9 +22464,9 @@ class ThriftHiveMetastore_truncate_table_args {
       {
         $output->writeListBegin(TType::STRING, count($this->partNames));
         {
-          foreach ($this->partNames as $iter1128)
+          foreach ($this->partNames as $iter1135)
           {
-            $xfer += $output->writeString($iter1128);
+            $xfer += $output->writeString($iter1135);
           }
         }
         $output->writeListEnd();
@@ -22902,14 +22902,14 @@ class ThriftHiveMetastore_get_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1129 = 0;
-            $_etype1132 = 0;
-            $xfer += $input->readListBegin($_etype1132, $_size1129);
-            for ($_i1133 = 0; $_i1133 < $_size1129; ++$_i1133)
+            $_size1136 = 0;
+            $_etype1139 = 0;
+            $xfer += $input->readListBegin($_etype1139, $_size1136);
+            for ($_i1140 = 0; $_i1140 < $_size1136; ++$_i1140)
             {
-              $elem1134 = null;
-              $xfer += $input->readString($elem1134);
-              $this->success []= $elem1134;
+              $elem1141 = null;
+              $xfer += $input->readString($elem1141);
+              $this->success []= $elem1141;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22945,9 +22945,9 @@ class ThriftHiveMetastore_get_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1135)
+          foreach ($this->success as $iter1142)
           {
-            $xfer += $output->writeString($iter1135);
+            $xfer += $output->writeString($iter1142);
           }
         }
         $output->writeListEnd();
@@ -23149,14 +23149,14 @@ class ThriftHiveMetastore_get_tables_by_type_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1136 = 0;
-            $_etype1139 = 0;
-            $xfer += $input->readListBegin($_etype1139, $_size1136);
-            for ($_i1140 = 0; $_i1140 < $_size1136; ++$_i1140)
+            $_size1143 = 0;
+            $_etype1146 = 0;
+            $xfer += $input->readListBegin($_etype1146, $_size1143);
+            for ($_i1147 = 0; $_i1147 < $_size1143; ++$_i1147)
             {
-              $elem1141 = null;
-              $xfer += $input->readString($elem1141);
-              $this->success []= $elem1141;
+              $elem1148 = null;
+              $xfer += $input->readString($elem1148);
+              $this->success []= $elem1148;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23192,9 +23192,9 @@ class ThriftHiveMetastore_get_tables_by_type_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1142)
+          foreach ($this->success as $iter1149)
           {
-            $xfer += $output->writeString($iter1142);
+            $xfer += $output->writeString($iter1149);
           }
         }
         $output->writeListEnd();
@@ -23326,15 +23326,15 @@ class ThriftHiveMetastore_get_all_materialized_view_objects_for_rewriting_result
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1143 = 0;
-            $_etype1146 = 0;
-            $xfer += $input->readListBegin($_etype1146, $_size1143);
-            for ($_i1147 = 0; $_i1147 < $_size1143; ++$_i1147)
+            $_size1150 = 0;
+            $_etype1153 = 0;
+            $xfer += $input->readListBegin($_etype1153, $_size1150);
+            for ($_i1154 = 0; $_i1154 < $_size1150; ++$_i1154)
             {
-              $elem1148 = null;
-              $elem1148 = new \metastore\Table();
-              $xfer += $elem1148->read($input);
-              $this->success []= $elem1148;
+              $elem1155 = null;
+              $elem1155 = new \metastore\Table();
+              $xfer += $elem1155->read($input);
+              $this->success []= $elem1155;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23370,9 +23370,9 @@ class ThriftHiveMetastore_get_all_materialized_view_objects_for_rewriting_result
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1149)
+          foreach ($this->success as $iter1156)
           {
-            $xfer += $iter1149->write($output);
+            $xfer += $iter1156->write($output);
           }
         }
         $output->writeListEnd();
@@ -23528,14 +23528,14 @@ class ThriftHiveMetastore_get_materialized_views_for_rewriting_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1150 = 0;
-            $_etype1153 = 0;
-            $xfer += $input->readListBegin($_etype1153, $_size1150);
-            for ($_i1154 = 0; $_i1154 < $_size1150; ++$_i1154)
+            $_size1157 = 0;
+            $_etype1160 = 0;
+            $xfer += $input->readListBegin($_etype1160, $_size1157);
+            for ($_i1161 = 0; $_i1161 < $_size1157; ++$_i1161)
             {
-              $elem1155 = null;
-              $xfer += $input->readString($elem1155);
-              $this->success []= $elem1155;
+              $elem1162 = null;
+              $xfer += $input->readString($elem1162);
+              $this->success []= $elem1162;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23571,9 +23571,9 @@ class ThriftHiveMetastore_get_materialized_views_for_rewriting_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1156)
+          foreach ($this->success as $iter1163)
           {
-            $xfer += $output->writeString($iter1156);
+            $xfer += $output->writeString($iter1163);
           }
         }
         $output->writeListEnd();
@@ -23678,14 +23678,14 @@ class ThriftHiveMetastore_get_table_meta_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->tbl_types = 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;
-              $xfer += $input->readString($elem1162);
-              $this->tbl_types []= $elem1162;
+              $elem1169 = null;
+              $xfer += $input->readString($elem1169);
+              $this->tbl_types []= $elem1169;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23723,9 +23723,9 @@ class ThriftHiveMetastore_get_table_meta_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_types));
         {
-          foreach ($this->tbl_types as $iter1163)
+          foreach ($this->tbl_types as $iter1170)
           {
-            $xfer += $output->writeString($iter1163);
+            $xfer += $output->writeString($iter1170);
           }
         }
         $output->writeListEnd();
@@ -23802,15 +23802,15 @@ class ThriftHiveMetastore_get_table_meta_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\TableMeta();
-              $xfer += $elem1169->read($input);
-              $this->success []= $elem1169;
+              $elem1176 = null;
+              $elem1176 = new \metastore\TableMeta();
+              $xfer += $elem1176->read($input);
+              $this->success []= $elem1176;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23846,9 +23846,9 @@ class ThriftHiveMetastore_get_table_meta_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();
@@ -24004,14 +24004,14 @@ class ThriftHiveMetastore_get_all_tables_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;
-              $xfer += $input->readString($elem1176);
-              $this->success []= $elem1176;
+              $elem1183 = null;
+              $xfer += $input->readString($elem1183);
+              $this->success []= $elem1183;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24047,9 +24047,9 @@ class ThriftHiveMetastore_get_all_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1177)
+          foreach ($this->success as $iter1184)
           {
-            $xfer += $output->writeString($iter1177);
+            $xfer += $output->writeString($iter1184);
           }
         }
         $output->writeListEnd();
@@ -24364,14 +24364,14 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = 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;
-              $xfer += $input->readString($elem1183);
-              $this->tbl_names []= $elem1183;
+              $elem1190 = null;
+              $xfer += $input->readString($elem1190);
+              $this->tbl_names []= $elem1190;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24404,9 +24404,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter1184)
+          foreach ($this->tbl_names as $iter1191)
           {
-            $xfer += $output->writeString($iter1184);
+            $xfer += $output->writeString($iter1191);
           }
         }
         $output->writeListEnd();
@@ -24471,15 +24471,15 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = 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\Table();
-              $xfer += $elem1190->read($input);
-              $this->success []= $elem1190;
+              $elem1197 = null;
+              $elem1197 = new \metastore\Table();
+              $xfer += $elem1197->read($input);
+              $this->success []= $elem1197;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24507,9 +24507,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1191)
+          foreach ($this->success as $iter1198)
           {
-            $xfer += $iter1191->write($output);
+            $xfer += $iter1198->write($output);
           }
         }
         $output->writeListEnd();
@@ -24666,15 +24666,15 @@ class ThriftHiveMetastore_get_tables_ext_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1192 = 0;
-            $_etype1195 = 0;
-            $xfer += $input->readListBegin($_etype1195, $_size1192);
-            for ($_i1196 = 0; $_i1196 < $_size1192; ++$_i1196)
+            $_size1199 = 0;
+            $_etype1202 = 0;
+            $xfer += $input->readListBegin($_etype1202, $_size1199);
+            for ($_i1203 = 0; $_i1203 < $_size1199; ++$_i1203)
             {
-              $elem1197 = null;
-              $elem1197 = new \metastore\ExtendedTableInfo();
-              $xfer += $elem1197->read($input);
-              $this->success []= $elem1197;
+              $elem1204 = null;
+              $elem1204 = new \metastore\ExtendedTableInfo();
+              $xfer += $elem1204->read($input);
+              $this->success []= $elem1204;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24710,9 +24710,9 @@ class ThriftHiveMetastore_get_tables_ext_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1198)
+          foreach ($this->success as $iter1205)
           {
-            $xfer += $iter1198->write($output);
+            $xfer += $iter1205->write($output);
           }
         }
         $output->writeListEnd();
@@ -25917,14 +25917,14 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1199 = 0;
-            $_etype1202 = 0;
-            $xfer += $input->readListBegin($_etype1202, $_size1199);
-            for ($_i1203 = 0; $_i1203 < $_size1199; ++$_i1203)
+            $_size1206 = 0;
+            $_etype1209 = 0;
+            $xfer += $input->readListBegin($_etype1209, $_size1206);
+            for ($_i1210 = 0; $_i1210 < $_size1206; ++$_i1210)
             {
-              $elem1204 = null;
-              $xfer += $input->readString($elem1204);
-              $this->success []= $elem1204;
+              $elem1211 = null;
+              $xfer += $input->readString($elem1211);
+              $this->success []= $elem1211;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25976,9 +25976,9 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1205)
+          foreach ($this->success as $iter1212)
           {
-            $xfer += $output->writeString($iter1205);
+            $xfer += $output->writeString($iter1212);
           }
         }
         $output->writeListEnd();
@@ -27501,15 +27501,15 @@ class ThriftHiveMetastore_add_partitions_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size1206 = 0;
-            $_etype1209 = 0;
-            $xfer += $input->readListBegin($_etype1209, $_size1206);
-            for ($_i1210 = 0; $_i1210 < $_size1206; ++$_i1210)
+            $_size1213 = 0;
+            $_etype1216 = 0;
+            $xfer += $input->readListBegin($_etype1216, $_size1213);
+            for ($_i1217 = 0; $_i1217 < $_size1213; ++$_i1217)
             {
-              $elem1211 = null;
-              $elem1211 = new \metastore\Partition();
-              $xfer += $elem1211->read($input);
-              $this->new_parts []= $elem1211;
+              $elem1218 = null;
+              $elem1218 = new \metastore\Partition();
+              $xfer += $elem1218->read($input);
+              $this->new_parts []= $elem1218;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27537,9 +27537,9 @@ class ThriftHiveMetastore_add_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1212)
+          foreach ($this->new_parts as $iter1219)
           {
-            $xfer += $iter1212->write($output);
+            $xfer += $iter1219->write($output);
           }
         }
         $output->writeListEnd();
@@ -27754,15 +27754,15 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size1213 = 0;
-            $_etype1216 = 0;
-            $xfer += $input->readListBegin($_etype1216, $_size1213);
-            for ($_i1217 = 0; $_i1217 < $_size1213; ++$_i1217)
+            $_size1220 = 0;
+            $_etype1223 = 0;
+            $xfer += $input->readListBegin($_etype1223, $_size1220);
+            for ($_i1224 = 0; $_i1224 < $_size1220; ++$_i1224)
             {
-              $elem1218 = null;
-              $elem1218 = new \metastore\PartitionSpec();
-              $xfer += $elem1218->read($input);
-              $this->new_parts []= $elem1218;
+              $elem1225 = null;
+              $elem1225 = new \metastore\PartitionSpec();
+              $xfer += $elem1225->read($input);
+              $this->new_parts []= $elem1225;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27790,9 +27790,9 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1219)
+          foreach ($this->new_parts as $iter1226)
           {
-            $xfer += $iter1219->write($output);
+            $xfer += $iter1226->write($output);
           }
         }
         $output->writeListEnd();
@@ -28042,14 +28042,14 @@ class ThriftHiveMetastore_append_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1220 = 0;
-            $_etype1223 = 0;
-            $xfer += $input->readListBegin($_etype1223, $_size1220);
-            for ($_i1224 = 0; $_i1224 < $_size1220; ++$_i1224)
+            $_size1227 = 0;
+            $_etype1230 = 0;
+            $xfer += $input->readListBegin($_etype1230, $_size1227);
+            for ($_i1231 = 0; $_i1231 < $_size1227; ++$_i1231)
             {
-              $elem1225 = null;
-              $xfer += $input->readString($elem1225);
-              $this->part_vals []= $elem1225;
+              $elem1232 = null;
+              $xfer += $input->readString($elem1232);
+              $this->part_vals []= $elem1232;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28087,9 +28087,9 @@ class ThriftHiveMetastore_append_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1226)
+          foreach ($this->part_vals as $iter1233)
           {
-            $xfer += $output->writeString($iter1226);
+            $xfer += $output->writeString($iter1233);
           }
         }
         $output->writeListEnd();
@@ -28591,14 +28591,14 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = 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->part_vals []= $elem1232;
+              $elem1239 = null;
+              $xfer += $input->readString($elem1239);
+              $this->part_vals []= $elem1239;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28644,9 +28644,9 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1233)
+          foreach ($this->part_vals as $iter1240)
           {
-            $xfer += $output->writeString($iter1233);
+            $xfer += $output->writeString($iter1240);
           }
         }
         $output->writeListEnd();
@@ -29500,14 +29500,14 @@ class ThriftHiveMetastore_drop_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = 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->part_vals []= $elem1239;
+              $elem1246 = null;
+              $xfer += $input->readString($elem1246);
+              $this->part_vals []= $elem1246;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29552,9 +29552,9 @@ class ThriftHiveMetastore_drop_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1240)
+          foreach ($this->part_vals as $iter1247)
           {
-            $xfer += $output->writeString($iter1240);
+            $xfer += $output->writeString($iter1247);
           }
         }
         $output->writeListEnd();
@@ -29807,14 +29807,14 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = 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->part_vals []= $elem1246;
+              $elem1253 = null;
+              $xfer += $input->readString($elem1253);
+              $this->part_vals []= $elem1253;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29867,9 +29867,9 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1247)
+          foreach ($this->part_vals as $iter1254)
           {
-            $xfer += $output->writeString($iter1247);
+            $xfer += $output->writeString($iter1254);
           }
         }
         $output->writeListEnd();
@@ -30883,14 +30883,14 @@ class ThriftHiveMetastore_get_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = 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;
-              $xfer += $input->readString($elem1253);
-              $this->part_vals []= $elem1253;
+              $elem1260 = null;
+              $xfer += $input->readString($elem1260);
+              $this->part_vals []= $elem1260;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30928,9 +30928,9 @@ class ThriftHiveMetastore_get_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1254)
+          foreach ($this->part_vals as $iter1261)
           {
-            $xfer += $output->writeString($iter1254);
+            $xfer += $output->writeString($iter1261);
           }
         }
         $output->writeListEnd();
@@ -31172,17 +31172,17 @@ class ThriftHiveMetastore_exchange_partition_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size1255 = 0;
-            $_ktype1256 = 0;
-            $_vtype1257 = 0;
-            $xfer += $input->readMapBegin($_ktype1256, $_vtype1257, $_size1255);
-            for ($_i1259 = 0; $_i1259 < $_size1255; ++$_i1259)
+            $_size1262 = 0;
+            $_ktype1263 = 0;
+            $_vtype1264 = 0;
+            $xfer += $input->readMapBegin($_ktype1263, $_vtype1264, $_size1262);
+            for ($_i1266 = 0; $_i1266 < $_size1262; ++$_i1266)
             {
-              $key1260 = '';
-              $val1261 = '';
-              $xfer += $input->readString($key1260);
-              $xfer += $input->readString($val1261);
-              $this->partitionSpecs[$key1260] = $val1261;
+              $key1267 = '';
+              $val1268 = '';
+              $xfer += $input->readString($key1267);
+              $xfer += $input->readString($val1268);
+              $this->partitionSpecs[$key1267] = $val1268;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -31238,10 +31238,10 @@ class ThriftHiveMetastore_exchange_partition_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter1262 => $viter1263)
+          foreach ($this->partitionSpecs as $kiter1269 => $viter1270)
           {
-            $xfer += $output->writeString($kiter1262);
-            $xfer += $output->writeString($viter1263);
+            $xfer += $output->writeString($kiter1269);
+            $xfer += $output->writeString($viter1270);
           }
         }
         $output->writeMapEnd();
@@ -31553,17 +31553,17 @@ class ThriftHiveMetastore_exchange_partitions_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size1264 = 0;
-            $_ktype1265 = 0;
-            $_vtype1266 = 0;
-            $xfer += $input->readMapBegin($_ktype1265, $_vtype1266, $_size1264);
-            for ($_i1268 = 0; $_i1268 < $_size1264; ++$_i1268)
+            $_size1271 = 0;
+            $_ktype1272 = 0;
+            $_vtype1273 = 0;
+            $xfer += $input->readMapBegin($_ktype1272, $_vtype1273, $_size1271);
+            for ($_i1275 = 0; $_i1275 < $_size1271; ++$_i1275)
             {
-              $key1269 = '';
-              $val1270 = '';
-              $xfer += $input->readString($key1269);
-              $xfer += $input->readString($val1270);
-              $this->partitionSpecs[$key1269] = $val1270;
+              $key1276 = '';
+              $val1277 = '';
+              $xfer += $input->readString($key1276);
+              $xfer += $input->readString($val1277);
+              $this->partitionSpecs[$key1276] = $val1277;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -31619,10 +31619,10 @@ class ThriftHiveMetastore_exchange_partitions_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter1271 => $viter1272)
+          foreach ($this->partitionSpecs as $kiter1278 => $viter1279)
           {
-            $xfer += $output->writeString($kiter1271);
-            $xfer += $output->writeString($viter1272);
+            $xfer += $output->writeString($kiter1278);
+            $xfer += $output->writeString($viter1279);
           }
         }
         $output->writeMapEnd();
@@ -31755,15 +31755,15 @@ class ThriftHiveMetastore_exchange_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1273 = 0;
-            $_etype1276 = 0;
-            $xfer += $input->readListBegin($_etype1276, $_size1273);
-            for ($_i1277 = 0; $_i1277 < $_size1273; ++$_i1277)
+            $_size1280 = 0;
+            $_etype1283 = 0;
+            $xfer += $input->readListBegin($_etype1283, $_size1280);
+            for ($_i1284 = 0; $_i1284 < $_size1280; ++$_i1284)
             {
-              $elem1278 = null;
-              $elem1278 = new \metastore\Partition();
-              $xfer += $elem1278->read($input);
-              $this->success []= $elem1278;
+              $elem1285 = null;
+              $elem1285 = new \metastore\Partition();
+              $xfer += $elem1285->read($input);
+              $this->success []= $elem1285;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31823,9 +31823,9 @@ class ThriftHiveMetastore_exchange_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1279)
+          foreach ($this->success as $iter1286)
           {
-            $xfer += $iter1279->write($output);
+            $xfer += $iter1286->write($output);
           }
         }
         $output->writeListEnd();
@@ -31971,14 +31971,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1280 = 0;
-            $_etype1283 = 0;
-            $xfer += $input->readListBegin($_etype1283, $_size1280);
-            for ($_i1284 = 0; $_i1284 < $_size1280; ++$_i1284)
+            $_size1287 = 0;
+            $_etype1290 = 0;
+            $xfer += $input->readListBegin($_etype1290, $_size1287);
+            for ($_i1291 = 0; $_i1291 < $_size1287; ++$_i1291)
             {
-              $elem1285 = null;
-              $xfer += $input->readString($elem1285);
-              $this->part_vals []= $elem1285;
+              $elem1292 = null;
+              $xfer += $input->readString($elem1292);
+              $this->part_vals []= $elem1292;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31995,14 +31995,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1286 = 0;
-            $_etype1289 = 0;
-            $xfer += $input->readListBegin($_etype1289, $_size1286);
-            for ($_i1290 = 0; $_i1290 < $_size1286; ++$_i1290)
+            $_size1293 = 0;
+            $_etype1296 = 0;
+            $xfer += $input->readListBegin($_etype1296, $_size1293);
+            for ($_i1297 = 0; $_i1297 < $_size1293; ++$_i1297)
             {
-              $elem1291 = null;
-              $xfer += $input->readString($elem1291);
-              $this->group_names []= $elem1291;
+              $elem1298 = null;
+              $xfer += $input->readString($elem1298);
+              $this->group_names []= $elem1298;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32040,9 +32040,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1292)
+          foreach ($this->part_vals as $iter1299)
           {
-            $xfer += $output->writeString($iter1292);
+            $xfer += $output->writeString($iter1299);
           }
         }
         $output->writeListEnd();
@@ -32062,9 +32062,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1293)
+          foreach ($this->group_names as $iter1300)
           {
-            $xfer += $output->writeString($iter1293);
+            $xfer += $output->writeString($iter1300);
           }
         }
         $output->writeListEnd();
@@ -32655,15 +32655,15 @@ class ThriftHiveMetastore_get_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1294 = 0;
-            $_etype1297 = 0;
-            $xfer += $input->readListBegin($_etype1297, $_size1294);
-            for ($_i1298 = 0; $_i1298 < $_size1294; ++$_i1298)
+            $_size1301 = 0;
+            $_etype1304 = 0;
+            $xfer += $input->readListBegin($_etype1304, $_size1301);
+            for ($_i1305 = 0; $_i1305 < $_size1301; ++$_i1305)
             {
-              $elem1299 = null;
-              $elem1299 = new \metastore\Partition();
-              $xfer += $elem1299->read($input);
-              $this->success []= $elem1299;
+              $elem1306 = null;
+              $elem1306 = new \metastore\Partition();
+              $xfer += $elem1306->read($input);
+              $this->success []= $elem1306;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32707,9 +32707,9 @@ class ThriftHiveMetastore_get_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1300)
+          foreach ($this->success as $iter1307)
           {
-            $xfer += $iter1300->write($output);
+            $xfer += $iter1307->write($output);
           }
         }
         $output->writeListEnd();
@@ -32855,14 +32855,14 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1301 = 0;
-            $_etype1304 = 0;
-            $xfer += $input->readListBegin($_etype1304, $_size1301);
-            for ($_i1305 = 0; $_i1305 < $_size1301; ++$_i1305)
+            $_size1308 = 0;
+            $_etype1311 = 0;
+            $xfer += $input->readListBegin($_etype1311, $_size1308);
+            for ($_i1312 = 0; $_i1312 < $_size1308; ++$_i1312)
             {
-              $elem1306 = null;
-              $xfer += $input->readString($elem1306);
-              $this->group_names []= $elem1306;
+              $elem1313 = null;
+              $xfer += $input->readString($elem1313);
+              $this->group_names []= $elem1313;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32910,9 +32910,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1307)
+          foreach ($this->group_names as $iter1314)
           {
-            $xfer += $output->writeString($iter1307);
+            $xfer += $output->writeString($iter1314);
           }
         }
         $output->writeListEnd();
@@ -33001,15 +33001,15 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1308 = 0;
-            $_etype1311 = 0;
-            $xfer += $input->readListBegin($_etype1311, $_size1308);
-            for ($_i1312 = 0; $_i1312 < $_size1308; ++$_i1312)
+            $_size1315 = 0;
+            $_etype1318 = 0;
+            $xfer += $input->readListBegin($_etype1318, $_size1315);
+            for ($_i1319 = 0; $_i1319 < $_size1315; ++$_i1319)
             {
-              $elem1313 = null;
-              $elem1313 = new \metastore\Partition();
-              $xfer += $elem1313->read($input);
-              $this->success []= $elem1313;
+              $elem1320 = null;
+              $elem1320 = new \metastore\Partition();
+              $xfer += $elem1320->read($input);
+              $this->success []= $elem1320;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33053,9 +33053,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1314)
+          foreach ($this->success as $iter1321)
           {
-            $xfer += $iter1314->write($output);
+            $xfer += $iter1321->write($output);
           }
         }
         $output->writeListEnd();
@@ -33275,15 +33275,15 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1315 = 0;
-            $_etype1318 = 0;
-            $xfer += $input->readListBegin($_etype1318, $_size1315);
-            for ($_i1319 = 0; $_i1319 < $_size1315; ++$_i1319)
+            $_size1322 = 0;
+            $_etype1325 = 0;
+            $xfer += $input->readListBegin($_etype1325, $_size1322);
+            for ($_i1326 = 0; $_i1326 < $_size1322; ++$_i1326)
             {
-              $elem1320 = null;
-              $elem1320 = new \metastore\PartitionSpec();
-              $xfer += $elem1320->read($input);
-              $this->success []= $elem1320;
+              $elem1327 = null;
+              $elem1327 = new \metastore\PartitionSpec();
+              $xfer += $elem1327->read($input);
+              $this->success []= $elem1327;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33327,9 +33327,9 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1321)
+          foreach ($this->success as $iter1328)
           {
-            $xfer += $iter1321->write($output);
+            $xfer += $iter1328->write($output);
           }
         }
         $output->writeListEnd();
@@ -33548,14 +33548,14 @@ class ThriftHiveMetastore_get_partition_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1322 = 0;
-            $_etype1325 = 0;
-            $xfer += $input->readListBegin($_etype1325, $_size1322);
-            for ($_i1326 = 0; $_i1326 < $_size1322; ++$_i1326)
+            $_size1329 = 0;
+            $_etype1332 = 0;
+            $xfer += $input->readListBegin($_etype1332, $_size1329);
+            for ($_i1333 = 0; $_i1333 < $_size1329; ++$_i1333)
             {
-              $elem1327 = null;
-              $xfer += $input->readString($elem1327);
-              $this->success []= $elem1327;
+              $elem1334 = null;
+              $xfer += $input->readString($elem1334);
+              $this->success []= $elem1334;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33599,9 +33599,9 @@ class ThriftHiveMetastore_get_partition_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1328)
+          foreach ($this->success as $iter1335)
           {
-            $xfer += $output->writeString($iter1328);
+            $xfer += $output->writeString($iter1335);
           }
         }
         $output->writeListEnd();
@@ -33932,14 +33932,14 @@ class ThriftHiveMetastore_get_partitions_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1329 = 0;
-            $_etype1332 = 0;
-            $xfer += $input->readListBegin($_etype1332, $_size1329);
-            for ($_i1333 = 0; $_i1333 < $_size1329; ++$_i1333)
+            $_size1336 = 0;
+            $_etype1339 = 0;
+            $xfer += $input->readListBegin($_etype1339, $_size1336);
+            for ($_i1340 = 0; $_i1340 < $_size1336; ++$_i1340)
             {
-              $elem1334 = null;
-              $xfer += $input->readString($elem1334);
-              $this->part_vals []= $elem1334;
+              $elem1341 = null;
+              $xfer += $input->readString($elem1341);
+              $this->part_vals []= $elem1341;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33984,9 +33984,9 @@ class ThriftHiveMetastore_get_partitions_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1335)
+          foreach ($this->part_vals as $iter1342)
           {
-            $xfer += $output->writeString($iter1335);
+            $xfer += $output->writeString($iter1342);
           }
         }
         $output->writeListEnd();
@@ -34080,15 +34080,15 @@ class ThriftHiveMetastore_get_partitions_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1336 = 0;
-            $_etype1339 = 0;
-            $xfer += $input->readListBegin($_etype1339, $_size1336);
-            for ($_i1340 = 0; $_i1340 < $_size1336; ++$_i1340)
+            $_size1343 = 0;
+            $_etype1346 = 0;
+            $xfer += $input->readListBegin($_etype1346, $_size1343);
+            for ($_i1347 = 0; $_i1347 < $_size1343; ++$_i1347)
             {
-              $elem1341 = null;
-              $elem1341 = new \metastore\Partition();
-              $xfer += $elem1341->read($input);
-              $this->success []= $elem1341;
+              $elem1348 = null;
+              $elem1348 = new \metastore\Partition();
+              $xfer += $elem1348->read($input);
+              $this->success []= $elem1348;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34132,9 +34132,9 @@ class ThriftHiveMetastore_get_partitions_ps_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1342)
+          foreach ($this->success as $iter1349)
           {
-            $xfer += $iter1342->write($output);
+            $xfer += $iter1349->write($output);
           }
         }
         $output->writeListEnd();
@@ -34281,14 +34281,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1343 = 0;
-            $_etype1346 = 0;
-            $xfer += $input->readListBegin($_etype1346, $_size1343);
-            for ($_i1347 = 0; $_i1347 < $_size1343; ++$_i1347)
+            $_size1350 = 0;
+            $_etype1353 = 0;
+            $xfer += $input->readListBegin($_etype1353, $_size1350);
+            for ($_i1354 = 0; $_i1354 < $_size1350; ++$_i1354)
             {
-              $elem1348 = null;
-              $xfer += $input->readString($elem1348);
-              $this->part_vals []= $elem1348;
+              $elem1355 = null;
+              $xfer += $input->readString($elem1355);
+              $this->part_vals []= $elem1355;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34312,14 +34312,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1349 = 0;
-            $_etype1352 = 0;
-            $xfer += $input->readListBegin($_etype1352, $_size1349);
-            for ($_i1353 = 0; $_i1353 < $_size1349; ++$_i1353)
+            $_size1356 = 0;
+            $_etype1359 = 0;
+            $xfer += $input->readListBegin($_etype1359, $_size1356);
+            for ($_i1360 = 0; $_i1360 < $_size1356; ++$_i1360)
             {
-              $elem1354 = null;
-              $xfer += $input->readString($elem1354);
-              $this->group_names []= $elem1354;
+              $elem1361 = null;
+              $xfer += $input->readString($elem1361);
+              $this->group_names []= $elem1361;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34357,9 +34357,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1355)
+          foreach ($this->part_vals as $iter1362)
           {
-            $xfer += $output->writeString($iter1355);
+            $xfer += $output->writeString($iter1362);
           }
         }
         $output->writeListEnd();
@@ -34384,9 +34384,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1356)
+          foreach ($this->group_names as $iter1363)
           {
-            $xfer += $output->writeString($iter1356);
+            $xfer += $output->writeString($iter1363);
           }
         }
         $output->writeListEnd();
@@ -34475,15 +34475,15 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1357 = 0;
-            $_etype1360 = 0;
-            $xfer += $input->readListBegin($_etype1360, $_size1357);
-            for ($_i1361 = 0; $_i1361 < $_size1357; ++$_i1361)
+            $_size1364 = 0;
+            $_etype1367 = 0;
+            $xfer += $input->readListBegin($_etype1367, $_size1364);
+            for ($_i1368 = 0; $_i1368 < $_size1364; ++$_i1368)
             {
-              $elem1362 = null;
-              $elem1362 = new \metastore\Partition();
-              $xfer += $elem1362->read($input);
-              $this->success []= $elem1362;
+              $elem1369 = null;
+              $elem1369 = new \metastore\Partition();
+              $xfer += $elem1369->read($input);
+              $this->success []= $elem1369;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34527,9 +34527,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1363)
+          foreach ($this->success as $iter1370)
           {
-            $xfer += $iter1363->write($output);
+            $xfer += $iter1370->write($output);
           }
         }
         $output->writeListEnd();
@@ -34650,14 +34650,14 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1364 = 0;
-            $_etype1367 = 0;
-            $xfer += $input->readListBegin($_etype1367, $_size1364);
-            for ($_i1368 = 0; $_i1368 < $_size1364; ++$_i1368)
+            $_size1371 = 0;
+            $_etype1374 = 0;
+            $xfer += $input->readListBegin($_etype1374, $_size1371);
+            for ($_i1375 = 0; $_i1375 < $_size1371; ++$_i1375)
             {
-              $elem1369 = null;
-              $xfer += $input->readString($elem1369);
-              $this->part_vals []= $elem1369;
+              $elem1376 = null;
+              $xfer += $input->readString($elem1376);
+              $this->part_vals []= $elem1376;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34702,9 +34702,9 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1370)
+          foreach ($this->part_vals as $iter1377)
           {
-            $xfer += $output->writeString($iter1370);
+            $xfer += $output->writeString($iter1377);
           }
         }
         $output->writeListEnd();
@@ -34797,14 +34797,14 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1371 = 0;
-            $_etype1374 = 0;
-            $xfer += $input->readListBegin($_etype1374, $_size1371);
-            for ($_i1375 = 0; $_i1375 < $_size1371; ++$_i1375)
+            $_size1378 = 0;
+            $_etype1381 = 0;
+            $xfer += $input->readListBegin($_etype1381, $_size1378);
+            for ($_i1382 = 0; $_i1382 < $_size1378; ++$_i1382)
             {
-              $elem1376 = null;
-              $xfer += $input->readString($elem1376);
-              $this->success []= $elem1376;
+              $elem1383 = null;
+              $xfer += $input->readString($elem1383);
+              $this->success []= $elem1383;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34848,9 +34848,9 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1377)
+          foreach ($this->success as $iter1384)
           {
-            $xfer += $output->writeString($iter1377);
+            $xfer += $output->writeString($iter1384);
           }
         }
         $output->writeListEnd();
@@ -35093,15 +35093,15 @@ class ThriftHiveMetastore_get_partitions_by_filter_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 {
@@ -35145,9 +35145,9 @@ class ThriftHiveMetastore_get_partitions_by_filter_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();
@@ -35390,15 +35390,15 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = 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;
-              $elem1390 = new \metastore\PartitionSpec();
-              $xfer += $elem1390->read($input);
-              $this->success []= $elem1390;
+              $elem1397 = null;
+              $elem1397 = new \metastore\PartitionSpec();
+              $xfer += $elem1397->read($input);
+              $this->success []= $elem1397;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -35442,9 +35442,9 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1391)
+          foreach ($this->success as $iter1398)
           {
-            $xfer += $iter1391->write($output);
+            $xfer += $iter1398->write($output);
           }
         }
         $output->writeListEnd();
@@ -36010,14 +36010,14 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->names = array();
-            $_size1392 = 0;
-            $_etype1395 = 0;
-            $xfer += $input->readListBegin($_etype1395, $_size1392);
-            for ($_i1396 = 0; $_i1396 < $_size1392; ++$_i1396)
+            $_size1399 = 0;
+            $_etype1402 = 0;
+            $xfer += $input->readListBegin($_etype1402, $_size1399);
+            for ($_i1403 = 0; $_i1403 < $_size1399; ++$_i1403)
             {
-              $elem1397 = null;
-              $xfer += $input->readString($elem1397);
-              $this->names []= $elem1397;
+              $elem1404 = null;
+              $xfer += $input->readString($elem1404);
+              $this->names []= $elem1404;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -36055,9 +36055,9 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
       {
         $output->writeListBegin(TType::STRING, count($this->names));
         {
-          foreach ($this->names as $iter1398)
+          foreach ($this->names as $iter1405)
           {
-            $xfer += $output->writeString($iter1398);
+            $xfer += $output->writeString($iter1405);
           }
         }
         $output->writeListEnd();
@@ -36146,15 +36146,15 @@ class ThriftHiveMetastore_get_partitions_by_names_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 {
@@ -36198,9 +36198,9 @@ class ThriftHiveMetastore_get_partitions_by_names_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();
@@ -36749,15 +36749,15 @@ class ThriftHiveMetastore_alter_partitions_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = 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;
-              $elem1411 = new \metastore\Partition();
-              $xfer += $elem1411->read($input);
-              $this->new_parts []= $elem1411;
+              $elem1418 = null;
+              $elem1418 = new \metastore\Partition();
+              $xfer += $elem1418->read($input);
+              $this->new_parts []= $elem1418;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -36795,9 +36795,9 @@ class ThriftHiveMetastore_alter_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1412)
+          foreach ($this->new_parts as $iter1419)
           {
-            $xfer += $iter1412->write($output);
+            $xfer += $iter1419->write($output);
           }
         }
         $output->writeListEnd();
@@ -37012,15 +37012,15 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = 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->new_parts []= $elem1418;
+              $elem1425 = null;
+              $elem1425 = new \metastore\Partition();
+              $xfer += $elem1425->read($input);
+              $this->new_parts []= $elem1425;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -37066,9 +37066,9 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1419)
+          foreach ($this->new_parts as $iter1426)
           {
-            $xfer += $iter1419->write($output);
+            $xfer += $iter1426->write($output);
           }
         }
         $output->writeListEnd();
@@ -37756,14 +37756,14 @@ class ThriftHiveMetastore_rename_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = 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;
-              $xfer += $input->readString($elem1425);
-              $this->part_vals []= $elem1425;
+              $elem1432 = null;
+              $xfer += $input->readString($elem1432);
+              $this->part_vals []= $elem1432;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -37809,9 +37809,9 @@ class ThriftHiveMetastore_rename_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1426)
+          foreach ($this->part_vals as $iter1433)
           {
-            $xfer += $output->writeString($iter1426);
+            $xfer += $output->writeString($iter1433);
           }
         }
         $output->writeListEnd();
@@ -38206,14 +38206,14 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->part_vals = 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->part_vals []= $elem1432;
+              $elem1439 = null;
+              $xfer += $input->readString($elem1439);
+              $this->part_vals []= $elem1439;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -38248,9 +38248,9 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1433)
+          foreach ($this->part_vals as $iter1440)
           {
-            $xfer += $output->writeString($iter1433);
+            $xfer += $output->writeString($iter1440);
           }
         }
         $output->writeListEnd();
@@ -38704,14 +38704,14 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = 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->success []= $elem1439;
+              $elem1446 = null;
+              $xfer += $input->readString($elem1446);
+              $this->success []= $elem1446;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -38747,9 +38747,9 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1440)
+          foreach ($this->success as $iter1447)
           {
-            $xfer += $output->writeString($iter1440);
+            $xfer += $output->writeString($iter1447);
           }
         }
         $output->writeListEnd();
@@ -38909,17 +38909,17 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size1441 = 0;
-            $_ktype1442 = 0;
-            $_vtype1443 = 0;
-            $xfer += $input->readMapBegin($_ktype1442, $_vtype1443, $_size1441);
-            for ($_i1445 = 0; $_i1445 < $_size1441; ++$_i1445)
+            $_size1448 = 0;
+            $_ktype1449 = 0;
+            $_vtype1450 = 0;
+            $xfer += $input->readMapBegin($_ktype1449, $_vtype1450, $_size1448);
+            for ($_i1452 = 0; $_i1452 < $_size1448; ++$_i1452)
             {
-              $key1446 = '';
-              $val1447 = '';
-              $xfer += $input->readString($key1446);
-              $xfer += $input->readString($val1447);
-              $this->success[$key1446] = $val1447;
+              $key1453 = '';
+              $val1454 = '';
+              $xfer += $input->readString($key1453);
+              $xfer += $input->readString($val1454);
+              $this->success[$key1453] = $val1454;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -38955,10 +38955,10 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->success));
         {
-          foreach ($this->success as $kiter1448 => $viter1449)
+          foreach ($this->success as $kiter1455 => $viter1456)
           {
-            $xfer += $output->writeString($kiter1448);
-            $xfer += $output->writeString($viter1449);
+            $xfer += $output->writeString($kiter1455);
+            $xfer += $output->writeString($viter1456);
           }
         }
         $output->writeMapEnd();
@@ -39078,17 +39078,17 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size1450 = 0;
-            $_ktype1451 = 0;
-            $_vtype1452 = 0;
-            $xfer += $input->readMapBegin($_ktype1451, $_vtype1452, $_size1450);
-            for ($_i1454 = 0; $_i1454 < $_size1450; ++$_i1454)
+            $_size1457 = 0;
+            $_ktype1458 = 0;
+            $_vtype1459 = 0;
+            $xfer += $input->readMapBegin($_ktype1458, $_vtype1459, $_size1457);
+            for ($_i1461 = 0; $_i1461 < $_size1457; ++$_i1461)
             {
-              $key1455 = '';
-              $val1456 = '';
-              $xfer += $input->readString($key1455);
-              $xfer += $input->readString($val1456);
-              $this->part_vals[$key1455] = $val1456;
+              $key1462 = '';
+              $val1463 = '';
+              $xfer += $input->readString($key1462);
+              $xfer += $input->readString($val1463);
+              $this->part_vals[$key1462] = $val1463;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -39133,10 +39133,10 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $kiter1457 => $viter1458)
+          foreach ($this->part_vals as $kiter1464 => $viter1465)
           {
-            $xfer += $output->writeString($kiter1457);
-            $xfer += $output->writeString($viter1458);
+            $xfer += $output->writeString($kiter1464);
+            $xfer += $output->writeString($viter1465);
           }
         }
         $output->writeMapEnd();
@@ -39458,17 +39458,17 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size1459 = 0;
-            $_ktype1460 = 0;
-            $_vtype1461 = 0;
-            $xfer += $input->readMapBegin($_ktype1460, $_vtype1461, $_size1459);
-            for ($_i1463 = 0; $_i1463 < $_size1459; ++$_i1463)
+            $_size1466 = 0;
+            $_ktype1467 = 0;
+            $_vtype1468 = 0;
+            $xfer += $input->readMapBegin($_ktype1467, $_vtype1468, $_size1466);
+            for ($_i1470 = 0; $_i1470 < $_size1466; ++$_i1470)
             {
-              $key1464 = '';
-              $val1465 = '';
-              $xfer += $input->readString($key1464);
-              $xfer += $input->readString($val1465);
-              $this->part_vals[$key1464] = $val1465;
+              $key1471 = '';
+              $val1472 = '';
+              $xfer += $input->readString($key1471);
+              $xfer += $input->readString($val1472);
+              $this->part_vals[$key1471] = $val1472;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -39513,10 +39513,10 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $kiter1466 => $viter1467)
+          foreach ($this->part_vals as $kiter1473 => $viter1474)
           {
-            $xfer += $output->writeString($kiter1466);
-            $xfer += $output->writeString($viter1467);
+            $xfer += $output->writeString($kiter1473);
+            $xfer += $output->writeString($viter1474);
           }
         }
         $output->writeMapEnd();
@@ -45041,14 +45041,14 @@ class ThriftHiveMetastore_get_functions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1468 = 0;
-            $_etype1471 = 0;
-            $xfer += $input->readListBegin($_etype1471, $_size1468);
-            for ($_i1472 = 0; $_i1472 < $_size1468; ++$_i1472)
+            $_size1475 = 0;
+            $_etype1478 = 0;
+            $xfer += $input->readListBegin($_etype1478, $_size1475);
+            for ($_i1479 = 0; $_i1479 < $_size1475; ++$_i1479)
             {
-              $elem1473 = null;
-              $xfer += $input->readString($elem1473);
-              $this->success []= $elem1473;
+              $elem1480 = null;
+              $xfer += $input->readString($elem1480);
+              $this->success []= $elem1480;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -45084,9 +45084,9 @@ class ThriftHiveMetastore_get_functions_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1474)
+          foreach ($this->success as $iter1481)
           {
-            $xfer += $output->writeString($iter1474);
+            $xfer += $output->writeString($iter1481);
           }
         }
         $output->writeListEnd();
@@ -45955,14 +45955,14 @@ class ThriftHiveMetastore_get_role_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1475 = 0;
-            $_etype1478 = 0;
-            $xfer += $input->readListBegin($_etype1478, $_size1475);
-            for ($_i1479 = 0; $_i1479 < $_size1475; ++$_i1479)
+            $_size1482 = 0;
+            $_etype1485 = 0;
+            $xfer += $input->readListBegin($_etype1485, $_size1482);
+            for ($_i1486 = 0; $_i1486 < $_size1482; ++$_i1486)
             {
-              $elem1480 = null;
-              $xfer += $input->readString($elem1480);
-              $this->success []= $elem1480;
+              $elem1487 = null;
+              $xfer += $input->readString($elem1487);
+              $this->success []= $elem1487;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -45998,9 +45998,9 @@ class ThriftHiveMetastore_get_role_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1481)
+          foreach ($this->success as $iter1488)
           {
-            $xfer += $output->writeString($iter1481);
+            $xfer += $output->writeString($iter1488);
           }
         }
         $output->writeListEnd();
@@ -46691,15 +46691,15 @@ class ThriftHiveMetastore_list_roles_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1482 = 0;
-            $_etype1485 = 0;
-            $xfer += $input->readListBegin($_etype1485, $_size1482);
-            for ($_i1486 = 0; $_i1486 < $_size1482; ++$_i1486)
+            $_size1489 = 0;
+            $_etype1492 = 0;
+            $xfer += $input->readListBegin($_etype1492, $_size1489);
+            for ($_i1493 = 0; $_i1493 < $_size1489; ++$_i1493)
             {
-              $elem1487 = null;
-              $elem1487 = new \metastore\Role();
-              $xfer += $elem1487->read($input);
-              $this->success []= $elem1487;
+              $elem1494 = null;
+              $elem1494 = new \metastore\Role();
+              $xfer += $elem1494->read($input);
+              $this->success []= $elem1494;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -46735,9 +46735,9 @@ class ThriftHiveMetastore_list_roles_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1488)
+          foreach ($this->success as $iter1495)
           {
-            $xfer += $iter1488->write($output);
+            $xfer += $iter1495->write($output);
           }
         }
         $output->writeListEnd();
@@ -47399,14 +47399,14 @@ class ThriftHiveMetastore_get_privilege_set_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1489 = 0;
-            $_etype1492 = 0;
-            $xfer += $input->readListBegin($_etype1492, $_size1489);
-            for ($_i1493 = 0; $_i1493 < $_size1489; ++$_i1493)
+            $_size1496 = 0;
+            $_etype1499 = 0;
+            $xfer += $input->readListBegin($_etype1499, $_size1496);
+            for ($_i1500 = 0; $_i1500 < $_size1496; ++$_i1500)
             {
-              $elem1494 = null;
-              $xfer += $input->readString($elem1494);
-              $this->group_names []= $elem1494;
+              $elem1501 = null;
+              $xfer += $input->readString($elem1501);
+              $this->group_names []= $elem1501;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -47447,9 +47447,9 @@ class ThriftHiveMetastore_get_privilege_set_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1495)
+          foreach ($this->group_names as $iter1502)
           {
-            $xfer += $output->writeString($iter1495);
+            $xfer += $output->writeString($iter1502);
           }
         }
         $output->writeListEnd();
@@ -47757,15 +47757,15 @@ class ThriftHiveMetastore_list_privileges_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1496 = 0;
-            $_etype1499 = 0;
-            $xfer += $input->readListBegin($_etype1499, $_size1496);
-            for ($_i1500 = 0; $_i1500 < $_size1496; ++$_i1500)
+            $_size1503 = 0;
+            $_etype1506 = 0;
+            $xfer += $input->readListBegin($_etype1506, $_size1503);
+            for ($_i1507 = 0; $_i1507 < $_size1503; ++$_i1507)
             {
-              $elem1501 = null;
-              $elem1501 = new \metastore\HiveObjectPrivilege();
-              $xfer += $elem1501->read($input);
-              $this->success []= $elem1501;
+              $elem1508 = null;
+              $elem1508 = new \metastore\HiveObjectPrivilege();
+              $xfer += $elem1508->read($input);
+              $this->success []= $elem1508;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -47801,9 +47801,9 @@ class ThriftHiveMetastore_list_privileges_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1502)
+          foreach ($this->success as $iter1509)
           {
-            $xfer += $iter1502->write($output);
+            $xfer += $iter1509->write($output);
           }
         }
         $output->writeListEnd();
@@ -48671,14 +48671,14 @@ class ThriftHiveMetastore_set_ugi_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1503 = 0;
-            $_etype1506 = 0;
-            $xfer += $input->readListBegin($_etype1506, $_size1503);
-            for ($_i1507 = 0; $_i1507 < $_size1503; ++$_i1507)
+            $_size1510 = 0;
+            $_etype1513 = 0;
+            $xfer += $input->readListBegin($_etype1513, $_size1510);
+            for ($_i1514 = 0; $_i1514 < $_size1510; ++$_i1514)
             {
-              $elem1508 = null;
-              $xfer += $input->readString($elem1508);
-              $this->group_names []= $elem1508;
+              $elem1515 = null;
+              $xfer += $input->readString($elem1515);
+              $this->group_names []= $elem1515;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -48711,9 +48711,9 @@ class ThriftHiveMetastore_set_ugi_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1509)
+          foreach ($this->group_names as $iter1516)
           {
-            $xfer += $output->writeString($iter1509);
+            $xfer += $output->writeString($iter1516);
           }
         }
         $output->writeListEnd();
@@ -48789,14 +48789,14 @@ class ThriftHiveMetastore_set_ugi_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1510 = 0;
-            $_etype1513 = 0;
-            $xfer += $input->readListBegin($_etype1513, $_size1510);
-            for ($_i1514 = 0; $_i1514 < $_size1510; ++$_i1514)
+            $_size1517 = 0;
+            $_etype1520 = 0;
+            $xfer += $input->readListBegin($_etype1520, $_size1517);
+            for ($_i1521 = 0; $_i1521 < $_size1517; ++$_i1521)
             {
-              $elem1515 = null;
-              $xfer += $input->readString($elem1515);
-              $this->success []= $elem1515;
+              $elem1522 = null;
+              $xfer += $input->readString($elem1522);
+              $this->success []= $elem1522;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -48832,9 +48832,9 @@ class ThriftHiveMetastore_set_ugi_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1516)
+          foreach ($this->success as $iter1523)
           {
-            $xfer += $output->writeString($iter1516);
+            $xfer += $output->writeString($iter1523);
           }
         }
         $output->writeListEnd();
@@ -49951,14 +49951,14 @@ class ThriftHiveMetastore_get_all_token_identifiers_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1517 = 0;
-            $_etype1520 = 0;
-            $xfer += $input->readListBegin($_etype1520, $_size1517);
-            for ($_i1521 = 0; $_i1521 < $_size1517; ++$_i1521)
+            $_size1524 = 0;
+            $_etype1527 = 0;
+            $xfer += $input->readListBegin($_etype1527, $_size1524);
+            for ($_i1528 = 0; $_i1528 < $_size1524; ++$_i1528)
             {
-              $elem1522 = null;
-              $xfer += $input->readString($elem1522);
-              $this->success []= $elem1522;
+              $elem1529 = null;
+              $xfer += $input->readString($elem1529);
+              $this->success []= $elem1529;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -49986,9 +49986,9 @@ class ThriftHiveMetastore_get_all_token_identifiers_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1523)
+          foreach ($this->success as $iter1530)
           {
-            $xfer += $output->writeString($iter1523);
+            $xfer += $output->writeString($iter1530);
           }
         }
         $output->writeListEnd();
@@ -50627,14 +50627,14 @@ class ThriftHiveMetastore_get_master_keys_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1524 = 0;
-            $_etype1527 = 0;
-            $xfer += $input->readListBegin($_etype1527, $_size1524);
-            for ($_i1528 = 0; $_i1528 < $_size1524; ++$_i1528)
+            $_size1531 = 0;
+            $_etype1534 = 0;
+            $xfer += $input->readListBegin($_etype1534, $_size1531);
+            for ($_i1535 = 0; $_i1535 < $_size1531; ++$_i1535)
             {
-              $elem1529 = null;
-              $xfer += $input->readString($elem1529);
-              $this->success []= $elem1529;
+              $elem1536 = null;
+              $xfer += $input->readString($elem1536);
+              $this->success []= $elem1536;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -50662,9 +50662,9 @@ class ThriftHiveMetastore_get_master_keys_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1530)
+          foreach ($this->success as $iter1537)
           {
-            $xfer += $output->writeString($iter1530);
+            $xfer += $output->writeString($iter1537);
           }
         }
         $output->writeListEnd();
@@ -54418,14 +54418,14 @@ class ThriftHiveMetastore_find_columns_with_stats_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1531 = 0;
-            $_etype1534 = 0;
-            $xfer += $input->readListBegin($_etype1534, $_size1531);
-            for ($_i1535 = 0; $_i1535 < $_size1531; ++$_i1535)
+            $_size1538 = 0;
+            $_etype1541 = 0;
+            $xfer += $input->readListBegin($_etype1541, $_size1538);
+            for ($_i1542 = 0; $_i1542 < $_size1538; ++$_i1542)
             {
-              $elem1536 = null;
-              $xfer += $input->readString($elem1536);
-              $this->success []= $elem1536;
+              $elem1543 = null;
+              $xfer += $input->readString($elem1543);
+              $this->success []= $elem1543;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -54453,9 +54453,9 @@ class ThriftHiveMetastore_find_columns_with_stats_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1537)
+          foreach ($this->success as $iter1544)
           {
-            $xfer += $output->writeString($iter1537);
+            $xfer += $output->writeString($iter1544);
           }
         }
         $output->writeListEnd();
@@ -62626,15 +62626,15 @@ class ThriftHiveMetastore_get_schema_all_versions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1538 = 0;
-            $_etype1541 = 0;
-            $xfer += $input->readListBegin($_etype1541, $_size1538);
-            for ($_i1542 = 0; $_i1542 < $_size1538; ++$_i1542)
+            $_size1545 = 0;
+            $_etype1548 = 0;
+            $xfer += $input->readListBegin($_etype1548, $_size1545);
+            for ($_i1549 = 0; $_i1549 < $_size1545; ++$_i1549)
             {
-              $elem1543 = null;
-              $elem1543 = new \metastore\SchemaVersion();
-              $xfer += $elem1543->read($input);
-              $this->success []= $elem1543;
+              $elem1550 = null;
+              $elem1550 = new \metastore\SchemaVersion();
+              $xfer += $elem1550->read($input);
+              $this->success []= $elem1550;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -62678,9 +62678,9 @@ class ThriftHiveMetastore_get_schema_all_versions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1544)
+          foreach ($this->success as $iter1551)
           {
-            $xfer += $iter1544->write($output);
+            $xfer += $iter1551->write($output);
           }
         }
         $output->writeListEnd();
@@ -64549,15 +64549,15 @@ class ThriftHiveMetastore_get_runtime_stats_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1545 = 0;
-            $_etype1548 = 0;
-            $xfer += $input->readListBegin($_etype1548, $_size1545);
-            for ($_i1549 = 0; $_i1549 < $_size1545; ++$_i1549)
+            $_size1552 = 0;
+            $_etype1555 = 0;
+            $xfer += $input->readListBegin($_etype1555, $_size1552);
+            for ($_i1556 = 0; $_i1556 < $_size1552; ++$_i1556)
             {
-              $elem1550 = null;
-              $elem1550 = new \metastore\RuntimeStat();
-              $xfer += $elem1550->read($input);
-              $this->success []= $elem1550;
+              $elem1557 = null;
+              $elem1557 = new \metastore\RuntimeStat();
+              $xfer += $elem1557->read($input);
+              $this->success []= $elem1557;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -64593,9 +64593,9 @@ class ThriftHiveMetastore_get_runtime_stats_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1551)
+          foreach ($this->success as $iter1558)
           {
-            $xfer += $iter1551->write($output);
+            $xfer += $iter1558->write($output);
           }
         }
         $output->writeListEnd();
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/Types.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/Types.php
index 481735c..c4add01 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/Types.php
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/Types.php
@@ -35476,6 +35476,14 @@ class AlterTableRequest {
    * @var string
    */
   public $validWriteIdList = null;
+  /**
+   * @var string[]
+   */
+  public $processorCapabilities = null;
+  /**
+   * @var string
+   */
+  public $processorIdentifier = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -35510,6 +35518,18 @@ class AlterTableRequest {
           'var' => 'validWriteIdList',
           'type' => TType::STRING,
           ),
+        8 => array(
+          'var' => 'processorCapabilities',
+          'type' => TType::LST,
+          'etype' => TType::STRING,
+          'elem' => array(
+            'type' => TType::STRING,
+            ),
+          ),
+        9 => array(
+          'var' => 'processorIdentifier',
+          'type' => TType::STRING,
+          ),
         );
     }
     if (is_array($vals)) {
@@ -35534,6 +35554,12 @@ class AlterTableRequest {
       if (isset($vals['validWriteIdList'])) {
         $this->validWriteIdList = $vals['validWriteIdList'];
       }
+      if (isset($vals['processorCapabilities'])) {
+        $this->processorCapabilities = $vals['processorCapabilities'];
+      }
+      if (isset($vals['processorIdentifier'])) {
+        $this->processorIdentifier = $vals['processorIdentifier'];
+      }
     }
   }
 
@@ -35607,6 +35633,30 @@ class AlterTableRequest {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 8:
+          if ($ftype == TType::LST) {
+            $this->processorCapabilities = array();
+            $_size994 = 0;
+            $_etype997 = 0;
+            $xfer += $input->readListBegin($_etype997, $_size994);
+            for ($_i998 = 0; $_i998 < $_size994; ++$_i998)
+            {
+              $elem999 = null;
+              $xfer += $input->readString($elem999);
+              $this->processorCapabilities []= $elem999;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 9:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->processorIdentifier);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -35661,6 +35711,28 @@ class AlterTableRequest {
       $xfer += $output->writeString($this->validWriteIdList);
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->processorCapabilities !== null) {
+      if (!is_array($this->processorCapabilities)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('processorCapabilities', TType::LST, 8);
+      {
+        $output->writeListBegin(TType::STRING, count($this->processorCapabilities));
+        {
+          foreach ($this->processorCapabilities as $iter1000)
+          {
+            $xfer += $output->writeString($iter1000);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->processorIdentifier !== null) {
+      $xfer += $output->writeFieldBegin('processorIdentifier', TType::STRING, 9);
+      $xfer += $output->writeString($this->processorIdentifier);
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;
@@ -35790,14 +35862,14 @@ class GetPartitionsProjectionSpec {
         case 1:
           if ($ftype == TType::LST) {
             $this->fieldList = array();
-            $_size994 = 0;
-            $_etype997 = 0;
-            $xfer += $input->readListBegin($_etype997, $_size994);
-            for ($_i998 = 0; $_i998 < $_size994; ++$_i998)
+            $_size1001 = 0;
+            $_etype1004 = 0;
+            $xfer += $input->readListBegin($_etype1004, $_size1001);
+            for ($_i1005 = 0; $_i1005 < $_size1001; ++$_i1005)
             {
-              $elem999 = null;
-              $xfer += $input->readString($elem999);
-              $this->fieldList []= $elem999;
+              $elem1006 = null;
+              $xfer += $input->readString($elem1006);
+              $this->fieldList []= $elem1006;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -35839,9 +35911,9 @@ class GetPartitionsProjectionSpec {
       {
         $output->writeListBegin(TType::STRING, count($this->fieldList));
         {
-          foreach ($this->fieldList as $iter1000)
+          foreach ($this->fieldList as $iter1007)
           {
-            $xfer += $output->writeString($iter1000);
+            $xfer += $output->writeString($iter1007);
           }
         }
         $output->writeListEnd();
@@ -35933,14 +36005,14 @@ class GetPartitionsFilterSpec {
         case 8:
           if ($ftype == TType::LST) {
             $this->filters = array();
-            $_size1001 = 0;
-            $_etype1004 = 0;
-            $xfer += $input->readListBegin($_etype1004, $_size1001);
-            for ($_i1005 = 0; $_i1005 < $_size1001; ++$_i1005)
+            $_size1008 = 0;
+            $_etype1011 = 0;
+            $xfer += $input->readListBegin($_etype1011, $_size1008);
+            for ($_i1012 = 0; $_i1012 < $_size1008; ++$_i1012)
             {
-              $elem1006 = null;
-              $xfer += $input->readString($elem1006);
-              $this->filters []= $elem1006;
+              $elem1013 = null;
+              $xfer += $input->readString($elem1013);
+              $this->filters []= $elem1013;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -35973,9 +36045,9 @@ class GetPartitionsFilterSpec {
       {
         $output->writeListBegin(TType::STRING, count($this->filters));
         {
-          foreach ($this->filters as $iter1007)
+          foreach ($this->filters as $iter1014)
           {
-            $xfer += $output->writeString($iter1007);
+            $xfer += $output->writeString($iter1014);
           }
         }
         $output->writeListEnd();
@@ -36040,15 +36112,15 @@ class GetPartitionsResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->partitionSpec = array();
-            $_size1008 = 0;
-            $_etype1011 = 0;
-            $xfer += $input->readListBegin($_etype1011, $_size1008);
-            for ($_i1012 = 0; $_i1012 < $_size1008; ++$_i1012)
+            $_size1015 = 0;
+            $_etype1018 = 0;
+            $xfer += $input->readListBegin($_etype1018, $_size1015);
+            for ($_i1019 = 0; $_i1019 < $_size1015; ++$_i1019)
             {
-              $elem1013 = null;
-              $elem1013 = new \metastore\PartitionSpec();
-              $xfer += $elem1013->read($input);
-              $this->partitionSpec []= $elem1013;
+              $elem1020 = null;
+              $elem1020 = new \metastore\PartitionSpec();
+              $xfer += $elem1020->read($input);
+              $this->partitionSpec []= $elem1020;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -36076,9 +36148,9 @@ class GetPartitionsResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->partitionSpec));
         {
-          foreach ($this->partitionSpec as $iter1014)
+          foreach ($this->partitionSpec as $iter1021)
           {
-            $xfer += $iter1014->write($output);
+            $xfer += $iter1021->write($output);
           }
         }
         $output->writeListEnd();
@@ -36282,14 +36354,14 @@ class GetPartitionsRequest {
         case 6:
           if ($ftype == TType::LST) {
             $this->groupNames = array();
-            $_size1015 = 0;
-            $_etype1018 = 0;
-            $xfer += $input->readListBegin($_etype1018, $_size1015);
-            for ($_i1019 = 0; $_i1019 < $_size1015; ++$_i1019)
+            $_size1022 = 0;
+            $_etype1025 = 0;
+            $xfer += $input->readListBegin($_etype1025, $_size1022);
+            for ($_i1026 = 0; $_i1026 < $_size1022; ++$_i1026)
             {
-              $elem1020 = null;
-              $xfer += $input->readString($elem1020);
-              $this->groupNames []= $elem1020;
+              $elem1027 = null;
+              $xfer += $input->readString($elem1027);
+              $this->groupNames []= $elem1027;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -36315,14 +36387,14 @@ class GetPartitionsRequest {
         case 9:
           if ($ftype == TType::LST) {
             $this->processorCapabilities = array();
-            $_size1021 = 0;
-            $_etype1024 = 0;
-            $xfer += $input->readListBegin($_etype1024, $_size1021);
-            for ($_i1025 = 0; $_i1025 < $_size1021; ++$_i1025)
+            $_size1028 = 0;
+            $_etype1031 = 0;
+            $xfer += $input->readListBegin($_etype1031, $_size1028);
+            for ($_i1032 = 0; $_i1032 < $_size1028; ++$_i1032)
             {
-              $elem1026 = null;
-              $xfer += $input->readString($elem1026);
-              $this->processorCapabilities []= $elem1026;
+              $elem1033 = null;
+              $xfer += $input->readString($elem1033);
+              $this->processorCapabilities []= $elem1033;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -36382,9 +36454,9 @@ class GetPartitionsRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->groupNames));
         {
-          foreach ($this->groupNames as $iter1027)
+          foreach ($this->groupNames as $iter1034)
           {
-            $xfer += $output->writeString($iter1027);
+            $xfer += $output->writeString($iter1034);
           }
         }
         $output->writeListEnd();
@@ -36415,9 +36487,9 @@ class GetPartitionsRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->processorCapabilities));
         {
-          foreach ($this->processorCapabilities as $iter1028)
+          foreach ($this->processorCapabilities as $iter1035)
           {
-            $xfer += $output->writeString($iter1028);
+            $xfer += $output->writeString($iter1035);
           }
         }
         $output->writeListEnd();
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
index 1bc6cde..c0ecb64 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -17447,10 +17447,10 @@ class get_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1029, _size1026) = iprot.readListBegin()
-          for _i1030 in xrange(_size1026):
-            _elem1031 = iprot.readString()
-            self.success.append(_elem1031)
+          (_etype1036, _size1033) = iprot.readListBegin()
+          for _i1037 in xrange(_size1033):
+            _elem1038 = iprot.readString()
+            self.success.append(_elem1038)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17473,8 +17473,8 @@ class get_databases_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1032 in self.success:
-        oprot.writeString(iter1032)
+      for iter1039 in self.success:
+        oprot.writeString(iter1039)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17579,10 +17579,10 @@ class get_all_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1036, _size1033) = iprot.readListBegin()
-          for _i1037 in xrange(_size1033):
-            _elem1038 = iprot.readString()
-            self.success.append(_elem1038)
+          (_etype1043, _size1040) = iprot.readListBegin()
+          for _i1044 in xrange(_size1040):
+            _elem1045 = iprot.readString()
+            self.success.append(_elem1045)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17605,8 +17605,8 @@ class get_all_databases_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1039 in self.success:
-        oprot.writeString(iter1039)
+      for iter1046 in self.success:
+        oprot.writeString(iter1046)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18376,12 +18376,12 @@ class get_type_all_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype1041, _vtype1042, _size1040 ) = iprot.readMapBegin()
-          for _i1044 in xrange(_size1040):
-            _key1045 = iprot.readString()
-            _val1046 = Type()
-            _val1046.read(iprot)
-            self.success[_key1045] = _val1046
+          (_ktype1048, _vtype1049, _size1047 ) = iprot.readMapBegin()
+          for _i1051 in xrange(_size1047):
+            _key1052 = iprot.readString()
+            _val1053 = Type()
+            _val1053.read(iprot)
+            self.success[_key1052] = _val1053
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -18404,9 +18404,9 @@ class get_type_all_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.MAP, 0)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.success))
-      for kiter1047,viter1048 in self.success.items():
-        oprot.writeString(kiter1047)
-        viter1048.write(oprot)
+      for kiter1054,viter1055 in self.success.items():
+        oprot.writeString(kiter1054)
+        viter1055.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -18549,11 +18549,11 @@ class get_fields_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1052, _size1049) = iprot.readListBegin()
-          for _i1053 in xrange(_size1049):
-            _elem1054 = FieldSchema()
-            _elem1054.read(iprot)
-            self.success.append(_elem1054)
+          (_etype1059, _size1056) = iprot.readListBegin()
+          for _i1060 in xrange(_size1056):
+            _elem1061 = FieldSchema()
+            _elem1061.read(iprot)
+            self.success.append(_elem1061)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18588,8 +18588,8 @@ class get_fields_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1055 in self.success:
-        iter1055.write(oprot)
+      for iter1062 in self.success:
+        iter1062.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18756,11 +18756,11 @@ class get_fields_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1059, _size1056) = iprot.readListBegin()
-          for _i1060 in xrange(_size1056):
-            _elem1061 = FieldSchema()
-            _elem1061.read(iprot)
-            self.success.append(_elem1061)
+          (_etype1066, _size1063) = iprot.readListBegin()
+          for _i1067 in xrange(_size1063):
+            _elem1068 = FieldSchema()
+            _elem1068.read(iprot)
+            self.success.append(_elem1068)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18795,8 +18795,8 @@ class get_fields_with_environment_context_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1062 in self.success:
-        iter1062.write(oprot)
+      for iter1069 in self.success:
+        iter1069.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18949,11 +18949,11 @@ class get_schema_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1066, _size1063) = iprot.readListBegin()
-          for _i1067 in xrange(_size1063):
-            _elem1068 = FieldSchema()
-            _elem1068.read(iprot)
-            self.success.append(_elem1068)
+          (_etype1073, _size1070) = iprot.readListBegin()
+          for _i1074 in xrange(_size1070):
+            _elem1075 = FieldSchema()
+            _elem1075.read(iprot)
+            self.success.append(_elem1075)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18988,8 +18988,8 @@ class get_schema_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1069 in self.success:
-        iter1069.write(oprot)
+      for iter1076 in self.success:
+        iter1076.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19156,11 +19156,11 @@ class get_schema_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1073, _size1070) = iprot.readListBegin()
-          for _i1074 in xrange(_size1070):
-            _elem1075 = FieldSchema()
-            _elem1075.read(iprot)
-            self.success.append(_elem1075)
+          (_etype1080, _size1077) = iprot.readListBegin()
+          for _i1081 in xrange(_size1077):
+            _elem1082 = FieldSchema()
+            _elem1082.read(iprot)
+            self.success.append(_elem1082)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19195,8 +19195,8 @@ class get_schema_with_environment_context_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1076 in self.success:
-        iter1076.write(oprot)
+      for iter1083 in self.success:
+        iter1083.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19649,66 +19649,66 @@ class create_table_with_constraints_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.primaryKeys = []
-          (_etype1080, _size1077) = iprot.readListBegin()
-          for _i1081 in xrange(_size1077):
-            _elem1082 = SQLPrimaryKey()
-            _elem1082.read(iprot)
-            self.primaryKeys.append(_elem1082)
+          (_etype1087, _size1084) = iprot.readListBegin()
+          for _i1088 in xrange(_size1084):
+            _elem1089 = SQLPrimaryKey()
+            _elem1089.read(iprot)
+            self.primaryKeys.append(_elem1089)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.foreignKeys = []
-          (_etype1086, _size1083) = iprot.readListBegin()
-          for _i1087 in xrange(_size1083):
-            _elem1088 = SQLForeignKey()
-            _elem1088.read(iprot)
-            self.foreignKeys.append(_elem1088)
+          (_etype1093, _size1090) = iprot.readListBegin()
+          for _i1094 in xrange(_size1090):
+            _elem1095 = SQLForeignKey()
+            _elem1095.read(iprot)
+            self.foreignKeys.append(_elem1095)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.LIST:
           self.uniqueConstraints = []
-          (_etype1092, _size1089) = iprot.readListBegin()
-          for _i1093 in xrange(_size1089):
-            _elem1094 = SQLUniqueConstraint()
-            _elem1094.read(iprot)
-            self.uniqueConstraints.append(_elem1094)
+          (_etype1099, _size1096) = iprot.readListBegin()
+          for _i1100 in xrange(_size1096):
+            _elem1101 = SQLUniqueConstraint()
+            _elem1101.read(iprot)
+            self.uniqueConstraints.append(_elem1101)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.LIST:
           self.notNullConstraints = []
-          (_etype1098, _size1095) = iprot.readListBegin()
-          for _i1099 in xrange(_size1095):
-            _elem1100 = SQLNotNullConstraint()
-            _elem1100.read(iprot)
-            self.notNullConstraints.append(_elem1100)
+          (_etype1105, _size1102) = iprot.readListBegin()
+          for _i1106 in xrange(_size1102):
+            _elem1107 = SQLNotNullConstraint()
+            _elem1107.read(iprot)
+            self.notNullConstraints.append(_elem1107)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 6:
         if ftype == TType.LIST:
           self.defaultConstraints = []
-          (_etype1104, _size1101) = iprot.readListBegin()
-          for _i1105 in xrange(_size1101):
-            _elem1106 = SQLDefaultConstraint()
-            _elem1106.read(iprot)
-            self.defaultConstraints.append(_elem1106)
+          (_etype1111, _size1108) = iprot.readListBegin()
+          for _i1112 in xrange(_size1108):
+            _elem1113 = SQLDefaultConstraint()
+            _elem1113.read(iprot)
+            self.defaultConstraints.append(_elem1113)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 7:
         if ftype == TType.LIST:
           self.checkConstraints = []
-          (_etype1110, _size1107) = iprot.readListBegin()
-          for _i1111 in xrange(_size1107):
-            _elem1112 = SQLCheckConstraint()
-            _elem1112.read(iprot)
-            self.checkConstraints.append(_elem1112)
+          (_etype1117, _size1114) = iprot.readListBegin()
+          for _i1118 in xrange(_size1114):
+            _elem1119 = SQLCheckConstraint()
+            _elem1119.read(iprot)
+            self.checkConstraints.append(_elem1119)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19729,43 +19729,43 @@ class create_table_with_constraints_args:
     if self.primaryKeys is not None:
       oprot.writeFieldBegin('primaryKeys', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.primaryKeys))
-      for iter1113 in self.primaryKeys:
-        iter1113.write(oprot)
+      for iter1120 in self.primaryKeys:
+        iter1120.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.foreignKeys is not None:
       oprot.writeFieldBegin('foreignKeys', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.foreignKeys))
-      for iter1114 in self.foreignKeys:
-        iter1114.write(oprot)
+      for iter1121 in self.foreignKeys:
+        iter1121.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.uniqueConstraints is not None:
       oprot.writeFieldBegin('uniqueConstraints', TType.LIST, 4)
       oprot.writeListBegin(TType.STRUCT, len(self.uniqueConstraints))
-      for iter1115 in self.uniqueConstraints:
-        iter1115.write(oprot)
+      for iter1122 in self.uniqueConstraints:
+        iter1122.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.notNullConstraints is not None:
       oprot.writeFieldBegin('notNullConstraints', TType.LIST, 5)
       oprot.writeListBegin(TType.STRUCT, len(self.notNullConstraints))
-      for iter1116 in self.notNullConstraints:
-        iter1116.write(oprot)
+      for iter1123 in self.notNullConstraints:
+        iter1123.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.defaultConstraints is not None:
       oprot.writeFieldBegin('defaultConstraints', TType.LIST, 6)
       oprot.writeListBegin(TType.STRUCT, len(self.defaultConstraints))
-      for iter1117 in self.defaultConstraints:
-        iter1117.write(oprot)
+      for iter1124 in self.defaultConstraints:
+        iter1124.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.checkConstraints is not None:
       oprot.writeFieldBegin('checkConstraints', TType.LIST, 7)
       oprot.writeListBegin(TType.STRUCT, len(self.checkConstraints))
-      for iter1118 in self.checkConstraints:
-        iter1118.write(oprot)
+      for iter1125 in self.checkConstraints:
+        iter1125.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21499,10 +21499,10 @@ class truncate_table_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.partNames = []
-          (_etype1122, _size1119) = iprot.readListBegin()
-          for _i1123 in xrange(_size1119):
-            _elem1124 = iprot.readString()
-            self.partNames.append(_elem1124)
+          (_etype1129, _size1126) = iprot.readListBegin()
+          for _i1130 in xrange(_size1126):
+            _elem1131 = iprot.readString()
+            self.partNames.append(_elem1131)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21527,8 +21527,8 @@ class truncate_table_args:
     if self.partNames is not None:
       oprot.writeFieldBegin('partNames', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.partNames))
-      for iter1125 in self.partNames:
-        oprot.writeString(iter1125)
+      for iter1132 in self.partNames:
+        oprot.writeString(iter1132)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21873,10 +21873,10 @@ class get_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1129, _size1126) = iprot.readListBegin()
-          for _i1130 in xrange(_size1126):
-            _elem1131 = iprot.readString()
-            self.success.append(_elem1131)
+          (_etype1136, _size1133) = iprot.readListBegin()
+          for _i1137 in xrange(_size1133):
+            _elem1138 = iprot.readString()
+            self.success.append(_elem1138)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21899,8 +21899,8 @@ class get_tables_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1132 in self.success:
-        oprot.writeString(iter1132)
+      for iter1139 in self.success:
+        oprot.writeString(iter1139)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -22050,10 +22050,10 @@ class get_tables_by_type_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1136, _size1133) = iprot.readListBegin()
-          for _i1137 in xrange(_size1133):
-            _elem1138 = iprot.readString()
-            self.success.append(_elem1138)
+          (_etype1143, _size1140) = iprot.readListBegin()
+          for _i1144 in xrange(_size1140):
+            _elem1145 = iprot.readString()
+            self.success.append(_elem1145)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22076,8 +22076,8 @@ class get_tables_by_type_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1139 in self.success:
-        oprot.writeString(iter1139)
+      for iter1146 in self.success:
+        oprot.writeString(iter1146)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -22182,11 +22182,11 @@ class get_all_materialized_view_objects_for_rewriting_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1143, _size1140) = iprot.readListBegin()
-          for _i1144 in xrange(_size1140):
-            _elem1145 = Table()
-            _elem1145.read(iprot)
-            self.success.append(_elem1145)
+          (_etype1150, _size1147) = iprot.readListBegin()
+          for _i1151 in xrange(_size1147):
+            _elem1152 = Table()
+            _elem1152.read(iprot)
+            self.success.append(_elem1152)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22209,8 +22209,8 @@ class get_all_materialized_view_objects_for_rewriting_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1146 in self.success:
-        iter1146.write(oprot)
+      for iter1153 in self.success:
+        iter1153.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -22334,10 +22334,10 @@ class get_materialized_views_for_rewriting_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1150, _size1147) = iprot.readListBegin()
-          for _i1151 in xrange(_size1147):
-            _elem1152 = iprot.readString()
-            self.success.append(_elem1152)
+          (_etype1157, _size1154) = iprot.readListBegin()
+          for _i1158 in xrange(_size1154):
+            _elem1159 = iprot.readString()
+            self.success.append(_elem1159)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22360,8 +22360,8 @@ class get_materialized_views_for_rewriting_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1153 in self.success:
-        oprot.writeString(iter1153)
+      for iter1160 in self.success:
+        oprot.writeString(iter1160)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -22434,10 +22434,10 @@ class get_table_meta_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.tbl_types = []
-          (_etype1157, _size1154) = iprot.readListBegin()
-          for _i1158 in xrange(_size1154):
-            _elem1159 = iprot.readString()
-            self.tbl_types.append(_elem1159)
+          (_etype1164, _size1161) = iprot.readListBegin()
+          for _i1165 in xrange(_size1161):
+            _elem1166 = iprot.readString()
+            self.tbl_types.append(_elem1166)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22462,8 +22462,8 @@ class get_table_meta_args:
     if self.tbl_types is not None:
       oprot.writeFieldBegin('tbl_types', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.tbl_types))
-      for iter1160 in self.tbl_types:
-        oprot.writeString(iter1160)
+      for iter1167 in self.tbl_types:
+        oprot.writeString(iter1167)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -22519,11 +22519,11 @@ class get_table_meta_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1164, _size1161) = iprot.readListBegin()
-          for _i1165 in xrange(_size1161):
-            _elem1166 = TableMeta()
-            _elem1166.read(iprot)
-            self.success.append(_elem1166)
+          (_etype1171, _size1168) = iprot.readListBegin()
+          for _i1172 in xrange(_size1168):
+            _elem1173 = TableMeta()
+            _elem1173.read(iprot)
+            self.success.append(_elem1173)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22546,8 +22546,8 @@ class get_table_meta_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1167 in self.success:
-        iter1167.write(oprot)
+      for iter1174 in self.success:
+        iter1174.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -22671,10 +22671,10 @@ class get_all_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1171, _size1168) = iprot.readListBegin()
-          for _i1172 in xrange(_size1168):
-            _elem1173 = iprot.readString()
-            self.success.append(_elem1173)
+          (_etype1178, _size1175) = iprot.readListBegin()
+          for _i1179 in xrange(_size1175):
+            _elem1180 = iprot.readString()
+            self.success.append(_elem1180)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22697,8 +22697,8 @@ class get_all_tables_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1174 in self.success:
-        oprot.writeString(iter1174)
+      for iter1181 in self.success:
+        oprot.writeString(iter1181)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -22934,10 +22934,10 @@ class get_table_objects_by_name_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype1178, _size1175) = iprot.readListBegin()
-          for _i1179 in xrange(_size1175):
-            _elem1180 = iprot.readString()
-            self.tbl_names.append(_elem1180)
+          (_etype1185, _size1182) = iprot.readListBegin()
+          for _i1186 in xrange(_size1182):
+            _elem1187 = iprot.readString()
+            self.tbl_names.append(_elem1187)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22958,8 +22958,8 @@ class get_table_objects_by_name_args:
     if self.tbl_names is not None:
       oprot.writeFieldBegin('tbl_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.tbl_names))
-      for iter1181 in self.tbl_names:
-        oprot.writeString(iter1181)
+      for iter1188 in self.tbl_names:
+        oprot.writeString(iter1188)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -23011,11 +23011,11 @@ class get_table_objects_by_name_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1185, _size1182) = iprot.readListBegin()
-          for _i1186 in xrange(_size1182):
-            _elem1187 = Table()
-            _elem1187.read(iprot)
-            self.success.append(_elem1187)
+          (_etype1192, _size1189) = iprot.readListBegin()
+          for _i1193 in xrange(_size1189):
+            _elem1194 = Table()
+            _elem1194.read(iprot)
+            self.success.append(_elem1194)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23032,8 +23032,8 @@ class get_table_objects_by_name_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1188 in self.success:
-        iter1188.write(oprot)
+      for iter1195 in self.success:
+        iter1195.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -23153,11 +23153,11 @@ class get_tables_ext_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1192, _size1189) = iprot.readListBegin()
-          for _i1193 in xrange(_size1189):
-            _elem1194 = ExtendedTableInfo()
-            _elem1194.read(iprot)
-            self.success.append(_elem1194)
+          (_etype1199, _size1196) = iprot.readListBegin()
+          for _i1200 in xrange(_size1196):
+            _elem1201 = ExtendedTableInfo()
+            _elem1201.read(iprot)
+            self.success.append(_elem1201)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23180,8 +23180,8 @@ class get_tables_ext_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1195 in self.success:
-        iter1195.write(oprot)
+      for iter1202 in self.success:
+        iter1202.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -24054,10 +24054,10 @@ class get_table_names_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1199, _size1196) = iprot.readListBegin()
-          for _i1200 in xrange(_size1196):
-            _elem1201 = iprot.readString()
-            self.success.append(_elem1201)
+          (_etype1206, _size1203) = iprot.readListBegin()
+          for _i1207 in xrange(_size1203):
+            _elem1208 = iprot.readString()
+            self.success.append(_elem1208)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24092,8 +24092,8 @@ class get_table_names_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1202 in self.success:
-        oprot.writeString(iter1202)
+      for iter1209 in self.success:
+        oprot.writeString(iter1209)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -25222,11 +25222,11 @@ class add_partitions_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1206, _size1203) = iprot.readListBegin()
-          for _i1207 in xrange(_size1203):
-            _elem1208 = Partition()
-            _elem1208.read(iprot)
-            self.new_parts.append(_elem1208)
+          (_etype1213, _size1210) = iprot.readListBegin()
+          for _i1214 in xrange(_size1210):
+            _elem1215 = Partition()
+            _elem1215.read(iprot)
+            self.new_parts.append(_elem1215)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25243,8 +25243,8 @@ class add_partitions_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter1209 in self.new_parts:
-        iter1209.write(oprot)
+      for iter1216 in self.new_parts:
+        iter1216.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -25402,11 +25402,11 @@ class add_partitions_pspec_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1213, _size1210) = iprot.readListBegin()
-          for _i1214 in xrange(_size1210):
-            _elem1215 = PartitionSpec()
-            _elem1215.read(iprot)
-            self.new_parts.append(_elem1215)
+          (_etype1220, _size1217) = iprot.readListBegin()
+          for _i1221 in xrange(_size1217):
+            _elem1222 = PartitionSpec()
+            _elem1222.read(iprot)
+            self.new_parts.append(_elem1222)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25423,8 +25423,8 @@ class add_partitions_pspec_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter1216 in self.new_parts:
-        iter1216.write(oprot)
+      for iter1223 in self.new_parts:
+        iter1223.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -25598,10 +25598,10 @@ class append_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1220, _size1217) = iprot.readListBegin()
-          for _i1221 in xrange(_size1217):
-            _elem1222 = iprot.readString()
-            self.part_vals.append(_elem1222)
+          (_etype1227, _size1224) = iprot.readListBegin()
+          for _i1228 in xrange(_size1224):
+            _elem1229 = iprot.readString()
+            self.part_vals.append(_elem1229)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25626,8 +25626,8 @@ class append_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1223 in self.part_vals:
-        oprot.writeString(iter1223)
+      for iter1230 in self.part_vals:
+        oprot.writeString(iter1230)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -25980,10 +25980,10 @@ class append_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1227, _size1224) = iprot.readListBegin()
-          for _i1228 in xrange(_size1224):
-            _elem1229 = iprot.readString()
-            self.part_vals.append(_elem1229)
+          (_etype1234, _size1231) = iprot.readListBegin()
+          for _i1235 in xrange(_size1231):
+            _elem1236 = iprot.readString()
+            self.part_vals.append(_elem1236)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26014,8 +26014,8 @@ class append_partition_with_environment_context_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1230 in self.part_vals:
-        oprot.writeString(iter1230)
+      for iter1237 in self.part_vals:
+        oprot.writeString(iter1237)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -26610,10 +26610,10 @@ class drop_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1234, _size1231) = iprot.readListBegin()
-          for _i1235 in xrange(_size1231):
-            _elem1236 = iprot.readString()
-            self.part_vals.append(_elem1236)
+          (_etype1241, _size1238) = iprot.readListBegin()
+          for _i1242 in xrange(_size1238):
+            _elem1243 = iprot.readString()
+            self.part_vals.append(_elem1243)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26643,8 +26643,8 @@ class drop_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1237 in self.part_vals:
-        oprot.writeString(iter1237)
+      for iter1244 in self.part_vals:
+        oprot.writeString(iter1244)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -26817,10 +26817,10 @@ class drop_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1241, _size1238) = iprot.readListBegin()
-          for _i1242 in xrange(_size1238):
-            _elem1243 = iprot.readString()
-            self.part_vals.append(_elem1243)
+          (_etype1248, _size1245) = iprot.readListBegin()
+          for _i1249 in xrange(_size1245):
+            _elem1250 = iprot.readString()
+            self.part_vals.append(_elem1250)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26856,8 +26856,8 @@ class drop_partition_with_environment_context_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1244 in self.part_vals:
-        oprot.writeString(iter1244)
+      for iter1251 in self.part_vals:
+        oprot.writeString(iter1251)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -27594,10 +27594,10 @@ class get_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1248, _size1245) = iprot.readListBegin()
-          for _i1249 in xrange(_size1245):
-            _elem1250 = iprot.readString()
-            self.part_vals.append(_elem1250)
+          (_etype1255, _size1252) = iprot.readListBegin()
+          for _i1256 in xrange(_size1252):
+            _elem1257 = iprot.readString()
+            self.part_vals.append(_elem1257)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27622,8 +27622,8 @@ class get_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1251 in self.part_vals:
-        oprot.writeString(iter1251)
+      for iter1258 in self.part_vals:
+        oprot.writeString(iter1258)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -27782,11 +27782,11 @@ class exchange_partition_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype1253, _vtype1254, _size1252 ) = iprot.readMapBegin()
-          for _i1256 in xrange(_size1252):
-            _key1257 = iprot.readString()
-            _val1258 = iprot.readString()
-            self.partitionSpecs[_key1257] = _val1258
+          (_ktype1260, _vtype1261, _size1259 ) = iprot.readMapBegin()
+          for _i1263 in xrange(_size1259):
+            _key1264 = iprot.readString()
+            _val1265 = iprot.readString()
+            self.partitionSpecs[_key1264] = _val1265
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -27823,9 +27823,9 @@ class exchange_partition_args:
     if self.partitionSpecs is not None:
       oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs))
-      for kiter1259,viter1260 in self.partitionSpecs.items():
-        oprot.writeString(kiter1259)
-        oprot.writeString(viter1260)
+      for kiter1266,viter1267 in self.partitionSpecs.items():
+        oprot.writeString(kiter1266)
+        oprot.writeString(viter1267)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -28030,11 +28030,11 @@ class exchange_partitions_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype1262, _vtype1263, _size1261 ) = iprot.readMapBegin()
-          for _i1265 in xrange(_size1261):
-            _key1266 = iprot.readString()
-            _val1267 = iprot.readString()
-            self.partitionSpecs[_key1266] = _val1267
+          (_ktype1269, _vtype1270, _size1268 ) = iprot.readMapBegin()
+          for _i1272 in xrange(_size1268):
+            _key1273 = iprot.readString()
+            _val1274 = iprot.readString()
+            self.partitionSpecs[_key1273] = _val1274
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -28071,9 +28071,9 @@ class exchange_partitions_args:
     if self.partitionSpecs is not None:
       oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs))
-      for kiter1268,viter1269 in self.partitionSpecs.items():
-        oprot.writeString(kiter1268)
-        oprot.writeString(viter1269)
+      for kiter1275,viter1276 in self.partitionSpecs.items():
+        oprot.writeString(kiter1275)
+        oprot.writeString(viter1276)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -28156,11 +28156,11 @@ class exchange_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1273, _size1270) = iprot.readListBegin()
-          for _i1274 in xrange(_size1270):
-            _elem1275 = Partition()
-            _elem1275.read(iprot)
-            self.success.append(_elem1275)
+          (_etype1280, _size1277) = iprot.readListBegin()
+          for _i1281 in xrange(_size1277):
+            _elem1282 = Partition()
+            _elem1282.read(iprot)
+            self.success.append(_elem1282)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28201,8 +28201,8 @@ class exchange_partitions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1276 in self.success:
-        iter1276.write(oprot)
+      for iter1283 in self.success:
+        iter1283.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -28296,10 +28296,10 @@ class get_partition_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1280, _size1277) = iprot.readListBegin()
-          for _i1281 in xrange(_size1277):
-            _elem1282 = iprot.readString()
-            self.part_vals.append(_elem1282)
+          (_etype1287, _size1284) = iprot.readListBegin()
+          for _i1288 in xrange(_size1284):
+            _elem1289 = iprot.readString()
+            self.part_vals.append(_elem1289)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28311,10 +28311,10 @@ class get_partition_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1286, _size1283) = iprot.readListBegin()
-          for _i1287 in xrange(_size1283):
-            _elem1288 = iprot.readString()
-            self.group_names.append(_elem1288)
+          (_etype1293, _size1290) = iprot.readListBegin()
+          for _i1294 in xrange(_size1290):
+            _elem1295 = iprot.readString()
+            self.group_names.append(_elem1295)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28339,8 +28339,8 @@ class get_partition_with_auth_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1289 in self.part_vals:
-        oprot.writeString(iter1289)
+      for iter1296 in self.part_vals:
+        oprot.writeString(iter1296)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.user_name is not None:
@@ -28350,8 +28350,8 @@ class get_partition_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter1290 in self.group_names:
-        oprot.writeString(iter1290)
+      for iter1297 in self.group_names:
+        oprot.writeString(iter1297)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -28780,11 +28780,11 @@ class get_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1294, _size1291) = iprot.readListBegin()
-          for _i1295 in xrange(_size1291):
-            _elem1296 = Partition()
-            _elem1296.read(iprot)
-            self.success.append(_elem1296)
+          (_etype1301, _size1298) = iprot.readListBegin()
+          for _i1302 in xrange(_size1298):
+            _elem1303 = Partition()
+            _elem1303.read(iprot)
+            self.success.append(_elem1303)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28813,8 +28813,8 @@ class get_partitions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1297 in self.success:
-        iter1297.write(oprot)
+      for iter1304 in self.success:
+        iter1304.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -28908,10 +28908,10 @@ class get_partitions_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1301, _size1298) = iprot.readListBegin()
-          for _i1302 in xrange(_size1298):
-            _elem1303 = iprot.readString()
-            self.group_names.append(_elem1303)
+          (_etype1308, _size1305) = iprot.readListBegin()
+          for _i1309 in xrange(_size1305):
+            _elem1310 = iprot.readString()
+            self.group_names.append(_elem1310)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28944,8 +28944,8 @@ class get_partitions_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter1304 in self.group_names:
-        oprot.writeString(iter1304)
+      for iter1311 in self.group_names:
+        oprot.writeString(iter1311)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -29006,11 +29006,11 @@ class get_partitions_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1308, _size1305) = iprot.readListBegin()
-          for _i1309 in xrange(_size1305):
-            _elem1310 = Partition()
-            _elem1310.read(iprot)
-            self.success.append(_elem1310)
+          (_etype1315, _size1312) = iprot.readListBegin()
+          for _i1316 in xrange(_size1312):
+            _elem1317 = Partition()
+            _elem1317.read(iprot)
+            self.success.append(_elem1317)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29039,8 +29039,8 @@ class get_partitions_with_auth_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1311 in self.success:
-        iter1311.write(oprot)
+      for iter1318 in self.success:
+        iter1318.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -29198,11 +29198,11 @@ class get_partitions_pspec_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1315, _size1312) = iprot.readListBegin()
-          for _i1316 in xrange(_size1312):
-            _elem1317 = PartitionSpec()
-            _elem1317.read(iprot)
-            self.success.append(_elem1317)
+          (_etype1322, _size1319) = iprot.readListBegin()
+          for _i1323 in xrange(_size1319):
+            _elem1324 = PartitionSpec()
+            _elem1324.read(iprot)
+            self.success.append(_elem1324)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29231,8 +29231,8 @@ class get_partitions_pspec_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1318 in self.success:
-        iter1318.write(oprot)
+      for iter1325 in self.success:
+        iter1325.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -29390,10 +29390,10 @@ class get_partition_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1322, _size1319) = iprot.readListBegin()
-          for _i1323 in xrange(_size1319):
-            _elem1324 = iprot.readString()
-            self.success.append(_elem1324)
+          (_etype1329, _size1326) = iprot.readListBegin()
+          for _i1330 in xrange(_size1326):
+            _elem1331 = iprot.readString()
+            self.success.append(_elem1331)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29422,8 +29422,8 @@ class get_partition_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1325 in self.success:
-        oprot.writeString(iter1325)
+      for iter1332 in self.success:
+        oprot.writeString(iter1332)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -29663,10 +29663,10 @@ class get_partitions_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1329, _size1326) = iprot.readListBegin()
-          for _i1330 in xrange(_size1326):
-            _elem1331 = iprot.readString()
-            self.part_vals.append(_elem1331)
+          (_etype1336, _size1333) = iprot.readListBegin()
+          for _i1337 in xrange(_size1333):
+            _elem1338 = iprot.readString()
+            self.part_vals.append(_elem1338)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29696,8 +29696,8 @@ class get_partitions_ps_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1332 in self.part_vals:
-        oprot.writeString(iter1332)
+      for iter1339 in self.part_vals:
+        oprot.writeString(iter1339)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -29761,11 +29761,11 @@ class get_partitions_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1336, _size1333) = iprot.readListBegin()
-          for _i1337 in xrange(_size1333):
-            _elem1338 = Partition()
-            _elem1338.read(iprot)
-            self.success.append(_elem1338)
+          (_etype1343, _size1340) = iprot.readListBegin()
+          for _i1344 in xrange(_size1340):
+            _elem1345 = Partition()
+            _elem1345.read(iprot)
+            self.success.append(_elem1345)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29794,8 +29794,8 @@ class get_partitions_ps_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1339 in self.success:
-        iter1339.write(oprot)
+      for iter1346 in self.success:
+        iter1346.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -29882,10 +29882,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1343, _size1340) = iprot.readListBegin()
-          for _i1344 in xrange(_size1340):
-            _elem1345 = iprot.readString()
-            self.part_vals.append(_elem1345)
+          (_etype1350, _size1347) = iprot.readListBegin()
+          for _i1351 in xrange(_size1347):
+            _elem1352 = iprot.readString()
+            self.part_vals.append(_elem1352)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29902,10 +29902,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 6:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1349, _size1346) = iprot.readListBegin()
-          for _i1350 in xrange(_size1346):
-            _elem1351 = iprot.readString()
-            self.group_names.append(_elem1351)
+          (_etype1356, _size1353) = iprot.readListBegin()
+          for _i1357 in xrange(_size1353):
+            _elem1358 = iprot.readString()
+            self.group_names.append(_elem1358)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29930,8 +29930,8 @@ class get_partitions_ps_with_auth_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1352 in self.part_vals:
-        oprot.writeString(iter1352)
+      for iter1359 in self.part_vals:
+        oprot.writeString(iter1359)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -29945,8 +29945,8 @@ class get_partitions_ps_with_auth_args:
     if self.group_names is not None:
... 1174 lines suppressed ...