You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by we...@apache.org on 2016/05/27 22:14:14 UTC

[1/7] hive git commit: HIVE-13354 : Add ability to specify Compaction options per table and per request (Wei Zheng, reviewed by Eugene Koifman)

Repository: hive
Updated Branches:
  refs/heads/branch-1 abaf88248 -> 6e0504d9a


http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
index f8798b7..0601a29 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
@@ -169,7 +169,7 @@ class CompactionTxnHandler extends TxnHandler {
         dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
         stmt = dbConn.createStatement();
         String s = "select cq_id, cq_database, cq_table, cq_partition, " +
-          "cq_type from COMPACTION_QUEUE where cq_state = '" + INITIATED_STATE + "'";
+          "cq_type, cq_tblproperties from COMPACTION_QUEUE where cq_state = '" + INITIATED_STATE + "'";
         LOG.debug("Going to execute query <" + s + ">");
         rs = stmt.executeQuery(s);
         if (!rs.next()) {
@@ -185,6 +185,7 @@ class CompactionTxnHandler extends TxnHandler {
           info.tableName = rs.getString(3);
           info.partName = rs.getString(4);
           info.type = dbCompactionType2ThriftType(rs.getString(5).charAt(0));
+          info.properties = rs.getString(6);
           // Now, update this record as being worked on by this worker.
           long now = getDbTime(dbConn);
           s = "update COMPACTION_QUEUE set cq_worker_id = '" + workerId + "', " +
@@ -329,7 +330,7 @@ class CompactionTxnHandler extends TxnHandler {
       try {
         dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
         stmt = dbConn.createStatement();
-        rs = stmt.executeQuery("select CQ_ID, CQ_DATABASE, CQ_TABLE, CQ_PARTITION, CQ_STATE, CQ_TYPE, CQ_WORKER_ID, CQ_START, CQ_RUN_AS, CQ_HIGHEST_TXN_ID, CQ_META_INFO, CQ_HADOOP_JOB_ID from COMPACTION_QUEUE WHERE CQ_ID = " + info.id);
+        rs = stmt.executeQuery("select CQ_ID, CQ_DATABASE, CQ_TABLE, CQ_PARTITION, CQ_STATE, CQ_TYPE, CQ_TBLPROPERTIES, CQ_WORKER_ID, CQ_START, CQ_RUN_AS, CQ_HIGHEST_TXN_ID, CQ_META_INFO, CQ_HADOOP_JOB_ID from COMPACTION_QUEUE WHERE CQ_ID = " + info.id);
         if(rs.next()) {
           info = CompactionInfo.loadFullFromCompactionQueue(rs);
         }
@@ -345,7 +346,7 @@ class CompactionTxnHandler extends TxnHandler {
           LOG.debug("Going to rollback");
           dbConn.rollback();
         }
-        pStmt = dbConn.prepareStatement("insert into COMPLETED_COMPACTIONS(CC_ID, CC_DATABASE, CC_TABLE, CC_PARTITION, CC_STATE, CC_TYPE, CC_WORKER_ID, CC_START, CC_END, CC_RUN_AS, CC_HIGHEST_TXN_ID, CC_META_INFO, CC_HADOOP_JOB_ID) VALUES(?,?,?,?,?, ?,?,?,?,?, ?,?,?)");
+        pStmt = dbConn.prepareStatement("insert into COMPLETED_COMPACTIONS(CC_ID, CC_DATABASE, CC_TABLE, CC_PARTITION, CC_STATE, CC_TYPE, CC_TBLPROPERTIES, CC_WORKER_ID, CC_START, CC_END, CC_RUN_AS, CC_HIGHEST_TXN_ID, CC_META_INFO, CC_HADOOP_JOB_ID) VALUES(?,?,?,?,?, ?,?,?,?,?, ?,?,?,?)");
         info.state = SUCCEEDED_STATE;
         CompactionInfo.insertIntoCompletedCompactions(pStmt, info, getDbTime(dbConn));
         updCount = pStmt.executeUpdate();
@@ -838,7 +839,7 @@ class CompactionTxnHandler extends TxnHandler {
       try {
         dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
         stmt = dbConn.createStatement();
-        rs = stmt.executeQuery("select CQ_ID, CQ_DATABASE, CQ_TABLE, CQ_PARTITION, CQ_STATE, CQ_TYPE, CQ_WORKER_ID, CQ_START, CQ_RUN_AS, CQ_HIGHEST_TXN_ID, CQ_META_INFO, CQ_HADOOP_JOB_ID from COMPACTION_QUEUE WHERE CQ_ID = " + ci.id);
+        rs = stmt.executeQuery("select CQ_ID, CQ_DATABASE, CQ_TABLE, CQ_PARTITION, CQ_STATE, CQ_TYPE, CQ_TBLPROPERTIES, CQ_WORKER_ID, CQ_START, CQ_RUN_AS, CQ_HIGHEST_TXN_ID, CQ_META_INFO, CQ_HADOOP_JOB_ID from COMPACTION_QUEUE WHERE CQ_ID = " + ci.id);
         if(rs.next()) {
           ci = CompactionInfo.loadFullFromCompactionQueue(rs);
           String s = "delete from COMPACTION_QUEUE where cq_id = " + ci.id;
@@ -866,7 +867,7 @@ class CompactionTxnHandler extends TxnHandler {
         }
         close(rs, stmt, null);
 
-        pStmt = dbConn.prepareStatement("insert into COMPLETED_COMPACTIONS(CC_ID, CC_DATABASE, CC_TABLE, CC_PARTITION, CC_STATE, CC_TYPE, CC_WORKER_ID, CC_START, CC_END, CC_RUN_AS, CC_HIGHEST_TXN_ID, CC_META_INFO, CC_HADOOP_JOB_ID) VALUES(?,?,?,?,?, ?,?,?,?,?, ?,?,?)");
+        pStmt = dbConn.prepareStatement("insert into COMPLETED_COMPACTIONS(CC_ID, CC_DATABASE, CC_TABLE, CC_PARTITION, CC_STATE, CC_TYPE, CC_TBLPROPERTIES, CC_WORKER_ID, CC_START, CC_END, CC_RUN_AS, CC_HIGHEST_TXN_ID, CC_META_INFO, CC_HADOOP_JOB_ID) VALUES(?,?,?,?,?, ?,?,?,?,?, ?,?,?,?)");
         CompactionInfo.insertIntoCompletedCompactions(pStmt, ci, getDbTime(dbConn));
         int updCount = pStmt.executeUpdate();
         LOG.debug("Going to commit");

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
index 5805966..b503652 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
@@ -123,6 +123,7 @@ public final class TxnDbUtil {
           " CQ_PARTITION varchar(767)," +
           " CQ_STATE char(1) NOT NULL," +
           " CQ_TYPE char(1) NOT NULL," +
+          " CQ_TBLPROPERTIES varchar(2048)," +
           " CQ_WORKER_ID varchar(128)," +
           " CQ_START bigint," +
           " CQ_RUN_AS varchar(128)," +
@@ -140,6 +141,7 @@ public final class TxnDbUtil {
         " CC_PARTITION varchar(767)," +
         " CC_STATE char(1) NOT NULL," +
         " CC_TYPE char(1) NOT NULL," +
+        " CC_TBLPROPERTIES varchar(2048)," +
         " CC_WORKER_ID varchar(128)," +
         " CC_START bigint," +
         " CC_END bigint," +

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
index 27fa820..f2658f2 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.HouseKeeperService;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.*;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils.StringableMap;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.util.StringUtils;
 
@@ -1385,6 +1386,9 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         String partName = rqst.getPartitionname();
         if (partName != null) buf.append("cq_partition, ");
         buf.append("cq_state, cq_type");
+        if (rqst.getProperties() != null) {
+          buf.append(", cq_tblproperties");
+        }
         if (rqst.getRunas() != null) buf.append(", cq_run_as");
         buf.append(") values (");
         buf.append(id);
@@ -1413,6 +1417,10 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
             dbConn.rollback();
             throw new MetaException("Unexpected compaction type " + rqst.getType().toString());
         }
+        if (rqst.getProperties() != null) {
+          buf.append("', '");
+          buf.append(new StringableMap(rqst.getProperties()).toString());
+        }
         if (rqst.getRunas() != null) {
           buf.append("', '");
           buf.append(rqst.getRunas());

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java
index 5391fb0..7212bfd 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java
@@ -30,8 +30,10 @@ import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Properties;
 import java.util.Set;
 
 public class TxnUtils {
@@ -208,4 +210,56 @@ public class TxnUtils {
     long sizeInBytes = 8 * (((sb.length() * 2) + 45) / 8);
     return sizeInBytes / 1024 > queryMemoryLimit;
   }
+
+  public static class StringableMap extends HashMap<String, String> {
+
+    public StringableMap(String s) {
+      String[] parts = s.split(":", 2);
+      // read that many chars
+      int numElements = Integer.parseInt(parts[0]);
+      s = parts[1];
+      for (int i = 0; i < numElements; i++) {
+        parts = s.split(":", 2);
+        int len = Integer.parseInt(parts[0]);
+        String key = null;
+        if (len > 0) key = parts[1].substring(0, len);
+        parts = parts[1].substring(len).split(":", 2);
+        len = Integer.parseInt(parts[0]);
+        String value = null;
+        if (len > 0) value = parts[1].substring(0, len);
+        s = parts[1].substring(len);
+        put(key, value);
+      }
+    }
+
+    public StringableMap(Map<String, String> m) {
+      super(m);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder buf = new StringBuilder();
+      buf.append(size());
+      buf.append(':');
+      if (size() > 0) {
+        for (Map.Entry<String, String> entry : entrySet()) {
+          int length = (entry.getKey() == null) ? 0 : entry.getKey().length();
+          buf.append(entry.getKey() == null ? 0 : length);
+          buf.append(':');
+          if (length > 0) buf.append(entry.getKey());
+          length = (entry.getValue() == null) ? 0 : entry.getValue().length();
+          buf.append(length);
+          buf.append(':');
+          if (length > 0) buf.append(entry.getValue());
+        }
+      }
+      return buf.toString();
+    }
+
+    public Properties toProperties() {
+      Properties props = new Properties();
+      props.putAll(this);
+      return props;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index be4753f..c7d9de7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -1738,7 +1738,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       }
       partName = partitions.get(0).getName();
     }
-    db.compact(tbl.getDbName(), tbl.getTableName(), partName, desc.getCompactionType());
+    db.compact(tbl.getDbName(), tbl.getTableName(), partName, desc.getCompactionType(), desc.getProps());
     console.printInfo("Compaction enqueued.");
     return 0;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
index a67f23a..6362d23 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
@@ -3240,16 +3240,18 @@ private void constructOneLBLocationMap(FileStatus fSta,
    * @param partName name of the partition, if null table will be compacted (valid only for
    *                 non-partitioned tables).
    * @param compactType major or minor
+   * @param tblproperties the list of tblproperties to overwrite for this compaction
    * @throws HiveException
    */
-  public void compact(String dbname, String tableName, String partName,  String compactType)
+  public void compact(String dbname, String tableName, String partName, String compactType,
+                      Map<String, String> tblproperties)
       throws HiveException {
     try {
       CompactionType cr = null;
       if ("major".equals(compactType)) cr = CompactionType.MAJOR;
       else if ("minor".equals(compactType)) cr = CompactionType.MINOR;
       else throw new RuntimeException("Unknown compaction type " + compactType);
-      getMSC().compact(dbname, tableName, partName, cr);
+      getMSC().compact(dbname, tableName, partName, cr, tblproperties);
     } catch (Exception e) {
       LOG.error(StringUtils.stringifyException(e));
       throw new HiveException(e);

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
index f7cd167..c836268 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
@@ -1727,6 +1727,11 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     AlterTableSimpleDesc desc = new AlterTableSimpleDesc(
         tableName, newPartSpec, type);
 
+    if (ast.getChildCount() > 1) {
+      HashMap<String, String> mapProp = getProps((ASTNode) (ast.getChild(1)).getChild(0));
+      desc.setProps(mapProp);
+    }
+
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc), conf));
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
index 6c3d42a..eda460f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
@@ -1344,8 +1344,8 @@ alterStatementSuffixBucketNum
 alterStatementSuffixCompact
 @init { msgs.push("compaction request"); }
 @after { msgs.pop(); }
-    : KW_COMPACT compactType=StringLiteral
-    -> ^(TOK_ALTERTABLE_COMPACT $compactType)
+    : KW_COMPACT compactType=StringLiteral (KW_WITH KW_OVERWRITE KW_TBLPROPERTIES tableProperties)?
+    -> ^(TOK_ALTERTABLE_COMPACT $compactType tableProperties?)
     ;
 
 

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableSimpleDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableSimpleDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableSimpleDesc.java
index d819d15..2ae70bb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableSimpleDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableSimpleDesc.java
@@ -33,6 +33,7 @@ public class AlterTableSimpleDesc extends DDLDesc {
   private String compactionType;
 
   AlterTableTypes type;
+  private Map<String, String> props;
 
   public AlterTableSimpleDesc() {
   }
@@ -99,4 +100,11 @@ public class AlterTableSimpleDesc extends DDLDesc {
     return compactionType;
   }
 
+  public Map<String, String> getProps() {
+    return props;
+  }
+
+  public void setProps(Map<String, String> props) {
+    this.props = props;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java
index 9f68fa6..03cd992 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorMR.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils.StringableMap;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter;
 import org.apache.hadoop.hive.ql.io.AcidInputFormat;
 import org.apache.hadoop.hive.ql.io.AcidOutputFormat;
@@ -40,7 +41,6 @@ import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.io.HiveInputFormat;
 import org.apache.hadoop.hive.ql.io.IOConstants;
 import org.apache.hadoop.hive.ql.io.RecordIdentifier;
-import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.io.NullWritable;
@@ -92,12 +92,16 @@ public class CompactorMR {
   static final private String DELTA_DIRS = "hive.compactor.delta.dirs";
   static final private String DIRS_TO_SEARCH = "hive.compactor.dirs.to.search";
   static final private String TMPDIR = "_tmp";
+  static final private String TBLPROPS_PREFIX = "tblprops.";
+  static final private String COMPACTOR_PREFIX = "compactor.";
+
+  private JobConf mrJob;  // the MR job for compaction
 
   public CompactorMR() {
   }
 
   private JobConf createBaseJobConf(HiveConf conf, String jobName, Table t, StorageDescriptor sd,
-                                    ValidTxnList txns) {
+                                    ValidTxnList txns, CompactionInfo ci) {
     JobConf job = new JobConf(conf);
     job.setJobName(jobName);
     job.setOutputKeyClass(NullWritable.class);
@@ -123,9 +127,52 @@ public class CompactorMR {
     job.set(TABLE_PROPS, new StringableMap(t.getParameters()).toString());
     job.setInt(NUM_BUCKETS, sd.getNumBuckets());
     job.set(ValidTxnList.VALID_TXNS_KEY, txns.toString());
+    overrideMRProps(job, t.getParameters()); // override MR properties from tblproperties if applicable
+    if (ci.properties != null) { // override MR properties and general tblproperties if applicable
+      overrideTblProps(job, t.getParameters(), ci.properties);
+    }
     setColumnTypes(job, sd.getCols());
     return job;
   }
+
+  /**
+   * Parse tblproperties specified on "ALTER TABLE ... COMPACT ... WITH OVERWRITE TBLPROPERTIES ..."
+   * and override two categories of properties:
+   * 1. properties of the compactor MR job (with prefix "compactor.")
+   * 2. general hive properties (with prefix "tblprops.")
+   * @param job the compactor MR job
+   * @param tblproperties existing tblproperties
+   * @param properties table properties
+   */
+  private void overrideTblProps(JobConf job, Map<String, String> tblproperties, String properties) {
+    StringableMap stringableMap = new StringableMap(properties);
+    overrideMRProps(job, stringableMap);
+    // mingle existing tblproperties with those specified on the ALTER TABLE command
+    for (String key : stringableMap.keySet()) {
+      if (key.startsWith(TBLPROPS_PREFIX)) {
+        String propKey = key.substring(9);  // 9 is the length of "tblprops.". We only keep the rest
+        tblproperties.put(propKey, stringableMap.get(key));
+      }
+    }
+    // re-set TABLE_PROPS with reloaded tblproperties
+    job.set(TABLE_PROPS, new StringableMap(tblproperties).toString());
+  }
+
+  /**
+   * Parse tblproperties to override relevant properties of compactor MR job with specified values.
+   * For example, compactor.mapreuce.map.memory.mb=1024
+   * @param job the compactor MR job
+   * @param properties table properties
+   */
+  private void overrideMRProps(JobConf job, Map<String, String> properties) {
+    for (String key : properties.keySet()) {
+      if (key.startsWith(COMPACTOR_PREFIX)) {
+        String mrKey = key.substring(10); // 10 is the length of "compactor." We only keep the rest.
+        job.set(mrKey, properties.get(key));
+      }
+    }
+  }
+
   /**
    * Run Compaction which may consist of several jobs on the cluster.
    * @param conf Hive configuration file
@@ -142,7 +189,7 @@ public class CompactorMR {
     if(conf.getBoolVar(HiveConf.ConfVars.HIVE_IN_TEST) && conf.getBoolVar(HiveConf.ConfVars.HIVETESTMODEFAILCOMPACTION)) {
       throw new RuntimeException(HiveConf.ConfVars.HIVETESTMODEFAILCOMPACTION.name() + "=true");
     }
-    JobConf job = createBaseJobConf(conf, jobName, t, sd, txns);
+    JobConf job = createBaseJobConf(conf, jobName, t, sd, txns, ci);
 
     // Figure out and encode what files we need to read.  We do this here (rather than in
     // getSplits below) because as part of this we discover our minimum and maximum transactions,
@@ -167,11 +214,11 @@ public class CompactorMR {
         "runaway/mis-configured process writing to ACID tables, especially using Streaming Ingest API.");
       int numMinorCompactions = parsedDeltas.size() / maxDeltastoHandle;
       for(int jobSubId = 0; jobSubId < numMinorCompactions; jobSubId++) {
-        JobConf jobMinorCompact = createBaseJobConf(conf, jobName + "_" + jobSubId, t, sd, txns);
+        JobConf jobMinorCompact = createBaseJobConf(conf, jobName + "_" + jobSubId, t, sd, txns, ci);
         launchCompactionJob(jobMinorCompact,
           null, CompactionType.MINOR, null,
           parsedDeltas.subList(jobSubId * maxDeltastoHandle, (jobSubId + 1) * maxDeltastoHandle),
-          maxDeltastoHandle, -1);
+          maxDeltastoHandle, -1, conf);
       }
       //now recompute state since we've done minor compactions and have different 'best' set of deltas
       dir = AcidUtils.getAcidState(new Path(sd.getLocation()), conf, txns);
@@ -209,14 +256,14 @@ public class CompactorMR {
     }
 
     launchCompactionJob(job, baseDir, ci.type, dirsToSearch, dir.getCurrentDirectories(),
-      dir.getCurrentDirectories().size(), dir.getObsolete().size());
+      dir.getCurrentDirectories().size(), dir.getObsolete().size(), conf);
 
     su.gatherStats();
   }
   private void launchCompactionJob(JobConf job, Path baseDir, CompactionType compactionType,
                                    StringableList dirsToSearch,
                                    List<AcidUtils.ParsedDelta> parsedDeltas,
-                                   int curDirNumber, int obsoleteDirNumber) throws IOException {
+                                   int curDirNumber, int obsoleteDirNumber, HiveConf hiveConf) throws IOException {
     job.setBoolean(IS_MAJOR, compactionType == CompactionType.MAJOR);
     if(dirsToSearch == null) {
       dirsToSearch = new StringableList();
@@ -238,6 +285,10 @@ public class CompactorMR {
     job.setLong(MIN_TXN, minTxn);
     job.setLong(MAX_TXN, maxTxn);
 
+    if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_IN_TEST)) {
+      mrJob = job;
+    }
+
     LOG.info("Submitting " + compactionType + " compaction job '" +
       job.getJobName() + "' to " + job.getQueueName() + " queue.  " +
       "(current delta dirs count=" + curDirNumber +
@@ -272,6 +323,10 @@ public class CompactorMR {
     HiveConf.setVar(job, HiveConf.ConfVars.HIVEINPUTFORMAT, HiveInputFormat.class.getName());
   }
 
+  public JobConf getMrJob() {
+    return mrJob;
+  }
+
   static class CompactorInputSplit implements InputSplit {
     private long length = 0;
     private List<String> locations;
@@ -621,58 +676,6 @@ public class CompactorMR {
 
   }
 
-  static class StringableMap extends HashMap<String, String> {
-
-    StringableMap(String s) {
-      String[] parts = s.split(":", 2);
-      // read that many chars
-      int numElements = Integer.valueOf(parts[0]);
-      s = parts[1];
-      for (int i = 0; i < numElements; i++) {
-        parts = s.split(":", 2);
-        int len = Integer.valueOf(parts[0]);
-        String key = null;
-        if (len > 0) key = parts[1].substring(0, len);
-        parts = parts[1].substring(len).split(":", 2);
-        len = Integer.valueOf(parts[0]);
-        String value = null;
-        if (len > 0) value = parts[1].substring(0, len);
-        s = parts[1].substring(len);
-        put(key, value);
-      }
-    }
-
-    StringableMap(Map<String, String> m) {
-      super(m);
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder buf = new StringBuilder();
-      buf.append(size());
-      buf.append(':');
-      if (size() > 0) {
-        for (Map.Entry<String, String> entry : entrySet()) {
-          int length = (entry.getKey() == null) ? 0 : entry.getKey().length();
-          buf.append(entry.getKey() == null ? 0 : length);
-          buf.append(':');
-          if (length > 0) buf.append(entry.getKey());
-          length = (entry.getValue() == null) ? 0 : entry.getValue().length();
-          buf.append(length);
-          buf.append(':');
-          if (length > 0) buf.append(entry.getValue());
-        }
-      }
-      return buf.toString();
-    }
-
-    public Properties toProperties() {
-      Properties props = new Properties();
-      props.putAll(this);
-      return props;
-    }
-  }
-
   static class StringableList extends ArrayList<Path> {
     StringableList() {
 

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
index 625e389..1a63f99 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.util.StringUtils;
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -58,6 +59,8 @@ public class Initiator extends CompactorThread {
   static final private String CLASS_NAME = Initiator.class.getName();
   static final private Log LOG = LogFactory.getLog(CLASS_NAME);
 
+  static final private String COMPACTORTHRESHOLD_PREFIX = "compactorthreshold.";
+
   private long checkInterval;
 
   @Override
@@ -144,7 +147,7 @@ public class Initiator extends CompactorThread {
               /*Future thought: checkForCompaction will check a lot of file metadata and may be expensive.
               * Long term we should consider having a thread pool here and running checkForCompactionS
               * in parallel*/
-              CompactionType compactionNeeded = checkForCompaction(ci, txns, sd, runAs);
+              CompactionType compactionNeeded = checkForCompaction(ci, txns, sd, t.getParameters(), runAs);
               if (compactionNeeded != null) requestCompaction(ci, runAs, compactionNeeded);
             } catch (Throwable t) {
               LOG.error("Caught exception while trying to determine if we should compact " +
@@ -213,6 +216,7 @@ public class Initiator extends CompactorThread {
   private CompactionType checkForCompaction(final CompactionInfo ci,
                                             final ValidTxnList txns,
                                             final StorageDescriptor sd,
+                                            final Map<String, String> tblproperties,
                                             final String runAs)
       throws IOException, InterruptedException {
     // If it's marked as too many aborted, we already know we need to compact
@@ -222,7 +226,7 @@ public class Initiator extends CompactorThread {
       return CompactionType.MAJOR;
     }
     if (runJobAsSelf(runAs)) {
-      return determineCompactionType(ci, txns, sd);
+      return determineCompactionType(ci, txns, sd, tblproperties);
     } else {
       LOG.info("Going to initiate as user " + runAs);
       UserGroupInformation ugi = UserGroupInformation.createProxyUser(runAs,
@@ -230,7 +234,7 @@ public class Initiator extends CompactorThread {
       CompactionType compactionType = ugi.doAs(new PrivilegedExceptionAction<CompactionType>() {
         @Override
         public CompactionType run() throws Exception {
-          return determineCompactionType(ci, txns, sd);
+          return determineCompactionType(ci, txns, sd, tblproperties);
         }
       });
       try {
@@ -244,7 +248,7 @@ public class Initiator extends CompactorThread {
   }
 
   private CompactionType determineCompactionType(CompactionInfo ci, ValidTxnList txns,
-                                                 StorageDescriptor sd)
+                                                 StorageDescriptor sd, Map<String, String> tblproperties)
       throws IOException, InterruptedException {
     boolean noBase = false;
     Path location = new Path(sd.getLocation());
@@ -282,8 +286,11 @@ public class Initiator extends CompactorThread {
     if (baseSize == 0 && deltaSize > 0) {
       noBase = true;
     } else {
-      float deltaPctThreshold = HiveConf.getFloatVar(conf,
+      String deltaPctProp = tblproperties.get(COMPACTORTHRESHOLD_PREFIX +
           HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_PCT_THRESHOLD);
+      float deltaPctThreshold = deltaPctProp == null ?
+          HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_PCT_THRESHOLD) :
+          Float.parseFloat(deltaPctProp);
       boolean bigEnough =   (float)deltaSize/(float)baseSize > deltaPctThreshold;
       if (LOG.isDebugEnabled()) {
         StringBuilder msg = new StringBuilder("delta size: ");
@@ -299,8 +306,11 @@ public class Initiator extends CompactorThread {
       if (bigEnough) return CompactionType.MAJOR;
     }
 
-    int deltaNumThreshold = HiveConf.getIntVar(conf,
+    String deltaNumProp = tblproperties.get(COMPACTORTHRESHOLD_PREFIX +
         HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_NUM_THRESHOLD);
+    int deltaNumThreshold = deltaNumProp == null ?
+        HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_NUM_THRESHOLD) :
+        Integer.parseInt(deltaNumProp);
     boolean enough = deltas.size() > deltaNumThreshold;
     if (enough) {
       LOG.debug("Found " + deltas.size() + " delta files, threshold is " + deltaNumThreshold +

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
index bf8e5cc..256e27b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
@@ -58,6 +59,7 @@ public class Worker extends CompactorThread {
   static final private int baseThreadNum = 10002;
 
   private String name;
+  private JobConf mrJob; // the MR job for compaction
 
   /**
    * Get the hostname that this worker is run on.  Made static and public so that other classes
@@ -182,6 +184,9 @@ public class Worker extends CompactorThread {
             }
           }
           txnHandler.markCompacted(ci);
+          if (conf.getBoolVar(HiveConf.ConfVars.HIVE_IN_TEST)) {
+            mrJob = mr.getMrJob();
+          }
         } catch (Exception e) {
           LOG.error("Caught exception while trying to compact " + ci +
               ".  Marking failed to avoid repeated failures, " + StringUtils.stringifyException(e));
@@ -215,6 +220,10 @@ public class Worker extends CompactorThread {
     setName(name.toString());
   }
 
+  public JobConf getMrJob() {
+    return mrJob;
+  }
+
   static final class StatsUpdater {
     static final private Log LOG = LogFactory.getLog(StatsUpdater.class);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java
index 381eeb3..b91bdc3 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.fs.*;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.*;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils.StringableMap;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.junit.Assert;
 import org.junit.Before;
@@ -68,19 +69,19 @@ public class TestWorker extends CompactorTest {
   @Test
   public void stringableMap() throws Exception {
     // Empty map case
-    CompactorMR.StringableMap m = new CompactorMR.StringableMap(new HashMap<String, String>());
+    StringableMap m = new StringableMap(new HashMap<String, String>());
     String s = m.toString();
     Assert.assertEquals("0:", s);
-    m = new CompactorMR.StringableMap(s);
+    m = new StringableMap(s);
     Assert.assertEquals(0, m.size());
 
     Map<String, String> base = new HashMap<String, String>();
     base.put("mary", "poppins");
     base.put("bert", null);
     base.put(null, "banks");
-    m = new CompactorMR.StringableMap(base);
+    m = new StringableMap(base);
     s = m.toString();
-    m = new CompactorMR.StringableMap(s);
+    m = new StringableMap(s);
     Assert.assertEquals(3, m.size());
     Map<String, Boolean> saw = new HashMap<String, Boolean>(3);
     saw.put("mary", false);


[3/7] hive git commit: HIVE-13354 : Add ability to specify Compaction options per table and per request (Wei Zheng, reviewed by Eugene Koifman)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
index 438e368..c85150d 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -9428,14 +9428,14 @@ class ThriftHiveMetastore_get_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size479 = 0;
-            $_etype482 = 0;
-            $xfer += $input->readListBegin($_etype482, $_size479);
-            for ($_i483 = 0; $_i483 < $_size479; ++$_i483)
+            $_size488 = 0;
+            $_etype491 = 0;
+            $xfer += $input->readListBegin($_etype491, $_size488);
+            for ($_i492 = 0; $_i492 < $_size488; ++$_i492)
             {
-              $elem484 = null;
-              $xfer += $input->readString($elem484);
-              $this->success []= $elem484;
+              $elem493 = null;
+              $xfer += $input->readString($elem493);
+              $this->success []= $elem493;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -9471,9 +9471,9 @@ class ThriftHiveMetastore_get_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter485)
+          foreach ($this->success as $iter494)
           {
-            $xfer += $output->writeString($iter485);
+            $xfer += $output->writeString($iter494);
           }
         }
         $output->writeListEnd();
@@ -9604,14 +9604,14 @@ class ThriftHiveMetastore_get_all_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size486 = 0;
-            $_etype489 = 0;
-            $xfer += $input->readListBegin($_etype489, $_size486);
-            for ($_i490 = 0; $_i490 < $_size486; ++$_i490)
+            $_size495 = 0;
+            $_etype498 = 0;
+            $xfer += $input->readListBegin($_etype498, $_size495);
+            for ($_i499 = 0; $_i499 < $_size495; ++$_i499)
             {
-              $elem491 = null;
-              $xfer += $input->readString($elem491);
-              $this->success []= $elem491;
+              $elem500 = null;
+              $xfer += $input->readString($elem500);
+              $this->success []= $elem500;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -9647,9 +9647,9 @@ class ThriftHiveMetastore_get_all_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter492)
+          foreach ($this->success as $iter501)
           {
-            $xfer += $output->writeString($iter492);
+            $xfer += $output->writeString($iter501);
           }
         }
         $output->writeListEnd();
@@ -10650,18 +10650,18 @@ class ThriftHiveMetastore_get_type_all_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size493 = 0;
-            $_ktype494 = 0;
-            $_vtype495 = 0;
-            $xfer += $input->readMapBegin($_ktype494, $_vtype495, $_size493);
-            for ($_i497 = 0; $_i497 < $_size493; ++$_i497)
+            $_size502 = 0;
+            $_ktype503 = 0;
+            $_vtype504 = 0;
+            $xfer += $input->readMapBegin($_ktype503, $_vtype504, $_size502);
+            for ($_i506 = 0; $_i506 < $_size502; ++$_i506)
             {
-              $key498 = '';
-              $val499 = new \metastore\Type();
-              $xfer += $input->readString($key498);
-              $val499 = new \metastore\Type();
-              $xfer += $val499->read($input);
-              $this->success[$key498] = $val499;
+              $key507 = '';
+              $val508 = new \metastore\Type();
+              $xfer += $input->readString($key507);
+              $val508 = new \metastore\Type();
+              $xfer += $val508->read($input);
+              $this->success[$key507] = $val508;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -10697,10 +10697,10 @@ class ThriftHiveMetastore_get_type_all_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $kiter500 => $viter501)
+          foreach ($this->success as $kiter509 => $viter510)
           {
-            $xfer += $output->writeString($kiter500);
-            $xfer += $viter501->write($output);
+            $xfer += $output->writeString($kiter509);
+            $xfer += $viter510->write($output);
           }
         }
         $output->writeMapEnd();
@@ -10904,15 +10904,15 @@ class ThriftHiveMetastore_get_fields_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size502 = 0;
-            $_etype505 = 0;
-            $xfer += $input->readListBegin($_etype505, $_size502);
-            for ($_i506 = 0; $_i506 < $_size502; ++$_i506)
+            $_size511 = 0;
+            $_etype514 = 0;
+            $xfer += $input->readListBegin($_etype514, $_size511);
+            for ($_i515 = 0; $_i515 < $_size511; ++$_i515)
             {
-              $elem507 = null;
-              $elem507 = new \metastore\FieldSchema();
-              $xfer += $elem507->read($input);
-              $this->success []= $elem507;
+              $elem516 = null;
+              $elem516 = new \metastore\FieldSchema();
+              $xfer += $elem516->read($input);
+              $this->success []= $elem516;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -10964,9 +10964,9 @@ class ThriftHiveMetastore_get_fields_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter508)
+          foreach ($this->success as $iter517)
           {
-            $xfer += $iter508->write($output);
+            $xfer += $iter517->write($output);
           }
         }
         $output->writeListEnd();
@@ -11208,15 +11208,15 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size509 = 0;
-            $_etype512 = 0;
-            $xfer += $input->readListBegin($_etype512, $_size509);
-            for ($_i513 = 0; $_i513 < $_size509; ++$_i513)
+            $_size518 = 0;
+            $_etype521 = 0;
+            $xfer += $input->readListBegin($_etype521, $_size518);
+            for ($_i522 = 0; $_i522 < $_size518; ++$_i522)
             {
-              $elem514 = null;
-              $elem514 = new \metastore\FieldSchema();
-              $xfer += $elem514->read($input);
-              $this->success []= $elem514;
+              $elem523 = null;
+              $elem523 = new \metastore\FieldSchema();
+              $xfer += $elem523->read($input);
+              $this->success []= $elem523;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11268,9 +11268,9 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter515)
+          foreach ($this->success as $iter524)
           {
-            $xfer += $iter515->write($output);
+            $xfer += $iter524->write($output);
           }
         }
         $output->writeListEnd();
@@ -11484,15 +11484,15 @@ class ThriftHiveMetastore_get_schema_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size516 = 0;
-            $_etype519 = 0;
-            $xfer += $input->readListBegin($_etype519, $_size516);
-            for ($_i520 = 0; $_i520 < $_size516; ++$_i520)
+            $_size525 = 0;
+            $_etype528 = 0;
+            $xfer += $input->readListBegin($_etype528, $_size525);
+            for ($_i529 = 0; $_i529 < $_size525; ++$_i529)
             {
-              $elem521 = null;
-              $elem521 = new \metastore\FieldSchema();
-              $xfer += $elem521->read($input);
-              $this->success []= $elem521;
+              $elem530 = null;
+              $elem530 = new \metastore\FieldSchema();
+              $xfer += $elem530->read($input);
+              $this->success []= $elem530;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11544,9 +11544,9 @@ class ThriftHiveMetastore_get_schema_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter522)
+          foreach ($this->success as $iter531)
           {
-            $xfer += $iter522->write($output);
+            $xfer += $iter531->write($output);
           }
         }
         $output->writeListEnd();
@@ -11788,15 +11788,15 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size523 = 0;
-            $_etype526 = 0;
-            $xfer += $input->readListBegin($_etype526, $_size523);
-            for ($_i527 = 0; $_i527 < $_size523; ++$_i527)
+            $_size532 = 0;
+            $_etype535 = 0;
+            $xfer += $input->readListBegin($_etype535, $_size532);
+            for ($_i536 = 0; $_i536 < $_size532; ++$_i536)
             {
-              $elem528 = null;
-              $elem528 = new \metastore\FieldSchema();
-              $xfer += $elem528->read($input);
-              $this->success []= $elem528;
+              $elem537 = null;
+              $elem537 = new \metastore\FieldSchema();
+              $xfer += $elem537->read($input);
+              $this->success []= $elem537;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11848,9 +11848,9 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter529)
+          foreach ($this->success as $iter538)
           {
-            $xfer += $iter529->write($output);
+            $xfer += $iter538->write($output);
           }
         }
         $output->writeListEnd();
@@ -13005,14 +13005,14 @@ class ThriftHiveMetastore_get_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size530 = 0;
-            $_etype533 = 0;
-            $xfer += $input->readListBegin($_etype533, $_size530);
-            for ($_i534 = 0; $_i534 < $_size530; ++$_i534)
+            $_size539 = 0;
+            $_etype542 = 0;
+            $xfer += $input->readListBegin($_etype542, $_size539);
+            for ($_i543 = 0; $_i543 < $_size539; ++$_i543)
             {
-              $elem535 = null;
-              $xfer += $input->readString($elem535);
-              $this->success []= $elem535;
+              $elem544 = null;
+              $xfer += $input->readString($elem544);
+              $this->success []= $elem544;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13048,9 +13048,9 @@ class ThriftHiveMetastore_get_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter536)
+          foreach ($this->success as $iter545)
           {
-            $xfer += $output->writeString($iter536);
+            $xfer += $output->writeString($iter545);
           }
         }
         $output->writeListEnd();
@@ -13206,14 +13206,14 @@ class ThriftHiveMetastore_get_all_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size537 = 0;
-            $_etype540 = 0;
-            $xfer += $input->readListBegin($_etype540, $_size537);
-            for ($_i541 = 0; $_i541 < $_size537; ++$_i541)
+            $_size546 = 0;
+            $_etype549 = 0;
+            $xfer += $input->readListBegin($_etype549, $_size546);
+            for ($_i550 = 0; $_i550 < $_size546; ++$_i550)
             {
-              $elem542 = null;
-              $xfer += $input->readString($elem542);
-              $this->success []= $elem542;
+              $elem551 = null;
+              $xfer += $input->readString($elem551);
+              $this->success []= $elem551;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13249,9 +13249,9 @@ class ThriftHiveMetastore_get_all_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter543)
+          foreach ($this->success as $iter552)
           {
-            $xfer += $output->writeString($iter543);
+            $xfer += $output->writeString($iter552);
           }
         }
         $output->writeListEnd();
@@ -13566,14 +13566,14 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size544 = 0;
-            $_etype547 = 0;
-            $xfer += $input->readListBegin($_etype547, $_size544);
-            for ($_i548 = 0; $_i548 < $_size544; ++$_i548)
+            $_size553 = 0;
+            $_etype556 = 0;
+            $xfer += $input->readListBegin($_etype556, $_size553);
+            for ($_i557 = 0; $_i557 < $_size553; ++$_i557)
             {
-              $elem549 = null;
-              $xfer += $input->readString($elem549);
-              $this->tbl_names []= $elem549;
+              $elem558 = null;
+              $xfer += $input->readString($elem558);
+              $this->tbl_names []= $elem558;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13606,9 +13606,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter550)
+          foreach ($this->tbl_names as $iter559)
           {
-            $xfer += $output->writeString($iter550);
+            $xfer += $output->writeString($iter559);
           }
         }
         $output->writeListEnd();
@@ -13709,15 +13709,15 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size551 = 0;
-            $_etype554 = 0;
-            $xfer += $input->readListBegin($_etype554, $_size551);
-            for ($_i555 = 0; $_i555 < $_size551; ++$_i555)
+            $_size560 = 0;
+            $_etype563 = 0;
+            $xfer += $input->readListBegin($_etype563, $_size560);
+            for ($_i564 = 0; $_i564 < $_size560; ++$_i564)
             {
-              $elem556 = null;
-              $elem556 = new \metastore\Table();
-              $xfer += $elem556->read($input);
-              $this->success []= $elem556;
+              $elem565 = null;
+              $elem565 = new \metastore\Table();
+              $xfer += $elem565->read($input);
+              $this->success []= $elem565;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13769,9 +13769,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter557)
+          foreach ($this->success as $iter566)
           {
-            $xfer += $iter557->write($output);
+            $xfer += $iter566->write($output);
           }
         }
         $output->writeListEnd();
@@ -14007,14 +14007,14 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size558 = 0;
-            $_etype561 = 0;
-            $xfer += $input->readListBegin($_etype561, $_size558);
-            for ($_i562 = 0; $_i562 < $_size558; ++$_i562)
+            $_size567 = 0;
+            $_etype570 = 0;
+            $xfer += $input->readListBegin($_etype570, $_size567);
+            for ($_i571 = 0; $_i571 < $_size567; ++$_i571)
             {
-              $elem563 = null;
-              $xfer += $input->readString($elem563);
-              $this->success []= $elem563;
+              $elem572 = null;
+              $xfer += $input->readString($elem572);
+              $this->success []= $elem572;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14066,9 +14066,9 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter564)
+          foreach ($this->success as $iter573)
           {
-            $xfer += $output->writeString($iter564);
+            $xfer += $output->writeString($iter573);
           }
         }
         $output->writeListEnd();
@@ -15381,15 +15381,15 @@ class ThriftHiveMetastore_add_partitions_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size565 = 0;
-            $_etype568 = 0;
-            $xfer += $input->readListBegin($_etype568, $_size565);
-            for ($_i569 = 0; $_i569 < $_size565; ++$_i569)
+            $_size574 = 0;
+            $_etype577 = 0;
+            $xfer += $input->readListBegin($_etype577, $_size574);
+            for ($_i578 = 0; $_i578 < $_size574; ++$_i578)
             {
-              $elem570 = null;
-              $elem570 = new \metastore\Partition();
-              $xfer += $elem570->read($input);
-              $this->new_parts []= $elem570;
+              $elem579 = null;
+              $elem579 = new \metastore\Partition();
+              $xfer += $elem579->read($input);
+              $this->new_parts []= $elem579;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15417,9 +15417,9 @@ class ThriftHiveMetastore_add_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter571)
+          foreach ($this->new_parts as $iter580)
           {
-            $xfer += $iter571->write($output);
+            $xfer += $iter580->write($output);
           }
         }
         $output->writeListEnd();
@@ -15634,15 +15634,15 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size572 = 0;
-            $_etype575 = 0;
-            $xfer += $input->readListBegin($_etype575, $_size572);
-            for ($_i576 = 0; $_i576 < $_size572; ++$_i576)
+            $_size581 = 0;
+            $_etype584 = 0;
+            $xfer += $input->readListBegin($_etype584, $_size581);
+            for ($_i585 = 0; $_i585 < $_size581; ++$_i585)
             {
-              $elem577 = null;
-              $elem577 = new \metastore\PartitionSpec();
-              $xfer += $elem577->read($input);
-              $this->new_parts []= $elem577;
+              $elem586 = null;
+              $elem586 = new \metastore\PartitionSpec();
+              $xfer += $elem586->read($input);
+              $this->new_parts []= $elem586;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15670,9 +15670,9 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter578)
+          foreach ($this->new_parts as $iter587)
           {
-            $xfer += $iter578->write($output);
+            $xfer += $iter587->write($output);
           }
         }
         $output->writeListEnd();
@@ -15922,14 +15922,14 @@ class ThriftHiveMetastore_append_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size579 = 0;
-            $_etype582 = 0;
-            $xfer += $input->readListBegin($_etype582, $_size579);
-            for ($_i583 = 0; $_i583 < $_size579; ++$_i583)
+            $_size588 = 0;
+            $_etype591 = 0;
+            $xfer += $input->readListBegin($_etype591, $_size588);
+            for ($_i592 = 0; $_i592 < $_size588; ++$_i592)
             {
-              $elem584 = null;
-              $xfer += $input->readString($elem584);
-              $this->part_vals []= $elem584;
+              $elem593 = null;
+              $xfer += $input->readString($elem593);
+              $this->part_vals []= $elem593;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15967,9 +15967,9 @@ class ThriftHiveMetastore_append_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter585)
+          foreach ($this->part_vals as $iter594)
           {
-            $xfer += $output->writeString($iter585);
+            $xfer += $output->writeString($iter594);
           }
         }
         $output->writeListEnd();
@@ -16471,14 +16471,14 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size586 = 0;
-            $_etype589 = 0;
-            $xfer += $input->readListBegin($_etype589, $_size586);
-            for ($_i590 = 0; $_i590 < $_size586; ++$_i590)
+            $_size595 = 0;
+            $_etype598 = 0;
+            $xfer += $input->readListBegin($_etype598, $_size595);
+            for ($_i599 = 0; $_i599 < $_size595; ++$_i599)
             {
-              $elem591 = null;
-              $xfer += $input->readString($elem591);
-              $this->part_vals []= $elem591;
+              $elem600 = null;
+              $xfer += $input->readString($elem600);
+              $this->part_vals []= $elem600;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16524,9 +16524,9 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter592)
+          foreach ($this->part_vals as $iter601)
           {
-            $xfer += $output->writeString($iter592);
+            $xfer += $output->writeString($iter601);
           }
         }
         $output->writeListEnd();
@@ -17380,14 +17380,14 @@ class ThriftHiveMetastore_drop_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size593 = 0;
-            $_etype596 = 0;
-            $xfer += $input->readListBegin($_etype596, $_size593);
-            for ($_i597 = 0; $_i597 < $_size593; ++$_i597)
+            $_size602 = 0;
+            $_etype605 = 0;
+            $xfer += $input->readListBegin($_etype605, $_size602);
+            for ($_i606 = 0; $_i606 < $_size602; ++$_i606)
             {
-              $elem598 = null;
-              $xfer += $input->readString($elem598);
-              $this->part_vals []= $elem598;
+              $elem607 = null;
+              $xfer += $input->readString($elem607);
+              $this->part_vals []= $elem607;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17432,9 +17432,9 @@ class ThriftHiveMetastore_drop_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter599)
+          foreach ($this->part_vals as $iter608)
           {
-            $xfer += $output->writeString($iter599);
+            $xfer += $output->writeString($iter608);
           }
         }
         $output->writeListEnd();
@@ -17687,14 +17687,14 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size600 = 0;
-            $_etype603 = 0;
-            $xfer += $input->readListBegin($_etype603, $_size600);
-            for ($_i604 = 0; $_i604 < $_size600; ++$_i604)
+            $_size609 = 0;
+            $_etype612 = 0;
+            $xfer += $input->readListBegin($_etype612, $_size609);
+            for ($_i613 = 0; $_i613 < $_size609; ++$_i613)
             {
-              $elem605 = null;
-              $xfer += $input->readString($elem605);
-              $this->part_vals []= $elem605;
+              $elem614 = null;
+              $xfer += $input->readString($elem614);
+              $this->part_vals []= $elem614;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17747,9 +17747,9 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter606)
+          foreach ($this->part_vals as $iter615)
           {
-            $xfer += $output->writeString($iter606);
+            $xfer += $output->writeString($iter615);
           }
         }
         $output->writeListEnd();
@@ -18763,14 +18763,14 @@ class ThriftHiveMetastore_get_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size607 = 0;
-            $_etype610 = 0;
-            $xfer += $input->readListBegin($_etype610, $_size607);
-            for ($_i611 = 0; $_i611 < $_size607; ++$_i611)
+            $_size616 = 0;
+            $_etype619 = 0;
+            $xfer += $input->readListBegin($_etype619, $_size616);
+            for ($_i620 = 0; $_i620 < $_size616; ++$_i620)
             {
-              $elem612 = null;
-              $xfer += $input->readString($elem612);
-              $this->part_vals []= $elem612;
+              $elem621 = null;
+              $xfer += $input->readString($elem621);
+              $this->part_vals []= $elem621;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18808,9 +18808,9 @@ class ThriftHiveMetastore_get_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter613)
+          foreach ($this->part_vals as $iter622)
           {
-            $xfer += $output->writeString($iter613);
+            $xfer += $output->writeString($iter622);
           }
         }
         $output->writeListEnd();
@@ -19052,17 +19052,17 @@ class ThriftHiveMetastore_exchange_partition_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size614 = 0;
-            $_ktype615 = 0;
-            $_vtype616 = 0;
-            $xfer += $input->readMapBegin($_ktype615, $_vtype616, $_size614);
-            for ($_i618 = 0; $_i618 < $_size614; ++$_i618)
+            $_size623 = 0;
+            $_ktype624 = 0;
+            $_vtype625 = 0;
+            $xfer += $input->readMapBegin($_ktype624, $_vtype625, $_size623);
+            for ($_i627 = 0; $_i627 < $_size623; ++$_i627)
             {
-              $key619 = '';
-              $val620 = '';
-              $xfer += $input->readString($key619);
-              $xfer += $input->readString($val620);
-              $this->partitionSpecs[$key619] = $val620;
+              $key628 = '';
+              $val629 = '';
+              $xfer += $input->readString($key628);
+              $xfer += $input->readString($val629);
+              $this->partitionSpecs[$key628] = $val629;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -19118,10 +19118,10 @@ class ThriftHiveMetastore_exchange_partition_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter621 => $viter622)
+          foreach ($this->partitionSpecs as $kiter630 => $viter631)
           {
-            $xfer += $output->writeString($kiter621);
-            $xfer += $output->writeString($viter622);
+            $xfer += $output->writeString($kiter630);
+            $xfer += $output->writeString($viter631);
           }
         }
         $output->writeMapEnd();
@@ -19433,17 +19433,17 @@ class ThriftHiveMetastore_exchange_partitions_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size623 = 0;
-            $_ktype624 = 0;
-            $_vtype625 = 0;
-            $xfer += $input->readMapBegin($_ktype624, $_vtype625, $_size623);
-            for ($_i627 = 0; $_i627 < $_size623; ++$_i627)
+            $_size632 = 0;
+            $_ktype633 = 0;
+            $_vtype634 = 0;
+            $xfer += $input->readMapBegin($_ktype633, $_vtype634, $_size632);
+            for ($_i636 = 0; $_i636 < $_size632; ++$_i636)
             {
-              $key628 = '';
-              $val629 = '';
-              $xfer += $input->readString($key628);
-              $xfer += $input->readString($val629);
-              $this->partitionSpecs[$key628] = $val629;
+              $key637 = '';
+              $val638 = '';
+              $xfer += $input->readString($key637);
+              $xfer += $input->readString($val638);
+              $this->partitionSpecs[$key637] = $val638;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -19499,10 +19499,10 @@ class ThriftHiveMetastore_exchange_partitions_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter630 => $viter631)
+          foreach ($this->partitionSpecs as $kiter639 => $viter640)
           {
-            $xfer += $output->writeString($kiter630);
-            $xfer += $output->writeString($viter631);
+            $xfer += $output->writeString($kiter639);
+            $xfer += $output->writeString($viter640);
           }
         }
         $output->writeMapEnd();
@@ -19635,15 +19635,15 @@ class ThriftHiveMetastore_exchange_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size632 = 0;
-            $_etype635 = 0;
-            $xfer += $input->readListBegin($_etype635, $_size632);
-            for ($_i636 = 0; $_i636 < $_size632; ++$_i636)
+            $_size641 = 0;
+            $_etype644 = 0;
+            $xfer += $input->readListBegin($_etype644, $_size641);
+            for ($_i645 = 0; $_i645 < $_size641; ++$_i645)
             {
-              $elem637 = null;
-              $elem637 = new \metastore\Partition();
-              $xfer += $elem637->read($input);
-              $this->success []= $elem637;
+              $elem646 = null;
+              $elem646 = new \metastore\Partition();
+              $xfer += $elem646->read($input);
+              $this->success []= $elem646;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19703,9 +19703,9 @@ class ThriftHiveMetastore_exchange_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter638)
+          foreach ($this->success as $iter647)
           {
-            $xfer += $iter638->write($output);
+            $xfer += $iter647->write($output);
           }
         }
         $output->writeListEnd();
@@ -19851,14 +19851,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size639 = 0;
-            $_etype642 = 0;
-            $xfer += $input->readListBegin($_etype642, $_size639);
-            for ($_i643 = 0; $_i643 < $_size639; ++$_i643)
+            $_size648 = 0;
+            $_etype651 = 0;
+            $xfer += $input->readListBegin($_etype651, $_size648);
+            for ($_i652 = 0; $_i652 < $_size648; ++$_i652)
             {
-              $elem644 = null;
-              $xfer += $input->readString($elem644);
-              $this->part_vals []= $elem644;
+              $elem653 = null;
+              $xfer += $input->readString($elem653);
+              $this->part_vals []= $elem653;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19875,14 +19875,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size645 = 0;
-            $_etype648 = 0;
-            $xfer += $input->readListBegin($_etype648, $_size645);
-            for ($_i649 = 0; $_i649 < $_size645; ++$_i649)
+            $_size654 = 0;
+            $_etype657 = 0;
+            $xfer += $input->readListBegin($_etype657, $_size654);
+            for ($_i658 = 0; $_i658 < $_size654; ++$_i658)
             {
-              $elem650 = null;
-              $xfer += $input->readString($elem650);
-              $this->group_names []= $elem650;
+              $elem659 = null;
+              $xfer += $input->readString($elem659);
+              $this->group_names []= $elem659;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19920,9 +19920,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter651)
+          foreach ($this->part_vals as $iter660)
           {
-            $xfer += $output->writeString($iter651);
+            $xfer += $output->writeString($iter660);
           }
         }
         $output->writeListEnd();
@@ -19942,9 +19942,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter652)
+          foreach ($this->group_names as $iter661)
           {
-            $xfer += $output->writeString($iter652);
+            $xfer += $output->writeString($iter661);
           }
         }
         $output->writeListEnd();
@@ -20535,15 +20535,15 @@ class ThriftHiveMetastore_get_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size653 = 0;
-            $_etype656 = 0;
-            $xfer += $input->readListBegin($_etype656, $_size653);
-            for ($_i657 = 0; $_i657 < $_size653; ++$_i657)
+            $_size662 = 0;
+            $_etype665 = 0;
+            $xfer += $input->readListBegin($_etype665, $_size662);
+            for ($_i666 = 0; $_i666 < $_size662; ++$_i666)
             {
-              $elem658 = null;
-              $elem658 = new \metastore\Partition();
-              $xfer += $elem658->read($input);
-              $this->success []= $elem658;
+              $elem667 = null;
+              $elem667 = new \metastore\Partition();
+              $xfer += $elem667->read($input);
+              $this->success []= $elem667;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20587,9 +20587,9 @@ class ThriftHiveMetastore_get_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter659)
+          foreach ($this->success as $iter668)
           {
-            $xfer += $iter659->write($output);
+            $xfer += $iter668->write($output);
           }
         }
         $output->writeListEnd();
@@ -20735,14 +20735,14 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size660 = 0;
-            $_etype663 = 0;
-            $xfer += $input->readListBegin($_etype663, $_size660);
-            for ($_i664 = 0; $_i664 < $_size660; ++$_i664)
+            $_size669 = 0;
+            $_etype672 = 0;
+            $xfer += $input->readListBegin($_etype672, $_size669);
+            for ($_i673 = 0; $_i673 < $_size669; ++$_i673)
             {
-              $elem665 = null;
-              $xfer += $input->readString($elem665);
-              $this->group_names []= $elem665;
+              $elem674 = null;
+              $xfer += $input->readString($elem674);
+              $this->group_names []= $elem674;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20790,9 +20790,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter666)
+          foreach ($this->group_names as $iter675)
           {
-            $xfer += $output->writeString($iter666);
+            $xfer += $output->writeString($iter675);
           }
         }
         $output->writeListEnd();
@@ -20881,15 +20881,15 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size667 = 0;
-            $_etype670 = 0;
-            $xfer += $input->readListBegin($_etype670, $_size667);
-            for ($_i671 = 0; $_i671 < $_size667; ++$_i671)
+            $_size676 = 0;
+            $_etype679 = 0;
+            $xfer += $input->readListBegin($_etype679, $_size676);
+            for ($_i680 = 0; $_i680 < $_size676; ++$_i680)
             {
-              $elem672 = null;
-              $elem672 = new \metastore\Partition();
-              $xfer += $elem672->read($input);
-              $this->success []= $elem672;
+              $elem681 = null;
+              $elem681 = new \metastore\Partition();
+              $xfer += $elem681->read($input);
+              $this->success []= $elem681;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20933,9 +20933,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter673)
+          foreach ($this->success as $iter682)
           {
-            $xfer += $iter673->write($output);
+            $xfer += $iter682->write($output);
           }
         }
         $output->writeListEnd();
@@ -21155,15 +21155,15 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size674 = 0;
-            $_etype677 = 0;
-            $xfer += $input->readListBegin($_etype677, $_size674);
-            for ($_i678 = 0; $_i678 < $_size674; ++$_i678)
+            $_size683 = 0;
+            $_etype686 = 0;
+            $xfer += $input->readListBegin($_etype686, $_size683);
+            for ($_i687 = 0; $_i687 < $_size683; ++$_i687)
             {
-              $elem679 = null;
-              $elem679 = new \metastore\PartitionSpec();
-              $xfer += $elem679->read($input);
-              $this->success []= $elem679;
+              $elem688 = null;
+              $elem688 = new \metastore\PartitionSpec();
+              $xfer += $elem688->read($input);
+              $this->success []= $elem688;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21207,9 +21207,9 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter680)
+          foreach ($this->success as $iter689)
           {
-            $xfer += $iter680->write($output);
+            $xfer += $iter689->write($output);
           }
         }
         $output->writeListEnd();
@@ -21416,14 +21416,14 @@ class ThriftHiveMetastore_get_partition_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size681 = 0;
-            $_etype684 = 0;
-            $xfer += $input->readListBegin($_etype684, $_size681);
-            for ($_i685 = 0; $_i685 < $_size681; ++$_i685)
+            $_size690 = 0;
+            $_etype693 = 0;
+            $xfer += $input->readListBegin($_etype693, $_size690);
+            for ($_i694 = 0; $_i694 < $_size690; ++$_i694)
             {
-              $elem686 = null;
-              $xfer += $input->readString($elem686);
-              $this->success []= $elem686;
+              $elem695 = null;
+              $xfer += $input->readString($elem695);
+              $this->success []= $elem695;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21459,9 +21459,9 @@ class ThriftHiveMetastore_get_partition_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter687)
+          foreach ($this->success as $iter696)
           {
-            $xfer += $output->writeString($iter687);
+            $xfer += $output->writeString($iter696);
           }
         }
         $output->writeListEnd();
@@ -21577,14 +21577,14 @@ class ThriftHiveMetastore_get_partitions_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size688 = 0;
-            $_etype691 = 0;
-            $xfer += $input->readListBegin($_etype691, $_size688);
-            for ($_i692 = 0; $_i692 < $_size688; ++$_i692)
+            $_size697 = 0;
+            $_etype700 = 0;
+            $xfer += $input->readListBegin($_etype700, $_size697);
+            for ($_i701 = 0; $_i701 < $_size697; ++$_i701)
             {
-              $elem693 = null;
-              $xfer += $input->readString($elem693);
-              $this->part_vals []= $elem693;
+              $elem702 = null;
+              $xfer += $input->readString($elem702);
+              $this->part_vals []= $elem702;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21629,9 +21629,9 @@ class ThriftHiveMetastore_get_partitions_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter694)
+          foreach ($this->part_vals as $iter703)
           {
-            $xfer += $output->writeString($iter694);
+            $xfer += $output->writeString($iter703);
           }
         }
         $output->writeListEnd();
@@ -21725,15 +21725,15 @@ class ThriftHiveMetastore_get_partitions_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size695 = 0;
-            $_etype698 = 0;
-            $xfer += $input->readListBegin($_etype698, $_size695);
-            for ($_i699 = 0; $_i699 < $_size695; ++$_i699)
+            $_size704 = 0;
+            $_etype707 = 0;
+            $xfer += $input->readListBegin($_etype707, $_size704);
+            for ($_i708 = 0; $_i708 < $_size704; ++$_i708)
             {
-              $elem700 = null;
-              $elem700 = new \metastore\Partition();
-              $xfer += $elem700->read($input);
-              $this->success []= $elem700;
+              $elem709 = null;
+              $elem709 = new \metastore\Partition();
+              $xfer += $elem709->read($input);
+              $this->success []= $elem709;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21777,9 +21777,9 @@ class ThriftHiveMetastore_get_partitions_ps_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter701)
+          foreach ($this->success as $iter710)
           {
-            $xfer += $iter701->write($output);
+            $xfer += $iter710->write($output);
           }
         }
         $output->writeListEnd();
@@ -21926,14 +21926,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size702 = 0;
-            $_etype705 = 0;
-            $xfer += $input->readListBegin($_etype705, $_size702);
-            for ($_i706 = 0; $_i706 < $_size702; ++$_i706)
+            $_size711 = 0;
+            $_etype714 = 0;
+            $xfer += $input->readListBegin($_etype714, $_size711);
+            for ($_i715 = 0; $_i715 < $_size711; ++$_i715)
             {
-              $elem707 = null;
-              $xfer += $input->readString($elem707);
-              $this->part_vals []= $elem707;
+              $elem716 = null;
+              $xfer += $input->readString($elem716);
+              $this->part_vals []= $elem716;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21957,14 +21957,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size708 = 0;
-            $_etype711 = 0;
-            $xfer += $input->readListBegin($_etype711, $_size708);
-            for ($_i712 = 0; $_i712 < $_size708; ++$_i712)
+            $_size717 = 0;
+            $_etype720 = 0;
+            $xfer += $input->readListBegin($_etype720, $_size717);
+            for ($_i721 = 0; $_i721 < $_size717; ++$_i721)
             {
-              $elem713 = null;
-              $xfer += $input->readString($elem713);
-              $this->group_names []= $elem713;
+              $elem722 = null;
+              $xfer += $input->readString($elem722);
+              $this->group_names []= $elem722;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22002,9 +22002,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter714)
+          foreach ($this->part_vals as $iter723)
           {
-            $xfer += $output->writeString($iter714);
+            $xfer += $output->writeString($iter723);
           }
         }
         $output->writeListEnd();
@@ -22029,9 +22029,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter715)
+          foreach ($this->group_names as $iter724)
           {
-            $xfer += $output->writeString($iter715);
+            $xfer += $output->writeString($iter724);
           }
         }
         $output->writeListEnd();
@@ -22120,15 +22120,15 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size716 = 0;
-            $_etype719 = 0;
-            $xfer += $input->readListBegin($_etype719, $_size716);
-            for ($_i720 = 0; $_i720 < $_size716; ++$_i720)
+            $_size725 = 0;
+            $_etype728 = 0;
+            $xfer += $input->readListBegin($_etype728, $_size725);
+            for ($_i729 = 0; $_i729 < $_size725; ++$_i729)
             {
-              $elem721 = null;
-              $elem721 = new \metastore\Partition();
-              $xfer += $elem721->read($input);
-              $this->success []= $elem721;
+              $elem730 = null;
+              $elem730 = new \metastore\Partition();
+              $xfer += $elem730->read($input);
+              $this->success []= $elem730;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22172,9 +22172,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter722)
+          foreach ($this->success as $iter731)
           {
-            $xfer += $iter722->write($output);
+            $xfer += $iter731->write($output);
           }
         }
         $output->writeListEnd();
@@ -22295,14 +22295,14 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size723 = 0;
-            $_etype726 = 0;
-            $xfer += $input->readListBegin($_etype726, $_size723);
-            for ($_i727 = 0; $_i727 < $_size723; ++$_i727)
+            $_size732 = 0;
+            $_etype735 = 0;
+            $xfer += $input->readListBegin($_etype735, $_size732);
+            for ($_i736 = 0; $_i736 < $_size732; ++$_i736)
             {
-              $elem728 = null;
-              $xfer += $input->readString($elem728);
-              $this->part_vals []= $elem728;
+              $elem737 = null;
+              $xfer += $input->readString($elem737);
+              $this->part_vals []= $elem737;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22347,9 +22347,9 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter729)
+          foreach ($this->part_vals as $iter738)
           {
-            $xfer += $output->writeString($iter729);
+            $xfer += $output->writeString($iter738);
           }
         }
         $output->writeListEnd();
@@ -22442,14 +22442,14 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size730 = 0;
-            $_etype733 = 0;
-            $xfer += $input->readListBegin($_etype733, $_size730);
-            for ($_i734 = 0; $_i734 < $_size730; ++$_i734)
+            $_size739 = 0;
+            $_etype742 = 0;
+            $xfer += $input->readListBegin($_etype742, $_size739);
+            for ($_i743 = 0; $_i743 < $_size739; ++$_i743)
             {
-              $elem735 = null;
-              $xfer += $input->readString($elem735);
-              $this->success []= $elem735;
+              $elem744 = null;
+              $xfer += $input->readString($elem744);
+              $this->success []= $elem744;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22493,9 +22493,9 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter736)
+          foreach ($this->success as $iter745)
           {
-            $xfer += $output->writeString($iter736);
+            $xfer += $output->writeString($iter745);
           }
         }
         $output->writeListEnd();
@@ -22738,15 +22738,15 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size737 = 0;
-            $_etype740 = 0;
-            $xfer += $input->readListBegin($_etype740, $_size737);
-            for ($_i741 = 0; $_i741 < $_size737; ++$_i741)
+            $_size746 = 0;
+            $_etype749 = 0;
+            $xfer += $input->readListBegin($_etype749, $_size746);
+            for ($_i750 = 0; $_i750 < $_size746; ++$_i750)
             {
-              $elem742 = null;
-              $elem742 = new \metastore\Partition();
-              $xfer += $elem742->read($input);
-              $this->success []= $elem742;
+              $elem751 = null;
+              $elem751 = new \metastore\Partition();
+              $xfer += $elem751->read($input);
+              $this->success []= $elem751;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22790,9 +22790,9 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter743)
+          foreach ($this->success as $iter752)
           {
-            $xfer += $iter743->write($output);
+            $xfer += $iter752->write($output);
           }
         }
         $output->writeListEnd();
@@ -23035,15 +23035,15 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size744 = 0;
-            $_etype747 = 0;
-            $xfer += $input->readListBegin($_etype747, $_size744);
-            for ($_i748 = 0; $_i748 < $_size744; ++$_i748)
+            $_size753 = 0;
+            $_etype756 = 0;
+            $xfer += $input->readListBegin($_etype756, $_size753);
+            for ($_i757 = 0; $_i757 < $_size753; ++$_i757)
             {
-              $elem749 = null;
-              $elem749 = new \metastore\PartitionSpec();
-              $xfer += $elem749->read($input);
-              $this->success []= $elem749;
+              $elem758 = null;
+              $elem758 = new \metastore\PartitionSpec();
+              $xfer += $elem758->read($input);
+              $this->success []= $elem758;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23087,9 +23087,9 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter750)
+          foreach ($this->success as $iter759)
           {
-            $xfer += $iter750->write($output);
+            $xfer += $iter759->write($output);
           }
         }
         $output->writeListEnd();
@@ -23409,14 +23409,14 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->names = array();
-            $_size751 = 0;
-            $_etype754 = 0;
-            $xfer += $input->readListBegin($_etype754, $_size751);
-            for ($_i755 = 0; $_i755 < $_size751; ++$_i755)
+            $_size760 = 0;
+            $_etype763 = 0;
+            $xfer += $input->readListBegin($_etype763, $_size760);
+            for ($_i764 = 0; $_i764 < $_size760; ++$_i764)
             {
-              $elem756 = null;
-              $xfer += $input->readString($elem756);
-              $this->names []= $elem756;
+              $elem765 = null;
+              $xfer += $input->readString($elem765);
+              $this->names []= $elem765;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23454,9 +23454,9 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
       {
         $output->writeListBegin(TType::STRING, count($this->names));
         {
-          foreach ($this->names as $iter757)
+          foreach ($this->names as $iter766)
           {
-            $xfer += $output->writeString($iter757);
+            $xfer += $output->writeString($iter766);
           }
         }
         $output->writeListEnd();
@@ -23545,15 +23545,15 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size758 = 0;
-            $_etype761 = 0;
-            $xfer += $input->readListBegin($_etype761, $_size758);
-            for ($_i762 = 0; $_i762 < $_size758; ++$_i762)
+            $_size767 = 0;
+            $_etype770 = 0;
+            $xfer += $input->readListBegin($_etype770, $_size767);
+            for ($_i771 = 0; $_i771 < $_size767; ++$_i771)
             {
-              $elem763 = null;
-              $elem763 = new \metastore\Partition();
-              $xfer += $elem763->read($input);
-              $this->success []= $elem763;
+              $elem772 = null;
+              $elem772 = new \metastore\Partition();
+              $xfer += $elem772->read($input);
+              $this->success []= $elem772;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23597,9 +23597,9 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter764)
+          foreach ($this->success as $iter773)
           {
-            $xfer += $iter764->write($output);
+            $xfer += $iter773->write($output);
           }
         }
         $output->writeListEnd();
@@ -23938,15 +23938,15 @@ class ThriftHiveMetastore_alter_partitions_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size765 = 0;
-            $_etype768 = 0;
-            $xfer += $input->readListBegin($_etype768, $_size765);
-            for ($_i769 = 0; $_i769 < $_size765; ++$_i769)
+            $_size774 = 0;
+            $_etype777 = 0;
+            $xfer += $input->readListBegin($_etype777, $_size774);
+            for ($_i778 = 0; $_i778 < $_size774; ++$_i778)
             {
-              $elem770 = null;
-              $elem770 = new \metastore\Partition();
-              $xfer += $elem770->read($input);
-              $this->new_parts []= $elem770;
+              $elem779 = null;
+              $elem779 = new \metastore\Partition();
+              $xfer += $elem779->read($input);
+              $this->new_parts []= $elem779;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23984,9 +23984,9 @@ class ThriftHiveMetastore_alter_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter771)
+          foreach ($this->new_parts as $iter780)
           {
-            $xfer += $iter771->write($output);
+            $xfer += $iter780->write($output);
           }
         }
         $output->writeListEnd();
@@ -24456,14 +24456,14 @@ class ThriftHiveMetastore_rename_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size772 = 0;
-            $_etype775 = 0;
-            $xfer += $input->readListBegin($_etype775, $_size772);
-            for ($_i776 = 0; $_i776 < $_size772; ++$_i776)
+            $_size781 = 0;
+            $_etype784 = 0;
+            $xfer += $input->readListBegin($_etype784, $_size781);
+            for ($_i785 = 0; $_i785 < $_size781; ++$_i785)
             {
-              $elem777 = null;
-              $xfer += $input->readString($elem777);
-              $this->part_vals []= $elem777;
+              $elem786 = null;
+              $xfer += $input->readString($elem786);
+              $this->part_vals []= $elem786;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24509,9 +24509,9 @@ class ThriftHiveMetastore_rename_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter778)
+          foreach ($this->part_vals as $iter787)
           {
-            $xfer += $output->writeString($iter778);
+            $xfer += $output->writeString($iter787);
           }
         }
         $output->writeListEnd();
@@ -24696,14 +24696,14 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size779 = 0;
-            $_etype782 = 0;
-            $xfer += $input->readListBegin($_etype782, $_size779);
-            for ($_i783 = 0; $_i783 < $_size779; ++$_i783)
+            $_size788 = 0;
+            $_etype791 = 0;
+            $xfer += $input->readListBegin($_etype791, $_size788);
+            for ($_i792 = 0; $_i792 < $_size788; ++$_i792)
             {
-              $elem784 = null;
-              $xfer += $input->readString($elem784);
-              $this->part_vals []= $elem784;
+              $elem793 = null;
+              $xfer += $input->readString($elem793);
+              $this->part_vals []= $elem793;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24738,9 +24738,9 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter785)
+          foreach ($this->part_vals as $iter794)
           {
-            $xfer += $output->writeString($iter785);
+            $xfer += $output->writeString($iter794);
           }
         }
         $output->writeListEnd();
@@ -25194,14 +25194,14 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size786 = 0;
-            $_etype789 = 0;
-            $xfer += $input->readListBegin($_etype789, $_size786);
-            for ($_i790 = 0; $_i790 < $_size786; ++$_i790)
+            $_size795 = 0;
+            $_etype798 = 0;
+            $xfer += $input->readListBegin($_etype798, $_size795);
+            for ($_i799 = 0; $_i799 < $_size795; ++$_i799)
             {
-              $elem791 = null;
-              $xfer += $input->readString($elem791);
-              $this->success []= $elem791;
+              $elem800 = null;
+              $xfer += $input->readString($elem800);
+              $this->success []= $elem800;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25237,9 +25237,9 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter792)
+          foreach ($this->success as $iter801)
           {
-            $xfer += $output->writeString($iter792);
+            $xfer += $output->writeString($iter801);
           }
         }
         $output->writeListEnd();
@@ -25399,17 +25399,17 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size793 = 0;
-            $_ktype794 = 0;
-            $_vtype795 = 0;
-            $xfer += $input->readMapBegin($_ktype794, $_vtype795, $_size793);
-            for ($_i797 = 0; $_i797 < $_size793; ++$_i797)
+            $_size802 = 0;
+            $_ktype803 = 0;
+            $_vtype804 = 0;
+            $xfer += $input->readMapBegin($_ktype803, $_vtype804, $_size802);
+            for ($_i806 = 0; $_i806 < $_size802; ++$_i806)
             {
-              $key798 = '';
-              $val799 = '';
-              $xfer += $input->readString($key798);
-              $xfer += $input->readString($val799);
-              $this->success[$key798] = $val799;
+              $key807 = '';
+              $val808 = '';
+              $xfer += $input->readString($key807);
+              $xfer += $input->readString($val808);
+              $this->success[$key807] = $val808;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -25445,10 +25445,10 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->success));
         {
-          foreach ($this->success as $kiter800 => $viter801)
+          foreach ($this->success as $kiter809 => $viter810)
           {
-            $xfer += $output->writeString($kiter800);
-            $xfer += $output->writeString($viter801);
+            $xfer += $output->writeString($kiter809);
+            $xfer += $output->writeString($viter810);
           }
         }
         $output->writeMapEnd();
@@ -25568,17 +25568,17 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size802 = 0;
-            $_ktype803 = 0;
-            $_vtype804 = 0;
-            $xfer += $input->readMapBegin($_ktype803, $_vtype804, $_size802);
-            for ($_i806 = 0; $_i806 < $_size802; ++$_i806)
+            $_size811 = 0;
+            $_ktype812 = 0;
+            $_vtype813 = 0;
+            $xfer += $input->readMapBegin($_ktype812, $_vtype813, $_size811);
+            for ($_i815 = 0; $_i815 < $_size811; ++$_i815)
             {
-              $key807 = '';
-              $val808 = '';
-              $xfer += $input->readString($key807);
-              $xfer += $input->readString($val808);
-              $this->part_vals[$key807] = $val808;
+              $key816 = '';
+              $val817 = '';
+              $xfer += $input->readString($key816);
+              $xfer += $input->readString($val817);
+              $this->part_vals[$key816] = $val817;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -25623,10 +25623,10 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $kiter809 => $viter810)
+          foreach ($this->part_vals as $kiter818 => $viter819)
           {
-            $xfer += $output->writeString($kiter809);
-            $xfer += $output->writeString($viter810);
+            $xfer += $output->writeString($kiter818);
+            $xfer += $output->writeString($viter819);
           }
         }
         $output->writeMapEnd();
@@ -25948,17 +25948,17 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size811 = 0;
-            $_ktype812 = 0;
-            $_vtype813 = 0;
-            $xfer += $input->readMapBegin($_ktype812, $_vtype813, $_size811);
-            for ($_i815 = 0; $_i815 < $_size811; ++$_i815)
+            $_size820 = 0;
+            $_ktype821 = 0;
+            $_vtype822 = 0;
+            $xfer += $input->readMapBegin($_ktype821, $_vtype822, $_size820);
+            for ($_i824 = 0; $_i824 < $_size820; ++$_i824)
             {
-              $key816 = '';
-              $val817 = '';
-              $xfer += $input->readString($key816);
-              $xfer += $input->readString($val817);
-              $this->part_vals[$key816] = $val817;
+              $key825 = '';
+              $val826 = '';
+              $xfer += $input->readString($key825);
+              $xfer += $input->readString($val826);
+              $this->part_vals[$key825] = $val826;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -26003,10 +26003,10 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $kiter818 => $viter819)
+          foreach ($this->part_vals as $kiter827 => $viter828)
           {
-            $xfer += $output->writeString($kiter818);
-            $xfer += $output->writeString($viter819);
+            $xfer += $output->writeString($kiter827);
+            $xfer += $output->writeString($viter828);
           }
         }
         $output->writeMapEnd();
@@ -27480,15 +27480,15 @@ class ThriftHiveMetastore_get_indexes_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size820 = 0;
-            $_etype823 = 0;
-            $xfer += $input->readListBegin($_etype823, $_size820);
-            for ($_i824 = 0; $_i824 < $_size820; ++$_i824)
+            $_size829 = 0;
+            $_etype832 = 0;
+            $xfer += $input->readListBegin($_etype832, $_size829);
+            for ($_i833 = 0; $_i833 < $_size829; ++$_i833)
             {
-              $elem825 = null;
-              $elem825 = new \metastore\Index();
-              $xfer += $elem825->read($input);
-              $this->success []= $elem825;
+              $elem834 = null;
+              $elem834 = new \metastore\Index();
+              $xfer += $elem834->read($input);
+              $this->success []= $elem834;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27532,9 +27532,9 @@ class ThriftHiveMetastore_get_indexes_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter826)
+          foreach ($this->success as $iter835)
           {
-            $xfer += $iter826->write($output);
+            $xfer += $iter835->write($output);
           }
         }
         $output->writeListEnd();
@@ -27741,14 +27741,14 @@ class ThriftHiveMetastore_get_index_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size827 = 0;
-            $_etype830 = 0;
-            $xfer += $input->readListBegin($_etype830, $_size827);
-            for ($_i831 = 0; $_i831 < $_size827; ++$_i831)
+            $_size836 = 0;
+            $_etype839 = 0;
+            $xfer += $input->readListBegin($_etype839, $_size836);
+            for ($_i840 = 0; $_i840 < $_size836; ++$_i840)
             {
-              $elem832 = null;
-              $xfer += $input->readString($elem832);
-              $this->success []= $elem832;
+              $elem841 = null;
+              $xfer += $input->readString($elem841);
+              $this->success []= $elem841;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27784,9 +27784,9 @@ class ThriftHiveMetastore_get_index_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter833)
+          foreach ($this->success as $iter842)
           {
-            $xfer += $output->writeString($iter833);
+            $xfer += $output->writeString($iter842);
           }
         }
         $output->writeListEnd();
@@ -31260,14 +31260,14 @@ class ThriftHiveMetastore_get_functions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size834 = 0;
-            $_etype837 = 0;
-            $xfer += $input->readListBegin($_etype837, $_size834);
-            for ($_i838 = 0; $_i838 < $_size834; ++$_i838)
+            $_size843 = 0;
+            $_etype846 = 0;
+            $xfer += $input->readListBegin($_etype846, $_size843);
+            for ($_i847 = 0; $_i847 < $_size843; ++$_i847)
             {
-              $elem839 = null;
-              $xfer += $input->readString($elem839);
-              $this->success []= $elem839;
+              $elem848 = null;
+              $xfer += $input->readString($elem848);
+              $this->success []= $elem848;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31303,9 +31303,9 @@ class ThriftHiveMetastore_get_functions_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter840)
+          foreach ($this->success as $iter849)
           {
-            $xfer += $output->writeString($iter840);
+            $xfer += $output->writeString($iter849);
           }
         }
         $output->writeListEnd();
@@ -32174,14 +32174,14 @@ class ThriftHiveMetastore_get_role_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size841 = 0;
-            $_etype844 = 0;
-            $xfer += $input->readListBegin($_etype844, $_size841);
-            for ($_i845 = 0; $_i845 < $_size841; ++$_i845)
+            $_size850 = 0;
+            $_etype853 = 0;
+            $xfer += $input->readListBegin($_etype853, $_size850);
+            for ($_i854 = 0; $_i854 < $_size850; ++$_i854)
             {
-              $elem846 = null;
-              $xfer += $input->readString($elem846);
-              $this->success []= $elem846;
+              $elem855 = null;
+              $xfer += $input->readString($elem855);
+              $this->success []= $elem855;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32217,9 +32217,9 @@ class ThriftHiveMetastore_get_role_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter847)
+          foreach ($this->success as $iter856)
           {
-            $xfer += $output->writeString($iter847);
+            $xfer += $output->writeString($iter856);
           }
         }
         $output->writeListEnd();
@@ -32910,15 +32910,15 @@ class ThriftHiveMetastore_list_roles_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size848 = 0;
-            $_etype851 = 0;
-            $xfer += $input->readListBegin($_etype851, $_size848);
-            for ($_i852 = 0; $_i852 < $_size848; ++$_i852)
+            $_size857 = 0;
+            $_etype860 = 0;
+            $xfer += $input->readListBegin($_etype860, $_size857);
+            for ($_i861 = 0; $_i861 < $_size857; ++$_i861)
             {
-              $elem853 = null;
-              $elem853 = new \metastore\Role();
-              $xfer += $elem853->read($input);
-              $this->success []= $elem853;
+              $elem862 = null;
+              $elem862 = new \metastore\Role();
+              $xfer += $elem862->read($input);
+              $this->success []= $elem862;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32954,9 +32954,9 @@ class ThriftHiveMetastore_list_roles_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter854)
+          foreach ($this->success as $iter863)
           {
-            $xfer += $iter854->write($output);
+            $xfer += $iter863->write($output);
           }
         }
         $output->writeListEnd();
@@ -33618,14 +33618,14 @@ class ThriftHiveMetastore_get_privilege_set_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size855 = 0;
-            $_etype858 = 0;
-            $xfer += $input->readListBegin($_etype858, $_size855);
-            for ($_i859 = 0; $_i859 < $_size855; ++$_i859)
+            $_size864 = 0;
+            $_etype867 = 0;
+            $xfer += $input->readListBegin($_etype867, $_size864);
+            for ($_i868 = 0; $_i868 < $_size864; ++$_i868)
             {
-              $elem860 = null;
-              $xfer += $input->readString($elem860);
-              $this->group_names []= $elem860;
+              $elem869 = null;
+              $xfer += $input->readString($elem869);
+              $this->group_names []= $elem869;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33666,9 +33666,9 @@ class ThriftHiveMetastore_get_privilege_set_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter861)
+          foreach ($this->group_names as $iter870)
           {
-            $xfer += $output->writeString($iter861);
+            $xfer += $output->writeString($iter870);
           }
         }
         $output->writeListEnd();
@@ -33976,15 +33976,15 @@ class ThriftHiveMetastore_list_privileges_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size862 = 0;
-            $_etype865 = 0;
-            $xfer += $input->readListBegin($_etype865, $_size862);
-            for ($_i866 = 0; $_i866 < $_size862; ++$_i866)
+            $_size871 = 0;
+            $_etype874 = 0;
+            $xfer += $input->readListBegin($_etype874, $_size871);
+            for ($_i875 = 0; $_i875 < $_size871; ++$_i875)
             {
-              $elem867 = null;
-              $elem867 = new \metastore\HiveObjectPrivilege();
-              $xfer += $elem867->read($input);
-              $this->success []= $elem867;
+              $elem876 = null;
+              $elem876 = new \metastore\HiveObjectPrivilege();
+              $xfer += $elem876->read($input);
+              $this->success []= $elem876;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34020,9 +34020,9 @@ class ThriftHiveMetastore_list_privileges_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter868)
+          foreach ($this->success as $iter877)
           {
-            $xfer += $iter868->write($output);
+            $xfer += $iter877->write($output);
           }
         }
         $output->writeListEnd();
@@ -34654,14 +34654,14 @@ class ThriftHiveMetastore_set_ugi_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size869 = 0;
-            $_etype872 = 0;
-            $xfer += $input->readListBegin($_etype872, $_size869);
-            for ($_i873 = 0; $_i873 < $_size869; ++$_i873)
+            $_size878 = 0;
+            $_etype881 = 0;
+            $xfer += $input->readListBegin($_etype881, $_size878);
+            for ($_i882 = 0; $_i882 < $_size878; ++$_i882)
             {
-              $elem874 = null;
-              $xfer += $input->readString($elem874);
-              $this->group_names []= $elem874;
+              $elem883 = null;
+              $xfer += $input->readString($elem883);
+              $this->group_names []= $elem883;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34694,9 +34694,9 @@ class ThriftHiveMetastore_set_ugi_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter875)
+          foreach ($this->group_names as $iter884)
           {
-            $xfer += $output->writeString($iter875);
+            $xfer += $output->writeString($iter884);
           }
         }
         $output->writeListEnd();
@@ -34772,14 +34772,14 @@ class ThriftHiveMetastore_set_ugi_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size876 = 0;
-            $_etype879 = 0;
-            $xfer += $input->readListBegin($_etype879, $_size876);
-            for ($_i880 = 0; $_i880 < $_size876; ++$_i880)
+            $_size885 = 0;
+            $_etype888 = 0;
+            $xfer += $input->readListBegin($_etype888, $_size885);
+            for ($_i889 = 0; $_i889 < $_size885; ++$_i889)
             {
-              $elem881 = null;
-              $xfer += $input->readString($elem881);
-              $this->success []= $elem881;
+              $elem890 = null;
+              $xfer += $input->readString($elem890);
+              $this->success []= $elem890;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34815,9 +34815,9 @@ class ThriftHiveMetastore_set_ugi_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter882)
+          foreach ($this->success as $iter891)
           {
-            $xfer += $output->writeString($iter882);
+            $xfer += $output->writeString($iter891);
           }
         }
         $output->writeListEnd();


[4/7] hive git commit: HIVE-13354 : Add ability to specify Compaction options per table and per request (Wei Zheng, reviewed by Eugene Koifman)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index e836154..957a256 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@ -25539,13 +25539,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list540 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list540.size);
-                  String _elem541;
-                  for (int _i542 = 0; _i542 < _list540.size; ++_i542)
+                  org.apache.thrift.protocol.TList _list550 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list550.size);
+                  String _elem551;
+                  for (int _i552 = 0; _i552 < _list550.size; ++_i552)
                   {
-                    _elem541 = iprot.readString();
-                    struct.success.add(_elem541);
+                    _elem551 = iprot.readString();
+                    struct.success.add(_elem551);
                   }
                   iprot.readListEnd();
                 }
@@ -25580,9 +25580,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter543 : struct.success)
+            for (String _iter553 : struct.success)
             {
-              oprot.writeString(_iter543);
+              oprot.writeString(_iter553);
             }
             oprot.writeListEnd();
           }
@@ -25621,9 +25621,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter544 : struct.success)
+            for (String _iter554 : struct.success)
             {
-              oprot.writeString(_iter544);
+              oprot.writeString(_iter554);
             }
           }
         }
@@ -25638,13 +25638,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list545 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list545.size);
-            String _elem546;
-            for (int _i547 = 0; _i547 < _list545.size; ++_i547)
+            org.apache.thrift.protocol.TList _list555 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list555.size);
+            String _elem556;
+            for (int _i557 = 0; _i557 < _list555.size; ++_i557)
             {
-              _elem546 = iprot.readString();
-              struct.success.add(_elem546);
+              _elem556 = iprot.readString();
+              struct.success.add(_elem556);
             }
           }
           struct.setSuccessIsSet(true);
@@ -26298,13 +26298,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list548 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list548.size);
-                  String _elem549;
-                  for (int _i550 = 0; _i550 < _list548.size; ++_i550)
+                  org.apache.thrift.protocol.TList _list558 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list558.size);
+                  String _elem559;
+                  for (int _i560 = 0; _i560 < _list558.size; ++_i560)
                   {
-                    _elem549 = iprot.readString();
-                    struct.success.add(_elem549);
+                    _elem559 = iprot.readString();
+                    struct.success.add(_elem559);
                   }
                   iprot.readListEnd();
                 }
@@ -26339,9 +26339,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter551 : struct.success)
+            for (String _iter561 : struct.success)
             {
-              oprot.writeString(_iter551);
+              oprot.writeString(_iter561);
             }
             oprot.writeListEnd();
           }
@@ -26380,9 +26380,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter552 : struct.success)
+            for (String _iter562 : struct.success)
             {
-              oprot.writeString(_iter552);
+              oprot.writeString(_iter562);
             }
           }
         }
@@ -26397,13 +26397,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list553 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list553.size);
-            String _elem554;
-            for (int _i555 = 0; _i555 < _list553.size; ++_i555)
+            org.apache.thrift.protocol.TList _list563 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list563.size);
+            String _elem564;
+            for (int _i565 = 0; _i565 < _list563.size; ++_i565)
             {
-              _elem554 = iprot.readString();
-              struct.success.add(_elem554);
+              _elem564 = iprot.readString();
+              struct.success.add(_elem564);
             }
           }
           struct.setSuccessIsSet(true);
@@ -31010,16 +31010,16 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map556 = iprot.readMapBegin();
-                  struct.success = new HashMap<String,Type>(2*_map556.size);
-                  String _key557;
-                  Type _val558;
-                  for (int _i559 = 0; _i559 < _map556.size; ++_i559)
+                  org.apache.thrift.protocol.TMap _map566 = iprot.readMapBegin();
+                  struct.success = new HashMap<String,Type>(2*_map566.size);
+                  String _key567;
+                  Type _val568;
+                  for (int _i569 = 0; _i569 < _map566.size; ++_i569)
                   {
-                    _key557 = iprot.readString();
-                    _val558 = new Type();
-                    _val558.read(iprot);
-                    struct.success.put(_key557, _val558);
+                    _key567 = iprot.readString();
+                    _val568 = new Type();
+                    _val568.read(iprot);
+                    struct.success.put(_key567, _val568);
                   }
                   iprot.readMapEnd();
                 }
@@ -31054,10 +31054,10 @@ public class ThriftHiveMetastore {
           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> _iter560 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter570 : struct.success.entrySet())
             {
-              oprot.writeString(_iter560.getKey());
-              _iter560.getValue().write(oprot);
+              oprot.writeString(_iter570.getKey());
+              _iter570.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -31096,10 +31096,10 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Map.Entry<String, Type> _iter561 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter571 : struct.success.entrySet())
             {
-              oprot.writeString(_iter561.getKey());
-              _iter561.getValue().write(oprot);
+              oprot.writeString(_iter571.getKey());
+              _iter571.getValue().write(oprot);
             }
           }
         }
@@ -31114,16 +31114,16 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map562 = 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*_map562.size);
-            String _key563;
-            Type _val564;
-            for (int _i565 = 0; _i565 < _map562.size; ++_i565)
+            org.apache.thrift.protocol.TMap _map572 = 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*_map572.size);
+            String _key573;
+            Type _val574;
+            for (int _i575 = 0; _i575 < _map572.size; ++_i575)
             {
-              _key563 = iprot.readString();
-              _val564 = new Type();
-              _val564.read(iprot);
-              struct.success.put(_key563, _val564);
+              _key573 = iprot.readString();
+              _val574 = new Type();
+              _val574.read(iprot);
+              struct.success.put(_key573, _val574);
             }
           }
           struct.setSuccessIsSet(true);
@@ -32158,14 +32158,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list566 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list566.size);
-                  FieldSchema _elem567;
-                  for (int _i568 = 0; _i568 < _list566.size; ++_i568)
+                  org.apache.thrift.protocol.TList _list576 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list576.size);
+                  FieldSchema _elem577;
+                  for (int _i578 = 0; _i578 < _list576.size; ++_i578)
                   {
-                    _elem567 = new FieldSchema();
-                    _elem567.read(iprot);
-                    struct.success.add(_elem567);
+                    _elem577 = new FieldSchema();
+                    _elem577.read(iprot);
+                    struct.success.add(_elem577);
                   }
                   iprot.readListEnd();
                 }
@@ -32218,9 +32218,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter569 : struct.success)
+            for (FieldSchema _iter579 : struct.success)
             {
-              _iter569.write(oprot);
+              _iter579.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -32275,9 +32275,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter570 : struct.success)
+            for (FieldSchema _iter580 : struct.success)
             {
-              _iter570.write(oprot);
+              _iter580.write(oprot);
             }
           }
         }
@@ -32298,14 +32298,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list571 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list571.size);
-            FieldSchema _elem572;
-            for (int _i573 = 0; _i573 < _list571.size; ++_i573)
+            org.apache.thrift.protocol.TList _list581 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list581.size);
+            FieldSchema _elem582;
+            for (int _i583 = 0; _i583 < _list581.size; ++_i583)
             {
-              _elem572 = new FieldSchema();
-              _elem572.read(iprot);
-              struct.success.add(_elem572);
+              _elem582 = new FieldSchema();
+              _elem582.read(iprot);
+              struct.success.add(_elem582);
             }
           }
           struct.setSuccessIsSet(true);
@@ -33459,14 +33459,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list574 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list574.size);
-                  FieldSchema _elem575;
-                  for (int _i576 = 0; _i576 < _list574.size; ++_i576)
+                  org.apache.thrift.protocol.TList _list584 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list584.size);
+                  FieldSchema _elem585;
+                  for (int _i586 = 0; _i586 < _list584.size; ++_i586)
                   {
-                    _elem575 = new FieldSchema();
-                    _elem575.read(iprot);
-                    struct.success.add(_elem575);
+                    _elem585 = new FieldSchema();
+                    _elem585.read(iprot);
+                    struct.success.add(_elem585);
                   }
                   iprot.readListEnd();
                 }
@@ -33519,9 +33519,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter577 : struct.success)
+            for (FieldSchema _iter587 : struct.success)
             {
-              _iter577.write(oprot);
+              _iter587.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -33576,9 +33576,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter578 : struct.success)
+            for (FieldSchema _iter588 : struct.success)
             {
-              _iter578.write(oprot);
+              _iter588.write(oprot);
             }
           }
         }
@@ -33599,14 +33599,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list579 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list579.size);
-            FieldSchema _elem580;
-            for (int _i581 = 0; _i581 < _list579.size; ++_i581)
+            org.apache.thrift.protocol.TList _list589 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list589.size);
+            FieldSchema _elem590;
+            for (int _i591 = 0; _i591 < _list589.size; ++_i591)
             {
-              _elem580 = new FieldSchema();
-              _elem580.read(iprot);
-              struct.success.add(_elem580);
+              _elem590 = new FieldSchema();
+              _elem590.read(iprot);
+              struct.success.add(_elem590);
             }
           }
           struct.setSuccessIsSet(true);
@@ -34651,14 +34651,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list582 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list582.size);
-                  FieldSchema _elem583;
-                  for (int _i584 = 0; _i584 < _list582.size; ++_i584)
+                  org.apache.thrift.protocol.TList _list592 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list592.size);
+                  FieldSchema _elem593;
+                  for (int _i594 = 0; _i594 < _list592.size; ++_i594)
                   {
-                    _elem583 = new FieldSchema();
-                    _elem583.read(iprot);
-                    struct.success.add(_elem583);
+                    _elem593 = new FieldSchema();
+                    _elem593.read(iprot);
+                    struct.success.add(_elem593);
                   }
                   iprot.readListEnd();
                 }
@@ -34711,9 +34711,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter585 : struct.success)
+            for (FieldSchema _iter595 : struct.success)
             {
-              _iter585.write(oprot);
+              _iter595.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -34768,9 +34768,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter586 : struct.success)
+            for (FieldSchema _iter596 : struct.success)
             {
-              _iter586.write(oprot);
+              _iter596.write(oprot);
             }
           }
         }
@@ -34791,14 +34791,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list587 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list587.size);
-            FieldSchema _elem588;
-            for (int _i589 = 0; _i589 < _list587.size; ++_i589)
+            org.apache.thrift.protocol.TList _list597 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list597.size);
+            FieldSchema _elem598;
+            for (int _i599 = 0; _i599 < _list597.size; ++_i599)
             {
-              _elem588 = new FieldSchema();
-              _elem588.read(iprot);
-              struct.success.add(_elem588);
+              _elem598 = new FieldSchema();
+              _elem598.read(iprot);
+              struct.success.add(_elem598);
             }
           }
           struct.setSuccessIsSet(true);
@@ -35952,14 +35952,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list590 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list590.size);
-                  FieldSchema _elem591;
-                  for (int _i592 = 0; _i592 < _list590.size; ++_i592)
+                  org.apache.thrift.protocol.TList _list600 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list600.size);
+                  FieldSchema _elem601;
+                  for (int _i602 = 0; _i602 < _list600.size; ++_i602)
                   {
-                    _elem591 = new FieldSchema();
-                    _elem591.read(iprot);
-                    struct.success.add(_elem591);
+                    _elem601 = new FieldSchema();
+                    _elem601.read(iprot);
+                    struct.success.add(_elem601);
                   }
                   iprot.readListEnd();
                 }
@@ -36012,9 +36012,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter593 : struct.success)
+            for (FieldSchema _iter603 : struct.success)
             {
-              _iter593.write(oprot);
+              _iter603.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -36069,9 +36069,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter594 : struct.success)
+            for (FieldSchema _iter604 : struct.success)
             {
-              _iter594.write(oprot);
+              _iter604.write(oprot);
             }
           }
         }
@@ -36092,14 +36092,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list595 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list595.size);
-            FieldSchema _elem596;
-            for (int _i597 = 0; _i597 < _list595.size; ++_i597)
+            org.apache.thrift.protocol.TList _list605 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list605.size);
+            FieldSchema _elem606;
+            for (int _i607 = 0; _i607 < _list605.size; ++_i607)
             {
-              _elem596 = new FieldSchema();
-              _elem596.read(iprot);
-              struct.success.add(_elem596);
+              _elem606 = new FieldSchema();
+              _elem606.read(iprot);
+              struct.success.add(_elem606);
             }
           }
           struct.setSuccessIsSet(true);
@@ -41339,13 +41339,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list598 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list598.size);
-                  String _elem599;
-                  for (int _i600 = 0; _i600 < _list598.size; ++_i600)
+                  org.apache.thrift.protocol.TList _list608 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list608.size);
+                  String _elem609;
+                  for (int _i610 = 0; _i610 < _list608.size; ++_i610)
                   {
-                    _elem599 = iprot.readString();
-                    struct.success.add(_elem599);
+                    _elem609 = iprot.readString();
+                    struct.success.add(_elem609);
                   }
                   iprot.readListEnd();
                 }
@@ -41380,9 +41380,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter601 : struct.success)
+            for (String _iter611 : struct.success)
             {
-              oprot.writeString(_iter601);
+              oprot.writeString(_iter611);
             }
             oprot.writeListEnd();
           }
@@ -41421,9 +41421,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter602 : struct.success)
+            for (String _iter612 : struct.success)
             {
-              oprot.writeString(_iter602);
+              oprot.writeString(_iter612);
             }
           }
         }
@@ -41438,13 +41438,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list603 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list603.size);
-            String _elem604;
-            for (int _i605 = 0; _i605 < _list603.size; ++_i605)
+            org.apache.thrift.protocol.TList _list613 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list613.size);
+            String _elem614;
+            for (int _i615 = 0; _i615 < _list613.size; ++_i615)
             {
-              _elem604 = iprot.readString();
-              struct.success.add(_elem604);
+              _elem614 = iprot.readString();
+              struct.success.add(_elem614);
             }
           }
           struct.setSuccessIsSet(true);
@@ -42210,13 +42210,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list606 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list606.size);
-                  String _elem607;
-                  for (int _i608 = 0; _i608 < _list606.size; ++_i608)
+                  org.apache.thrift.protocol.TList _list616 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list616.size);
+                  String _elem617;
+                  for (int _i618 = 0; _i618 < _list616.size; ++_i618)
                   {
-                    _elem607 = iprot.readString();
-                    struct.success.add(_elem607);
+                    _elem617 = iprot.readString();
+                    struct.success.add(_elem617);
                   }
                   iprot.readListEnd();
                 }
@@ -42251,9 +42251,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter609 : struct.success)
+            for (String _iter619 : struct.success)
             {
-              oprot.writeString(_iter609);
+              oprot.writeString(_iter619);
             }
             oprot.writeListEnd();
           }
@@ -42292,9 +42292,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter610 : struct.success)
+            for (String _iter620 : struct.success)
             {
-              oprot.writeString(_iter610);
+              oprot.writeString(_iter620);
             }
           }
         }
@@ -42309,13 +42309,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list611 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list611.size);
-            String _elem612;
-            for (int _i613 = 0; _i613 < _list611.size; ++_i613)
+            org.apache.thrift.protocol.TList _list621 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list621.size);
+            String _elem622;
+            for (int _i623 = 0; _i623 < _list621.size; ++_i623)
             {
-              _elem612 = iprot.readString();
-              struct.success.add(_elem612);
+              _elem622 = iprot.readString();
+              struct.success.add(_elem622);
             }
           }
           struct.setSuccessIsSet(true);
@@ -43768,13 +43768,13 @@ public class ThriftHiveMetastore {
             case 2: // TBL_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list614 = iprot.readListBegin();
-                  struct.tbl_names = new ArrayList<String>(_list614.size);
-                  String _elem615;
-                  for (int _i616 = 0; _i616 < _list614.size; ++_i616)
+                  org.apache.thrift.protocol.TList _list624 = iprot.readListBegin();
+                  struct.tbl_names = new ArrayList<String>(_list624.size);
+                  String _elem625;
+                  for (int _i626 = 0; _i626 < _list624.size; ++_i626)
                   {
-                    _elem615 = iprot.readString();
-                    struct.tbl_names.add(_elem615);
+                    _elem625 = iprot.readString();
+                    struct.tbl_names.add(_elem625);
                   }
                   iprot.readListEnd();
                 }
@@ -43805,9 +43805,9 @@ public class ThriftHiveMetastore {
           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 _iter617 : struct.tbl_names)
+            for (String _iter627 : struct.tbl_names)
             {
-              oprot.writeString(_iter617);
+              oprot.writeString(_iter627);
             }
             oprot.writeListEnd();
           }
@@ -43844,9 +43844,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetTbl_names()) {
           {
             oprot.writeI32(struct.tbl_names.size());
-            for (String _iter618 : struct.tbl_names)
+            for (String _iter628 : struct.tbl_names)
             {
-              oprot.writeString(_iter618);
+              oprot.writeString(_iter628);
             }
           }
         }
@@ -43862,13 +43862,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list619 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_names = new ArrayList<String>(_list619.size);
-            String _elem620;
-            for (int _i621 = 0; _i621 < _list619.size; ++_i621)
+            org.apache.thrift.protocol.TList _list629 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_names = new ArrayList<String>(_list629.size);
+            String _elem630;
+            for (int _i631 = 0; _i631 < _list629.size; ++_i631)
             {
-              _elem620 = iprot.readString();
-              struct.tbl_names.add(_elem620);
+              _elem630 = iprot.readString();
+              struct.tbl_names.add(_elem630);
             }
           }
           struct.setTbl_namesIsSet(true);
@@ -44436,14 +44436,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list622 = iprot.readListBegin();
-                  struct.success = new ArrayList<Table>(_list622.size);
-                  Table _elem623;
-                  for (int _i624 = 0; _i624 < _list622.size; ++_i624)
+                  org.apache.thrift.protocol.TList _list632 = iprot.readListBegin();
+                  struct.success = new ArrayList<Table>(_list632.size);
+                  Table _elem633;
+                  for (int _i634 = 0; _i634 < _list632.size; ++_i634)
                   {
-                    _elem623 = new Table();
-                    _elem623.read(iprot);
-                    struct.success.add(_elem623);
+                    _elem633 = new Table();
+                    _elem633.read(iprot);
+                    struct.success.add(_elem633);
                   }
                   iprot.readListEnd();
                 }
@@ -44496,9 +44496,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Table _iter625 : struct.success)
+            for (Table _iter635 : struct.success)
             {
-              _iter625.write(oprot);
+              _iter635.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -44553,9 +44553,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Table _iter626 : struct.success)
+            for (Table _iter636 : struct.success)
             {
-              _iter626.write(oprot);
+              _iter636.write(oprot);
             }
           }
         }
@@ -44576,14 +44576,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list627 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Table>(_list627.size);
-            Table _elem628;
-            for (int _i629 = 0; _i629 < _list627.size; ++_i629)
+            org.apache.thrift.protocol.TList _list637 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Table>(_list637.size);
+            Table _elem638;
+            for (int _i639 = 0; _i639 < _list637.size; ++_i639)
             {
-              _elem628 = new Table();
-              _elem628.read(iprot);
-              struct.success.add(_elem628);
+              _elem638 = new Table();
+              _elem638.read(iprot);
+              struct.success.add(_elem638);
             }
           }
           struct.setSuccessIsSet(true);
@@ -45729,13 +45729,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list630 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list630.size);
-                  String _elem631;
-                  for (int _i632 = 0; _i632 < _list630.size; ++_i632)
+                  org.apache.thrift.protocol.TList _list640 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list640.size);
+                  String _elem641;
+                  for (int _i642 = 0; _i642 < _list640.size; ++_i642)
                   {
-                    _elem631 = iprot.readString();
-                    struct.success.add(_elem631);
+                    _elem641 = iprot.readString();
+                    struct.success.add(_elem641);
                   }
                   iprot.readListEnd();
                 }
@@ -45788,9 +45788,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter633 : struct.success)
+            for (String _iter643 : struct.success)
             {
-              oprot.writeString(_iter633);
+              oprot.writeString(_iter643);
             }
             oprot.writeListEnd();
           }
@@ -45845,9 +45845,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter634 : struct.success)
+            for (String _iter644 : struct.success)
             {
-              oprot.writeString(_iter634);
+              oprot.writeString(_iter644);
             }
           }
         }
@@ -45868,13 +45868,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list635 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list635.size);
-            String _elem636;
-            for (int _i637 = 0; _i637 < _list635.size; ++_i637)
+            org.apache.thrift.protocol.TList _list645 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list645.size);
+            String _elem646;
+            for (int _i647 = 0; _i647 < _list645.size; ++_i647)
             {
-              _elem636 = iprot.readString();
-              struct.success.add(_elem636);
+              _elem646 = iprot.readString();
+              struct.success.add(_elem646);
             }
           }
           struct.setSuccessIsSet(true);
@@ -51733,14 +51733,14 @@ public class ThriftHiveMetastore {
             case 1: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list638 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<Partition>(_list638.size);
-                  Partition _elem639;
-                  for (int _i640 = 0; _i640 < _list638.size; ++_i640)
+                  org.apache.thrift.protocol.TList _list648 = iprot.readListBegin();
+                  struct.new_parts = new ArrayList<Partition>(_list648.size);
+                  Partition _elem649;
+                  for (int _i650 = 0; _i650 < _list648.size; ++_i650)
                   {
-                    _elem639 = new Partition();
-                    _elem639.read(iprot);
-                    struct.new_parts.add(_elem639);
+                    _elem649 = new Partition();
+                    _elem649.read(iprot);
+                    struct.new_parts.add(_elem649);
                   }
                   iprot.readListEnd();
                 }
@@ -51766,9 +51766,9 @@ public class ThriftHiveMetastore {
           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 _iter641 : struct.new_parts)
+            for (Partition _iter651 : struct.new_parts)
             {
-              _iter641.write(oprot);
+              _iter651.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -51799,9 +51799,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (Partition _iter642 : struct.new_parts)
+            for (Partition _iter652 : struct.new_parts)
             {
-              _iter642.write(oprot);
+              _iter652.write(oprot);
             }
           }
         }
@@ -51813,14 +51813,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list643 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.new_parts = new ArrayList<Partition>(_list643.size);
-            Partition _elem644;
-            for (int _i645 = 0; _i645 < _list643.size; ++_i645)
+            org.apache.thrift.protocol.TList _list653 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.new_parts = new ArrayList<Partition>(_list653.size);
+            Partition _elem654;
+            for (int _i655 = 0; _i655 < _list653.size; ++_i655)
             {
-              _elem644 = new Partition();
-              _elem644.read(iprot);
-              struct.new_parts.add(_elem644);
+              _elem654 = new Partition();
+              _elem654.read(iprot);
+              struct.new_parts.add(_elem654);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -52821,14 +52821,14 @@ public class ThriftHiveMetastore {
             case 1: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list646 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<PartitionSpec>(_list646.size);
-                  PartitionSpec _elem647;
-                  for (int _i648 = 0; _i648 < _list646.size; ++_i648)
+                  org.apache.thrift.protocol.TList _list656 = iprot.readListBegin();
+                  struct.new_parts = new ArrayList<PartitionSpec>(_list656.size);
+                  PartitionSpec _elem657;
+                  for (int _i658 = 0; _i658 < _list656.size; ++_i658)
                   {
-                    _elem647 = new PartitionSpec();
-                    _elem647.read(iprot);
-                    struct.new_parts.add(_elem647);
+                    _elem657 = new PartitionSpec();
+                    _elem657.read(iprot);
+                    struct.new_parts.add(_elem657);
                   }
                   iprot.readListEnd();
                 }
@@ -52854,9 +52854,9 @@ public class ThriftHiveMetastore {
           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 _iter649 : struct.new_parts)
+            for (PartitionSpec _iter659 : struct.new_parts)
             {
-              _iter649.write(oprot);
+              _iter659.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -52887,9 +52887,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (PartitionSpec _iter650 : struct.new_parts)
+            for (PartitionSpec _iter660 : struct.new_parts)
             {
-              _iter650.write(oprot);
+              _iter660.write(oprot);
             }
           }
         }
@@ -52901,14 +52901,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list651 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.new_parts = new ArrayList<PartitionSpec>(_list651.size);
-            PartitionSpec _elem652;
-            for (int _i653 = 0; _i653 < _list651.size; ++_i653)
+            org.apache.thrift.protocol.TList _list661 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.new_parts = new ArrayList<PartitionSpec>(_list661.size);
+            PartitionSpec _elem662;
+            for (int _i663 = 0; _i663 < _list661.size; ++_i663)
             {
-              _elem652 = new PartitionSpec();
-              _elem652.read(iprot);
-              struct.new_parts.add(_elem652);
+              _elem662 = new PartitionSpec();
+              _elem662.read(iprot);
+              struct.new_parts.add(_elem662);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -54084,13 +54084,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list654 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list654.size);
-                  String _elem655;
-                  for (int _i656 = 0; _i656 < _list654.size; ++_i656)
+                  org.apache.thrift.protocol.TList _list664 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list664.size);
+                  String _elem665;
+                  for (int _i666 = 0; _i666 < _list664.size; ++_i666)
                   {
-                    _elem655 = iprot.readString();
-                    struct.part_vals.add(_elem655);
+                    _elem665 = iprot.readString();
+                    struct.part_vals.add(_elem665);
                   }
                   iprot.readListEnd();
                 }
@@ -54126,9 +54126,9 @@ public class ThriftHiveMetastore {
           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 _iter657 : struct.part_vals)
+            for (String _iter667 : struct.part_vals)
             {
-              oprot.writeString(_iter657);
+              oprot.writeString(_iter667);
             }
             oprot.writeListEnd();
           }
@@ -54171,9 +54171,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter658 : struct.part_vals)
+            for (String _iter668 : struct.part_vals)
             {
-              oprot.writeString(_iter658);
+              oprot.writeString(_iter668);
             }
           }
         }
@@ -54193,13 +54193,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list659 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list659.size);
-            String _elem660;
-            for (int _i661 = 0; _i661 < _list659.size; ++_i661)
+            org.apache.thrift.protocol.TList _list669 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list669.size);
+            String _elem670;
+            for (int _i671 = 0; _i671 < _list669.size; ++_i671)
             {
-              _elem660 = iprot.readString();
-              struct.part_vals.add(_elem660);
+              _elem670 = iprot.readString();
+              struct.part_vals.add(_elem670);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -56508,13 +56508,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list662 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list662.size);
-                  String _elem663;
-                  for (int _i664 = 0; _i664 < _list662.size; ++_i664)
+                  org.apache.thrift.protocol.TList _list672 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list672.size);
+                  String _elem673;
+                  for (int _i674 = 0; _i674 < _list672.size; ++_i674)
                   {
-                    _elem663 = iprot.readString();
-                    struct.part_vals.add(_elem663);
+                    _elem673 = iprot.readString();
+                    struct.part_vals.add(_elem673);
                   }
                   iprot.readListEnd();
                 }
@@ -56559,9 +56559,9 @@ public class ThriftHiveMetastore {
           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 _iter665 : struct.part_vals)
+            for (String _iter675 : struct.part_vals)
             {
-              oprot.writeString(_iter665);
+              oprot.writeString(_iter675);
             }
             oprot.writeListEnd();
           }
@@ -56612,9 +56612,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter666 : struct.part_vals)
+            for (String _iter676 : struct.part_vals)
             {
-              oprot.writeString(_iter666);
+              oprot.writeString(_iter676);
             }
           }
         }
@@ -56637,13 +56637,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list667 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list667.size);
-            String _elem668;
-            for (int _i669 = 0; _i669 < _list667.size; ++_i669)
+            org.apache.thrift.protocol.TList _list677 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list677.size);
+            String _elem678;
+            for (int _i679 = 0; _i679 < _list677.size; ++_i679)
             {
-              _elem668 = iprot.readString();
-              struct.part_vals.add(_elem668);
+              _elem678 = iprot.readString();
+              struct.part_vals.add(_elem678);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -60513,13 +60513,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list670 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list670.size);
-                  String _elem671;
-                  for (int _i672 = 0; _i672 < _list670.size; ++_i672)
+                  org.apache.thrift.protocol.TList _list680 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list680.size);
+                  String _elem681;
+                  for (int _i682 = 0; _i682 < _list680.size; ++_i682)
                   {
-                    _elem671 = iprot.readString();
-                    struct.part_vals.add(_elem671);
+                    _elem681 = iprot.readString();
+                    struct.part_vals.add(_elem681);
                   }
                   iprot.readListEnd();
                 }
@@ -60563,9 +60563,9 @@ public class ThriftHiveMetastore {
           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 _iter673 : struct.part_vals)
+            for (String _iter683 : struct.part_vals)
             {
-              oprot.writeString(_iter673);
+              oprot.writeString(_iter683);
             }
             oprot.writeListEnd();
           }
@@ -60614,9 +60614,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter674 : struct.part_vals)
+            for (String _iter684 : struct.part_vals)
             {
-              oprot.writeString(_iter674);
+              oprot.writeString(_iter684);
             }
           }
         }
@@ -60639,13 +60639,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list675 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list675.size);
-            String _elem676;
-            for (int _i677 = 0; _i677 < _list675.size; ++_i677)
+            org.apache.thrift.protocol.TList _list685 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list685.size);
+            String _elem686;
+            for (int _i687 = 0; _i687 < _list685.size; ++_i687)
             {
-              _elem676 = iprot.readString();
-              struct.part_vals.add(_elem676);
+              _elem686 = iprot.readString();
+              struct.part_vals.add(_elem686);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -61884,13 +61884,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list678 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list678.size);
-                  String _elem679;
-                  for (int _i680 = 0; _i680 < _list678.size; ++_i680)
+                  org.apache.thrift.protocol.TList _list688 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list688.size);
+                  String _elem689;
+                  for (int _i690 = 0; _i690 < _list688.size; ++_i690)
                   {
-                    _elem679 = iprot.readString();
-                    struct.part_vals.add(_elem679);
+                    _elem689 = iprot.readString();
+                    struct.part_vals.add(_elem689);
                   }
                   iprot.readListEnd();
                 }
@@ -61943,9 +61943,9 @@ public class ThriftHiveMetastore {
           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 _iter681 : struct.part_vals)
+            for (String _iter691 : struct.part_vals)
             {
-              oprot.writeString(_iter681);
+              oprot.writeString(_iter691);
             }
             oprot.writeListEnd();
           }
@@ -62002,9 +62002,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter682 : struct.part_vals)
+            for (String _iter692 : struct.part_vals)
             {
-              oprot.writeString(_iter682);
+              oprot.writeString(_iter692);
             }
           }
         }
@@ -62030,13 +62030,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list683 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list683.size);
-            String _elem684;
-            for (int _i685 = 0; _i685 < _list683.size; ++_i685)
+            org.apache.thrift.protocol.TList _list693 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list693.size);
+            String _elem694;
+            for (int _i695 = 0; _i695 < _list693.size; ++_i695)
             {
-              _elem684 = iprot.readString();
-              struct.part_vals.add(_elem684);
+              _elem694 = iprot.readString();
+              struct.part_vals.add(_elem694);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -66638,13 +66638,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list686 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list686.size);
-                  String _elem687;
-                  for (int _i688 = 0; _i688 < _list686.size; ++_i688)
+                  org.apache.thrift.protocol.TList _list696 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list696.size);
+                  String _elem697;
+                  for (int _i698 = 0; _i698 < _list696.size; ++_i698)
                   {
-                    _elem687 = iprot.readString();
-                    struct.part_vals.add(_elem687);
+                    _elem697 = iprot.readString();
+                    struct.part_vals.add(_elem697);
                   }
                   iprot.readListEnd();
                 }
@@ -66680,9 +66680,9 @@ public class ThriftHiveMetastore {
           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 _iter689 : struct.part_vals)
+            for (String _iter699 : struct.part_vals)
             {
-              oprot.writeString(_iter689);
+              oprot.writeString(_iter699);
             }
             oprot.writeListEnd();
           }
@@ -66725,9 +66725,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter690 : struct.part_vals)
+            for (String _iter700 : struct.part_vals)
             {
-              oprot.writeString(_iter690);
+              oprot.writeString(_iter700);
             }
           }
         }
@@ -66747,13 +66747,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list691 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list691.size);
-            String _elem692;
-            for (int _i693 = 0; _i693 < _list691.size; ++_i693)
+            org.apache.thrift.protocol.TList _list701 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list701.size);
+            String _elem702;
+            for (int _i703 = 0; _i703 < _list701.size; ++_i703)
             {
-              _elem692 = iprot.readString();
-              struct.part_vals.add(_elem692);
+              _elem702 = iprot.readString();
+              struct.part_vals.add(_elem702);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -67971,15 +67971,15 @@ public class ThriftHiveMetastore {
             case 1: // PARTITION_SPECS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map694 = iprot.readMapBegin();
-                  struct.partitionSpecs = new HashMap<String,String>(2*_map694.size);
-                  String _key695;
-                  String _val696;
-                  for (int _i697 = 0; _i697 < _map694.size; ++_i697)
+                  org.apache.thrift.protocol.TMap _map704 = iprot.readMapBegin();
+                  struct.partitionSpecs = new HashMap<String,String>(2*_map704.size);
+                  String _key705;
+                  String _val706;
+                  for (int _i707 = 0; _i707 < _map704.size; ++_i707)
                   {
-                    _key695 = iprot.readString();
-                    _val696 = iprot.readString();
-                    struct.partitionSpecs.put(_key695, _val696);
+                    _key705 = iprot.readString();
+                    _val706 = iprot.readString();
+                    struct.partitionSpecs.put(_key705, _val706);
                   }
                   iprot.readMapEnd();
                 }
@@ -68037,10 +68037,10 @@ public class ThriftHiveMetastore {
           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> _iter698 : struct.partitionSpecs.entrySet())
+            for (Map.Entry<String, String> _iter708 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter698.getKey());
-              oprot.writeString(_iter698.getValue());
+              oprot.writeString(_iter708.getKey());
+              oprot.writeString(_iter708.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -68103,10 +68103,10 @@ public class ThriftHiveMetastore {
         if (struct.isSetPartitionSpecs()) {
           {
             oprot.writeI32(struct.partitionSpecs.size());
-            for (Map.Entry<String, String> _iter699 : struct.partitionSpecs.entrySet())
+            for (Map.Entry<String, String> _iter709 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter699.getKey());
-              oprot.writeString(_iter699.getValue());
+              oprot.writeString(_iter709.getKey());
+              oprot.writeString(_iter709.getValue());
             }
           }
         }
@@ -68130,15 +68130,15 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map700 = 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*_map700.size);
-            String _key701;
-            String _val702;
-            for (int _i703 = 0; _i703 < _map700.size; ++_i703)
+            org.apache.thrift.protocol.TMap _map710 = 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*_map710.size);
+            String _key711;
+            String _val712;
+            for (int _i713 = 0; _i713 < _map710.size; ++_i713)
             {
-              _key701 = iprot.readString();
-              _val702 = iprot.readString();
-              struct.partitionSpecs.put(_key701, _val702);
+              _key711 = iprot.readString();
+              _val712 = iprot.readString();
+              struct.partitionSpecs.put(_key711, _val712);
             }
           }
           struct.setPartitionSpecsIsSet(true);
@@ -69584,15 +69584,15 @@ public class ThriftHiveMetastore {
             case 1: // PARTITION_SPECS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map704 = iprot.readMapBegin();
-                  struct.partitionSpecs = new HashMap<String,String>(2*_map704.size);
-                  String _key705;
-                  String _val706;
-                  for (int _i707 = 0; _i707 < _map704.size; ++_i707)
+                  org.apache.thrift.protocol.TMap _map714 = iprot.readMapBegin();
+                  struct.partitionSpecs = new HashMap<String,String>(2*_map714.size);
+                  String _key715;
+                  String _val716;
+                  for (int _i717 = 0; _i717 < _map714.size; ++_i717)
                   {
-                    _key705 = iprot.readString();
-                    _val706 = iprot.readString();
-                    struct.partitionSpecs.put(_key705, _val706);
+                    _key715 = iprot.readString();
+                    _val716 = iprot.readString();
+                    struct.partitionSpecs.put(_key715, _val716);
                   }
                   iprot.readMapEnd();
                 }
@@ -69650,10 +69650,10 @@ public class ThriftHiveMetastore {
           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> _iter708 : struct.partitionSpecs.entrySet())
+            for (Map.Entry<String, String> _iter718 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter708.getKey());
-              oprot.writeString(_iter708.getValue());
+              oprot.writeString(_iter718.getKey());
+              oprot.writeString(_iter718.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -69716,10 +69716,10 @@ public class ThriftHiveMetastore {
         if (struct.isSetPartitionSpecs()) {
           {
             oprot.writeI32(struct.partitionSpecs.size());
-            for (Map.Entry<String, String> _iter709 : struct.partitionSpecs.entrySet())
+            for (Map.Entry<String, String> _iter719 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter709.getKey());
-              oprot.writeString(_iter709.getValue());
+              oprot.writeString(_iter719.getKey());
+              oprot.writeString(_iter719.getValue());
             }
           }
         }
@@ -69743,15 +69743,15 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map710 = 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*_map710.size);
-            String _key711;
-            String _val712;
-            for (int _i713 = 0; _i713 < _map710.size; ++_i713)
+            org.apache.thrift.protocol.TMap _map720 = 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*_map720.size);
+            String _key721;
+            String _val722;
+            for (int _i723 = 0; _i723 < _map720.size; ++_i723)
             {
-              _key711 = iprot.readString();
-              _val712 = iprot.readString();
-              struct.partitionSpecs.put(_key711, _val712);
+              _key721 = iprot.readString();
+              _val722 = iprot.readString();
+              struct.partitionSpecs.put(_key721, _val722);
             }
           }
           struct.setPartitionSpecsIsSet(true);
@@ -70416,14 +70416,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list714 = iprot.readListBegin();
-                  struct.success = new ArrayList<Partition>(_list714.size);
-                  Partition _elem715;
-                  for (int _i716 = 0; _i716 < _list714.size; ++_i716)
+                  org.apache.thrift.protocol.TList _list724 = iprot.readListBegin();
+                  struct.success = new ArrayList<Partition>(_list724.size);
+                  Partition _elem725;
+                  for (int _i726 = 0; _i726 < _list724.size; ++_i726)
                   {
-                    _elem715 = new Partition();
-                    _elem715.read(iprot);
-                    struct.success.add(_elem715);
+                    _elem725 = new Partition();
+                    _elem725.read(iprot);
+                    struct.success.add(_elem725);
                   }
                   iprot.readListEnd();
                 }
@@ -70485,9 +70485,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Partition _iter717 : struct.success)
+            for (Partition _iter727 : struct.success)
             {
-              _iter717.write(oprot);
+              _iter727.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -70550,9 +70550,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Partition _iter718 : struct.success)
+            for (Partition _iter728 : struct.success)
             {
-              _iter718.write(oprot);
+              _iter728.write(oprot);
             }
           }
         }
@@ -70576,14 +70576,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list719 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Partition>(_list719.size);
-            Partition _elem720;
-            for (int _i721 = 0; _i721 < _list719.size; ++_i721)
+            org.apache.thrift.protocol.TList _list729 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Partition>(_list729.size);
+            Partition _elem730;
+            for (int _i731 = 0; _i731 < _list729.size; ++_i731)
             {
-              _elem720 = new Partition();
-              _elem720.read(iprot);
-              struct.success.add(_elem720);
+              _elem730 = new Partition();
+              _elem730.read(iprot);
+              struct.success.add(_elem730);
             }
           }
           struct.setSuccessIsSet(true);
@@ -71282,13 +71282,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list722 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list722.size);
-                  String _elem723;
-                  for (int _i724 = 0; _i724 < _list722.size; ++_i724)
+                  org.apache.thrift.protocol.TList _list732 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list732.size);
+                  String _elem733;
+                  for (int _i734 = 0; _i734 < _list732.size; ++_i734)
                   {
-                    _elem723 = iprot.readString();
-                    struct.part_vals.add(_elem723);
+                    _elem733 = iprot.readString();
+                    struct.part_vals.add(_elem733);
                   }
                   iprot.readListEnd();
                 }
@@ -71308,13 +71308,13 @@ public class ThriftHiveMetastore {
             case 5: // GROUP_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list725 = iprot.readListBegin();
-                  struct.group_names = new ArrayList<String>(_list725.size);
-                  String _elem726;
-                  for (int _i727 = 0; _i727 < _list725.size; ++_i727)
+                  org.apache.thrift.protocol.TList _list735 = iprot.readListBegin();
+                  struct.group_names = new ArrayList<String>(_list735.size);
+                  String _elem736;
+                  for (int _i737 = 0; _i737 < _list735.size; ++_i737)
                   {
-                    _elem726 = iprot.readString();
-                    struct.group_names.add(_elem726);
+                    _elem736 = iprot.readString();
+                    struct.group_names.add(_elem736);
                   }
                   iprot.readListEnd();
                 }
@@ -71350,9 +71350,9 @@ public class ThriftHiveMetastore {
           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 _iter728 : struct.part_vals)
+            for (String _iter738 : struct.part_vals)
             {
-              oprot.writeString(_iter728);
+              oprot.writeString(_iter738);
             }
             oprot.writeListEnd();
           }
@@ -71367,9 +71367,9 @@ public class ThriftHiveMetastore {
           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 _iter729 : struct.group_names)
+            for (String _iter739 : struct.group_names)
             {
-              oprot.writeString(_iter729);
+              oprot.writeString(_iter739);
             }
             oprot.writeListEnd();
           }
@@ -71418,9 +71418,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter730 : struct.part_vals)
+            for (String _iter740 : struct.part_vals)
             {
-              oprot.writeString(_iter730);
+              oprot.writeString(_iter740);
             }
           }
         }
@@ -71430,9 +71430,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetGroup_names()) {
           {
             oprot.writeI32(struct.group_names.size());
-            for (String _iter731 : struct.group_names)
+            for (String _iter741 : struct.group_names)
             {
-              oprot.writeString(_iter731);
+              oprot.writeString(_iter741);
             }
           }
         }
@@ -71452,13 +71452,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list732 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list732.size);
-            String _elem733;
-            for (int _i734 = 0; _i734 < _list732.size; ++_i734)
+            org.apache.thrift.protocol.TList _list742 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list742.size);
+            String _elem743;
+            for (int _i744 = 0; _i744 < _list742.size; ++_i744)
             {
-              _elem733 = iprot.readString();
-              struct.part_vals.add(_elem733);
+              _elem743 = iprot.readString();
+              struct.part_vals.add(_elem743);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -71469,13 +71469,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol.TList _list735 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.group_names = new ArrayList<String>(_list735.size);
-            String _elem736;
-            for (int _i737 = 0; _i737 < _list735.size; ++_i737)
+            org.apache.thrift.protocol.TList _list745 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.group_names = new ArrayList<String>(_list745.size);
+            String _elem746;
+            for (int _i747 = 0; _i747 < _list745.size; ++_i747)
             {
-              _elem736 = iprot.readString();
-              struct.group_names.add(_elem736);
+              _elem746 = iprot.readString();
+              struct.group_names.add(_elem746);
             }
           }
           struct.setGroup_namesIsSet(true);
@@ -74244,14 +74244,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list738 = iprot.readListBegin();
-                  struct.success = new ArrayList<Partition>(_list738.size);
-                  Partition _elem739;
-                  for (int _i740 = 0; _i740 < _list738.size; ++_i740)
+                  org.apache.thrift.protocol.TList _list748 = iprot.readListBegin();
+                  struct.success = new ArrayList<Partition>(_list748.size);
+                  Partition _elem749;
+                  for (int _i750 = 0; _i750 < _list748.size; ++_i750)
                   {
-                    _elem739 = new Partition();
-                    _elem739.read(iprot);
-                    struct.success.add(_elem739);
+                    _elem749 = new Partition();
+                    _elem749.read(iprot);
+                    struct.success.add(_elem749);
                   }
                   iprot.readListEnd();
                 }
@@ -74295,9 +74295,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Partition _iter741 : struct.success)
+            for (Partition _iter751 : struct.success)
             {
-              _iter741.write(oprot);
+              _iter751.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -74344,9 +74344,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Partition _iter742 : struct.success)
+            for (Partition _iter752 : struct.success)
             {
-              _iter742.write(oprot);
+              _iter752.write(oprot);
             }
           }
         }
@@ -74364,14 +74364,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list743 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Partition>(_list743.size);
-            Partition _elem744;
-            for (int _i745 = 0; _i745 < _list743.size; ++_i745)
+            org.apache.thrift.protocol.TList _list753 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Partition>(_list753.size);
+            Partition _elem754;
+            for (int _i755 = 0; _i755 < _list753.size; ++_i755)
             {
-              _elem744 = new Partition();
-              _elem744.read(iprot);
-              struct.success.add(_elem744);
+              _elem754 = new Partition();
+              _elem754.read(iprot);
+              struct.success.add(_elem754);
             }
           }
           struct.setSuccessIsSet(true);
@@ -75061,13 +75061,13 @@ public class ThriftHiveMetastore {
             case 5: // GROUP_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list746 = iprot.readListBegin();
-                  struct.group_names = new ArrayList<String>(_list746.size);
-                  String _elem747;
-                  for (int _i748 = 0; _i748 < _list746.size; ++_i748)
+                  org.apache.thrift.protocol.TList _list756 = iprot.readListBegin();
+                  struct.group_names = new ArrayList<String>(_list756.size);
+                  String _elem757;
+                  for (int _i758 = 0; _i758 < _list756.size; ++_i758)
                   {
-                    _elem747 = iprot.readString();
-                    struct.group_names.add(_elem747);
+                    _elem757 = iprot.readString();
+                    struct.group_names.add(_elem757);
                   }
                   iprot.readListEnd();
                 }
@@ -75111,9 +75111,9 @@ public class ThriftHiveMetastore {
           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 _iter749 : struct.group_names)
+            for (String _iter759 : struct.group_names)
             {
-              oprot.writeString(_iter749);
+              oprot.writeString(_iter759);
             }
             oprot.writeListEnd();
           }
@@ -75168,9 +75168,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetGroup_names()) {
           {
             oprot.writeI32(struct.group_names.size());
-            for (String _iter750 : struct.group_names)
+            for (String _iter760 : struct.group_names)
             {
-              oprot.writeString(_iter750);
+              oprot.writeString(_iter760);
             }
           }
         }
@@ -75198,13 +75198,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol.TList _list751 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.group_names = new ArrayList<String>(_list751.size);
-            String _elem752;
-            for (int _i753 = 0; _i753 < _list751.size; ++_i753)
+            org.apache.thrift.protocol.TList _list761 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.group_names = new ArrayList<String>(_list761.size);
+            String _elem762;
+            for (int _i763 = 0; _i763 < _list761.size; ++_i763)
             {
-              _elem752 = iprot.readString();
-              struct.group_names.add(_elem752);
+              _elem762 = iprot.readString();
+              struct.group_names.add(_elem762);
             }
           }
           struct.setGroup_namesIsSet(true);
@@ -75691,14 +75691,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list754 = iprot.readListBegin();
-                  struct.success = new ArrayList<Partition>(_list754.size);
-                  Partition _elem755;
-                  for (int _i756 = 0; _i756 < _list754.size; ++_i756)
+                  org.apache.thrift.protocol.TList _list764 = iprot.readListBegin();
+                  struct.success = new ArrayList<Partition>(_list764.size);
+                  Partition _elem765;
+                  for (int _i766 = 0; _i766 < _list764.size; ++_i766)
                   {
-                    _elem755 = new Partition();
-                    _elem755.read(iprot);
-                    struct.success.add(_elem755);
+                    _elem765 = new Partition();
+                    _elem765.read(iprot);
+                    struct.success.add(_elem765);
                   }
                   iprot.readListEnd();
                 }
@@ -75742,9 +75742,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Partition _iter757 : struct.success)
+            for (Partition _iter767 : struct.success)
             {
-              _iter757.write(oprot);
+              _iter767.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -75791,9 +75791,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Partition _iter758 : struct.success)
+            for (Partition _iter768 : struct.success)
             {
-              _iter758.write(oprot);
+              _iter768.write(oprot);
             }
           }
         }
@@ -75811,14 +75811,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list759 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Partition>(_list759.size);
-            Partition _elem760;
-            for (int _i761 = 0; _i761 < _list759.size; ++_i761)
+            org.apache.thrift.protocol.TList _list769 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Partition>(_list769.size);
+            Partition _elem770;
+            for (int _i771 = 0; _i771 < _list769.size; ++_i771)
             {
-              _elem760 = new Partition();
-              _elem760.read(iprot);
-              struct.success.add(_elem760);
+              _elem770 = new Partition();
+              _elem770.read(iprot);
+              struct.success.add(_elem770);
             }
           }
           struct.setSuccessIsSet(true);
@@ -76881,14 +76881,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list762 = iprot.readListBegin();
-                  struct.success = new ArrayList<PartitionSpec>(_list762.size);
-                  PartitionSpec _elem763;
-                  for (int _i764 = 0; _i764 < _list762.size; ++_i764)
+                  org.apache.thrift.protocol.TList _list772 = iprot.readListBegin();
+                  struct.success = new ArrayList<PartitionSpec>(_list772.size);
+                  PartitionSpec _elem773;
+                  for (int _i774 = 0; _i774 < _list772.size; ++_i774)
                   {
-                    _elem763 = new PartitionSpec();
-                    _elem763.read(iprot);
-                    struct.success.add(_elem763);
+                    _elem773 = new PartitionSpec();
+                    _elem773.read(iprot);
+                    struct.success.add(_elem773);
                   }
                   iprot.readListEnd();
                 }
@@ -76932,9 +76932,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (PartitionSpec _iter765 : struct.success)
+            for (PartitionSpec _iter775 : struct.success)
             {
-              _iter765.write(oprot);
+              _iter775.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -76981,9 +76981,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (PartitionSpec _iter766 : struct.success)
+            for (PartitionSpec _iter776 : struct.success)
             {
-              _iter766.write(oprot);
+              _iter776.write(oprot);
             }
           }
         }
@@ -77001,14 +77001,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list767 = new org.apache.thrift.protocol.TList(org.apache.th

<TRUNCATED>

[6/7] hive git commit: HIVE-13354 : Add ability to specify Compaction options per table and per request (Wei Zheng, reviewed by Eugene Koifman)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
index 028c647..c2a208f 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@ -1240,14 +1240,14 @@ uint32_t ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size659;
-            ::apache::thrift::protocol::TType _etype662;
-            xfer += iprot->readListBegin(_etype662, _size659);
-            this->success.resize(_size659);
-            uint32_t _i663;
-            for (_i663 = 0; _i663 < _size659; ++_i663)
+            uint32_t _size667;
+            ::apache::thrift::protocol::TType _etype670;
+            xfer += iprot->readListBegin(_etype670, _size667);
+            this->success.resize(_size667);
+            uint32_t _i671;
+            for (_i671 = 0; _i671 < _size667; ++_i671)
             {
-              xfer += iprot->readString(this->success[_i663]);
+              xfer += iprot->readString(this->success[_i671]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1286,10 +1286,10 @@ uint32_t ThriftHiveMetastore_get_databases_result::write(::apache::thrift::proto
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter664;
-      for (_iter664 = this->success.begin(); _iter664 != this->success.end(); ++_iter664)
+      std::vector<std::string> ::const_iterator _iter672;
+      for (_iter672 = this->success.begin(); _iter672 != this->success.end(); ++_iter672)
       {
-        xfer += oprot->writeString((*_iter664));
+        xfer += oprot->writeString((*_iter672));
       }
       xfer += oprot->writeListEnd();
     }
@@ -1334,14 +1334,14 @@ uint32_t ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size665;
-            ::apache::thrift::protocol::TType _etype668;
-            xfer += iprot->readListBegin(_etype668, _size665);
-            (*(this->success)).resize(_size665);
-            uint32_t _i669;
-            for (_i669 = 0; _i669 < _size665; ++_i669)
+            uint32_t _size673;
+            ::apache::thrift::protocol::TType _etype676;
+            xfer += iprot->readListBegin(_etype676, _size673);
+            (*(this->success)).resize(_size673);
+            uint32_t _i677;
+            for (_i677 = 0; _i677 < _size673; ++_i677)
             {
-              xfer += iprot->readString((*(this->success))[_i669]);
+              xfer += iprot->readString((*(this->success))[_i677]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1458,14 +1458,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size670;
-            ::apache::thrift::protocol::TType _etype673;
-            xfer += iprot->readListBegin(_etype673, _size670);
-            this->success.resize(_size670);
-            uint32_t _i674;
-            for (_i674 = 0; _i674 < _size670; ++_i674)
+            uint32_t _size678;
+            ::apache::thrift::protocol::TType _etype681;
+            xfer += iprot->readListBegin(_etype681, _size678);
+            this->success.resize(_size678);
+            uint32_t _i682;
+            for (_i682 = 0; _i682 < _size678; ++_i682)
             {
-              xfer += iprot->readString(this->success[_i674]);
+              xfer += iprot->readString(this->success[_i682]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1504,10 +1504,10 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::write(::apache::thrift::p
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter675;
-      for (_iter675 = this->success.begin(); _iter675 != this->success.end(); ++_iter675)
+      std::vector<std::string> ::const_iterator _iter683;
+      for (_iter683 = this->success.begin(); _iter683 != this->success.end(); ++_iter683)
       {
-        xfer += oprot->writeString((*_iter675));
+        xfer += oprot->writeString((*_iter683));
       }
       xfer += oprot->writeListEnd();
     }
@@ -1552,14 +1552,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size676;
-            ::apache::thrift::protocol::TType _etype679;
-            xfer += iprot->readListBegin(_etype679, _size676);
-            (*(this->success)).resize(_size676);
-            uint32_t _i680;
-            for (_i680 = 0; _i680 < _size676; ++_i680)
+            uint32_t _size684;
+            ::apache::thrift::protocol::TType _etype687;
+            xfer += iprot->readListBegin(_etype687, _size684);
+            (*(this->success)).resize(_size684);
+            uint32_t _i688;
+            for (_i688 = 0; _i688 < _size684; ++_i688)
             {
-              xfer += iprot->readString((*(this->success))[_i680]);
+              xfer += iprot->readString((*(this->success))[_i688]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2621,17 +2621,17 @@ uint32_t ThriftHiveMetastore_get_type_all_result::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size681;
-            ::apache::thrift::protocol::TType _ktype682;
-            ::apache::thrift::protocol::TType _vtype683;
-            xfer += iprot->readMapBegin(_ktype682, _vtype683, _size681);
-            uint32_t _i685;
-            for (_i685 = 0; _i685 < _size681; ++_i685)
+            uint32_t _size689;
+            ::apache::thrift::protocol::TType _ktype690;
+            ::apache::thrift::protocol::TType _vtype691;
+            xfer += iprot->readMapBegin(_ktype690, _vtype691, _size689);
+            uint32_t _i693;
+            for (_i693 = 0; _i693 < _size689; ++_i693)
             {
-              std::string _key686;
-              xfer += iprot->readString(_key686);
-              Type& _val687 = this->success[_key686];
-              xfer += _val687.read(iprot);
+              std::string _key694;
+              xfer += iprot->readString(_key694);
+              Type& _val695 = this->success[_key694];
+              xfer += _val695.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -2670,11 +2670,11 @@ uint32_t ThriftHiveMetastore_get_type_all_result::write(::apache::thrift::protoc
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::map<std::string, Type> ::const_iterator _iter688;
-      for (_iter688 = this->success.begin(); _iter688 != this->success.end(); ++_iter688)
+      std::map<std::string, Type> ::const_iterator _iter696;
+      for (_iter696 = this->success.begin(); _iter696 != this->success.end(); ++_iter696)
       {
-        xfer += oprot->writeString(_iter688->first);
-        xfer += _iter688->second.write(oprot);
+        xfer += oprot->writeString(_iter696->first);
+        xfer += _iter696->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -2719,17 +2719,17 @@ uint32_t ThriftHiveMetastore_get_type_all_presult::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size689;
-            ::apache::thrift::protocol::TType _ktype690;
-            ::apache::thrift::protocol::TType _vtype691;
-            xfer += iprot->readMapBegin(_ktype690, _vtype691, _size689);
-            uint32_t _i693;
-            for (_i693 = 0; _i693 < _size689; ++_i693)
+            uint32_t _size697;
+            ::apache::thrift::protocol::TType _ktype698;
+            ::apache::thrift::protocol::TType _vtype699;
+            xfer += iprot->readMapBegin(_ktype698, _vtype699, _size697);
+            uint32_t _i701;
+            for (_i701 = 0; _i701 < _size697; ++_i701)
             {
-              std::string _key694;
-              xfer += iprot->readString(_key694);
-              Type& _val695 = (*(this->success))[_key694];
-              xfer += _val695.read(iprot);
+              std::string _key702;
+              xfer += iprot->readString(_key702);
+              Type& _val703 = (*(this->success))[_key702];
+              xfer += _val703.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -2883,14 +2883,14 @@ uint32_t ThriftHiveMetastore_get_fields_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size696;
-            ::apache::thrift::protocol::TType _etype699;
-            xfer += iprot->readListBegin(_etype699, _size696);
-            this->success.resize(_size696);
-            uint32_t _i700;
-            for (_i700 = 0; _i700 < _size696; ++_i700)
+            uint32_t _size704;
+            ::apache::thrift::protocol::TType _etype707;
+            xfer += iprot->readListBegin(_etype707, _size704);
+            this->success.resize(_size704);
+            uint32_t _i708;
+            for (_i708 = 0; _i708 < _size704; ++_i708)
             {
-              xfer += this->success[_i700].read(iprot);
+              xfer += this->success[_i708].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -2945,10 +2945,10 @@ uint32_t ThriftHiveMetastore_get_fields_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter701;
-      for (_iter701 = this->success.begin(); _iter701 != this->success.end(); ++_iter701)
+      std::vector<FieldSchema> ::const_iterator _iter709;
+      for (_iter709 = this->success.begin(); _iter709 != this->success.end(); ++_iter709)
       {
-        xfer += (*_iter701).write(oprot);
+        xfer += (*_iter709).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3001,14 +3001,14 @@ uint32_t ThriftHiveMetastore_get_fields_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size702;
-            ::apache::thrift::protocol::TType _etype705;
-            xfer += iprot->readListBegin(_etype705, _size702);
-            (*(this->success)).resize(_size702);
-            uint32_t _i706;
-            for (_i706 = 0; _i706 < _size702; ++_i706)
+            uint32_t _size710;
+            ::apache::thrift::protocol::TType _etype713;
+            xfer += iprot->readListBegin(_etype713, _size710);
+            (*(this->success)).resize(_size710);
+            uint32_t _i714;
+            for (_i714 = 0; _i714 < _size710; ++_i714)
             {
-              xfer += (*(this->success))[_i706].read(iprot);
+              xfer += (*(this->success))[_i714].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3194,14 +3194,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size707;
-            ::apache::thrift::protocol::TType _etype710;
-            xfer += iprot->readListBegin(_etype710, _size707);
-            this->success.resize(_size707);
-            uint32_t _i711;
-            for (_i711 = 0; _i711 < _size707; ++_i711)
+            uint32_t _size715;
+            ::apache::thrift::protocol::TType _etype718;
+            xfer += iprot->readListBegin(_etype718, _size715);
+            this->success.resize(_size715);
+            uint32_t _i719;
+            for (_i719 = 0; _i719 < _size715; ++_i719)
             {
-              xfer += this->success[_i711].read(iprot);
+              xfer += this->success[_i719].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3256,10 +3256,10 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::write(:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter712;
-      for (_iter712 = this->success.begin(); _iter712 != this->success.end(); ++_iter712)
+      std::vector<FieldSchema> ::const_iterator _iter720;
+      for (_iter720 = this->success.begin(); _iter720 != this->success.end(); ++_iter720)
       {
-        xfer += (*_iter712).write(oprot);
+        xfer += (*_iter720).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3312,14 +3312,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size713;
-            ::apache::thrift::protocol::TType _etype716;
-            xfer += iprot->readListBegin(_etype716, _size713);
-            (*(this->success)).resize(_size713);
-            uint32_t _i717;
-            for (_i717 = 0; _i717 < _size713; ++_i717)
+            uint32_t _size721;
+            ::apache::thrift::protocol::TType _etype724;
+            xfer += iprot->readListBegin(_etype724, _size721);
+            (*(this->success)).resize(_size721);
+            uint32_t _i725;
+            for (_i725 = 0; _i725 < _size721; ++_i725)
             {
-              xfer += (*(this->success))[_i717].read(iprot);
+              xfer += (*(this->success))[_i725].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3489,14 +3489,14 @@ uint32_t ThriftHiveMetastore_get_schema_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size718;
-            ::apache::thrift::protocol::TType _etype721;
-            xfer += iprot->readListBegin(_etype721, _size718);
-            this->success.resize(_size718);
-            uint32_t _i722;
-            for (_i722 = 0; _i722 < _size718; ++_i722)
+            uint32_t _size726;
+            ::apache::thrift::protocol::TType _etype729;
+            xfer += iprot->readListBegin(_etype729, _size726);
+            this->success.resize(_size726);
+            uint32_t _i730;
+            for (_i730 = 0; _i730 < _size726; ++_i730)
             {
-              xfer += this->success[_i722].read(iprot);
+              xfer += this->success[_i730].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3551,10 +3551,10 @@ uint32_t ThriftHiveMetastore_get_schema_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter723;
-      for (_iter723 = this->success.begin(); _iter723 != this->success.end(); ++_iter723)
+      std::vector<FieldSchema> ::const_iterator _iter731;
+      for (_iter731 = this->success.begin(); _iter731 != this->success.end(); ++_iter731)
       {
-        xfer += (*_iter723).write(oprot);
+        xfer += (*_iter731).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3607,14 +3607,14 @@ uint32_t ThriftHiveMetastore_get_schema_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size724;
-            ::apache::thrift::protocol::TType _etype727;
-            xfer += iprot->readListBegin(_etype727, _size724);
-            (*(this->success)).resize(_size724);
-            uint32_t _i728;
-            for (_i728 = 0; _i728 < _size724; ++_i728)
+            uint32_t _size732;
+            ::apache::thrift::protocol::TType _etype735;
+            xfer += iprot->readListBegin(_etype735, _size732);
+            (*(this->success)).resize(_size732);
+            uint32_t _i736;
+            for (_i736 = 0; _i736 < _size732; ++_i736)
             {
-              xfer += (*(this->success))[_i728].read(iprot);
+              xfer += (*(this->success))[_i736].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3800,14 +3800,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size729;
-            ::apache::thrift::protocol::TType _etype732;
-            xfer += iprot->readListBegin(_etype732, _size729);
-            this->success.resize(_size729);
-            uint32_t _i733;
-            for (_i733 = 0; _i733 < _size729; ++_i733)
+            uint32_t _size737;
+            ::apache::thrift::protocol::TType _etype740;
+            xfer += iprot->readListBegin(_etype740, _size737);
+            this->success.resize(_size737);
+            uint32_t _i741;
+            for (_i741 = 0; _i741 < _size737; ++_i741)
             {
-              xfer += this->success[_i733].read(iprot);
+              xfer += this->success[_i741].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3862,10 +3862,10 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::write(:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter734;
-      for (_iter734 = this->success.begin(); _iter734 != this->success.end(); ++_iter734)
+      std::vector<FieldSchema> ::const_iterator _iter742;
+      for (_iter742 = this->success.begin(); _iter742 != this->success.end(); ++_iter742)
       {
-        xfer += (*_iter734).write(oprot);
+        xfer += (*_iter742).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3918,14 +3918,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size735;
-            ::apache::thrift::protocol::TType _etype738;
-            xfer += iprot->readListBegin(_etype738, _size735);
-            (*(this->success)).resize(_size735);
-            uint32_t _i739;
-            for (_i739 = 0; _i739 < _size735; ++_i739)
+            uint32_t _size743;
+            ::apache::thrift::protocol::TType _etype746;
+            xfer += iprot->readListBegin(_etype746, _size743);
+            (*(this->success)).resize(_size743);
+            uint32_t _i747;
+            for (_i747 = 0; _i747 < _size743; ++_i747)
             {
-              xfer += (*(this->success))[_i739].read(iprot);
+              xfer += (*(this->success))[_i747].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5099,14 +5099,14 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size740;
-            ::apache::thrift::protocol::TType _etype743;
-            xfer += iprot->readListBegin(_etype743, _size740);
-            this->success.resize(_size740);
-            uint32_t _i744;
-            for (_i744 = 0; _i744 < _size740; ++_i744)
+            uint32_t _size748;
+            ::apache::thrift::protocol::TType _etype751;
+            xfer += iprot->readListBegin(_etype751, _size748);
+            this->success.resize(_size748);
+            uint32_t _i752;
+            for (_i752 = 0; _i752 < _size748; ++_i752)
             {
-              xfer += iprot->readString(this->success[_i744]);
+              xfer += iprot->readString(this->success[_i752]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5145,10 +5145,10 @@ uint32_t ThriftHiveMetastore_get_tables_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter745;
-      for (_iter745 = this->success.begin(); _iter745 != this->success.end(); ++_iter745)
+      std::vector<std::string> ::const_iterator _iter753;
+      for (_iter753 = this->success.begin(); _iter753 != this->success.end(); ++_iter753)
       {
-        xfer += oprot->writeString((*_iter745));
+        xfer += oprot->writeString((*_iter753));
       }
       xfer += oprot->writeListEnd();
     }
@@ -5193,14 +5193,14 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size746;
-            ::apache::thrift::protocol::TType _etype749;
-            xfer += iprot->readListBegin(_etype749, _size746);
-            (*(this->success)).resize(_size746);
-            uint32_t _i750;
-            for (_i750 = 0; _i750 < _size746; ++_i750)
+            uint32_t _size754;
+            ::apache::thrift::protocol::TType _etype757;
+            xfer += iprot->readListBegin(_etype757, _size754);
+            (*(this->success)).resize(_size754);
+            uint32_t _i758;
+            for (_i758 = 0; _i758 < _size754; ++_i758)
             {
-              xfer += iprot->readString((*(this->success))[_i750]);
+              xfer += iprot->readString((*(this->success))[_i758]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5338,14 +5338,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size751;
-            ::apache::thrift::protocol::TType _etype754;
-            xfer += iprot->readListBegin(_etype754, _size751);
-            this->success.resize(_size751);
-            uint32_t _i755;
-            for (_i755 = 0; _i755 < _size751; ++_i755)
+            uint32_t _size759;
+            ::apache::thrift::protocol::TType _etype762;
+            xfer += iprot->readListBegin(_etype762, _size759);
+            this->success.resize(_size759);
+            uint32_t _i763;
+            for (_i763 = 0; _i763 < _size759; ++_i763)
             {
-              xfer += iprot->readString(this->success[_i755]);
+              xfer += iprot->readString(this->success[_i763]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5384,10 +5384,10 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter756;
-      for (_iter756 = this->success.begin(); _iter756 != this->success.end(); ++_iter756)
+      std::vector<std::string> ::const_iterator _iter764;
+      for (_iter764 = this->success.begin(); _iter764 != this->success.end(); ++_iter764)
       {
-        xfer += oprot->writeString((*_iter756));
+        xfer += oprot->writeString((*_iter764));
       }
       xfer += oprot->writeListEnd();
     }
@@ -5432,14 +5432,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size757;
-            ::apache::thrift::protocol::TType _etype760;
-            xfer += iprot->readListBegin(_etype760, _size757);
-            (*(this->success)).resize(_size757);
-            uint32_t _i761;
-            for (_i761 = 0; _i761 < _size757; ++_i761)
+            uint32_t _size765;
+            ::apache::thrift::protocol::TType _etype768;
+            xfer += iprot->readListBegin(_etype768, _size765);
+            (*(this->success)).resize(_size765);
+            uint32_t _i769;
+            for (_i769 = 0; _i769 < _size765; ++_i769)
             {
-              xfer += iprot->readString((*(this->success))[_i761]);
+              xfer += iprot->readString((*(this->success))[_i769]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5749,14 +5749,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_names.clear();
-            uint32_t _size762;
-            ::apache::thrift::protocol::TType _etype765;
-            xfer += iprot->readListBegin(_etype765, _size762);
-            this->tbl_names.resize(_size762);
-            uint32_t _i766;
-            for (_i766 = 0; _i766 < _size762; ++_i766)
+            uint32_t _size770;
+            ::apache::thrift::protocol::TType _etype773;
+            xfer += iprot->readListBegin(_etype773, _size770);
+            this->tbl_names.resize(_size770);
+            uint32_t _i774;
+            for (_i774 = 0; _i774 < _size770; ++_i774)
             {
-              xfer += iprot->readString(this->tbl_names[_i766]);
+              xfer += iprot->readString(this->tbl_names[_i774]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5789,10 +5789,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::write(::apache::thr
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_names.size()));
-    std::vector<std::string> ::const_iterator _iter767;
-    for (_iter767 = this->tbl_names.begin(); _iter767 != this->tbl_names.end(); ++_iter767)
+    std::vector<std::string> ::const_iterator _iter775;
+    for (_iter775 = this->tbl_names.begin(); _iter775 != this->tbl_names.end(); ++_iter775)
     {
-      xfer += oprot->writeString((*_iter767));
+      xfer += oprot->writeString((*_iter775));
     }
     xfer += oprot->writeListEnd();
   }
@@ -5820,10 +5820,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_pargs::write(::apache::th
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_names)).size()));
-    std::vector<std::string> ::const_iterator _iter768;
-    for (_iter768 = (*(this->tbl_names)).begin(); _iter768 != (*(this->tbl_names)).end(); ++_iter768)
+    std::vector<std::string> ::const_iterator _iter776;
+    for (_iter776 = (*(this->tbl_names)).begin(); _iter776 != (*(this->tbl_names)).end(); ++_iter776)
     {
-      xfer += oprot->writeString((*_iter768));
+      xfer += oprot->writeString((*_iter776));
     }
     xfer += oprot->writeListEnd();
   }
@@ -5864,14 +5864,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size769;
-            ::apache::thrift::protocol::TType _etype772;
-            xfer += iprot->readListBegin(_etype772, _size769);
-            this->success.resize(_size769);
-            uint32_t _i773;
-            for (_i773 = 0; _i773 < _size769; ++_i773)
+            uint32_t _size777;
+            ::apache::thrift::protocol::TType _etype780;
+            xfer += iprot->readListBegin(_etype780, _size777);
+            this->success.resize(_size777);
+            uint32_t _i781;
+            for (_i781 = 0; _i781 < _size777; ++_i781)
             {
-              xfer += this->success[_i773].read(iprot);
+              xfer += this->success[_i781].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5926,10 +5926,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::write(::apache::t
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Table> ::const_iterator _iter774;
-      for (_iter774 = this->success.begin(); _iter774 != this->success.end(); ++_iter774)
+      std::vector<Table> ::const_iterator _iter782;
+      for (_iter782 = this->success.begin(); _iter782 != this->success.end(); ++_iter782)
       {
-        xfer += (*_iter774).write(oprot);
+        xfer += (*_iter782).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -5982,14 +5982,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size775;
-            ::apache::thrift::protocol::TType _etype778;
-            xfer += iprot->readListBegin(_etype778, _size775);
-            (*(this->success)).resize(_size775);
-            uint32_t _i779;
-            for (_i779 = 0; _i779 < _size775; ++_i779)
+            uint32_t _size783;
+            ::apache::thrift::protocol::TType _etype786;
+            xfer += iprot->readListBegin(_etype786, _size783);
+            (*(this->success)).resize(_size783);
+            uint32_t _i787;
+            for (_i787 = 0; _i787 < _size783; ++_i787)
             {
-              xfer += (*(this->success))[_i779].read(iprot);
+              xfer += (*(this->success))[_i787].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6175,14 +6175,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size780;
-            ::apache::thrift::protocol::TType _etype783;
-            xfer += iprot->readListBegin(_etype783, _size780);
-            this->success.resize(_size780);
-            uint32_t _i784;
-            for (_i784 = 0; _i784 < _size780; ++_i784)
+            uint32_t _size788;
+            ::apache::thrift::protocol::TType _etype791;
+            xfer += iprot->readListBegin(_etype791, _size788);
+            this->success.resize(_size788);
+            uint32_t _i792;
+            for (_i792 = 0; _i792 < _size788; ++_i792)
             {
-              xfer += iprot->readString(this->success[_i784]);
+              xfer += iprot->readString(this->success[_i792]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6237,10 +6237,10 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::write(::apache::t
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter785;
-      for (_iter785 = this->success.begin(); _iter785 != this->success.end(); ++_iter785)
+      std::vector<std::string> ::const_iterator _iter793;
+      for (_iter793 = this->success.begin(); _iter793 != this->success.end(); ++_iter793)
       {
-        xfer += oprot->writeString((*_iter785));
+        xfer += oprot->writeString((*_iter793));
       }
       xfer += oprot->writeListEnd();
     }
@@ -6293,14 +6293,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size786;
-            ::apache::thrift::protocol::TType _etype789;
-            xfer += iprot->readListBegin(_etype789, _size786);
-            (*(this->success)).resize(_size786);
-            uint32_t _i790;
-            for (_i790 = 0; _i790 < _size786; ++_i790)
+            uint32_t _size794;
+            ::apache::thrift::protocol::TType _etype797;
+            xfer += iprot->readListBegin(_etype797, _size794);
+            (*(this->success)).resize(_size794);
+            uint32_t _i798;
+            for (_i798 = 0; _i798 < _size794; ++_i798)
             {
-              xfer += iprot->readString((*(this->success))[_i790]);
+              xfer += iprot->readString((*(this->success))[_i798]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7634,14 +7634,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size791;
-            ::apache::thrift::protocol::TType _etype794;
-            xfer += iprot->readListBegin(_etype794, _size791);
-            this->new_parts.resize(_size791);
-            uint32_t _i795;
-            for (_i795 = 0; _i795 < _size791; ++_i795)
+            uint32_t _size799;
+            ::apache::thrift::protocol::TType _etype802;
+            xfer += iprot->readListBegin(_etype802, _size799);
+            this->new_parts.resize(_size799);
+            uint32_t _i803;
+            for (_i803 = 0; _i803 < _size799; ++_i803)
             {
-              xfer += this->new_parts[_i795].read(iprot);
+              xfer += this->new_parts[_i803].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7670,10 +7670,10 @@ uint32_t ThriftHiveMetastore_add_partitions_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->new_parts.size()));
-    std::vector<Partition> ::const_iterator _iter796;
-    for (_iter796 = this->new_parts.begin(); _iter796 != this->new_parts.end(); ++_iter796)
+    std::vector<Partition> ::const_iterator _iter804;
+    for (_iter804 = this->new_parts.begin(); _iter804 != this->new_parts.end(); ++_iter804)
     {
-      xfer += (*_iter796).write(oprot);
+      xfer += (*_iter804).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7697,10 +7697,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->new_parts)).size()));
-    std::vector<Partition> ::const_iterator _iter797;
-    for (_iter797 = (*(this->new_parts)).begin(); _iter797 != (*(this->new_parts)).end(); ++_iter797)
+    std::vector<Partition> ::const_iterator _iter805;
+    for (_iter805 = (*(this->new_parts)).begin(); _iter805 != (*(this->new_parts)).end(); ++_iter805)
     {
-      xfer += (*_iter797).write(oprot);
+      xfer += (*_iter805).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7909,14 +7909,14 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_args::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size798;
-            ::apache::thrift::protocol::TType _etype801;
-            xfer += iprot->readListBegin(_etype801, _size798);
-            this->new_parts.resize(_size798);
-            uint32_t _i802;
-            for (_i802 = 0; _i802 < _size798; ++_i802)
+            uint32_t _size806;
+            ::apache::thrift::protocol::TType _etype809;
+            xfer += iprot->readListBegin(_etype809, _size806);
+            this->new_parts.resize(_size806);
+            uint32_t _i810;
+            for (_i810 = 0; _i810 < _size806; ++_i810)
             {
-              xfer += this->new_parts[_i802].read(iprot);
+              xfer += this->new_parts[_i810].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7945,10 +7945,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_args::write(::apache::thrift::
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->new_parts.size()));
-    std::vector<PartitionSpec> ::const_iterator _iter803;
-    for (_iter803 = this->new_parts.begin(); _iter803 != this->new_parts.end(); ++_iter803)
+    std::vector<PartitionSpec> ::const_iterator _iter811;
+    for (_iter811 = this->new_parts.begin(); _iter811 != this->new_parts.end(); ++_iter811)
     {
-      xfer += (*_iter803).write(oprot);
+      xfer += (*_iter811).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7972,10 +7972,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_pargs::write(::apache::thrift:
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->new_parts)).size()));
-    std::vector<PartitionSpec> ::const_iterator _iter804;
-    for (_iter804 = (*(this->new_parts)).begin(); _iter804 != (*(this->new_parts)).end(); ++_iter804)
+    std::vector<PartitionSpec> ::const_iterator _iter812;
+    for (_iter812 = (*(this->new_parts)).begin(); _iter812 != (*(this->new_parts)).end(); ++_iter812)
     {
-      xfer += (*_iter804).write(oprot);
+      xfer += (*_iter812).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8200,14 +8200,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size805;
-            ::apache::thrift::protocol::TType _etype808;
-            xfer += iprot->readListBegin(_etype808, _size805);
-            this->part_vals.resize(_size805);
-            uint32_t _i809;
-            for (_i809 = 0; _i809 < _size805; ++_i809)
+            uint32_t _size813;
+            ::apache::thrift::protocol::TType _etype816;
+            xfer += iprot->readListBegin(_etype816, _size813);
+            this->part_vals.resize(_size813);
+            uint32_t _i817;
+            for (_i817 = 0; _i817 < _size813; ++_i817)
             {
-              xfer += iprot->readString(this->part_vals[_i809]);
+              xfer += iprot->readString(this->part_vals[_i817]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8244,10 +8244,10 @@ uint32_t ThriftHiveMetastore_append_partition_args::write(::apache::thrift::prot
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter810;
-    for (_iter810 = this->part_vals.begin(); _iter810 != this->part_vals.end(); ++_iter810)
+    std::vector<std::string> ::const_iterator _iter818;
+    for (_iter818 = this->part_vals.begin(); _iter818 != this->part_vals.end(); ++_iter818)
     {
-      xfer += oprot->writeString((*_iter810));
+      xfer += oprot->writeString((*_iter818));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8279,10 +8279,10 @@ uint32_t ThriftHiveMetastore_append_partition_pargs::write(::apache::thrift::pro
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter811;
-    for (_iter811 = (*(this->part_vals)).begin(); _iter811 != (*(this->part_vals)).end(); ++_iter811)
+    std::vector<std::string> ::const_iterator _iter819;
+    for (_iter819 = (*(this->part_vals)).begin(); _iter819 != (*(this->part_vals)).end(); ++_iter819)
     {
-      xfer += oprot->writeString((*_iter811));
+      xfer += oprot->writeString((*_iter819));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8754,14 +8754,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size812;
-            ::apache::thrift::protocol::TType _etype815;
-            xfer += iprot->readListBegin(_etype815, _size812);
-            this->part_vals.resize(_size812);
-            uint32_t _i816;
-            for (_i816 = 0; _i816 < _size812; ++_i816)
+            uint32_t _size820;
+            ::apache::thrift::protocol::TType _etype823;
+            xfer += iprot->readListBegin(_etype823, _size820);
+            this->part_vals.resize(_size820);
+            uint32_t _i824;
+            for (_i824 = 0; _i824 < _size820; ++_i824)
             {
-              xfer += iprot->readString(this->part_vals[_i816]);
+              xfer += iprot->readString(this->part_vals[_i824]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8806,10 +8806,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::wri
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter817;
-    for (_iter817 = this->part_vals.begin(); _iter817 != this->part_vals.end(); ++_iter817)
+    std::vector<std::string> ::const_iterator _iter825;
+    for (_iter825 = this->part_vals.begin(); _iter825 != this->part_vals.end(); ++_iter825)
     {
-      xfer += oprot->writeString((*_iter817));
+      xfer += oprot->writeString((*_iter825));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8845,10 +8845,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_pargs::wr
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter818;
-    for (_iter818 = (*(this->part_vals)).begin(); _iter818 != (*(this->part_vals)).end(); ++_iter818)
+    std::vector<std::string> ::const_iterator _iter826;
+    for (_iter826 = (*(this->part_vals)).begin(); _iter826 != (*(this->part_vals)).end(); ++_iter826)
     {
-      xfer += oprot->writeString((*_iter818));
+      xfer += oprot->writeString((*_iter826));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9651,14 +9651,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size819;
-            ::apache::thrift::protocol::TType _etype822;
-            xfer += iprot->readListBegin(_etype822, _size819);
-            this->part_vals.resize(_size819);
-            uint32_t _i823;
-            for (_i823 = 0; _i823 < _size819; ++_i823)
+            uint32_t _size827;
+            ::apache::thrift::protocol::TType _etype830;
+            xfer += iprot->readListBegin(_etype830, _size827);
+            this->part_vals.resize(_size827);
+            uint32_t _i831;
+            for (_i831 = 0; _i831 < _size827; ++_i831)
             {
-              xfer += iprot->readString(this->part_vals[_i823]);
+              xfer += iprot->readString(this->part_vals[_i831]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9703,10 +9703,10 @@ uint32_t ThriftHiveMetastore_drop_partition_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter824;
-    for (_iter824 = this->part_vals.begin(); _iter824 != this->part_vals.end(); ++_iter824)
+    std::vector<std::string> ::const_iterator _iter832;
+    for (_iter832 = this->part_vals.begin(); _iter832 != this->part_vals.end(); ++_iter832)
     {
-      xfer += oprot->writeString((*_iter824));
+      xfer += oprot->writeString((*_iter832));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9742,10 +9742,10 @@ uint32_t ThriftHiveMetastore_drop_partition_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter825;
-    for (_iter825 = (*(this->part_vals)).begin(); _iter825 != (*(this->part_vals)).end(); ++_iter825)
+    std::vector<std::string> ::const_iterator _iter833;
+    for (_iter833 = (*(this->part_vals)).begin(); _iter833 != (*(this->part_vals)).end(); ++_iter833)
     {
-      xfer += oprot->writeString((*_iter825));
+      xfer += oprot->writeString((*_iter833));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9954,14 +9954,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size826;
-            ::apache::thrift::protocol::TType _etype829;
-            xfer += iprot->readListBegin(_etype829, _size826);
-            this->part_vals.resize(_size826);
-            uint32_t _i830;
-            for (_i830 = 0; _i830 < _size826; ++_i830)
+            uint32_t _size834;
+            ::apache::thrift::protocol::TType _etype837;
+            xfer += iprot->readListBegin(_etype837, _size834);
+            this->part_vals.resize(_size834);
+            uint32_t _i838;
+            for (_i838 = 0; _i838 < _size834; ++_i838)
             {
-              xfer += iprot->readString(this->part_vals[_i830]);
+              xfer += iprot->readString(this->part_vals[_i838]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10014,10 +10014,10 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::write
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter831;
-    for (_iter831 = this->part_vals.begin(); _iter831 != this->part_vals.end(); ++_iter831)
+    std::vector<std::string> ::const_iterator _iter839;
+    for (_iter839 = this->part_vals.begin(); _iter839 != this->part_vals.end(); ++_iter839)
     {
-      xfer += oprot->writeString((*_iter831));
+      xfer += oprot->writeString((*_iter839));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10057,10 +10057,10 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_pargs::writ
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter832;
-    for (_iter832 = (*(this->part_vals)).begin(); _iter832 != (*(this->part_vals)).end(); ++_iter832)
+    std::vector<std::string> ::const_iterator _iter840;
+    for (_iter840 = (*(this->part_vals)).begin(); _iter840 != (*(this->part_vals)).end(); ++_iter840)
     {
-      xfer += oprot->writeString((*_iter832));
+      xfer += oprot->writeString((*_iter840));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11066,14 +11066,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size833;
-            ::apache::thrift::protocol::TType _etype836;
-            xfer += iprot->readListBegin(_etype836, _size833);
-            this->part_vals.resize(_size833);
-            uint32_t _i837;
-            for (_i837 = 0; _i837 < _size833; ++_i837)
+            uint32_t _size841;
+            ::apache::thrift::protocol::TType _etype844;
+            xfer += iprot->readListBegin(_etype844, _size841);
+            this->part_vals.resize(_size841);
+            uint32_t _i845;
+            for (_i845 = 0; _i845 < _size841; ++_i845)
             {
-              xfer += iprot->readString(this->part_vals[_i837]);
+              xfer += iprot->readString(this->part_vals[_i845]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11110,10 +11110,10 @@ uint32_t ThriftHiveMetastore_get_partition_args::write(::apache::thrift::protoco
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter838;
-    for (_iter838 = this->part_vals.begin(); _iter838 != this->part_vals.end(); ++_iter838)
+    std::vector<std::string> ::const_iterator _iter846;
+    for (_iter846 = this->part_vals.begin(); _iter846 != this->part_vals.end(); ++_iter846)
     {
-      xfer += oprot->writeString((*_iter838));
+      xfer += oprot->writeString((*_iter846));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11145,10 +11145,10 @@ uint32_t ThriftHiveMetastore_get_partition_pargs::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter839;
-    for (_iter839 = (*(this->part_vals)).begin(); _iter839 != (*(this->part_vals)).end(); ++_iter839)
+    std::vector<std::string> ::const_iterator _iter847;
+    for (_iter847 = (*(this->part_vals)).begin(); _iter847 != (*(this->part_vals)).end(); ++_iter847)
     {
-      xfer += oprot->writeString((*_iter839));
+      xfer += oprot->writeString((*_iter847));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11337,17 +11337,17 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size840;
-            ::apache::thrift::protocol::TType _ktype841;
-            ::apache::thrift::protocol::TType _vtype842;
-            xfer += iprot->readMapBegin(_ktype841, _vtype842, _size840);
-            uint32_t _i844;
-            for (_i844 = 0; _i844 < _size840; ++_i844)
+            uint32_t _size848;
+            ::apache::thrift::protocol::TType _ktype849;
+            ::apache::thrift::protocol::TType _vtype850;
+            xfer += iprot->readMapBegin(_ktype849, _vtype850, _size848);
+            uint32_t _i852;
+            for (_i852 = 0; _i852 < _size848; ++_i852)
             {
-              std::string _key845;
-              xfer += iprot->readString(_key845);
-              std::string& _val846 = this->partitionSpecs[_key845];
-              xfer += iprot->readString(_val846);
+              std::string _key853;
+              xfer += iprot->readString(_key853);
+              std::string& _val854 = this->partitionSpecs[_key853];
+              xfer += iprot->readString(_val854);
             }
             xfer += iprot->readMapEnd();
           }
@@ -11408,11 +11408,11 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::write(::apache::thrift::pr
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionSpecs.size()));
-    std::map<std::string, std::string> ::const_iterator _iter847;
-    for (_iter847 = this->partitionSpecs.begin(); _iter847 != this->partitionSpecs.end(); ++_iter847)
+    std::map<std::string, std::string> ::const_iterator _iter855;
+    for (_iter855 = this->partitionSpecs.begin(); _iter855 != this->partitionSpecs.end(); ++_iter855)
     {
-      xfer += oprot->writeString(_iter847->first);
-      xfer += oprot->writeString(_iter847->second);
+      xfer += oprot->writeString(_iter855->first);
+      xfer += oprot->writeString(_iter855->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -11452,11 +11452,11 @@ uint32_t ThriftHiveMetastore_exchange_partition_pargs::write(::apache::thrift::p
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partitionSpecs)).size()));
-    std::map<std::string, std::string> ::const_iterator _iter848;
-    for (_iter848 = (*(this->partitionSpecs)).begin(); _iter848 != (*(this->partitionSpecs)).end(); ++_iter848)
+    std::map<std::string, std::string> ::const_iterator _iter856;
+    for (_iter856 = (*(this->partitionSpecs)).begin(); _iter856 != (*(this->partitionSpecs)).end(); ++_iter856)
     {
-      xfer += oprot->writeString(_iter848->first);
-      xfer += oprot->writeString(_iter848->second);
+      xfer += oprot->writeString(_iter856->first);
+      xfer += oprot->writeString(_iter856->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -11701,17 +11701,17 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size849;
-            ::apache::thrift::protocol::TType _ktype850;
-            ::apache::thrift::protocol::TType _vtype851;
-            xfer += iprot->readMapBegin(_ktype850, _vtype851, _size849);
-            uint32_t _i853;
-            for (_i853 = 0; _i853 < _size849; ++_i853)
+            uint32_t _size857;
+            ::apache::thrift::protocol::TType _ktype858;
+            ::apache::thrift::protocol::TType _vtype859;
+            xfer += iprot->readMapBegin(_ktype858, _vtype859, _size857);
+            uint32_t _i861;
+            for (_i861 = 0; _i861 < _size857; ++_i861)
             {
-              std::string _key854;
-              xfer += iprot->readString(_key854);
-              std::string& _val855 = this->partitionSpecs[_key854];
-              xfer += iprot->readString(_val855);
+              std::string _key862;
+              xfer += iprot->readString(_key862);
+              std::string& _val863 = this->partitionSpecs[_key862];
+              xfer += iprot->readString(_val863);
             }
             xfer += iprot->readMapEnd();
           }
@@ -11772,11 +11772,11 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::write(::apache::thrift::p
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionSpecs.size()));
-    std::map<std::string, std::string> ::const_iterator _iter856;
-    for (_iter856 = this->partitionSpecs.begin(); _iter856 != this->partitionSpecs.end(); ++_iter856)
+    std::map<std::string, std::string> ::const_iterator _iter864;
+    for (_iter864 = this->partitionSpecs.begin(); _iter864 != this->partitionSpecs.end(); ++_iter864)
     {
-      xfer += oprot->writeString(_iter856->first);
-      xfer += oprot->writeString(_iter856->second);
+      xfer += oprot->writeString(_iter864->first);
+      xfer += oprot->writeString(_iter864->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -11816,11 +11816,11 @@ uint32_t ThriftHiveMetastore_exchange_partitions_pargs::write(::apache::thrift::
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partitionSpecs)).size()));
-    std::map<std::string, std::string> ::const_iterator _iter857;
-    for (_iter857 = (*(this->partitionSpecs)).begin(); _iter857 != (*(this->partitionSpecs)).end(); ++_iter857)
+    std::map<std::string, std::string> ::const_iterator _iter865;
+    for (_iter865 = (*(this->partitionSpecs)).begin(); _iter865 != (*(this->partitionSpecs)).end(); ++_iter865)
     {
-      xfer += oprot->writeString(_iter857->first);
-      xfer += oprot->writeString(_iter857->second);
+      xfer += oprot->writeString(_iter865->first);
+      xfer += oprot->writeString(_iter865->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -11877,14 +11877,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size858;
-            ::apache::thrift::protocol::TType _etype861;
-            xfer += iprot->readListBegin(_etype861, _size858);
-            this->success.resize(_size858);
-            uint32_t _i862;
-            for (_i862 = 0; _i862 < _size858; ++_i862)
+            uint32_t _size866;
+            ::apache::thrift::protocol::TType _etype869;
+            xfer += iprot->readListBegin(_etype869, _size866);
+            this->success.resize(_size866);
+            uint32_t _i870;
+            for (_i870 = 0; _i870 < _size866; ++_i870)
             {
-              xfer += this->success[_i862].read(iprot);
+              xfer += this->success[_i870].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -11947,10 +11947,10 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::write(::apache::thrift:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter863;
-      for (_iter863 = this->success.begin(); _iter863 != this->success.end(); ++_iter863)
+      std::vector<Partition> ::const_iterator _iter871;
+      for (_iter871 = this->success.begin(); _iter871 != this->success.end(); ++_iter871)
       {
-        xfer += (*_iter863).write(oprot);
+        xfer += (*_iter871).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -12007,14 +12007,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_presult::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size864;
-            ::apache::thrift::protocol::TType _etype867;
-            xfer += iprot->readListBegin(_etype867, _size864);
-            (*(this->success)).resize(_size864);
-            uint32_t _i868;
-            for (_i868 = 0; _i868 < _size864; ++_i868)
+            uint32_t _size872;
+            ::apache::thrift::protocol::TType _etype875;
+            xfer += iprot->readListBegin(_etype875, _size872);
+            (*(this->success)).resize(_size872);
+            uint32_t _i876;
+            for (_i876 = 0; _i876 < _size872; ++_i876)
             {
-              xfer += (*(this->success))[_i868].read(iprot);
+              xfer += (*(this->success))[_i876].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12113,14 +12113,14 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size869;
-            ::apache::thrift::protocol::TType _etype872;
-            xfer += iprot->readListBegin(_etype872, _size869);
-            this->part_vals.resize(_size869);
-            uint32_t _i873;
-            for (_i873 = 0; _i873 < _size869; ++_i873)
+            uint32_t _size877;
+            ::apache::thrift::protocol::TType _etype880;
+            xfer += iprot->readListBegin(_etype880, _size877);
+            this->part_vals.resize(_size877);
+            uint32_t _i881;
+            for (_i881 = 0; _i881 < _size877; ++_i881)
             {
-              xfer += iprot->readString(this->part_vals[_i873]);
+              xfer += iprot->readString(this->part_vals[_i881]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12141,14 +12141,14 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size874;
-            ::apache::thrift::protocol::TType _etype877;
-            xfer += iprot->readListBegin(_etype877, _size874);
-            this->group_names.resize(_size874);
-            uint32_t _i878;
-            for (_i878 = 0; _i878 < _size874; ++_i878)
+            uint32_t _size882;
+            ::apache::thrift::protocol::TType _etype885;
+            xfer += iprot->readListBegin(_etype885, _size882);
+            this->group_names.resize(_size882);
+            uint32_t _i886;
+            for (_i886 = 0; _i886 < _size882; ++_i886)
             {
-              xfer += iprot->readString(this->group_names[_i878]);
+              xfer += iprot->readString(this->group_names[_i886]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12185,10 +12185,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrif
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter879;
-    for (_iter879 = this->part_vals.begin(); _iter879 != this->part_vals.end(); ++_iter879)
+    std::vector<std::string> ::const_iterator _iter887;
+    for (_iter887 = this->part_vals.begin(); _iter887 != this->part_vals.end(); ++_iter887)
     {
-      xfer += oprot->writeString((*_iter879));
+      xfer += oprot->writeString((*_iter887));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12201,10 +12201,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrif
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter880;
-    for (_iter880 = this->group_names.begin(); _iter880 != this->group_names.end(); ++_iter880)
+    std::vector<std::string> ::const_iterator _iter888;
+    for (_iter888 = this->group_names.begin(); _iter888 != this->group_names.end(); ++_iter888)
     {
-      xfer += oprot->writeString((*_iter880));
+      xfer += oprot->writeString((*_iter888));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12236,10 +12236,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thri
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter881;
-    for (_iter881 = (*(this->part_vals)).begin(); _iter881 != (*(this->part_vals)).end(); ++_iter881)
+    std::vector<std::string> ::const_iterator _iter889;
+    for (_iter889 = (*(this->part_vals)).begin(); _iter889 != (*(this->part_vals)).end(); ++_iter889)
     {
-      xfer += oprot->writeString((*_iter881));
+      xfer += oprot->writeString((*_iter889));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12252,10 +12252,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thri
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter882;
-    for (_iter882 = (*(this->group_names)).begin(); _iter882 != (*(this->group_names)).end(); ++_iter882)
+    std::vector<std::string> ::const_iterator _iter890;
+    for (_iter890 = (*(this->group_names)).begin(); _iter890 != (*(this->group_names)).end(); ++_iter890)
     {
-      xfer += oprot->writeString((*_iter882));
+      xfer += oprot->writeString((*_iter890));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12814,14 +12814,14 @@ uint32_t ThriftHiveMetastore_get_partitions_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size883;
-            ::apache::thrift::protocol::TType _etype886;
-            xfer += iprot->readListBegin(_etype886, _size883);
-            this->success.resize(_size883);
-            uint32_t _i887;
-            for (_i887 = 0; _i887 < _size883; ++_i887)
+            uint32_t _size891;
+            ::apache::thrift::protocol::TType _etype894;
+            xfer += iprot->readListBegin(_etype894, _size891);
+            this->success.resize(_size891);
+            uint32_t _i895;
+            for (_i895 = 0; _i895 < _size891; ++_i895)
             {
-              xfer += this->success[_i887].read(iprot);
+              xfer += this->success[_i895].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12868,10 +12868,10 @@ uint32_t ThriftHiveMetastore_get_partitions_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter888;
-      for (_iter888 = this->success.begin(); _iter888 != this->success.end(); ++_iter888)
+      std::vector<Partition> ::const_iterator _iter896;
+      for (_iter896 = this->success.begin(); _iter896 != this->success.end(); ++_iter896)
       {
-        xfer += (*_iter888).write(oprot);
+        xfer += (*_iter896).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -12920,14 +12920,14 @@ uint32_t ThriftHiveMetastore_get_partitions_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size889;
-            ::apache::thrift::protocol::TType _etype892;
-            xfer += iprot->readListBegin(_etype892, _size889);
-            (*(this->success)).resize(_size889);
-            uint32_t _i893;
-            for (_i893 = 0; _i893 < _size889; ++_i893)
+            uint32_t _size897;
+            ::apache::thrift::protocol::TType _etype900;
+            xfer += iprot->readListBegin(_etype900, _size897);
+            (*(this->success)).resize(_size897);
+            uint32_t _i901;
+            for (_i901 = 0; _i901 < _size897; ++_i901)
             {
-              xfer += (*(this->success))[_i893].read(iprot);
+              xfer += (*(this->success))[_i901].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13026,14 +13026,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::read(::apache::thrif
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size894;
-            ::apache::thrift::protocol::TType _etype897;
-            xfer += iprot->readListBegin(_etype897, _size894);
-            this->group_names.resize(_size894);
-            uint32_t _i898;
-            for (_i898 = 0; _i898 < _size894; ++_i898)
+            uint32_t _size902;
+            ::apache::thrift::protocol::TType _etype905;
+            xfer += iprot->readListBegin(_etype905, _size902);
+            this->group_names.resize(_size902);
+            uint32_t _i906;
+            for (_i906 = 0; _i906 < _size902; ++_i906)
             {
-              xfer += iprot->readString(this->group_names[_i898]);
+              xfer += iprot->readString(this->group_names[_i906]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13078,10 +13078,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::write(::apache::thri
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter899;
-    for (_iter899 = this->group_names.begin(); _iter899 != this->group_names.end(); ++_iter899)
+    std::vector<std::string> ::const_iterator _iter907;
+    for (_iter907 = this->group_names.begin(); _iter907 != this->group_names.end(); ++_iter907)
     {
-      xfer += oprot->writeString((*_iter899));
+      xfer += oprot->writeString((*_iter907));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13121,10 +13121,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_pargs::write(::apache::thr
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter900;
-    for (_iter900 = (*(this->group_names)).begin(); _iter900 != (*(this->group_names)).end(); ++_iter900)
+    std::vector<std::string> ::const_iterator _iter908;
+    for (_iter908 = (*(this->group_names)).begin(); _iter908 != (*(this->group_names)).end(); ++_iter908)
     {
-      xfer += oprot->writeString((*_iter900));
+      xfer += oprot->writeString((*_iter908));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13165,14 +13165,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size901;
-            ::apache::thrift::protocol::TType _etype904;
-            xfer += iprot->readListBegin(_etype904, _size901);
-            this->success.resize(_size901);
-            uint32_t _i905;
-            for (_i905 = 0; _i905 < _size901; ++_i905)
+            uint32_t _size909;
+            ::apache::thrift::protocol::TType _etype912;
+            xfer += iprot->readListBegin(_etype912, _size909);
+            this->success.resize(_size909);
+            uint32_t _i913;
+            for (_i913 = 0; _i913 < _size909; ++_i913)
             {
-              xfer += this->success[_i905].read(iprot);
+              xfer += this->success[_i913].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13219,10 +13219,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::write(::apache::th
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter906;
-      for (_iter906 = this->success.begin(); _iter906 != this->success.end(); ++_iter906)
+      std::vector<Partition> ::const_iterator _iter914;
+      for (_iter914 = this->success.begin(); _iter914 != this->success.end(); ++_iter914)
       {
-        xfer += (*_iter906).write(oprot);
+        xfer += (*_iter914).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -13271,14 +13271,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_presult::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size907;
-            ::apache::thrift::protocol::TType _etype910;
-            xfer += iprot->readListBegin(_etype910, _size907);
-            (*(this->success)).resize(_size907);
-            uint32_t _i911;
-            for (_i911 = 0; _i911 < _size907; ++_i911)
+            uint32_t _size915;
+            ::apache::thrift::protocol::TType _etype918;
+            xfer += iprot->readListBegin(_etype918, _size915);
+            (*(this->success)).resize(_size915);
+            uint32_t _i919;
+            for (_i919 = 0; _i919 < _size915; ++_i919)
             {
-              xfer += (*(this->success))[_i911].read(iprot);
+              xfer += (*(this->success))[_i919].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13456,14 +13456,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size912;
-            ::apache::thrift::protocol::TType _etype915;
-            xfer += iprot->readListBegin(_etype915, _size912);
-            this->success.resize(_size912);
-            uint32_t _i916;
-            for (_i916 = 0; _i916 < _size912; ++_i916)
+            uint32_t _size920;
+            ::apache::thrift::protocol::TType _etype923;
+            xfer += iprot->readListBegin(_etype923, _size920);
+            this->success.resize(_size920);
+            uint32_t _i924;
+            for (_i924 = 0; _i924 < _size920; ++_i924)
             {
-              xfer += this->success[_i916].read(iprot);
+              xfer += this->success[_i924].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13510,10 +13510,10 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::write(::apache::thrift
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<PartitionSpec> ::const_iterator _iter917;
-      for (_iter917 = this->success.begin(); _iter917 != this->success.end(); ++_iter917)
+      std::vector<PartitionSpec> ::const_iterator _iter925;
+      for (_iter925 = this->success.begin(); _iter925 != this->success.end(); ++_iter925)
       {
-        xfer += (*_iter917).write(oprot);
+        xfer += (*_iter925).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -13562,14 +13562,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_presult::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size918;
-            ::apache::thrift::protocol::TType _etype921;
-            xfer += iprot->readListBegin(_etype921, _size918);
-            (*(this->success)).resize(_size918);
-            uint32_t _i922;
-            for (_i922 = 0; _i922 < _size918; ++_i922)
+            uint32_t _size926;
+            ::apache::thrift::protocol::TType _etype929;
+            xfer += iprot->readListBegin(_etype929, _size926);
+            (*(this->success)).resize(_size926);
+            uint32_t _i930;
+            for (_i930 = 0; _i930 < _size926; ++_i930)
             {
-              xfer += (*(this->success))[_i922].read(iprot);
+              xfer += (*(this->success))[_i930].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13747,14 +13747,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size923;
-            ::apache::thrift::protocol::TType _etype926;
-            xfer += iprot->readListBegin(_etype926, _size923);
-            this->success.resize(_size923);
-            uint32_t _i927;
-            for (_i927 = 0; _i927 < _size923; ++_i927)
+            uint32_t _size931;
+            ::apache::thrift::protocol::TType _etype934;
+            xfer += iprot->readListBegin(_etype934, _size931);
+            this->success.resize(_size931);
+            uint32_t _i935;
+            for (_i935 = 0; _i935 < _size931; ++_i935)
             {
-              xfer += iprot->readString(this->success[_i927]);
+              xfer += iprot->readString(this->success[_i935]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13793,10 +13793,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::write(::apache::thrift:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter928;
-      for (_iter928 = this->success.begin(); _iter928 != this->success.end(); ++_iter928)
+      std::vector<std::string> ::const_iterator _iter936;
+      for (_iter936 = this->success.begin(); _iter936 != this->success.end(); ++_iter936)
       {
-        xfer += oprot->writeString((*_iter928));
+        xfer += oprot->writeString((*_iter936));
       }
       xfer += oprot->writeListEnd();
     }
@@ -13841,14 +13841,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_presult::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size929;
-            ::apache::thrift::protocol::TType _etype932;
-            xfer += iprot->readListBegin(_etype932, _size929);
-            (*(this->success)).resize(_size929);
-            uint32_t _i933;
-            for (_i933 = 0; _i933 < _size929; ++_i933)
+            uint32_t _size937;
+            ::apache::thrift::protocol::TType _etype940;
+            xfer += iprot->readListBegin(_etype940, _size937);
+            (*(this->success)).resize(_size937);
+            uint32_t _i941;
+            for (_i941 = 0; _i941 < _size937; ++_i941)
             {
-              xfer += iprot->readString((*(this->success))[_i933]);
+              xfer += iprot->readString((*(this->success))[_i941]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13923,14 +13923,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_args::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size934;
-            ::apache::thrift::protocol::TType _etype937;
-            xfer += iprot->readListBegin(_etype937, _size934);
-            this->part_vals.resize(_size934);
-            uint32_t _i938;
-            for (_i938 = 0; _i938 < _size934; ++_i938)
+            uint32_t _size942;
+            ::apache::thrift::protocol::TType _etype945;
+            xfer += iprot->readListBegin(_etype945, _size942);
+            this->part_vals.resize(_size942);
+            uint32_t _i946;
+            for (_i946 = 0; _i946 < _size942; ++_i946)
             {
-              xfer += iprot->readString(this->part_vals[_i938]);
+              xfer += iprot->readString(this->part_vals[_i946]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13975,10 +13975,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_args::write(::apache::thrift::pro
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter939;
-    for (_iter939 = this->part_vals.begin(); _iter939 != this->part_vals.end(); ++_iter939)
+    std::vector<std::string> ::const_iterator _iter947;
+    for (_iter947 = this->part_vals.begin(); _iter947 != this->part_vals.end(); ++_iter947)
     {
-      xfer += oprot->writeString((*_iter939));
+      xfer += oprot->writeString((*_iter947));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14014,10 +14014,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_pargs::write(::apache::thrift::pr
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter940;
-    for (_iter940 = (*(this->part_vals)).begin(); _iter940 != (*(this->part_vals)).end(); ++_iter940)
+    std::vector<std::string> ::const_iterator _iter948;
+    for (_iter948 = (*(this->part_vals)).begin(); _iter948 != (*(this->part_vals)).end(); ++_iter948)
     {
-      xfer += oprot->writeString((*_iter940));
+      xfer += oprot->writeString((*_iter948));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14062,14 +14062,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size941;
-            ::apache::thrift::protocol::TType _etype944;
-            xfer += iprot->readListBegin(_etype944, _size941);
-            this->success.resize(_size941);
-            uint32_t _i945;
-            for (_i945 = 0; _i945 < _size941; ++_i945)
+            uint32_t _size949;
+            ::apache::thrift::protocol::TType _etype952;
+            xfer += iprot->readListBegin(_etype952, _size949);
+            this->success.resize(_size949);
+            uint32_t _i953;
+            for (_i953 = 0; _i953 < _size949; ++_i953)
             {
-              xfer += this->success[_i945].read(iprot);
+              xfer += this->success[_i953].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14116,10 +14116,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_result::write(::apache::thrift::p
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter946;
-      for (_iter946 = this->success.begin(); _iter946 != this->success.end(); ++_iter946)
+      std::vector<Partition> ::const_iterator _iter954;
+      for (_iter954 = this->success.begin(); _iter954 != this->success.end(); ++_iter954)
       {
-        xfer += (*_iter946).write(oprot);
+        xfer += (*_iter954).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -14168,14 +14168,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size947;
-            ::apache::thrift::protocol::TType _etype950;
-            xfer += iprot->readListBegin(_etype950, _size947);
-            (*(this->success)).resize(_size947);
-            uint32_t _i951;
-            for (_i951 = 0; _i951 < _size947; ++_i951)
+            uint32_t _size955;
+            ::apache::thrift::protocol::TType _etype958;
+            xfer += iprot->readListBegin(_etype958, _size955);
+            (*(this->success)).resize(_size955);
+            uint32_t _i959;
+            for (_i959 = 0; _i959 < _size955; ++_i959)
             {
-              xfer += (*(this->success))[_i951].read(iprot);
+              xfer += (*(this->success))[_i959].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14258,14 +14258,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size952;
-            ::apache::thrift::protocol::TType _etype955;
-            xfer += iprot->readListBegin(_etype955, _size952);
-            this->part_vals.resize(_size952);
-            uint32_t _i956;
-            for (_i956 = 0; _i956 < _size952; ++_i956)
+            uint32_t _size960;
+            ::apache::thrift::protocol::TType _etype963;
+            xfer += iprot->readListBegin(_etype963, _size960);
+            this->part_vals.resize(_size960);
+            uint32_t _i964;
+            for (_i964 = 0; _i964 < _size960; ++_i964)
             {
-              xfer += iprot->readString(this->part_vals[_i956]);
+              xfer += iprot->readString(this->part_vals[_i964]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14294,14 +14294,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size957;
-            ::apache::thrift::protocol::TType _etype960;
-            xfer += iprot->readListBegin(_etype960, _size957);
-            this->group_names.resize(_size957);
-            uint32_t _i961;
-            for (_i961 = 0; _i961 < _size957; ++_i961)
+            uint32_t _size965;
+            ::apache::thrift::protocol::TType _etype968;
+            xfer += iprot->readListBegin(_etype968, _size965);
+            this->group_names.resize(_size965);
+            uint32_t _i969;
+            for (_i969 = 0; _i969 < _size965; ++_i969)
             {
-              xfer += iprot->readString(this->group_names[_i961]);
+              xfer += iprot->readString(this->group_names[_i969]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14338,10 +14338,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::write(::apache::t
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter962;
-    for (_iter962 = this->part_vals.begin(); _iter962 != this->part_vals.end(); ++_iter962)
+    std::vector<std::string> ::const_iterator _iter970;
+    for (_iter970 = this->part_vals.begin(); _iter970 != this->part_vals.end(); ++_iter970)
     {
-      xfer += oprot->writeString((*_iter962));
+      xfer += oprot->writeString((*_iter970));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14358,10 +14358,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::write(::apache::t
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 6);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter963;
-    for (_iter963 = this->group_names.begin(); _iter963 != this->group_names.end(); ++_iter963)
+    std::vector<std::string> ::const_iterator _iter971;
+    for (_iter971 = this->group_names.begin(); _iter971 != this->group_names.end(); ++_iter971)
     {
-      xfer += oprot->writeString((*_iter963));
+      xfer += oprot->writeString((*_iter971));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14393,10 +14393,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_pargs

<TRUNCATED>

[7/7] hive git commit: HIVE-13354 : Add ability to specify Compaction options per table and per request (Wei Zheng, reviewed by Eugene Koifman)

Posted by we...@apache.org.
HIVE-13354 : Add ability to specify Compaction options per table and per request (Wei Zheng, reviewed by Eugene Koifman)


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

Branch: refs/heads/branch-1
Commit: 6e0504d9a8552471f7137a513abe2a0a15c124de
Parents: abaf882
Author: Wei Zheng <we...@apache.org>
Authored: Fri May 27 15:13:53 2016 -0700
Committer: Wei Zheng <we...@apache.org>
Committed: Fri May 27 15:13:53 2016 -0700

----------------------------------------------------------------------
 .../hive/ql/txn/compactor/TestCompactor.java    |  164 ++
 metastore/if/hive_metastore.thrift              |    1 +
 .../upgrade/derby/036-HIVE-13354.derby.sql      |    2 +
 .../derby/hive-txn-schema-1.3.0.derby.sql       |    2 +
 .../derby/upgrade-1.2.0-to-1.3.0.derby.sql      |    1 +
 .../upgrade/mssql/021-HIVE-13354.mssql.sql      |    2 +
 .../upgrade/mssql/hive-schema-1.3.0.mssql.sql   |    2 +
 .../mssql/upgrade-1.2.0-to-1.3.0.mssql.sql      |    1 +
 .../upgrade/mysql/036-HIVE-13354.mysql.sql      |    2 +
 .../mysql/hive-txn-schema-1.3.0.mysql.sql       |    2 +
 .../mysql/upgrade-1.2.0-to-1.3.0.mysql.sql      |    1 +
 .../upgrade/oracle/036-HIVE-13354.oracle.sql    |    2 +
 .../oracle/hive-txn-schema-1.3.0.oracle.sql     |    2 +
 .../oracle/upgrade-1.2.0-to-1.3.0.oracle.sql    |    1 +
 .../postgres/035-HIVE-13354.postgres.sql        |    2 +
 .../postgres/hive-txn-schema-1.3.0.postgres.sql |    2 +
 .../upgrade-1.2.0-to-1.3.0.postgres.sql         |    1 +
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 1814 ++++++++--------
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp |  640 +++---
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |   10 +-
 .../metastore/api/AddDynamicPartitions.java     |   32 +-
 .../hive/metastore/api/CompactionRequest.java   |  166 +-
 .../hive/metastore/api/FireEventRequest.java    |   32 +-
 .../metastore/api/GetAllFunctionsResponse.java  |   36 +-
 .../metastore/api/InsertEventRequestData.java   |   32 +-
 .../api/NotificationEventResponse.java          |   36 +-
 .../hive/metastore/api/ShowCompactResponse.java |   36 +-
 .../hive/metastore/api/ThriftHiveMetastore.java | 1948 +++++++++---------
 .../gen-php/metastore/ThriftHiveMetastore.php   | 1098 +++++-----
 .../src/gen/thrift/gen-php/metastore/Types.php  |  171 +-
 .../hive_metastore/ThriftHiveMetastore.py       |  738 +++----
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |  103 +-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |    4 +-
 .../hive/metastore/HiveMetaStoreClient.java     |   14 +
 .../hadoop/hive/metastore/IMetaStoreClient.java |   18 +
 .../hive/metastore/txn/CompactionInfo.java      |   30 +-
 .../metastore/txn/CompactionTxnHandler.java     |   11 +-
 .../hadoop/hive/metastore/txn/TxnDbUtil.java    |    2 +
 .../hadoop/hive/metastore/txn/TxnHandler.java   |    8 +
 .../hadoop/hive/metastore/txn/TxnUtils.java     |   54 +
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |    2 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |    6 +-
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |    5 +
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |    4 +-
 .../hive/ql/plan/AlterTableSimpleDesc.java      |    8 +
 .../hive/ql/txn/compactor/CompactorMR.java      |  121 +-
 .../hadoop/hive/ql/txn/compactor/Initiator.java |   22 +-
 .../hadoop/hive/ql/txn/compactor/Worker.java    |    9 +
 .../hive/ql/txn/compactor/TestWorker.java       |    9 +-
 49 files changed, 4016 insertions(+), 3393 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java
index 37bbab8..9c8bcc1 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java
@@ -14,6 +14,7 @@ import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 import org.apache.hadoop.hive.metastore.api.CompactionRequest;
 import org.apache.hadoop.hive.metastore.api.CompactionType;
 import org.apache.hadoop.hive.metastore.api.LongColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.ShowCompactRequest;
 import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
 import org.apache.hadoop.hive.metastore.api.ShowCompactResponseElement;
@@ -34,6 +35,7 @@ import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
 import org.apache.hadoop.hive.ql.io.orc.OrcStruct;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.mapred.JobConf;
 import org.apache.hive.hcatalog.common.HCatUtil;
 import org.apache.hive.hcatalog.streaming.DelimitedInputWriter;
 import org.apache.hive.hcatalog.streaming.HiveEndPoint;
@@ -61,6 +63,7 @@ import java.util.SortedSet;
 import java.util.TreeSet;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.TimeUnit;
 
 /**
  */
@@ -834,6 +837,137 @@ public class TestCompactor {
       connection.close();
     }
   }
+
+  /**
+   * Users have the choice of specifying compaction related tblproperties either in CREATE TABLE
+   * statement or in ALTER TABLE .. COMPACT statement. This tests both cases.
+   * @throws Exception
+   */
+  @Test
+  public void testTableProperties() throws Exception {
+    String tblName1 = "ttp1"; // plain acid table
+    String tblName2 = "ttp2"; // acid table with customized tblproperties
+    executeStatementOnDriver("drop table if exists " + tblName1, driver);
+    executeStatementOnDriver("drop table if exists " + tblName2, driver);
+    executeStatementOnDriver("CREATE TABLE " + tblName1 + "(a INT, b STRING) " +
+        " CLUSTERED BY(a) INTO 2 BUCKETS STORED AS ORC TBLPROPERTIES ('transactional'='true')", driver);
+    executeStatementOnDriver("CREATE TABLE " + tblName2 + "(a INT, b STRING) " +
+        " CLUSTERED BY(a) INTO 2 BUCKETS STORED AS ORC TBLPROPERTIES (" +
+        "'transactional'='true'," +
+        "'compactor.mapreduce.map.memory.mb'='2048'," + // 2048 MB memory for compaction map job
+        "'compactorthreshold.hive.compactor.delta.num.threshold'='4'," +  // minor compaction if more than 4 delta dirs
+        "'compactorthreshold.hive.compactor.delta.pct.threshold'='0.5'" + // major compaction if more than 50%
+        ")", driver);
+
+    // Insert 5 rows to both tables
+    executeStatementOnDriver("insert into " + tblName1 + " values (1, 'a')", driver);
+    executeStatementOnDriver("insert into " + tblName1 + " values (2, 'b')", driver);
+    executeStatementOnDriver("insert into " + tblName1 + " values (3, 'c')", driver);
+    executeStatementOnDriver("insert into " + tblName1 + " values (4, 'd')", driver);
+    executeStatementOnDriver("insert into " + tblName1 + " values (5, 'e')", driver);
+
+    executeStatementOnDriver("insert into " + tblName2 + " values (1, 'a')", driver);
+    executeStatementOnDriver("insert into " + tblName2 + " values (2, 'b')", driver);
+    executeStatementOnDriver("insert into " + tblName2 + " values (3, 'c')", driver);
+    executeStatementOnDriver("insert into " + tblName2 + " values (4, 'd')", driver);
+    executeStatementOnDriver("insert into " + tblName2 + " values (5, 'e')", driver);
+
+    runInitiator(conf);
+
+    // Compactor should only schedule compaction for ttp2 (delta.num.threshold=4), not ttp1
+    TxnStore txnHandler = TxnUtils.getTxnStore(conf);
+    ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
+    Assert.assertEquals(1, rsp.getCompacts().size());
+    Assert.assertEquals(TxnStore.INITIATED_RESPONSE, rsp.getCompacts().get(0).getState());
+    Assert.assertEquals("ttp2", rsp.getCompacts().get(0).getTablename());
+    Assert.assertEquals(CompactionType.MAJOR, rsp.getCompacts().get(0).getType()); // type is MAJOR since there's no base yet
+
+    // Finish the scheduled compaction for ttp2, and manually compact ttp1, to make them comparable again
+    executeStatementOnDriver("alter table " + tblName1 + " compact 'major'", driver);
+    rsp = txnHandler.showCompact(new ShowCompactRequest());
+    Assert.assertEquals(2, rsp.getCompacts().size());
+    Assert.assertEquals("ttp2", rsp.getCompacts().get(0).getTablename());
+    Assert.assertEquals(TxnStore.INITIATED_RESPONSE, rsp.getCompacts().get(0).getState());
+    Assert.assertEquals("ttp1", rsp.getCompacts().get(1).getTablename());
+    Assert.assertEquals(TxnStore.INITIATED_RESPONSE, rsp.getCompacts().get(1).getState());
+    // compact ttp2, by running the Worker explicitly, in order to get the reference to the compactor MR job
+    AtomicBoolean stop = new AtomicBoolean(true);
+    Worker t = new Worker();
+    t.setThreadId((int) t.getId());
+    t.setHiveConf(conf);
+    AtomicBoolean looped = new AtomicBoolean();
+    t.init(stop, looped);
+    t.run();
+    JobConf job = t.getMrJob();
+    Assert.assertEquals("2048", job.get("mapreduce.map.memory.mb"));  // 2048 comes from tblproperties
+    // Compact ttp1
+    stop = new AtomicBoolean(true);
+    t = new Worker();
+    t.setThreadId((int) t.getId());
+    t.setHiveConf(conf);
+    looped = new AtomicBoolean();
+    t.init(stop, looped);
+    t.run();
+    job = t.getMrJob();
+    Assert.assertEquals("1024", job.get("mapreduce.map.memory.mb"));  // 1024 is the default value
+    // Clean up
+    runCleaner(conf);
+    rsp = txnHandler.showCompact(new ShowCompactRequest());
+    Assert.assertEquals(2, rsp.getCompacts().size());
+    Assert.assertEquals("ttp2", rsp.getCompacts().get(0).getTablename());
+    Assert.assertEquals(TxnStore.SUCCEEDED_RESPONSE, rsp.getCompacts().get(0).getState());
+    Assert.assertEquals("ttp1", rsp.getCompacts().get(1).getTablename());
+    Assert.assertEquals(TxnStore.SUCCEEDED_RESPONSE, rsp.getCompacts().get(1).getState());
+
+    // Insert one more row - this should trigger hive.compactor.delta.pct.threshold to be reached for ttp2
+    executeStatementOnDriver("insert into " + tblName1 + " values (6, 'f')", driver);
+    executeStatementOnDriver("insert into " + tblName2 + " values (6, 'f')", driver);
+
+    // Intentionally set this high so that it will not trigger major compaction for ttp1.
+    // Only trigger major compaction for ttp2 (delta.pct.threshold=0.5) because of the newly inserted row (actual pct: 0.66)
+    conf.setFloatVar(HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_PCT_THRESHOLD, 0.8f);
+    runInitiator(conf);
+    rsp = txnHandler.showCompact(new ShowCompactRequest());
+    Assert.assertEquals(4, rsp.getCompacts().size());
+    Assert.assertEquals("ttp1", rsp.getCompacts().get(0).getTablename());
+    Assert.assertEquals(TxnStore.INITIATED_RESPONSE, rsp.getCompacts().get(0).getState());
+    Assert.assertEquals("ttp2", rsp.getCompacts().get(1).getTablename());
+    Assert.assertEquals(TxnStore.INITIATED_RESPONSE, rsp.getCompacts().get(1).getState());
+
+    // Finish the scheduled compaction for ttp2
+    runWorker(conf);
+    runWorker(conf);
+    runCleaner(conf);
+    rsp = txnHandler.showCompact(new ShowCompactRequest());
+    Assert.assertEquals(4, rsp.getCompacts().size());
+    Assert.assertEquals("ttp2", rsp.getCompacts().get(0).getTablename());
+    Assert.assertEquals(TxnStore.SUCCEEDED_RESPONSE, rsp.getCompacts().get(0).getState());
+
+    // Now test tblproperties specified on ALTER TABLE .. COMPACT .. statement
+    executeStatementOnDriver("insert into " + tblName2 + " values (7, 'g')", driver);
+    executeStatementOnDriver("alter table " + tblName2 + " compact 'major'" +
+        " with overwrite tblproperties (" +
+        "'compactor.mapreduce.map.memory.mb'='3072'," +
+        "'tblprops.orc.compress.size'='8192')", driver);
+
+    rsp = txnHandler.showCompact(new ShowCompactRequest());
+    Assert.assertEquals(5, rsp.getCompacts().size());
+    Assert.assertEquals("ttp2", rsp.getCompacts().get(0).getTablename());
+    Assert.assertEquals(TxnStore.INITIATED_RESPONSE, rsp.getCompacts().get(0).getState());
+
+    // Run the Worker explicitly, in order to get the reference to the compactor MR job
+    stop = new AtomicBoolean(true);
+    t = new Worker();
+    t.setThreadId((int) t.getId());
+    t.setHiveConf(conf);
+    looped = new AtomicBoolean();
+    t.init(stop, looped);
+    t.run();
+    job = t.getMrJob();
+    Assert.assertEquals("3072", job.get("mapreduce.map.memory.mb"));
+    Assert.assertTrue(job.get("hive.compactor.table.props").contains("orc.compress.size4:8192"));
+  }
+
   private void writeBatch(StreamingConnection connection, DelimitedInputWriter writer,
                           boolean closeEarly)
       throws InterruptedException, StreamingException {
@@ -957,4 +1091,34 @@ public class TestCompactor {
     }
 
   }
+
+  static void runInitiator(HiveConf hiveConf) throws MetaException {
+    AtomicBoolean stop = new AtomicBoolean(true);
+    Initiator t = new Initiator();
+    t.setThreadId((int) t.getId());
+    t.setHiveConf(hiveConf);
+    AtomicBoolean looped = new AtomicBoolean();
+    t.init(stop, looped);
+    t.run();
+  }
+
+  static void runWorker(HiveConf hiveConf) throws MetaException {
+    AtomicBoolean stop = new AtomicBoolean(true);
+    Worker t = new Worker();
+    t.setThreadId((int) t.getId());
+    t.setHiveConf(hiveConf);
+    AtomicBoolean looped = new AtomicBoolean();
+    t.init(stop, looped);
+    t.run();
+  }
+
+  static void runCleaner(HiveConf hiveConf) throws MetaException {
+    AtomicBoolean stop = new AtomicBoolean(true);
+    Cleaner t = new Cleaner();
+    t.setThreadId((int) t.getId());
+    t.setHiveConf(hiveConf);
+    AtomicBoolean looped = new AtomicBoolean();
+    t.init(stop, looped);
+    t.run();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/if/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/metastore/if/hive_metastore.thrift b/metastore/if/hive_metastore.thrift
index f3d1d42..c7b6fd8 100755
--- a/metastore/if/hive_metastore.thrift
+++ b/metastore/if/hive_metastore.thrift
@@ -674,6 +674,7 @@ struct CompactionRequest {
     3: optional string partitionname,
     4: required CompactionType type,
     5: optional string runas,
+    6: optional map<string, string> properties
 }
 
 struct ShowCompactRequest {

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/scripts/upgrade/derby/036-HIVE-13354.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/036-HIVE-13354.derby.sql b/metastore/scripts/upgrade/derby/036-HIVE-13354.derby.sql
new file mode 100644
index 0000000..2f691b1
--- /dev/null
+++ b/metastore/scripts/upgrade/derby/036-HIVE-13354.derby.sql
@@ -0,0 +1,2 @@
+ALTER TABLE COMPACTION_QUEUE ADD CQ_TBLPROPERTIES varchar(2048);
+ALTER TABLE COMPLETED_COMPACTIONS ADD CC_TBLPROPERTIES varchar(2048);

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/scripts/upgrade/derby/hive-txn-schema-1.3.0.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/hive-txn-schema-1.3.0.derby.sql b/metastore/scripts/upgrade/derby/hive-txn-schema-1.3.0.derby.sql
index 480c19e..634dd73 100644
--- a/metastore/scripts/upgrade/derby/hive-txn-schema-1.3.0.derby.sql
+++ b/metastore/scripts/upgrade/derby/hive-txn-schema-1.3.0.derby.sql
@@ -82,6 +82,7 @@ CREATE TABLE COMPACTION_QUEUE (
   CQ_PARTITION varchar(767),
   CQ_STATE char(1) NOT NULL,
   CQ_TYPE char(1) NOT NULL,
+  CQ_TBLPROPERTIES varchar(2048),
   CQ_WORKER_ID varchar(128),
   CQ_START bigint,
   CQ_RUN_AS varchar(128),
@@ -102,6 +103,7 @@ CREATE TABLE COMPLETED_COMPACTIONS (
   CC_PARTITION varchar(767),
   CC_STATE char(1) NOT NULL,
   CC_TYPE char(1) NOT NULL,
+  CC_TBLPROPERTIES varchar(2048),
   CC_WORKER_ID varchar(128),
   CC_START bigint,
   CC_END bigint,

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/scripts/upgrade/derby/upgrade-1.2.0-to-1.3.0.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/upgrade-1.2.0-to-1.3.0.derby.sql b/metastore/scripts/upgrade/derby/upgrade-1.2.0-to-1.3.0.derby.sql
index 1b9e171..d4484bd 100644
--- a/metastore/scripts/upgrade/derby/upgrade-1.2.0-to-1.3.0.derby.sql
+++ b/metastore/scripts/upgrade/derby/upgrade-1.2.0-to-1.3.0.derby.sql
@@ -11,5 +11,6 @@ RUN '030-HIVE-12823.derby.sql';
 RUN '031-HIVE-12831.derby.sql';
 RUN '032-HIVE-12832.derby.sql';
 RUN '035-HIVE-13395.derby.sql';
+RUN '036-HIVE-13354.derby.sql';
 
 UPDATE "APP".VERSION SET SCHEMA_VERSION='1.3.0', VERSION_COMMENT='Hive release version 1.3.0' where VER_ID=1;

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/scripts/upgrade/mssql/021-HIVE-13354.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/021-HIVE-13354.mssql.sql b/metastore/scripts/upgrade/mssql/021-HIVE-13354.mssql.sql
new file mode 100644
index 0000000..518b142
--- /dev/null
+++ b/metastore/scripts/upgrade/mssql/021-HIVE-13354.mssql.sql
@@ -0,0 +1,2 @@
+ALTER TABLE COMPACTION_QUEUE ADD CQ_TBLPROPERTIES nvarchar(2048) NULL;
+ALTER TABLE COMPLETED_COMPACTIONS ADD CC_TBLPROPERTIES nvarchar(2048) NULL;

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/scripts/upgrade/mssql/hive-schema-1.3.0.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/hive-schema-1.3.0.mssql.sql b/metastore/scripts/upgrade/mssql/hive-schema-1.3.0.mssql.sql
index 7e0e24f..931edfc 100644
--- a/metastore/scripts/upgrade/mssql/hive-schema-1.3.0.mssql.sql
+++ b/metastore/scripts/upgrade/mssql/hive-schema-1.3.0.mssql.sql
@@ -863,6 +863,7 @@ CREATE TABLE COMPACTION_QUEUE(
 	CQ_PARTITION nvarchar(767) NULL,
 	CQ_STATE char(1) NOT NULL,
 	CQ_TYPE char(1) NOT NULL,
+	CQ_TBLPROPERTIES nvarchar(2048) NULL,
 	CQ_WORKER_ID nvarchar(128) NULL,
 	CQ_START bigint NULL,
 	CQ_RUN_AS nvarchar(128) NULL,
@@ -882,6 +883,7 @@ CREATE TABLE COMPLETED_COMPACTIONS (
 	CC_PARTITION nvarchar(767) NULL,
 	CC_STATE char(1) NOT NULL,
 	CC_TYPE char(1) NOT NULL,
+	CC_TBLPROPERTIES nvarchar(2048) NULL,
 	CC_WORKER_ID nvarchar(128) NULL,
 	CC_START bigint NULL,
 	CC_END bigint NULL,

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/scripts/upgrade/mssql/upgrade-1.2.0-to-1.3.0.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/upgrade-1.2.0-to-1.3.0.mssql.sql b/metastore/scripts/upgrade/mssql/upgrade-1.2.0-to-1.3.0.mssql.sql
index 18da152..32782eb 100644
--- a/metastore/scripts/upgrade/mssql/upgrade-1.2.0-to-1.3.0.mssql.sql
+++ b/metastore/scripts/upgrade/mssql/upgrade-1.2.0-to-1.3.0.mssql.sql
@@ -11,6 +11,7 @@ SELECT 'Upgrading MetaStore schema from 1.2.0 to 1.3.0' AS MESSAGE;
 :r 016-HIVE-12831.mssql.sql;
 :r 017-HIVE-12832.mssql.sql;
 :r 020-HIVE-13395.mssql.sql;
+:r 021-HIVE-13354.mssql.sql
 
 UPDATE VERSION SET SCHEMA_VERSION='1.3.0', VERSION_COMMENT='Hive release version 1.3.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 1.2.0 to 1.3.0' AS MESSAGE;

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/scripts/upgrade/mysql/036-HIVE-13354.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/036-HIVE-13354.mysql.sql b/metastore/scripts/upgrade/mysql/036-HIVE-13354.mysql.sql
new file mode 100644
index 0000000..2f691b1
--- /dev/null
+++ b/metastore/scripts/upgrade/mysql/036-HIVE-13354.mysql.sql
@@ -0,0 +1,2 @@
+ALTER TABLE COMPACTION_QUEUE ADD CQ_TBLPROPERTIES varchar(2048);
+ALTER TABLE COMPLETED_COMPACTIONS ADD CC_TBLPROPERTIES varchar(2048);

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/scripts/upgrade/mysql/hive-txn-schema-1.3.0.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/hive-txn-schema-1.3.0.mysql.sql b/metastore/scripts/upgrade/mysql/hive-txn-schema-1.3.0.mysql.sql
index e852fc9..21b6c9b 100644
--- a/metastore/scripts/upgrade/mysql/hive-txn-schema-1.3.0.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/hive-txn-schema-1.3.0.mysql.sql
@@ -85,6 +85,7 @@ CREATE TABLE COMPACTION_QUEUE (
   CQ_PARTITION varchar(767),
   CQ_STATE char(1) NOT NULL,
   CQ_TYPE char(1) NOT NULL,
+  CQ_TBLPROPERTIES varchar(2048),
   CQ_WORKER_ID varchar(128),
   CQ_START bigint,
   CQ_RUN_AS varchar(128),
@@ -100,6 +101,7 @@ CREATE TABLE COMPLETED_COMPACTIONS (
   CC_PARTITION varchar(767),
   CC_STATE char(1) NOT NULL,
   CC_TYPE char(1) NOT NULL,
+  CC_TBLPROPERTIES varchar(2048),
   CC_WORKER_ID varchar(128),
   CC_START bigint,
   CC_END bigint,

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/scripts/upgrade/mysql/upgrade-1.2.0-to-1.3.0.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/upgrade-1.2.0-to-1.3.0.mysql.sql b/metastore/scripts/upgrade/mysql/upgrade-1.2.0-to-1.3.0.mysql.sql
index 021b802..284f366 100644
--- a/metastore/scripts/upgrade/mysql/upgrade-1.2.0-to-1.3.0.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/upgrade-1.2.0-to-1.3.0.mysql.sql
@@ -12,6 +12,7 @@ SOURCE 030-HIVE-12823.mysql.sql;
 SOURCE 031-HIVE-12831.mysql.sql;
 SOURCE 032-HIVE-12832.mysql.sql;
 SOURCE 035-HIVE-13395.mysql.sql;
+SOURCE 036-HIVE-13354.mysql.sql;
 
 UPDATE VERSION SET SCHEMA_VERSION='1.3.0', VERSION_COMMENT='Hive release version 1.3.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 1.2.0 to 1.3.0' AS ' ';

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/scripts/upgrade/oracle/036-HIVE-13354.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/036-HIVE-13354.oracle.sql b/metastore/scripts/upgrade/oracle/036-HIVE-13354.oracle.sql
new file mode 100644
index 0000000..2f691b1
--- /dev/null
+++ b/metastore/scripts/upgrade/oracle/036-HIVE-13354.oracle.sql
@@ -0,0 +1,2 @@
+ALTER TABLE COMPACTION_QUEUE ADD CQ_TBLPROPERTIES varchar(2048);
+ALTER TABLE COMPLETED_COMPACTIONS ADD CC_TBLPROPERTIES varchar(2048);

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/scripts/upgrade/oracle/hive-txn-schema-1.3.0.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/hive-txn-schema-1.3.0.oracle.sql b/metastore/scripts/upgrade/oracle/hive-txn-schema-1.3.0.oracle.sql
index 199ff4c..89f6374 100644
--- a/metastore/scripts/upgrade/oracle/hive-txn-schema-1.3.0.oracle.sql
+++ b/metastore/scripts/upgrade/oracle/hive-txn-schema-1.3.0.oracle.sql
@@ -83,6 +83,7 @@ CREATE TABLE COMPACTION_QUEUE (
   CQ_PARTITION varchar(767),
   CQ_STATE char(1) NOT NULL,
   CQ_TYPE char(1) NOT NULL,
+  CQ_TBLPROPERTIES varchar(2048),
   CQ_WORKER_ID varchar(128),
   CQ_START NUMBER(19),
   CQ_RUN_AS varchar(128),
@@ -103,6 +104,7 @@ CREATE TABLE COMPLETED_COMPACTIONS (
   CC_PARTITION varchar(767),
   CC_STATE char(1) NOT NULL,
   CC_TYPE char(1) NOT NULL,
+  CC_TBLPROPERTIES varchar(2048),
   CC_WORKER_ID varchar(128),
   CC_START NUMBER(19),
   CC_END NUMBER(19),

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/scripts/upgrade/oracle/upgrade-1.2.0-to-1.3.0.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/upgrade-1.2.0-to-1.3.0.oracle.sql b/metastore/scripts/upgrade/oracle/upgrade-1.2.0-to-1.3.0.oracle.sql
index ce86e67..7673b6e 100644
--- a/metastore/scripts/upgrade/oracle/upgrade-1.2.0-to-1.3.0.oracle.sql
+++ b/metastore/scripts/upgrade/oracle/upgrade-1.2.0-to-1.3.0.oracle.sql
@@ -11,6 +11,7 @@ SELECT 'Upgrading MetaStore schema from 1.2.0 to 1.3.0' AS Status from dual;
 @031-HIVE-12381.oracle.sql;
 @032-HIVE-12832.oracle.sql;
 @035-HIVE-13395.oracle.sql;
+@036-HIVE-13354.oracle.sql;
 
 UPDATE VERSION SET SCHEMA_VERSION='1.3.0', VERSION_COMMENT='Hive release version 1.3.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 1.2.0 to 1.3.0' AS Status from dual;

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/scripts/upgrade/postgres/035-HIVE-13354.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/035-HIVE-13354.postgres.sql b/metastore/scripts/upgrade/postgres/035-HIVE-13354.postgres.sql
new file mode 100644
index 0000000..2f691b1
--- /dev/null
+++ b/metastore/scripts/upgrade/postgres/035-HIVE-13354.postgres.sql
@@ -0,0 +1,2 @@
+ALTER TABLE COMPACTION_QUEUE ADD CQ_TBLPROPERTIES varchar(2048);
+ALTER TABLE COMPLETED_COMPACTIONS ADD CC_TBLPROPERTIES varchar(2048);

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/scripts/upgrade/postgres/hive-txn-schema-1.3.0.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/hive-txn-schema-1.3.0.postgres.sql b/metastore/scripts/upgrade/postgres/hive-txn-schema-1.3.0.postgres.sql
index b606f81..f998d18 100644
--- a/metastore/scripts/upgrade/postgres/hive-txn-schema-1.3.0.postgres.sql
+++ b/metastore/scripts/upgrade/postgres/hive-txn-schema-1.3.0.postgres.sql
@@ -83,6 +83,7 @@ CREATE TABLE COMPACTION_QUEUE (
   CQ_PARTITION varchar(767),
   CQ_STATE char(1) NOT NULL,
   CQ_TYPE char(1) NOT NULL,
+  CQ_TBLPROPERTIES varchar(2048),
   CQ_WORKER_ID varchar(128),
   CQ_START bigint,
   CQ_RUN_AS varchar(128),
@@ -103,6 +104,7 @@ CREATE TABLE COMPLETED_COMPACTIONS (
   CC_PARTITION varchar(767),
   CC_STATE char(1) NOT NULL,
   CC_TYPE char(1) NOT NULL,
+  CC_TBLPROPERTIES varchar(2048),
   CC_WORKER_ID varchar(128),
   CC_START bigint,
   CC_END bigint,

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/scripts/upgrade/postgres/upgrade-1.2.0-to-1.3.0.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/upgrade-1.2.0-to-1.3.0.postgres.sql b/metastore/scripts/upgrade/postgres/upgrade-1.2.0-to-1.3.0.postgres.sql
index 624dde6..260f5c2 100644
--- a/metastore/scripts/upgrade/postgres/upgrade-1.2.0-to-1.3.0.postgres.sql
+++ b/metastore/scripts/upgrade/postgres/upgrade-1.2.0-to-1.3.0.postgres.sql
@@ -11,6 +11,7 @@ SELECT 'Upgrading MetaStore schema from 1.2.0 to 1.3.0';
 \i 030-HIVE-12831.postgres.sql;
 \i 031-HIVE-12832.postgres.sql;
 \i 034-HIVE-13395.postgres.sql;
+\i 035-HIVE-13354.postgres.sql;
 
 UPDATE "VERSION" SET "SCHEMA_VERSION"='1.3.0', "VERSION_COMMENT"='Hive release version 1.3.0' where "VER_ID"=1;
 SELECT 'Finished upgrading MetaStore schema from 1.2.0 to 1.3.0';


[2/7] hive git commit: HIVE-13354 : Add ability to specify Compaction options per table and per request (Wei Zheng, reviewed by Eugene Koifman)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/src/gen/thrift/gen-php/metastore/Types.php
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-php/metastore/Types.php b/metastore/src/gen/thrift/gen-php/metastore/Types.php
index 045864a..925eea4 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -12797,6 +12797,10 @@ class CompactionRequest {
    * @var string
    */
   public $runas = null;
+  /**
+   * @var array
+   */
+  public $properties = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -12821,6 +12825,18 @@ class CompactionRequest {
           'var' => 'runas',
           'type' => TType::STRING,
           ),
+        6 => array(
+          'var' => 'properties',
+          'type' => TType::MAP,
+          'ktype' => TType::STRING,
+          'vtype' => TType::STRING,
+          'key' => array(
+            'type' => TType::STRING,
+          ),
+          'val' => array(
+            'type' => TType::STRING,
+            ),
+          ),
         );
     }
     if (is_array($vals)) {
@@ -12839,6 +12855,9 @@ class CompactionRequest {
       if (isset($vals['runas'])) {
         $this->runas = $vals['runas'];
       }
+      if (isset($vals['properties'])) {
+        $this->properties = $vals['properties'];
+      }
     }
   }
 
@@ -12896,6 +12915,26 @@ class CompactionRequest {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 6:
+          if ($ftype == TType::MAP) {
+            $this->properties = array();
+            $_size437 = 0;
+            $_ktype438 = 0;
+            $_vtype439 = 0;
+            $xfer += $input->readMapBegin($_ktype438, $_vtype439, $_size437);
+            for ($_i441 = 0; $_i441 < $_size437; ++$_i441)
+            {
+              $key442 = '';
+              $val443 = '';
+              $xfer += $input->readString($key442);
+              $xfer += $input->readString($val443);
+              $this->properties[$key442] = $val443;
+            }
+            $xfer += $input->readMapEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -12934,6 +12973,24 @@ class CompactionRequest {
       $xfer += $output->writeString($this->runas);
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->properties !== null) {
+      if (!is_array($this->properties)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('properties', TType::MAP, 6);
+      {
+        $output->writeMapBegin(TType::STRING, TType::STRING, count($this->properties));
+        {
+          foreach ($this->properties as $kiter444 => $viter445)
+          {
+            $xfer += $output->writeString($kiter444);
+            $xfer += $output->writeString($viter445);
+          }
+        }
+        $output->writeMapEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;
@@ -13370,15 +13427,15 @@ class ShowCompactResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->compacts = array();
-            $_size437 = 0;
-            $_etype440 = 0;
-            $xfer += $input->readListBegin($_etype440, $_size437);
-            for ($_i441 = 0; $_i441 < $_size437; ++$_i441)
+            $_size446 = 0;
+            $_etype449 = 0;
+            $xfer += $input->readListBegin($_etype449, $_size446);
+            for ($_i450 = 0; $_i450 < $_size446; ++$_i450)
             {
-              $elem442 = null;
-              $elem442 = new \metastore\ShowCompactResponseElement();
-              $xfer += $elem442->read($input);
-              $this->compacts []= $elem442;
+              $elem451 = null;
+              $elem451 = new \metastore\ShowCompactResponseElement();
+              $xfer += $elem451->read($input);
+              $this->compacts []= $elem451;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13406,9 +13463,9 @@ class ShowCompactResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->compacts));
         {
-          foreach ($this->compacts as $iter443)
+          foreach ($this->compacts as $iter452)
           {
-            $xfer += $iter443->write($output);
+            $xfer += $iter452->write($output);
           }
         }
         $output->writeListEnd();
@@ -13537,14 +13594,14 @@ class AddDynamicPartitions {
         case 4:
           if ($ftype == TType::LST) {
             $this->partitionnames = array();
-            $_size444 = 0;
-            $_etype447 = 0;
-            $xfer += $input->readListBegin($_etype447, $_size444);
-            for ($_i448 = 0; $_i448 < $_size444; ++$_i448)
+            $_size453 = 0;
+            $_etype456 = 0;
+            $xfer += $input->readListBegin($_etype456, $_size453);
+            for ($_i457 = 0; $_i457 < $_size453; ++$_i457)
             {
-              $elem449 = null;
-              $xfer += $input->readString($elem449);
-              $this->partitionnames []= $elem449;
+              $elem458 = null;
+              $xfer += $input->readString($elem458);
+              $this->partitionnames []= $elem458;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13594,9 +13651,9 @@ class AddDynamicPartitions {
       {
         $output->writeListBegin(TType::STRING, count($this->partitionnames));
         {
-          foreach ($this->partitionnames as $iter450)
+          foreach ($this->partitionnames as $iter459)
           {
-            $xfer += $output->writeString($iter450);
+            $xfer += $output->writeString($iter459);
           }
         }
         $output->writeListEnd();
@@ -13954,15 +14011,15 @@ class NotificationEventResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->events = array();
-            $_size451 = 0;
-            $_etype454 = 0;
-            $xfer += $input->readListBegin($_etype454, $_size451);
-            for ($_i455 = 0; $_i455 < $_size451; ++$_i455)
+            $_size460 = 0;
+            $_etype463 = 0;
+            $xfer += $input->readListBegin($_etype463, $_size460);
+            for ($_i464 = 0; $_i464 < $_size460; ++$_i464)
             {
-              $elem456 = null;
-              $elem456 = new \metastore\NotificationEvent();
-              $xfer += $elem456->read($input);
-              $this->events []= $elem456;
+              $elem465 = null;
+              $elem465 = new \metastore\NotificationEvent();
+              $xfer += $elem465->read($input);
+              $this->events []= $elem465;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13990,9 +14047,9 @@ class NotificationEventResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->events));
         {
-          foreach ($this->events as $iter457)
+          foreach ($this->events as $iter466)
           {
-            $xfer += $iter457->write($output);
+            $xfer += $iter466->write($output);
           }
         }
         $output->writeListEnd();
@@ -14131,14 +14188,14 @@ class InsertEventRequestData {
         case 1:
           if ($ftype == TType::LST) {
             $this->filesAdded = array();
-            $_size458 = 0;
-            $_etype461 = 0;
-            $xfer += $input->readListBegin($_etype461, $_size458);
-            for ($_i462 = 0; $_i462 < $_size458; ++$_i462)
+            $_size467 = 0;
+            $_etype470 = 0;
+            $xfer += $input->readListBegin($_etype470, $_size467);
+            for ($_i471 = 0; $_i471 < $_size467; ++$_i471)
             {
-              $elem463 = null;
-              $xfer += $input->readString($elem463);
-              $this->filesAdded []= $elem463;
+              $elem472 = null;
+              $xfer += $input->readString($elem472);
+              $this->filesAdded []= $elem472;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14166,9 +14223,9 @@ class InsertEventRequestData {
       {
         $output->writeListBegin(TType::STRING, count($this->filesAdded));
         {
-          foreach ($this->filesAdded as $iter464)
+          foreach ($this->filesAdded as $iter473)
           {
-            $xfer += $output->writeString($iter464);
+            $xfer += $output->writeString($iter473);
           }
         }
         $output->writeListEnd();
@@ -14386,14 +14443,14 @@ class FireEventRequest {
         case 5:
           if ($ftype == TType::LST) {
             $this->partitionVals = array();
-            $_size465 = 0;
-            $_etype468 = 0;
-            $xfer += $input->readListBegin($_etype468, $_size465);
-            for ($_i469 = 0; $_i469 < $_size465; ++$_i469)
+            $_size474 = 0;
+            $_etype477 = 0;
+            $xfer += $input->readListBegin($_etype477, $_size474);
+            for ($_i478 = 0; $_i478 < $_size474; ++$_i478)
             {
-              $elem470 = null;
-              $xfer += $input->readString($elem470);
-              $this->partitionVals []= $elem470;
+              $elem479 = null;
+              $xfer += $input->readString($elem479);
+              $this->partitionVals []= $elem479;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14444,9 +14501,9 @@ class FireEventRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->partitionVals));
         {
-          foreach ($this->partitionVals as $iter471)
+          foreach ($this->partitionVals as $iter480)
           {
-            $xfer += $output->writeString($iter471);
+            $xfer += $output->writeString($iter480);
           }
         }
         $output->writeListEnd();
@@ -14561,15 +14618,15 @@ class GetAllFunctionsResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->functions = array();
-            $_size472 = 0;
-            $_etype475 = 0;
-            $xfer += $input->readListBegin($_etype475, $_size472);
-            for ($_i476 = 0; $_i476 < $_size472; ++$_i476)
+            $_size481 = 0;
+            $_etype484 = 0;
+            $xfer += $input->readListBegin($_etype484, $_size481);
+            for ($_i485 = 0; $_i485 < $_size481; ++$_i485)
             {
-              $elem477 = null;
-              $elem477 = new \metastore\Function();
-              $xfer += $elem477->read($input);
-              $this->functions []= $elem477;
+              $elem486 = null;
+              $elem486 = new \metastore\Function();
+              $xfer += $elem486->read($input);
+              $this->functions []= $elem486;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14597,9 +14654,9 @@ class GetAllFunctionsResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->functions));
         {
-          foreach ($this->functions as $iter478)
+          foreach ($this->functions as $iter487)
           {
-            $xfer += $iter478->write($output);
+            $xfer += $iter487->write($output);
           }
         }
         $output->writeListEnd();

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
index d739752..73a375d 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -9862,10 +9862,10 @@ class get_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype479, _size476) = iprot.readListBegin()
-          for _i480 in xrange(_size476):
-            _elem481 = iprot.readString()
-            self.success.append(_elem481)
+          (_etype488, _size485) = iprot.readListBegin()
+          for _i489 in xrange(_size485):
+            _elem490 = iprot.readString()
+            self.success.append(_elem490)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9888,8 +9888,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 iter482 in self.success:
-        oprot.writeString(iter482)
+      for iter491 in self.success:
+        oprot.writeString(iter491)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -9994,10 +9994,10 @@ class get_all_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype486, _size483) = iprot.readListBegin()
-          for _i487 in xrange(_size483):
-            _elem488 = iprot.readString()
-            self.success.append(_elem488)
+          (_etype495, _size492) = iprot.readListBegin()
+          for _i496 in xrange(_size492):
+            _elem497 = iprot.readString()
+            self.success.append(_elem497)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10020,8 +10020,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 iter489 in self.success:
-        oprot.writeString(iter489)
+      for iter498 in self.success:
+        oprot.writeString(iter498)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -10791,12 +10791,12 @@ class get_type_all_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype491, _vtype492, _size490 ) = iprot.readMapBegin()
-          for _i494 in xrange(_size490):
-            _key495 = iprot.readString()
-            _val496 = Type()
-            _val496.read(iprot)
-            self.success[_key495] = _val496
+          (_ktype500, _vtype501, _size499 ) = iprot.readMapBegin()
+          for _i503 in xrange(_size499):
+            _key504 = iprot.readString()
+            _val505 = Type()
+            _val505.read(iprot)
+            self.success[_key504] = _val505
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -10819,9 +10819,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 kiter497,viter498 in self.success.items():
-        oprot.writeString(kiter497)
-        viter498.write(oprot)
+      for kiter506,viter507 in self.success.items():
+        oprot.writeString(kiter506)
+        viter507.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -10964,11 +10964,11 @@ class get_fields_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype502, _size499) = iprot.readListBegin()
-          for _i503 in xrange(_size499):
-            _elem504 = FieldSchema()
-            _elem504.read(iprot)
-            self.success.append(_elem504)
+          (_etype511, _size508) = iprot.readListBegin()
+          for _i512 in xrange(_size508):
+            _elem513 = FieldSchema()
+            _elem513.read(iprot)
+            self.success.append(_elem513)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11003,8 +11003,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 iter505 in self.success:
-        iter505.write(oprot)
+      for iter514 in self.success:
+        iter514.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -11171,11 +11171,11 @@ class get_fields_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype509, _size506) = iprot.readListBegin()
-          for _i510 in xrange(_size506):
-            _elem511 = FieldSchema()
-            _elem511.read(iprot)
-            self.success.append(_elem511)
+          (_etype518, _size515) = iprot.readListBegin()
+          for _i519 in xrange(_size515):
+            _elem520 = FieldSchema()
+            _elem520.read(iprot)
+            self.success.append(_elem520)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11210,8 +11210,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 iter512 in self.success:
-        iter512.write(oprot)
+      for iter521 in self.success:
+        iter521.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -11364,11 +11364,11 @@ class get_schema_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype516, _size513) = iprot.readListBegin()
-          for _i517 in xrange(_size513):
-            _elem518 = FieldSchema()
-            _elem518.read(iprot)
-            self.success.append(_elem518)
+          (_etype525, _size522) = iprot.readListBegin()
+          for _i526 in xrange(_size522):
+            _elem527 = FieldSchema()
+            _elem527.read(iprot)
+            self.success.append(_elem527)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11403,8 +11403,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 iter519 in self.success:
-        iter519.write(oprot)
+      for iter528 in self.success:
+        iter528.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -11571,11 +11571,11 @@ class get_schema_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype523, _size520) = iprot.readListBegin()
-          for _i524 in xrange(_size520):
-            _elem525 = FieldSchema()
-            _elem525.read(iprot)
-            self.success.append(_elem525)
+          (_etype532, _size529) = iprot.readListBegin()
+          for _i533 in xrange(_size529):
+            _elem534 = FieldSchema()
+            _elem534.read(iprot)
+            self.success.append(_elem534)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11610,8 +11610,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 iter526 in self.success:
-        iter526.write(oprot)
+      for iter535 in self.success:
+        iter535.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -12476,10 +12476,10 @@ class get_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype530, _size527) = iprot.readListBegin()
-          for _i531 in xrange(_size527):
-            _elem532 = iprot.readString()
-            self.success.append(_elem532)
+          (_etype539, _size536) = iprot.readListBegin()
+          for _i540 in xrange(_size536):
+            _elem541 = iprot.readString()
+            self.success.append(_elem541)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12502,8 +12502,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 iter533 in self.success:
-        oprot.writeString(iter533)
+      for iter542 in self.success:
+        oprot.writeString(iter542)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -12627,10 +12627,10 @@ class get_all_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype537, _size534) = iprot.readListBegin()
-          for _i538 in xrange(_size534):
-            _elem539 = iprot.readString()
-            self.success.append(_elem539)
+          (_etype546, _size543) = iprot.readListBegin()
+          for _i547 in xrange(_size543):
+            _elem548 = iprot.readString()
+            self.success.append(_elem548)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12653,8 +12653,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 iter540 in self.success:
-        oprot.writeString(iter540)
+      for iter549 in self.success:
+        oprot.writeString(iter549)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -12890,10 +12890,10 @@ class get_table_objects_by_name_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype544, _size541) = iprot.readListBegin()
-          for _i545 in xrange(_size541):
-            _elem546 = iprot.readString()
-            self.tbl_names.append(_elem546)
+          (_etype553, _size550) = iprot.readListBegin()
+          for _i554 in xrange(_size550):
+            _elem555 = iprot.readString()
+            self.tbl_names.append(_elem555)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12914,8 +12914,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 iter547 in self.tbl_names:
-        oprot.writeString(iter547)
+      for iter556 in self.tbl_names:
+        oprot.writeString(iter556)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -12976,11 +12976,11 @@ class get_table_objects_by_name_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype551, _size548) = iprot.readListBegin()
-          for _i552 in xrange(_size548):
-            _elem553 = Table()
-            _elem553.read(iprot)
-            self.success.append(_elem553)
+          (_etype560, _size557) = iprot.readListBegin()
+          for _i561 in xrange(_size557):
+            _elem562 = Table()
+            _elem562.read(iprot)
+            self.success.append(_elem562)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13015,8 +13015,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 iter554 in self.success:
-        iter554.write(oprot)
+      for iter563 in self.success:
+        iter563.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -13182,10 +13182,10 @@ class get_table_names_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype558, _size555) = iprot.readListBegin()
-          for _i559 in xrange(_size555):
-            _elem560 = iprot.readString()
-            self.success.append(_elem560)
+          (_etype567, _size564) = iprot.readListBegin()
+          for _i568 in xrange(_size564):
+            _elem569 = iprot.readString()
+            self.success.append(_elem569)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13220,8 +13220,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 iter561 in self.success:
-        oprot.writeString(iter561)
+      for iter570 in self.success:
+        oprot.writeString(iter570)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -14191,11 +14191,11 @@ class add_partitions_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype565, _size562) = iprot.readListBegin()
-          for _i566 in xrange(_size562):
-            _elem567 = Partition()
-            _elem567.read(iprot)
-            self.new_parts.append(_elem567)
+          (_etype574, _size571) = iprot.readListBegin()
+          for _i575 in xrange(_size571):
+            _elem576 = Partition()
+            _elem576.read(iprot)
+            self.new_parts.append(_elem576)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14212,8 +14212,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 iter568 in self.new_parts:
-        iter568.write(oprot)
+      for iter577 in self.new_parts:
+        iter577.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14371,11 +14371,11 @@ class add_partitions_pspec_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype572, _size569) = iprot.readListBegin()
-          for _i573 in xrange(_size569):
-            _elem574 = PartitionSpec()
-            _elem574.read(iprot)
-            self.new_parts.append(_elem574)
+          (_etype581, _size578) = iprot.readListBegin()
+          for _i582 in xrange(_size578):
+            _elem583 = PartitionSpec()
+            _elem583.read(iprot)
+            self.new_parts.append(_elem583)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14392,8 +14392,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 iter575 in self.new_parts:
-        iter575.write(oprot)
+      for iter584 in self.new_parts:
+        iter584.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14567,10 +14567,10 @@ class append_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype579, _size576) = iprot.readListBegin()
-          for _i580 in xrange(_size576):
-            _elem581 = iprot.readString()
-            self.part_vals.append(_elem581)
+          (_etype588, _size585) = iprot.readListBegin()
+          for _i589 in xrange(_size585):
+            _elem590 = iprot.readString()
+            self.part_vals.append(_elem590)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14595,8 +14595,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 iter582 in self.part_vals:
-        oprot.writeString(iter582)
+      for iter591 in self.part_vals:
+        oprot.writeString(iter591)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14949,10 +14949,10 @@ class append_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype586, _size583) = iprot.readListBegin()
-          for _i587 in xrange(_size583):
-            _elem588 = iprot.readString()
-            self.part_vals.append(_elem588)
+          (_etype595, _size592) = iprot.readListBegin()
+          for _i596 in xrange(_size592):
+            _elem597 = iprot.readString()
+            self.part_vals.append(_elem597)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14983,8 +14983,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 iter589 in self.part_vals:
-        oprot.writeString(iter589)
+      for iter598 in self.part_vals:
+        oprot.writeString(iter598)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -15579,10 +15579,10 @@ class drop_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype593, _size590) = iprot.readListBegin()
-          for _i594 in xrange(_size590):
-            _elem595 = iprot.readString()
-            self.part_vals.append(_elem595)
+          (_etype602, _size599) = iprot.readListBegin()
+          for _i603 in xrange(_size599):
+            _elem604 = iprot.readString()
+            self.part_vals.append(_elem604)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15612,8 +15612,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 iter596 in self.part_vals:
-        oprot.writeString(iter596)
+      for iter605 in self.part_vals:
+        oprot.writeString(iter605)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -15786,10 +15786,10 @@ class drop_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype600, _size597) = iprot.readListBegin()
-          for _i601 in xrange(_size597):
-            _elem602 = iprot.readString()
-            self.part_vals.append(_elem602)
+          (_etype609, _size606) = iprot.readListBegin()
+          for _i610 in xrange(_size606):
+            _elem611 = iprot.readString()
+            self.part_vals.append(_elem611)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15825,8 +15825,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 iter603 in self.part_vals:
-        oprot.writeString(iter603)
+      for iter612 in self.part_vals:
+        oprot.writeString(iter612)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -16563,10 +16563,10 @@ class get_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype607, _size604) = iprot.readListBegin()
-          for _i608 in xrange(_size604):
-            _elem609 = iprot.readString()
-            self.part_vals.append(_elem609)
+          (_etype616, _size613) = iprot.readListBegin()
+          for _i617 in xrange(_size613):
+            _elem618 = iprot.readString()
+            self.part_vals.append(_elem618)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16591,8 +16591,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 iter610 in self.part_vals:
-        oprot.writeString(iter610)
+      for iter619 in self.part_vals:
+        oprot.writeString(iter619)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16751,11 +16751,11 @@ class exchange_partition_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype612, _vtype613, _size611 ) = iprot.readMapBegin()
-          for _i615 in xrange(_size611):
-            _key616 = iprot.readString()
-            _val617 = iprot.readString()
-            self.partitionSpecs[_key616] = _val617
+          (_ktype621, _vtype622, _size620 ) = iprot.readMapBegin()
+          for _i624 in xrange(_size620):
+            _key625 = iprot.readString()
+            _val626 = iprot.readString()
+            self.partitionSpecs[_key625] = _val626
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -16792,9 +16792,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 kiter618,viter619 in self.partitionSpecs.items():
-        oprot.writeString(kiter618)
-        oprot.writeString(viter619)
+      for kiter627,viter628 in self.partitionSpecs.items():
+        oprot.writeString(kiter627)
+        oprot.writeString(viter628)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -16999,11 +16999,11 @@ class exchange_partitions_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype621, _vtype622, _size620 ) = iprot.readMapBegin()
-          for _i624 in xrange(_size620):
-            _key625 = iprot.readString()
-            _val626 = iprot.readString()
-            self.partitionSpecs[_key625] = _val626
+          (_ktype630, _vtype631, _size629 ) = iprot.readMapBegin()
+          for _i633 in xrange(_size629):
+            _key634 = iprot.readString()
+            _val635 = iprot.readString()
+            self.partitionSpecs[_key634] = _val635
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -17040,9 +17040,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 kiter627,viter628 in self.partitionSpecs.items():
-        oprot.writeString(kiter627)
-        oprot.writeString(viter628)
+      for kiter636,viter637 in self.partitionSpecs.items():
+        oprot.writeString(kiter636)
+        oprot.writeString(viter637)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -17125,11 +17125,11 @@ class exchange_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype632, _size629) = iprot.readListBegin()
-          for _i633 in xrange(_size629):
-            _elem634 = Partition()
-            _elem634.read(iprot)
-            self.success.append(_elem634)
+          (_etype641, _size638) = iprot.readListBegin()
+          for _i642 in xrange(_size638):
+            _elem643 = Partition()
+            _elem643.read(iprot)
+            self.success.append(_elem643)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17170,8 +17170,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 iter635 in self.success:
-        iter635.write(oprot)
+      for iter644 in self.success:
+        iter644.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17265,10 +17265,10 @@ class get_partition_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype639, _size636) = iprot.readListBegin()
-          for _i640 in xrange(_size636):
-            _elem641 = iprot.readString()
-            self.part_vals.append(_elem641)
+          (_etype648, _size645) = iprot.readListBegin()
+          for _i649 in xrange(_size645):
+            _elem650 = iprot.readString()
+            self.part_vals.append(_elem650)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17280,10 +17280,10 @@ class get_partition_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype645, _size642) = iprot.readListBegin()
-          for _i646 in xrange(_size642):
-            _elem647 = iprot.readString()
-            self.group_names.append(_elem647)
+          (_etype654, _size651) = iprot.readListBegin()
+          for _i655 in xrange(_size651):
+            _elem656 = iprot.readString()
+            self.group_names.append(_elem656)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17308,8 +17308,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 iter648 in self.part_vals:
-        oprot.writeString(iter648)
+      for iter657 in self.part_vals:
+        oprot.writeString(iter657)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.user_name is not None:
@@ -17319,8 +17319,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 iter649 in self.group_names:
-        oprot.writeString(iter649)
+      for iter658 in self.group_names:
+        oprot.writeString(iter658)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17749,11 +17749,11 @@ class get_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype653, _size650) = iprot.readListBegin()
-          for _i654 in xrange(_size650):
-            _elem655 = Partition()
-            _elem655.read(iprot)
-            self.success.append(_elem655)
+          (_etype662, _size659) = iprot.readListBegin()
+          for _i663 in xrange(_size659):
+            _elem664 = Partition()
+            _elem664.read(iprot)
+            self.success.append(_elem664)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17782,8 +17782,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 iter656 in self.success:
-        iter656.write(oprot)
+      for iter665 in self.success:
+        iter665.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17877,10 +17877,10 @@ class get_partitions_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype660, _size657) = iprot.readListBegin()
-          for _i661 in xrange(_size657):
-            _elem662 = iprot.readString()
-            self.group_names.append(_elem662)
+          (_etype669, _size666) = iprot.readListBegin()
+          for _i670 in xrange(_size666):
+            _elem671 = iprot.readString()
+            self.group_names.append(_elem671)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17913,8 +17913,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 iter663 in self.group_names:
-        oprot.writeString(iter663)
+      for iter672 in self.group_names:
+        oprot.writeString(iter672)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17975,11 +17975,11 @@ class get_partitions_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype667, _size664) = iprot.readListBegin()
-          for _i668 in xrange(_size664):
-            _elem669 = Partition()
-            _elem669.read(iprot)
-            self.success.append(_elem669)
+          (_etype676, _size673) = iprot.readListBegin()
+          for _i677 in xrange(_size673):
+            _elem678 = Partition()
+            _elem678.read(iprot)
+            self.success.append(_elem678)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18008,8 +18008,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 iter670 in self.success:
-        iter670.write(oprot)
+      for iter679 in self.success:
+        iter679.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18167,11 +18167,11 @@ class get_partitions_pspec_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype674, _size671) = iprot.readListBegin()
-          for _i675 in xrange(_size671):
-            _elem676 = PartitionSpec()
-            _elem676.read(iprot)
-            self.success.append(_elem676)
+          (_etype683, _size680) = iprot.readListBegin()
+          for _i684 in xrange(_size680):
+            _elem685 = PartitionSpec()
+            _elem685.read(iprot)
+            self.success.append(_elem685)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18200,8 +18200,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 iter677 in self.success:
-        iter677.write(oprot)
+      for iter686 in self.success:
+        iter686.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18356,10 +18356,10 @@ class get_partition_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype681, _size678) = iprot.readListBegin()
-          for _i682 in xrange(_size678):
-            _elem683 = iprot.readString()
-            self.success.append(_elem683)
+          (_etype690, _size687) = iprot.readListBegin()
+          for _i691 in xrange(_size687):
+            _elem692 = iprot.readString()
+            self.success.append(_elem692)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18382,8 +18382,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 iter684 in self.success:
-        oprot.writeString(iter684)
+      for iter693 in self.success:
+        oprot.writeString(iter693)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -18459,10 +18459,10 @@ class get_partitions_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype688, _size685) = iprot.readListBegin()
-          for _i689 in xrange(_size685):
-            _elem690 = iprot.readString()
-            self.part_vals.append(_elem690)
+          (_etype697, _size694) = iprot.readListBegin()
+          for _i698 in xrange(_size694):
+            _elem699 = iprot.readString()
+            self.part_vals.append(_elem699)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18492,8 +18492,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 iter691 in self.part_vals:
-        oprot.writeString(iter691)
+      for iter700 in self.part_vals:
+        oprot.writeString(iter700)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -18557,11 +18557,11 @@ class get_partitions_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype695, _size692) = iprot.readListBegin()
-          for _i696 in xrange(_size692):
-            _elem697 = Partition()
-            _elem697.read(iprot)
-            self.success.append(_elem697)
+          (_etype704, _size701) = iprot.readListBegin()
+          for _i705 in xrange(_size701):
+            _elem706 = Partition()
+            _elem706.read(iprot)
+            self.success.append(_elem706)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18590,8 +18590,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 iter698 in self.success:
-        iter698.write(oprot)
+      for iter707 in self.success:
+        iter707.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18678,10 +18678,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype702, _size699) = iprot.readListBegin()
-          for _i703 in xrange(_size699):
-            _elem704 = iprot.readString()
-            self.part_vals.append(_elem704)
+          (_etype711, _size708) = iprot.readListBegin()
+          for _i712 in xrange(_size708):
+            _elem713 = iprot.readString()
+            self.part_vals.append(_elem713)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18698,10 +18698,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 6:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype708, _size705) = iprot.readListBegin()
-          for _i709 in xrange(_size705):
-            _elem710 = iprot.readString()
-            self.group_names.append(_elem710)
+          (_etype717, _size714) = iprot.readListBegin()
+          for _i718 in xrange(_size714):
+            _elem719 = iprot.readString()
+            self.group_names.append(_elem719)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18726,8 +18726,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 iter711 in self.part_vals:
-        oprot.writeString(iter711)
+      for iter720 in self.part_vals:
+        oprot.writeString(iter720)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -18741,8 +18741,8 @@ class get_partitions_ps_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 6)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter712 in self.group_names:
-        oprot.writeString(iter712)
+      for iter721 in self.group_names:
+        oprot.writeString(iter721)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18804,11 +18804,11 @@ class get_partitions_ps_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype716, _size713) = iprot.readListBegin()
-          for _i717 in xrange(_size713):
-            _elem718 = Partition()
-            _elem718.read(iprot)
-            self.success.append(_elem718)
+          (_etype725, _size722) = iprot.readListBegin()
+          for _i726 in xrange(_size722):
+            _elem727 = Partition()
+            _elem727.read(iprot)
+            self.success.append(_elem727)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18837,8 +18837,8 @@ class get_partitions_ps_with_auth_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter719 in self.success:
-        iter719.write(oprot)
+      for iter728 in self.success:
+        iter728.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18919,10 +18919,10 @@ class get_partition_names_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype723, _size720) = iprot.readListBegin()
-          for _i724 in xrange(_size720):
-            _elem725 = iprot.readString()
-            self.part_vals.append(_elem725)
+          (_etype732, _size729) = iprot.readListBegin()
+          for _i733 in xrange(_size729):
+            _elem734 = iprot.readString()
+            self.part_vals.append(_elem734)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18952,8 +18952,8 @@ class get_partition_names_ps_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter726 in self.part_vals:
-        oprot.writeString(iter726)
+      for iter735 in self.part_vals:
+        oprot.writeString(iter735)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -19017,10 +19017,10 @@ class get_partition_names_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype730, _size727) = iprot.readListBegin()
-          for _i731 in xrange(_size727):
-            _elem732 = iprot.readString()
-            self.success.append(_elem732)
+          (_etype739, _size736) = iprot.readListBegin()
+          for _i740 in xrange(_size736):
+            _elem741 = iprot.readString()
+            self.success.append(_elem741)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19049,8 +19049,8 @@ class get_partition_names_ps_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter733 in self.success:
-        oprot.writeString(iter733)
+      for iter742 in self.success:
+        oprot.writeString(iter742)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19221,11 +19221,11 @@ class get_partitions_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype737, _size734) = iprot.readListBegin()
-          for _i738 in xrange(_size734):
-            _elem739 = Partition()
-            _elem739.read(iprot)
-            self.success.append(_elem739)
+          (_etype746, _size743) = iprot.readListBegin()
+          for _i747 in xrange(_size743):
+            _elem748 = Partition()
+            _elem748.read(iprot)
+            self.success.append(_elem748)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19254,8 +19254,8 @@ class get_partitions_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter740 in self.success:
-        iter740.write(oprot)
+      for iter749 in self.success:
+        iter749.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19426,11 +19426,11 @@ class get_part_specs_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype744, _size741) = iprot.readListBegin()
-          for _i745 in xrange(_size741):
-            _elem746 = PartitionSpec()
-            _elem746.read(iprot)
-            self.success.append(_elem746)
+          (_etype753, _size750) = iprot.readListBegin()
+          for _i754 in xrange(_size750):
+            _elem755 = PartitionSpec()
+            _elem755.read(iprot)
+            self.success.append(_elem755)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19459,8 +19459,8 @@ class get_part_specs_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter747 in self.success:
-        iter747.write(oprot)
+      for iter756 in self.success:
+        iter756.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19697,10 +19697,10 @@ class get_partitions_by_names_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.names = []
-          (_etype751, _size748) = iprot.readListBegin()
-          for _i752 in xrange(_size748):
-            _elem753 = iprot.readString()
-            self.names.append(_elem753)
+          (_etype760, _size757) = iprot.readListBegin()
+          for _i761 in xrange(_size757):
+            _elem762 = iprot.readString()
+            self.names.append(_elem762)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19725,8 +19725,8 @@ class get_partitions_by_names_args:
     if self.names is not None:
       oprot.writeFieldBegin('names', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.names))
-      for iter754 in self.names:
-        oprot.writeString(iter754)
+      for iter763 in self.names:
+        oprot.writeString(iter763)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -19785,11 +19785,11 @@ class get_partitions_by_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype758, _size755) = iprot.readListBegin()
-          for _i759 in xrange(_size755):
-            _elem760 = Partition()
-            _elem760.read(iprot)
-            self.success.append(_elem760)
+          (_etype767, _size764) = iprot.readListBegin()
+          for _i768 in xrange(_size764):
+            _elem769 = Partition()
+            _elem769.read(iprot)
+            self.success.append(_elem769)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19818,8 +19818,8 @@ class get_partitions_by_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter761 in self.success:
-        iter761.write(oprot)
+      for iter770 in self.success:
+        iter770.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20069,11 +20069,11 @@ class alter_partitions_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype765, _size762) = iprot.readListBegin()
-          for _i766 in xrange(_size762):
-            _elem767 = Partition()
-            _elem767.read(iprot)
-            self.new_parts.append(_elem767)
+          (_etype774, _size771) = iprot.readListBegin()
+          for _i775 in xrange(_size771):
+            _elem776 = Partition()
+            _elem776.read(iprot)
+            self.new_parts.append(_elem776)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20098,8 +20098,8 @@ class alter_partitions_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter768 in self.new_parts:
-        iter768.write(oprot)
+      for iter777 in self.new_parts:
+        iter777.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20438,10 +20438,10 @@ class rename_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype772, _size769) = iprot.readListBegin()
-          for _i773 in xrange(_size769):
-            _elem774 = iprot.readString()
-            self.part_vals.append(_elem774)
+          (_etype781, _size778) = iprot.readListBegin()
+          for _i782 in xrange(_size778):
+            _elem783 = iprot.readString()
+            self.part_vals.append(_elem783)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20472,8 +20472,8 @@ class rename_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter775 in self.part_vals:
-        oprot.writeString(iter775)
+      for iter784 in self.part_vals:
+        oprot.writeString(iter784)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.new_part is not None:
@@ -20615,10 +20615,10 @@ class partition_name_has_valid_characters_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype779, _size776) = iprot.readListBegin()
-          for _i780 in xrange(_size776):
-            _elem781 = iprot.readString()
-            self.part_vals.append(_elem781)
+          (_etype788, _size785) = iprot.readListBegin()
+          for _i789 in xrange(_size785):
+            _elem790 = iprot.readString()
+            self.part_vals.append(_elem790)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20640,8 +20640,8 @@ class partition_name_has_valid_characters_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter782 in self.part_vals:
-        oprot.writeString(iter782)
+      for iter791 in self.part_vals:
+        oprot.writeString(iter791)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.throw_exception is not None:
@@ -20999,10 +20999,10 @@ class partition_name_to_vals_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype786, _size783) = iprot.readListBegin()
-          for _i787 in xrange(_size783):
-            _elem788 = iprot.readString()
-            self.success.append(_elem788)
+          (_etype795, _size792) = iprot.readListBegin()
+          for _i796 in xrange(_size792):
+            _elem797 = iprot.readString()
+            self.success.append(_elem797)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21025,8 +21025,8 @@ class partition_name_to_vals_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter789 in self.success:
-        oprot.writeString(iter789)
+      for iter798 in self.success:
+        oprot.writeString(iter798)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21150,11 +21150,11 @@ class partition_name_to_spec_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype791, _vtype792, _size790 ) = iprot.readMapBegin()
-          for _i794 in xrange(_size790):
-            _key795 = iprot.readString()
-            _val796 = iprot.readString()
-            self.success[_key795] = _val796
+          (_ktype800, _vtype801, _size799 ) = iprot.readMapBegin()
+          for _i803 in xrange(_size799):
+            _key804 = iprot.readString()
+            _val805 = iprot.readString()
+            self.success[_key804] = _val805
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -21177,9 +21177,9 @@ class partition_name_to_spec_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.MAP, 0)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.success))
-      for kiter797,viter798 in self.success.items():
-        oprot.writeString(kiter797)
-        oprot.writeString(viter798)
+      for kiter806,viter807 in self.success.items():
+        oprot.writeString(kiter806)
+        oprot.writeString(viter807)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21255,11 +21255,11 @@ class markPartitionForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype800, _vtype801, _size799 ) = iprot.readMapBegin()
-          for _i803 in xrange(_size799):
-            _key804 = iprot.readString()
-            _val805 = iprot.readString()
-            self.part_vals[_key804] = _val805
+          (_ktype809, _vtype810, _size808 ) = iprot.readMapBegin()
+          for _i812 in xrange(_size808):
+            _key813 = iprot.readString()
+            _val814 = iprot.readString()
+            self.part_vals[_key813] = _val814
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -21289,9 +21289,9 @@ class markPartitionForEvent_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals))
-      for kiter806,viter807 in self.part_vals.items():
-        oprot.writeString(kiter806)
-        oprot.writeString(viter807)
+      for kiter815,viter816 in self.part_vals.items():
+        oprot.writeString(kiter815)
+        oprot.writeString(viter816)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -21505,11 +21505,11 @@ class isPartitionMarkedForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype809, _vtype810, _size808 ) = iprot.readMapBegin()
-          for _i812 in xrange(_size808):
-            _key813 = iprot.readString()
-            _val814 = iprot.readString()
-            self.part_vals[_key813] = _val814
+          (_ktype818, _vtype819, _size817 ) = iprot.readMapBegin()
+          for _i821 in xrange(_size817):
+            _key822 = iprot.readString()
+            _val823 = iprot.readString()
+            self.part_vals[_key822] = _val823
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -21539,9 +21539,9 @@ class isPartitionMarkedForEvent_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals))
-      for kiter815,viter816 in self.part_vals.items():
-        oprot.writeString(kiter815)
-        oprot.writeString(viter816)
+      for kiter824,viter825 in self.part_vals.items():
+        oprot.writeString(kiter824)
+        oprot.writeString(viter825)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -22596,11 +22596,11 @@ class get_indexes_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype820, _size817) = iprot.readListBegin()
-          for _i821 in xrange(_size817):
-            _elem822 = Index()
-            _elem822.read(iprot)
-            self.success.append(_elem822)
+          (_etype829, _size826) = iprot.readListBegin()
+          for _i830 in xrange(_size826):
+            _elem831 = Index()
+            _elem831.read(iprot)
+            self.success.append(_elem831)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22629,8 +22629,8 @@ class get_indexes_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter823 in self.success:
-        iter823.write(oprot)
+      for iter832 in self.success:
+        iter832.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -22785,10 +22785,10 @@ class get_index_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype827, _size824) = iprot.readListBegin()
-          for _i828 in xrange(_size824):
-            _elem829 = iprot.readString()
-            self.success.append(_elem829)
+          (_etype836, _size833) = iprot.readListBegin()
+          for _i837 in xrange(_size833):
+            _elem838 = iprot.readString()
+            self.success.append(_elem838)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22811,8 +22811,8 @@ class get_index_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter830 in self.success:
-        oprot.writeString(iter830)
+      for iter839 in self.success:
+        oprot.writeString(iter839)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -25360,10 +25360,10 @@ class get_functions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype834, _size831) = iprot.readListBegin()
-          for _i835 in xrange(_size831):
-            _elem836 = iprot.readString()
-            self.success.append(_elem836)
+          (_etype843, _size840) = iprot.readListBegin()
+          for _i844 in xrange(_size840):
+            _elem845 = iprot.readString()
+            self.success.append(_elem845)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25386,8 +25386,8 @@ class get_functions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter837 in self.success:
-        oprot.writeString(iter837)
+      for iter846 in self.success:
+        oprot.writeString(iter846)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26075,10 +26075,10 @@ class get_role_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype841, _size838) = iprot.readListBegin()
-          for _i842 in xrange(_size838):
-            _elem843 = iprot.readString()
-            self.success.append(_elem843)
+          (_etype850, _size847) = iprot.readListBegin()
+          for _i851 in xrange(_size847):
+            _elem852 = iprot.readString()
+            self.success.append(_elem852)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26101,8 +26101,8 @@ class get_role_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter844 in self.success:
-        oprot.writeString(iter844)
+      for iter853 in self.success:
+        oprot.writeString(iter853)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26616,11 +26616,11 @@ class list_roles_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype848, _size845) = iprot.readListBegin()
-          for _i849 in xrange(_size845):
-            _elem850 = Role()
-            _elem850.read(iprot)
-            self.success.append(_elem850)
+          (_etype857, _size854) = iprot.readListBegin()
+          for _i858 in xrange(_size854):
+            _elem859 = Role()
+            _elem859.read(iprot)
+            self.success.append(_elem859)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26643,8 +26643,8 @@ class list_roles_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter851 in self.success:
-        iter851.write(oprot)
+      for iter860 in self.success:
+        iter860.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27153,10 +27153,10 @@ class get_privilege_set_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype855, _size852) = iprot.readListBegin()
-          for _i856 in xrange(_size852):
-            _elem857 = iprot.readString()
-            self.group_names.append(_elem857)
+          (_etype864, _size861) = iprot.readListBegin()
+          for _i865 in xrange(_size861):
+            _elem866 = iprot.readString()
+            self.group_names.append(_elem866)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27181,8 +27181,8 @@ class get_privilege_set_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter858 in self.group_names:
-        oprot.writeString(iter858)
+      for iter867 in self.group_names:
+        oprot.writeString(iter867)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -27409,11 +27409,11 @@ class list_privileges_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype862, _size859) = iprot.readListBegin()
-          for _i863 in xrange(_size859):
-            _elem864 = HiveObjectPrivilege()
-            _elem864.read(iprot)
-            self.success.append(_elem864)
+          (_etype871, _size868) = iprot.readListBegin()
+          for _i872 in xrange(_size868):
+            _elem873 = HiveObjectPrivilege()
+            _elem873.read(iprot)
+            self.success.append(_elem873)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27436,8 +27436,8 @@ class list_privileges_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter865 in self.success:
-        iter865.write(oprot)
+      for iter874 in self.success:
+        iter874.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27935,10 +27935,10 @@ class set_ugi_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype869, _size866) = iprot.readListBegin()
-          for _i870 in xrange(_size866):
-            _elem871 = iprot.readString()
-            self.group_names.append(_elem871)
+          (_etype878, _size875) = iprot.readListBegin()
+          for _i879 in xrange(_size875):
+            _elem880 = iprot.readString()
+            self.group_names.append(_elem880)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27959,8 +27959,8 @@ class set_ugi_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter872 in self.group_names:
-        oprot.writeString(iter872)
+      for iter881 in self.group_names:
+        oprot.writeString(iter881)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -28015,10 +28015,10 @@ class set_ugi_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype876, _size873) = iprot.readListBegin()
-          for _i877 in xrange(_size873):
-            _elem878 = iprot.readString()
-            self.success.append(_elem878)
+          (_etype885, _size882) = iprot.readListBegin()
+          for _i886 in xrange(_size882):
+            _elem887 = iprot.readString()
+            self.success.append(_elem887)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28041,8 +28041,8 @@ class set_ugi_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter879 in self.success:
-        oprot.writeString(iter879)
+      for iter888 in self.success:
+        oprot.writeString(iter888)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index 29ba9b1..2c34c66 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -8892,6 +8892,7 @@ class CompactionRequest:
    - partitionname
    - type
    - runas
+   - properties
   """
 
   thrift_spec = (
@@ -8901,14 +8902,16 @@ class CompactionRequest:
     (3, TType.STRING, 'partitionname', None, None, ), # 3
     (4, TType.I32, 'type', None, None, ), # 4
     (5, TType.STRING, 'runas', None, None, ), # 5
+    (6, TType.MAP, 'properties', (TType.STRING,None,TType.STRING,None), None, ), # 6
   )
 
-  def __init__(self, dbname=None, tablename=None, partitionname=None, type=None, runas=None,):
+  def __init__(self, dbname=None, tablename=None, partitionname=None, type=None, runas=None, properties=None,):
     self.dbname = dbname
     self.tablename = tablename
     self.partitionname = partitionname
     self.type = type
     self.runas = runas
+    self.properties = properties
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -8944,6 +8947,17 @@ class CompactionRequest:
           self.runas = iprot.readString()
         else:
           iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.MAP:
+          self.properties = {}
+          (_ktype435, _vtype436, _size434 ) = iprot.readMapBegin()
+          for _i438 in xrange(_size434):
+            _key439 = iprot.readString()
+            _val440 = iprot.readString()
+            self.properties[_key439] = _val440
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -8974,6 +8988,14 @@ class CompactionRequest:
       oprot.writeFieldBegin('runas', TType.STRING, 5)
       oprot.writeString(self.runas)
       oprot.writeFieldEnd()
+    if self.properties is not None:
+      oprot.writeFieldBegin('properties', TType.MAP, 6)
+      oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.properties))
+      for kiter441,viter442 in self.properties.items():
+        oprot.writeString(kiter441)
+        oprot.writeString(viter442)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -8994,6 +9016,7 @@ class CompactionRequest:
     value = (value * 31) ^ hash(self.partitionname)
     value = (value * 31) ^ hash(self.type)
     value = (value * 31) ^ hash(self.runas)
+    value = (value * 31) ^ hash(self.properties)
     return value
 
   def __repr__(self):
@@ -9295,11 +9318,11 @@ class ShowCompactResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.compacts = []
-          (_etype437, _size434) = iprot.readListBegin()
-          for _i438 in xrange(_size434):
-            _elem439 = ShowCompactResponseElement()
-            _elem439.read(iprot)
-            self.compacts.append(_elem439)
+          (_etype446, _size443) = iprot.readListBegin()
+          for _i447 in xrange(_size443):
+            _elem448 = ShowCompactResponseElement()
+            _elem448.read(iprot)
+            self.compacts.append(_elem448)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9316,8 +9339,8 @@ class ShowCompactResponse:
     if self.compacts is not None:
       oprot.writeFieldBegin('compacts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.compacts))
-      for iter440 in self.compacts:
-        iter440.write(oprot)
+      for iter449 in self.compacts:
+        iter449.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -9398,10 +9421,10 @@ class AddDynamicPartitions:
       elif fid == 4:
         if ftype == TType.LIST:
           self.partitionnames = []
-          (_etype444, _size441) = iprot.readListBegin()
-          for _i445 in xrange(_size441):
-            _elem446 = iprot.readString()
-            self.partitionnames.append(_elem446)
+          (_etype453, _size450) = iprot.readListBegin()
+          for _i454 in xrange(_size450):
+            _elem455 = iprot.readString()
+            self.partitionnames.append(_elem455)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9435,8 +9458,8 @@ class AddDynamicPartitions:
     if self.partitionnames is not None:
       oprot.writeFieldBegin('partitionnames', TType.LIST, 4)
       oprot.writeListBegin(TType.STRING, len(self.partitionnames))
-      for iter447 in self.partitionnames:
-        oprot.writeString(iter447)
+      for iter456 in self.partitionnames:
+        oprot.writeString(iter456)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.operationType is not None:
@@ -9722,11 +9745,11 @@ class NotificationEventResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.events = []
-          (_etype451, _size448) = iprot.readListBegin()
-          for _i452 in xrange(_size448):
-            _elem453 = NotificationEvent()
-            _elem453.read(iprot)
-            self.events.append(_elem453)
+          (_etype460, _size457) = iprot.readListBegin()
+          for _i461 in xrange(_size457):
+            _elem462 = NotificationEvent()
+            _elem462.read(iprot)
+            self.events.append(_elem462)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9743,8 +9766,8 @@ class NotificationEventResponse:
     if self.events is not None:
       oprot.writeFieldBegin('events', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.events))
-      for iter454 in self.events:
-        iter454.write(oprot)
+      for iter463 in self.events:
+        iter463.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -9865,10 +9888,10 @@ class InsertEventRequestData:
       if fid == 1:
         if ftype == TType.LIST:
           self.filesAdded = []
-          (_etype458, _size455) = iprot.readListBegin()
-          for _i459 in xrange(_size455):
-            _elem460 = iprot.readString()
-            self.filesAdded.append(_elem460)
+          (_etype467, _size464) = iprot.readListBegin()
+          for _i468 in xrange(_size464):
+            _elem469 = iprot.readString()
+            self.filesAdded.append(_elem469)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9885,8 +9908,8 @@ class InsertEventRequestData:
     if self.filesAdded is not None:
       oprot.writeFieldBegin('filesAdded', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.filesAdded))
-      for iter461 in self.filesAdded:
-        oprot.writeString(iter461)
+      for iter470 in self.filesAdded:
+        oprot.writeString(iter470)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -10039,10 +10062,10 @@ class FireEventRequest:
       elif fid == 5:
         if ftype == TType.LIST:
           self.partitionVals = []
-          (_etype465, _size462) = iprot.readListBegin()
-          for _i466 in xrange(_size462):
-            _elem467 = iprot.readString()
-            self.partitionVals.append(_elem467)
+          (_etype474, _size471) = iprot.readListBegin()
+          for _i475 in xrange(_size471):
+            _elem476 = iprot.readString()
+            self.partitionVals.append(_elem476)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10075,8 +10098,8 @@ class FireEventRequest:
     if self.partitionVals is not None:
       oprot.writeFieldBegin('partitionVals', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.partitionVals))
-      for iter468 in self.partitionVals:
-        oprot.writeString(iter468)
+      for iter477 in self.partitionVals:
+        oprot.writeString(iter477)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -10182,11 +10205,11 @@ class GetAllFunctionsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.functions = []
-          (_etype472, _size469) = iprot.readListBegin()
-          for _i473 in xrange(_size469):
-            _elem474 = Function()
-            _elem474.read(iprot)
-            self.functions.append(_elem474)
+          (_etype481, _size478) = iprot.readListBegin()
+          for _i482 in xrange(_size478):
+            _elem483 = Function()
+            _elem483.read(iprot)
+            self.functions.append(_elem483)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10203,8 +10226,8 @@ class GetAllFunctionsResponse:
     if self.functions is not None:
       oprot.writeFieldBegin('functions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.functions))
-      for iter475 in self.functions:
-        iter475.write(oprot)
+      for iter484 in self.functions:
+        iter484.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb b/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
index 662658c..762b6d7 100644
--- a/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -2015,13 +2015,15 @@ class CompactionRequest
   PARTITIONNAME = 3
   TYPE = 4
   RUNAS = 5
+  PROPERTIES = 6
 
   FIELDS = {
     DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbname'},
     TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tablename'},
     PARTITIONNAME => {:type => ::Thrift::Types::STRING, :name => 'partitionname', :optional => true},
     TYPE => {:type => ::Thrift::Types::I32, :name => 'type', :enum_class => ::CompactionType},
-    RUNAS => {:type => ::Thrift::Types::STRING, :name => 'runas', :optional => true}
+    RUNAS => {:type => ::Thrift::Types::STRING, :name => 'runas', :optional => true},
+    PROPERTIES => {:type => ::Thrift::Types::MAP, :name => 'properties', :key => {:type => ::Thrift::Types::STRING}, :value => {:type => ::Thrift::Types::STRING}, :optional => true}
   }
 
   def struct_fields; FIELDS; end

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index 83d533f..585a647 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -1943,6 +1943,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
   }
 
   @Override
+  @Deprecated
   public void compact(String dbname, String tableName, String partitionName,  CompactionType type)
       throws TException {
     CompactionRequest cr = new CompactionRequest();
@@ -1955,6 +1956,19 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
   }
 
   @Override
+  public void compact(String dbname, String tableName, String partitionName, CompactionType type,
+                      Map<String, String> tblproperties) throws TException {
+    CompactionRequest cr = new CompactionRequest();
+    if (dbname == null) cr.setDbname(DEFAULT_DATABASE_NAME);
+    else cr.setDbname(dbname);
+    cr.setTablename(tableName);
+    if (partitionName != null) cr.setPartitionname(partitionName);
+    cr.setType(type);
+    cr.setProperties(tblproperties);
+    client.compact(cr);
+  }
+
+  @Override
   public ShowCompactResponse showCompactions() throws TException {
     return client.show_compact(new ShowCompactRequest());
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index fc7b70f..6d3b4a5 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -1390,10 +1390,28 @@ public interface IMetaStoreClient {
    * @param type Whether this is a major or minor compaction.
    * @throws TException
    */
+  @Deprecated
   void compact(String dbname, String tableName, String partitionName,  CompactionType type)
       throws TException;
 
   /**
+   * Send a request to compact a table or partition.  This will not block until the compaction is
+   * complete.  It will instead put a request on the queue for that table or partition to be
+   * compacted.  No checking is done on the dbname, tableName, or partitionName to make sure they
+   * refer to valid objects.  It is assumed this has already been done by the caller.
+   * @param dbname Name of the database the table is in.  If null, this will be assumed to be
+   *               'default'.
+   * @param tableName Name of the table to be compacted.  This cannot be null.  If partitionName
+   *                  is null, this must be a non-partitioned table.
+   * @param partitionName Name of the partition to be compacted
+   * @param type Whether this is a major or minor compaction.
+   * @param tblproperties the list of tblproperties to override for this compact. Can be null.
+   * @throws TException
+   */
+  void compact(String dbname, String tableName, String partitionName, CompactionType type,
+               Map<String, String> tblproperties) throws TException;
+
+  /**
    * Get a list of all current compactions.
    * @return List of all current compactions.  This includes compactions waiting to happen,
    * in progress, and finished but waiting to clean the existing files.

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionInfo.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionInfo.java b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionInfo.java
index bea1473..85e0885 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionInfo.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionInfo.java
@@ -37,6 +37,7 @@ public class CompactionInfo implements Comparable<CompactionInfo> {
   String workerId;
   long start;
   public String runAs;
+  public String properties;
   public boolean tooManyAborts = false;
   /**
    * {@code 0} means it wasn't set (e.g. in case of upgrades, since ResultSet.getLong() will return 0 if field is NULL) 
@@ -102,6 +103,7 @@ public class CompactionInfo implements Comparable<CompactionInfo> {
       "partName:" + partName + "," +
       "state:" + state + "," +
       "type:" + type + "," +
+      "properties:" + properties + "," +
       "runAs:" + runAs + "," +
       "tooManyAborts:" + tooManyAborts + "," +
       "highestTxnId:" + highestTxnId;
@@ -120,12 +122,13 @@ public class CompactionInfo implements Comparable<CompactionInfo> {
     fullCi.partName = rs.getString(4);
     fullCi.state = rs.getString(5).charAt(0);//cq_state
     fullCi.type = TxnHandler.dbCompactionType2ThriftType(rs.getString(6).charAt(0));
-    fullCi.workerId = rs.getString(7);
-    fullCi.start = rs.getLong(8);
-    fullCi.runAs = rs.getString(9);
-    fullCi.highestTxnId = rs.getLong(10);
-    fullCi.metaInfo = rs.getBytes(11);
-    fullCi.hadoopJobId = rs.getString(12);
+    fullCi.properties = rs.getString(7);
+    fullCi.workerId = rs.getString(8);
+    fullCi.start = rs.getLong(9);
+    fullCi.runAs = rs.getString(10);
+    fullCi.highestTxnId = rs.getLong(11);
+    fullCi.metaInfo = rs.getBytes(12);
+    fullCi.hadoopJobId = rs.getString(13);
     return fullCi;
   }
   static void insertIntoCompletedCompactions(PreparedStatement pStmt, CompactionInfo ci, long endTime) throws SQLException {
@@ -135,12 +138,13 @@ public class CompactionInfo implements Comparable<CompactionInfo> {
     pStmt.setString(4, ci.partName);
     pStmt.setString(5, Character.toString(ci.state));
     pStmt.setString(6, Character.toString(TxnHandler.thriftCompactionType2DbType(ci.type)));
-    pStmt.setString(7, ci.workerId);
-    pStmt.setLong(8, ci.start);
-    pStmt.setLong(9, endTime);
-    pStmt.setString(10, ci.runAs);
-    pStmt.setLong(11, ci.highestTxnId);
-    pStmt.setBytes(12, ci.metaInfo);
-    pStmt.setString(13, ci.hadoopJobId);
+    pStmt.setString(7, ci.properties);
+    pStmt.setString(8, ci.workerId);
+    pStmt.setLong(9, ci.start);
+    pStmt.setLong(10, endTime);
+    pStmt.setString(11, ci.runAs);
+    pStmt.setLong(12, ci.highestTxnId);
+    pStmt.setBytes(13, ci.metaInfo);
+    pStmt.setString(14, ci.hadoopJobId);
   }
 }


[5/7] hive git commit: HIVE-13354 : Add ability to specify Compaction options per table and per request (Wei Zheng, reviewed by Eugene Koifman)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
index 9e2e883..f2a715a 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -12834,6 +12834,11 @@ void CompactionRequest::__set_runas(const std::string& val) {
 __isset.runas = true;
 }
 
+void CompactionRequest::__set_properties(const std::map<std::string, std::string> & val) {
+  this->properties = val;
+__isset.properties = true;
+}
+
 uint32_t CompactionRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
@@ -12900,6 +12905,29 @@ uint32_t CompactionRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
           xfer += iprot->skip(ftype);
         }
         break;
+      case 6:
+        if (ftype == ::apache::thrift::protocol::T_MAP) {
+          {
+            this->properties.clear();
+            uint32_t _size561;
+            ::apache::thrift::protocol::TType _ktype562;
+            ::apache::thrift::protocol::TType _vtype563;
+            xfer += iprot->readMapBegin(_ktype562, _vtype563, _size561);
+            uint32_t _i565;
+            for (_i565 = 0; _i565 < _size561; ++_i565)
+            {
+              std::string _key566;
+              xfer += iprot->readString(_key566);
+              std::string& _val567 = this->properties[_key566];
+              xfer += iprot->readString(_val567);
+            }
+            xfer += iprot->readMapEnd();
+          }
+          this->__isset.properties = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -12945,6 +12973,20 @@ uint32_t CompactionRequest::write(::apache::thrift::protocol::TProtocol* oprot)
     xfer += oprot->writeString(this->runas);
     xfer += oprot->writeFieldEnd();
   }
+  if (this->__isset.properties) {
+    xfer += oprot->writeFieldBegin("properties", ::apache::thrift::protocol::T_MAP, 6);
+    {
+      xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->properties.size()));
+      std::map<std::string, std::string> ::const_iterator _iter568;
+      for (_iter568 = this->properties.begin(); _iter568 != this->properties.end(); ++_iter568)
+      {
+        xfer += oprot->writeString(_iter568->first);
+        xfer += oprot->writeString(_iter568->second);
+      }
+      xfer += oprot->writeMapEnd();
+    }
+    xfer += oprot->writeFieldEnd();
+  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
@@ -12957,24 +12999,27 @@ void swap(CompactionRequest &a, CompactionRequest &b) {
   swap(a.partitionname, b.partitionname);
   swap(a.type, b.type);
   swap(a.runas, b.runas);
+  swap(a.properties, b.properties);
   swap(a.__isset, b.__isset);
 }
 
-CompactionRequest::CompactionRequest(const CompactionRequest& other561) {
-  dbname = other561.dbname;
-  tablename = other561.tablename;
-  partitionname = other561.partitionname;
-  type = other561.type;
-  runas = other561.runas;
-  __isset = other561.__isset;
-}
-CompactionRequest& CompactionRequest::operator=(const CompactionRequest& other562) {
-  dbname = other562.dbname;
-  tablename = other562.tablename;
-  partitionname = other562.partitionname;
-  type = other562.type;
-  runas = other562.runas;
-  __isset = other562.__isset;
+CompactionRequest::CompactionRequest(const CompactionRequest& other569) {
+  dbname = other569.dbname;
+  tablename = other569.tablename;
+  partitionname = other569.partitionname;
+  type = other569.type;
+  runas = other569.runas;
+  properties = other569.properties;
+  __isset = other569.__isset;
+}
+CompactionRequest& CompactionRequest::operator=(const CompactionRequest& other570) {
+  dbname = other570.dbname;
+  tablename = other570.tablename;
+  partitionname = other570.partitionname;
+  type = other570.type;
+  runas = other570.runas;
+  properties = other570.properties;
+  __isset = other570.__isset;
   return *this;
 }
 void CompactionRequest::printTo(std::ostream& out) const {
@@ -12985,6 +13030,7 @@ void CompactionRequest::printTo(std::ostream& out) const {
   out << ", " << "partitionname="; (__isset.partitionname ? (out << to_string(partitionname)) : (out << "<null>"));
   out << ", " << "type=" << to_string(type);
   out << ", " << "runas="; (__isset.runas ? (out << to_string(runas)) : (out << "<null>"));
+  out << ", " << "properties="; (__isset.properties ? (out << to_string(properties)) : (out << "<null>"));
   out << ")";
 }
 
@@ -13037,11 +13083,11 @@ void swap(ShowCompactRequest &a, ShowCompactRequest &b) {
   (void) b;
 }
 
-ShowCompactRequest::ShowCompactRequest(const ShowCompactRequest& other563) {
-  (void) other563;
+ShowCompactRequest::ShowCompactRequest(const ShowCompactRequest& other571) {
+  (void) other571;
 }
-ShowCompactRequest& ShowCompactRequest::operator=(const ShowCompactRequest& other564) {
-  (void) other564;
+ShowCompactRequest& ShowCompactRequest::operator=(const ShowCompactRequest& other572) {
+  (void) other572;
   return *this;
 }
 void ShowCompactRequest::printTo(std::ostream& out) const {
@@ -13162,9 +13208,9 @@ uint32_t ShowCompactResponseElement::read(::apache::thrift::protocol::TProtocol*
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast565;
-          xfer += iprot->readI32(ecast565);
-          this->type = (CompactionType::type)ecast565;
+          int32_t ecast573;
+          xfer += iprot->readI32(ecast573);
+          this->type = (CompactionType::type)ecast573;
           isset_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -13337,35 +13383,35 @@ void swap(ShowCompactResponseElement &a, ShowCompactResponseElement &b) {
   swap(a.__isset, b.__isset);
 }
 
-ShowCompactResponseElement::ShowCompactResponseElement(const ShowCompactResponseElement& other566) {
-  dbname = other566.dbname;
-  tablename = other566.tablename;
-  partitionname = other566.partitionname;
-  type = other566.type;
-  state = other566.state;
-  workerid = other566.workerid;
-  start = other566.start;
-  runAs = other566.runAs;
-  hightestTxnId = other566.hightestTxnId;
-  metaInfo = other566.metaInfo;
-  endTime = other566.endTime;
-  hadoopJobId = other566.hadoopJobId;
-  __isset = other566.__isset;
-}
-ShowCompactResponseElement& ShowCompactResponseElement::operator=(const ShowCompactResponseElement& other567) {
-  dbname = other567.dbname;
-  tablename = other567.tablename;
-  partitionname = other567.partitionname;
-  type = other567.type;
-  state = other567.state;
-  workerid = other567.workerid;
-  start = other567.start;
-  runAs = other567.runAs;
-  hightestTxnId = other567.hightestTxnId;
-  metaInfo = other567.metaInfo;
-  endTime = other567.endTime;
-  hadoopJobId = other567.hadoopJobId;
-  __isset = other567.__isset;
+ShowCompactResponseElement::ShowCompactResponseElement(const ShowCompactResponseElement& other574) {
+  dbname = other574.dbname;
+  tablename = other574.tablename;
+  partitionname = other574.partitionname;
+  type = other574.type;
+  state = other574.state;
+  workerid = other574.workerid;
+  start = other574.start;
+  runAs = other574.runAs;
+  hightestTxnId = other574.hightestTxnId;
+  metaInfo = other574.metaInfo;
+  endTime = other574.endTime;
+  hadoopJobId = other574.hadoopJobId;
+  __isset = other574.__isset;
+}
+ShowCompactResponseElement& ShowCompactResponseElement::operator=(const ShowCompactResponseElement& other575) {
+  dbname = other575.dbname;
+  tablename = other575.tablename;
+  partitionname = other575.partitionname;
+  type = other575.type;
+  state = other575.state;
+  workerid = other575.workerid;
+  start = other575.start;
+  runAs = other575.runAs;
+  hightestTxnId = other575.hightestTxnId;
+  metaInfo = other575.metaInfo;
+  endTime = other575.endTime;
+  hadoopJobId = other575.hadoopJobId;
+  __isset = other575.__isset;
   return *this;
 }
 void ShowCompactResponseElement::printTo(std::ostream& out) const {
@@ -13421,14 +13467,14 @@ uint32_t ShowCompactResponse::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->compacts.clear();
-            uint32_t _size568;
-            ::apache::thrift::protocol::TType _etype571;
-            xfer += iprot->readListBegin(_etype571, _size568);
-            this->compacts.resize(_size568);
-            uint32_t _i572;
-            for (_i572 = 0; _i572 < _size568; ++_i572)
+            uint32_t _size576;
+            ::apache::thrift::protocol::TType _etype579;
+            xfer += iprot->readListBegin(_etype579, _size576);
+            this->compacts.resize(_size576);
+            uint32_t _i580;
+            for (_i580 = 0; _i580 < _size576; ++_i580)
             {
-              xfer += this->compacts[_i572].read(iprot);
+              xfer += this->compacts[_i580].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13459,10 +13505,10 @@ uint32_t ShowCompactResponse::write(::apache::thrift::protocol::TProtocol* oprot
   xfer += oprot->writeFieldBegin("compacts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->compacts.size()));
-    std::vector<ShowCompactResponseElement> ::const_iterator _iter573;
-    for (_iter573 = this->compacts.begin(); _iter573 != this->compacts.end(); ++_iter573)
+    std::vector<ShowCompactResponseElement> ::const_iterator _iter581;
+    for (_iter581 = this->compacts.begin(); _iter581 != this->compacts.end(); ++_iter581)
     {
-      xfer += (*_iter573).write(oprot);
+      xfer += (*_iter581).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -13478,11 +13524,11 @@ void swap(ShowCompactResponse &a, ShowCompactResponse &b) {
   swap(a.compacts, b.compacts);
 }
 
-ShowCompactResponse::ShowCompactResponse(const ShowCompactResponse& other574) {
-  compacts = other574.compacts;
+ShowCompactResponse::ShowCompactResponse(const ShowCompactResponse& other582) {
+  compacts = other582.compacts;
 }
-ShowCompactResponse& ShowCompactResponse::operator=(const ShowCompactResponse& other575) {
-  compacts = other575.compacts;
+ShowCompactResponse& ShowCompactResponse::operator=(const ShowCompactResponse& other583) {
+  compacts = other583.compacts;
   return *this;
 }
 void ShowCompactResponse::printTo(std::ostream& out) const {
@@ -13571,14 +13617,14 @@ uint32_t AddDynamicPartitions::read(::apache::thrift::protocol::TProtocol* iprot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitionnames.clear();
-            uint32_t _size576;
-            ::apache::thrift::protocol::TType _etype579;
-            xfer += iprot->readListBegin(_etype579, _size576);
-            this->partitionnames.resize(_size576);
-            uint32_t _i580;
-            for (_i580 = 0; _i580 < _size576; ++_i580)
+            uint32_t _size584;
+            ::apache::thrift::protocol::TType _etype587;
+            xfer += iprot->readListBegin(_etype587, _size584);
+            this->partitionnames.resize(_size584);
+            uint32_t _i588;
+            for (_i588 = 0; _i588 < _size584; ++_i588)
             {
-              xfer += iprot->readString(this->partitionnames[_i580]);
+              xfer += iprot->readString(this->partitionnames[_i588]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13589,9 +13635,9 @@ uint32_t AddDynamicPartitions::read(::apache::thrift::protocol::TProtocol* iprot
         break;
       case 5:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast581;
-          xfer += iprot->readI32(ecast581);
-          this->operationType = (DataOperationType::type)ecast581;
+          int32_t ecast589;
+          xfer += iprot->readI32(ecast589);
+          this->operationType = (DataOperationType::type)ecast589;
           this->__isset.operationType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -13637,10 +13683,10 @@ uint32_t AddDynamicPartitions::write(::apache::thrift::protocol::TProtocol* opro
   xfer += oprot->writeFieldBegin("partitionnames", ::apache::thrift::protocol::T_LIST, 4);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionnames.size()));
-    std::vector<std::string> ::const_iterator _iter582;
-    for (_iter582 = this->partitionnames.begin(); _iter582 != this->partitionnames.end(); ++_iter582)
+    std::vector<std::string> ::const_iterator _iter590;
+    for (_iter590 = this->partitionnames.begin(); _iter590 != this->partitionnames.end(); ++_iter590)
     {
-      xfer += oprot->writeString((*_iter582));
+      xfer += oprot->writeString((*_iter590));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13666,21 +13712,21 @@ void swap(AddDynamicPartitions &a, AddDynamicPartitions &b) {
   swap(a.__isset, b.__isset);
 }
 
-AddDynamicPartitions::AddDynamicPartitions(const AddDynamicPartitions& other583) {
-  txnid = other583.txnid;
-  dbname = other583.dbname;
-  tablename = other583.tablename;
-  partitionnames = other583.partitionnames;
-  operationType = other583.operationType;
-  __isset = other583.__isset;
-}
-AddDynamicPartitions& AddDynamicPartitions::operator=(const AddDynamicPartitions& other584) {
-  txnid = other584.txnid;
-  dbname = other584.dbname;
-  tablename = other584.tablename;
-  partitionnames = other584.partitionnames;
-  operationType = other584.operationType;
-  __isset = other584.__isset;
+AddDynamicPartitions::AddDynamicPartitions(const AddDynamicPartitions& other591) {
+  txnid = other591.txnid;
+  dbname = other591.dbname;
+  tablename = other591.tablename;
+  partitionnames = other591.partitionnames;
+  operationType = other591.operationType;
+  __isset = other591.__isset;
+}
+AddDynamicPartitions& AddDynamicPartitions::operator=(const AddDynamicPartitions& other592) {
+  txnid = other592.txnid;
+  dbname = other592.dbname;
+  tablename = other592.tablename;
+  partitionnames = other592.partitionnames;
+  operationType = other592.operationType;
+  __isset = other592.__isset;
   return *this;
 }
 void AddDynamicPartitions::printTo(std::ostream& out) const {
@@ -13786,15 +13832,15 @@ void swap(NotificationEventRequest &a, NotificationEventRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-NotificationEventRequest::NotificationEventRequest(const NotificationEventRequest& other585) {
-  lastEvent = other585.lastEvent;
-  maxEvents = other585.maxEvents;
-  __isset = other585.__isset;
+NotificationEventRequest::NotificationEventRequest(const NotificationEventRequest& other593) {
+  lastEvent = other593.lastEvent;
+  maxEvents = other593.maxEvents;
+  __isset = other593.__isset;
 }
-NotificationEventRequest& NotificationEventRequest::operator=(const NotificationEventRequest& other586) {
-  lastEvent = other586.lastEvent;
-  maxEvents = other586.maxEvents;
-  __isset = other586.__isset;
+NotificationEventRequest& NotificationEventRequest::operator=(const NotificationEventRequest& other594) {
+  lastEvent = other594.lastEvent;
+  maxEvents = other594.maxEvents;
+  __isset = other594.__isset;
   return *this;
 }
 void NotificationEventRequest::printTo(std::ostream& out) const {
@@ -13976,23 +14022,23 @@ void swap(NotificationEvent &a, NotificationEvent &b) {
   swap(a.__isset, b.__isset);
 }
 
-NotificationEvent::NotificationEvent(const NotificationEvent& other587) {
-  eventId = other587.eventId;
-  eventTime = other587.eventTime;
-  eventType = other587.eventType;
-  dbName = other587.dbName;
-  tableName = other587.tableName;
-  message = other587.message;
-  __isset = other587.__isset;
-}
-NotificationEvent& NotificationEvent::operator=(const NotificationEvent& other588) {
-  eventId = other588.eventId;
-  eventTime = other588.eventTime;
-  eventType = other588.eventType;
-  dbName = other588.dbName;
-  tableName = other588.tableName;
-  message = other588.message;
-  __isset = other588.__isset;
+NotificationEvent::NotificationEvent(const NotificationEvent& other595) {
+  eventId = other595.eventId;
+  eventTime = other595.eventTime;
+  eventType = other595.eventType;
+  dbName = other595.dbName;
+  tableName = other595.tableName;
+  message = other595.message;
+  __isset = other595.__isset;
+}
+NotificationEvent& NotificationEvent::operator=(const NotificationEvent& other596) {
+  eventId = other596.eventId;
+  eventTime = other596.eventTime;
+  eventType = other596.eventType;
+  dbName = other596.dbName;
+  tableName = other596.tableName;
+  message = other596.message;
+  __isset = other596.__isset;
   return *this;
 }
 void NotificationEvent::printTo(std::ostream& out) const {
@@ -14042,14 +14088,14 @@ uint32_t NotificationEventResponse::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->events.clear();
-            uint32_t _size589;
-            ::apache::thrift::protocol::TType _etype592;
-            xfer += iprot->readListBegin(_etype592, _size589);
-            this->events.resize(_size589);
-            uint32_t _i593;
-            for (_i593 = 0; _i593 < _size589; ++_i593)
+            uint32_t _size597;
+            ::apache::thrift::protocol::TType _etype600;
+            xfer += iprot->readListBegin(_etype600, _size597);
+            this->events.resize(_size597);
+            uint32_t _i601;
+            for (_i601 = 0; _i601 < _size597; ++_i601)
             {
-              xfer += this->events[_i593].read(iprot);
+              xfer += this->events[_i601].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14080,10 +14126,10 @@ uint32_t NotificationEventResponse::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("events", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->events.size()));
-    std::vector<NotificationEvent> ::const_iterator _iter594;
-    for (_iter594 = this->events.begin(); _iter594 != this->events.end(); ++_iter594)
+    std::vector<NotificationEvent> ::const_iterator _iter602;
+    for (_iter602 = this->events.begin(); _iter602 != this->events.end(); ++_iter602)
     {
-      xfer += (*_iter594).write(oprot);
+      xfer += (*_iter602).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -14099,11 +14145,11 @@ void swap(NotificationEventResponse &a, NotificationEventResponse &b) {
   swap(a.events, b.events);
 }
 
-NotificationEventResponse::NotificationEventResponse(const NotificationEventResponse& other595) {
-  events = other595.events;
+NotificationEventResponse::NotificationEventResponse(const NotificationEventResponse& other603) {
+  events = other603.events;
 }
-NotificationEventResponse& NotificationEventResponse::operator=(const NotificationEventResponse& other596) {
-  events = other596.events;
+NotificationEventResponse& NotificationEventResponse::operator=(const NotificationEventResponse& other604) {
+  events = other604.events;
   return *this;
 }
 void NotificationEventResponse::printTo(std::ostream& out) const {
@@ -14185,11 +14231,11 @@ void swap(CurrentNotificationEventId &a, CurrentNotificationEventId &b) {
   swap(a.eventId, b.eventId);
 }
 
-CurrentNotificationEventId::CurrentNotificationEventId(const CurrentNotificationEventId& other597) {
-  eventId = other597.eventId;
+CurrentNotificationEventId::CurrentNotificationEventId(const CurrentNotificationEventId& other605) {
+  eventId = other605.eventId;
 }
-CurrentNotificationEventId& CurrentNotificationEventId::operator=(const CurrentNotificationEventId& other598) {
-  eventId = other598.eventId;
+CurrentNotificationEventId& CurrentNotificationEventId::operator=(const CurrentNotificationEventId& other606) {
+  eventId = other606.eventId;
   return *this;
 }
 void CurrentNotificationEventId::printTo(std::ostream& out) const {
@@ -14234,14 +14280,14 @@ uint32_t InsertEventRequestData::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->filesAdded.clear();
-            uint32_t _size599;
-            ::apache::thrift::protocol::TType _etype602;
-            xfer += iprot->readListBegin(_etype602, _size599);
-            this->filesAdded.resize(_size599);
-            uint32_t _i603;
-            for (_i603 = 0; _i603 < _size599; ++_i603)
+            uint32_t _size607;
+            ::apache::thrift::protocol::TType _etype610;
+            xfer += iprot->readListBegin(_etype610, _size607);
+            this->filesAdded.resize(_size607);
+            uint32_t _i611;
+            for (_i611 = 0; _i611 < _size607; ++_i611)
             {
-              xfer += iprot->readString(this->filesAdded[_i603]);
+              xfer += iprot->readString(this->filesAdded[_i611]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14272,10 +14318,10 @@ uint32_t InsertEventRequestData::write(::apache::thrift::protocol::TProtocol* op
   xfer += oprot->writeFieldBegin("filesAdded", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->filesAdded.size()));
-    std::vector<std::string> ::const_iterator _iter604;
-    for (_iter604 = this->filesAdded.begin(); _iter604 != this->filesAdded.end(); ++_iter604)
+    std::vector<std::string> ::const_iterator _iter612;
+    for (_iter612 = this->filesAdded.begin(); _iter612 != this->filesAdded.end(); ++_iter612)
     {
-      xfer += oprot->writeString((*_iter604));
+      xfer += oprot->writeString((*_iter612));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14291,11 +14337,11 @@ void swap(InsertEventRequestData &a, InsertEventRequestData &b) {
   swap(a.filesAdded, b.filesAdded);
 }
 
-InsertEventRequestData::InsertEventRequestData(const InsertEventRequestData& other605) {
-  filesAdded = other605.filesAdded;
+InsertEventRequestData::InsertEventRequestData(const InsertEventRequestData& other613) {
+  filesAdded = other613.filesAdded;
 }
-InsertEventRequestData& InsertEventRequestData::operator=(const InsertEventRequestData& other606) {
-  filesAdded = other606.filesAdded;
+InsertEventRequestData& InsertEventRequestData::operator=(const InsertEventRequestData& other614) {
+  filesAdded = other614.filesAdded;
   return *this;
 }
 void InsertEventRequestData::printTo(std::ostream& out) const {
@@ -14375,13 +14421,13 @@ void swap(FireEventRequestData &a, FireEventRequestData &b) {
   swap(a.__isset, b.__isset);
 }
 
-FireEventRequestData::FireEventRequestData(const FireEventRequestData& other607) {
-  insertData = other607.insertData;
-  __isset = other607.__isset;
+FireEventRequestData::FireEventRequestData(const FireEventRequestData& other615) {
+  insertData = other615.insertData;
+  __isset = other615.__isset;
 }
-FireEventRequestData& FireEventRequestData::operator=(const FireEventRequestData& other608) {
-  insertData = other608.insertData;
-  __isset = other608.__isset;
+FireEventRequestData& FireEventRequestData::operator=(const FireEventRequestData& other616) {
+  insertData = other616.insertData;
+  __isset = other616.__isset;
   return *this;
 }
 void FireEventRequestData::printTo(std::ostream& out) const {
@@ -14478,14 +14524,14 @@ uint32_t FireEventRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitionVals.clear();
-            uint32_t _size609;
-            ::apache::thrift::protocol::TType _etype612;
-            xfer += iprot->readListBegin(_etype612, _size609);
-            this->partitionVals.resize(_size609);
-            uint32_t _i613;
-            for (_i613 = 0; _i613 < _size609; ++_i613)
+            uint32_t _size617;
+            ::apache::thrift::protocol::TType _etype620;
+            xfer += iprot->readListBegin(_etype620, _size617);
+            this->partitionVals.resize(_size617);
+            uint32_t _i621;
+            for (_i621 = 0; _i621 < _size617; ++_i621)
             {
-              xfer += iprot->readString(this->partitionVals[_i613]);
+              xfer += iprot->readString(this->partitionVals[_i621]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14537,10 +14583,10 @@ uint32_t FireEventRequest::write(::apache::thrift::protocol::TProtocol* oprot) c
     xfer += oprot->writeFieldBegin("partitionVals", ::apache::thrift::protocol::T_LIST, 5);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionVals.size()));
-      std::vector<std::string> ::const_iterator _iter614;
-      for (_iter614 = this->partitionVals.begin(); _iter614 != this->partitionVals.end(); ++_iter614)
+      std::vector<std::string> ::const_iterator _iter622;
+      for (_iter622 = this->partitionVals.begin(); _iter622 != this->partitionVals.end(); ++_iter622)
       {
-        xfer += oprot->writeString((*_iter614));
+        xfer += oprot->writeString((*_iter622));
       }
       xfer += oprot->writeListEnd();
     }
@@ -14561,21 +14607,21 @@ void swap(FireEventRequest &a, FireEventRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-FireEventRequest::FireEventRequest(const FireEventRequest& other615) {
-  successful = other615.successful;
-  data = other615.data;
-  dbName = other615.dbName;
-  tableName = other615.tableName;
-  partitionVals = other615.partitionVals;
-  __isset = other615.__isset;
-}
-FireEventRequest& FireEventRequest::operator=(const FireEventRequest& other616) {
-  successful = other616.successful;
-  data = other616.data;
-  dbName = other616.dbName;
-  tableName = other616.tableName;
-  partitionVals = other616.partitionVals;
-  __isset = other616.__isset;
+FireEventRequest::FireEventRequest(const FireEventRequest& other623) {
+  successful = other623.successful;
+  data = other623.data;
+  dbName = other623.dbName;
+  tableName = other623.tableName;
+  partitionVals = other623.partitionVals;
+  __isset = other623.__isset;
+}
+FireEventRequest& FireEventRequest::operator=(const FireEventRequest& other624) {
+  successful = other624.successful;
+  data = other624.data;
+  dbName = other624.dbName;
+  tableName = other624.tableName;
+  partitionVals = other624.partitionVals;
+  __isset = other624.__isset;
   return *this;
 }
 void FireEventRequest::printTo(std::ostream& out) const {
@@ -14638,11 +14684,11 @@ void swap(FireEventResponse &a, FireEventResponse &b) {
   (void) b;
 }
 
-FireEventResponse::FireEventResponse(const FireEventResponse& other617) {
-  (void) other617;
+FireEventResponse::FireEventResponse(const FireEventResponse& other625) {
+  (void) other625;
 }
-FireEventResponse& FireEventResponse::operator=(const FireEventResponse& other618) {
-  (void) other618;
+FireEventResponse& FireEventResponse::operator=(const FireEventResponse& other626) {
+  (void) other626;
   return *this;
 }
 void FireEventResponse::printTo(std::ostream& out) const {
@@ -14686,14 +14732,14 @@ uint32_t GetAllFunctionsResponse::read(::apache::thrift::protocol::TProtocol* ip
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->functions.clear();
-            uint32_t _size619;
-            ::apache::thrift::protocol::TType _etype622;
-            xfer += iprot->readListBegin(_etype622, _size619);
-            this->functions.resize(_size619);
-            uint32_t _i623;
-            for (_i623 = 0; _i623 < _size619; ++_i623)
+            uint32_t _size627;
+            ::apache::thrift::protocol::TType _etype630;
+            xfer += iprot->readListBegin(_etype630, _size627);
+            this->functions.resize(_size627);
+            uint32_t _i631;
+            for (_i631 = 0; _i631 < _size627; ++_i631)
             {
-              xfer += this->functions[_i623].read(iprot);
+              xfer += this->functions[_i631].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14723,10 +14769,10 @@ uint32_t GetAllFunctionsResponse::write(::apache::thrift::protocol::TProtocol* o
     xfer += oprot->writeFieldBegin("functions", ::apache::thrift::protocol::T_LIST, 1);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->functions.size()));
-      std::vector<Function> ::const_iterator _iter624;
-      for (_iter624 = this->functions.begin(); _iter624 != this->functions.end(); ++_iter624)
+      std::vector<Function> ::const_iterator _iter632;
+      for (_iter632 = this->functions.begin(); _iter632 != this->functions.end(); ++_iter632)
       {
-        xfer += (*_iter624).write(oprot);
+        xfer += (*_iter632).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -14743,13 +14789,13 @@ void swap(GetAllFunctionsResponse &a, GetAllFunctionsResponse &b) {
   swap(a.__isset, b.__isset);
 }
 
-GetAllFunctionsResponse::GetAllFunctionsResponse(const GetAllFunctionsResponse& other625) {
-  functions = other625.functions;
-  __isset = other625.__isset;
+GetAllFunctionsResponse::GetAllFunctionsResponse(const GetAllFunctionsResponse& other633) {
+  functions = other633.functions;
+  __isset = other633.__isset;
 }
-GetAllFunctionsResponse& GetAllFunctionsResponse::operator=(const GetAllFunctionsResponse& other626) {
-  functions = other626.functions;
-  __isset = other626.__isset;
+GetAllFunctionsResponse& GetAllFunctionsResponse::operator=(const GetAllFunctionsResponse& other634) {
+  functions = other634.functions;
+  __isset = other634.__isset;
   return *this;
 }
 void GetAllFunctionsResponse::printTo(std::ostream& out) const {
@@ -14829,13 +14875,13 @@ void swap(MetaException &a, MetaException &b) {
   swap(a.__isset, b.__isset);
 }
 
-MetaException::MetaException(const MetaException& other627) : TException() {
-  message = other627.message;
-  __isset = other627.__isset;
+MetaException::MetaException(const MetaException& other635) : TException() {
+  message = other635.message;
+  __isset = other635.__isset;
 }
-MetaException& MetaException::operator=(const MetaException& other628) {
-  message = other628.message;
-  __isset = other628.__isset;
+MetaException& MetaException::operator=(const MetaException& other636) {
+  message = other636.message;
+  __isset = other636.__isset;
   return *this;
 }
 void MetaException::printTo(std::ostream& out) const {
@@ -14926,13 +14972,13 @@ void swap(UnknownTableException &a, UnknownTableException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownTableException::UnknownTableException(const UnknownTableException& other629) : TException() {
-  message = other629.message;
-  __isset = other629.__isset;
+UnknownTableException::UnknownTableException(const UnknownTableException& other637) : TException() {
+  message = other637.message;
+  __isset = other637.__isset;
 }
-UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other630) {
-  message = other630.message;
-  __isset = other630.__isset;
+UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other638) {
+  message = other638.message;
+  __isset = other638.__isset;
   return *this;
 }
 void UnknownTableException::printTo(std::ostream& out) const {
@@ -15023,13 +15069,13 @@ void swap(UnknownDBException &a, UnknownDBException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownDBException::UnknownDBException(const UnknownDBException& other631) : TException() {
-  message = other631.message;
-  __isset = other631.__isset;
+UnknownDBException::UnknownDBException(const UnknownDBException& other639) : TException() {
+  message = other639.message;
+  __isset = other639.__isset;
 }
-UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other632) {
-  message = other632.message;
-  __isset = other632.__isset;
+UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other640) {
+  message = other640.message;
+  __isset = other640.__isset;
   return *this;
 }
 void UnknownDBException::printTo(std::ostream& out) const {
@@ -15120,13 +15166,13 @@ void swap(AlreadyExistsException &a, AlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other633) : TException() {
-  message = other633.message;
-  __isset = other633.__isset;
+AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other641) : TException() {
+  message = other641.message;
+  __isset = other641.__isset;
 }
-AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other634) {
-  message = other634.message;
-  __isset = other634.__isset;
+AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other642) {
+  message = other642.message;
+  __isset = other642.__isset;
   return *this;
 }
 void AlreadyExistsException::printTo(std::ostream& out) const {
@@ -15217,13 +15263,13 @@ void swap(InvalidPartitionException &a, InvalidPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other635) : TException() {
-  message = other635.message;
-  __isset = other635.__isset;
+InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other643) : TException() {
+  message = other643.message;
+  __isset = other643.__isset;
 }
-InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other636) {
-  message = other636.message;
-  __isset = other636.__isset;
+InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other644) {
+  message = other644.message;
+  __isset = other644.__isset;
   return *this;
 }
 void InvalidPartitionException::printTo(std::ostream& out) const {
@@ -15314,13 +15360,13 @@ void swap(UnknownPartitionException &a, UnknownPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other637) : TException() {
-  message = other637.message;
-  __isset = other637.__isset;
+UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other645) : TException() {
+  message = other645.message;
+  __isset = other645.__isset;
 }
-UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other638) {
-  message = other638.message;
-  __isset = other638.__isset;
+UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other646) {
+  message = other646.message;
+  __isset = other646.__isset;
   return *this;
 }
 void UnknownPartitionException::printTo(std::ostream& out) const {
@@ -15411,13 +15457,13 @@ void swap(InvalidObjectException &a, InvalidObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidObjectException::InvalidObjectException(const InvalidObjectException& other639) : TException() {
-  message = other639.message;
-  __isset = other639.__isset;
+InvalidObjectException::InvalidObjectException(const InvalidObjectException& other647) : TException() {
+  message = other647.message;
+  __isset = other647.__isset;
 }
-InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other640) {
-  message = other640.message;
-  __isset = other640.__isset;
+InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other648) {
+  message = other648.message;
+  __isset = other648.__isset;
   return *this;
 }
 void InvalidObjectException::printTo(std::ostream& out) const {
@@ -15508,13 +15554,13 @@ void swap(NoSuchObjectException &a, NoSuchObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other641) : TException() {
-  message = other641.message;
-  __isset = other641.__isset;
+NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other649) : TException() {
+  message = other649.message;
+  __isset = other649.__isset;
 }
-NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other642) {
-  message = other642.message;
-  __isset = other642.__isset;
+NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other650) {
+  message = other650.message;
+  __isset = other650.__isset;
   return *this;
 }
 void NoSuchObjectException::printTo(std::ostream& out) const {
@@ -15605,13 +15651,13 @@ void swap(IndexAlreadyExistsException &a, IndexAlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-IndexAlreadyExistsException::IndexAlreadyExistsException(const IndexAlreadyExistsException& other643) : TException() {
-  message = other643.message;
-  __isset = other643.__isset;
+IndexAlreadyExistsException::IndexAlreadyExistsException(const IndexAlreadyExistsException& other651) : TException() {
+  message = other651.message;
+  __isset = other651.__isset;
 }
-IndexAlreadyExistsException& IndexAlreadyExistsException::operator=(const IndexAlreadyExistsException& other644) {
-  message = other644.message;
-  __isset = other644.__isset;
+IndexAlreadyExistsException& IndexAlreadyExistsException::operator=(const IndexAlreadyExistsException& other652) {
+  message = other652.message;
+  __isset = other652.__isset;
   return *this;
 }
 void IndexAlreadyExistsException::printTo(std::ostream& out) const {
@@ -15702,13 +15748,13 @@ void swap(InvalidOperationException &a, InvalidOperationException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidOperationException::InvalidOperationException(const InvalidOperationException& other645) : TException() {
-  message = other645.message;
-  __isset = other645.__isset;
+InvalidOperationException::InvalidOperationException(const InvalidOperationException& other653) : TException() {
+  message = other653.message;
+  __isset = other653.__isset;
 }
-InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other646) {
-  message = other646.message;
-  __isset = other646.__isset;
+InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other654) {
+  message = other654.message;
+  __isset = other654.__isset;
   return *this;
 }
 void InvalidOperationException::printTo(std::ostream& out) const {
@@ -15799,13 +15845,13 @@ void swap(ConfigValSecurityException &a, ConfigValSecurityException &b) {
   swap(a.__isset, b.__isset);
 }
 
-ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other647) : TException() {
-  message = other647.message;
-  __isset = other647.__isset;
+ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other655) : TException() {
+  message = other655.message;
+  __isset = other655.__isset;
 }
-ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other648) {
-  message = other648.message;
-  __isset = other648.__isset;
+ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other656) {
+  message = other656.message;
+  __isset = other656.__isset;
   return *this;
 }
 void ConfigValSecurityException::printTo(std::ostream& out) const {
@@ -15896,13 +15942,13 @@ void swap(InvalidInputException &a, InvalidInputException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidInputException::InvalidInputException(const InvalidInputException& other649) : TException() {
-  message = other649.message;
-  __isset = other649.__isset;
+InvalidInputException::InvalidInputException(const InvalidInputException& other657) : TException() {
+  message = other657.message;
+  __isset = other657.__isset;
 }
-InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other650) {
-  message = other650.message;
-  __isset = other650.__isset;
+InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other658) {
+  message = other658.message;
+  __isset = other658.__isset;
   return *this;
 }
 void InvalidInputException::printTo(std::ostream& out) const {
@@ -15993,13 +16039,13 @@ void swap(NoSuchTxnException &a, NoSuchTxnException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other651) : TException() {
-  message = other651.message;
-  __isset = other651.__isset;
+NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other659) : TException() {
+  message = other659.message;
+  __isset = other659.__isset;
 }
-NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other652) {
-  message = other652.message;
-  __isset = other652.__isset;
+NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other660) {
+  message = other660.message;
+  __isset = other660.__isset;
   return *this;
 }
 void NoSuchTxnException::printTo(std::ostream& out) const {
@@ -16090,13 +16136,13 @@ void swap(TxnAbortedException &a, TxnAbortedException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnAbortedException::TxnAbortedException(const TxnAbortedException& other653) : TException() {
-  message = other653.message;
-  __isset = other653.__isset;
+TxnAbortedException::TxnAbortedException(const TxnAbortedException& other661) : TException() {
+  message = other661.message;
+  __isset = other661.__isset;
 }
-TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other654) {
-  message = other654.message;
-  __isset = other654.__isset;
+TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other662) {
+  message = other662.message;
+  __isset = other662.__isset;
   return *this;
 }
 void TxnAbortedException::printTo(std::ostream& out) const {
@@ -16187,13 +16233,13 @@ void swap(TxnOpenException &a, TxnOpenException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnOpenException::TxnOpenException(const TxnOpenException& other655) : TException() {
-  message = other655.message;
-  __isset = other655.__isset;
+TxnOpenException::TxnOpenException(const TxnOpenException& other663) : TException() {
+  message = other663.message;
+  __isset = other663.__isset;
 }
-TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other656) {
-  message = other656.message;
-  __isset = other656.__isset;
+TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other664) {
+  message = other664.message;
+  __isset = other664.__isset;
   return *this;
 }
 void TxnOpenException::printTo(std::ostream& out) const {
@@ -16284,13 +16330,13 @@ void swap(NoSuchLockException &a, NoSuchLockException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchLockException::NoSuchLockException(const NoSuchLockException& other657) : TException() {
-  message = other657.message;
-  __isset = other657.__isset;
+NoSuchLockException::NoSuchLockException(const NoSuchLockException& other665) : TException() {
+  message = other665.message;
+  __isset = other665.__isset;
 }
-NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other658) {
-  message = other658.message;
-  __isset = other658.__isset;
+NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other666) {
+  message = other666.message;
+  __isset = other666.__isset;
   return *this;
 }
 void NoSuchLockException::printTo(std::ostream& out) const {

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
index 390db33..4574214 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -5202,9 +5202,10 @@ inline std::ostream& operator<<(std::ostream& out, const HeartbeatTxnRangeRespon
 }
 
 typedef struct _CompactionRequest__isset {
-  _CompactionRequest__isset() : partitionname(false), runas(false) {}
+  _CompactionRequest__isset() : partitionname(false), runas(false), properties(false) {}
   bool partitionname :1;
   bool runas :1;
+  bool properties :1;
 } _CompactionRequest__isset;
 
 class CompactionRequest {
@@ -5221,6 +5222,7 @@ class CompactionRequest {
   std::string partitionname;
   CompactionType::type type;
   std::string runas;
+  std::map<std::string, std::string>  properties;
 
   _CompactionRequest__isset __isset;
 
@@ -5234,6 +5236,8 @@ class CompactionRequest {
 
   void __set_runas(const std::string& val);
 
+  void __set_properties(const std::map<std::string, std::string> & val);
+
   bool operator == (const CompactionRequest & rhs) const
   {
     if (!(dbname == rhs.dbname))
@@ -5250,6 +5254,10 @@ class CompactionRequest {
       return false;
     else if (__isset.runas && !(runas == rhs.runas))
       return false;
+    if (__isset.properties != rhs.__isset.properties)
+      return false;
+    else if (__isset.properties && !(properties == rhs.properties))
+      return false;
     return true;
   }
   bool operator != (const CompactionRequest &rhs) const {

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
index 7543ef4..9350b2d 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
@@ -727,13 +727,13 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
           case 4: // PARTITIONNAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list500 = iprot.readListBegin();
-                struct.partitionnames = new ArrayList<String>(_list500.size);
-                String _elem501;
-                for (int _i502 = 0; _i502 < _list500.size; ++_i502)
+                org.apache.thrift.protocol.TList _list510 = iprot.readListBegin();
+                struct.partitionnames = new ArrayList<String>(_list510.size);
+                String _elem511;
+                for (int _i512 = 0; _i512 < _list510.size; ++_i512)
                 {
-                  _elem501 = iprot.readString();
-                  struct.partitionnames.add(_elem501);
+                  _elem511 = iprot.readString();
+                  struct.partitionnames.add(_elem511);
                 }
                 iprot.readListEnd();
               }
@@ -780,9 +780,9 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
         oprot.writeFieldBegin(PARTITIONNAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partitionnames.size()));
-          for (String _iter503 : struct.partitionnames)
+          for (String _iter513 : struct.partitionnames)
           {
-            oprot.writeString(_iter503);
+            oprot.writeString(_iter513);
           }
           oprot.writeListEnd();
         }
@@ -817,9 +817,9 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
       oprot.writeString(struct.tablename);
       {
         oprot.writeI32(struct.partitionnames.size());
-        for (String _iter504 : struct.partitionnames)
+        for (String _iter514 : struct.partitionnames)
         {
-          oprot.writeString(_iter504);
+          oprot.writeString(_iter514);
         }
       }
       BitSet optionals = new BitSet();
@@ -842,13 +842,13 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
       struct.tablename = iprot.readString();
       struct.setTablenameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list505 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.partitionnames = new ArrayList<String>(_list505.size);
-        String _elem506;
-        for (int _i507 = 0; _i507 < _list505.size; ++_i507)
+        org.apache.thrift.protocol.TList _list515 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.partitionnames = new ArrayList<String>(_list515.size);
+        String _elem516;
+        for (int _i517 = 0; _i517 < _list515.size; ++_i517)
         {
-          _elem506 = iprot.readString();
-          struct.partitionnames.add(_elem506);
+          _elem516 = iprot.readString();
+          struct.partitionnames.add(_elem516);
         }
       }
       struct.setPartitionnamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
index e028ecb..638ba73 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
@@ -43,6 +43,7 @@ public class CompactionRequest implements org.apache.thrift.TBase<CompactionRequ
   private static final org.apache.thrift.protocol.TField PARTITIONNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("partitionname", org.apache.thrift.protocol.TType.STRING, (short)3);
   private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.I32, (short)4);
   private static final org.apache.thrift.protocol.TField RUNAS_FIELD_DESC = new org.apache.thrift.protocol.TField("runas", org.apache.thrift.protocol.TType.STRING, (short)5);
+  private static final org.apache.thrift.protocol.TField PROPERTIES_FIELD_DESC = new org.apache.thrift.protocol.TField("properties", org.apache.thrift.protocol.TType.MAP, (short)6);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -55,6 +56,7 @@ public class CompactionRequest implements org.apache.thrift.TBase<CompactionRequ
   private String partitionname; // optional
   private CompactionType type; // required
   private String runas; // optional
+  private Map<String,String> properties; // 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 {
@@ -66,7 +68,8 @@ public class CompactionRequest implements org.apache.thrift.TBase<CompactionRequ
      * @see CompactionType
      */
     TYPE((short)4, "type"),
-    RUNAS((short)5, "runas");
+    RUNAS((short)5, "runas"),
+    PROPERTIES((short)6, "properties");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -91,6 +94,8 @@ public class CompactionRequest implements org.apache.thrift.TBase<CompactionRequ
           return TYPE;
         case 5: // RUNAS
           return RUNAS;
+        case 6: // PROPERTIES
+          return PROPERTIES;
         default:
           return null;
       }
@@ -131,7 +136,7 @@ public class CompactionRequest implements org.apache.thrift.TBase<CompactionRequ
   }
 
   // isset id assignments
-  private static final _Fields optionals[] = {_Fields.PARTITIONNAME,_Fields.RUNAS};
+  private static final _Fields optionals[] = {_Fields.PARTITIONNAME,_Fields.RUNAS,_Fields.PROPERTIES};
   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);
@@ -145,6 +150,10 @@ public class CompactionRequest implements org.apache.thrift.TBase<CompactionRequ
         new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, CompactionType.class)));
     tmpMap.put(_Fields.RUNAS, new org.apache.thrift.meta_data.FieldMetaData("runas", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.PROPERTIES, new org.apache.thrift.meta_data.FieldMetaData("properties", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CompactionRequest.class, metaDataMap);
   }
@@ -182,6 +191,10 @@ public class CompactionRequest implements org.apache.thrift.TBase<CompactionRequ
     if (other.isSetRunas()) {
       this.runas = other.runas;
     }
+    if (other.isSetProperties()) {
+      Map<String,String> __this__properties = new HashMap<String,String>(other.properties);
+      this.properties = __this__properties;
+    }
   }
 
   public CompactionRequest deepCopy() {
@@ -195,6 +208,7 @@ public class CompactionRequest implements org.apache.thrift.TBase<CompactionRequ
     this.partitionname = null;
     this.type = null;
     this.runas = null;
+    this.properties = null;
   }
 
   public String getDbname() {
@@ -320,6 +334,40 @@ public class CompactionRequest implements org.apache.thrift.TBase<CompactionRequ
     }
   }
 
+  public int getPropertiesSize() {
+    return (this.properties == null) ? 0 : this.properties.size();
+  }
+
+  public void putToProperties(String key, String val) {
+    if (this.properties == null) {
+      this.properties = new HashMap<String,String>();
+    }
+    this.properties.put(key, val);
+  }
+
+  public Map<String,String> getProperties() {
+    return this.properties;
+  }
+
+  public void setProperties(Map<String,String> properties) {
+    this.properties = properties;
+  }
+
+  public void unsetProperties() {
+    this.properties = null;
+  }
+
+  /** Returns true if field properties is set (has been assigned a value) and false otherwise */
+  public boolean isSetProperties() {
+    return this.properties != null;
+  }
+
+  public void setPropertiesIsSet(boolean value) {
+    if (!value) {
+      this.properties = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case DBNAME:
@@ -362,6 +410,14 @@ public class CompactionRequest implements org.apache.thrift.TBase<CompactionRequ
       }
       break;
 
+    case PROPERTIES:
+      if (value == null) {
+        unsetProperties();
+      } else {
+        setProperties((Map<String,String>)value);
+      }
+      break;
+
     }
   }
 
@@ -382,6 +438,9 @@ public class CompactionRequest implements org.apache.thrift.TBase<CompactionRequ
     case RUNAS:
       return getRunas();
 
+    case PROPERTIES:
+      return getProperties();
+
     }
     throw new IllegalStateException();
   }
@@ -403,6 +462,8 @@ public class CompactionRequest implements org.apache.thrift.TBase<CompactionRequ
       return isSetType();
     case RUNAS:
       return isSetRunas();
+    case PROPERTIES:
+      return isSetProperties();
     }
     throw new IllegalStateException();
   }
@@ -465,6 +526,15 @@ public class CompactionRequest implements org.apache.thrift.TBase<CompactionRequ
         return false;
     }
 
+    boolean this_present_properties = true && this.isSetProperties();
+    boolean that_present_properties = true && that.isSetProperties();
+    if (this_present_properties || that_present_properties) {
+      if (!(this_present_properties && that_present_properties))
+        return false;
+      if (!this.properties.equals(that.properties))
+        return false;
+    }
+
     return true;
   }
 
@@ -497,6 +567,11 @@ public class CompactionRequest implements org.apache.thrift.TBase<CompactionRequ
     if (present_runas)
       list.add(runas);
 
+    boolean present_properties = true && (isSetProperties());
+    list.add(present_properties);
+    if (present_properties)
+      list.add(properties);
+
     return list.hashCode();
   }
 
@@ -558,6 +633,16 @@ public class CompactionRequest implements org.apache.thrift.TBase<CompactionRequ
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetProperties()).compareTo(other.isSetProperties());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetProperties()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.properties, other.properties);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -621,6 +706,16 @@ public class CompactionRequest implements org.apache.thrift.TBase<CompactionRequ
       }
       first = false;
     }
+    if (isSetProperties()) {
+      if (!first) sb.append(", ");
+      sb.append("properties:");
+      if (this.properties == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.properties);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -716,6 +811,26 @@ public class CompactionRequest implements org.apache.thrift.TBase<CompactionRequ
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 6: // PROPERTIES
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map492 = iprot.readMapBegin();
+                struct.properties = new HashMap<String,String>(2*_map492.size);
+                String _key493;
+                String _val494;
+                for (int _i495 = 0; _i495 < _map492.size; ++_i495)
+                {
+                  _key493 = iprot.readString();
+                  _val494 = iprot.readString();
+                  struct.properties.put(_key493, _val494);
+                }
+                iprot.readMapEnd();
+              }
+              struct.setPropertiesIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -758,6 +873,21 @@ public class CompactionRequest implements org.apache.thrift.TBase<CompactionRequ
           oprot.writeFieldEnd();
         }
       }
+      if (struct.properties != null) {
+        if (struct.isSetProperties()) {
+          oprot.writeFieldBegin(PROPERTIES_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.properties.size()));
+            for (Map.Entry<String, String> _iter496 : struct.properties.entrySet())
+            {
+              oprot.writeString(_iter496.getKey());
+              oprot.writeString(_iter496.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -785,13 +915,26 @@ public class CompactionRequest implements org.apache.thrift.TBase<CompactionRequ
       if (struct.isSetRunas()) {
         optionals.set(1);
       }
-      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetProperties()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
       if (struct.isSetPartitionname()) {
         oprot.writeString(struct.partitionname);
       }
       if (struct.isSetRunas()) {
         oprot.writeString(struct.runas);
       }
+      if (struct.isSetProperties()) {
+        {
+          oprot.writeI32(struct.properties.size());
+          for (Map.Entry<String, String> _iter497 : struct.properties.entrySet())
+          {
+            oprot.writeString(_iter497.getKey());
+            oprot.writeString(_iter497.getValue());
+          }
+        }
+      }
     }
 
     @Override
@@ -803,7 +946,7 @@ public class CompactionRequest implements org.apache.thrift.TBase<CompactionRequ
       struct.setTablenameIsSet(true);
       struct.type = org.apache.hadoop.hive.metastore.api.CompactionType.findByValue(iprot.readI32());
       struct.setTypeIsSet(true);
-      BitSet incoming = iprot.readBitSet(2);
+      BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         struct.partitionname = iprot.readString();
         struct.setPartitionnameIsSet(true);
@@ -812,6 +955,21 @@ public class CompactionRequest implements org.apache.thrift.TBase<CompactionRequ
         struct.runas = iprot.readString();
         struct.setRunasIsSet(true);
       }
+      if (incoming.get(2)) {
+        {
+          org.apache.thrift.protocol.TMap _map498 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.properties = new HashMap<String,String>(2*_map498.size);
+          String _key499;
+          String _val500;
+          for (int _i501 = 0; _i501 < _map498.size; ++_i501)
+          {
+            _key499 = iprot.readString();
+            _val500 = iprot.readString();
+            struct.properties.put(_key499, _val500);
+          }
+        }
+        struct.setPropertiesIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
index 551c46a..a3a6171 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
@@ -713,13 +713,13 @@ public class FireEventRequest implements org.apache.thrift.TBase<FireEventReques
           case 5: // PARTITION_VALS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list524 = iprot.readListBegin();
-                struct.partitionVals = new ArrayList<String>(_list524.size);
-                String _elem525;
-                for (int _i526 = 0; _i526 < _list524.size; ++_i526)
+                org.apache.thrift.protocol.TList _list534 = iprot.readListBegin();
+                struct.partitionVals = new ArrayList<String>(_list534.size);
+                String _elem535;
+                for (int _i536 = 0; _i536 < _list534.size; ++_i536)
                 {
-                  _elem525 = iprot.readString();
-                  struct.partitionVals.add(_elem525);
+                  _elem535 = iprot.readString();
+                  struct.partitionVals.add(_elem535);
                 }
                 iprot.readListEnd();
               }
@@ -768,9 +768,9 @@ public class FireEventRequest implements org.apache.thrift.TBase<FireEventReques
           oprot.writeFieldBegin(PARTITION_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partitionVals.size()));
-            for (String _iter527 : struct.partitionVals)
+            for (String _iter537 : struct.partitionVals)
             {
-              oprot.writeString(_iter527);
+              oprot.writeString(_iter537);
             }
             oprot.writeListEnd();
           }
@@ -816,9 +816,9 @@ public class FireEventRequest implements org.apache.thrift.TBase<FireEventReques
       if (struct.isSetPartitionVals()) {
         {
           oprot.writeI32(struct.partitionVals.size());
-          for (String _iter528 : struct.partitionVals)
+          for (String _iter538 : struct.partitionVals)
           {
-            oprot.writeString(_iter528);
+            oprot.writeString(_iter538);
           }
         }
       }
@@ -843,13 +843,13 @@ public class FireEventRequest implements org.apache.thrift.TBase<FireEventReques
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list529 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.partitionVals = new ArrayList<String>(_list529.size);
-          String _elem530;
-          for (int _i531 = 0; _i531 < _list529.size; ++_i531)
+          org.apache.thrift.protocol.TList _list539 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.partitionVals = new ArrayList<String>(_list539.size);
+          String _elem540;
+          for (int _i541 = 0; _i541 < _list539.size; ++_i541)
           {
-            _elem530 = iprot.readString();
-            struct.partitionVals.add(_elem530);
+            _elem540 = iprot.readString();
+            struct.partitionVals.add(_elem540);
           }
         }
         struct.setPartitionValsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
index f23c8e5..9728f5e 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
@@ -346,14 +346,14 @@ public class GetAllFunctionsResponse implements org.apache.thrift.TBase<GetAllFu
           case 1: // FUNCTIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list532 = iprot.readListBegin();
-                struct.functions = new ArrayList<Function>(_list532.size);
-                Function _elem533;
-                for (int _i534 = 0; _i534 < _list532.size; ++_i534)
+                org.apache.thrift.protocol.TList _list542 = iprot.readListBegin();
+                struct.functions = new ArrayList<Function>(_list542.size);
+                Function _elem543;
+                for (int _i544 = 0; _i544 < _list542.size; ++_i544)
                 {
-                  _elem533 = new Function();
-                  _elem533.read(iprot);
-                  struct.functions.add(_elem533);
+                  _elem543 = new Function();
+                  _elem543.read(iprot);
+                  struct.functions.add(_elem543);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ public class GetAllFunctionsResponse implements org.apache.thrift.TBase<GetAllFu
           oprot.writeFieldBegin(FUNCTIONS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.functions.size()));
-            for (Function _iter535 : struct.functions)
+            for (Function _iter545 : struct.functions)
             {
-              _iter535.write(oprot);
+              _iter545.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ public class GetAllFunctionsResponse implements org.apache.thrift.TBase<GetAllFu
       if (struct.isSetFunctions()) {
         {
           oprot.writeI32(struct.functions.size());
-          for (Function _iter536 : struct.functions)
+          for (Function _iter546 : struct.functions)
           {
-            _iter536.write(oprot);
+            _iter546.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ public class GetAllFunctionsResponse implements org.apache.thrift.TBase<GetAllFu
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list537 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.functions = new ArrayList<Function>(_list537.size);
-          Function _elem538;
-          for (int _i539 = 0; _i539 < _list537.size; ++_i539)
+          org.apache.thrift.protocol.TList _list547 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.functions = new ArrayList<Function>(_list547.size);
+          Function _elem548;
+          for (int _i549 = 0; _i549 < _list547.size; ++_i549)
           {
-            _elem538 = new Function();
-            _elem538.read(iprot);
-            struct.functions.add(_elem538);
+            _elem548 = new Function();
+            _elem548.read(iprot);
+            struct.functions.add(_elem548);
           }
         }
         struct.setFunctionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
index 8a799c8..594c28e 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
@@ -351,13 +351,13 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
           case 1: // FILES_ADDED
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list516 = iprot.readListBegin();
-                struct.filesAdded = new ArrayList<String>(_list516.size);
-                String _elem517;
-                for (int _i518 = 0; _i518 < _list516.size; ++_i518)
+                org.apache.thrift.protocol.TList _list526 = iprot.readListBegin();
+                struct.filesAdded = new ArrayList<String>(_list526.size);
+                String _elem527;
+                for (int _i528 = 0; _i528 < _list526.size; ++_i528)
                 {
-                  _elem517 = iprot.readString();
-                  struct.filesAdded.add(_elem517);
+                  _elem527 = iprot.readString();
+                  struct.filesAdded.add(_elem527);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
         oprot.writeFieldBegin(FILES_ADDED_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filesAdded.size()));
-          for (String _iter519 : struct.filesAdded)
+          for (String _iter529 : struct.filesAdded)
           {
-            oprot.writeString(_iter519);
+            oprot.writeString(_iter529);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.filesAdded.size());
-        for (String _iter520 : struct.filesAdded)
+        for (String _iter530 : struct.filesAdded)
         {
-          oprot.writeString(_iter520);
+          oprot.writeString(_iter530);
         }
       }
     }
@@ -421,13 +421,13 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
     public void read(org.apache.thrift.protocol.TProtocol prot, InsertEventRequestData struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list521 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.filesAdded = new ArrayList<String>(_list521.size);
-        String _elem522;
-        for (int _i523 = 0; _i523 < _list521.size; ++_i523)
+        org.apache.thrift.protocol.TList _list531 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.filesAdded = new ArrayList<String>(_list531.size);
+        String _elem532;
+        for (int _i533 = 0; _i533 < _list531.size; ++_i533)
         {
-          _elem522 = iprot.readString();
-          struct.filesAdded.add(_elem522);
+          _elem532 = iprot.readString();
+          struct.filesAdded.add(_elem532);
         }
       }
       struct.setFilesAddedIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
index f6671d5..fbc19b7 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
@@ -354,14 +354,14 @@ public class NotificationEventResponse implements org.apache.thrift.TBase<Notifi
           case 1: // EVENTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list508 = iprot.readListBegin();
-                struct.events = new ArrayList<NotificationEvent>(_list508.size);
-                NotificationEvent _elem509;
-                for (int _i510 = 0; _i510 < _list508.size; ++_i510)
+                org.apache.thrift.protocol.TList _list518 = iprot.readListBegin();
+                struct.events = new ArrayList<NotificationEvent>(_list518.size);
+                NotificationEvent _elem519;
+                for (int _i520 = 0; _i520 < _list518.size; ++_i520)
                 {
-                  _elem509 = new NotificationEvent();
-                  _elem509.read(iprot);
-                  struct.events.add(_elem509);
+                  _elem519 = new NotificationEvent();
+                  _elem519.read(iprot);
+                  struct.events.add(_elem519);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ public class NotificationEventResponse implements org.apache.thrift.TBase<Notifi
         oprot.writeFieldBegin(EVENTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.events.size()));
-          for (NotificationEvent _iter511 : struct.events)
+          for (NotificationEvent _iter521 : struct.events)
           {
-            _iter511.write(oprot);
+            _iter521.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ public class NotificationEventResponse implements org.apache.thrift.TBase<Notifi
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.events.size());
-        for (NotificationEvent _iter512 : struct.events)
+        for (NotificationEvent _iter522 : struct.events)
         {
-          _iter512.write(oprot);
+          _iter522.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ public class NotificationEventResponse implements org.apache.thrift.TBase<Notifi
     public void read(org.apache.thrift.protocol.TProtocol prot, NotificationEventResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list513 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.events = new ArrayList<NotificationEvent>(_list513.size);
-        NotificationEvent _elem514;
-        for (int _i515 = 0; _i515 < _list513.size; ++_i515)
+        org.apache.thrift.protocol.TList _list523 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.events = new ArrayList<NotificationEvent>(_list523.size);
+        NotificationEvent _elem524;
+        for (int _i525 = 0; _i525 < _list523.size; ++_i525)
         {
-          _elem514 = new NotificationEvent();
-          _elem514.read(iprot);
-          struct.events.add(_elem514);
+          _elem524 = new NotificationEvent();
+          _elem524.read(iprot);
+          struct.events.add(_elem524);
         }
       }
       struct.setEventsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/6e0504d9/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
index 1cb446d..6ee26ad 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
@@ -354,14 +354,14 @@ public class ShowCompactResponse implements org.apache.thrift.TBase<ShowCompactR
           case 1: // COMPACTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list492 = iprot.readListBegin();
-                struct.compacts = new ArrayList<ShowCompactResponseElement>(_list492.size);
-                ShowCompactResponseElement _elem493;
-                for (int _i494 = 0; _i494 < _list492.size; ++_i494)
+                org.apache.thrift.protocol.TList _list502 = iprot.readListBegin();
+                struct.compacts = new ArrayList<ShowCompactResponseElement>(_list502.size);
+                ShowCompactResponseElement _elem503;
+                for (int _i504 = 0; _i504 < _list502.size; ++_i504)
                 {
-                  _elem493 = new ShowCompactResponseElement();
-                  _elem493.read(iprot);
-                  struct.compacts.add(_elem493);
+                  _elem503 = new ShowCompactResponseElement();
+                  _elem503.read(iprot);
+                  struct.compacts.add(_elem503);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ public class ShowCompactResponse implements org.apache.thrift.TBase<ShowCompactR
         oprot.writeFieldBegin(COMPACTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.compacts.size()));
-          for (ShowCompactResponseElement _iter495 : struct.compacts)
+          for (ShowCompactResponseElement _iter505 : struct.compacts)
           {
-            _iter495.write(oprot);
+            _iter505.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ public class ShowCompactResponse implements org.apache.thrift.TBase<ShowCompactR
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.compacts.size());
-        for (ShowCompactResponseElement _iter496 : struct.compacts)
+        for (ShowCompactResponseElement _iter506 : struct.compacts)
         {
-          _iter496.write(oprot);
+          _iter506.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ public class ShowCompactResponse implements org.apache.thrift.TBase<ShowCompactR
     public void read(org.apache.thrift.protocol.TProtocol prot, ShowCompactResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list497 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.compacts = new ArrayList<ShowCompactResponseElement>(_list497.size);
-        ShowCompactResponseElement _elem498;
-        for (int _i499 = 0; _i499 < _list497.size; ++_i499)
+        org.apache.thrift.protocol.TList _list507 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.compacts = new ArrayList<ShowCompactResponseElement>(_list507.size);
+        ShowCompactResponseElement _elem508;
+        for (int _i509 = 0; _i509 < _list507.size; ++_i509)
         {
-          _elem498 = new ShowCompactResponseElement();
-          _elem498.read(iprot);
-          struct.compacts.add(_elem498);
+          _elem508 = new ShowCompactResponseElement();
+          _elem508.read(iprot);
+          struct.compacts.add(_elem508);
         }
       }
       struct.setCompactsIsSet(true);